diff --git a/DEPS.bzl b/DEPS.bzl index 2a9004c21455b..f05385986ad79 100644 --- a/DEPS.bzl +++ b/DEPS.bzl @@ -3495,8 +3495,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:x5L32xr5KE2C0UhlKfrsDTWQ/8fpFYQLq5xn+ph0sQ0=", - version = "v2.0.3-0.20221103084528-ec1202a9d26f", + sum = "h1:nFVdyTXcQYZwQQCdSJcFI1vBFyzG1hVuZ39MAK6wqK4=", + version = "v2.0.3-0.20221108030801-9c0835c80eba", ) go_repository( name = "com_github_tikv_pd_client", diff --git a/Makefile b/Makefile index d45e258eaaa83..66b3ba0686917 100644 --- a/Makefile +++ b/Makefile @@ -409,7 +409,7 @@ bazel_coverage_test: failpoint-enable bazel_ci_prepare -- //... -//cmd/... -//tests/graceshutdown/... \ -//tests/globalkilltest/... -//tests/readonlytest/... -//br/pkg/task:task_test -//tests/realtikvtest/... bazel $(BAZEL_GLOBAL_CONFIG) coverage $(BAZEL_CMD_CONFIG) \ - --build_event_json_file=bazel_2.json --@io_bazel_rules_go//go/config:cover_format=go_cover \ + --build_event_json_file=bazel_2.json --@io_bazel_rules_go//go/config:cover_format=go_cover --define gotags=featuretag \ -- //... -//cmd/... -//tests/graceshutdown/... \ -//tests/globalkilltest/... -//tests/readonlytest/... -//br/pkg/task:task_test -//tests/realtikvtest/... diff --git a/autoid_service/BUILD.bazel b/autoid_service/BUILD.bazel index 7e29b826e7b29..df3d361d412ed 100644 --- a/autoid_service/BUILD.bazel +++ b/autoid_service/BUILD.bazel @@ -6,6 +6,7 @@ go_library( importpath = "github.com/pingcap/tidb/autoid_service", visibility = ["//visibility:public"], deps = [ + "//config", "//kv", "//meta", "//metrics", @@ -16,6 +17,7 @@ go_library( "@com_github_pingcap_kvproto//pkg/autoid", "@io_etcd_go_etcd_client_v3//:client", "@org_golang_google_grpc//:grpc", + "@org_golang_google_grpc//keepalive", "@org_uber_go_zap//:zap", ], ) diff --git a/autoid_service/autoid.go b/autoid_service/autoid.go index 7e0edb7ea3e63..d20c78fd06098 100644 --- a/autoid_service/autoid.go +++ b/autoid_service/autoid.go @@ -16,12 +16,14 @@ package autoid import ( "context" + "crypto/tls" "math" "sync" "time" "github.com/pingcap/errors" "github.com/pingcap/kvproto/pkg/autoid" + "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/meta" "github.com/pingcap/tidb/metrics" @@ -31,6 +33,7 @@ import ( clientv3 "go.etcd.io/etcd/client/v3" "go.uber.org/zap" "google.golang.org/grpc" + "google.golang.org/grpc/keepalive" ) var ( @@ -245,10 +248,22 @@ type Service struct { } // New return a Service instance. -func New(selfAddr string, etcdAddr []string, store kv.Storage) *Service { +func New(selfAddr string, etcdAddr []string, store kv.Storage, tlsConfig *tls.Config) *Service { + cfg := config.GetGlobalConfig() + etcdLogCfg := zap.NewProductionConfig() cli, err := clientv3.New(clientv3.Config{ - Endpoints: etcdAddr, - DialTimeout: time.Second, + LogConfig: &etcdLogCfg, + Endpoints: etcdAddr, + AutoSyncInterval: 30 * time.Second, + DialTimeout: 5 * time.Second, + DialOptions: []grpc.DialOption{ + grpc.WithBackoffMaxDelay(time.Second * 3), + grpc.WithKeepaliveParams(keepalive.ClientParameters{ + Time: time.Duration(cfg.TiKVClient.GrpcKeepAliveTime) * time.Second, + Timeout: time.Duration(cfg.TiKVClient.GrpcKeepAliveTimeout) * time.Second, + }), + }, + TLS: tlsConfig, }) if err != nil { panic(err) diff --git a/bindinfo/handle.go b/bindinfo/handle.go index c69f3e45fb2bb..f4a9d372e9d85 100644 --- a/bindinfo/handle.go +++ b/bindinfo/handle.go @@ -121,7 +121,8 @@ func (h *BindHandle) Reset(ctx sessionctx.Context) { h.bindInfo.parser = parser.New() h.invalidBindRecordMap.Value.Store(make(map[string]*bindRecordUpdate)) h.invalidBindRecordMap.flushFunc = func(record *BindRecord) error { - return h.DropBindRecord(record.OriginalSQL, record.Db, &record.Bindings[0]) + _, err := h.DropBindRecord(record.OriginalSQL, record.Db, &record.Bindings[0]) + return err } h.pendingVerifyBindRecordMap.Value.Store(make(map[string]*bindRecordUpdate)) h.pendingVerifyBindRecordMap.flushFunc = func(record *BindRecord) error { @@ -368,7 +369,7 @@ func (h *BindHandle) AddBindRecord(sctx sessionctx.Context, record *BindRecord) } // DropBindRecord drops a BindRecord to the storage and BindRecord int the cache. -func (h *BindHandle) DropBindRecord(originalSQL, db string, binding *Binding) (err error) { +func (h *BindHandle) DropBindRecord(originalSQL, db string, binding *Binding) (deletedRows uint64, err error) { db = strings.ToLower(db) h.bindInfo.Lock() h.sctx.Lock() @@ -380,9 +381,8 @@ func (h *BindHandle) DropBindRecord(originalSQL, db string, binding *Binding) (e exec, _ := h.sctx.Context.(sqlexec.SQLExecutor) _, err = exec.ExecuteInternal(ctx, "BEGIN PESSIMISTIC") if err != nil { - return err + return 0, err } - var deleteRows int defer func() { if err != nil { _, err1 := exec.ExecuteInternal(ctx, "ROLLBACK") @@ -391,7 +391,7 @@ func (h *BindHandle) DropBindRecord(originalSQL, db string, binding *Binding) (e } _, err = exec.ExecuteInternal(ctx, "COMMIT") - if err != nil || deleteRows == 0 { + if err != nil || deletedRows == 0 { return } @@ -404,7 +404,7 @@ func (h *BindHandle) DropBindRecord(originalSQL, db string, binding *Binding) (e // Lock mysql.bind_info to synchronize with CreateBindRecord / AddBindRecord / DropBindRecord on other tidb instances. if err = h.lockBindInfoTable(); err != nil { - return err + return 0, err } updateTs := types.NewTime(types.FromGoTime(time.Now()), mysql.TypeTimestamp, 3).String() @@ -416,9 +416,11 @@ func (h *BindHandle) DropBindRecord(originalSQL, db string, binding *Binding) (e _, err = exec.ExecuteInternal(ctx, `UPDATE mysql.bind_info SET status = %?, update_time = %? WHERE original_sql = %? AND update_time < %? AND bind_sql = %? and status != %?`, deleted, updateTs, originalSQL, updateTs, binding.BindSQL, deleted) } + if err != nil { + return 0, err + } - deleteRows = int(h.sctx.Context.GetSessionVars().StmtCtx.AffectedRows()) - return err + return h.sctx.Context.GetSessionVars().StmtCtx.AffectedRows(), nil } // SetBindRecordStatus set a BindRecord's status to the storage and bind cache. @@ -1185,7 +1187,8 @@ func (h *BindHandle) HandleEvolvePlanTask(sctx sessionctx.Context, adminEvolve b // since it is still in the bind record. Now we just drop it and if it is actually retryable, // we will hope for that we can capture this evolve task again. if err != nil { - return h.DropBindRecord(originalSQL, db, &binding) + _, err = h.DropBindRecord(originalSQL, db, &binding) + return err } // If the accepted plan timeouts, it is hard to decide the timeout for verify plan. // Currently we simply mark the verify plan as `using` if it could run successfully within maxTime. @@ -1195,7 +1198,8 @@ func (h *BindHandle) HandleEvolvePlanTask(sctx sessionctx.Context, adminEvolve b sctx.GetSessionVars().UsePlanBaselines = false verifyPlanTime, err := h.getRunningDuration(sctx, db, binding.BindSQL, maxTime) if err != nil { - return h.DropBindRecord(originalSQL, db, &binding) + _, err = h.DropBindRecord(originalSQL, db, &binding) + return err } if verifyPlanTime == -1 || (float64(verifyPlanTime)*acceptFactor > float64(currentPlanTime)) { binding.Status = Rejected diff --git a/bindinfo/handle_test.go b/bindinfo/handle_test.go index 7831dc1358775..01ff0628feb5c 100644 --- a/bindinfo/handle_test.go +++ b/bindinfo/handle_test.go @@ -547,6 +547,7 @@ func TestGlobalBinding(t *testing.T) { require.NotNil(t, bind.UpdateTime) _, err = tk.Exec("drop global " + testSQL.dropSQL) + require.Equal(t, uint64(1), tk.Session().AffectedRows()) require.NoError(t, err) bindData = dom.BindHandle().GetBindRecord(hash, sql, "test") require.Nil(t, bindData) diff --git a/br/pkg/restore/BUILD.bazel b/br/pkg/restore/BUILD.bazel index 2c9ce90552f7b..2c42717bafe36 100644 --- a/br/pkg/restore/BUILD.bazel +++ b/br/pkg/restore/BUILD.bazel @@ -149,7 +149,6 @@ go_test( "//testkit/testsetup", "//types", "//util/codec", - "//util/mathutil", "@com_github_fsouza_fake_gcs_server//fakestorage", "@com_github_golang_protobuf//proto", "@com_github_pingcap_errors//:errors", diff --git a/br/pkg/restore/log_client_test.go b/br/pkg/restore/log_client_test.go index c4c6d46d15b3f..71db52cf7678f 100644 --- a/br/pkg/restore/log_client_test.go +++ b/br/pkg/restore/log_client_test.go @@ -300,12 +300,19 @@ func testReadFromMetadataWithVersion(t *testing.T, m metaMaker) { meta := new(StreamMetadataSet) meta.Helper = stream.NewMetadataHelper() - meta.LoadUntil(ctx, loc, c.untilTS) + meta.LoadUntilAndCalculateShiftTS(ctx, loc, c.untilTS) var metas []*backuppb.Metadata - for _, m := range meta.metadata { + for path := range meta.metadataInfos { + data, err := loc.ReadFile(ctx, path) + require.NoError(t, err) + + m, err := meta.Helper.ParseToMetadataHard(data) + require.NoError(t, err) + metas = append(metas, m) } + actualStoreIDs := make([]int64, 0, len(metas)) for _, meta := range metas { actualStoreIDs = append(actualStoreIDs, meta.StoreId) diff --git a/br/pkg/restore/stream_metas.go b/br/pkg/restore/stream_metas.go index 24736c9298a70..2aa9c8f11a9db 100644 --- a/br/pkg/restore/stream_metas.go +++ b/br/pkg/restore/stream_metas.go @@ -12,67 +12,116 @@ import ( backuppb "github.com/pingcap/kvproto/pkg/brpb" "github.com/pingcap/log" berrors "github.com/pingcap/tidb/br/pkg/errors" + "github.com/pingcap/tidb/br/pkg/logutil" "github.com/pingcap/tidb/br/pkg/storage" "github.com/pingcap/tidb/br/pkg/stream" + "github.com/pingcap/tidb/br/pkg/utils" "github.com/pingcap/tidb/util/mathutil" "go.uber.org/zap" + "golang.org/x/sync/errgroup" ) +const notDeletedBecameFatalThreshold = 128 + type StreamMetadataSet struct { - metadata map[string]*backuppb.Metadata - // The metadata after changed that needs to be write back. - writeback map[string]*backuppb.Metadata + // if set true, the metadata and datafile won't be removed + DryRun bool + + // keeps the meta-information of metadata as little as possible + // to save the memory + metadataInfos map[string]*MetadataInfo + // a parser of metadata Helper *stream.MetadataHelper - BeforeDoWriteBack func(path string, last, current *backuppb.Metadata) (skip bool) + // for test + BeforeDoWriteBack func(path string, replaced *backuppb.Metadata) (skip bool) } -// MetaLen returns the length of read metadata. -func (ms *StreamMetadataSet) MetaLen() int { - return len(ms.metadata) +// keep these meta-information for statistics and filtering +type FileGroupInfo struct { + MaxTS uint64 + Length uint64 + KVCount int64 } -// LoadUntil loads the metadata until the specified timestamp. -// This would load all metadata files that *may* contain data from transaction committed before that TS. -// Note: maybe record the timestamp and reject reading data files after this TS? -func (ms *StreamMetadataSet) LoadUntil(ctx context.Context, s storage.ExternalStorage, until uint64) error { +// keep these meta-information for statistics and filtering +type MetadataInfo struct { + MinTS uint64 + FileGroupInfos []*FileGroupInfo +} + +// LoadUntilAndCalculateShiftTS loads the metadata until the specified timestamp and calculate the shift-until-ts by the way. +// This would record all metadata files that *may* contain data from transaction committed before that TS. +func (ms *StreamMetadataSet) LoadUntilAndCalculateShiftTS(ctx context.Context, s storage.ExternalStorage, until uint64) (uint64, error) { metadataMap := struct { sync.Mutex - metas map[string]*backuppb.Metadata + metas map[string]*MetadataInfo + shiftUntilTS uint64 }{} - ms.writeback = make(map[string]*backuppb.Metadata) - metadataMap.metas = make(map[string]*backuppb.Metadata) + metadataMap.metas = make(map[string]*MetadataInfo) + // `shiftUntilTS` must be less than `until` + metadataMap.shiftUntilTS = until err := stream.FastUnmarshalMetaData(ctx, s, func(path string, raw []byte) error { m, err := ms.Helper.ParseToMetadataHard(raw) if err != nil { return err } - metadataMap.Lock() // If the meta file contains only files with ts grater than `until`, when the file is from // `Default`: it should be kept, because its corresponding `write` must has commit ts grater than it, which should not be considered. // `Write`: it should trivially not be considered. if m.MinTs <= until { - metadataMap.metas[path] = m + // record these meta-information for statistics and filtering + fileGroupInfos := make([]*FileGroupInfo, 0, len(m.FileGroups)) + for _, group := range m.FileGroups { + var kvCount int64 = 0 + for _, file := range group.DataFilesInfo { + kvCount += file.NumberOfEntries + } + fileGroupInfos = append(fileGroupInfos, &FileGroupInfo{ + MaxTS: group.MaxTs, + Length: group.Length, + KVCount: kvCount, + }) + } + metadataMap.Lock() + metadataMap.metas[path] = &MetadataInfo{ + MinTS: m.MinTs, + FileGroupInfos: fileGroupInfos, + } + metadataMap.Unlock() + } + // filter out the metadatas whose ts-range is overlap with [until, +inf) + // and calculate their minimum begin-default-ts + ts, ok := UpdateShiftTS(m, until, mathutil.MaxUint) + if ok { + metadataMap.Lock() + if ts < metadataMap.shiftUntilTS { + metadataMap.shiftUntilTS = ts + } + metadataMap.Unlock() } - metadataMap.Unlock() return nil }) if err != nil { - return errors.Trace(err) + return 0, errors.Trace(err) } - ms.metadata = metadataMap.metas - return nil + ms.metadataInfos = metadataMap.metas + if metadataMap.shiftUntilTS != until { + log.Warn("calculate shift-ts", zap.Uint64("start-ts", until), zap.Uint64("shift-ts", metadataMap.shiftUntilTS)) + } + return metadataMap.shiftUntilTS, nil } -// LoadFrom loads data from an external storage into the stream metadata set. +// LoadFrom loads data from an external storage into the stream metadata set. (Now only for test) func (ms *StreamMetadataSet) LoadFrom(ctx context.Context, s storage.ExternalStorage) error { - return ms.LoadUntil(ctx, s, math.MaxUint64) + _, err := ms.LoadUntilAndCalculateShiftTS(ctx, s, math.MaxUint64) + return err } -func (ms *StreamMetadataSet) iterateDataFiles(f func(d *backuppb.DataFileGroup) (shouldBreak bool)) { - for _, m := range ms.metadata { - for _, d := range m.FileGroups { +func (ms *StreamMetadataSet) iterateDataFiles(f func(d *FileGroupInfo) (shouldBreak bool)) { + for _, m := range ms.metadataInfos { + for _, d := range m.FileGroupInfos { if f(d) { return } @@ -80,21 +129,6 @@ func (ms *StreamMetadataSet) iterateDataFiles(f func(d *backuppb.DataFileGroup) } } -// CalculateShiftTS calculates the shift-ts. -func (ms *StreamMetadataSet) CalculateShiftTS(startTS uint64) uint64 { - metadatas := make([]*backuppb.Metadata, 0, len(ms.metadata)) - for _, m := range ms.metadata { - metadatas = append(metadatas, m) - } - - minBeginTS, exist := CalculateShiftTS(metadatas, startTS, mathutil.MaxUint) - if !exist { - minBeginTS = startTS - } - log.Warn("calculate shift-ts", zap.Uint64("start-ts", startTS), zap.Uint64("shift-ts", minBeginTS)) - return minBeginTS -} - // IterateFilesFullyBefore runs the function over all files contain data before the timestamp only. // // 0 before @@ -103,83 +137,145 @@ func (ms *StreamMetadataSet) CalculateShiftTS(startTS uint64) uint64 { // |-file2--------------| <- File contains any record out of this won't be found. // // This function would call the `f` over file1 only. -func (ms *StreamMetadataSet) IterateFilesFullyBefore(before uint64, f func(d *backuppb.DataFileGroup) (shouldBreak bool)) { - ms.iterateDataFiles(func(d *backuppb.DataFileGroup) (shouldBreak bool) { - if d.MaxTs >= before { +func (ms *StreamMetadataSet) IterateFilesFullyBefore(before uint64, f func(d *FileGroupInfo) (shouldBreak bool)) { + ms.iterateDataFiles(func(d *FileGroupInfo) (shouldBreak bool) { + if d.MaxTS >= before { return false } return f(d) }) } -// RemoveDataBefore would find files contains only records before the timestamp, mark them as removed from meta, -// and returning their information. -func (ms *StreamMetadataSet) RemoveDataBefore(from uint64) []*backuppb.DataFileGroup { - removed := []*backuppb.DataFileGroup{} - for metaPath, m := range ms.metadata { - remainedDataFiles := make([]*backuppb.DataFileGroup, 0) - // can we assume those files are sorted to avoid traversing here? (by what?) - for _, ds := range m.FileGroups { - if ds.MaxTs < from { - removed = append(removed, ds) - } else { - remainedDataFiles = append(remainedDataFiles, ds) +// RemoveDataFilesAndUpdateMetadataInBatch concurrently remove datafilegroups and update metadata. +// Only one metadata is processed in each thread, including deleting its datafilegroup and updating it. +// Returns the not deleted datafilegroups. +func (ms *StreamMetadataSet) RemoveDataFilesAndUpdateMetadataInBatch(ctx context.Context, from uint64, storage storage.ExternalStorage, updateFn func(num int64)) ([]string, error) { + var notDeleted struct { + item []string + sync.Mutex + } + worker := utils.NewWorkerPool(128, "delete files") + eg, cx := errgroup.WithContext(ctx) + for path, metaInfo := range ms.metadataInfos { + path := path + minTS := metaInfo.MinTS + // It's safety to remove the item within a range loop + delete(ms.metadataInfos, path) + if minTS >= from { + // That means all the datafiles wouldn't be removed, + // so that the metadata is skipped. + continue + } + worker.ApplyOnErrorGroup(eg, func() error { + if cx.Err() != nil { + return cx.Err() + } + + data, err := storage.ReadFile(ctx, path) + if err != nil { + return err + } + + meta, err := ms.Helper.ParseToMetadataHard(data) + if err != nil { + return err + } + + num, notDeletedItems, err := ms.removeDataFilesAndUpdateMetadata(ctx, storage, from, meta, path) + if err != nil { + return err + } + + updateFn(num) + + notDeleted.Lock() + notDeleted.item = append(notDeleted.item, notDeletedItems...) + notDeleted.Unlock() + return nil + }) + } + + if err := eg.Wait(); err != nil { + return nil, errors.Trace(err) + } + + return notDeleted.item, nil +} + +// removeDataFilesAndUpdateMetadata removes some datafilegroups of the metadata, if their max-ts is less than `from` +func (ms *StreamMetadataSet) removeDataFilesAndUpdateMetadata(ctx context.Context, storage storage.ExternalStorage, from uint64, meta *backuppb.Metadata, metaPath string) (num int64, notDeleted []string, err error) { + removed := make([]*backuppb.DataFileGroup, 0) + remainedDataFiles := make([]*backuppb.DataFileGroup, 0) + notDeleted = make([]string, 0) + // can we assume those files are sorted to avoid traversing here? (by what?) + for _, ds := range meta.FileGroups { + if ds.MaxTs < from { + removed = append(removed, ds) + } else { + // That means some kvs in the datafilegroup shouldn't be removed, + // so it will be kept out being removed. + remainedDataFiles = append(remainedDataFiles, ds) + } + } + + num = int64(len(removed)) + + if ms.DryRun { + log.Debug("dry run, skip deletion ...") + return num, notDeleted, nil + } + + // remove data file groups + for _, f := range removed { + log.Debug("Deleting file", zap.String("path", f.Path)) + if err := storage.DeleteFile(ctx, f.Path); err != nil { + log.Warn("File not deleted.", zap.String("path", f.Path), logutil.ShortError(err)) + notDeleted = append(notDeleted, f.Path) + if len(notDeleted) > notDeletedBecameFatalThreshold { + return num, notDeleted, errors.Annotatef(berrors.ErrPiTRMalformedMetadata, "too many failure when truncating") } } - if len(remainedDataFiles) != len(m.FileGroups) { - mCopy := *m - mCopy.FileGroups = remainedDataFiles - ms.WriteBack(metaPath, &mCopy) + } + + // update metadata + if len(remainedDataFiles) != len(meta.FileGroups) { + // rewrite metadata + log.Info("Updating metadata.", zap.String("file", metaPath), + zap.Int("data-file-before", len(meta.FileGroups)), + zap.Int("data-file-after", len(remainedDataFiles))) + + // replace the filegroups and update the ts of the replaced metadata + ReplaceMetadata(meta, remainedDataFiles) + + if ms.BeforeDoWriteBack != nil && ms.BeforeDoWriteBack(metaPath, meta) { + return num, notDeleted, nil + } + + if err := ms.doWriteBackForFile(ctx, storage, metaPath, meta); err != nil { + // NOTE: Maybe we'd better roll back all writebacks? (What will happen if roll back fails too?) + return num, notDeleted, errors.Annotatef(err, "failed to write back file %s", metaPath) } } - return removed -} -func (ms *StreamMetadataSet) WriteBack(path string, file *backuppb.Metadata) { - ms.writeback[path] = file + return num, notDeleted, nil } -func (ms *StreamMetadataSet) doWriteBackForFile(ctx context.Context, s storage.ExternalStorage, path string) error { - data, ok := ms.writeback[path] - if !ok { - return errors.Annotatef(berrors.ErrInvalidArgument, "There is no write back for path %s", path) - } +func (ms *StreamMetadataSet) doWriteBackForFile(ctx context.Context, s storage.ExternalStorage, path string, meta *backuppb.Metadata) error { // If the metadata file contains no data file, remove it due to it is meanless. - if len(data.FileGroups) == 0 { + if len(meta.FileGroups) == 0 { if err := s.DeleteFile(ctx, path); err != nil { return errors.Annotatef(err, "failed to remove the empty meta %s", path) } return nil } - bs, err := ms.Helper.Marshal(data) + bs, err := ms.Helper.Marshal(meta) if err != nil { return errors.Annotatef(err, "failed to marshal the file %s", path) } return truncateAndWrite(ctx, s, path, bs) } -// PendingMeta returns the length of metadata waiting for be written back. -func (ms *StreamMetadataSet) PendingMeta() int { - return len(ms.writeback) -} - -func (ms *StreamMetadataSet) DoWriteBack(ctx context.Context, s storage.ExternalStorage) error { - for path := range ms.writeback { - if ms.BeforeDoWriteBack != nil && ms.BeforeDoWriteBack(path, ms.metadata[path], ms.writeback[path]) { - continue - } - err := ms.doWriteBackForFile(ctx, s, path) - // NOTE: Maybe we'd better roll back all writebacks? (What will happen if roll back fails too?) - if err != nil { - return errors.Annotatef(err, "failed to write back file %s", path) - } - - delete(ms.writeback, path) - } - return nil -} - func truncateAndWrite(ctx context.Context, s storage.ExternalStorage, path string, data []byte) error { // Performance hack: the `Write` implemention would truncate the file if it exists. if err := s.WriteFile(ctx, path, data); err != nil { @@ -258,26 +354,30 @@ func UpdateShiftTS(m *backuppb.Metadata, startTS uint64, restoreTS uint64) (uint return minBeginTS, isExist } -// CalculateShiftTS gets the minimal begin-ts about transaction according to the kv-event in write-cf. -func CalculateShiftTS( - metas []*backuppb.Metadata, - startTS uint64, - restoreTS uint64, -) (uint64, bool) { - var ( - minBeginTS uint64 - isExist bool - ) - for _, m := range metas { - if len(m.FileGroups) == 0 || m.MinTs > restoreTS || m.MaxTs < startTS { - continue +// replace the filegroups and update the ts of the replaced metadata +func ReplaceMetadata(meta *backuppb.Metadata, filegroups []*backuppb.DataFileGroup) { + // replace the origin metadata + meta.FileGroups = filegroups + + if len(meta.FileGroups) == 0 { + meta.MinTs = 0 + meta.MaxTs = 0 + meta.ResolvedTs = 0 + return + } + + meta.MinTs = meta.FileGroups[0].MinTs + meta.MaxTs = meta.FileGroups[0].MaxTs + meta.ResolvedTs = meta.FileGroups[0].MinResolvedTs + for _, group := range meta.FileGroups { + if group.MinTs < meta.MinTs { + meta.MinTs = group.MinTs + } + if group.MaxTs > meta.MaxTs { + meta.MaxTs = group.MaxTs } - ts, ok := UpdateShiftTS(m, startTS, restoreTS) - if ok && (!isExist || ts < minBeginTS) { - minBeginTS = ts - isExist = true + if group.MinResolvedTs < meta.ResolvedTs { + meta.ResolvedTs = group.MinResolvedTs } } - - return minBeginTS, isExist } diff --git a/br/pkg/restore/stream_metas_test.go b/br/pkg/restore/stream_metas_test.go index 8e75f7544885e..5b75e9de6b3d8 100644 --- a/br/pkg/restore/stream_metas_test.go +++ b/br/pkg/restore/stream_metas_test.go @@ -6,7 +6,10 @@ import ( "context" "fmt" "math/rand" + "os" + "path" "path/filepath" + "sync" "testing" "github.com/fsouza/fake-gcs-server/fakestorage" @@ -16,7 +19,6 @@ import ( "github.com/pingcap/tidb/br/pkg/restore" "github.com/pingcap/tidb/br/pkg/storage" "github.com/pingcap/tidb/br/pkg/stream" - "github.com/pingcap/tidb/util/mathutil" "github.com/stretchr/testify/require" "go.uber.org/zap" ) @@ -36,13 +38,59 @@ func fakeDataFiles(s storage.ExternalStorage, base, item int) (result []*backupp return } +func fakeDataFilesV2(s storage.ExternalStorage, base, item int) (result []*backuppb.DataFileGroup) { + ctx := context.Background() + for i := base; i < base+item; i++ { + path := fmt.Sprintf("%04d_to_%04d.log", i, i+2) + s.WriteFile(ctx, path, []byte("test")) + data := &backuppb.DataFileGroup{ + Path: path, + MinTs: uint64(i), + MaxTs: uint64(i + 2), + } + result = append(result, data) + } + return +} + +func tsOfFile(dfs []*backuppb.DataFileInfo) (uint64, uint64) { + var minTS uint64 = 9876543210 + var maxTS uint64 = 0 + for _, df := range dfs { + if df.MaxTs > maxTS { + maxTS = df.MaxTs + } + if df.MinTs < minTS { + minTS = df.MinTs + } + } + return minTS, maxTS +} + +func tsOfFileGroup(dfs []*backuppb.DataFileGroup) (uint64, uint64) { + var minTS uint64 = 9876543210 + var maxTS uint64 = 0 + for _, df := range dfs { + if df.MaxTs > maxTS { + maxTS = df.MaxTs + } + if df.MinTs < minTS { + minTS = df.MinTs + } + } + return minTS, maxTS +} + func fakeStreamBackup(s storage.ExternalStorage) error { ctx := context.Background() base := 0 for i := 0; i < 6; i++ { dfs := fakeDataFiles(s, base, 4) base += 4 + minTS, maxTS := tsOfFile(dfs) meta := &backuppb.Metadata{ + MinTs: minTS, + MaxTs: maxTS, Files: dfs, StoreId: int64(i%3 + 1), } @@ -64,43 +112,13 @@ func fakeStreamBackupV2(s storage.ExternalStorage) error { ctx := context.Background() base := 0 for i := 0; i < 6; i++ { - dfs := fakeDataFiles(s, base, 4) - minTs1 := uint64(18446744073709551615) - maxTs1 := uint64(0) - for _, f := range dfs[0:2] { - f.Path = fmt.Sprintf("%d", i) - if minTs1 > f.MinTs { - minTs1 = f.MinTs - } - if maxTs1 < f.MaxTs { - maxTs1 = f.MaxTs - } - } - minTs2 := uint64(18446744073709551615) - maxTs2 := uint64(0) - for _, f := range dfs[2:] { - f.Path = fmt.Sprintf("%d", i) - if minTs2 > f.MinTs { - minTs2 = f.MinTs - } - if maxTs2 < f.MaxTs { - maxTs2 = f.MaxTs - } - } + dfs := fakeDataFilesV2(s, base, 4) base += 4 + minTS, maxTS := tsOfFileGroup(dfs) meta := &backuppb.Metadata{ - FileGroups: []*backuppb.DataFileGroup{ - { - DataFilesInfo: dfs[0:2], - MinTs: minTs1, - MaxTs: maxTs1, - }, - { - DataFilesInfo: dfs[2:], - MinTs: minTs2, - MaxTs: maxTs2, - }, - }, + MinTs: minTS, + MaxTs: maxTS, + FileGroups: dfs, StoreId: int64(i%3 + 1), MetaVersion: backuppb.MetaVersion_V2, } @@ -135,42 +153,59 @@ func TestTruncateLog(t *testing.T) { } require.NoError(t, s.LoadFrom(ctx, l)) - fs := []*backuppb.DataFileGroup{} - s.IterateFilesFullyBefore(17, func(d *backuppb.DataFileGroup) (shouldBreak bool) { + fs := []*restore.FileGroupInfo{} + s.IterateFilesFullyBefore(17, func(d *restore.FileGroupInfo) (shouldBreak bool) { fs = append(fs, d) - require.Less(t, d.MaxTs, uint64(17)) + require.Less(t, d.MaxTS, uint64(17)) return false }) require.Len(t, fs, 15) - s.RemoveDataBefore(17) - deletedFiles := []string{} - modifiedFiles := []string{} - s.BeforeDoWriteBack = func(path string, last, current *backuppb.Metadata) bool { - require.NotNil(t, last) - if len(current.GetFileGroups()) == 0 { - deletedFiles = append(deletedFiles, path) - } else if len(current.GetFileGroups()) != len(last.GetFileGroups()) { - modifiedFiles = append(modifiedFiles, path) + var lock sync.Mutex + remainedFiles := []string{} + remainedDataFiles := []string{} + removedMetaFiles := []string{} + s.BeforeDoWriteBack = func(path string, replaced *backuppb.Metadata) bool { + lock.Lock() + require.NotNil(t, replaced) + if len(replaced.GetFileGroups()) > 0 { + remainedFiles = append(remainedFiles, path) + for _, ds := range replaced.FileGroups { + remainedDataFiles = append(remainedDataFiles, ds.Path) + } + } else { + removedMetaFiles = append(removedMetaFiles, path) } + lock.Unlock() return false } - require.NoError(t, s.DoWriteBack(ctx, l)) - require.ElementsMatch(t, deletedFiles, []string{"v1/backupmeta/0000.meta", "v1/backupmeta/0001.meta", "v1/backupmeta/0002.meta"}) - require.ElementsMatch(t, modifiedFiles, []string{"v1/backupmeta/0003.meta"}) + + var total int64 = 0 + notDeleted, err := s.RemoveDataFilesAndUpdateMetadataInBatch(ctx, 17, l, func(num int64) { + lock.Lock() + total += num + lock.Unlock() + }) + require.NoError(t, err) + require.Equal(t, len(notDeleted), 0) + require.ElementsMatch(t, remainedFiles, []string{"v1/backupmeta/0003.meta"}) + require.ElementsMatch(t, removedMetaFiles, []string{"v1/backupmeta/0000.meta", "v1/backupmeta/0001.meta", "v1/backupmeta/0002.meta"}) + require.ElementsMatch(t, remainedDataFiles, []string{"0015_to_0017.log"}) + require.Equal(t, total, int64(15)) require.NoError(t, s.LoadFrom(ctx, l)) - s.IterateFilesFullyBefore(17, func(d *backuppb.DataFileGroup) (shouldBreak bool) { + s.IterateFilesFullyBefore(17, func(d *restore.FileGroupInfo) (shouldBreak bool) { t.Errorf("some of log files still not truncated, it is %#v", d) return true }) - l.WalkDir(ctx, &storage.WalkOption{ + err = l.WalkDir(ctx, &storage.WalkOption{ SubDir: stream.GetStreamBackupMetaPrefix(), }, func(s string, i int64) error { - require.NotContains(t, deletedFiles, s) + require.NotContains(t, removedMetaFiles, s) return nil }) + require.NoError(t, err) } func TestTruncateLogV2(t *testing.T) { @@ -190,42 +225,59 @@ func TestTruncateLogV2(t *testing.T) { } require.NoError(t, s.LoadFrom(ctx, l)) - fs := []*backuppb.DataFileGroup{} - s.IterateFilesFullyBefore(17, func(d *backuppb.DataFileGroup) (shouldBreak bool) { + fs := []*restore.FileGroupInfo{} + s.IterateFilesFullyBefore(17, func(d *restore.FileGroupInfo) (shouldBreak bool) { fs = append(fs, d) - require.Less(t, d.MaxTs, uint64(17)) + require.Less(t, d.MaxTS, uint64(17)) return false }) - require.Len(t, fs, 7) - - s.RemoveDataBefore(17) - deletedFiles := []string{} - modifiedFiles := []string{} - s.BeforeDoWriteBack = func(path string, last, current *backuppb.Metadata) bool { - require.NotNil(t, last) - if len(current.GetFileGroups()) == 0 { - deletedFiles = append(deletedFiles, path) - } else if len(current.GetFileGroups()) != len(last.GetFileGroups()) { - modifiedFiles = append(modifiedFiles, path) + require.Len(t, fs, 15) + + var lock sync.Mutex + remainedFiles := []string{} + remainedDataFiles := []string{} + removedMetaFiles := []string{} + s.BeforeDoWriteBack = func(path string, replaced *backuppb.Metadata) bool { + lock.Lock() + require.NotNil(t, replaced) + if len(replaced.GetFileGroups()) > 0 { + remainedFiles = append(remainedFiles, path) + for _, ds := range replaced.FileGroups { + remainedDataFiles = append(remainedDataFiles, ds.Path) + } + } else { + removedMetaFiles = append(removedMetaFiles, path) } + lock.Unlock() return false } - require.NoError(t, s.DoWriteBack(ctx, l)) - require.ElementsMatch(t, deletedFiles, []string{"v1/backupmeta/0000.meta", "v1/backupmeta/0001.meta", "v1/backupmeta/0002.meta"}) - require.ElementsMatch(t, modifiedFiles, []string{"v1/backupmeta/0003.meta"}) + + var total int64 = 0 + notDeleted, err := s.RemoveDataFilesAndUpdateMetadataInBatch(ctx, 17, l, func(num int64) { + lock.Lock() + total += num + lock.Unlock() + }) + require.NoError(t, err) + require.Equal(t, len(notDeleted), 0) + require.ElementsMatch(t, remainedFiles, []string{"v1/backupmeta/0003.meta"}) + require.ElementsMatch(t, removedMetaFiles, []string{"v1/backupmeta/0000.meta", "v1/backupmeta/0001.meta", "v1/backupmeta/0002.meta"}) + require.ElementsMatch(t, remainedDataFiles, []string{"0015_to_0017.log"}) + require.Equal(t, total, int64(15)) require.NoError(t, s.LoadFrom(ctx, l)) - s.IterateFilesFullyBefore(17, func(d *backuppb.DataFileGroup) (shouldBreak bool) { + s.IterateFilesFullyBefore(17, func(d *restore.FileGroupInfo) (shouldBreak bool) { t.Errorf("some of log files still not truncated, it is %#v", d) return true }) - l.WalkDir(ctx, &storage.WalkOption{ + err = l.WalkDir(ctx, &storage.WalkOption{ SubDir: stream.GetStreamBackupMetaPrefix(), }, func(s string, i int64) error { - require.NotContains(t, deletedFiles, s) + require.NotContains(t, removedMetaFiles, s) return nil }) + require.NoError(t, err) } func TestTruncateSafepoint(t *testing.T) { @@ -425,52 +477,1835 @@ func fakeMetaDataV2s(t *testing.T, helper *stream.MetadataHelper, cf string) []* return m2s } +func ff(minTS, maxTS uint64) *backuppb.DataFileGroup { + return f(0, minTS, maxTS, stream.DefaultCF, 0) +} + +func TestReplaceMetadataTs(t *testing.T) { + m := &backuppb.Metadata{} + restore.ReplaceMetadata(m, []*backuppb.DataFileGroup{ + ff(1, 3), + ff(4, 5), + }) + require.Equal(t, m.MinTs, uint64(1)) + require.Equal(t, m.MaxTs, uint64(5)) + + restore.ReplaceMetadata(m, []*backuppb.DataFileGroup{ + ff(1, 4), + ff(3, 5), + }) + require.Equal(t, m.MinTs, uint64(1)) + require.Equal(t, m.MaxTs, uint64(5)) + + restore.ReplaceMetadata(m, []*backuppb.DataFileGroup{ + ff(1, 6), + ff(0, 5), + }) + require.Equal(t, m.MinTs, uint64(0)) + require.Equal(t, m.MaxTs, uint64(6)) + + restore.ReplaceMetadata(m, []*backuppb.DataFileGroup{ + ff(1, 3), + }) + require.Equal(t, m.MinTs, uint64(1)) + require.Equal(t, m.MaxTs, uint64(3)) + + restore.ReplaceMetadata(m, []*backuppb.DataFileGroup{}) + require.Equal(t, m.MinTs, uint64(0)) + require.Equal(t, m.MaxTs, uint64(0)) + + restore.ReplaceMetadata(m, []*backuppb.DataFileGroup{ + ff(1, 3), + ff(2, 4), + ff(0, 2), + }) + require.Equal(t, m.MinTs, uint64(0)) + require.Equal(t, m.MaxTs, uint64(4)) +} + +func m(storeId int64, minTS, maxTS uint64) *backuppb.Metadata { + return &backuppb.Metadata{ + StoreId: storeId, + MinTs: minTS, + MaxTs: maxTS, + MetaVersion: backuppb.MetaVersion_V2, + } +} + +func f(storeId int64, minTS, maxTS uint64, cf string, defaultTS uint64) *backuppb.DataFileGroup { + return &backuppb.DataFileGroup{ + Path: logName(storeId, minTS, maxTS), + DataFilesInfo: []*backuppb.DataFileInfo{ + { + NumberOfEntries: 1, + MinTs: minTS, + MaxTs: maxTS, + Cf: cf, + MinBeginTsInDefaultCf: defaultTS, + }, + }, + MinTs: minTS, + MaxTs: maxTS, + } +} + +// get the metadata with only one datafilegroup +func m_1(storeId int64, minTS, maxTS uint64, cf string, defaultTS uint64) *backuppb.Metadata { + meta := m(storeId, minTS, maxTS) + meta.FileGroups = []*backuppb.DataFileGroup{ + f(storeId, minTS, maxTS, cf, defaultTS), + } + return meta +} + +// get the metadata with 2 datafilegroup +func m_2( + storeId int64, + minTSL, maxTSL uint64, cfL string, defaultTSL uint64, + minTSR, maxTSR uint64, cfR string, defaultTSR uint64, +) *backuppb.Metadata { + meta := m(storeId, minTSL, maxTSR) + meta.FileGroups = []*backuppb.DataFileGroup{ + f(storeId, minTSL, maxTSL, cfL, defaultTSL), + f(storeId, minTSR, maxTSR, cfR, defaultTSR), + } + return meta +} + +// clean the files in the external storage +func cleanFiles(ctx context.Context, s storage.ExternalStorage) error { + names := make([]string, 0) + err := s.WalkDir(ctx, &storage.WalkOption{}, func(path string, size int64) error { + names = append(names, path) + return nil + }) + if err != nil { + return err + } + for _, path := range names { + err := s.DeleteFile(ctx, path) + if err != nil { + return err + } + } + return nil +} + +func metaName(storeId int64) string { + return fmt.Sprintf("%s/%04d.meta", stream.GetStreamBackupMetaPrefix(), storeId) +} + +func logName(storeId int64, minTS, maxTS uint64) string { + return fmt.Sprintf("%04d_%04d_%04d.log", storeId, minTS, maxTS) +} + +// generate the files to the external storage +func generateFiles(ctx context.Context, s storage.ExternalStorage, metas []*backuppb.Metadata, tmpDir string) error { + if err := cleanFiles(ctx, s); err != nil { + return err + } + fname := path.Join(tmpDir, stream.GetStreamBackupMetaPrefix()) + os.MkdirAll(fname, 0777) + for _, meta := range metas { + data, err := meta.Marshal() + if err != nil { + return err + } + + fname := metaName(meta.StoreId) + err = s.WriteFile(ctx, fname, data) + if err != nil { + return err + } + + for _, group := range meta.FileGroups { + fname := logName(meta.StoreId, group.MinTs, group.MaxTs) + err = s.WriteFile(ctx, fname, []byte("test")) + if err != nil { + return err + } + } + } + + return nil +} + +// check the files in the external storage +func checkFiles(ctx context.Context, s storage.ExternalStorage, metas []*backuppb.Metadata, t *testing.T) { + pathSet := make(map[string]struct{}) + for _, meta := range metas { + metaPath := metaName(meta.StoreId) + pathSet[metaPath] = struct{}{} + exists, err := s.FileExists(ctx, metaPath) + require.NoError(t, err) + require.True(t, exists) + + data, err := s.ReadFile(ctx, metaPath) + require.NoError(t, err) + metaRead := &backuppb.Metadata{} + err = metaRead.Unmarshal(data) + require.NoError(t, err) + require.Equal(t, meta.MinTs, metaRead.MinTs) + require.Equal(t, meta.MaxTs, metaRead.MaxTs) + for i, group := range meta.FileGroups { + require.Equal(t, metaRead.FileGroups[i].Path, group.Path) + logPath := logName(meta.StoreId, group.MinTs, group.MaxTs) + pathSet[logPath] = struct{}{} + exists, err := s.FileExists(ctx, logPath) + require.NoError(t, err) + require.True(t, exists) + } + } + + err := s.WalkDir(ctx, &storage.WalkOption{}, func(path string, size int64) error { + _, exists := pathSet[path] + require.True(t, exists, path) + return nil + }) + require.NoError(t, err) +} + +type testParam struct { + until []uint64 + shiftUntilTS uint64 + restMetadata []*backuppb.Metadata +} + +func TestTruncate1(t *testing.T) { + ctx := context.Background() + tmpDir := t.TempDir() + s, err := storage.NewLocalStorage(tmpDir) + require.NoError(t, err) + + cases := []struct { + metas []*backuppb.Metadata + testParams []*testParam + }{ + { + // metadata 10-----------20 + // ↑ ↑ + // +-----------+ + // ↓ ↓ + // filegroup 10-----d-----20 + metas: []*backuppb.Metadata{ + m_1(1, 10, 20, stream.DefaultCF, 0), + }, + testParams: []*testParam{ + { + until: []uint64{5}, + shiftUntilTS: 5, restMetadata: []*backuppb.Metadata{ + m_1(1, 10, 20, stream.DefaultCF, 0), + }, + }, { + until: []uint64{10}, + shiftUntilTS: 10, restMetadata: []*backuppb.Metadata{ + m_1(1, 10, 20, stream.DefaultCF, 0), + }, + }, { + until: []uint64{15}, + shiftUntilTS: 15, restMetadata: []*backuppb.Metadata{ + m_1(1, 10, 20, stream.DefaultCF, 0), + }, + }, { + until: []uint64{20}, + shiftUntilTS: 20, restMetadata: []*backuppb.Metadata{ + m_1(1, 10, 20, stream.DefaultCF, 0), + }, + }, { + until: []uint64{25}, + shiftUntilTS: 25, restMetadata: []*backuppb.Metadata{}, + }, + }, + }, { + // metadata 10-----------20 + // ↑ ↑ + // +-----------+ + // ↓ ↓ + // filegroup 5-d--10-----w-----20 + metas: []*backuppb.Metadata{ + m_1(1, 10, 20, stream.WriteCF, 5), + }, + testParams: []*testParam{ + { + until: []uint64{3}, + shiftUntilTS: 3, restMetadata: []*backuppb.Metadata{ + m_1(1, 10, 20, stream.WriteCF, 5), + }, + }, { + until: []uint64{5, 7, 10, 15, 20}, + shiftUntilTS: 5, restMetadata: []*backuppb.Metadata{ + m_1(1, 10, 20, stream.WriteCF, 5), + }, + }, { + until: []uint64{25}, + shiftUntilTS: 25, restMetadata: []*backuppb.Metadata{}, + }, + }, + }, { + // metadata 5----8 10-----------20 + // ↑ ↑ ↑ ↑ + // +----+ +-----------+ + // ↓ ↓ ↓ ↓ + // filegroup 5--d-8 ↓ ↓ + // filegroup 5--d---10-----w-----20 + metas: []*backuppb.Metadata{ + m_1(1, 5, 8, stream.DefaultCF, 0), + m_1(2, 10, 20, stream.WriteCF, 5), + }, + testParams: []*testParam{ + { + until: []uint64{3}, + shiftUntilTS: 3, restMetadata: []*backuppb.Metadata{ + m_1(1, 5, 8, stream.DefaultCF, 0), + m_1(2, 10, 20, stream.WriteCF, 5), + }, + }, { + until: []uint64{5, 8, 9, 10, 15, 20}, + shiftUntilTS: 5, restMetadata: []*backuppb.Metadata{ + m_1(1, 5, 8, stream.DefaultCF, 0), + m_1(2, 10, 20, stream.WriteCF, 5), + }, + }, { + until: []uint64{25}, + shiftUntilTS: 25, restMetadata: []*backuppb.Metadata{}, + }, + }, + }, { + // metadata 10-----------20 + // metadata 5------10 ↑ + // ↑ ↑ ↑ + // +-------+-----------+ + // ↓ ↓ ↓ + // filegroup 5--d---10 ↓ + // filegroup 5--d---10-----w-----20 + metas: []*backuppb.Metadata{ + m_1(1, 5, 10, stream.DefaultCF, 0), + m_1(2, 10, 20, stream.WriteCF, 5), + }, + testParams: []*testParam{ + { + until: []uint64{3}, + shiftUntilTS: 3, restMetadata: []*backuppb.Metadata{ + m_1(1, 5, 10, stream.DefaultCF, 0), + m_1(2, 10, 20, stream.WriteCF, 5), + }, + }, { + until: []uint64{5, 8, 9, 10, 15, 20}, + shiftUntilTS: 5, restMetadata: []*backuppb.Metadata{ + m_1(1, 5, 10, stream.DefaultCF, 0), + m_1(2, 10, 20, stream.WriteCF, 5), + }, + }, { + until: []uint64{25}, + shiftUntilTS: 25, restMetadata: []*backuppb.Metadata{}, + }, + }, + }, { + // metadata 10-----------20 + // metadata 5-------↑-12 ↑ + // ↑ ↑ ↑ ↑ + // +-------+-+---------+ + // ↓ ↓ ↓ ↓ + // filegroup 5--d----↓-12 ↓ + // filegroup 5--d---10-----w-----20 + metas: []*backuppb.Metadata{ + m_1(1, 5, 12, stream.DefaultCF, 0), + m_1(2, 10, 20, stream.WriteCF, 5), + }, + testParams: []*testParam{ + { + until: []uint64{3}, + shiftUntilTS: 3, restMetadata: []*backuppb.Metadata{ + m_1(1, 5, 12, stream.DefaultCF, 0), + m_1(2, 10, 20, stream.WriteCF, 5), + }, + }, { + until: []uint64{5, 8, 9, 10, 15, 20}, + shiftUntilTS: 5, restMetadata: []*backuppb.Metadata{ + m_1(1, 5, 12, stream.DefaultCF, 0), + m_1(2, 10, 20, stream.WriteCF, 5), + }, + }, { + until: []uint64{25}, + shiftUntilTS: 25, restMetadata: []*backuppb.Metadata{}, + }, + }, + }, { + // metadata 10-----------20 + // metadata 5-------↑-----------20 + // ↑ ↑ ↑ + // +-------+-----------+ + // ↓ ↓ ↓ + // filegroup 5--d----↓-----------20 + // filegroup 5--d---10-----w-----20 + metas: []*backuppb.Metadata{ + m_1(1, 5, 20, stream.DefaultCF, 0), + m_1(2, 10, 20, stream.WriteCF, 5), + }, + testParams: []*testParam{ + { + until: []uint64{3}, + shiftUntilTS: 3, restMetadata: []*backuppb.Metadata{ + m_1(1, 5, 20, stream.DefaultCF, 0), + m_1(2, 10, 20, stream.WriteCF, 5), + }, + }, { + until: []uint64{5, 8, 10, 15, 20}, + shiftUntilTS: 5, restMetadata: []*backuppb.Metadata{ + m_1(1, 5, 20, stream.DefaultCF, 0), + m_1(2, 10, 20, stream.WriteCF, 5), + }, + }, { + until: []uint64{25}, + shiftUntilTS: 25, restMetadata: []*backuppb.Metadata{}, + }, + }, + }, { + // metadata 10-----------20 + // metadata 5-------↑-----------↑--22 + // ↑ ↑ ↑ ↑ + // +-------+-----------+--+ + // ↓ ↓ ↓ ↓ + // filegroup 5--d----↓-----------↓--22 + // filegroup 5--d---10-----w-----20 + metas: []*backuppb.Metadata{ + m_1(1, 5, 22, stream.DefaultCF, 0), + m_1(2, 10, 20, stream.WriteCF, 5), + }, + testParams: []*testParam{ + { + until: []uint64{3}, + shiftUntilTS: 3, restMetadata: []*backuppb.Metadata{ + m_1(1, 5, 22, stream.DefaultCF, 0), + m_1(2, 10, 20, stream.WriteCF, 5), + }, + }, { + until: []uint64{5, 8, 10, 15, 20}, + shiftUntilTS: 5, restMetadata: []*backuppb.Metadata{ + m_1(1, 5, 22, stream.DefaultCF, 0), + m_1(2, 10, 20, stream.WriteCF, 5), + }, + }, { + until: []uint64{21}, + shiftUntilTS: 21, restMetadata: []*backuppb.Metadata{ + m_1(1, 5, 22, stream.DefaultCF, 0), + }, + }, { + until: []uint64{22}, + shiftUntilTS: 22, restMetadata: []*backuppb.Metadata{ + m_1(1, 5, 22, stream.DefaultCF, 0), + }, + }, { + until: []uint64{25}, + shiftUntilTS: 25, restMetadata: []*backuppb.Metadata{}, + }, + }, + }, { + // metadata 10-----------20 + // metadata 10---14 ↑ + // ↑ ↑ ↑ + // +----+-------+ + // ↓ ↓ ↓ + // filegroup 10-d-14 ↓ + // filegroup 5--d---10-----w-----20 + metas: []*backuppb.Metadata{ + m_1(1, 10, 14, stream.DefaultCF, 0), + m_1(2, 10, 20, stream.WriteCF, 5), + }, + testParams: []*testParam{ + { + until: []uint64{3}, + shiftUntilTS: 3, restMetadata: []*backuppb.Metadata{ + m_1(1, 10, 14, stream.DefaultCF, 0), + m_1(2, 10, 20, stream.WriteCF, 5), + }, + }, { + until: []uint64{5, 8, 10, 12, 14, 18, 20}, + shiftUntilTS: 5, restMetadata: []*backuppb.Metadata{ + m_1(1, 10, 14, stream.DefaultCF, 0), + m_1(2, 10, 20, stream.WriteCF, 5), + }, + }, { + until: []uint64{25}, + shiftUntilTS: 25, restMetadata: []*backuppb.Metadata{}, + }, + }, + }, { + // metadata 10-----------20 + // metadata 10-----------20 + // ↑ ↑ + // +------------+ + // ↓ ↓ + // filegroup 10----d------20 + // filegroup 5--d---10-----w-----20 + metas: []*backuppb.Metadata{ + m_1(1, 10, 20, stream.DefaultCF, 0), + m_1(2, 10, 20, stream.WriteCF, 5), + }, + testParams: []*testParam{ + { + until: []uint64{3}, + shiftUntilTS: 3, restMetadata: []*backuppb.Metadata{ + m_1(1, 10, 20, stream.DefaultCF, 0), + m_1(2, 10, 20, stream.WriteCF, 5), + }, + }, { + until: []uint64{5, 8, 10, 14, 20}, + shiftUntilTS: 5, restMetadata: []*backuppb.Metadata{ + m_1(1, 10, 20, stream.DefaultCF, 0), + m_1(2, 10, 20, stream.WriteCF, 5), + }, + }, { + until: []uint64{25}, + shiftUntilTS: 25, restMetadata: []*backuppb.Metadata{}, + }, + }, + }, { + // metadata 10-----------20 + // metadata 10------------↑--22 + // ↑ ↑ ↑ + // +------------+---+ + // ↓ ↓ ↓ + // filegroup 10----d-------↓--22 + // filegroup 5--d---10-----w-----20 + metas: []*backuppb.Metadata{ + m_1(1, 10, 22, stream.DefaultCF, 0), + m_1(2, 10, 20, stream.WriteCF, 5), + }, + testParams: []*testParam{ + { + until: []uint64{3}, + shiftUntilTS: 3, restMetadata: []*backuppb.Metadata{ + m_1(1, 10, 22, stream.DefaultCF, 0), + m_1(2, 10, 20, stream.WriteCF, 5), + }, + }, { + until: []uint64{5, 8, 10, 14, 20}, + shiftUntilTS: 5, restMetadata: []*backuppb.Metadata{ + m_1(1, 10, 22, stream.DefaultCF, 0), + m_1(2, 10, 20, stream.WriteCF, 5), + }, + }, { + until: []uint64{21}, + shiftUntilTS: 21, restMetadata: []*backuppb.Metadata{ + m_1(1, 10, 22, stream.DefaultCF, 0), + }, + }, { + until: []uint64{22}, + shiftUntilTS: 22, restMetadata: []*backuppb.Metadata{ + m_1(1, 10, 22, stream.DefaultCF, 0), + }, + }, { + until: []uint64{25}, + shiftUntilTS: 25, restMetadata: []*backuppb.Metadata{}, + }, + }, + }, { + // metadata 10-----------20 + // metadata ↑ 12-----18 ↑ + // ↑ ↑ ↑ ↑ + // +--+------+--+ + // ↓ ↓ ↓ ↓ + // filegroup ↓ 12--d--18 ↓ + // filegroup 5--d---10-----w-----20 + metas: []*backuppb.Metadata{ + m_1(1, 12, 18, stream.DefaultCF, 0), + m_1(2, 10, 20, stream.WriteCF, 5), + }, + testParams: []*testParam{ + { + until: []uint64{3}, + shiftUntilTS: 3, restMetadata: []*backuppb.Metadata{ + m_1(1, 12, 18, stream.DefaultCF, 0), + m_1(2, 10, 20, stream.WriteCF, 5), + }, + }, { + until: []uint64{5, 8, 10, 11, 12, 15, 18, 19, 20}, + shiftUntilTS: 5, restMetadata: []*backuppb.Metadata{ + m_1(1, 12, 18, stream.DefaultCF, 0), + m_1(2, 10, 20, stream.WriteCF, 5), + }, + }, { + until: []uint64{25}, + shiftUntilTS: 25, restMetadata: []*backuppb.Metadata{}, + }, + }, + }, { + // metadata 10-----------20 + // metadata ↑ 14----20 + // ↑ ↑ ↑ + // +------+-----+ + // ↓ ↓ ↓ + // filegroup ↓ 14--d-20 + // filegroup 5--d---10-----w-----20 + metas: []*backuppb.Metadata{ + m_1(1, 14, 20, stream.DefaultCF, 0), + m_1(2, 10, 20, stream.WriteCF, 5), + }, + testParams: []*testParam{ + { + until: []uint64{3}, + shiftUntilTS: 3, restMetadata: []*backuppb.Metadata{ + m_1(1, 14, 20, stream.DefaultCF, 0), + m_1(2, 10, 20, stream.WriteCF, 5), + }, + }, { + until: []uint64{5, 8, 10, 14, 20}, + shiftUntilTS: 5, restMetadata: []*backuppb.Metadata{ + m_1(1, 14, 20, stream.DefaultCF, 0), + m_1(2, 10, 20, stream.WriteCF, 5), + }, + }, { + until: []uint64{25}, + shiftUntilTS: 25, restMetadata: []*backuppb.Metadata{}, + }, + }, + }, { + // metadata 10-----------20 + // metadata ↑ 14-----↑--22 + // ↑ ↑ ↑ ↑ + // +------+-----+---+ + // ↓ ↓ ↓ ↓ + // filegroup ↓ 14-d--↓--22 + // filegroup 5--d---10-----w-----20 + metas: []*backuppb.Metadata{ + m_1(1, 14, 22, stream.DefaultCF, 0), + m_1(2, 10, 20, stream.WriteCF, 5), + }, + testParams: []*testParam{ + { + until: []uint64{3}, + shiftUntilTS: 3, restMetadata: []*backuppb.Metadata{ + m_1(1, 14, 22, stream.DefaultCF, 0), + m_1(2, 10, 20, stream.WriteCF, 5), + }, + }, { + until: []uint64{5, 8, 10, 14, 20}, + shiftUntilTS: 5, restMetadata: []*backuppb.Metadata{ + m_1(1, 14, 22, stream.DefaultCF, 0), + m_1(2, 10, 20, stream.WriteCF, 5), + }, + }, { + until: []uint64{21}, + shiftUntilTS: 21, restMetadata: []*backuppb.Metadata{ + m_1(1, 14, 22, stream.DefaultCF, 0), + }, + }, { + until: []uint64{22}, + shiftUntilTS: 22, restMetadata: []*backuppb.Metadata{ + m_1(1, 14, 22, stream.DefaultCF, 0), + }, + }, { + until: []uint64{25}, + shiftUntilTS: 25, restMetadata: []*backuppb.Metadata{}, + }, + }, + }, { + // metadata 10-----------20 + // metadata ↑ 20--22 + // ↑ ↑ ↑ + // +------------+---+ + // ↓ ↓ ↓ + // filegroup ↓ 20--22 + // filegroup 5--d---10-----w-----20 + metas: []*backuppb.Metadata{ + m_1(1, 20, 22, stream.DefaultCF, 0), + m_1(2, 10, 20, stream.WriteCF, 5), + }, + testParams: []*testParam{ + { + until: []uint64{3}, + shiftUntilTS: 3, restMetadata: []*backuppb.Metadata{ + m_1(1, 20, 22, stream.DefaultCF, 0), + m_1(2, 10, 20, stream.WriteCF, 5), + }, + }, { + until: []uint64{5, 8, 10, 14, 20}, + shiftUntilTS: 5, restMetadata: []*backuppb.Metadata{ + m_1(1, 20, 22, stream.DefaultCF, 0), + m_1(2, 10, 20, stream.WriteCF, 5), + }, + }, { + until: []uint64{21}, + shiftUntilTS: 21, restMetadata: []*backuppb.Metadata{ + m_1(1, 20, 22, stream.DefaultCF, 0), + }, + }, { + until: []uint64{22}, + shiftUntilTS: 22, restMetadata: []*backuppb.Metadata{ + m_1(1, 20, 22, stream.DefaultCF, 0), + }, + }, { + until: []uint64{25}, + shiftUntilTS: 25, restMetadata: []*backuppb.Metadata{}, + }, + }, + }, { + // metadata 10-----------20 + // metadata ↑ ↑ 21---24 + // ↑ ↑ ↑ ↑ + // +------------+--+----+ + // ↓ ↓ ↓ ↓ + // filegroup ↓ ↓ 21-d-24 + // filegroup 5--d---10-----w-----20 + metas: []*backuppb.Metadata{ + m_1(1, 21, 24, stream.DefaultCF, 0), + m_1(2, 10, 20, stream.WriteCF, 5), + }, + testParams: []*testParam{ + { + until: []uint64{3}, + shiftUntilTS: 3, restMetadata: []*backuppb.Metadata{ + m_1(1, 21, 24, stream.DefaultCF, 0), + m_1(2, 10, 20, stream.WriteCF, 5), + }, + }, { + until: []uint64{5, 8, 10, 14, 20}, + shiftUntilTS: 5, restMetadata: []*backuppb.Metadata{ + m_1(1, 21, 24, stream.DefaultCF, 0), + m_1(2, 10, 20, stream.WriteCF, 5), + }, + }, { + until: []uint64{21}, + shiftUntilTS: 21, restMetadata: []*backuppb.Metadata{ + m_1(1, 21, 24, stream.DefaultCF, 0), + }, + }, { + until: []uint64{22}, + shiftUntilTS: 22, restMetadata: []*backuppb.Metadata{ + m_1(1, 21, 24, stream.DefaultCF, 0), + }, + }, { + until: []uint64{25}, + shiftUntilTS: 25, restMetadata: []*backuppb.Metadata{}, + }, + }, + }, + } + + for i, cs := range cases { + for j, ts := range cs.testParams { + for _, until := range ts.until { + t.Logf("case %d, param %d, until %d", i, j, until) + metas := restore.StreamMetadataSet{ + Helper: stream.NewMetadataHelper(), + } + err := generateFiles(ctx, s, cs.metas, tmpDir) + require.NoError(t, err) + shiftUntilTS, err := metas.LoadUntilAndCalculateShiftTS(ctx, s, until) + require.NoError(t, err) + require.Equal(t, shiftUntilTS, ts.shiftUntilTS) + n, err := metas.RemoveDataFilesAndUpdateMetadataInBatch(ctx, shiftUntilTS, s, func(num int64) {}) + require.Equal(t, len(n), 0) + require.NoError(t, err) + + // check the result + checkFiles(ctx, s, ts.restMetadata, t) + } + } + } +} + +type testParam2 struct { + until []uint64 + shiftUntilTS func(uint64) uint64 + restMetadata []*backuppb.Metadata +} + +func returnV(v uint64) func(uint64) uint64 { + return func(uint64) uint64 { + return v + } +} + +func returnSelf() func(uint64) uint64 { + return func(u uint64) uint64 { + return u + } +} + +func TestTruncate2(t *testing.T) { + ctx := context.Background() + tmpDir := t.TempDir() + s, err := storage.NewLocalStorage(tmpDir) + require.NoError(t, err) + + cases := []struct { + metas []*backuppb.Metadata + testParams []*testParam2 + }{ + { + // metadata 10-----------20 + // ↑ ↑ + // +-----------+ + // ↓ ↓ ↓ ↓ + // filegroup 10-d-13 ↓ ↓ + // filegroup 8----d--15-w-20 + metas: []*backuppb.Metadata{ + m_2(1, + 10, 13, stream.DefaultCF, 0, + 15, 20, stream.WriteCF, 8, + ), + }, + testParams: []*testParam2{ + { + until: []uint64{5}, + shiftUntilTS: returnV(5), restMetadata: []*backuppb.Metadata{ + m_2(1, + 10, 13, stream.DefaultCF, 0, + 15, 20, stream.WriteCF, 8, + ), + }, + }, { + until: []uint64{8, 9, 10, 12, 13, 14, 15, 18, 20}, + shiftUntilTS: returnV(8), restMetadata: []*backuppb.Metadata{ + m_2(1, + 10, 13, stream.DefaultCF, 0, + 15, 20, stream.WriteCF, 8, + ), + }, + }, { + until: []uint64{25}, + shiftUntilTS: returnV(25), restMetadata: []*backuppb.Metadata{}, + }, + }, + }, { + // metadata 3---6 10----------20 + // ↑ ↑ ↑ ↑ + // +---+ +-----------+ + // ↓ ↓ ↓ ↓ ↓ ↓ + // filegroup 3 6 10-d-13 ↓ ↓ + // filegroup 1-----------d--15-w-20 + metas: []*backuppb.Metadata{ + m_1(1, 3, 6, stream.DefaultCF, 0), + m_2(2, + 10, 13, stream.DefaultCF, 0, + 15, 20, stream.WriteCF, 1, + ), + }, + testParams: []*testParam2{ + { + until: []uint64{0}, + shiftUntilTS: returnV(0), restMetadata: []*backuppb.Metadata{ + m_1(1, 3, 6, stream.DefaultCF, 0), + m_2(2, + 10, 13, stream.DefaultCF, 0, + 15, 20, stream.WriteCF, 8, + ), + }, + }, { + until: []uint64{1, 2, 3, 4, 6, 9, 10, 12, 13, 14, 15, 18, 20}, + shiftUntilTS: returnV(1), restMetadata: []*backuppb.Metadata{ + m_1(1, 3, 6, stream.DefaultCF, 0), + m_2(2, + 10, 13, stream.DefaultCF, 0, + 15, 20, stream.WriteCF, 8, + ), + }, + }, { + until: []uint64{25}, + shiftUntilTS: returnV(25), restMetadata: []*backuppb.Metadata{}, + }, + }, + }, { + // metadata 3---6 10----------20 + // ↑ ↑ ↑ ↑ + // +---+ +-----------+ + // ↓ ↓ ↓ ↓ ↓ ↓ + // filegroup 3 6 10-d-13 ↓ ↓ + // filegroup 3----------d--15-w-20 + metas: []*backuppb.Metadata{ + m_1(1, 3, 6, stream.DefaultCF, 0), + m_2(2, + 10, 13, stream.DefaultCF, 0, + 15, 20, stream.WriteCF, 3, + ), + }, + testParams: []*testParam2{ + { + until: []uint64{2}, + shiftUntilTS: returnV(2), restMetadata: []*backuppb.Metadata{ + m_1(1, 3, 6, stream.DefaultCF, 0), + m_2(2, + 10, 13, stream.DefaultCF, 0, + 15, 20, stream.WriteCF, 3, + ), + }, + }, { + until: []uint64{3, 4, 6, 9, 10, 12, 13, 14, 15, 18, 20}, + shiftUntilTS: returnV(3), restMetadata: []*backuppb.Metadata{ + m_1(1, 3, 6, stream.DefaultCF, 0), + m_2(2, + 10, 13, stream.DefaultCF, 0, + 15, 20, stream.WriteCF, 3, + ), + }, + }, { + until: []uint64{25}, + shiftUntilTS: returnV(25), restMetadata: []*backuppb.Metadata{}, + }, + }, + }, { + // metadata 3---7 10----------20 + // ↑ ↑ ↑ ↑ + // +---+ +----+-+----+ + // ↓ ↓ ↓ ↓ ↓ ↓ + // filegroup 3 7 10-d-13 ↓ ↓ + // filegroup 5--------d--15-w-20 + metas: []*backuppb.Metadata{ + m_1(1, 3, 7, stream.DefaultCF, 0), + m_2(2, + 10, 13, stream.DefaultCF, 0, + 15, 20, stream.WriteCF, 5, + ), + }, + testParams: []*testParam2{ + { + until: []uint64{2, 3, 4}, + shiftUntilTS: returnSelf(), restMetadata: []*backuppb.Metadata{ + m_1(1, 3, 7, stream.DefaultCF, 0), + m_2(2, + 10, 13, stream.DefaultCF, 0, + 15, 20, stream.WriteCF, 5, + ), + }, + }, { + until: []uint64{5, 6, 7, 9, 10, 12, 13, 14, 15, 18, 20}, + shiftUntilTS: returnV(5), restMetadata: []*backuppb.Metadata{ + m_1(1, 3, 7, stream.DefaultCF, 0), + m_2(2, + 10, 13, stream.DefaultCF, 0, + 15, 20, stream.WriteCF, 5, + ), + }, + }, { + until: []uint64{25}, + shiftUntilTS: returnV(25), restMetadata: []*backuppb.Metadata{}, + }, + }, + }, { + // metadata 3---7 10----------20 + // ↑ ↑ ↑ ↑ + // +---+ +----+-+----+ + // ↓ ↓ ↓ ↓ ↓ ↓ + // filegroup 3 7 10-d-13 ↓ ↓ + // filegroup 7------d--15-w-20 + metas: []*backuppb.Metadata{ + m_1(1, 3, 7, stream.DefaultCF, 0), + m_2(2, + 10, 13, stream.DefaultCF, 0, + 15, 20, stream.WriteCF, 7, + ), + }, + testParams: []*testParam2{ + { + until: []uint64{2, 3, 4, 6, 7}, + shiftUntilTS: returnSelf(), restMetadata: []*backuppb.Metadata{ + m_1(1, 3, 7, stream.DefaultCF, 0), + m_2(2, + 10, 13, stream.DefaultCF, 0, + 15, 20, stream.WriteCF, 7, + ), + }, + }, { + until: []uint64{9, 10, 12, 13, 14, 15, 18, 20}, + shiftUntilTS: returnV(7), restMetadata: []*backuppb.Metadata{ + m_1(1, 3, 7, stream.DefaultCF, 0), + m_2(2, + 10, 13, stream.DefaultCF, 0, + 15, 20, stream.WriteCF, 7, + ), + }, + }, { + until: []uint64{25}, + shiftUntilTS: returnV(25), restMetadata: []*backuppb.Metadata{}, + }, + }, + }, { + // metadata 3---6 10----------20 + // ↑ ↑ ↑ ↑ + // +---+ +----+-+----+ + // ↓ ↓ ↓ ↓ ↓ ↓ + // filegroup 3-d-6 10-d-13 ↓ ↓ + // filegroup 8----d--15-w-20 + metas: []*backuppb.Metadata{ + m_1(1, 3, 6, stream.DefaultCF, 0), + m_2(2, + 10, 13, stream.DefaultCF, 0, + 15, 20, stream.WriteCF, 8, + ), + }, + testParams: []*testParam2{ + { + until: []uint64{2, 3, 4, 6}, + shiftUntilTS: returnSelf(), restMetadata: []*backuppb.Metadata{ + m_1(1, 3, 6, stream.DefaultCF, 0), + m_2(2, + 10, 13, stream.DefaultCF, 0, + 15, 20, stream.WriteCF, 8, + ), + }, + }, { + until: []uint64{7}, + shiftUntilTS: returnSelf(), restMetadata: []*backuppb.Metadata{ + m_2(2, + 10, 13, stream.DefaultCF, 0, + 15, 20, stream.WriteCF, 8, + ), + }, + }, { + until: []uint64{8, 9, 10, 12, 13, 14, 15, 18, 20}, + shiftUntilTS: returnV(8), restMetadata: []*backuppb.Metadata{ + m_2(2, + 10, 13, stream.DefaultCF, 0, + 15, 20, stream.WriteCF, 8, + ), + }, + }, { + until: []uint64{25}, + shiftUntilTS: returnV(25), restMetadata: []*backuppb.Metadata{}, + }, + }, + }, { + // metadata 3---6 10----------20 + // ↑ ↑ ↑ ↑ + // +---+ +----+-+----+ + // ↓ ↓ ↓ ↓ ↓ ↓ + // filegroup 3-d-6 10-d-13 ↓ ↓ + // filegroup 10--d--15-w-20 + metas: []*backuppb.Metadata{ + m_1(1, 3, 6, stream.DefaultCF, 0), + m_2(2, + 10, 13, stream.DefaultCF, 0, + 15, 20, stream.WriteCF, 10, + ), + }, + testParams: []*testParam2{ + { + until: []uint64{2, 3, 4, 6}, + shiftUntilTS: returnSelf(), restMetadata: []*backuppb.Metadata{ + m_1(1, 3, 6, stream.DefaultCF, 0), + m_2(2, + 10, 13, stream.DefaultCF, 0, + 15, 20, stream.WriteCF, 10, + ), + }, + }, { + until: []uint64{7, 8, 9}, + shiftUntilTS: returnSelf(), restMetadata: []*backuppb.Metadata{ + m_2(2, + 10, 13, stream.DefaultCF, 0, + 15, 20, stream.WriteCF, 10, + ), + }, + }, { + until: []uint64{10, 12, 13, 14, 15, 18, 20}, + shiftUntilTS: returnV(10), restMetadata: []*backuppb.Metadata{ + m_2(2, + 10, 13, stream.DefaultCF, 0, + 15, 20, stream.WriteCF, 10, + ), + }, + }, { + until: []uint64{25}, + shiftUntilTS: returnV(25), restMetadata: []*backuppb.Metadata{}, + }, + }, + }, { + // metadata 3---6 10----------20 + // ↑ ↑ ↑ ↑ + // +---+ +----+-+----+ + // ↓ ↓ ↓ ↓ ↓ ↓ + // filegroup 3-d-6 9-d-13 ↓ ↓ + // filegroup 11-d-15-w-20 + metas: []*backuppb.Metadata{ + m_1(1, 3, 6, stream.DefaultCF, 0), + m_2(2, + 9, 13, stream.DefaultCF, 0, + 15, 20, stream.WriteCF, 11, + ), + }, + testParams: []*testParam2{ + { + until: []uint64{2, 3, 4, 6}, + shiftUntilTS: returnSelf(), restMetadata: []*backuppb.Metadata{ + m_1(1, 3, 6, stream.DefaultCF, 0), + m_2(2, + 9, 13, stream.DefaultCF, 0, + 15, 20, stream.WriteCF, 11, + ), + }, + }, { + until: []uint64{7, 8, 9, 10}, + shiftUntilTS: returnSelf(), restMetadata: []*backuppb.Metadata{ + m_2(2, + 9, 13, stream.DefaultCF, 0, + 15, 20, stream.WriteCF, 11, + ), + }, + }, { + until: []uint64{11, 12, 13, 14, 15, 18, 20}, + shiftUntilTS: returnV(11), restMetadata: []*backuppb.Metadata{ + m_2(2, + 9, 13, stream.DefaultCF, 0, + 15, 20, stream.WriteCF, 11, + ), + }, + }, { + until: []uint64{25}, + shiftUntilTS: returnV(25), restMetadata: []*backuppb.Metadata{}, + }, + }, + }, { + // metadata 3---6 10----------20 + // ↑ ↑ ↑ ↑ + // +---+ +----+-+----+ + // ↓ ↓ ↓ ↓ ↓ ↓ + // filegroup 3-d-6 10-d-13 ↓ ↓ + // filegroup 13d15-w-20 + metas: []*backuppb.Metadata{ + m_1(1, 3, 6, stream.DefaultCF, 0), + m_2(2, + 10, 13, stream.DefaultCF, 0, + 15, 20, stream.WriteCF, 13, + ), + }, + testParams: []*testParam2{ + { + until: []uint64{2, 3, 4, 6}, + shiftUntilTS: returnSelf(), restMetadata: []*backuppb.Metadata{ + m_1(1, 3, 6, stream.DefaultCF, 0), + m_2(2, + 10, 13, stream.DefaultCF, 0, + 15, 20, stream.WriteCF, 13, + ), + }, + }, { + until: []uint64{7, 8, 9, 10, 12}, + shiftUntilTS: returnSelf(), restMetadata: []*backuppb.Metadata{ + m_2(2, + 10, 13, stream.DefaultCF, 0, + 15, 20, stream.WriteCF, 13, + ), + }, + }, { + until: []uint64{13, 14, 15, 18, 20}, + shiftUntilTS: returnV(13), restMetadata: []*backuppb.Metadata{ + m_2(2, + 10, 13, stream.DefaultCF, 0, + 15, 20, stream.WriteCF, 13, + ), + }, + }, { + until: []uint64{25}, + shiftUntilTS: returnV(25), restMetadata: []*backuppb.Metadata{}, + }, + }, + }, { + // metadata 3---6 10----------20 + // ↑ ↑ ↑ ↑ + // +---+ +----+--+---+ + // ↓ ↓ ↓ ↓ ↓ ↓ + // filegroup 3-d-6 10-d-12 ↓ ↓ + // filegroup 14d16-w-20 + metas: []*backuppb.Metadata{ + m_1(1, 3, 6, stream.DefaultCF, 0), + m_2(2, + 10, 12, stream.DefaultCF, 0, + 16, 20, stream.WriteCF, 14, + ), + }, + testParams: []*testParam2{ + { + until: []uint64{2, 3, 4, 6}, + shiftUntilTS: returnSelf(), restMetadata: []*backuppb.Metadata{ + m_1(1, 3, 6, stream.DefaultCF, 0), + m_2(2, + 10, 12, stream.DefaultCF, 0, + 16, 20, stream.WriteCF, 14, + ), + }, + }, { + until: []uint64{7, 8, 9, 10, 11, 12}, + shiftUntilTS: returnSelf(), restMetadata: []*backuppb.Metadata{ + m_2(2, + 10, 12, stream.DefaultCF, 0, + 16, 20, stream.WriteCF, 14, + ), + }, + }, { + until: []uint64{13}, + shiftUntilTS: returnSelf(), restMetadata: []*backuppb.Metadata{ + m_1(2, 16, 20, stream.WriteCF, 14), + }, + }, { + until: []uint64{14, 15, 18, 20}, + shiftUntilTS: returnV(14), restMetadata: []*backuppb.Metadata{ + m_1(2, 16, 20, stream.WriteCF, 14), + }, + }, { + until: []uint64{25}, + shiftUntilTS: returnV(25), restMetadata: []*backuppb.Metadata{}, + }, + }, + }, { + // metadata 3---6 10----------20 + // ↑ ↑ ↑ ↑ + // +---+ +----+--+---+ + // ↓ ↓ ↓ ↓ ↓ ↓ + // filegroup 3-d-6 10-d-12 ↓ ↓ + // filegroup 14d16-w-20 + metas: []*backuppb.Metadata{ + m_1(1, 3, 6, stream.DefaultCF, 0), + m_2(2, + 10, 12, stream.DefaultCF, 0, + 16, 20, stream.WriteCF, 14, + ), + }, + testParams: []*testParam2{ + { + until: []uint64{2, 3, 4, 6}, + shiftUntilTS: returnSelf(), restMetadata: []*backuppb.Metadata{ + m_1(1, 3, 6, stream.DefaultCF, 0), + m_2(2, + 10, 12, stream.DefaultCF, 0, + 16, 20, stream.WriteCF, 14, + ), + }, + }, { + until: []uint64{7, 8, 9, 10, 11, 12}, + shiftUntilTS: returnSelf(), restMetadata: []*backuppb.Metadata{ + m_2(2, + 10, 12, stream.DefaultCF, 0, + 16, 20, stream.WriteCF, 14, + ), + }, + }, { + until: []uint64{13}, + shiftUntilTS: returnSelf(), restMetadata: []*backuppb.Metadata{ + m_1(2, 16, 20, stream.WriteCF, 14), + }, + }, { + until: []uint64{14, 15, 18, 20}, + shiftUntilTS: returnV(14), restMetadata: []*backuppb.Metadata{ + m_1(2, 16, 20, stream.WriteCF, 14), + }, + }, { + until: []uint64{25}, + shiftUntilTS: returnV(25), restMetadata: []*backuppb.Metadata{}, + }, + }, + }, + } + + for i, cs := range cases { + for j, ts := range cs.testParams { + for _, until := range ts.until { + t.Logf("case %d, param %d, until %d", i, j, until) + metas := restore.StreamMetadataSet{ + Helper: stream.NewMetadataHelper(), + } + err := generateFiles(ctx, s, cs.metas, tmpDir) + require.NoError(t, err) + shiftUntilTS, err := metas.LoadUntilAndCalculateShiftTS(ctx, s, until) + require.NoError(t, err) + require.Equal(t, shiftUntilTS, ts.shiftUntilTS(until)) + n, err := metas.RemoveDataFilesAndUpdateMetadataInBatch(ctx, shiftUntilTS, s, func(num int64) {}) + require.Equal(t, len(n), 0) + require.NoError(t, err) + + // check the result + checkFiles(ctx, s, ts.restMetadata, t) + } + } + } +} + +func TestTruncate3(t *testing.T) { + ctx := context.Background() + tmpDir := t.TempDir() + s, err := storage.NewLocalStorage(tmpDir) + require.NoError(t, err) + + cases := []struct { + metas []*backuppb.Metadata + testParams []*testParam2 + }{ + { + // metadata 3------10 12----------20 + // ↑ ↑ ↑ ↑ + // +-+--+--+ +----+--+---+ + // ↓ ↓ ↓ ↓ ↓ ↓ ↓ ↓ + // filegroup 3--d-7 ↓ ↓ ↓ ↓ ↓ + // filegroup 5--d-10 ↓ ↓ ↓ ↓ + // filegroup 3----d-----12---w--18 ↓ + // filegroup 5----d--------15--w--20 + metas: []*backuppb.Metadata{ + m_2(1, + 3, 7, stream.DefaultCF, 0, + 5, 10, stream.DefaultCF, 0, + ), + m_2(2, + 12, 18, stream.WriteCF, 3, + 15, 20, stream.WriteCF, 5, + ), + }, + testParams: []*testParam2{ + { + until: []uint64{2}, + shiftUntilTS: returnV(2), restMetadata: []*backuppb.Metadata{ + m_2(1, + 3, 7, stream.DefaultCF, 0, + 5, 10, stream.DefaultCF, 0, + ), + m_2(2, + 12, 18, stream.WriteCF, 3, + 15, 20, stream.WriteCF, 5, + ), + }, + }, { + until: []uint64{3, 4, 5, 6, 7, 8, 10, 11, 12, 13, 14, 15, 16, 18}, + shiftUntilTS: returnV(3), restMetadata: []*backuppb.Metadata{ + m_2(1, + 3, 7, stream.DefaultCF, 0, + 5, 10, stream.DefaultCF, 0, + ), + m_2(2, + 12, 18, stream.WriteCF, 3, + 15, 20, stream.WriteCF, 5, + ), + }, + }, { + until: []uint64{19, 20}, + shiftUntilTS: returnV(5), restMetadata: []*backuppb.Metadata{ + m_2(1, + 3, 7, stream.DefaultCF, 0, + 5, 10, stream.DefaultCF, 0, + ), + m_2(2, + 12, 18, stream.WriteCF, 3, + 15, 20, stream.WriteCF, 5, + ), + }, + }, { + until: []uint64{25}, + shiftUntilTS: returnV(25), restMetadata: []*backuppb.Metadata{}, + }, + }, + }, { + // metadata 2------10 12----------20 + // ↑ ↑ ↑ ↑ + // +-+--+--+ +----+--+---+ + // ↓ ↓ ↓ ↓ ↓ ↓ ↓ ↓ + // filegroup 2--d-6 ↓ ↓ ↓ ↓ ↓ + // filegroup 4--d-10 ↓ ↓ ↓ ↓ + // filegroup 2----d-----12---w--18 ↓ + // filegroup 8---d----15--w--20 + metas: []*backuppb.Metadata{ + m_2(1, + 2, 6, stream.DefaultCF, 0, + 4, 10, stream.DefaultCF, 0, + ), + m_2(2, + 12, 18, stream.WriteCF, 2, + 15, 20, stream.WriteCF, 8, + ), + }, + testParams: []*testParam2{ + { + until: []uint64{1}, + shiftUntilTS: returnV(1), restMetadata: []*backuppb.Metadata{ + m_2(1, + 2, 6, stream.DefaultCF, 0, + 4, 10, stream.DefaultCF, 0, + ), + m_2(2, + 12, 18, stream.WriteCF, 2, + 15, 20, stream.WriteCF, 8, + ), + }, + }, { + until: []uint64{2, 3, 4, 5, 6, 7, 8, 10, 11, 12, 13, 14, 15, 16, 18}, + shiftUntilTS: returnV(2), restMetadata: []*backuppb.Metadata{ + m_2(1, + 2, 6, stream.DefaultCF, 0, + 4, 10, stream.DefaultCF, 0, + ), + m_2(2, + 12, 18, stream.WriteCF, 2, + 15, 20, stream.WriteCF, 8, + ), + }, + }, { + until: []uint64{19, 20}, + shiftUntilTS: returnV(8), restMetadata: []*backuppb.Metadata{ + m_1(1, + 4, 10, stream.DefaultCF, 0, + ), + m_2(2, + 12, 18, stream.WriteCF, 2, + 15, 20, stream.WriteCF, 8, + ), + }, + }, { + until: []uint64{25}, + shiftUntilTS: returnV(25), restMetadata: []*backuppb.Metadata{}, + }, + }, + }, { + // metadata 2------10 14----------20 + // ↑ ↑ ↑ ↑ + // +-+--+--+ +----+--+---+ + // ↓ ↓ ↓ ↓ ↓ ↓ ↓ ↓ + // filegroup 2--d-6 ↓ ↓ ↓ ↓ ↓ + // filegroup 4--d-10 ↓ ↓ ↓ ↓ + // filegroup 2----d-------14---w--18 ↓ + // filegroup 12---d--16--w--20 + metas: []*backuppb.Metadata{ + m_2(1, + 2, 6, stream.DefaultCF, 0, + 4, 10, stream.DefaultCF, 0, + ), + m_2(2, + 14, 18, stream.WriteCF, 2, + 16, 20, stream.WriteCF, 12, + ), + }, + testParams: []*testParam2{ + { + until: []uint64{1}, + shiftUntilTS: returnV(1), restMetadata: []*backuppb.Metadata{ + m_2(1, + 2, 6, stream.DefaultCF, 0, + 4, 10, stream.DefaultCF, 0, + ), + m_2(2, + 14, 18, stream.WriteCF, 2, + 16, 20, stream.WriteCF, 12, + ), + }, + }, { + until: []uint64{2, 3, 4, 5, 6, 7, 8, 10, 11, 12, 13, 14, 15, 16, 18}, + shiftUntilTS: returnV(2), restMetadata: []*backuppb.Metadata{ + m_2(1, + 2, 6, stream.DefaultCF, 0, + 4, 10, stream.DefaultCF, 0, + ), + m_2(2, + 14, 18, stream.WriteCF, 2, + 16, 20, stream.WriteCF, 12, + ), + }, + }, { + until: []uint64{19, 20}, + shiftUntilTS: returnV(12), restMetadata: []*backuppb.Metadata{ + m_2(2, + 14, 18, stream.WriteCF, 2, + 16, 20, stream.WriteCF, 8, + ), + }, + }, { + until: []uint64{25}, + shiftUntilTS: returnV(25), restMetadata: []*backuppb.Metadata{}, + }, + }, + }, { + // metadata 2-------10 14----------20 + // ↑ ↑ ↑ ↑ + // +-+--+---+ +----+--+---+ + // ↓ ↓ ↓ ↓ ↓ ↓ ↓ ↓ + // filegroup 2--d-6 ↓ ↓ ↓ ↓ ↓ + // filegroup 4-d-8w10 ↓ ↓ ↓ ↓ + // filegroup 14--d---18 ↓ + // filegroup 14-d--16-w--20 + metas: []*backuppb.Metadata{ + m_2(1, + 2, 6, stream.DefaultCF, 0, + 8, 10, stream.WriteCF, 4, + ), + m_2(2, + 14, 18, stream.DefaultCF, 0, + 16, 20, stream.WriteCF, 14, + ), + }, + testParams: []*testParam2{ + { + until: []uint64{1}, + shiftUntilTS: returnV(1), restMetadata: []*backuppb.Metadata{ + m_2(1, + 2, 6, stream.DefaultCF, 0, + 8, 10, stream.WriteCF, 4, + ), + m_2(2, + 14, 18, stream.DefaultCF, 0, + 16, 20, stream.WriteCF, 14, + ), + }, + }, { + until: []uint64{2, 3}, + shiftUntilTS: returnSelf(), restMetadata: []*backuppb.Metadata{ + m_2(1, + 2, 6, stream.DefaultCF, 0, + 8, 10, stream.WriteCF, 4, + ), + m_2(2, + 14, 18, stream.DefaultCF, 0, + 16, 20, stream.WriteCF, 14, + ), + }, + }, { + until: []uint64{4, 5, 6, 7, 8, 9, 10}, + shiftUntilTS: returnV(4), restMetadata: []*backuppb.Metadata{ + m_2(1, + 2, 6, stream.DefaultCF, 0, + 8, 10, stream.WriteCF, 4, + ), + m_2(2, + 14, 18, stream.DefaultCF, 0, + 16, 20, stream.WriteCF, 14, + ), + }, + }, { + until: []uint64{12}, + shiftUntilTS: returnV(12), restMetadata: []*backuppb.Metadata{ + m_2(2, + 14, 18, stream.DefaultCF, 0, + 16, 20, stream.WriteCF, 14, + ), + }, + }, { + until: []uint64{14, 15, 16, 17, 18, 19, 20}, + shiftUntilTS: returnV(14), restMetadata: []*backuppb.Metadata{ + m_2(2, + 14, 18, stream.DefaultCF, 0, + 16, 20, stream.WriteCF, 14, + ), + }, + }, { + until: []uint64{25}, + shiftUntilTS: returnV(25), restMetadata: []*backuppb.Metadata{}, + }, + }, + }, { + // metadata 2-------10 14----------22 24-w-26 + // ↑ ↑ ↑ ↑ ↑ ↑ + // +-+--+---+ +----+--+---+ +----+ + // ↓ ↓ ↓ ↓ ↓ ↓ ↓ ↓ ↓ ↓ + // filegroup 2--d-6 ↓ ↓ ↓ ↓ ↓ ↓ ↓ + // filegroup 8d10 ↓ ↓ ↓ ↓ ↓ ↓ + // filegroup 9--d--14--w---18 ↓ ↓ ↓ + // filegroup 16-d--22 ↓ ↓ + // filegroup 20---d-24-w-26 + metas: []*backuppb.Metadata{ + m_2(1, + 2, 6, stream.DefaultCF, 0, + 8, 10, stream.DefaultCF, 0, + ), + m_2(2, + 14, 18, stream.WriteCF, 9, + 16, 22, stream.DefaultCF, 0, + ), + m_1(3, + 24, 26, stream.WriteCF, 20, + ), + }, + testParams: []*testParam2{ + { + until: []uint64{1, 2, 3, 6}, + shiftUntilTS: returnSelf(), restMetadata: []*backuppb.Metadata{ + m_2(1, + 2, 6, stream.DefaultCF, 0, + 8, 10, stream.DefaultCF, 0, + ), + m_2(2, + 14, 18, stream.WriteCF, 9, + 16, 22, stream.DefaultCF, 0, + ), + m_1(3, + 24, 26, stream.WriteCF, 20, + ), + }, + }, { + until: []uint64{7, 8}, + shiftUntilTS: returnSelf(), restMetadata: []*backuppb.Metadata{ + m_1(1, + 8, 10, stream.DefaultCF, 0, + ), + m_2(2, + 14, 18, stream.WriteCF, 9, + 16, 22, stream.DefaultCF, 0, + ), + m_1(3, + 24, 26, stream.WriteCF, 20, + ), + }, + }, { + until: []uint64{9, 10, 11, 14, 15, 16, 17, 18}, + shiftUntilTS: returnV(9), restMetadata: []*backuppb.Metadata{ + m_1(1, + 8, 10, stream.DefaultCF, 0, + ), + m_2(2, + 14, 18, stream.WriteCF, 9, + 16, 22, stream.DefaultCF, 0, + ), + m_1(3, + 24, 26, stream.WriteCF, 20, + ), + }, + }, { + until: []uint64{19}, + shiftUntilTS: returnV(19), restMetadata: []*backuppb.Metadata{ + m_1(2, + 16, 22, stream.DefaultCF, 0, + ), + m_1(3, + 24, 26, stream.WriteCF, 20, + ), + }, + }, { + until: []uint64{20, 21, 22, 23, 24, 25, 26}, + shiftUntilTS: returnV(20), restMetadata: []*backuppb.Metadata{ + m_1(2, + 16, 22, stream.DefaultCF, 0, + ), + m_1(3, + 24, 26, stream.WriteCF, 20, + ), + }, + }, { + until: []uint64{28}, + shiftUntilTS: returnV(28), restMetadata: []*backuppb.Metadata{}, + }, + }, + }, + } + + for i, cs := range cases { + for j, ts := range cs.testParams { + for _, until := range ts.until { + t.Logf("case %d, param %d, until %d", i, j, until) + metas := restore.StreamMetadataSet{ + Helper: stream.NewMetadataHelper(), + } + err := generateFiles(ctx, s, cs.metas, tmpDir) + require.NoError(t, err) + shiftUntilTS, err := metas.LoadUntilAndCalculateShiftTS(ctx, s, until) + require.NoError(t, err) + require.Equal(t, shiftUntilTS, ts.shiftUntilTS(until)) + n, err := metas.RemoveDataFilesAndUpdateMetadataInBatch(ctx, shiftUntilTS, s, func(num int64) {}) + require.Equal(t, len(n), 0) + require.NoError(t, err) + + // check the result + checkFiles(ctx, s, ts.restMetadata, t) + } + } + } +} + +type testParam3 struct { + until []uint64 + shiftUntilTS func(uint64) uint64 +} + +func fi(minTS, maxTS uint64, cf string, defaultTS uint64) *backuppb.DataFileInfo { + return &backuppb.DataFileInfo{ + NumberOfEntries: 1, + MinTs: minTS, + MaxTs: maxTS, + Cf: cf, + MinBeginTsInDefaultCf: defaultTS, + } +} + +func getTsFromFiles(files []*backuppb.DataFileInfo) (uint64, uint64, uint64) { + if len(files) == 0 { + return 0, 0, 0 + } + f := files[0] + minTs, maxTs, resolvedTs := f.MinTs, f.MaxTs, f.ResolvedTs + for _, file := range files { + if file.MinTs < minTs { + minTs = file.MinTs + } + if file.MaxTs > maxTs { + maxTs = file.MaxTs + } + if file.ResolvedTs < resolvedTs { + resolvedTs = file.ResolvedTs + } + } + return minTs, maxTs, resolvedTs +} + +func mf(id int64, filess [][]*backuppb.DataFileInfo) *backuppb.Metadata { + filegroups := make([]*backuppb.DataFileGroup, 0) + for _, files := range filess { + minTs, maxTs, resolvedTs := getTsFromFiles(files) + filegroups = append(filegroups, &backuppb.DataFileGroup{ + DataFilesInfo: files, + MinTs: minTs, + MaxTs: maxTs, + MinResolvedTs: resolvedTs, + }) + } + + m := &backuppb.Metadata{ + StoreId: id, + MetaVersion: backuppb.MetaVersion_V2, + } + restore.ReplaceMetadata(m, filegroups) + return m +} + func TestCalculateShiftTS(t *testing.T) { - var ( - startTs uint64 = 2900 - restoreTS uint64 = 4500 - ) - - helper := stream.NewMetadataHelper() - ms := fakeMetaDatas(t, helper, stream.WriteCF) - shiftTS, exist := restore.CalculateShiftTS(ms, startTs, restoreTS) - require.Equal(t, shiftTS, uint64(2000)) - require.Equal(t, exist, true) - - shiftTS, exist = restore.CalculateShiftTS(ms, startTs, mathutil.MaxUint) - require.Equal(t, shiftTS, uint64(1800)) - require.Equal(t, exist, true) - - shiftTS, exist = restore.CalculateShiftTS(ms, 1999, 3001) - require.Equal(t, shiftTS, uint64(800)) - require.Equal(t, exist, true) - - ms = fakeMetaDatas(t, helper, stream.DefaultCF) - _, exist = restore.CalculateShiftTS(ms, startTs, restoreTS) - require.Equal(t, exist, false) -} - -func TestCalculateShiftTSV2(t *testing.T) { - var ( - startTs uint64 = 2900 - restoreTS uint64 = 5100 - ) - - helper := stream.NewMetadataHelper() - ms := fakeMetaDataV2s(t, helper, stream.WriteCF) - shiftTS, exist := restore.CalculateShiftTS(ms, startTs, restoreTS) - require.Equal(t, shiftTS, uint64(1800)) - require.Equal(t, exist, true) - - shiftTS, exist = restore.CalculateShiftTS(ms, startTs, mathutil.MaxUint) - require.Equal(t, shiftTS, uint64(1700)) - require.Equal(t, exist, true) - - shiftTS, exist = restore.CalculateShiftTS(ms, 1999, 3001) - require.Equal(t, shiftTS, uint64(800)) - require.Equal(t, exist, true) - - ms = fakeMetaDataV2s(t, helper, stream.DefaultCF) - _, exist = restore.CalculateShiftTS(ms, startTs, restoreTS) - require.Equal(t, exist, false) + ctx := context.Background() + tmpDir := t.TempDir() + s, err := storage.NewLocalStorage(tmpDir) + require.NoError(t, err) + + cases := []struct { + metas []*backuppb.Metadata + testParams []*testParam3 + }{ + { + // filegroup 10 35 + // ↑ ↑ + // +----+-++---+ + // ↓ ↓ ↓↓ ↓ + // fileinfo 10-d-20 + // fileinfo 8--d-15--w-30 + // fileinfo 11-d-25-w-35 + metas: []*backuppb.Metadata{ + mf(1, [][]*backuppb.DataFileInfo{ + { + fi(10, 20, stream.DefaultCF, 0), + fi(15, 30, stream.WriteCF, 8), + fi(25, 35, stream.WriteCF, 11), + }, + }), + }, + testParams: []*testParam3{ + { + until: []uint64{3}, + shiftUntilTS: returnV(3), + }, { + until: []uint64{8, 9, 10, 11, 12, 15, 16, 20, 21, 25, 26, 30}, + shiftUntilTS: returnV(8), + }, { + until: []uint64{31, 35}, + shiftUntilTS: returnV(11), + }, { + until: []uint64{36}, + shiftUntilTS: returnV(36), + }, + }, + }, { + // filegroup 50 85 + // ↑ ↑ + // +-+-+--+--+------+ + // ↓ ↓ ↓ ↓ ↓ ↓ + // fileinfo 55-d-65-70 + // fileinfo 50-d60 + // fileinfo 72d80w85 + metas: []*backuppb.Metadata{ + mf(1, [][]*backuppb.DataFileInfo{ + { + fi(65, 70, stream.WriteCF, 55), + fi(50, 60, stream.DefaultCF, 0), + fi(80, 85, stream.WriteCF, 72), + }, + }), + }, + testParams: []*testParam3{ + { + until: []uint64{45, 50, 52}, + shiftUntilTS: returnSelf(), + }, { + until: []uint64{55, 56, 60, 61, 65, 66, 70}, + shiftUntilTS: returnV(55), + }, { + until: []uint64{71}, + shiftUntilTS: returnV(71), + }, { + until: []uint64{72, 73, 80, 81, 85}, + shiftUntilTS: returnV(72), + }, { + until: []uint64{86}, + shiftUntilTS: returnV(86), + }, + }, + }, { + // filegroup 10 35 50 85 + // ↑ ↑ ↑ ↑ + // +----+-++---+ +-+-+--+--+------+ + // ↓ ↓ ↓↓ ↓ ↓ ↓ ↓ ↓ ↓ ↓ + // fileinfo 10-d-20 55-d-65-70 + // fileinfo 8--d-15--w-30 50-d60 + // fileinfo 11-d-25-w-35 72d80w85 + metas: []*backuppb.Metadata{ + mf(1, [][]*backuppb.DataFileInfo{ + { + fi(10, 20, stream.DefaultCF, 0), + fi(15, 30, stream.WriteCF, 8), + fi(25, 35, stream.WriteCF, 11), + }, + }), + mf(2, [][]*backuppb.DataFileInfo{ + { + fi(65, 70, stream.WriteCF, 55), + fi(50, 60, stream.DefaultCF, 0), + fi(80, 85, stream.WriteCF, 72), + }, + }), + }, + testParams: []*testParam3{ + { + until: []uint64{3}, + shiftUntilTS: returnV(3), + }, { + until: []uint64{8, 9, 10, 11, 12, 15, 16, 20, 21, 25, 26, 30}, + shiftUntilTS: returnV(8), + }, { + until: []uint64{31, 35}, + shiftUntilTS: returnV(11), + }, { + until: []uint64{36}, + shiftUntilTS: returnV(36), + }, { + until: []uint64{45, 50, 52}, + shiftUntilTS: returnSelf(), + }, { + until: []uint64{55, 56, 60, 61, 65, 66, 70}, + shiftUntilTS: returnV(55), + }, { + until: []uint64{71}, + shiftUntilTS: returnV(71), + }, { + until: []uint64{72, 73, 80, 81, 85}, + shiftUntilTS: returnV(72), + }, { + until: []uint64{86}, + shiftUntilTS: returnV(86), + }, + }, + }, + } + + for i, cs := range cases { + for j, ts := range cs.testParams { + for _, until := range ts.until { + t.Logf("case %d, param %d, until %d", i, j, until) + metas := restore.StreamMetadataSet{ + Helper: stream.NewMetadataHelper(), + } + err := generateFiles(ctx, s, cs.metas, tmpDir) + require.NoError(t, err) + shiftUntilTS, err := metas.LoadUntilAndCalculateShiftTS(ctx, s, until) + require.NoError(t, err) + require.Equal(t, shiftUntilTS, ts.shiftUntilTS(until), cs.metas) + } + } + } } diff --git a/br/pkg/task/stream.go b/br/pkg/task/stream.go index ff4a1c449e181..a259452b14b2d 100644 --- a/br/pkg/task/stream.go +++ b/br/pkg/task/stream.go @@ -57,7 +57,6 @@ import ( "go.uber.org/zap" "go.uber.org/zap/zapcore" "golang.org/x/exp/slices" - "golang.org/x/sync/errgroup" ) const ( @@ -68,8 +67,6 @@ const ( flagStreamStartTS = "start-ts" flagStreamEndTS = "end-ts" flagGCSafePointTTS = "gc-ttl" - - notDeletedBecameFatalThreshold = 128 ) var ( @@ -937,31 +934,24 @@ func RunStreamTruncate(c context.Context, g glue.Glue, cmdName string, cfg *Stre readMetaDone := console.ShowTask("Reading Metadata... ", glue.WithTimeCost()) metas := restore.StreamMetadataSet{ Helper: stream.NewMetadataHelper(), - BeforeDoWriteBack: func(path string, last, current *backuppb.Metadata) (skip bool) { - log.Info("Updating metadata.", zap.String("file", path), - zap.Int("data-file-before", len(last.GetFileGroups())), - zap.Int("data-file-after", len(current.GetFileGroups()))) - return cfg.DryRun - }, + DryRun: cfg.DryRun, } - if err := metas.LoadUntil(ctx, storage, cfg.Until); err != nil { + shiftUntilTS, err := metas.LoadUntilAndCalculateShiftTS(ctx, storage, cfg.Until) + if err != nil { return err } readMetaDone() var ( - fileCount uint64 = 0 - kvCount int64 = 0 - totalSize uint64 = 0 - shiftUntilTS = metas.CalculateShiftTS(cfg.Until) + fileCount int = 0 + kvCount int64 = 0 + totalSize uint64 = 0 ) - metas.IterateFilesFullyBefore(shiftUntilTS, func(d *backuppb.DataFileGroup) (shouldBreak bool) { + metas.IterateFilesFullyBefore(shiftUntilTS, func(d *restore.FileGroupInfo) (shouldBreak bool) { fileCount++ totalSize += d.Length - for _, f := range d.DataFilesInfo { - kvCount += f.NumberOfEntries - } + kvCount += d.KVCount return }) console.Printf("We are going to remove %s files, until %s.\n", @@ -979,83 +969,39 @@ func RunStreamTruncate(c context.Context, g glue.Glue, cmdName string, cfg *Stre } } - removed := metas.RemoveDataBefore(shiftUntilTS) - - // remove log + // begin to remove p := console.StartProgressBar( - "Clearing Data Files", len(removed), + "Clearing Data Files and Metadata", fileCount, glue.WithTimeCost(), glue.WithConstExtraField("kv-count", kvCount), glue.WithConstExtraField("kv-size", fmt.Sprintf("%d(%s)", totalSize, units.HumanSize(float64(totalSize)))), ) defer p.Close() - worker := utils.NewWorkerPool(128, "delete files") - eg, cx := errgroup.WithContext(ctx) - const keepFirstNFailure = 16 - var notDeleted struct { - item []string - sync.Mutex - } - for _, f := range removed { - f := f - worker.ApplyOnErrorGroup(eg, func() error { - if cx.Err() != nil { - p.Close() - return cx.Err() - } - defer p.Inc() - log.Debug("Deleting file", zap.String("path", f.Path)) - if cfg.DryRun { - return nil - } - if err := storage.DeleteFile(ctx, f.Path); err != nil { - log.Warn("File not deleted.", zap.String("path", f.Path), logutil.ShortError(err)) - notDeleted.Lock() - defer notDeleted.Unlock() - notDeleted.item = append(notDeleted.item, f.Path) - if len(notDeleted.item) > notDeletedBecameFatalThreshold { - return errors.Annotatef(berrors.ErrPiTRMalformedMetadata, "too many failure when truncating") - } - } - return nil - }) - } - if err := p.Wait(ctx); err != nil { + + notDeleted, err := metas.RemoveDataFilesAndUpdateMetadataInBatch(ctx, shiftUntilTS, storage, p.IncBy) + if err != nil { return err } - if err := eg.Wait(); err != nil { + + if err := p.Wait(ctx); err != nil { return err } - notDeleted.Lock() - if len(notDeleted.item) > 0 { + if len(notDeleted) > 0 { + const keepFirstNFailure = 16 console.Println("Files below are not deleted due to error, you may clear it manually, check log for detail error:") - console.Println("- Total", em(len(notDeleted.item)), "items.") - if len(notDeleted.item) > keepFirstNFailure { - console.Println("-", em(len(notDeleted.item)-keepFirstNFailure), "items omitted.") + console.Println("- Total", em(len(notDeleted)), "items.") + if len(notDeleted) > keepFirstNFailure { + console.Println("-", em(len(notDeleted)-keepFirstNFailure), "items omitted.") // TODO: maybe don't add them at the very first. - notDeleted.item = notDeleted.item[:keepFirstNFailure] + notDeleted = notDeleted[:keepFirstNFailure] } - for _, f := range notDeleted.item { + for _, f := range notDeleted { console.Println(f) } } - notDeleted.Unlock() - // remove metadata - pw := console.StartProgressBar("Removing Metadata", metas.PendingMeta(), glue.WithTimeCost(), glue.WithConstExtraField("metas", metas.PendingMeta())) - defer pw.Close() - metas.BeforeDoWriteBack = func(path string, last, current *backuppb.Metadata) (skip bool) { - log.Info("Updating metadata.", zap.String("file", path), - zap.Int("data-file-before", len(last.GetFiles())), - zap.Int("data-file-after", len(current.GetFiles()))) - pw.Inc() - return cfg.DryRun - } - if err := metas.DoWriteBack(ctx, storage); err != nil { - return err - } - return pw.Wait(ctx) + return nil } // RunStreamRestore restores stream log. diff --git a/br/pkg/utils/db.go b/br/pkg/utils/db.go index be2bd87a6ccb8..23911fa6f0a93 100644 --- a/br/pkg/utils/db.go +++ b/br/pkg/utils/db.go @@ -71,7 +71,7 @@ func CheckLogBackupEnabled(ctx sessionctx.Context) bool { // we use `sqlexec.RestrictedSQLExecutor` as parameter because it's easy to mock. // it should return error. func IsLogBackupEnabled(ctx sqlexec.RestrictedSQLExecutor) (bool, error) { - valStr := "show config where name = 'log-backup.enable'" + valStr := "show config where name = 'log-backup.enable' and type = 'tikv'" internalCtx := kv.WithInternalSourceType(context.Background(), kv.InternalTxnBR) rows, fields, errSQL := ctx.ExecRestrictedSQL(internalCtx, nil, valStr) if errSQL != nil { diff --git a/ddl/BUILD.bazel b/ddl/BUILD.bazel index 1e6357664afeb..b54f52a6d192c 100644 --- a/ddl/BUILD.bazel +++ b/ddl/BUILD.bazel @@ -137,9 +137,10 @@ go_library( go_test( name = "ddl_test", - timeout = "long", + timeout = "moderate", srcs = [ "attributes_sql_test.go", + "backfilling_test.go", "callback_test.go", "cancel_test.go", "cluster_test.go", @@ -256,6 +257,7 @@ go_test( "@com_github_tikv_client_go_v2//oracle", "@com_github_tikv_client_go_v2//testutils", "@com_github_tikv_client_go_v2//tikv", + "@com_github_tikv_client_go_v2//util", "@io_etcd_go_etcd_client_v3//:client", "@org_golang_google_grpc//:grpc", "@org_golang_x_exp//slices", diff --git a/ddl/attributes_sql_test.go b/ddl/attributes_sql_test.go index de00a63a7a661..95f881e6fb3fe 100644 --- a/ddl/attributes_sql_test.go +++ b/ddl/attributes_sql_test.go @@ -28,6 +28,7 @@ import ( "github.com/pingcap/tidb/testkit" "github.com/pingcap/tidb/util/gcutil" "github.com/stretchr/testify/require" + tikvutil "github.com/tikv/client-go/v2/util" ) // MockGC is used to make GC work in the test environment. @@ -44,9 +45,8 @@ func MockGC(tk *testkit.TestKit) (string, string, string, func()) { // disable emulator GC. // Otherwise emulator GC will delete table record as soon as possible after execute drop table ddl. util.EmulatorGCDisable() - gcTimeFormat := "20060102-15:04:05 -0700 MST" - timeBeforeDrop := time.Now().Add(0 - 48*60*60*time.Second).Format(gcTimeFormat) - timeAfterDrop := time.Now().Add(48 * 60 * 60 * time.Second).Format(gcTimeFormat) + timeBeforeDrop := time.Now().Add(0 - 48*60*60*time.Second).Format(tikvutil.GCTimeFormat) + timeAfterDrop := time.Now().Add(48 * 60 * 60 * time.Second).Format(tikvutil.GCTimeFormat) safePointSQL := `INSERT HIGH_PRIORITY INTO mysql.tidb VALUES ('tikv_gc_safe_point', '%[1]s', '') ON DUPLICATE KEY UPDATE variable_value = '%[1]s'` diff --git a/ddl/backfilling.go b/ddl/backfilling.go index 7761003d78e23..dfd213652ab2e 100644 --- a/ddl/backfilling.go +++ b/ddl/backfilling.go @@ -42,6 +42,7 @@ import ( "github.com/pingcap/tidb/util" "github.com/pingcap/tidb/util/dbterror" "github.com/pingcap/tidb/util/logutil" + "github.com/pingcap/tidb/util/mathutil" decoder "github.com/pingcap/tidb/util/rowDecoder" "github.com/pingcap/tidb/util/timeutil" "github.com/pingcap/tidb/util/topsql" @@ -131,6 +132,7 @@ type backfiller interface { } type backfillResult struct { + taskID int addedCount int scanCount int nextKey kv.Key @@ -149,6 +151,7 @@ type backfillTaskContext struct { } type reorgBackfillTask struct { + id int physicalTableID int64 startKey kv.Key endKey kv.Key @@ -181,30 +184,32 @@ type backfillWorker struct { taskCh chan *reorgBackfillTask resultCh chan *backfillResult table table.Table - closed bool priority int tp backfillWorkerType + ctx context.Context + cancel func() } -func newBackfillWorker(sessCtx sessionctx.Context, id int, t table.PhysicalTable, +func newBackfillWorker(ctx context.Context, sessCtx sessionctx.Context, id int, t table.PhysicalTable, reorgInfo *reorgInfo, tp backfillWorkerType) *backfillWorker { + bfCtx, cancel := context.WithCancel(ctx) return &backfillWorker{ id: id, table: t, reorgInfo: reorgInfo, batchCnt: int(variable.GetDDLReorgBatchSize()), sessCtx: sessCtx, - taskCh: make(chan *reorgBackfillTask, 1), - resultCh: make(chan *backfillResult, 1), priority: reorgInfo.Job.Priority, tp: tp, + ctx: bfCtx, + cancel: cancel, } } func (w *backfillWorker) Close() { - if !w.closed { - w.closed = true - close(w.taskCh) + if w.cancel != nil { + w.cancel() + w.cancel = nil } } @@ -214,10 +219,14 @@ func closeBackfillWorkers(workers []*backfillWorker) { } } +// ResultCounterForTest is used for test. +var ResultCounterForTest *atomic.Int32 + // handleBackfillTask backfills range [task.startHandle, task.endHandle) handle's index to table. func (w *backfillWorker) handleBackfillTask(d *ddlCtx, task *reorgBackfillTask, bf backfiller) *backfillResult { handleRange := *task result := &backfillResult{ + taskID: task.id, err: nil, addedCount: 0, nextKey: handleRange.startKey, @@ -282,6 +291,9 @@ func (w *backfillWorker) handleBackfillTask(d *ddlCtx, task *reorgBackfillTask, zap.Int("scanCount", result.scanCount), zap.String("nextHandle", tryDecodeToHandleString(result.nextKey)), zap.String("takeTime", time.Since(startTime).String())) + if ResultCounterForTest != nil && result.err == nil { + ResultCounterForTest.Add(1) + } return result } @@ -289,21 +301,29 @@ func (w *backfillWorker) run(d *ddlCtx, bf backfiller, job *model.Job) { logutil.BgLogger().Info("[ddl] backfill worker start", zap.Stringer("type", w.tp), zap.Int("workerID", w.id)) - defer func() { - w.resultCh <- &backfillResult{err: dbterror.ErrReorgPanic} - }() - defer util.Recover(metrics.LabelDDL, "backfillWorker.run", nil, false) + var curTaskID int + defer util.Recover(metrics.LabelDDL, "backfillWorker.run", func() { + w.resultCh <- &backfillResult{taskID: curTaskID, err: dbterror.ErrReorgPanic} + }, false) for { + if util.HasCancelled(w.ctx) { + logutil.BgLogger().Info("[ddl] backfill worker exit on context done", + zap.Stringer("type", w.tp), zap.Int("workerID", w.id)) + return + } task, more := <-w.taskCh if !more { - break + logutil.BgLogger().Info("[ddl] backfill worker exit", + zap.Stringer("type", w.tp), zap.Int("workerID", w.id)) + return } + curTaskID = task.id d.setDDLLabelForTopSQL(job) logutil.BgLogger().Debug("[ddl] backfill worker got task", zap.Int("workerID", w.id), zap.String("task", task.String())) failpoint.Inject("mockBackfillRunErr", func() { if w.id == 0 { - result := &backfillResult{addedCount: 0, nextKey: nil, err: errors.Errorf("mock backfill error")} + result := &backfillResult{taskID: task.id, addedCount: 0, nextKey: nil, err: errors.Errorf("mock backfill error")} w.resultCh <- result failpoint.Continue() } @@ -318,14 +338,11 @@ func (w *backfillWorker) run(d *ddlCtx, bf backfiller, job *model.Job) { time.Sleep(100 * time.Millisecond) }) - // Dynamic change batch size. + // Change the batch size dynamically. w.batchCnt = int(variable.GetDDLReorgBatchSize()) result := w.handleBackfillTask(d, task, bf) w.resultCh <- result } - logutil.BgLogger().Info("[ddl] backfill worker exit", - zap.Stringer("type", w.tp), - zap.Int("workerID", w.id)) } // splitTableRanges uses PD region's key ranges to split the backfilling table key range space, @@ -357,48 +374,66 @@ func splitTableRanges(t table.PhysicalTable, store kv.Storage, startKey, endKey return ranges, nil } -func waitTaskResults(workers []*backfillWorker, taskCnt int, - totalAddedCount *int64, startKey kv.Key) (kv.Key, int64, error) { +func waitTaskResults(scheduler *backfillScheduler, batchTasks []*reorgBackfillTask, + totalAddedCount *int64) (kv.Key, int64, error) { var ( - addedCount int64 - nextKey = startKey firstErr error + addedCount int64 ) - for i := 0; i < taskCnt; i++ { - worker := workers[i] - result := <-worker.resultCh - if firstErr == nil && result.err != nil { - firstErr = result.err - // We should wait all working workers exits, any way. - continue - } - + keeper := newDoneTaskKeeper(batchTasks[0].startKey) + taskSize := len(batchTasks) + for i := 0; i < taskSize; i++ { + result := <-scheduler.resultCh if result.err != nil { - logutil.BgLogger().Warn("[ddl] backfill worker failed", zap.Int("workerID", worker.id), + if firstErr == nil { + firstErr = result.err + } + logutil.BgLogger().Warn("[ddl] backfill worker failed", + zap.String("result next key", hex.EncodeToString(result.nextKey)), zap.Error(result.err)) + // Drain tasks. + cnt := drainTasks(scheduler.taskCh) + // We need to wait all the tasks to finish before closing it + // to prevent send on closed channel error. + taskSize -= cnt + continue } - - if firstErr == nil { - *totalAddedCount += int64(result.addedCount) - addedCount += int64(result.addedCount) - nextKey = result.nextKey + *totalAddedCount += int64(result.addedCount) + addedCount += int64(result.addedCount) + keeper.updateNextKey(result.taskID, result.nextKey) + if i%scheduler.workerSize()*4 == 0 { + // We try to adjust the worker size regularly to reduce + // the overhead of loading the DDL related global variables. + err := scheduler.adjustWorkerSize() + if err != nil { + logutil.BgLogger().Warn("[ddl] cannot adjust backfill worker size", zap.Error(err)) + } } } + return keeper.nextKey, addedCount, errors.Trace(firstErr) +} - return nextKey, addedCount, errors.Trace(firstErr) +func drainTasks(taskCh chan *reorgBackfillTask) int { + cnt := 0 + for len(taskCh) > 0 { + <-taskCh + cnt++ + } + return cnt } // sendTasksAndWait sends tasks to workers, and waits for all the running workers to return results, // there are taskCnt running workers. -func (dc *ddlCtx) sendTasksAndWait(sessPool *sessionPool, reorgInfo *reorgInfo, totalAddedCount *int64, workers []*backfillWorker, batchTasks []*reorgBackfillTask) error { - for i, task := range batchTasks { - workers[i].taskCh <- task +func (dc *ddlCtx) sendTasksAndWait(scheduler *backfillScheduler, totalAddedCount *int64, + batchTasks []*reorgBackfillTask) error { + reorgInfo := scheduler.reorgInfo + for _, task := range batchTasks { + scheduler.taskCh <- task } startKey := batchTasks[0].startKey - taskCnt := len(batchTasks) startTime := time.Now() - nextKey, taskAddedCount, err := waitTaskResults(workers, taskCnt, totalAddedCount, startKey) + nextKey, taskAddedCount, err := waitTaskResults(scheduler, batchTasks, totalAddedCount) elapsedTime := time.Since(startTime) if err == nil { err = dc.isReorgRunnable(reorgInfo.Job) @@ -406,10 +441,9 @@ func (dc *ddlCtx) sendTasksAndWait(sessPool *sessionPool, reorgInfo *reorgInfo, if err != nil { // Update the reorg handle that has been processed. - err1 := reorgInfo.UpdateReorgMeta(nextKey, sessPool) + err1 := reorgInfo.UpdateReorgMeta(nextKey, scheduler.sessPool) metrics.BatchAddIdxHistogram.WithLabelValues(metrics.LblError).Observe(elapsedTime.Seconds()) logutil.BgLogger().Warn("[ddl] backfill worker handle batch tasks failed", - zap.Stringer("type", workers[0].tp), zap.ByteString("elementType", reorgInfo.currElement.TypeKey), zap.Int64("elementID", reorgInfo.currElement.ID), zap.Int64("totalAddedCount", *totalAddedCount), @@ -426,7 +460,6 @@ func (dc *ddlCtx) sendTasksAndWait(sessPool *sessionPool, reorgInfo *reorgInfo, dc.getReorgCtx(reorgInfo.Job).setNextKey(nextKey) metrics.BatchAddIdxHistogram.WithLabelValues(metrics.LblOK).Observe(elapsedTime.Seconds()) logutil.BgLogger().Info("[ddl] backfill workers successfully processed batch", - zap.Stringer("type", workers[0].tp), zap.ByteString("elementType", reorgInfo.currElement.TypeKey), zap.Int64("elementID", reorgInfo.currElement.ID), zap.Int64("totalAddedCount", *totalAddedCount), @@ -465,11 +498,11 @@ func tryDecodeToHandleString(key kv.Key) string { } // handleRangeTasks sends tasks to workers, and returns remaining kvRanges that is not handled. -func (dc *ddlCtx) handleRangeTasks(sessPool *sessionPool, t table.Table, workers []*backfillWorker, reorgInfo *reorgInfo, +func (dc *ddlCtx) handleRangeTasks(scheduler *backfillScheduler, t table.Table, totalAddedCount *int64, kvRanges []kv.KeyRange) ([]kv.KeyRange, error) { - batchTasks := make([]*reorgBackfillTask, 0, len(workers)) + batchTasks := make([]*reorgBackfillTask, 0, backfillTaskChanSize) + reorgInfo := scheduler.reorgInfo physicalTableID := reorgInfo.PhysicalTableID - var prefix kv.Key if tbl, ok := t.(table.PartitionedTable); ok { t = tbl.GetPartition(physicalTableID) @@ -480,9 +513,10 @@ func (dc *ddlCtx) handleRangeTasks(sessPool *sessionPool, t table.Table, workers prefix = t.RecordPrefix() } // Build reorg tasks. + job := reorgInfo.Job for i, keyRange := range kvRanges { endKey := keyRange.EndKey - endK, err := getRangeEndKey(reorgInfo.d.jobContext(reorgInfo.Job), workers[0].sessCtx.GetStore(), workers[0].priority, prefix, keyRange.StartKey, endKey) + endK, err := getRangeEndKey(scheduler.jobCtx, dc.store, job.Priority, prefix, keyRange.StartKey, endKey) if err != nil { logutil.BgLogger().Info("[ddl] send range task to workers, get reverse key failed", zap.Error(err)) } else { @@ -492,6 +526,7 @@ func (dc *ddlCtx) handleRangeTasks(sessPool *sessionPool, t table.Table, workers } task := &reorgBackfillTask{ + id: i, physicalTableID: physicalTableID, startKey: keyRange.StartKey, endKey: endKey, @@ -499,7 +534,7 @@ func (dc *ddlCtx) handleRangeTasks(sessPool *sessionPool, t table.Table, workers endInclude: endK.Cmp(keyRange.EndKey) != 0 || i == len(kvRanges)-1} batchTasks = append(batchTasks, task) - if len(batchTasks) >= len(workers) { + if len(batchTasks) >= backfillTaskChanSize { break } } @@ -509,7 +544,7 @@ func (dc *ddlCtx) handleRangeTasks(sessPool *sessionPool, t table.Table, workers } // Wait tasks finish. - err := dc.sendTasksAndWait(sessPool, reorgInfo, totalAddedCount, workers, batchTasks) + err := dc.sendTasksAndWait(scheduler, totalAddedCount, batchTasks) if err != nil { return nil, errors.Trace(err) } @@ -527,7 +562,7 @@ var ( // TestCheckWorkerNumCh use for test adjust backfill worker. TestCheckWorkerNumCh = make(chan *sync.WaitGroup) // TestCheckWorkerNumber use for test adjust backfill worker. - TestCheckWorkerNumber = int32(16) + TestCheckWorkerNumber = int32(1) // TestCheckReorgTimeout is used to mock timeout when reorg data. TestCheckReorgTimeout = int32(0) ) @@ -572,6 +607,132 @@ func setSessCtxLocation(sctx sessionctx.Context, info *reorgInfo) error { return nil } +type backfillScheduler struct { + ctx context.Context + reorgInfo *reorgInfo + sessPool *sessionPool + tp backfillWorkerType + tbl table.PhysicalTable + decodeColMap map[int64]decoder.Column + jobCtx *JobContext + + workers []*backfillWorker + maxSize int + + taskCh chan *reorgBackfillTask + resultCh chan *backfillResult +} + +const backfillTaskChanSize = 1024 + +func newBackfillScheduler(ctx context.Context, info *reorgInfo, sessPool *sessionPool, + tp backfillWorkerType, tbl table.PhysicalTable, decColMap map[int64]decoder.Column, + jobCtx *JobContext) *backfillScheduler { + return &backfillScheduler{ + ctx: ctx, + reorgInfo: info, + sessPool: sessPool, + tp: tp, + tbl: tbl, + decodeColMap: decColMap, + jobCtx: jobCtx, + workers: make([]*backfillWorker, 0, variable.GetDDLReorgWorkerCounter()), + taskCh: make(chan *reorgBackfillTask, backfillTaskChanSize), + resultCh: make(chan *backfillResult, backfillTaskChanSize), + } +} + +func (b *backfillScheduler) newSessCtx() (sessionctx.Context, error) { + reorgInfo := b.reorgInfo + sessCtx := newContext(reorgInfo.d.store) + sessCtx.GetSessionVars().StmtCtx.IsDDLJobInQueue = true + // Set the row encode format version. + rowFormat := variable.GetDDLReorgRowFormat() + sessCtx.GetSessionVars().RowEncoder.Enable = rowFormat != variable.DefTiDBRowFormatV1 + // Simulate the sql mode environment in the worker sessionCtx. + sqlMode := reorgInfo.ReorgMeta.SQLMode + sessCtx.GetSessionVars().SQLMode = sqlMode + if err := setSessCtxLocation(sessCtx, reorgInfo); err != nil { + return nil, errors.Trace(err) + } + sessCtx.GetSessionVars().StmtCtx.BadNullAsWarning = !sqlMode.HasStrictMode() + sessCtx.GetSessionVars().StmtCtx.TruncateAsWarning = !sqlMode.HasStrictMode() + sessCtx.GetSessionVars().StmtCtx.OverflowAsWarning = !sqlMode.HasStrictMode() + sessCtx.GetSessionVars().StmtCtx.AllowInvalidDate = sqlMode.HasAllowInvalidDatesMode() + sessCtx.GetSessionVars().StmtCtx.DividedByZeroAsWarning = !sqlMode.HasStrictMode() + sessCtx.GetSessionVars().StmtCtx.IgnoreZeroInDate = !sqlMode.HasStrictMode() || sqlMode.HasAllowInvalidDatesMode() + sessCtx.GetSessionVars().StmtCtx.NoZeroDate = sqlMode.HasStrictMode() + return sessCtx, nil +} + +func (b *backfillScheduler) setMaxWorkerSize(maxSize int) { + b.maxSize = maxSize +} + +func (b *backfillScheduler) workerSize() int { + return len(b.workers) +} + +func (b *backfillScheduler) adjustWorkerSize() error { + reorgInfo := b.reorgInfo + job := reorgInfo.Job + jc := b.jobCtx + if err := loadDDLReorgVars(b.ctx, b.sessPool); err != nil { + logutil.BgLogger().Error("[ddl] load DDL reorganization variable failed", zap.Error(err)) + } + workerCnt := int(variable.GetDDLReorgWorkerCounter()) + workerCnt = mathutil.Min(workerCnt, b.maxSize) + // Increase the worker. + for i := len(b.workers); i < workerCnt; i++ { + sessCtx, err := b.newSessCtx() + if err != nil { + return err + } + var ( + runner *backfillWorker + worker backfiller + ) + switch b.tp { + case typeAddIndexWorker: + idxWorker, err := newAddIndexWorker(sessCtx, i, b.tbl, b.decodeColMap, reorgInfo, jc, job) + if err != nil { + return errors.Trace(err) + } + worker, runner = idxWorker, idxWorker.backfillWorker + case typeAddIndexMergeTmpWorker: + tmpIdxWorker := newMergeTempIndexWorker(sessCtx, i, b.tbl, reorgInfo, jc) + worker, runner = tmpIdxWorker, tmpIdxWorker.backfillWorker + case typeUpdateColumnWorker: + // Setting InCreateOrAlterStmt tells the difference between SELECT casting and ALTER COLUMN casting. + sessCtx.GetSessionVars().StmtCtx.InCreateOrAlterStmt = true + updateWorker := newUpdateColumnWorker(sessCtx, i, b.tbl, b.decodeColMap, reorgInfo, jc) + worker, runner = updateWorker, updateWorker.backfillWorker + case typeCleanUpIndexWorker: + idxWorker := newCleanUpIndexWorker(sessCtx, i, b.tbl, b.decodeColMap, reorgInfo, jc) + worker, runner = idxWorker, idxWorker.backfillWorker + default: + return errors.New("unknown backfill type") + } + runner.taskCh = b.taskCh + runner.resultCh = b.resultCh + b.workers = append(b.workers, runner) + go runner.run(reorgInfo.d, worker, job) + } + // Decrease the worker. + if len(b.workers) > workerCnt { + workers := b.workers[workerCnt:] + b.workers = b.workers[:workerCnt] + closeBackfillWorkers(workers) + } + return injectCheckBackfillWorkerNum(len(b.workers)) +} + +func (b *backfillScheduler) Close() { + closeBackfillWorkers(b.workers) + close(b.taskCh) + close(b.resultCh) +} + // writePhysicalTableRecord handles the "add index" or "modify/change column" reorganization state for a non-partitioned table or a partition. // For a partitioned table, it should be handled partition by partition. // @@ -612,107 +773,25 @@ func (dc *ddlCtx) writePhysicalTableRecord(sessPool *sessionPool, t table.Physic } }) - // variable.ddlReorgWorkerCounter can be modified by system variable "tidb_ddl_reorg_worker_cnt". - workerCnt := variable.GetDDLReorgWorkerCounter() - backfillWorkers := make([]*backfillWorker, 0, workerCnt) - defer func() { - closeBackfillWorkers(backfillWorkers) - }() jc := dc.jobContext(job) + scheduler := newBackfillScheduler(dc.ctx, reorgInfo, sessPool, bfWorkerType, t, decodeColMap, jc) + defer scheduler.Close() for { kvRanges, err := splitTableRanges(t, reorgInfo.d.store, startKey, endKey) if err != nil { return errors.Trace(err) } + scheduler.setMaxWorkerSize(len(kvRanges)) - // For dynamic adjust backfill worker number. - if err := loadDDLReorgVars(dc.ctx, sessPool); err != nil { - logutil.BgLogger().Error("[ddl] load DDL reorganization variable failed", zap.Error(err)) - } - workerCnt = variable.GetDDLReorgWorkerCounter() - rowFormat := variable.GetDDLReorgRowFormat() - // If only have 1 range, we can only start 1 worker. - if len(kvRanges) < int(workerCnt) { - workerCnt = int32(len(kvRanges)) - } - // Enlarge the worker size. - for i := len(backfillWorkers); i < int(workerCnt); i++ { - sessCtx := newContext(reorgInfo.d.store) - sessCtx.GetSessionVars().StmtCtx.IsDDLJobInQueue = true - // Set the row encode format version. - sessCtx.GetSessionVars().RowEncoder.Enable = rowFormat != variable.DefTiDBRowFormatV1 - // Simulate the sql mode environment in the worker sessionCtx. - sqlMode := reorgInfo.ReorgMeta.SQLMode - sessCtx.GetSessionVars().SQLMode = sqlMode - if err := setSessCtxLocation(sessCtx, reorgInfo); err != nil { - return errors.Trace(err) - } - - sessCtx.GetSessionVars().StmtCtx.BadNullAsWarning = !sqlMode.HasStrictMode() - sessCtx.GetSessionVars().StmtCtx.TruncateAsWarning = !sqlMode.HasStrictMode() - sessCtx.GetSessionVars().StmtCtx.OverflowAsWarning = !sqlMode.HasStrictMode() - sessCtx.GetSessionVars().StmtCtx.AllowInvalidDate = sqlMode.HasAllowInvalidDatesMode() - sessCtx.GetSessionVars().StmtCtx.DividedByZeroAsWarning = !sqlMode.HasStrictMode() - sessCtx.GetSessionVars().StmtCtx.IgnoreZeroInDate = !sqlMode.HasStrictMode() || sqlMode.HasAllowInvalidDatesMode() - sessCtx.GetSessionVars().StmtCtx.NoZeroDate = sqlMode.HasStrictMode() - - switch bfWorkerType { - case typeAddIndexWorker: - idxWorker, err := newAddIndexWorker(sessCtx, i, t, decodeColMap, reorgInfo, jc, job) - if err != nil { - return errors.Trace(err) - } - backfillWorkers = append(backfillWorkers, idxWorker.backfillWorker) - go idxWorker.backfillWorker.run(reorgInfo.d, idxWorker, job) - case typeAddIndexMergeTmpWorker: - tmpIdxWorker := newMergeTempIndexWorker(sessCtx, i, t, reorgInfo, jc) - backfillWorkers = append(backfillWorkers, tmpIdxWorker.backfillWorker) - go tmpIdxWorker.backfillWorker.run(reorgInfo.d, tmpIdxWorker, job) - case typeUpdateColumnWorker: - // Setting InCreateOrAlterStmt tells the difference between SELECT casting and ALTER COLUMN casting. - sessCtx.GetSessionVars().StmtCtx.InCreateOrAlterStmt = true - updateWorker := newUpdateColumnWorker(sessCtx, i, t, decodeColMap, reorgInfo, jc) - backfillWorkers = append(backfillWorkers, updateWorker.backfillWorker) - go updateWorker.backfillWorker.run(reorgInfo.d, updateWorker, job) - case typeCleanUpIndexWorker: - idxWorker := newCleanUpIndexWorker(sessCtx, i, t, decodeColMap, reorgInfo, jc) - backfillWorkers = append(backfillWorkers, idxWorker.backfillWorker) - go idxWorker.backfillWorker.run(reorgInfo.d, idxWorker, job) - default: - return errors.New("unknow backfill type") - } - } - // Shrink the worker size. - if len(backfillWorkers) > int(workerCnt) { - workers := backfillWorkers[workerCnt:] - backfillWorkers = backfillWorkers[:workerCnt] - closeBackfillWorkers(workers) + err = scheduler.adjustWorkerSize() + if err != nil { + return errors.Trace(err) } - failpoint.Inject("checkBackfillWorkerNum", func(val failpoint.Value) { - //nolint:forcetypeassert - if val.(bool) { - num := int(atomic.LoadInt32(&TestCheckWorkerNumber)) - if num != 0 { - if num > len(kvRanges) { - if len(backfillWorkers) != len(kvRanges) { - failpoint.Return(errors.Errorf("check backfill worker num error, len kv ranges is: %v, check backfill worker num is: %v, actual record num is: %v", len(kvRanges), num, len(backfillWorkers))) - } - } else if num != len(backfillWorkers) { - failpoint.Return(errors.Errorf("check backfill worker num error, len kv ranges is: %v, check backfill worker num is: %v, actual record num is: %v", len(kvRanges), num, len(backfillWorkers))) - } - var wg sync.WaitGroup - wg.Add(1) - TestCheckWorkerNumCh <- &wg - wg.Wait() - } - } - }) - logutil.BgLogger().Info("[ddl] start backfill workers to reorg record", zap.Stringer("type", bfWorkerType), - zap.Int("workerCnt", len(backfillWorkers)), + zap.Int("workerCnt", scheduler.workerSize()), zap.Int("regionCnt", len(kvRanges)), zap.String("startKey", hex.EncodeToString(startKey)), zap.String("endKey", hex.EncodeToString(endKey))) @@ -726,7 +805,7 @@ func (dc *ddlCtx) writePhysicalTableRecord(sessPool *sessionPool, t table.Physic return errors.New(ingest.LitErrGetBackendFail) } } - remains, err := dc.handleRangeTasks(sessPool, t, backfillWorkers, reorgInfo, &totalAddedCount, kvRanges) + remains, err := dc.handleRangeTasks(scheduler, t, &totalAddedCount, kvRanges) if err != nil { return errors.Trace(err) } @@ -739,6 +818,25 @@ func (dc *ddlCtx) writePhysicalTableRecord(sessPool *sessionPool, t table.Physic return nil } +func injectCheckBackfillWorkerNum(curWorkerSize int) error { + failpoint.Inject("checkBackfillWorkerNum", func(val failpoint.Value) { + //nolint:forcetypeassert + if val.(bool) { + num := int(atomic.LoadInt32(&TestCheckWorkerNumber)) + if num != 0 { + if num != curWorkerSize { + failpoint.Return(errors.Errorf("expected backfill worker num: %v, actual record num: %v", num, curWorkerSize)) + } + var wg sync.WaitGroup + wg.Add(1) + TestCheckWorkerNumCh <- &wg + wg.Wait() + } + } + }) + return nil +} + // recordIterFunc is used for low-level record iteration. type recordIterFunc func(h kv.Handle, rowKey kv.Key, rawRecord []byte) (more bool, err error) @@ -850,3 +948,36 @@ func logSlowOperations(elapsed time.Duration, slowMsg string, threshold uint32) logutil.BgLogger().Info("[ddl] slow operations", zap.Duration("takeTimes", elapsed), zap.String("msg", slowMsg)) } } + +// doneTaskKeeper keeps the done tasks and update the latest next key. +type doneTaskKeeper struct { + doneTaskNextKey map[int]kv.Key + current int + nextKey kv.Key +} + +func newDoneTaskKeeper(start kv.Key) *doneTaskKeeper { + return &doneTaskKeeper{ + doneTaskNextKey: make(map[int]kv.Key), + current: 0, + nextKey: start, + } +} + +func (n *doneTaskKeeper) updateNextKey(doneTaskID int, next kv.Key) { + if doneTaskID == n.current { + n.current++ + n.nextKey = next + for { + if nKey, ok := n.doneTaskNextKey[n.current]; ok { + delete(n.doneTaskNextKey, n.current) + n.current++ + n.nextKey = nKey + } else { + break + } + } + return + } + n.doneTaskNextKey[doneTaskID] = next +} diff --git a/ddl/backfilling_test.go b/ddl/backfilling_test.go new file mode 100644 index 0000000000000..167b809dd4487 --- /dev/null +++ b/ddl/backfilling_test.go @@ -0,0 +1,45 @@ +// Copyright 2022 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 ddl + +import ( + "bytes" + "testing" + + "github.com/pingcap/tidb/kv" + "github.com/stretchr/testify/require" +) + +func TestDoneTaskKeeper(t *testing.T) { + n := newDoneTaskKeeper(kv.Key("a")) + n.updateNextKey(0, kv.Key("b")) + n.updateNextKey(1, kv.Key("c")) + require.True(t, bytes.Equal(n.nextKey, kv.Key("c"))) + require.Len(t, n.doneTaskNextKey, 0) + + n.updateNextKey(4, kv.Key("f")) + require.True(t, bytes.Equal(n.nextKey, kv.Key("c"))) + require.Len(t, n.doneTaskNextKey, 1) + n.updateNextKey(3, kv.Key("e")) + n.updateNextKey(5, kv.Key("g")) + require.True(t, bytes.Equal(n.nextKey, kv.Key("c"))) + require.Len(t, n.doneTaskNextKey, 3) + n.updateNextKey(2, kv.Key("d")) + require.True(t, bytes.Equal(n.nextKey, kv.Key("g"))) + require.Len(t, n.doneTaskNextKey, 0) + + n.updateNextKey(6, kv.Key("h")) + require.True(t, bytes.Equal(n.nextKey, kv.Key("h"))) +} diff --git a/ddl/cluster.go b/ddl/cluster.go index 7d9070578af50..ebb833156cec2 100644 --- a/ddl/cluster.go +++ b/ddl/cluster.go @@ -60,12 +60,15 @@ var pdScheduleKey = []string{ const ( flashbackMaxBackoff = 1800000 // 1800s flashbackTimeout = 3 * time.Minute // 3min +) - pdScheduleArgsOffset = 1 - gcEnabledArgsOffset = 2 - autoAnalyzeOffset = 3 - totalLockedRegionsOffset = 4 - commitTSOffset = 5 +const ( + pdScheduleArgsOffset = 1 + iota + gcEnabledOffset + autoAnalyzeOffset + readOnlyOffset + totalLockedRegionsOffset + commitTSOffset ) func closePDSchedule() error { @@ -122,8 +125,8 @@ func ValidateFlashbackTS(ctx context.Context, sctx sessionctx.Context, flashBack return gcutil.ValidateSnapshotWithGCSafePoint(flashBackTS, gcSafePoint) } -func setTiDBEnableAutoAnalyze(sess sessionctx.Context, value string) error { - return sess.GetSessionVars().GlobalVarsAccessor.SetGlobalSysVar(context.Background(), variable.TiDBEnableAutoAnalyze, value) +func setTiDBEnableAutoAnalyze(ctx context.Context, sess sessionctx.Context, value string) error { + return sess.GetSessionVars().GlobalVarsAccessor.SetGlobalSysVar(ctx, variable.TiDBEnableAutoAnalyze, value) } func getTiDBEnableAutoAnalyze(sess sessionctx.Context) (string, error) { @@ -134,6 +137,18 @@ func getTiDBEnableAutoAnalyze(sess sessionctx.Context) (string, error) { return val, nil } +func setTiDBSuperReadOnly(ctx context.Context, sess sessionctx.Context, value string) error { + return sess.GetSessionVars().GlobalVarsAccessor.SetGlobalSysVar(ctx, variable.TiDBSuperReadOnly, value) +} + +func getTiDBSuperReadOnly(sess sessionctx.Context) (string, error) { + val, err := sess.GetSessionVars().GlobalVarsAccessor.GetGlobalSysVar(variable.TiDBSuperReadOnly) + if err != nil { + return "", errors.Trace(err) + } + return val, nil +} + func checkAndSetFlashbackClusterInfo(sess sessionctx.Context, d *ddlCtx, t *meta.Meta, job *model.Job, flashbackTS uint64) (err error) { if err = ValidateFlashbackTS(d.ctx, sess, flashbackTS); err != nil { return err @@ -145,7 +160,10 @@ func checkAndSetFlashbackClusterInfo(sess sessionctx.Context, d *ddlCtx, t *meta if err = closePDSchedule(); err != nil { return err } - if err = setTiDBEnableAutoAnalyze(sess, variable.Off); err != nil { + if err = setTiDBEnableAutoAnalyze(d.ctx, sess, variable.Off); err != nil { + return err + } + if err = setTiDBSuperReadOnly(d.ctx, sess, variable.On); err != nil { return err } @@ -465,9 +483,9 @@ func (w *worker) onFlashbackCluster(d *ddlCtx, t *meta.Meta, job *model.Job) (ve var flashbackTS, lockedRegions, commitTS uint64 var pdScheduleValue map[string]interface{} - var autoAnalyzeValue string + var autoAnalyzeValue, readOnlyValue string var gcEnabledValue bool - if err := job.DecodeArgs(&flashbackTS, &pdScheduleValue, &gcEnabledValue, &autoAnalyzeValue, &lockedRegions, &commitTS); err != nil { + if err := job.DecodeArgs(&flashbackTS, &pdScheduleValue, &gcEnabledValue, &autoAnalyzeValue, &readOnlyValue, &lockedRegions, &commitTS); err != nil { job.State = model.JobStateCancelled return ver, errors.Trace(err) } @@ -494,13 +512,19 @@ func (w *worker) onFlashbackCluster(d *ddlCtx, t *meta.Meta, job *model.Job) (ve job.State = model.JobStateCancelled return ver, errors.Trace(err) } - job.Args[gcEnabledArgsOffset] = &gcEnableValue + job.Args[gcEnabledOffset] = &gcEnableValue autoAnalyzeValue, err = getTiDBEnableAutoAnalyze(sess) if err != nil { job.State = model.JobStateCancelled return ver, errors.Trace(err) } job.Args[autoAnalyzeOffset] = &autoAnalyzeValue + readOnlyValue, err = getTiDBSuperReadOnly(sess) + if err != nil { + job.State = model.JobStateCancelled + return ver, errors.Trace(err) + } + job.Args[readOnlyOffset] = &readOnlyValue job.SchemaState = model.StateDeleteOnly return ver, nil // Stage 2, check flashbackTS, close GC and PD schedule. @@ -593,10 +617,10 @@ func finishFlashbackCluster(w *worker, job *model.Job) error { var flashbackTS, lockedRegions, commitTS uint64 var pdScheduleValue map[string]interface{} - var autoAnalyzeValue string + var autoAnalyzeValue, readOnlyValue string var gcEnabled bool - if err := job.DecodeArgs(&flashbackTS, &pdScheduleValue, &gcEnabled, &autoAnalyzeValue, &lockedRegions, &commitTS); err != nil { + if err := job.DecodeArgs(&flashbackTS, &pdScheduleValue, &gcEnabled, &autoAnalyzeValue, &readOnlyValue, &lockedRegions, &commitTS); err != nil { return errors.Trace(err) } sess, err := w.sessPool.get() @@ -614,7 +638,10 @@ func finishFlashbackCluster(w *worker, job *model.Job) error { return err } } - return setTiDBEnableAutoAnalyze(sess, autoAnalyzeValue) + if err = setTiDBSuperReadOnly(w.ctx, sess, readOnlyValue); err != nil { + return err + } + return setTiDBEnableAutoAnalyze(w.ctx, sess, autoAnalyzeValue) }) if err != nil { return err diff --git a/ddl/cluster_test.go b/ddl/cluster_test.go index b32fcd709e924..db406230050fb 100644 --- a/ddl/cluster_test.go +++ b/ddl/cluster_test.go @@ -202,23 +202,35 @@ func TestGlobalVariablesOnFlashback(t *testing.T) { rs, err = tk.Exec("show variables like 'tidb_enable_auto_analyze'") assert.NoError(t, err) assert.Equal(t, tk.ResultSetToResult(rs, "").Rows()[0][1], variable.Off) + rs, err = tk.Exec("show variables like 'tidb_super_read_only'") + assert.NoError(t, err) + assert.Equal(t, tk.ResultSetToResult(rs, "").Rows()[0][1], variable.On) } } dom.DDL().SetHook(hook) - // first try with `tidb_gc_enable` = on + // first try with `tidb_gc_enable` = on and `tidb_super_read_only` = off tk.MustExec("set global tidb_gc_enable = on") + tk.MustExec("set global tidb_super_read_only = off") tk.MustExec(fmt.Sprintf("flashback cluster to timestamp '%s'", oracle.GetTimeFromTS(ts))) - rs, err := tk.Exec("show variables like 'tidb_gc_enable'") + + rs, err := tk.Exec("show variables like 'tidb_super_read_only'") + require.NoError(t, err) + require.Equal(t, tk.ResultSetToResult(rs, "").Rows()[0][1], variable.Off) + rs, err = tk.Exec("show variables like 'tidb_gc_enable'") require.NoError(t, err) require.Equal(t, tk.ResultSetToResult(rs, "").Rows()[0][1], variable.On) - // second try with `tidb_gc_enable` = off + // second try with `tidb_gc_enable` = off and `tidb_super_read_only` = on tk.MustExec("set global tidb_gc_enable = off") + tk.MustExec("set global tidb_super_read_only = on") ts, err = tk.Session().GetStore().GetOracle().GetTimestamp(context.Background(), &oracle.Option{}) require.NoError(t, err) tk.MustExec(fmt.Sprintf("flashback cluster to timestamp '%s'", oracle.GetTimeFromTS(ts))) + rs, err = tk.Exec("show variables like 'tidb_super_read_only'") + require.NoError(t, err) + require.Equal(t, tk.ResultSetToResult(rs, "").Rows()[0][1], variable.On) rs, err = tk.Exec("show variables like 'tidb_gc_enable'") require.NoError(t, err) require.Equal(t, tk.ResultSetToResult(rs, "").Rows()[0][1], variable.Off) diff --git a/ddl/column.go b/ddl/column.go index 10db5120e9351..06f6e2897f3ae 100644 --- a/ddl/column.go +++ b/ddl/column.go @@ -1165,7 +1165,7 @@ func newUpdateColumnWorker(sessCtx sessionctx.Context, id int, t table.PhysicalT } rowDecoder := decoder.NewRowDecoder(t, t.WritableCols(), decodeColMap) return &updateColumnWorker{ - backfillWorker: newBackfillWorker(sessCtx, id, t, reorgInfo, typeUpdateColumnWorker), + backfillWorker: newBackfillWorker(jc.ddlJobCtx, sessCtx, id, t, reorgInfo, typeUpdateColumnWorker), oldColInfo: oldCol, newColInfo: newCol, metricCounter: metrics.BackfillTotalCounter.WithLabelValues(metrics.GenerateReorgLabel("update_col_rate", reorgInfo.SchemaName, t.Meta().Name.String())), diff --git a/ddl/column_type_change_test.go b/ddl/column_type_change_test.go index ae0adda97b99b..818d0714080f1 100644 --- a/ddl/column_type_change_test.go +++ b/ddl/column_type_change_test.go @@ -1946,7 +1946,7 @@ func TestChangeIntToBitWillPanicInBackfillIndexes(t *testing.T) { " KEY `idx3` (`a`,`b`),\n" + " KEY `idx4` (`a`,`b`,`c`)\n" + ") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin")) - tk.MustQuery("select * from t").Check(testkit.Rows("\x13 1 1.00", "\x11 2 2.00")) + tk.MustQuery("select * from t").Sort().Check(testkit.Rows("\x11 2 2.00", "\x13 1 1.00")) } // Close issue #24584 diff --git a/ddl/db_partition_test.go b/ddl/db_partition_test.go index b10f01c272ef9..e5ad2aa2bbfec 100644 --- a/ddl/db_partition_test.go +++ b/ddl/db_partition_test.go @@ -4535,6 +4535,7 @@ func TestPartitionTableWithAnsiQuotes(t *testing.T) { ` PARTITION "p4" VALUES LESS THAN ('\\''\t\n','\\''\t\n'),` + "\n" + ` PARTITION "pMax" VALUES LESS THAN (MAXVALUE,MAXVALUE))`)) } + func TestAlterModifyColumnOnPartitionedTable(t *testing.T) { store := testkit.CreateMockStore(t) tk := testkit.NewTestKit(t, store) @@ -4665,6 +4666,28 @@ func TestAlterModifyColumnOnPartitionedTable(t *testing.T) { tk.MustGetErrCode(`alter table t modify a varchar(20)`, errno.ErrUnsupportedDDLOperation) } +func TestAlterModifyColumnOnPartitionedTableRename(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + schemaName := "modColPartRename" + tk.MustExec("create database " + schemaName) + tk.MustExec("use " + schemaName) + tk.MustExec(`create table t (a int, b char) partition by range (a) (partition p0 values less than (10))`) + tk.MustContainErrMsg(`alter table t change a c int`, "[planner:1054]Unknown column 'a' in 'expression'") + tk.MustExec(`drop table t`) + tk.MustExec(`create table t (a char, b char) partition by range columns (a) (partition p0 values less than ('z'))`) + tk.MustContainErrMsg(`alter table t change a c char`, "[ddl:8200]New column does not match partition definitions: [ddl:1567]partition column name cannot be found") + tk.MustExec(`drop table t`) + tk.MustExec(`create table t (a int, b char) partition by list (a) (partition p0 values in (10))`) + tk.MustContainErrMsg(`alter table t change a c int`, "[planner:1054]Unknown column 'a' in 'expression'") + tk.MustExec(`drop table t`) + tk.MustExec(`create table t (a char, b char) partition by list columns (a) (partition p0 values in ('z'))`) + tk.MustContainErrMsg(`alter table t change a c char`, "[ddl:8200]New column does not match partition definitions: [ddl:1567]partition column name cannot be found") + tk.MustExec(`drop table t`) + tk.MustExec(`create table t (a int, b char) partition by hash (a) partitions 3`) + tk.MustContainErrMsg(`alter table t change a c int`, "[planner:1054]Unknown column 'a' in 'expression'") +} + func TestAlterModifyColumnOnPartitionedTableFail(t *testing.T) { store := testkit.CreateMockStore(t) tk := testkit.NewTestKit(t, store) diff --git a/ddl/ddl.go b/ddl/ddl.go index c8edf0019ef13..7eeaa8c1f357e 100644 --- a/ddl/ddl.go +++ b/ddl/ddl.go @@ -1216,14 +1216,32 @@ func (d *ddl) SwitchConcurrentDDL(toConcurrentDDL bool) error { // SwitchMDL enables MDL or disable DDL. func (d *ddl) SwitchMDL(enable bool) error { + ctx, cancel := context.WithTimeout(context.Background(), time.Second*30) + defer cancel() + + // Disable MDL for test. + if enable && !variable.DefTiDBEnableConcurrentDDL { + sql := fmt.Sprintf("UPDATE HIGH_PRIORITY %[1]s.%[2]s SET VARIABLE_VALUE = %[4]d WHERE VARIABLE_NAME = '%[3]s'", + mysql.SystemDB, mysql.GlobalVariablesTable, variable.TiDBEnableMDL, 0) + sess, err := d.sessPool.get() + if err != nil { + logutil.BgLogger().Warn("[ddl] get session failed", zap.Error(err)) + return nil + } + defer d.sessPool.put(sess) + se := newSession(sess) + _, err = se.execute(ctx, sql, "disableMDL") + if err != nil { + logutil.BgLogger().Warn("[ddl] disable MDL failed", zap.Error(err)) + } + return nil + } + isEnableBefore := variable.EnableMDL.Load() if isEnableBefore == enable { return nil } - ctx, cancel := context.WithTimeout(context.Background(), time.Second*30) - defer cancel() - // Check if there is any DDL running. // This check can not cover every corner cases, so users need to guarantee that there is no DDL running by themselves. sess, err := d.sessPool.get() diff --git a/ddl/ddl_api.go b/ddl/ddl_api.go index e67c0eb001128..f60a80ece0d06 100644 --- a/ddl/ddl_api.go +++ b/ddl/ddl_api.go @@ -2716,9 +2716,10 @@ func (d *ddl) FlashbackCluster(ctx sessionctx.Context, flashbackTS uint64) error Args: []interface{}{ flashbackTS, map[string]interface{}{}, - true, /* tidb_gc_enable */ - variable.On, /* tidb_enable_auto_analyze */ - 0, /* totalRegions */ + true, /* tidb_gc_enable */ + variable.On, /* tidb_enable_auto_analyze */ + variable.Off, /* tidb_super_read_only */ + 0, /* totalRegions */ 0 /* newCommitTS */}, } err := d.DoDDLJob(ctx, job) @@ -4670,6 +4671,7 @@ func GetModifiableColumnJob( newTblInfo.Columns = newCols var buf bytes.Buffer + // TODO: update the partitioning columns with new names if column is renamed AppendPartitionInfo(tblInfo.GetPartitionInfo(), &buf, mysql.ModeNone) // The parser supports ALTER TABLE ... PARTITION BY ... even if the ddl code does not yet :) // Ignoring warnings diff --git a/ddl/ddl_workerpool_test.go b/ddl/ddl_workerpool_test.go index 39a3a6b1452d0..e9f324ce9dff8 100644 --- a/ddl/ddl_workerpool_test.go +++ b/ddl/ddl_workerpool_test.go @@ -15,6 +15,7 @@ package ddl import ( + "context" "testing" "github.com/ngaut/pools" @@ -38,7 +39,7 @@ func TestBackfillWorkerPool(t *testing.T) { reorgInfo := &reorgInfo{Job: &model.Job{ID: 1}} f := func() func() (pools.Resource, error) { return func() (pools.Resource, error) { - wk := newBackfillWorker(nil, 1, nil, reorgInfo, typeAddIndexWorker) + wk := newBackfillWorker(context.Background(), nil, 1, nil, reorgInfo, typeAddIndexWorker) return wk, nil } } diff --git a/ddl/index.go b/ddl/index.go index 4404ea4f551bd..b93c968bced1b 100644 --- a/ddl/index.go +++ b/ddl/index.go @@ -1214,7 +1214,7 @@ func newAddIndexWorker(sessCtx sessionctx.Context, id int, t table.PhysicalTable return &addIndexWorker{ baseIndexWorker: baseIndexWorker{ - backfillWorker: newBackfillWorker(sessCtx, id, t, reorgInfo, typeAddIndexWorker), + backfillWorker: newBackfillWorker(jc.ddlJobCtx, sessCtx, id, t, reorgInfo, typeAddIndexWorker), indexes: []table.Index{index}, rowDecoder: rowDecoder, defaultVals: make([]types.Datum, len(t.WritableCols())), @@ -1712,7 +1712,7 @@ func newCleanUpIndexWorker(sessCtx sessionctx.Context, id int, t table.PhysicalT } return &cleanUpIndexWorker{ baseIndexWorker: baseIndexWorker{ - backfillWorker: newBackfillWorker(sessCtx, id, t, reorgInfo, typeCleanUpIndexWorker), + backfillWorker: newBackfillWorker(jc.ddlJobCtx, sessCtx, id, t, reorgInfo, typeCleanUpIndexWorker), indexes: indexes, rowDecoder: rowDecoder, defaultVals: make([]types.Datum, len(t.WritableCols())), diff --git a/ddl/index_merge_tmp.go b/ddl/index_merge_tmp.go index 9159b47c47951..b33111119facc 100644 --- a/ddl/index_merge_tmp.go +++ b/ddl/index_merge_tmp.go @@ -95,7 +95,7 @@ func newMergeTempIndexWorker(sessCtx sessionctx.Context, id int, t table.Physica index := tables.NewIndex(t.GetPhysicalID(), t.Meta(), indexInfo) return &mergeIndexWorker{ - backfillWorker: newBackfillWorker(sessCtx, id, t, reorgInfo, typeAddIndexMergeTmpWorker), + backfillWorker: newBackfillWorker(jc.ddlJobCtx, sessCtx, id, t, reorgInfo, typeAddIndexMergeTmpWorker), index: index, jobContext: jc, } diff --git a/ddl/index_modify_test.go b/ddl/index_modify_test.go index 38f091f41bc10..c24f2dfb1b783 100644 --- a/ddl/index_modify_test.go +++ b/ddl/index_modify_test.go @@ -1069,15 +1069,18 @@ func TestAddIndexWithDupIndex(t *testing.T) { } func TestAddIndexUniqueFailOnDuplicate(t *testing.T) { + ddl.ResultCounterForTest = &atomic.Int32{} store := testkit.CreateMockStore(t) tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("create table t (a bigint primary key clustered, b int);") - tk.MustExec("set @@global.tidb_ddl_reorg_worker_cnt = 2;") + tk.MustExec("set @@global.tidb_ddl_reorg_worker_cnt = 1;") for i := 1; i <= 12; i++ { tk.MustExec("insert into t values (?, ?)", i, i) } tk.MustExec("insert into t values (0, 1);") // Insert a duplicate key. tk.MustQuery("split table t by (0), (1), (2), (3), (4), (5), (6), (7), (8), (9), (10), (11), (12);").Check(testkit.Rows("13 1")) tk.MustGetErrCode("alter table t add unique index idx (b);", errno.ErrDupEntry) + require.Less(t, int(ddl.ResultCounterForTest.Load()), 6) + ddl.ResultCounterForTest = nil } diff --git a/ddl/partition.go b/ddl/partition.go index 5a6a5b561b146..7bba0b1006332 100644 --- a/ddl/partition.go +++ b/ddl/partition.go @@ -782,6 +782,9 @@ func generatePartitionDefinitionsFromInterval(ctx sessionctx.Context, partOption } if len(tbInfo.Partition.Columns) > 0 { colTypes := collectColumnsType(tbInfo) + if len(colTypes) != len(tbInfo.Partition.Columns) { + return dbterror.ErrWrongPartitionName.GenWithStack("partition column name cannot be found") + } if _, err := checkAndGetColumnsTypeAndValuesMatch(ctx, colTypes, first.Exprs); err != nil { return err } @@ -1081,6 +1084,9 @@ func buildListPartitionDefinitions(ctx sessionctx.Context, defs []*ast.Partition definitions := make([]model.PartitionDefinition, 0, len(defs)) exprChecker := newPartitionExprChecker(ctx, nil, checkPartitionExprAllowed) colTypes := collectColumnsType(tbInfo) + if len(colTypes) != len(tbInfo.Partition.Columns) { + return nil, dbterror.ErrWrongPartitionName.GenWithStack("partition column name cannot be found") + } for _, def := range defs { if err := def.Clause.Validate(model.PartitionTypeList, len(tbInfo.Partition.Columns)); err != nil { return nil, err @@ -1139,7 +1145,11 @@ func collectColumnsType(tbInfo *model.TableInfo) []types.FieldType { if len(tbInfo.Partition.Columns) > 0 { colTypes := make([]types.FieldType, 0, len(tbInfo.Partition.Columns)) for _, col := range tbInfo.Partition.Columns { - colTypes = append(colTypes, findColumnByName(col.L, tbInfo).FieldType) + c := findColumnByName(col.L, tbInfo) + if c == nil { + return nil + } + colTypes = append(colTypes, c.FieldType) } return colTypes @@ -1152,6 +1162,9 @@ func buildRangePartitionDefinitions(ctx sessionctx.Context, defs []*ast.Partitio definitions := make([]model.PartitionDefinition, 0, len(defs)) exprChecker := newPartitionExprChecker(ctx, nil, checkPartitionExprAllowed) colTypes := collectColumnsType(tbInfo) + if len(colTypes) != len(tbInfo.Partition.Columns) { + return nil, dbterror.ErrWrongPartitionName.GenWithStack("partition column name cannot be found") + } for _, def := range defs { if err := def.Clause.Validate(model.PartitionTypeRange, len(tbInfo.Partition.Columns)); err != nil { return nil, err diff --git a/distsql/select_result.go b/distsql/select_result.go index a2d6215987a32..0e807b360d0ad 100644 --- a/distsql/select_result.go +++ b/distsql/select_result.go @@ -311,7 +311,7 @@ func (r *selectResult) readFromDefault(ctx context.Context, chk *chunk.Chunk) er func (r *selectResult) readFromChunk(ctx context.Context, chk *chunk.Chunk) error { if r.respChunkDecoder == nil { r.respChunkDecoder = chunk.NewDecoder( - r.ctx.GetSessionVars().GetNewChunk(r.fieldTypes, 0), + chunk.NewChunkWithCapacity(r.fieldTypes, 0), r.fieldTypes, ) } diff --git a/dumpling/export/config.go b/dumpling/export/config.go index 62ef45b45a0ae..548535ed486ac 100644 --- a/dumpling/export/config.go +++ b/dumpling/export/config.go @@ -221,6 +221,9 @@ func (conf *Config) GetDriverConfig(db string) *mysql.Config { driverCfg.MaxAllowedPacket = 0 if conf.Security.DriveTLSName != "" { driverCfg.TLSConfig = conf.Security.DriveTLSName + } else { + // Use TLS first. + driverCfg.TLSConfig = "preferred" } if conf.AllowCleartextPasswords { driverCfg.AllowCleartextPasswords = true diff --git a/executor/BUILD.bazel b/executor/BUILD.bazel index 8d9eb3af53211..787a7ed8429f2 100644 --- a/executor/BUILD.bazel +++ b/executor/BUILD.bazel @@ -244,7 +244,7 @@ go_library( go_test( name = "executor_test", - timeout = "long", + timeout = "moderate", srcs = [ "adapter_test.go", "admin_test.go", diff --git a/executor/aggregate_test.go b/executor/aggregate_test.go index d29fa86d961fe..1868802d65d8d 100644 --- a/executor/aggregate_test.go +++ b/executor/aggregate_test.go @@ -1642,11 +1642,11 @@ func TestIssue26885(t *testing.T) { tk.MustExec("INSERT INTO t1 (c1) VALUES ('');") tk.MustExec("INSERT INTO t1 (c1) VALUES (0);") tk.MustQuery("select * from t1").Check(testkit.Rows("b", "", "a", "", "")) - tk.MustQuery("select c1 + 0 from t1").Check(testkit.Rows("3", "2", "1", "2", "0")) + tk.MustQuery("select c1 + 0 from t1").Sort().Check(testkit.Rows("0", "1", "2", "2", "3")) tk.MustQuery("SELECT c1 + 0, COUNT(c1) FROM t1 GROUP BY c1 order by c1;").Check(testkit.Rows("0 1", "1 1", "2 2", "3 1")) tk.MustExec("alter table t1 add index idx(c1); ") - tk.MustQuery("select c1 + 0 from t1").Check(testkit.Rows("3", "2", "1", "2", "0")) + tk.MustQuery("select c1 + 0 from t1").Sort().Check(testkit.Rows("0", "1", "2", "2", "3")) tk.MustQuery("SELECT c1 + 0, COUNT(c1) FROM t1 GROUP BY c1 order by c1;").Check(testkit.Rows("0 1", "1 1", "2 2", "3 1")) tk.MustExec(`DROP TABLE IF EXISTS t1;`) diff --git a/executor/analyzetest/analyze_test.go b/executor/analyzetest/analyze_test.go index d789b6a276fdf..c5043935e1650 100644 --- a/executor/analyzetest/analyze_test.go +++ b/executor/analyzetest/analyze_test.go @@ -975,6 +975,7 @@ func TestSavedAnalyzeOptions(t *testing.T) { tk.MustExec("use test") tk.MustExec("set @@session.tidb_analyze_version = 2") + tk.MustExec("set @@session.tidb_stats_load_sync_wait = 20000") // to stabilise test tk.MustExec("create table t(a int, b int, c int, primary key(a), key idx(b))") tk.MustExec("insert into t values (1,1,1),(2,1,2),(3,1,3),(4,1,4),(5,1,5),(6,1,6),(7,7,7),(8,8,8),(9,9,9)") @@ -1062,6 +1063,7 @@ func TestSavedPartitionAnalyzeOptions(t *testing.T) { tk.MustExec("use test") tk.MustExec("set @@session.tidb_analyze_version = 2") + tk.MustExec("set @@session.tidb_stats_load_sync_wait = 20000") // to stabilise test tk.MustExec("set @@session.tidb_partition_prune_mode = 'static'") createTable := `CREATE TABLE t (a int, b int, c varchar(10), primary key(a), index idx(b)) PARTITION BY RANGE ( a ) ( @@ -2580,6 +2582,7 @@ func TestAnalyzePartitionTableWithDynamicMode(t *testing.T) { tk.MustExec("use test") tk.MustExec("set @@session.tidb_analyze_version = 2") + tk.MustExec("set @@session.tidb_stats_load_sync_wait = 20000") // to stabilise test tk.MustExec("set @@session.tidb_partition_prune_mode = 'dynamic'") createTable := `CREATE TABLE t (a int, b int, c varchar(10), d int, primary key(a), index idx(b)) PARTITION BY RANGE ( a ) ( @@ -2673,6 +2676,7 @@ func TestAnalyzePartitionTableStaticToDynamic(t *testing.T) { tk.MustExec("use test") tk.MustExec("set @@session.tidb_analyze_version = 2") + tk.MustExec("set @@session.tidb_stats_load_sync_wait = 20000") // to stabilise test tk.MustExec("set @@session.tidb_partition_prune_mode = 'static'") createTable := `CREATE TABLE t (a int, b int, c varchar(10), d int, primary key(a), index idx(b)) PARTITION BY RANGE ( a ) ( @@ -2864,6 +2868,7 @@ func TestAnalyzePartitionStaticToDynamic(t *testing.T) { tk.MustExec("use test") tk.MustExec("set @@session.tidb_analyze_version = 2") + tk.MustExec("set @@session.tidb_stats_load_sync_wait = 20000") // to stabilise test createTable := `CREATE TABLE t (a int, b int, c varchar(10), d int, primary key(a), index idx(b)) PARTITION BY RANGE ( a ) ( PARTITION p0 VALUES LESS THAN (10), @@ -2943,6 +2948,7 @@ func TestAnalyzePartitionUnderV1Dynamic(t *testing.T) { tk.MustExec("use test") tk.MustExec("set @@session.tidb_analyze_version = 1") + tk.MustExec("set @@session.tidb_stats_load_sync_wait = 20000") // to stabilise test tk.MustExec("set @@session.tidb_partition_prune_mode = 'dynamic'") createTable := `CREATE TABLE t (a int, b int, c varchar(10), d int, primary key(a), index idx(b)) PARTITION BY RANGE ( a ) ( diff --git a/executor/bind.go b/executor/bind.go index aed1ee3460e68..cf337968d4130 100644 --- a/executor/bind.go +++ b/executor/bind.go @@ -75,9 +75,12 @@ func (e *SQLBindExec) dropSQLBind() error { } if !e.isGlobal { handle := e.ctx.Value(bindinfo.SessionBindInfoKeyType).(*bindinfo.SessionHandle) - return handle.DropBindRecord(e.normdOrigSQL, e.db, bindInfo) + err := handle.DropBindRecord(e.normdOrigSQL, e.db, bindInfo) + return err } - return domain.GetDomain(e.ctx).BindHandle().DropBindRecord(e.normdOrigSQL, e.db, bindInfo) + affectedRows, err := domain.GetDomain(e.ctx).BindHandle().DropBindRecord(e.normdOrigSQL, e.db, bindInfo) + e.ctx.GetSessionVars().StmtCtx.AddAffectedRows(affectedRows) + return err } func (e *SQLBindExec) setBindingStatus() error { diff --git a/executor/builder.go b/executor/builder.go index 1b88b6e4047c6..70d91e64137a4 100644 --- a/executor/builder.go +++ b/executor/builder.go @@ -1540,7 +1540,7 @@ func (b *executorBuilder) buildHashAgg(v *plannercore.PhysicalHashAgg) Executor e.defaultVal = nil } else { if v.IsFinalAgg() { - e.defaultVal = e.ctx.GetSessionVars().GetNewChunk(retTypes(e), 1) + e.defaultVal = e.ctx.GetSessionVars().GetNewChunkWithCapacity(retTypes(e), 1, 1, e.AllocPool) } } for _, aggDesc := range v.AggFuncs { @@ -1603,7 +1603,7 @@ func (b *executorBuilder) buildStreamAgg(v *plannercore.PhysicalStreamAgg) Execu } else { // Only do this for final agg, see issue #35295, #30923 if v.IsFinalAgg() { - e.defaultVal = e.ctx.GetSessionVars().GetNewChunk(retTypes(e), 1) + e.defaultVal = e.ctx.GetSessionVars().GetNewChunkWithCapacity(retTypes(e), 1, 1, e.AllocPool) } } for i, aggDesc := range v.AggFuncs { diff --git a/executor/distsql.go b/executor/distsql.go index 182831bc90021..0cef7e66d441e 100644 --- a/executor/distsql.go +++ b/executor/distsql.go @@ -866,7 +866,7 @@ func (w *indexWorker) fetchHandles(ctx context.Context, result distsql.SelectRes } }() retTps := w.idxLookup.getRetTpsByHandle() - chk := w.idxLookup.ctx.GetSessionVars().GetNewChunk(retTps, w.idxLookup.maxChunkSize) + chk := w.idxLookup.ctx.GetSessionVars().GetNewChunkWithCapacity(retTps, w.idxLookup.maxChunkSize, w.idxLookup.maxChunkSize, w.idxLookup.AllocPool) idxID := w.idxLookup.getIndexPlanRootID() if w.idxLookup.ctx.GetSessionVars().StmtCtx.RuntimeStatsColl != nil { if idxID != w.idxLookup.id && w.idxLookup.stats != nil { diff --git a/executor/executor.go b/executor/executor.go index 542734e4b1a75..3d10ff32a5168 100644 --- a/executor/executor.go +++ b/executor/executor.go @@ -132,6 +132,7 @@ type baseExecutor struct { children []Executor retFieldTypes []*types.FieldType runtimeStats *execdetails.BasicRuntimeStats + AllocPool chunk.Allocator } const ( @@ -234,7 +235,7 @@ func newFirstChunk(e Executor) *chunk.Chunk { func tryNewCacheChunk(e Executor) *chunk.Chunk { base := e.base() s := base.ctx.GetSessionVars() - return s.GetNewChunkWithCapacity(base.retFieldTypes, base.initCap, base.maxChunkSize) + return s.GetNewChunkWithCapacity(base.retFieldTypes, base.initCap, base.maxChunkSize, base.AllocPool) } // newList creates a new List to buffer current executor's result. @@ -267,6 +268,7 @@ func newBaseExecutor(ctx sessionctx.Context, schema *expression.Schema, id int, schema: schema, initCap: ctx.GetSessionVars().InitChunkSize, maxChunkSize: ctx.GetSessionVars().MaxChunkSize, + AllocPool: ctx.GetSessionVars().ChunkPool.Alloc, } if ctx.GetSessionVars().StmtCtx.RuntimeStatsColl != nil { if e.id > 0 { @@ -1954,6 +1956,7 @@ func ResetContextOfStmt(ctx sessionctx.Context, s ast.StmtNode) (err error) { vars.MemTracker.ResetMaxConsumed() vars.DiskTracker.ResetMaxConsumed() vars.MemTracker.SessionID = vars.ConnectionID + vars.StmtCtx.TableStats = make(map[int64]interface{}) if _, ok := s.(*ast.AnalyzeTableStmt); ok { sc.InitMemTracker(memory.LabelForAnalyzeMemory, -1) diff --git a/executor/explainfor_test.go b/executor/explainfor_test.go index 6d7cc1f11f7dd..db80c89f8d26e 100644 --- a/executor/explainfor_test.go +++ b/executor/explainfor_test.go @@ -443,6 +443,7 @@ func TestPointGetUserVarPlanCache(t *testing.T) { tk.Session().Auth(&auth.UserIdentity{Username: "root", Hostname: "localhost", CurrentUser: true, AuthUsername: "root", AuthHostname: "%"}, nil, []byte("012345678901234567890")) tk.MustExec("use test") + tk.MustExec("set tidb_cost_model_version=2") tk.MustExec("set @@tidb_enable_collect_execution_info=0;") tk.Session().GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeOn tk.MustExec("drop table if exists t1") @@ -461,12 +462,12 @@ func TestPointGetUserVarPlanCache(t *testing.T) { tk.Session().SetSessionManager(&testkit.MockSessionManager{PS: ps}) 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`, - `└─IndexJoin_17 1.00 root inner join, inner:TableReader_13, outer key:test.t2.a, inner key:test.t1.a, equal cond:eq(test.t2.a, test.t1.a)`, - ` ├─Selection_44(Build) 0.80 root not(isnull(test.t2.a))`, - ` │ └─Point_Get_43 1.00 root table:t2, index:idx_a(a) `, - ` └─TableReader_13(Probe) 0.00 root data:Selection_12`, - ` └─Selection_12 0.00 cop[tikv] eq(test.t1.a, 1)`, - ` └─TableRangeScan_11 0.80 cop[tikv] table:t1 range: decided by [eq(test.t1.a, test.t2.a)], keep order:false, stats:pseudo`)) + `└─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`, + ` └─Selection_39(Probe) 0.80 root not(isnull(test.t2.a))`, + ` └─Point_Get_38 1.00 root table:t2, index:idx_a(a) `)) tk.MustExec("set @a=2") tk.MustQuery("execute stmt using @a").Check(testkit.Rows( @@ -477,12 +478,12 @@ func TestPointGetUserVarPlanCache(t *testing.T) { tk.Session().SetSessionManager(&testkit.MockSessionManager{PS: ps}) 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`, - `└─IndexJoin_17 1.00 root inner join, inner:TableReader_13, outer key:test.t2.a, inner key:test.t1.a, equal cond:eq(test.t2.a, test.t1.a)`, - ` ├─Selection_44(Build) 0.80 root not(isnull(test.t2.a))`, - ` │ └─Point_Get_43 1.00 root table:t2, index:idx_a(a) `, - ` └─TableReader_13(Probe) 0.00 root data:Selection_12`, - ` └─Selection_12 0.00 cop[tikv] eq(test.t1.a, 2)`, - ` └─TableRangeScan_11 0.80 cop[tikv] table:t1 range: decided by [eq(test.t1.a, test.t2.a)], keep order:false, stats:pseudo`)) + `└─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`, + ` └─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( "2 4 2 2", )) diff --git a/executor/index_lookup_hash_join.go b/executor/index_lookup_hash_join.go index 9601dffc77900..c54b60749601d 100644 --- a/executor/index_lookup_hash_join.go +++ b/executor/index_lookup_hash_join.go @@ -418,7 +418,7 @@ func (e *IndexNestedLoopHashJoin) newInnerWorker(taskCh chan *indexHashJoinTask, innerCtx: e.innerCtx, outerCtx: e.outerCtx, ctx: e.ctx, - executorChk: e.ctx.GetSessionVars().GetNewChunk(e.innerCtx.rowTypes, e.maxChunkSize), + executorChk: e.ctx.GetSessionVars().GetNewChunkWithCapacity(e.innerCtx.rowTypes, e.maxChunkSize, e.maxChunkSize, e.AllocPool), indexRanges: copiedRanges, keyOff2IdxOff: e.keyOff2IdxOff, stats: innerStats, diff --git a/executor/index_lookup_join.go b/executor/index_lookup_join.go index 92f195985a191..05cc337d3d7ee 100644 --- a/executor/index_lookup_join.go +++ b/executor/index_lookup_join.go @@ -226,7 +226,7 @@ func (e *IndexLookUpJoin) newInnerWorker(taskCh chan *lookUpJoinTask) *innerWork outerCtx: e.outerCtx, taskCh: taskCh, ctx: e.ctx, - executorChk: e.ctx.GetSessionVars().GetNewChunk(e.innerCtx.rowTypes, e.maxChunkSize), + executorChk: e.ctx.GetSessionVars().GetNewChunkWithCapacity(e.innerCtx.rowTypes, e.maxChunkSize, e.maxChunkSize, e.AllocPool), indexRanges: copiedRanges, keyOff2IdxOff: e.keyOff2IdxOff, stats: innerStats, @@ -431,7 +431,7 @@ func (ow *outerWorker) buildTask(ctx context.Context) (*lookUpJoinTask, error) { } maxChunkSize := ow.ctx.GetSessionVars().MaxChunkSize for requiredRows > task.outerResult.Len() { - chk := ow.ctx.GetSessionVars().GetNewChunk(ow.outerCtx.rowTypes, maxChunkSize) + chk := ow.ctx.GetSessionVars().GetNewChunkWithCapacity(ow.outerCtx.rowTypes, maxChunkSize, maxChunkSize, ow.executor.base().AllocPool) chk = chk.SetRequiredRows(requiredRows, maxChunkSize) err := Next(ctx, ow.executor, chk) if err != nil { @@ -462,7 +462,7 @@ func (ow *outerWorker) buildTask(ctx context.Context) (*lookUpJoinTask, error) { } task.encodedLookUpKeys = make([]*chunk.Chunk, task.outerResult.NumChunks()) for i := range task.encodedLookUpKeys { - task.encodedLookUpKeys[i] = ow.ctx.GetSessionVars().GetNewChunk([]*types.FieldType{types.NewFieldType(mysql.TypeBlob)}, task.outerResult.GetChunk(i).NumRows()) + task.encodedLookUpKeys[i] = ow.ctx.GetSessionVars().GetNewChunkWithCapacity([]*types.FieldType{types.NewFieldType(mysql.TypeBlob)}, task.outerResult.GetChunk(i).NumRows(), task.outerResult.GetChunk(i).NumRows(), ow.executor.base().AllocPool) } return task, nil } diff --git a/executor/index_lookup_merge_join.go b/executor/index_lookup_merge_join.go index 1ba2c2940c3fd..8bd379944c825 100644 --- a/executor/index_lookup_merge_join.go +++ b/executor/index_lookup_merge_join.go @@ -706,7 +706,7 @@ func (imw *innerMergeWorker) dedupDatumLookUpKeys(lookUpContents []*indexJoinLoo // fetchNextInnerResult collects a chunk of inner results from inner child executor. func (imw *innerMergeWorker) fetchNextInnerResult(ctx context.Context, task *lookUpMergeJoinTask) (beginRow chunk.Row, err error) { - task.innerResult = imw.ctx.GetSessionVars().GetNewChunk(retTypes(imw.innerExec), imw.ctx.GetSessionVars().MaxChunkSize) + task.innerResult = imw.ctx.GetSessionVars().GetNewChunkWithCapacity(retTypes(imw.innerExec), imw.ctx.GetSessionVars().MaxChunkSize, imw.ctx.GetSessionVars().MaxChunkSize, imw.innerExec.base().AllocPool) err = Next(ctx, imw.innerExec, task.innerResult) task.innerIter = chunk.NewIterator4Chunk(task.innerResult) beginRow = task.innerIter.Begin() diff --git a/executor/index_merge_reader.go b/executor/index_merge_reader.go index 82c6ab2f50817..0e7eb394710fd 100644 --- a/executor/index_merge_reader.go +++ b/executor/index_merge_reader.go @@ -508,7 +508,7 @@ func (w *partialTableWorker) syncErr(resultCh chan<- *lookupTableTask, err error func (w *partialTableWorker) fetchHandles(ctx context.Context, exitCh <-chan struct{}, fetchCh chan<- *lookupTableTask, resultCh chan<- *lookupTableTask, finished <-chan struct{}, handleCols plannercore.HandleCols) (count int64, err error) { - chk := w.sc.GetSessionVars().GetNewChunk(retTypes(w.tableReader), w.maxChunkSize) + chk := w.sc.GetSessionVars().GetNewChunkWithCapacity(retTypes(w.tableReader), w.maxChunkSize, w.maxChunkSize, w.tableReader.base().AllocPool) var basic *execdetails.BasicRuntimeStats if be := w.tableReader.base(); be != nil && be.runtimeStats != nil { basic = be.runtimeStats @@ -817,7 +817,7 @@ func (w *partialIndexWorker) fetchHandles( resultCh chan<- *lookupTableTask, finished <-chan struct{}, handleCols plannercore.HandleCols) (count int64, err error) { - chk := w.sc.GetSessionVars().GetNewChunk(handleCols.GetFieldsTypes(), w.maxChunkSize) + chk := chunk.NewChunkWithCapacity(handleCols.GetFieldsTypes(), w.maxChunkSize) var basicStats *execdetails.BasicRuntimeStats if w.stats != nil { if w.idxID != 0 { diff --git a/executor/join.go b/executor/join.go index 48d3e5d5a56f8..87781a49d7c50 100644 --- a/executor/join.go +++ b/executor/join.go @@ -297,7 +297,7 @@ func (e *HashJoinExec) fetchBuildSideRows(ctx context.Context, chkCh chan<- *chu if e.finished.Load().(bool) { return } - chk := e.ctx.GetSessionVars().GetNewChunk(e.buildSideExec.base().retFieldTypes, e.ctx.GetSessionVars().MaxChunkSize) + chk := e.ctx.GetSessionVars().GetNewChunkWithCapacity(e.buildSideExec.base().retFieldTypes, e.ctx.GetSessionVars().MaxChunkSize, e.ctx.GetSessionVars().MaxChunkSize, e.AllocPool) err = Next(ctx, e.buildSideExec, chk) if err != nil { e.buildFinished <- errors.Trace(err) diff --git a/executor/joiner.go b/executor/joiner.go index 5fe4d92eba2a2..842135802444f 100644 --- a/executor/joiner.go +++ b/executor/joiner.go @@ -192,7 +192,7 @@ func newJoiner(ctx sessionctx.Context, joinType plannercore.JoinType, return &antiLeftOuterSemiJoiner{base} case plannercore.LeftOuterJoin, plannercore.RightOuterJoin, plannercore.InnerJoin: if len(base.conditions) > 0 { - base.chk = ctx.GetSessionVars().GetNewChunk(shallowRowType, ctx.GetSessionVars().MaxChunkSize) + base.chk = chunk.NewChunkWithCapacity(shallowRowType, ctx.GetSessionVars().MaxChunkSize) } switch joinType { case plannercore.LeftOuterJoin: diff --git a/executor/merge_join_test.go b/executor/merge_join_test.go index 63e8b8f9d0b3e..7246589e40412 100644 --- a/executor/merge_join_test.go +++ b/executor/merge_join_test.go @@ -18,6 +18,7 @@ import ( "bytes" "fmt" "math/rand" + "sort" "strconv" "testing" @@ -255,19 +256,28 @@ func TestShuffleMergeJoinInDisk(t *testing.T) { tk.MustExec("set @@tidb_mem_quota_query=1;") tk.MustExec("set @@tidb_merge_join_concurrency=4;") + tk.MustExec("set @@tidb_max_chunk_size=32;") tk.MustExec("drop table if exists t") tk.MustExec("drop table if exists t1") tk.MustExec("create table t(c1 int, c2 int)") tk.MustExec("create table t1(c1 int, c2 int)") - tk.MustExec("insert into t values(1,1)") - tk.MustExec("insert into t1 values(1,3),(4,4)") - + tk.MustExec("insert into t values(1,1),(2,2),(3,3),(4,4)") + for i := 1; i <= 1024; i += 4 { + tk.MustExec(fmt.Sprintf("insert into t1 values(%v,%v),(%v,%v),(%v,%v),(%v,%v)", i, i, i+1, i+1, i+2, i+2, i+3, i+3)) + } result := checkMergeAndRun(tk, t, "select /*+ TIDB_SMJ(t) */ * from t1 left outer join t on t.c1 = t1.c1 where t.c1 = 1 or t1.c2 > 20") - result.Check(testkit.Rows("1 3 1 1")) + + var expect []string + expect = append(expect, "1 1 1 1") + for i := 21; i <= 1024; i++ { + expect = append(expect, fmt.Sprintf("%v %v ", i, i)) + } + sort.Strings(expect) + result.Sort().Check(testkit.Rows(expect...)) require.Equal(t, int64(0), tk.Session().GetSessionVars().MemTracker.BytesConsumed()) require.Greater(t, tk.Session().GetSessionVars().MemTracker.MaxConsumed(), int64(0)) require.Equal(t, int64(0), tk.Session().GetSessionVars().DiskTracker.BytesConsumed()) - require.GreaterOrEqual(t, tk.Session().GetSessionVars().DiskTracker.MaxConsumed(), int64(0)) + require.Greater(t, tk.Session().GetSessionVars().DiskTracker.MaxConsumed(), int64(0)) } func TestMergeJoinInDisk(t *testing.T) { diff --git a/executor/pipelined_window.go b/executor/pipelined_window.go index 505cf09f415d7..cda1d9c389fd0 100644 --- a/executor/pipelined_window.go +++ b/executor/pipelined_window.go @@ -217,7 +217,7 @@ func (e *PipelinedWindowExec) fetchChild(ctx context.Context) (EOF bool, err err } // TODO: reuse chunks - resultChk := e.ctx.GetSessionVars().GetNewChunkWithCapacity(e.retFieldTypes, 0, numRows) + resultChk := e.ctx.GetSessionVars().GetNewChunkWithCapacity(e.retFieldTypes, 0, numRows, e.AllocPool) err = e.copyChk(childResult, resultChk) if err != nil { return false, err diff --git a/executor/recover_test.go b/executor/recover_test.go index 26ed1a765f58f..aad1c93d9fb87 100644 --- a/executor/recover_test.go +++ b/executor/recover_test.go @@ -31,6 +31,7 @@ import ( "github.com/pingcap/tidb/util/gcutil" "github.com/stretchr/testify/require" "github.com/tikv/client-go/v2/oracle" + tikvutil "github.com/tikv/client-go/v2/util" ) func TestRecoverTable(t *testing.T) { @@ -472,9 +473,8 @@ func MockGC(tk *testkit.TestKit) (string, string, string, func()) { // disable emulator GC. // Otherwise emulator GC will delete table record as soon as possible after execute drop table ddl. ddlutil.EmulatorGCDisable() - gcTimeFormat := "20060102-15:04:05 -0700 MST" - timeBeforeDrop := time.Now().Add(0 - 48*60*60*time.Second).Format(gcTimeFormat) - timeAfterDrop := time.Now().Add(48 * 60 * 60 * time.Second).Format(gcTimeFormat) + timeBeforeDrop := time.Now().Add(0 - 48*60*60*time.Second).Format(tikvutil.GCTimeFormat) + timeAfterDrop := time.Now().Add(48 * 60 * 60 * time.Second).Format(tikvutil.GCTimeFormat) safePointSQL := `INSERT HIGH_PRIORITY INTO mysql.tidb VALUES ('tikv_gc_safe_point', '%[1]s', '') ON DUPLICATE KEY UPDATE variable_value = '%[1]s'` diff --git a/executor/seqtest/BUILD.bazel b/executor/seqtest/BUILD.bazel index 77c4b30c0edef..fa66357c81cc7 100644 --- a/executor/seqtest/BUILD.bazel +++ b/executor/seqtest/BUILD.bazel @@ -2,7 +2,7 @@ load("@io_bazel_rules_go//go:def.bzl", "go_test") go_test( name = "seqtest_test", - timeout = "long", + timeout = "moderate", srcs = [ "main_test.go", "prepared_test.go", diff --git a/executor/showtest/show_test.go b/executor/showtest/show_test.go index cd06bd56b2229..11fa95dc0bb8c 100644 --- a/executor/showtest/show_test.go +++ b/executor/showtest/show_test.go @@ -775,6 +775,7 @@ func TestShowWarningsForExprPushdown(t *testing.T) { store, dom := testkit.CreateMockStoreAndDomain(t) tk := testkit.NewTestKit(t, store) tk.MustExec("use test") + tk.MustExec(`set tidb_cost_model_version=2`) testSQL := `create table if not exists show_warnings_expr_pushdown (a int, value date)` tk.MustExec(testSQL) @@ -796,10 +797,10 @@ func TestShowWarningsForExprPushdown(t *testing.T) { tk.MustExec("explain select * from show_warnings_expr_pushdown t where md5(value) = '2020-01-01'") require.Equal(t, uint16(1), tk.Session().GetSessionVars().StmtCtx.WarningCount()) tk.MustQuery("show warnings").Check(testkit.RowsWithSep("|", "Warning|1105|Scalar function 'md5'(signature: MD5, return type: var_string(32)) is not supported to push down to tiflash now.")) - tk.MustExec("explain select max(md5(value)) from show_warnings_expr_pushdown group by a") + tk.MustExec("explain select /*+ read_from_storage(tiflash[show_warnings_expr_pushdown]) */ max(md5(value)) from show_warnings_expr_pushdown group by a") require.Equal(t, uint16(2), tk.Session().GetSessionVars().StmtCtx.WarningCount()) tk.MustQuery("show warnings").Check(testkit.RowsWithSep("|", "Warning|1105|Scalar function 'md5'(signature: MD5, return type: var_string(32)) is not supported to push down to tiflash now.", "Warning|1105|Aggregation can not be pushed to tiflash because arguments of AggFunc `max` contains unsupported exprs")) - tk.MustExec("explain select max(a) from show_warnings_expr_pushdown group by md5(value)") + tk.MustExec("explain select /*+ read_from_storage(tiflash[show_warnings_expr_pushdown]) */ max(a) from show_warnings_expr_pushdown group by md5(value)") require.Equal(t, uint16(2), tk.Session().GetSessionVars().StmtCtx.WarningCount()) tk.MustQuery("show warnings").Check(testkit.RowsWithSep("|", "Warning|1105|Scalar function 'md5'(signature: MD5, return type: var_string(32)) is not supported to push down to tiflash now.", "Warning|1105|Aggregation can not be pushed to tiflash because groupByItems contain unsupported exprs")) tk.MustExec("set tidb_opt_distinct_agg_push_down=0") diff --git a/executor/window.go b/executor/window.go index ef284344d0c8c..aaa1e51cacc85 100644 --- a/executor/window.go +++ b/executor/window.go @@ -162,7 +162,7 @@ func (e *WindowExec) fetchChild(ctx context.Context) (EOF bool, err error) { return true, nil } - resultChk := e.ctx.GetSessionVars().GetNewChunkWithCapacity(e.retFieldTypes, 0, numRows) + resultChk := e.ctx.GetSessionVars().GetNewChunkWithCapacity(e.retFieldTypes, 0, numRows, e.AllocPool) err = e.copyChk(childResult, resultChk) if err != nil { return false, err diff --git a/extension/event_listener_test.go b/extension/event_listener_test.go index ece046f820b64..6e108262dbf8c 100644 --- a/extension/event_listener_test.go +++ b/extension/event_listener_test.go @@ -137,6 +137,7 @@ func TestExtensionStmtEvents(t *testing.T) { require.NoError(t, conn.HandleQuery(context.Background(), "SET tidb_multi_statement_mode='ON'")) require.NoError(t, conn.HandleQuery(context.Background(), "use test")) require.NoError(t, conn.HandleQuery(context.Background(), "create table t1(a int, b int)")) + require.NoError(t, conn.HandleQuery(context.Background(), "create table t2(id int primary key)")) require.NoError(t, conn.HandleQuery(context.Background(), "create database test2")) require.NoError(t, conn.HandleQuery(context.Background(), "create table test2.t1(c int, d int)")) require.NoError(t, conn.HandleQuery(context.Background(), "set @a=1")) @@ -158,6 +159,10 @@ func TestExtensionStmtEvents(t *testing.T) { stmtID3 := getPreparedID(t, conn.Context()) require.NoError(t, conn.HandleQuery(context.Background(), "drop table tnoexist")) + cmd = append([]byte{mysql.ComStmtPrepare}, []byte("insert into t2 values(?)")...) + require.NoError(t, conn.Dispatch(context.Background(), cmd)) + stmtID4 := getPreparedID(t, conn.Context()) + connID := conn.Context().Session.GetSessionVars().ConnectionID require.NotEqual(t, uint64(0), connID) @@ -186,6 +191,14 @@ func TestExtensionStmtEvents(t *testing.T) { {DB: "test", Table: "t1"}, }, }, + { + sql: "insert into t2 values(1)", + redactText: "insert into `t2` values ( ? )", + affectedRows: 1, + tables: []stmtctx.TableEntry{ + {DB: "test", Table: "t2"}, + }, + }, { binaryExecute: stmtID2, executeParams: []paramInfo{ @@ -206,7 +219,22 @@ func TestExtensionStmtEvents(t *testing.T) { }, originalText: "select * from tnoexist where n=?", redactText: "select * from `tnoexist` where `n` = ?", - err: "select * from tnoexist where n=? [arguments: 5]: [planner:8113]Schema change caused error: [schema:1146]Table 'test.tnoexist' doesn't exist", + tables: []stmtctx.TableEntry{ + {DB: "test", Table: "tnoexist"}, + }, + err: "select * from tnoexist where n=? [arguments: 5]: [planner:8113]Schema change caused error: [schema:1146]Table 'test.tnoexist' doesn't exist", + }, + { + binaryExecute: stmtID4, + executeParams: []paramInfo{ + {value: 3}, + }, + originalText: "insert into t2 values(?)", + redactText: "insert into `t2` values ( ? )", + affectedRows: 1, + tables: []stmtctx.TableEntry{ + {DB: "test", Table: "t2"}, + }, }, { sql: "prepare s from 'select * from t1 where a=1 and b>? and b 1", multiQueryCases: []stmtEventCase{ @@ -258,6 +295,19 @@ func TestExtensionStmtEvents(t *testing.T) { }, }, }, + { + binaryExecute: stmtID4, + executeParams: []paramInfo{ + {value: 3}, + }, + err: "insert into t2 values(?) [arguments: 3]: [kv:1062]Duplicate entry '3' for key 't2.PRIMARY'", + originalText: "insert into t2 values(?)", + redactText: "insert into `t2` values ( ? )", + affectedRows: 0, + tables: []stmtctx.TableEntry{ + {DB: "test", Table: "t2"}, + }, + }, } for i, c := range cases { diff --git a/go.mod b/go.mod index 200010fa4ac82..952503212d3ee 100644 --- a/go.mod +++ b/go.mod @@ -86,7 +86,7 @@ require ( github.com/stretchr/testify v1.8.0 github.com/tdakkota/asciicheck v0.1.1 github.com/tiancaiamao/appdash v0.0.0-20181126055449-889f96f722a2 - github.com/tikv/client-go/v2 v2.0.3-0.20221103084528-ec1202a9d26f + github.com/tikv/client-go/v2 v2.0.3-0.20221108030801-9c0835c80eba github.com/tikv/pd/client v0.0.0-20221031025758-80f0d8ca4d07 github.com/timakin/bodyclose v0.0.0-20210704033933-f49887972144 github.com/twmb/murmur3 v1.1.3 diff --git a/go.sum b/go.sum index deccee72b3f3d..eb7dd447f9d5e 100644 --- a/go.sum +++ b/go.sum @@ -926,8 +926,8 @@ github.com/tenntenn/text/transform v0.0.0-20200319021203-7eef512accb3 h1:f+jULpR github.com/tenntenn/text/transform v0.0.0-20200319021203-7eef512accb3/go.mod h1:ON8b8w4BN/kE1EOhwT0o+d62W65a6aPw1nouo9LMgyY= github.com/tiancaiamao/appdash v0.0.0-20181126055449-889f96f722a2 h1:mbAskLJ0oJfDRtkanvQPiooDH8HvJ2FBh+iKT/OmiQQ= github.com/tiancaiamao/appdash v0.0.0-20181126055449-889f96f722a2/go.mod h1:2PfKggNGDuadAa0LElHrByyrz4JPZ9fFx6Gs7nx7ZZU= -github.com/tikv/client-go/v2 v2.0.3-0.20221103084528-ec1202a9d26f h1:x5L32xr5KE2C0UhlKfrsDTWQ/8fpFYQLq5xn+ph0sQ0= -github.com/tikv/client-go/v2 v2.0.3-0.20221103084528-ec1202a9d26f/go.mod h1:X9s4ct/MLk1sFqe5mU79KClKegLFDTa/FCx3hzexGtk= +github.com/tikv/client-go/v2 v2.0.3-0.20221108030801-9c0835c80eba h1:nFVdyTXcQYZwQQCdSJcFI1vBFyzG1hVuZ39MAK6wqK4= +github.com/tikv/client-go/v2 v2.0.3-0.20221108030801-9c0835c80eba/go.mod h1:X9s4ct/MLk1sFqe5mU79KClKegLFDTa/FCx3hzexGtk= github.com/tikv/pd/client v0.0.0-20221031025758-80f0d8ca4d07 h1:ckPpxKcl75mO2N6a4cJXiZH43hvcHPpqc9dh1TmH1nc= github.com/tikv/pd/client v0.0.0-20221031025758-80f0d8ca4d07/go.mod h1:CipBxPfxPUME+BImx9MUYXCnAVLS3VJUr3mnSJwh40A= github.com/timakin/bodyclose v0.0.0-20210704033933-f49887972144 h1:kl4KhGNsJIbDHS9/4U9yQo1UcPQM0kOMJHn29EoH/Ro= diff --git a/infoschema/tables_test.go b/infoschema/tables_test.go index 64b38de1c9db3..3650f2833a1a6 100644 --- a/infoschema/tables_test.go +++ b/infoschema/tables_test.go @@ -1513,6 +1513,10 @@ func TestVariablesInfo(t *testing.T) { tk := testkit.NewTestKit(t, store) + if !variable.EnableConcurrentDDL.Load() { + t.Skip("skip test when concurrent DDL is disabled") + } + tk.MustExec("use information_schema") tk.MustExec("SET GLOBAL innodb_compression_level = 8;") @@ -1543,14 +1547,15 @@ func TestVariablesInfo(t *testing.T) { // See session/bootstrap.go:doDMLWorks() for where the exceptions are defined. stmt := tk.MustQuery(`SELECT variable_name, default_value, current_value FROM information_schema.variables_info WHERE current_value != default_value and default_value != '' ORDER BY variable_name`) stmt.Check(testkit.Rows( - "last_sql_use_alloc OFF ON", // for test stability - "tidb_enable_auto_analyze ON OFF", // always changed for tests - "tidb_enable_collect_execution_info ON OFF", // for test stability - "tidb_enable_mutation_checker OFF ON", // for new installs - "tidb_mem_oom_action CANCEL LOG", // always changed for tests - "tidb_row_format_version 1 2", // for new installs - "tidb_txn_assertion_level OFF FAST", // for new installs - "timestamp 0 123456789", // always dynamic + "last_sql_use_alloc OFF ON", // for test stability + "tidb_enable_auto_analyze ON OFF", // always changed for tests + "tidb_enable_collect_execution_info ON OFF", // for test stability + "tidb_enable_mutation_checker OFF ON", // for new installs + "tidb_enable_plan_replayer_capture OFF false", // for enable plan replayer capture + "tidb_mem_oom_action CANCEL LOG", // always changed for tests + "tidb_row_format_version 1 2", // for new installs + "tidb_txn_assertion_level OFF FAST", // for new installs + "timestamp 0 123456789", // always dynamic )) } diff --git a/planner/core/BUILD.bazel b/planner/core/BUILD.bazel index 711c4af298925..9db73e6d5fcb8 100644 --- a/planner/core/BUILD.bazel +++ b/planner/core/BUILD.bazel @@ -188,6 +188,7 @@ go_test( "plan_cost_detail_test.go", "plan_cost_ver1_test.go", "plan_cost_ver2_test.go", + "plan_replayer_capture_test.go", "plan_stats_test.go", "plan_test.go", "plan_to_pb_test.go", diff --git a/planner/core/cbo_test.go b/planner/core/cbo_test.go index c7e663f43ff1e..340c0e2830162 100644 --- a/planner/core/cbo_test.go +++ b/planner/core/cbo_test.go @@ -264,6 +264,7 @@ func TestEmptyTable(t *testing.T) { store := testkit.CreateMockStore(t) testKit := testkit.NewTestKit(t, store) testKit.MustExec("use test") + testKit.MustExec("set tidb_cost_model_version=2") testKit.MustExec("drop table if exists t, t1") testKit.MustExec("create table t (c1 int)") testKit.MustExec("create table t1 (c1 int)") @@ -434,6 +435,7 @@ func TestCorrelatedEstimation(t *testing.T) { store := testkit.CreateMockStore(t) tk := testkit.NewTestKit(t, store) tk.MustExec("use test") + tk.MustExec("set tidb_cost_model_version=2") tk.MustExec("set sql_mode='STRICT_TRANS_TABLES'") // disable only full group by tk.MustExec("create table t(a int, b int, c int, index idx(c,b,a))") tk.MustExec("insert into t values(1,1,1), (2,2,2), (3,3,3), (4,4,4), (5,5,5), (6,6,6), (7,7,7), (8,8,8), (9,9,9),(10,10,10)") diff --git a/planner/core/collect_column_stats_usage.go b/planner/core/collect_column_stats_usage.go index 49f5cfdfd7011..49fffb149b85d 100644 --- a/planner/core/collect_column_stats_usage.go +++ b/planner/core/collect_column_stats_usage.go @@ -17,6 +17,7 @@ package core import ( "github.com/pingcap/tidb/expression" "github.com/pingcap/tidb/parser/model" + "github.com/pingcap/tidb/sessionctx/variable" ) const ( @@ -43,6 +44,11 @@ type columnStatsUsageCollector struct { histNeededCols map[model.TableItemID]struct{} // cols is used to store columns collected from expressions and saves some allocation. cols []*expression.Column + + // collectVisitedTable indicates whether to collect visited table + collectVisitedTable bool + // visitedtbls indicates the visited table + visitedtbls map[int64]struct{} } func newColumnStatsUsageCollector(collectMode uint64) *columnStatsUsageCollector { @@ -58,6 +64,10 @@ func newColumnStatsUsageCollector(collectMode uint64) *columnStatsUsageCollector if collectMode&collectHistNeededColumns != 0 { collector.histNeededCols = make(map[model.TableItemID]struct{}) } + if variable.EnablePlanReplayerCapture.Load() { + collector.collectVisitedTable = true + collector.visitedtbls = map[int64]struct{}{} + } return collector } @@ -103,6 +113,9 @@ func (c *columnStatsUsageCollector) collectPredicateColumnsForDataSource(ds *Dat // For partition tables, no matter whether it is static or dynamic pruning mode, we use table ID rather than partition ID to // set TableColumnID.TableID. In this way, we keep the set of predicate columns consistent between different partitions and global table. tblID := ds.TableInfo().ID + if c.collectVisitedTable { + c.visitedtbls[tblID] = struct{}{} + } for _, col := range ds.Schema().Columns { tblColID := model.TableItemID{TableID: tblID, ID: col.ID, IsIndex: false} c.colMap[col.UniqueID] = map[model.TableItemID]struct{}{tblColID: {}} @@ -147,6 +160,10 @@ func (c *columnStatsUsageCollector) collectPredicateColumnsForUnionAll(p *Logica } func (c *columnStatsUsageCollector) addHistNeededColumns(ds *DataSource) { + if c.collectVisitedTable { + tblID := ds.TableInfo().ID + c.visitedtbls[tblID] = struct{}{} + } columns := expression.ExtractColumnsFromExpressions(c.cols[:0], ds.pushedDownConds, nil) for _, col := range columns { tblColID := model.TableItemID{TableID: ds.physicalTableID, ID: col.ID, IsIndex: false} @@ -285,6 +302,9 @@ func CollectColumnStatsUsage(lp LogicalPlan, predicate, histNeeded bool) ([]mode } collector := newColumnStatsUsageCollector(mode) collector.collectFromPlan(lp) + if collector.collectVisitedTable { + recordTableRuntimeStats(lp.SCtx(), collector.visitedtbls) + } set2slice := func(set map[model.TableItemID]struct{}) []model.TableItemID { ret := make([]model.TableItemID, 0, len(set)) for tblColID := range set { diff --git a/planner/core/expression_rewriter_test.go b/planner/core/expression_rewriter_test.go index 3de3509c3e093..2e4f9e036b830 100644 --- a/planner/core/expression_rewriter_test.go +++ b/planner/core/expression_rewriter_test.go @@ -381,6 +381,7 @@ func TestMultiColInExpression(t *testing.T) { store := testkit.CreateMockStore(t) tk := testkit.NewTestKit(t, store) tk.MustExec("use test;") + tk.MustExec("set tidb_cost_model_version=2") tk.MustExec("drop table if exists t1, t2") tk.MustExec("create table t1(a int, b int)") tk.MustExec("insert into t1 values(1,1),(2,null),(null,3),(4,4)") diff --git a/planner/core/integration_test.go b/planner/core/integration_test.go index a89b85b736d62..9f842f2b3c85b 100644 --- a/planner/core/integration_test.go +++ b/planner/core/integration_test.go @@ -615,6 +615,7 @@ func TestMPPJoin(t *testing.T) { store := testkit.CreateMockStore(t) tk := testkit.NewTestKit(t, store) tk.MustExec("use test") + tk.MustExec("set tidb_cost_model_version=2") tk.MustExec("drop table if exists d1_t") tk.MustExec("create table d1_t(d1_k int, value int)") tk.MustExec("insert into d1_t values(1,2),(2,3)") @@ -1320,6 +1321,7 @@ func TestReadFromStorageHintAndIsolationRead(t *testing.T) { tk := testkit.NewTestKit(t, store) tk.MustExec("use test") + tk.MustExec("set tidb_cost_model_version=2") tk.MustExec("drop table if exists t, tt, ttt") tk.MustExec("create table t(a int, b int, index ia(a))") tk.MustExec("set @@session.tidb_isolation_read_engines=\"tikv\"") @@ -5697,6 +5699,7 @@ func TestOutputSkylinePruningInfo(t *testing.T) { store := testkit.CreateMockStore(t) tk := testkit.NewTestKit(t, store) tk.MustExec("use test") + tk.MustExec("set tidb_cost_model_version=2") tk.MustExec("drop table if exists t") tk.MustExec("create table t(a int, b int, c int, d int, e int, f int, g int, primary key (a), unique key c_d_e (c, d, e), unique key f (f), unique key f_g (f, g), key g (g))") @@ -5726,6 +5729,7 @@ func TestPreferRangeScanForUnsignedIntHandle(t *testing.T) { store := testkit.CreateMockStore(t) tk := testkit.NewTestKit(t, store) tk.MustExec("use test") + tk.MustExec("set tidb_cost_model_version=2") tk.MustExec("drop table if exists t") tk.MustExec("create table t(a int unsigned primary key, b int, c int, index idx_b(b))") tk.MustExec("insert into t values (1,2,3), (4,5,6), (7,8,9), (10,11,12), (13,14,15)") @@ -6083,6 +6087,7 @@ func TestRejectSortForMPP(t *testing.T) { store := testkit.CreateMockStore(t) tk := testkit.NewTestKit(t, store) tk.MustExec("use test") + tk.MustExec("set tidb_cost_model_version=2") tk.MustExec("drop table if exists t") tk.MustExec("create table t (id int, value decimal(6,3), name char(128))") tk.MustExec("analyze table t") @@ -6709,6 +6714,7 @@ func TestTiFlashFineGrainedShuffle(t *testing.T) { store, dom := testkit.CreateMockStoreAndDomain(t) tk := testkit.NewTestKit(t, store) tk.MustExec("use test") + tk.MustExec("set tidb_cost_model_version=2") tk.MustExec("set @@tidb_isolation_read_engines = 'tiflash'") tk.MustExec("set @@tidb_enforce_mpp = on") tk.MustExec("drop table if exists t1;") @@ -7086,6 +7092,7 @@ func TestAggWithJsonPushDownToTiFlash(t *testing.T) { store := testkit.CreateMockStore(t) tk := testkit.NewTestKit(t, store) tk.MustExec("use test") + tk.MustExec("set tidb_cost_model_version=2") tk.MustExec("drop table if exists t") tk.MustExec("create table t(a json);") tk.MustExec("insert into t values(null);") @@ -7107,26 +7114,26 @@ func TestAggWithJsonPushDownToTiFlash(t *testing.T) { } rows := [][]interface{}{ - {"HashAgg_6", "root", "funcs:avg(Column#4)->Column#3"}, + {"HashAgg_8", "root", "funcs:avg(Column#4)->Column#3"}, {"└─Projection_19", "root", "cast(test.t.a, double BINARY)->Column#4"}, - {" └─TableReader_12", "root", "data:TableFullScan_11"}, - {" └─TableFullScan_11", "cop[tiflash]", "keep order:false, stats:pseudo"}, + {" └─TableReader_14", "root", "data:TableFullScan_13"}, + {" └─TableFullScan_13", "cop[tiflash]", "keep order:false, stats:pseudo"}, } tk.MustQuery("explain select avg(a) from t;").CheckAt([]int{0, 2, 4}, rows) rows = [][]interface{}{ - {"HashAgg_6", "root", "funcs:sum(Column#4)->Column#3"}, + {"HashAgg_8", "root", "funcs:sum(Column#4)->Column#3"}, {"└─Projection_19", "root", "cast(test.t.a, double BINARY)->Column#4"}, - {" └─TableReader_12", "root", "data:TableFullScan_11"}, - {" └─TableFullScan_11", "cop[tiflash]", "keep order:false, stats:pseudo"}, + {" └─TableReader_14", "root", "data:TableFullScan_13"}, + {" └─TableFullScan_13", "cop[tiflash]", "keep order:false, stats:pseudo"}, } tk.MustQuery("explain select sum(a) from t;").CheckAt([]int{0, 2, 4}, rows) rows = [][]interface{}{ - {"HashAgg_6", "root", "funcs:group_concat(Column#4 separator \",\")->Column#3"}, + {"HashAgg_8", "root", "funcs:group_concat(Column#4 separator \",\")->Column#3"}, {"└─Projection_13", "root", "cast(test.t.a, var_string(4294967295))->Column#4"}, - {" └─TableReader_10", "root", "data:TableFullScan_9"}, - {" └─TableFullScan_9", "cop[tiflash]", "keep order:false, stats:pseudo"}, + {" └─TableReader_12", "root", "data:TableFullScan_11"}, + {" └─TableFullScan_11", "cop[tiflash]", "keep order:false, stats:pseudo"}, } tk.MustQuery("explain select /*+ hash_agg() */ group_concat(a) from t;").CheckAt([]int{0, 2, 4}, rows) } diff --git a/planner/core/physical_plan_test.go b/planner/core/physical_plan_test.go index c78ef7e8fd27d..36676cb9b1fdc 100644 --- a/planner/core/physical_plan_test.go +++ b/planner/core/physical_plan_test.go @@ -54,6 +54,7 @@ func TestDAGPlanBuilderSimpleCase(t *testing.T) { tk := testkit.NewTestKit(t, store) tk.MustExec("use test") + tk.MustExec("set tidb_cost_model_version=2") tk.MustExec("set tidb_opt_limit_push_down_threshold=0") var input []string var output []struct { @@ -406,6 +407,7 @@ func TestDAGPlanBuilderAgg(t *testing.T) { store := testkit.CreateMockStore(t) tk := testkit.NewTestKit(t, store) tk.MustExec("use test") + tk.MustExec("set tidb_cost_model_version=2") tk.MustExec("set sql_mode='STRICT_TRANS_TABLES'") // disable only full group by sessionVars := tk.Session().GetSessionVars() sessionVars.SetHashAggFinalConcurrency(1) @@ -442,6 +444,7 @@ func TestRefine(t *testing.T) { tk := testkit.NewTestKit(t, store) tk.MustExec("use test") + tk.MustExec("set tidb_cost_model_version=2") var input []string var output []struct { @@ -599,6 +602,7 @@ func TestIndexJoinUnionScan(t *testing.T) { tk := testkit.NewTestKit(t, store) tk.MustExec("use test") + tk.MustExec("set tidb_cost_model_version=2") tk.MustExec("create table t (a int primary key, b int, index idx(a))") tk.MustExec("create table tt (a int primary key) partition by range (a) (partition p0 values less than (100), partition p1 values less than (200))") tk.MustExec(`set @@tidb_partition_prune_mode='` + string(variable.Static) + `'`) @@ -720,6 +724,7 @@ func TestSemiJoinToInner(t *testing.T) { store := testkit.CreateMockStore(t) tk := testkit.NewTestKit(t, store) tk.MustExec("use test") + tk.MustExec("set tidb_cost_model_version=2") var input []string var output []struct { @@ -976,6 +981,7 @@ func TestSemiJoinRewriteHints(t *testing.T) { store := testkit.CreateMockStore(t) tk := testkit.NewTestKit(t, store) tk.MustExec("use test") + tk.MustExec("set tidb_cost_model_version=2") tk.MustExec("create table t(a int, b int, c int)") sessionVars := tk.Session().GetSessionVars() @@ -1218,6 +1224,7 @@ func TestForceInlineCTE(t *testing.T) { store := testkit.CreateMockStore(t) tk := testkit.NewTestKit(t, store) tk.MustExec("use test") + tk.MustExec("set tidb_cost_model_version=2") tk.MustExec("drop table if exists t;") tk.MustExec("CREATE TABLE `t` (`a` int(11));") tk.MustExec("insert into t values (1), (5), (10), (15), (20), (30), (50);") @@ -1606,6 +1613,7 @@ func TestQueryBlockHint(t *testing.T) { tk := testkit.NewTestKit(t, store) tk.MustExec("use test") + tk.MustExec("set tidb_cost_model_version=2") var input []string var output []struct { @@ -1761,6 +1769,7 @@ func TestIndexJoinHint(t *testing.T) { tk := testkit.NewTestKit(t, store) tk.MustExec("use test") + tk.MustExec("set tidb_cost_model_version=2") tk.MustExec(`drop table if exists test.t1, test.t2, test.t;`) tk.MustExec(`create table test.t1(a bigint, b bigint, index idx_a(a), index idx_b(b));`) tk.MustExec(`create table test.t2(a bigint, b bigint, index idx_a(a), index idx_b(b));`) @@ -2111,6 +2120,7 @@ func TestSkewDistinctAgg(t *testing.T) { store := testkit.CreateMockStore(t) tk := testkit.NewTestKit(t, store) tk.MustExec("use test") + tk.MustExec("set tidb_cost_model_version=2") tk.MustExec("drop table if exists t") tk.MustExec("CREATE TABLE `t` (`a` int(11), `b` int(11), `c` int(11), `d` date)") tk.MustExec("insert into t (a,b,c,d) value(1,4,5,'2019-06-01')") @@ -2163,6 +2173,7 @@ func TestHJBuildAndProbeHint(t *testing.T) { store := testkit.CreateMockStore(t) tk := testkit.NewTestKit(t, store) tk.MustExec("use test") + tk.MustExec("set tidb_cost_model_version=2") tk.MustExec("drop table if exists t1, t2, t3") tk.MustExec("create table t1(a int primary key, b int not null)") tk.MustExec("create table t2(a int primary key, b int not null)") @@ -2199,6 +2210,7 @@ func TestHJBuildAndProbeHint4StaticPartitionTable(t *testing.T) { store := testkit.CreateMockStore(t) tk := testkit.NewTestKit(t, store) tk.MustExec("use test") + tk.MustExec("set tidb_cost_model_version=2") tk.MustExec("drop table if exists t1, t2, t3") tk.MustExec(`create table t1(a int, b int) partition by hash(a) partitions 4`) tk.MustExec(`create table t2(a int, b int) partition by hash(a) partitions 5`) @@ -2273,6 +2285,7 @@ func TestHJBuildAndProbeHint4TiFlash(t *testing.T) { store := testkit.CreateMockStore(t) tk := testkit.NewTestKit(t, store) tk.MustExec("use test") + tk.MustExec("set tidb_cost_model_version=2") tk.MustExec("drop table if exists t1, t2, t3") tk.MustExec("create table t1(a int primary key, b int not null)") tk.MustExec("create table t2(a int primary key, b int not null)") @@ -2311,6 +2324,7 @@ func TestHJBuildAndProbeHintWithBinding(t *testing.T) { tk := testkit.NewTestKit(t, store) tk.MustExec("use test") + tk.MustExec("set tidb_cost_model_version=2") tk.MustExec("drop table if exists t, t1, t2, t3;") tk.MustExec("create table t(a int, b int, key(a));") tk.MustExec("create table t1(a int, b int, key(a));") diff --git a/planner/core/plan_cost_ver1.go b/planner/core/plan_cost_ver1.go index 0cd8c5eabd473..d1a7e6e95a9c5 100644 --- a/planner/core/plan_cost_ver1.go +++ b/planner/core/plan_cost_ver1.go @@ -852,7 +852,7 @@ func (p *PhysicalHashJoin) GetCost(lCnt, rCnt float64, isMPP bool, costFlag uint sessVars := p.ctx.GetSessionVars() oomUseTmpStorage := variable.EnableTmpStorageOnOOM.Load() memQuota := sessVars.MemTracker.GetBytesLimit() // sessVars.MemQuotaQuery && hint - rowSize := getAvgRowSize(build.statsInfo(), build.Schema()) + rowSize := getAvgRowSize(build.statsInfo(), build.Schema().Columns) spill := oomUseTmpStorage && memQuota > 0 && rowSize*buildCnt > float64(memQuota) && p.storeTp != kv.TiFlash // Cost of building hash table. cpuFactor := sessVars.GetCPUFactor() @@ -1049,7 +1049,7 @@ func (p *PhysicalSort) GetCost(count float64, schema *expression.Schema) float64 oomUseTmpStorage := variable.EnableTmpStorageOnOOM.Load() memQuota := sessVars.MemTracker.GetBytesLimit() // sessVars.MemQuotaQuery && hint - rowSize := getAvgRowSize(p.statsInfo(), schema) + rowSize := getAvgRowSize(p.statsInfo(), schema.Columns) spill := oomUseTmpStorage && memQuota > 0 && rowSize*count > float64(memQuota) diskCost := count * sessVars.GetDiskFactor() * rowSize if !spill { diff --git a/planner/core/plan_cost_ver2.go b/planner/core/plan_cost_ver2.go index 0242777ae6ef5..0f4c936fa4e46 100644 --- a/planner/core/plan_cost_ver2.go +++ b/planner/core/plan_cost_ver2.go @@ -115,7 +115,7 @@ func (p *PhysicalIndexScan) getPlanCostVer2(taskType property.TaskType, option * } rows := getCardinality(p, option.CostFlag) - rowSize := math.Max(p.getScanRowSize(), 2.0) + rowSize := math.Max(getAvgRowSize(p.stats, p.schema.Columns), 2.0) // consider all index columns scanFactor := getTaskScanFactorVer2(p, taskType) p.planCostVer2 = scanCostVer2(option, rows, rowSize, scanFactor) @@ -132,7 +132,13 @@ func (p *PhysicalTableScan) getPlanCostVer2(taskType property.TaskType, option * } rows := getCardinality(p, option.CostFlag) - rowSize := math.Max(p.getScanRowSize(), 2.0) + var rowSize float64 + if p.StoreType == kv.TiKV { + rowSize = getAvgRowSize(p.stats, p.tblCols) // consider all columns if TiKV + } else { // TiFlash + rowSize = getAvgRowSize(p.stats, p.schema.Columns) + } + rowSize = math.Max(rowSize, 2.0) scanFactor := getTaskScanFactorVer2(p, taskType) p.planCostVer2 = scanCostVer2(option, rows, rowSize, scanFactor) @@ -155,7 +161,7 @@ func (p *PhysicalIndexReader) getPlanCostVer2(taskType property.TaskType, option } rows := getCardinality(p.indexPlan, option.CostFlag) - rowSize := getAvgRowSize(p.indexPlan.Stats(), p.indexPlan.Schema()) + rowSize := getAvgRowSize(p.stats, p.schema.Columns) netFactor := getTaskNetFactorVer2(p, taskType) concurrency := float64(p.ctx.GetSessionVars().DistSQLScanConcurrency()) @@ -180,7 +186,7 @@ func (p *PhysicalTableReader) getPlanCostVer2(taskType property.TaskType, option } rows := getCardinality(p.tablePlan, option.CostFlag) - rowSize := getAvgRowSize(p.tablePlan.Stats(), p.tablePlan.Schema()) + rowSize := getAvgRowSize(p.stats, p.schema.Columns) netFactor := getTaskNetFactorVer2(p, taskType) concurrency := float64(p.ctx.GetSessionVars().DistSQLScanConcurrency()) childType := property.CopSingleReadTaskType @@ -246,13 +252,14 @@ func (p *PhysicalIndexLookUpReader) getPlanCostVer2(taskType property.TaskType, } tableSideCost := divCostVer2(sumCostVer2(tableNetCost, tableChildCost), distConcurrency) - // double-read + // double-read: assume at least 1 row to double-read to avoid 0 double-read cost. + doubleReadRows := math.Max(indexRows, 1) doubleReadCPUCost := newCostVer2(option, cpuFactor, indexRows*cpuFactor.Value, - "double-read-cpu(%v*%v)", indexRows, cpuFactor) + "double-read-cpu(%v*%v)", doubleReadRows, cpuFactor) batchSize := float64(p.ctx.GetSessionVars().IndexLookupSize) taskPerBatch := 32.0 // TODO: remove this magic number - doubleReadTasks := indexRows / batchSize * taskPerBatch + doubleReadTasks := doubleReadRows / batchSize * taskPerBatch doubleReadRequestCost := doubleReadCostVer2(option, doubleReadTasks, requestFactor) doubleReadCost := sumCostVer2(doubleReadCPUCost, doubleReadRequestCost) @@ -283,7 +290,7 @@ func (p *PhysicalIndexMergeReader) getPlanCostVer2(taskType property.TaskType, o var tableSideCost costVer2 if tablePath := p.tablePlan; tablePath != nil { rows := getCardinality(tablePath, option.CostFlag) - rowSize := getAvgRowSize(tablePath.Stats(), tablePath.Schema()) + rowSize := getAvgRowSize(tablePath.Stats(), tablePath.Schema().Columns) tableNetCost := netCostVer2(option, rows, rowSize, netFactor) tableChildCost, err := tablePath.getPlanCostVer2(taskType, option) @@ -296,7 +303,7 @@ func (p *PhysicalIndexMergeReader) getPlanCostVer2(taskType property.TaskType, o indexSideCost := make([]costVer2, 0, len(p.partialPlans)) for _, indexPath := range p.partialPlans { rows := getCardinality(indexPath, option.CostFlag) - rowSize := getAvgRowSize(indexPath.Stats(), indexPath.Schema()) + rowSize := getAvgRowSize(indexPath.Stats(), indexPath.Schema().Columns) indexNetCost := netCostVer2(option, rows, rowSize, netFactor) indexChildCost, err := indexPath.getPlanCostVer2(taskType, option) @@ -328,7 +335,7 @@ func (p *PhysicalSort) getPlanCostVer2(taskType property.TaskType, option *PlanC } rows := math.Max(getCardinality(p.children[0], option.CostFlag), 1) - rowSize := getAvgRowSize(p.statsInfo(), p.Schema()) + rowSize := getAvgRowSize(p.statsInfo(), p.Schema().Columns) cpuFactor := getTaskCPUFactorVer2(p, taskType) memFactor := getTaskMemFactorVer2(p, taskType) diskFactor := defaultVer2Factors.TiDBDisk @@ -377,7 +384,7 @@ func (p *PhysicalTopN) getPlanCostVer2(taskType property.TaskType, option *PlanC rows := getCardinality(p.children[0], option.CostFlag) N := math.Max(1, float64(p.Count+p.Offset)) - rowSize := getAvgRowSize(p.statsInfo(), p.Schema()) + rowSize := getAvgRowSize(p.statsInfo(), p.Schema().Columns) cpuFactor := getTaskCPUFactorVer2(p, taskType) memFactor := getTaskMemFactorVer2(p, taskType) @@ -428,7 +435,7 @@ func (p *PhysicalHashAgg) getPlanCostVer2(taskType property.TaskType, option *Pl inputRows := getCardinality(p.children[0], option.CostFlag) outputRows := getCardinality(p, option.CostFlag) - outputRowSize := getAvgRowSize(p.Stats(), p.Schema()) + outputRowSize := getAvgRowSize(p.Stats(), p.Schema().Columns) cpuFactor := getTaskCPUFactorVer2(p, taskType) memFactor := getTaskMemFactorVer2(p, taskType) concurrency := float64(p.ctx.GetSessionVars().HashAggFinalConcurrency()) @@ -500,7 +507,7 @@ func (p *PhysicalHashJoin) getPlanCostVer2(taskType property.TaskType, option *P } buildRows := getCardinality(build, option.CostFlag) probeRows := getCardinality(probe, option.CostFlag) - buildRowSize := getAvgRowSize(build.Stats(), build.Schema()) + buildRowSize := getAvgRowSize(build.Stats(), build.Schema().Columns) tidbConcurrency := float64(p.Concurrency) mppConcurrency := float64(3) // TODO: remove this empirical value cpuFactor := getTaskCPUFactorVer2(p, taskType) @@ -644,7 +651,7 @@ func (p *PhysicalExchangeReceiver) getPlanCostVer2(taskType property.TaskType, o } rows := getCardinality(p, option.CostFlag) - rowSize := getAvgRowSize(p.stats, p.Schema()) + rowSize := getAvgRowSize(p.stats, p.Schema().Columns) netFactor := getTaskNetFactorVer2(p, taskType) isBCast := false if sender, ok := p.children[0].(*PhysicalExchangeSender); ok { @@ -677,7 +684,7 @@ func (p *PointGetPlan) getPlanCostVer2(taskType property.TaskType, option *PlanC p.planCostInit = true return zeroCostVer2, nil } - rowSize := getAvgRowSize(p.stats, p.schema) + rowSize := getAvgRowSize(p.stats, p.schema.Columns) netFactor := getTaskNetFactorVer2(p, taskType) p.planCostVer2 = netCostVer2(option, 1, rowSize, netFactor) @@ -697,7 +704,7 @@ func (p *BatchPointGetPlan) getPlanCostVer2(taskType property.TaskType, option * return zeroCostVer2, nil } rows := getCardinality(p, option.CostFlag) - rowSize := getAvgRowSize(p.stats, p.schema) + rowSize := getAvgRowSize(p.stats, p.schema.Columns) netFactor := getTaskNetFactorVer2(p, taskType) p.planCostVer2 = netCostVer2(option, rows, rowSize, netFactor) diff --git a/planner/core/plan_cost_ver2_test.go b/planner/core/plan_cost_ver2_test.go index 0043aa0be898e..27b5b913980e4 100644 --- a/planner/core/plan_cost_ver2_test.go +++ b/planner/core/plan_cost_ver2_test.go @@ -142,15 +142,55 @@ func TestCostModelShowFormula(t *testing.T) { actual := make([][]interface{}, 0, len(plan)) for _, row := range plan { actual = append(actual, []interface{}{row[0], row[3]}) // id,costFormula - fmt.Println(actual) } require.Equal(t, actual, [][]interface{}{ {"TableReader_7", "((Selection_6) + (net(2*rowsize(16)*tidb_kv_net_factor(3.96))))/15"}, {"└─Selection_6", "(cpu(3*filters(1)*tikv_cpu_factor(49.9))) + (TableFullScan_5)"}, - {" └─TableFullScan_5", "scan(3*logrowsize(29)*tikv_scan_factor(40.7))"}, + {" └─TableFullScan_5", "scan(3*logrowsize(32)*tikv_scan_factor(40.7))"}, }) } +func TestCostModelVer2ScanRowSize(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec(`create table t (pk int, a int, b int, c int, d int, primary key(pk), index ab(a, b), index abc(a, b, c))`) + tk.MustExec("insert into t values (1, 1, 1, 1, 1)") + tk.MustExec(`set @@tidb_cost_model_version=2`) + + cases := []struct { + query string + scanFormula string + }{ + // index scan row-size on idx_ab is always equal to row-size(index_ab) + {"select a from t use index(ab) where a=1", "scan(1*logrowsize(32)*tikv_scan_factor(40.7))"}, + {"select a, b from t use index(ab) where a=1", "scan(1*logrowsize(32)*tikv_scan_factor(40.7))"}, + {"select b from t use index(ab) where a=1 and b=1", "scan(1*logrowsize(32)*tikv_scan_factor(40.7))"}, + // index scan row-size on idx_abc is always equal to row-size(index_abc) + {"select a from t use index(abc) where a=1", "scan(1*logrowsize(48)*tikv_scan_factor(40.7))"}, + {"select a from t use index(abc) where a=1 and b=1", "scan(1*logrowsize(48)*tikv_scan_factor(40.7))"}, + {"select a, b from t use index(abc) where a=1 and b=1", "scan(1*logrowsize(48)*tikv_scan_factor(40.7))"}, + {"select a, b, c from t use index(abc) where a=1 and b=1 and c=1", "scan(1*logrowsize(48)*tikv_scan_factor(40.7))"}, + // table scan row-size is always equal to row-size(*) + {"select a from t use index(primary) where a=1", "scan(1*logrowsize(80)*tikv_scan_factor(40.7))"}, + {"select a, d from t use index(primary) where a=1", "scan(1*logrowsize(80)*tikv_scan_factor(40.7))"}, + {"select * from t use index(primary) where a=1", "scan(1*logrowsize(80)*tikv_scan_factor(40.7))"}, + } + for _, c := range cases { + rs := tk.MustQuery("explain analyze format=true_card_cost " + c.query).Rows() + scan := rs[len(rs)-1] + formula := scan[3] + require.Equal(t, formula, c.scanFormula) + } + + tk.MustQuery("explain select a from t where a=1").Check(testkit.Rows( + `IndexReader_6 10.00 root index:IndexRangeScan_5`, // use idx_ab automatically since it has the smallest row-size in all access paths. + `└─IndexRangeScan_5 10.00 cop[tikv] table:t, index:ab(a, b) range:[1,1], keep order:false, stats:pseudo`)) + tk.MustQuery("explain select a, b, c from t where a=1").Check(testkit.Rows( + `IndexReader_6 10.00 root index:IndexRangeScan_5`, // use idx_abc automatically + `└─IndexRangeScan_5 10.00 cop[tikv] table:t, index:abc(a, b, c) range:[1,1], keep order:false, stats:pseudo`)) +} + func TestCostModelTraceVer2(t *testing.T) { store := testkit.CreateMockStore(t) tk := testkit.NewTestKit(t, store) diff --git a/planner/core/plan_replayer_capture_test.go b/planner/core/plan_replayer_capture_test.go new file mode 100644 index 0000000000000..2e88f090bd784 --- /dev/null +++ b/planner/core/plan_replayer_capture_test.go @@ -0,0 +1,83 @@ +// Copyright 2022 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 core_test + +import ( + "context" + "testing" + + "github.com/pingcap/tidb/domain" + "github.com/pingcap/tidb/parser" + "github.com/pingcap/tidb/planner/core" + "github.com/pingcap/tidb/sessionctx" + "github.com/pingcap/tidb/statistics" + "github.com/pingcap/tidb/testkit" + "github.com/pingcap/tidb/util/hint" + "github.com/stretchr/testify/require" +) + +func TestPlanReplayerCaptureRecordJsonStats(t *testing.T) { + store, dom := testkit.CreateMockStoreAndDomain(t) + tk := testkit.NewTestKit(t, store) + ctx := tk.Session().(sessionctx.Context) + tk.MustExec("use test") + tk.MustExec("create table t1(a int)") + tk.MustExec("create table t2(a int)") + tk.MustExec("SET global tidb_enable_plan_replayer_capture = ON;") + tk.MustExec("analyze table t1") + tk.MustExec("analyze table t2") + testcases := []struct { + sql string + count int + }{ + { + sql: "select * from t1", + count: 1, + }, + { + sql: "select * from t2", + count: 1, + }, + { + sql: "select * from t1,t2", + count: 2, + }, + } + for _, tc := range testcases { + tableStats := getTableStats(tc.sql, t, ctx, dom) + require.Equal(t, tc.count, len(tableStats)) + } +} + +func getTableStats(sql string, t *testing.T, ctx sessionctx.Context, dom *domain.Domain) map[int64]*statistics.Table { + p := parser.New() + stmt, err := p.ParseOneStmt(sql, "", "") + require.NoError(t, err) + err = core.Preprocess(context.Background(), ctx, stmt, core.WithPreprocessorReturn(&core.PreprocessorReturn{InfoSchema: dom.InfoSchema()})) + require.NoError(t, err) + sctx := core.MockContext() + builder, _ := core.NewPlanBuilder().Init(sctx, dom.InfoSchema(), &hint.BlockHintProcessor{}) + domain.GetDomain(sctx).MockInfoCacheAndLoadInfoSchema(dom.InfoSchema()) + plan, err := builder.Build(context.TODO(), stmt) + require.NoError(t, err) + _, _, err = core.DoOptimize(context.TODO(), sctx, builder.GetOptFlag(), plan.(core.LogicalPlan)) + require.NoError(t, err) + tableStats := sctx.GetSessionVars().StmtCtx.TableStats + r := make(map[int64]*statistics.Table) + for key, v := range tableStats { + r[key] = v.(*statistics.Table) + } + return r +} diff --git a/planner/core/plan_stats.go b/planner/core/plan_stats.go index 681b5c1e8a78e..9bea8edb8b0ba 100644 --- a/planner/core/plan_stats.go +++ b/planner/core/plan_stats.go @@ -25,6 +25,7 @@ import ( "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/sessionctx/variable" + "github.com/pingcap/tidb/statistics" "github.com/pingcap/tidb/util/logutil" "github.com/pingcap/tidb/util/mathutil" "go.uber.org/zap" @@ -154,7 +155,7 @@ func collectSyncIndices(ctx sessionctx.Context, histNeededColumns []model.TableI continue } idxStats, ok := tblStats.Indices[idx.Meta().ID] - if !ok || idxStats == nil || !idxStats.IsFullLoad() { + if ok && idxStats.IsStatsInitialized() && !idxStats.IsFullLoad() { histNeededIndices[model.TableItemID{TableID: column.TableID, ID: idxID, IsIndex: true}] = struct{}{} } } @@ -170,3 +171,34 @@ func collectHistNeededItems(histNeededColumns []model.TableItemID, histNeededInd histNeededItems = append(histNeededItems, histNeededColumns...) return } + +func recordTableRuntimeStats(sctx sessionctx.Context, tbls map[int64]struct{}) { + tblStats := sctx.GetSessionVars().StmtCtx.TableStats + if tblStats == nil { + tblStats = map[int64]interface{}{} + } + for tblID := range tbls { + tblJSONStats, err := recordSingleTableRuntimeStats(sctx, tblID) + if err != nil { + logutil.BgLogger().Warn("record table json stats failed", zap.Int64("tblID", tblID), zap.Error(err)) + } + if tblJSONStats == nil { + logutil.BgLogger().Warn("record table json stats failed due to empty", zap.Int64("tblID", tblID)) + } + tblStats[tblID] = tblJSONStats + } + sctx.GetSessionVars().StmtCtx.TableStats = tblStats +} + +func recordSingleTableRuntimeStats(sctx sessionctx.Context, tblID int64) (*statistics.Table, error) { + dom := domain.GetDomain(sctx) + is := dom.InfoSchema() + statsHandle := dom.StatsHandle() + tbl, ok := is.TableByID(tblID) + if !ok { + return nil, nil + } + tableInfo := tbl.Meta() + stats := statsHandle.GetTableStats(tableInfo) + return stats, nil +} diff --git a/planner/core/planbuilder.go b/planner/core/planbuilder.go index a07f3c13d6717..6815088d3f0af 100644 --- a/planner/core/planbuilder.go +++ b/planner/core/planbuilder.go @@ -633,7 +633,7 @@ func (b *PlanBuilder) GetIsForUpdateRead() bool { } // GetDBTableInfo gets the accessed dbs and tables info. -func (b *PlanBuilder) GetDBTableInfo() []stmtctx.TableEntry { +func GetDBTableInfo(visitInfo []visitInfo) []stmtctx.TableEntry { var tables []stmtctx.TableEntry existsFunc := func(tbls []stmtctx.TableEntry, tbl *stmtctx.TableEntry) bool { for _, t := range tbls { @@ -643,7 +643,7 @@ func (b *PlanBuilder) GetDBTableInfo() []stmtctx.TableEntry { } return false } - for _, v := range b.visitInfo { + for _, v := range visitInfo { tbl := &stmtctx.TableEntry{DB: v.db, Table: v.table} if !existsFunc(tables, tbl) { tables = append(tables, *tbl) diff --git a/planner/core/rule_join_reorder_test.go b/planner/core/rule_join_reorder_test.go index 609b2dde4499a..d4d35f5de80a4 100644 --- a/planner/core/rule_join_reorder_test.go +++ b/planner/core/rule_join_reorder_test.go @@ -261,6 +261,7 @@ func TestJoinOrderHint4DifferentJoinType(t *testing.T) { tk := testkit.NewTestKit(t, store) tk.MustExec("use test") + tk.MustExec("set tidb_cost_model_version=2") tk.MustExec("drop table if exists t, t1, t2, t3, t4, t5, t6, t7, t8;") tk.MustExec("create table t(a int, b int, key(a));") tk.MustExec("create table t1(a int, b int, key(a));") diff --git a/planner/core/rule_result_reorder_test.go b/planner/core/rule_result_reorder_test.go index 060b44da245aa..09e0f1da35783 100644 --- a/planner/core/rule_result_reorder_test.go +++ b/planner/core/rule_result_reorder_test.go @@ -130,6 +130,7 @@ func TestOrderedResultModeOnSubQuery(t *testing.T) { tk := testkit.NewTestKit(t, store) tk.MustExec("use test") + tk.MustExec("set tidb_cost_model_version=2") tk.MustExec("set tidb_enable_ordered_result_mode=1") tk.MustExec("drop table if exists t1") tk.MustExec("drop table if exists t2") @@ -157,6 +158,7 @@ func TestOrderedResultModeOnOtherOperators(t *testing.T) { tk := testkit.NewTestKit(t, store) tk.MustExec("use test") + tk.MustExec("set tidb_cost_model_version=2") tk.MustExec("set tidb_enable_ordered_result_mode=1") tk.MustExec("drop table if exists t1") tk.MustExec("drop table if exists t2") diff --git a/planner/core/stats_test.go b/planner/core/stats_test.go index d9f4b5a015b38..2948c90abb4df 100644 --- a/planner/core/stats_test.go +++ b/planner/core/stats_test.go @@ -123,6 +123,7 @@ func TestNDVGroupCols(t *testing.T) { store := testkit.CreateMockStore(t) tk := testkit.NewTestKit(t, store) tk.MustExec("use test") + tk.MustExec("set tidb_cost_model_version=2") tk.MustExec("drop table if exists t1, t2") tk.MustExec("create table t1(a int not null, b int not null, key(a,b))") tk.MustExec("insert into t1 values(1,1),(1,2),(2,1),(2,2)") diff --git a/planner/core/task.go b/planner/core/task.go index 44295ca6b4def..11d502568d3ae 100644 --- a/planner/core/task.go +++ b/planner/core/task.go @@ -297,12 +297,11 @@ func (p *PhysicalIndexJoin) attach2Task(tasks ...task) task { return t } -func getAvgRowSize(stats *property.StatsInfo, schema *expression.Schema) (size float64) { +func getAvgRowSize(stats *property.StatsInfo, cols []*expression.Column) (size float64) { if stats.HistColl != nil { - size = stats.HistColl.GetAvgRowSizeListInDisk(schema.Columns) + size = stats.HistColl.GetAvgRowSizeListInDisk(cols) } else { // Estimate using just the type info. - cols := schema.Columns for _, col := range cols { size += float64(chunk.EstimateTypeWidth(col.GetType())) } diff --git a/planner/core/testdata/analyze_suite_out.json b/planner/core/testdata/analyze_suite_out.json index d9345a1a5bcce..38fc33be76916 100644 --- a/planner/core/testdata/analyze_suite_out.json +++ b/planner/core/testdata/analyze_suite_out.json @@ -380,22 +380,22 @@ [ "Projection 10.00 root Column#22", "└─Apply 10.00 root CARTESIAN left outer semi join, other cond:eq(test.t.c, Column#21)", - " ├─TableReader(Build) 10.00 root data:TableFullScan", - " │ └─TableFullScan 10.00 cop[tikv] table:t keep order:false", + " ├─IndexReader(Build) 10.00 root index:IndexFullScan", + " │ └─IndexFullScan 10.00 cop[tikv] table:t, index:idx(c, b, a) keep order:false", " └─StreamAgg(Probe) 10.00 root funcs:count(1)->Column#21", " └─HashJoin 10.00 root inner join, equal:[eq(test.t.a, test.t.a)]", - " ├─TableReader(Build) 10.00 root data:Selection", + " ├─IndexReader(Build) 10.00 root index:Selection", " │ └─Selection 10.00 cop[tikv] eq(test.t.a, test.t.a), not(isnull(test.t.a))", - " │ └─TableFullScan 100.00 cop[tikv] table:t1 keep order:false", - " └─TableReader(Probe) 10.00 root data:Selection", + " │ └─IndexFullScan 100.00 cop[tikv] table:t1, index:idx(c, b, a) keep order:false", + " └─IndexReader(Probe) 10.00 root index:Selection", " └─Selection 10.00 cop[tikv] eq(test.t.a, test.t.a), not(isnull(test.t.a))", - " └─TableFullScan 100.00 cop[tikv] table:s keep order:false" + " └─IndexFullScan 100.00 cop[tikv] table:s, index:idx(c, b, a) keep order:false" ], [ "Projection 10.00 root Column#13", "└─Apply 10.00 root CARTESIAN left outer join", - " ├─TableReader(Build) 10.00 root data:TableFullScan", - " │ └─TableFullScan 10.00 cop[tikv] table:t keep order:false", + " ├─IndexReader(Build) 10.00 root index:IndexFullScan", + " │ └─IndexFullScan 10.00 cop[tikv] table:t, index:idx(c, b, a) keep order:false", " └─MaxOneRow(Probe) 10.00 root ", " └─Projection 1.00 root concat(cast(test.t.a, var_string(20)), ,, cast(test.t.b, var_string(20)))->Column#13", " └─IndexReader 1.00 root index:Selection", @@ -424,7 +424,7 @@ "Name": "TestEmptyTable", "Cases": [ "TableReader(Table(t)->Sel([le(test.t.c1, 50)]))", - "LeftHashJoin{TableReader(Table(t)->Sel([not(isnull(test.t.c1))]))->TableReader(Table(t1)->Sel([not(isnull(test.t1.c1))]))->HashAgg}(test.t.c1,test.t1.c1)", + "LeftHashJoin{TableReader(Table(t)->Sel([not(isnull(test.t.c1))]))->TableReader(Table(t1)->Sel([not(isnull(test.t1.c1))])->HashAgg)->HashAgg}(test.t.c1,test.t1.c1)", "LeftHashJoin{TableReader(Table(t)->Sel([not(isnull(test.t.c1))]))->TableReader(Table(t1)->Sel([not(isnull(test.t1.c1))]))}(test.t.c1,test.t1.c1)", "Dual" ] diff --git a/planner/core/testdata/enforce_mpp_suite_out.json b/planner/core/testdata/enforce_mpp_suite_out.json index 20f349233aed4..3f8a2667a4dc2 100644 --- a/planner/core/testdata/enforce_mpp_suite_out.json +++ b/planner/core/testdata/enforce_mpp_suite_out.json @@ -31,30 +31,30 @@ { "SQL": "explain format='verbose' select count(*) from t where a=1", "Plan": [ - "StreamAgg_24 1.00 217.09 root funcs:count(Column#6)->Column#4", - "└─IndexReader_25 1.00 167.19 root index:StreamAgg_9", - " └─StreamAgg_9 1.00 2476.20 cop[tikv] funcs:count(1)->Column#6", - " └─IndexRangeScan_23 10.00 1977.20 cop[tikv] table:t, index:idx(a) range:[1,1], keep order:false, stats:pseudo" + "StreamAgg_24 1.00 193.81 root funcs:count(Column#6)->Column#4", + "└─IndexReader_25 1.00 143.91 root index:StreamAgg_9", + " └─StreamAgg_9 1.00 2127.00 cop[tikv] funcs:count(1)->Column#6", + " └─IndexRangeScan_23 10.00 1628.00 cop[tikv] table:t, index:idx(a) range:[1,1], keep order:false, stats:pseudo" ], "Warn": null }, { "SQL": "explain format='verbose' select /*+ read_from_storage(tikv[t]) */ count(*) from t where a=1", "Plan": [ - "StreamAgg_17 1.00 217.09 root funcs:count(Column#6)->Column#4", - "└─IndexReader_18 1.00 167.19 root index:StreamAgg_9", - " └─StreamAgg_9 1.00 2476.20 cop[tikv] funcs:count(1)->Column#6", - " └─IndexRangeScan_16 10.00 1977.20 cop[tikv] table:t, index:idx(a) range:[1,1], keep order:false, stats:pseudo" + "StreamAgg_17 1.00 193.81 root funcs:count(Column#6)->Column#4", + "└─IndexReader_18 1.00 143.91 root index:StreamAgg_9", + " └─StreamAgg_9 1.00 2127.00 cop[tikv] funcs:count(1)->Column#6", + " └─IndexRangeScan_16 10.00 1628.00 cop[tikv] table:t, index:idx(a) range:[1,1], keep order:false, stats:pseudo" ], "Warn": null }, { "SQL": "explain format='verbose' select /*+ read_from_storage(tiflash[t]) */ count(*) from t where a=1", "Plan": [ - "StreamAgg_10 1.00 65360.67 root funcs:count(1)->Column#4", - "└─TableReader_24 10.00 64861.67 root data:Selection_23", - " └─Selection_23 10.00 972291.38 cop[tiflash] eq(test.t.a, 1)", - " └─TableFullScan_22 10000.00 948291.38 cop[tiflash] table:t keep order:false, stats:pseudo" + "StreamAgg_10 1.00 64007.91 root funcs:count(1)->Column#4", + "└─TableReader_24 10.00 63508.91 root data:Selection_23", + " └─Selection_23 10.00 952000.00 cop[tiflash] eq(test.t.a, 1)", + " └─TableFullScan_22 10000.00 928000.00 cop[tiflash] table:t keep order:false, stats:pseudo" ], "Warn": null }, @@ -71,30 +71,30 @@ { "SQL": "explain format='verbose' select count(*) from t where a=1", "Plan": [ - "StreamAgg_31 1.00 217.09 root funcs:count(Column#7)->Column#4", - "└─IndexReader_32 1.00 167.19 root index:StreamAgg_11", - " └─StreamAgg_11 1.00 2476.20 cop[tikv] funcs:count(1)->Column#7", - " └─IndexRangeScan_30 10.00 1977.20 cop[tikv] table:t, index:idx(a) range:[1,1], keep order:false, stats:pseudo" + "StreamAgg_31 1.00 193.81 root funcs:count(Column#7)->Column#4", + "└─IndexReader_32 1.00 143.91 root index:StreamAgg_11", + " └─StreamAgg_11 1.00 2127.00 cop[tikv] funcs:count(1)->Column#7", + " └─IndexRangeScan_30 10.00 1628.00 cop[tikv] table:t, index:idx(a) range:[1,1], keep order:false, stats:pseudo" ], "Warn": null }, { "SQL": "explain format='verbose' select /*+ read_from_storage(tikv[t]) */ count(*) from t where a=1", "Plan": [ - "StreamAgg_19 1.00 217.09 root funcs:count(Column#6)->Column#4", - "└─IndexReader_20 1.00 167.19 root index:StreamAgg_11", - " └─StreamAgg_11 1.00 2476.20 cop[tikv] funcs:count(1)->Column#6", - " └─IndexRangeScan_18 10.00 1977.20 cop[tikv] table:t, index:idx(a) range:[1,1], keep order:false, stats:pseudo" + "StreamAgg_19 1.00 193.81 root funcs:count(Column#6)->Column#4", + "└─IndexReader_20 1.00 143.91 root index:StreamAgg_11", + " └─StreamAgg_11 1.00 2127.00 cop[tikv] funcs:count(1)->Column#6", + " └─IndexRangeScan_18 10.00 1628.00 cop[tikv] table:t, index:idx(a) range:[1,1], keep order:false, stats:pseudo" ], "Warn": null }, { "SQL": "explain format='verbose' select /*+ read_from_storage(tiflash[t]) */ count(*) from t where a=1", "Plan": [ - "StreamAgg_12 1.00 65360.67 root funcs:count(1)->Column#4", - "└─TableReader_31 10.00 64861.67 root data:Selection_30", - " └─Selection_30 10.00 972291.38 cop[tiflash] eq(test.t.a, 1)", - " └─TableFullScan_29 10000.00 948291.38 cop[tiflash] table:t keep order:false, stats:pseudo" + "StreamAgg_12 1.00 64007.91 root funcs:count(1)->Column#4", + "└─TableReader_31 10.00 63508.91 root data:Selection_30", + " └─Selection_30 10.00 952000.00 cop[tiflash] eq(test.t.a, 1)", + " └─TableFullScan_29 10000.00 928000.00 cop[tiflash] table:t keep order:false, stats:pseudo" ], "Warn": null }, @@ -106,30 +106,30 @@ { "SQL": "explain format='verbose' select count(*) from t where a=1", "Plan": [ - "StreamAgg_31 1.00 217.09 root funcs:count(Column#7)->Column#4", - "└─IndexReader_32 1.00 167.19 root index:StreamAgg_11", - " └─StreamAgg_11 1.00 2476.20 cop[tikv] funcs:count(1)->Column#7", - " └─IndexRangeScan_30 10.00 1977.20 cop[tikv] table:t, index:idx(a) range:[1,1], keep order:false, stats:pseudo" + "StreamAgg_31 1.00 193.81 root funcs:count(Column#7)->Column#4", + "└─IndexReader_32 1.00 143.91 root index:StreamAgg_11", + " └─StreamAgg_11 1.00 2127.00 cop[tikv] funcs:count(1)->Column#7", + " └─IndexRangeScan_30 10.00 1628.00 cop[tikv] table:t, index:idx(a) range:[1,1], keep order:false, stats:pseudo" ], "Warn": null }, { "SQL": "explain format='verbose' select /*+ read_from_storage(tikv[t]) */ count(*) from t where a=1", "Plan": [ - "StreamAgg_19 1.00 217.09 root funcs:count(Column#6)->Column#4", - "└─IndexReader_20 1.00 167.19 root index:StreamAgg_11", - " └─StreamAgg_11 1.00 2476.20 cop[tikv] funcs:count(1)->Column#6", - " └─IndexRangeScan_18 10.00 1977.20 cop[tikv] table:t, index:idx(a) range:[1,1], keep order:false, stats:pseudo" + "StreamAgg_19 1.00 193.81 root funcs:count(Column#6)->Column#4", + "└─IndexReader_20 1.00 143.91 root index:StreamAgg_11", + " └─StreamAgg_11 1.00 2127.00 cop[tikv] funcs:count(1)->Column#6", + " └─IndexRangeScan_18 10.00 1628.00 cop[tikv] table:t, index:idx(a) range:[1,1], keep order:false, stats:pseudo" ], "Warn": null }, { "SQL": "explain format='verbose' select /*+ read_from_storage(tiflash[t]) */ count(*) from t where a=1", "Plan": [ - "StreamAgg_12 1.00 65360.67 root funcs:count(1)->Column#4", - "└─TableReader_31 10.00 64861.67 root data:Selection_30", - " └─Selection_30 10.00 972291.38 cop[tiflash] eq(test.t.a, 1)", - " └─TableFullScan_29 10000.00 948291.38 cop[tiflash] table:t keep order:false, stats:pseudo" + "StreamAgg_12 1.00 64007.91 root funcs:count(1)->Column#4", + "└─TableReader_31 10.00 63508.91 root data:Selection_30", + " └─Selection_30 10.00 952000.00 cop[tiflash] eq(test.t.a, 1)", + " └─TableFullScan_29 10000.00 928000.00 cop[tiflash] table:t keep order:false, stats:pseudo" ], "Warn": null }, @@ -141,20 +141,20 @@ { "SQL": "explain format='verbose' select count(*) from t where a=1", "Plan": [ - "StreamAgg_31 1.00 217.09 root funcs:count(Column#7)->Column#4", - "└─IndexReader_32 1.00 167.19 root index:StreamAgg_11", - " └─StreamAgg_11 1.00 2476.20 cop[tikv] funcs:count(1)->Column#7", - " └─IndexRangeScan_30 10.00 1977.20 cop[tikv] table:t, index:idx(a) range:[1,1], keep order:false, stats:pseudo" + "StreamAgg_31 1.00 193.81 root funcs:count(Column#7)->Column#4", + "└─IndexReader_32 1.00 143.91 root index:StreamAgg_11", + " └─StreamAgg_11 1.00 2127.00 cop[tikv] funcs:count(1)->Column#7", + " └─IndexRangeScan_30 10.00 1628.00 cop[tikv] table:t, index:idx(a) range:[1,1], keep order:false, stats:pseudo" ], "Warn": null }, { "SQL": "explain format='verbose' select /*+ read_from_storage(tikv[t]) */ count(*) from t where a=1", "Plan": [ - "StreamAgg_19 1.00 217.09 root funcs:count(Column#6)->Column#4", - "└─IndexReader_20 1.00 167.19 root index:StreamAgg_11", - " └─StreamAgg_11 1.00 2476.20 cop[tikv] funcs:count(1)->Column#6", - " └─IndexRangeScan_18 10.00 1977.20 cop[tikv] table:t, index:idx(a) range:[1,1], keep order:false, stats:pseudo" + "StreamAgg_19 1.00 193.81 root funcs:count(Column#6)->Column#4", + "└─IndexReader_20 1.00 143.91 root index:StreamAgg_11", + " └─StreamAgg_11 1.00 2127.00 cop[tikv] funcs:count(1)->Column#6", + " └─IndexRangeScan_18 10.00 1628.00 cop[tikv] table:t, index:idx(a) range:[1,1], keep order:false, stats:pseudo" ], "Warn": [ "MPP mode may be blocked because you have set a hint to read table `t` from TiKV." @@ -165,10 +165,10 @@ "Plan": [ "HashAgg_22 1.00 39930.30 root funcs:count(Column#6)->Column#4", "└─TableReader_24 1.00 0.00 root data:ExchangeSender_23", - " └─ExchangeSender_23 1.00 974211.94 mpp[tiflash] ExchangeType: PassThrough", - " └─HashAgg_10 1.00 974211.94 mpp[tiflash] funcs:count(1)->Column#6", - " └─Selection_21 10.00 972291.38 mpp[tiflash] eq(test.t.a, 1)", - " └─TableFullScan_20 10000.00 948291.38 mpp[tiflash] table:t keep order:false, stats:pseudo" + " └─ExchangeSender_23 1.00 953920.56 mpp[tiflash] ExchangeType: PassThrough", + " └─HashAgg_10 1.00 953920.56 mpp[tiflash] funcs:count(1)->Column#6", + " └─Selection_21 10.00 952000.00 mpp[tiflash] eq(test.t.a, 1)", + " └─TableFullScan_20 10000.00 928000.00 mpp[tiflash] table:t keep order:false, stats:pseudo" ], "Warn": null } diff --git a/planner/core/testdata/expression_rewriter_suite_out.json b/planner/core/testdata/expression_rewriter_suite_out.json index 3c850e434fc9b..72b86884a1412 100644 --- a/planner/core/testdata/expression_rewriter_suite_out.json +++ b/planner/core/testdata/expression_rewriter_suite_out.json @@ -7,10 +7,9 @@ "Plan": [ "HashJoin 2.25 root inner join, equal:[eq(test.t1.a, test.t2.a) eq(test.t1.b, test.t2.b)]", "├─HashAgg(Build) 1.69 root group by:test.t2.a, test.t2.b, funcs:firstrow(test.t2.a)->test.t2.a, funcs:firstrow(test.t2.b)->test.t2.b", - "│ └─TableReader 1.69 root data:HashAgg", - "│ └─HashAgg 1.69 cop[tikv] group by:test.t2.a, test.t2.b, ", - "│ └─Selection 2.25 cop[tikv] not(isnull(test.t2.a)), not(isnull(test.t2.b))", - "│ └─TableFullScan 4.00 cop[tikv] table:t2 keep order:false", + "│ └─TableReader 2.25 root data:Selection", + "│ └─Selection 2.25 cop[tikv] not(isnull(test.t2.a)), not(isnull(test.t2.b))", + "│ └─TableFullScan 4.00 cop[tikv] table:t2 keep order:false", "└─TableReader(Probe) 2.25 root data:Selection", " └─Selection 2.25 cop[tikv] not(isnull(test.t1.a)), not(isnull(test.t1.b))", " └─TableFullScan 4.00 cop[tikv] table:t1 keep order:false" @@ -37,10 +36,9 @@ "Plan": [ "HashJoin 1.69 root inner join, equal:[eq(test.t2.a, test.t1.a) eq(test.t2.b, Column#7)]", "├─HashAgg(Build) 1.69 root group by:test.t2.a, test.t2.b, funcs:firstrow(test.t2.a)->test.t2.a, funcs:firstrow(test.t2.b)->test.t2.b", - "│ └─TableReader 1.69 root data:HashAgg", - "│ └─HashAgg 1.69 cop[tikv] group by:test.t2.a, test.t2.b, ", - "│ └─Selection 2.25 cop[tikv] not(isnull(test.t2.a)), not(isnull(test.t2.b))", - "│ └─TableFullScan 4.00 cop[tikv] table:t2 keep order:false", + "│ └─TableReader 2.25 root data:Selection", + "│ └─Selection 2.25 cop[tikv] not(isnull(test.t2.a)), not(isnull(test.t2.b))", + "│ └─TableFullScan 4.00 cop[tikv] table:t2 keep order:false", "└─HashAgg(Probe) 2.25 root group by:test.t1.a, funcs:count(1)->Column#7, funcs:firstrow(test.t1.a)->test.t1.a", " └─TableReader 3.00 root data:Selection", " └─Selection 3.00 cop[tikv] not(isnull(test.t1.a))", diff --git a/planner/core/testdata/integration_suite_out.json b/planner/core/testdata/integration_suite_out.json index 0a4b38ba0b05b..304d84c59486a 100644 --- a/planner/core/testdata/integration_suite_out.json +++ b/planner/core/testdata/integration_suite_out.json @@ -1861,8 +1861,8 @@ { "SQL": "select f, g from t1 where f = 2 and g > 3", "Plan": [ - "IndexReader_6 33.33 752.77 root index:IndexRangeScan_5", - "└─IndexRangeScan_5 33.33 7067.49 cop[tikv] table:t1, index:f_g(f, g) range:(2 3,2 +inf], keep order:false, stats:pseudo" + "IndexReader_6 33.33 733.82 root index:IndexRangeScan_5", + "└─IndexRangeScan_5 33.33 6783.33 cop[tikv] table:t1, index:f_g(f, g) range:(2 3,2 +inf], keep order:false, stats:pseudo" ], "Warnings": [ "Note 1105 unique index f_g of t1 is selected since the path only fetches limited number of rows with single scan" @@ -1964,21 +1964,20 @@ { "SQL": "select * from t where a > 1 order by f", "Plan": [ - "IndexLookUp_14 3333.33 136747.00 root ", - "├─Selection_13(Build) 3333.33 585000.00 cop[tikv] gt(test.t.a, 1)", - "│ └─IndexFullScan_11 10000.00 555000.00 cop[tikv] table:t, index:f(f) keep order:true, stats:pseudo", - "└─TableRowIDScan_12(Probe) 3333.33 370000.00 cop[tikv] table:t keep order:false, stats:pseudo" + "Sort_5 3333.33 2146348.14 root test.t.f", + "└─TableReader_9 3333.33 160128.74 root data:TableRangeScan_8", + " └─TableRangeScan_8 3333.33 923531.15 cop[tikv] table:t range:(1,+inf], keep order:false, stats:pseudo" ], "Warnings": [ - "Note 1105 [t,f,f_g] remain after pruning paths for t given Prop{SortItems: [{test.t.f asc}], TaskTp: rootTask}" + "Note 1105 [t] remain after pruning paths for t given Prop{SortItems: [], TaskTp: rootTask}" ] }, { "SQL": "select * from t where f > 1", "Plan": [ - "IndexLookUp_10 3333.33 86674.83 root ", - "├─IndexRangeScan_8(Build) 3333.33 185000.00 cop[tikv] table:t, index:f(f) range:(1,+inf], keep order:false, stats:pseudo", - "└─TableRowIDScan_9(Probe) 3333.33 370000.00 cop[tikv] table:t keep order:false, stats:pseudo" + "TableReader_7 3333.33 316532.90 root data:Selection_6", + "└─Selection_6 3333.33 3269593.45 cop[tikv] gt(test.t.f, 1)", + " └─TableFullScan_5 10000.00 2770593.45 cop[tikv] table:t keep order:false, stats:pseudo" ], "Warnings": [ "Note 1105 [t,f,f_g] remain after pruning paths for t given Prop{SortItems: [], TaskTp: rootTask}" @@ -1987,8 +1986,8 @@ { "SQL": "select f from t where f > 1", "Plan": [ - "IndexReader_6 3333.33 11140.22 root index:IndexRangeScan_5", - "└─IndexRangeScan_5 3333.33 140000.00 cop[tikv] table:t, index:f(f) range:(1,+inf], keep order:false, stats:pseudo" + "IndexReader_6 3333.33 50257.78 root index:IndexRangeScan_5", + "└─IndexRangeScan_5 3333.33 542666.67 cop[tikv] table:t, index:f(f) range:(1,+inf], keep order:false, stats:pseudo" ], "Warnings": [ "Note 1105 [f,f_g] remain after pruning paths for t given Prop{SortItems: [], TaskTp: rootTask}" @@ -1997,10 +1996,10 @@ { "SQL": "select * from t where f > 3 and g = 5", "Plan": [ - "IndexLookUp_15 3.33 206.74 root ", - "├─IndexRangeScan_12(Build) 10.00 570.00 cop[tikv] table:t, index:g(g) range:[5,5], keep order:false, stats:pseudo", - "└─Selection_14(Probe) 3.33 1140.00 cop[tikv] gt(test.t.f, 3)", - " └─TableRowIDScan_13 10.00 1110.00 cop[tikv] table:t keep order:false, stats:pseudo" + "IndexLookUp_15 3.33 19551.99 root ", + "├─IndexRangeScan_12(Build) 10.00 2035.00 cop[tikv] table:t, index:g(g) range:[5,5], keep order:false, stats:pseudo", + "└─Selection_14(Probe) 3.33 3269.59 cop[tikv] gt(test.t.f, 3)", + " └─TableRowIDScan_13 10.00 2770.59 cop[tikv] table:t keep order:false, stats:pseudo" ], "Warnings": [ "Note 1105 [t,f_g,g] remain after pruning paths for t given Prop{SortItems: [], TaskTp: rootTask}" @@ -2009,10 +2008,10 @@ { "SQL": "select * from t where g = 5 order by f", "Plan": [ - "Sort_5 10.00 353.68 root test.t.f", - "└─IndexLookUp_13 10.00 230.01 root ", - " ├─IndexRangeScan_11(Build) 10.00 570.00 cop[tikv] table:t, index:g(g) range:[5,5], keep order:false, stats:pseudo", - " └─TableRowIDScan_12(Probe) 10.00 1110.00 cop[tikv] table:t keep order:false, stats:pseudo" + "Sort_5 10.00 21321.97 root test.t.f", + "└─IndexLookUp_13 10.00 19545.34 root ", + " ├─IndexRangeScan_11(Build) 10.00 2035.00 cop[tikv] table:t, index:g(g) range:[5,5], keep order:false, stats:pseudo", + " └─TableRowIDScan_12(Probe) 10.00 2770.59 cop[tikv] table:t keep order:false, stats:pseudo" ], "Warnings": [ "Note 1105 [t,g] remain after pruning paths for t given Prop{SortItems: [], TaskTp: rootTask}" @@ -2021,10 +2020,10 @@ { "SQL": "select * from t where d = 3 order by c, e", "Plan": [ - "IndexLookUp_15 10.00 57222.78 root ", - "├─Selection_14(Build) 10.00 855000.00 cop[tikv] eq(test.t.d, 3)", - "│ └─IndexFullScan_12 10000.00 825000.00 cop[tikv] table:t, index:c_d_e(c, d, e) keep order:true, stats:pseudo", - "└─TableRowIDScan_13(Probe) 10.00 1110.00 cop[tikv] table:t keep order:false, stats:pseudo" + "IndexLookUp_15 10.00 215519.24 root ", + "├─Selection_14(Build) 10.00 2941000.00 cop[tikv] eq(test.t.d, 3)", + "│ └─IndexFullScan_12 10000.00 2442000.00 cop[tikv] table:t, index:c_d_e(c, d, e) keep order:true, stats:pseudo", + "└─TableRowIDScan_13(Probe) 10.00 2770.59 cop[tikv] table:t keep order:false, stats:pseudo" ], "Warnings": [ "Note 1105 [t,c_d_e] remain after pruning paths for t given Prop{SortItems: [{test.t.c asc} {test.t.e asc}], TaskTp: rootTask}" @@ -2043,31 +2042,31 @@ { "SQL": "explain format = 'verbose' select * from t where b > 5", "Plan": [ - "TableReader_7 3.00 19.21 root data:Selection_6", - "└─Selection_6 3.00 195.00 cop[tikv] gt(test.t.b, 5)", - " └─TableFullScan_5 5.00 180.00 cop[tikv] table:t keep order:false" + "TableReader_7 3.00 130.42 root data:Selection_6", + "└─Selection_6 3.00 1386.04 cop[tikv] gt(test.t.b, 5)", + " └─TableFullScan_5 5.00 1136.54 cop[tikv] table:t keep order:false" ], "Warnings": null }, { "SQL": "explain format = 'verbose' select * from t where b = 6 order by a limit 1", "Plan": [ - "Limit_11 0.00 14.33 root offset:0, count:1", - "└─TableReader_24 0.00 14.33 root data:Limit_23", - " └─Limit_23 0.00 195.00 cop[tikv] offset:0, count:1", - " └─Selection_22 0.00 195.00 cop[tikv] eq(test.t.b, 6)", - " └─TableFullScan_21 5.00 180.00 cop[tikv] table:t keep order:true" + "Limit_11 0.00 92.40 root offset:0, count:1", + "└─TableReader_24 0.00 92.40 root data:Limit_23", + " └─Limit_23 0.00 1386.04 cop[tikv] offset:0, count:1", + " └─Selection_22 0.00 1386.04 cop[tikv] eq(test.t.b, 6)", + " └─TableFullScan_21 5.00 1136.54 cop[tikv] table:t keep order:true" ], "Warnings": null }, { "SQL": "explain format = 'verbose' select * from t where b = 6 limit 1", "Plan": [ - "Limit_8 0.00 14.33 root offset:0, count:1", - "└─TableReader_13 0.00 14.33 root data:Limit_12", - " └─Limit_12 0.00 195.00 cop[tikv] offset:0, count:1", - " └─Selection_11 0.00 195.00 cop[tikv] eq(test.t.b, 6)", - " └─TableFullScan_10 5.00 180.00 cop[tikv] table:t keep order:false" + "Limit_8 0.00 92.40 root offset:0, count:1", + "└─TableReader_13 0.00 92.40 root data:Limit_12", + " └─Limit_12 0.00 1386.04 cop[tikv] offset:0, count:1", + " └─Selection_11 0.00 1386.04 cop[tikv] eq(test.t.b, 6)", + " └─TableFullScan_10 5.00 1136.54 cop[tikv] table:t keep order:false" ], "Warnings": null }, @@ -2079,9 +2078,9 @@ { "SQL": "explain format = 'verbose' select * from t where b > 5", "Plan": [ - "IndexLookUp_7 3.00 57.91 root ", - "├─IndexRangeScan_5(Build) 3.00 171.00 cop[tikv] table:t, index:idx_b(b) range:(5,+inf], keep order:false", - "└─TableRowIDScan_6(Probe) 3.00 108.00 cop[tikv] table:t keep order:false" + "IndexLookUp_7 3.00 5856.46 root ", + "├─IndexRangeScan_5(Build) 3.00 610.50 cop[tikv] table:t, index:idx_b(b) range:(5,+inf], keep order:false", + "└─TableRowIDScan_6(Probe) 3.00 681.92 cop[tikv] table:t keep order:false" ], "Warnings": [ "Note 1105 [idx_b] remain after pruning paths for t given Prop{SortItems: [], TaskTp: rootTask}" @@ -2090,9 +2089,9 @@ { "SQL": "explain format = 'verbose' select * from t where b = 6 order by a limit 1", "Plan": [ - "TopN_9 0.00 19.34 root test.t.a, offset:0, count:1", - "└─IndexLookUp_16 0.00 19.33 root ", - " ├─TopN_15(Build) 0.00 0.00 cop[tikv] test.t.a, offset:0, count:1", + "TopN_9 0.00 1925.01 root test.t.a, offset:0, count:1", + "└─IndexLookUp_16 0.00 1920.21 root ", + " ├─TopN_15(Build) 0.00 3.20 cop[tikv] test.t.a, offset:0, count:1", " │ └─IndexRangeScan_13 0.00 0.00 cop[tikv] table:t, index:idx_b(b) range:[6,6], keep order:false", " └─TableRowIDScan_14(Probe) 0.00 0.00 cop[tikv] table:t keep order:false" ], @@ -2103,7 +2102,7 @@ { "SQL": "explain format = 'verbose' select * from t where b = 6 limit 1", "Plan": [ - "IndexLookUp_13 0.00 19.33 root limit embedded(offset:0, count:1)", + "IndexLookUp_13 0.00 1152.00 root limit embedded(offset:0, count:1)", "├─Limit_12(Build) 0.00 0.00 cop[tikv] offset:0, count:1", "│ └─IndexRangeScan_10 0.00 0.00 cop[tikv] table:t, index:idx_b(b) range:[6,6], keep order:false", "└─TableRowIDScan_11(Probe) 0.00 0.00 cop[tikv] table:t keep order:false" @@ -2486,65 +2485,65 @@ { "SQL": "explain format = 'verbose' select count(*) from t3", "Plan": [ - "StreamAgg_20 1.00 99.31 root funcs:count(Column#9)->Column#4", - "└─TableReader_21 1.00 49.41 root data:StreamAgg_8", - " └─StreamAgg_8 1.00 709.52 cop[tikv] funcs:count(1)->Column#9", - " └─TableFullScan_18 3.00 559.82 cop[tikv] table:t3 keep order:false" + "StreamAgg_20 1.00 102.69 root funcs:count(Column#9)->Column#4", + "└─IndexReader_21 1.00 52.79 root index:StreamAgg_8", + " └─StreamAgg_8 1.00 760.20 cop[tikv] funcs:count(1)->Column#9", + " └─IndexFullScan_19 3.00 610.50 cop[tikv] table:t3, index:c(b) keep order:false" ] }, { "SQL": "explain format = 'verbose' select count(*) from t2", "Plan": [ - "StreamAgg_26 1.00 99.31 root funcs:count(Column#7)->Column#4", - "└─TableReader_27 1.00 49.41 root data:StreamAgg_10", - " └─StreamAgg_10 1.00 709.52 cop[tikv] funcs:count(1)->Column#7", - " └─TableFullScan_24 3.00 559.82 cop[tikv] table:t2 keep order:false" + "StreamAgg_26 1.00 107.45 root funcs:count(Column#7)->Column#4", + "└─TableReader_27 1.00 57.55 root data:StreamAgg_10", + " └─StreamAgg_10 1.00 831.62 cop[tikv] funcs:count(1)->Column#7", + " └─TableFullScan_24 3.00 681.92 cop[tikv] table:t2 keep order:false" ] }, { "SQL": "explain format = 'verbose' select * from t3 order by a", "Plan": [ - "Sort_4 3.00 310.13 root test.t3.a", - "└─TableReader_8 3.00 62.67 root data:TableFullScan_7", - " └─TableFullScan_7 3.00 559.82 cop[tikv] table:t3 keep order:false" + "Sort_4 3.00 318.27 root test.t3.a", + "└─TableReader_8 3.00 70.81 root data:TableFullScan_7", + " └─TableFullScan_7 3.00 681.92 cop[tikv] table:t3 keep order:false" ] }, { "SQL": "explain format = 'verbose' select * from t3 order by b", "Plan": [ - "Sort_4 3.00 310.13 root test.t3.b", - "└─TableReader_8 3.00 62.67 root data:TableFullScan_7", - " └─TableFullScan_7 3.00 559.82 cop[tikv] table:t3 keep order:false" + "Sort_4 3.00 318.27 root test.t3.b", + "└─TableReader_8 3.00 70.81 root data:TableFullScan_7", + " └─TableFullScan_7 3.00 681.92 cop[tikv] table:t3 keep order:false" ] }, { "SQL": "explain format = 'verbose' select * from t3 order by a limit 1", "Plan": [ - "TopN_7 1.00 44.96 root test.t3.a, offset:0, count:1", - "└─TableReader_16 1.00 41.76 root data:TopN_15", - " └─TopN_15 1.00 563.02 cop[tikv] test.t3.a, offset:0, count:1", - " └─TableFullScan_14 3.00 559.82 cop[tikv] table:t3 keep order:false" + "TopN_7 1.00 53.10 root test.t3.a, offset:0, count:1", + "└─TableReader_16 1.00 49.90 root data:TopN_15", + " └─TopN_15 1.00 685.12 cop[tikv] test.t3.a, offset:0, count:1", + " └─TableFullScan_14 3.00 681.92 cop[tikv] table:t3 keep order:false" ] }, { "SQL": "explain format = 'verbose' select * from t3 order by b limit 1", "Plan": [ - "TopN_7 1.00 44.96 root test.t3.b, offset:0, count:1", - "└─TableReader_16 1.00 41.76 root data:TopN_15", - " └─TopN_15 1.00 563.02 cop[tikv] test.t3.b, offset:0, count:1", - " └─TableFullScan_14 3.00 559.82 cop[tikv] table:t3 keep order:false" + "TopN_7 1.00 53.10 root test.t3.b, offset:0, count:1", + "└─TableReader_16 1.00 49.90 root data:TopN_15", + " └─TopN_15 1.00 685.12 cop[tikv] test.t3.b, offset:0, count:1", + " └─TableFullScan_14 3.00 681.92 cop[tikv] table:t3 keep order:false" ] }, { "SQL": "explain format = 'verbose' select count(*) from t2 group by a", "Plan": [ - "TableReader_44 3.00 25896.85 root data:ExchangeSender_43", - "└─ExchangeSender_43 3.00 388400.01 mpp[tiflash] ExchangeType: PassThrough", - " └─Projection_38 3.00 388400.00 mpp[tiflash] Column#4", - " └─HashAgg_36 3.00 388399.98 mpp[tiflash] group by:test.t2.a, funcs:count(1)->Column#4", - " └─ExchangeReceiver_22 3.00 385507.26 mpp[tiflash] ", - " └─ExchangeSender_21 3.00 385459.26 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t2.a, collate: binary]", - " └─TableFullScan_20 3.00 385459.26 mpp[tiflash] table:t2 keep order:false" + "TableReader_44 3.00 31142.18 root data:ExchangeSender_43", + "└─ExchangeSender_43 3.00 467079.95 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection_38 3.00 467079.93 mpp[tiflash] Column#4", + " └─HashAgg_36 3.00 467079.92 mpp[tiflash] group by:test.t2.a, funcs:count(1)->Column#4", + " └─ExchangeReceiver_22 3.00 464187.20 mpp[tiflash] ", + " └─ExchangeSender_21 3.00 464139.20 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t2.a, collate: binary]", + " └─TableFullScan_20 3.00 464139.20 mpp[tiflash] table:t2 keep order:false" ] }, { @@ -2558,8 +2557,8 @@ { "SQL": "explain format = 'verbose' select /*+ use_index(t3, c) */ count(a) from t3 where b = 0", "Plan": [ - "StreamAgg_10 1.00 0.00 root funcs:count(test.t3.a)->Column#4", - "└─IndexLookUp_17 0.00 0.00 root ", + "StreamAgg_10 1.00 1920.00 root funcs:count(test.t3.a)->Column#4", + "└─IndexLookUp_17 0.00 1920.00 root ", " ├─IndexRangeScan_15(Build) 0.00 0.00 cop[tikv] table:t3, index:c(b) range:[0,0], keep order:false", " └─TableRowIDScan_16(Probe) 0.00 0.00 cop[tikv] table:t3 keep order:false" ] @@ -2567,90 +2566,90 @@ { "SQL": "explain format = 'verbose' select count(*) from t2 where a = 0", "Plan": [ - "StreamAgg_12 1.00 47.30 root funcs:count(1)->Column#4", - "└─TableReader_21 0.00 47.30 root data:Selection_20", - " └─Selection_20 0.00 709.52 cop[tikv] eq(test.t2.a, 0)", - " └─TableFullScan_19 3.00 559.82 cop[tikv] table:t2 keep order:false" + "StreamAgg_12 1.00 55.44 root funcs:count(1)->Column#4", + "└─TableReader_21 0.00 55.44 root data:Selection_20", + " └─Selection_20 0.00 831.62 cop[tikv] eq(test.t2.a, 0)", + " └─TableFullScan_19 3.00 681.92 cop[tikv] table:t2 keep order:false" ] }, { "SQL": "explain format = 'verbose' select count(*) from t3 t join t3 on t.a = t3.b", "Plan": [ - "StreamAgg_10 1.00 630.77 root funcs:count(1)->Column#7", - "└─HashJoin_40 3.00 481.07 root inner join, equal:[eq(test.t3.a, test.t3.b)]", - " ├─IndexReader_28(Build) 3.00 52.22 root index:IndexFullScan_27", - " │ └─IndexFullScan_27 3.00 593.16 cop[tikv] table:t3, index:c(b) keep order:false", - " └─TableReader_26(Probe) 3.00 59.97 root data:Selection_25", - " └─Selection_25 3.00 709.52 cop[tikv] not(isnull(test.t3.a))", - " └─TableFullScan_24 3.00 559.82 cop[tikv] table:t keep order:false" + "StreamAgg_10 1.00 631.93 root funcs:count(1)->Column#7", + "└─HashJoin_40 3.00 482.23 root inner join, equal:[eq(test.t3.a, test.t3.b)]", + " ├─IndexReader_28(Build) 3.00 45.23 root index:IndexFullScan_27", + " │ └─IndexFullScan_27 3.00 488.40 cop[tikv] table:t3, index:c(b) keep order:false", + " └─TableReader_26(Probe) 3.00 68.11 root data:Selection_25", + " └─Selection_25 3.00 831.62 cop[tikv] not(isnull(test.t3.a))", + " └─TableFullScan_24 3.00 681.92 cop[tikv] table:t keep order:false" ] }, { "SQL": "explain format = 'verbose' select /*+ read_from_storage(tiflash[t1, t2]) */ count(*) from t1 join t2 on t1.a = t2.a", "Plan": [ - "StreamAgg_15 1.00 51562.56 root funcs:count(1)->Column#7", - "└─TableReader_41 3.00 51412.86 root data:ExchangeSender_40", - " └─ExchangeSender_40 3.00 771087.32 mpp[tiflash] ExchangeType: PassThrough", - " └─HashJoin_37 3.00 771087.32 mpp[tiflash] inner join, equal:[eq(test.t1.a, test.t2.a)]", - " ├─ExchangeReceiver_22(Build) 3.00 385610.46 mpp[tiflash] ", - " │ └─ExchangeSender_21 3.00 385466.46 mpp[tiflash] ExchangeType: Broadcast", - " │ └─Selection_20 3.00 385466.46 mpp[tiflash] not(isnull(test.t1.a))", - " │ └─TableFullScan_19 3.00 385459.26 mpp[tiflash] table:t1 keep order:false", - " └─Selection_24(Probe) 3.00 385466.46 mpp[tiflash] not(isnull(test.t2.a))", - " └─TableFullScan_23 3.00 385459.26 mpp[tiflash] table:t2 keep order:false" + "StreamAgg_15 1.00 62053.22 root funcs:count(1)->Column#7", + "└─TableReader_41 3.00 61903.52 root data:ExchangeSender_40", + " └─ExchangeSender_40 3.00 928447.20 mpp[tiflash] ExchangeType: PassThrough", + " └─HashJoin_37 3.00 928447.20 mpp[tiflash] inner join, equal:[eq(test.t1.a, test.t2.a)]", + " ├─ExchangeReceiver_22(Build) 3.00 464290.40 mpp[tiflash] ", + " │ └─ExchangeSender_21 3.00 464146.40 mpp[tiflash] ExchangeType: Broadcast", + " │ └─Selection_20 3.00 464146.40 mpp[tiflash] not(isnull(test.t1.a))", + " │ └─TableFullScan_19 3.00 464139.20 mpp[tiflash] table:t1 keep order:false", + " └─Selection_24(Probe) 3.00 464146.40 mpp[tiflash] not(isnull(test.t2.a))", + " └─TableFullScan_23 3.00 464139.20 mpp[tiflash] table:t2 keep order:false" ] }, { "SQL": "explain format = 'verbose' select /*+ read_from_storage(tiflash[t1, t2]) */ count(*) from t1 join t2 on t1.a = t2.a join t3 on t1.b = t3.b", "Plan": [ - "StreamAgg_15 1.00 54032.05 root funcs:count(1)->Column#10", - "└─HashJoin_59 3.00 53882.35 root inner join, equal:[eq(test.t1.b, test.t3.b)]", - " ├─IndexReader_47(Build) 3.00 52.22 root index:IndexFullScan_46", - " │ └─IndexFullScan_46 3.00 593.16 cop[tikv] table:t3, index:c(b) keep order:false", - " └─TableReader_39(Probe) 3.00 53461.26 root data:ExchangeSender_38", - " └─ExchangeSender_38 3.00 801760.47 mpp[tiflash] ExchangeType: PassThrough", - " └─HashJoin_29 3.00 801760.47 mpp[tiflash] inner join, equal:[eq(test.t1.a, test.t2.a)]", - " ├─ExchangeReceiver_35(Build) 3.00 416282.81 mpp[tiflash] ", - " │ └─ExchangeSender_34 3.00 415994.81 mpp[tiflash] ExchangeType: Broadcast", - " │ └─Selection_33 3.00 415994.81 mpp[tiflash] not(isnull(test.t1.a)), not(isnull(test.t1.b))", - " │ └─TableFullScan_32 3.00 415980.41 mpp[tiflash] table:t1 keep order:false", - " └─Selection_37(Probe) 3.00 385466.46 mpp[tiflash] not(isnull(test.t2.a))", - " └─TableFullScan_36 3.00 385459.26 mpp[tiflash] table:t2 keep order:false" + "StreamAgg_15 1.00 70216.64 root funcs:count(1)->Column#10", + "└─HashJoin_59 3.00 70066.94 root inner join, equal:[eq(test.t1.b, test.t3.b)]", + " ├─IndexReader_47(Build) 3.00 45.23 root index:IndexFullScan_46", + " │ └─IndexFullScan_46 3.00 488.40 cop[tikv] table:t3, index:c(b) keep order:false", + " └─TableReader_39(Probe) 3.00 69652.83 root data:ExchangeSender_38", + " └─ExchangeSender_38 3.00 1044634.00 mpp[tiflash] ExchangeType: PassThrough", + " └─HashJoin_29 3.00 1044634.00 mpp[tiflash] inner join, equal:[eq(test.t1.a, test.t2.a)]", + " ├─ExchangeReceiver_35(Build) 3.00 580476.40 mpp[tiflash] ", + " │ └─ExchangeSender_34 3.00 580188.40 mpp[tiflash] ExchangeType: Broadcast", + " │ └─Selection_33 3.00 580188.40 mpp[tiflash] not(isnull(test.t1.a)), not(isnull(test.t1.b))", + " │ └─TableFullScan_32 3.00 580174.00 mpp[tiflash] table:t1 keep order:false", + " └─Selection_37(Probe) 3.00 464146.40 mpp[tiflash] not(isnull(test.t2.a))", + " └─TableFullScan_36 3.00 464139.20 mpp[tiflash] table:t2 keep order:false" ] }, { "SQL": "explain format = 'verbose' select (2) in (select /*+ read_from_storage(tiflash[t1]) */ count(*) from t1) from (select t.b < (select /*+ read_from_storage(tiflash[t2]) */ t.b from t2 limit 1 ) from t3 t) t", "Plan": [ - "HashJoin_19 3.00 110724.79 root CARTESIAN left outer semi join", - "├─Selection_38(Build) 0.80 24733.71 root eq(2, Column#18)", - "│ └─StreamAgg_45 1.00 24683.81 root funcs:count(1)->Column#18", - "│ └─TableReader_59 3.00 24534.11 root data:TableFullScan_58", - "│ └─TableFullScan_58 3.00 367821.61 cop[tiflash] table:t1 keep order:false", - "└─Projection_20(Probe) 3.00 85919.94 root 1->Column#28", - " └─Apply_22 3.00 85919.64 root CARTESIAN left outer join", - " ├─TableReader_24(Build) 3.00 49.99 root data:TableFullScan_23", - " │ └─TableFullScan_23 3.00 559.82 cop[tikv] table:t keep order:false", - " └─Projection_27(Probe) 3.00 28623.22 root 1->Column#26", - " └─Limit_30 3.00 28623.12 root offset:0, count:1", - " └─TableReader_37 3.00 28623.12 root data:ExchangeSender_36", - " └─ExchangeSender_36 3.00 429293.93 mpp[tiflash] ExchangeType: PassThrough", - " └─Limit_35 3.00 429293.93 mpp[tiflash] offset:0, count:1", - " └─TableFullScan_34 3.00 429293.93 mpp[tiflash] table:t2 keep order:false" + "HashJoin_19 3.00 160874.64 root CARTESIAN left outer semi join", + "├─Selection_38(Build) 0.80 31154.89 root eq(2, Column#18)", + "│ └─StreamAgg_45 1.00 31104.99 root funcs:count(1)->Column#18", + "│ └─TableReader_59 3.00 30955.29 root data:TableFullScan_58", + "│ └─TableFullScan_58 3.00 464139.20 cop[tiflash] table:t1 keep order:false", + "└─Projection_20(Probe) 3.00 129648.62 root 1->Column#28", + " └─Apply_22 3.00 129648.32 root CARTESIAN left outer join", + " ├─IndexReader_26(Build) 3.00 53.37 root index:IndexFullScan_25", + " │ └─IndexFullScan_25 3.00 610.50 cop[tikv] table:t, index:c(b) keep order:false", + " └─Projection_27(Probe) 3.00 43198.32 root 1->Column#26", + " └─Limit_30 3.00 43198.22 root offset:0, count:1", + " └─TableReader_37 3.00 43198.22 root data:ExchangeSender_36", + " └─ExchangeSender_36 3.00 647920.44 mpp[tiflash] ExchangeType: PassThrough", + " └─Limit_35 3.00 647920.44 mpp[tiflash] offset:0, count:1", + " └─TableFullScan_34 3.00 647920.44 mpp[tiflash] table:t2 keep order:false" ] }, { "SQL": "explain format = 'verbose' select /*+ merge_join(t1), read_from_storage(tiflash[t1, t2]) */ count(*) from t1 join t2 on t1.a = t2.a", "Plan": [ - "StreamAgg_14 1.00 52067.30 root funcs:count(1)->Column#7", - "└─MergeJoin_26 3.00 51917.60 root inner join, left key:test.t1.a, right key:test.t2.a", - " ├─Sort_24(Build) 3.00 25957.31 root test.t2.a", - " │ └─TableReader_23 3.00 25710.44 root data:Selection_22", - " │ └─Selection_22 3.00 385466.46 cop[tiflash] not(isnull(test.t2.a))", - " │ └─TableFullScan_21 3.00 385459.26 cop[tiflash] table:t2 keep order:false", - " └─Sort_20(Probe) 3.00 25957.31 root test.t1.a", - " └─TableReader_19 3.00 25710.44 root data:Selection_18", - " └─Selection_18 3.00 385466.46 cop[tiflash] not(isnull(test.t1.a))", - " └─TableFullScan_17 3.00 385459.26 cop[tiflash] table:t1 keep order:false" + "StreamAgg_14 1.00 62557.96 root funcs:count(1)->Column#7", + "└─MergeJoin_26 3.00 62408.26 root inner join, left key:test.t1.a, right key:test.t2.a", + " ├─Sort_24(Build) 3.00 31202.63 root test.t2.a", + " │ └─TableReader_23 3.00 30955.77 root data:Selection_22", + " │ └─Selection_22 3.00 464146.40 cop[tiflash] not(isnull(test.t2.a))", + " │ └─TableFullScan_21 3.00 464139.20 cop[tiflash] table:t2 keep order:false", + " └─Sort_20(Probe) 3.00 31202.63 root test.t1.a", + " └─TableReader_19 3.00 30955.77 root data:Selection_18", + " └─Selection_18 3.00 464146.40 cop[tiflash] not(isnull(test.t1.a))", + " └─TableFullScan_17 3.00 464139.20 cop[tiflash] table:t1 keep order:false" ] } ] @@ -3196,47 +3195,44 @@ { "SQL": "explain format = 'brief' select count(*) from fact_t join d1_t on fact_t.d1_k > d1_t.d1_k", "Plan": [ - "HashAgg 1.00 root funcs:count(Column#12)->Column#11", - "└─TableReader 1.00 root data:ExchangeSender", - " └─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", - " └─HashAgg 1.00 mpp[tiflash] funcs:count(1)->Column#12", - " └─HashJoin 16.00 mpp[tiflash] CARTESIAN inner join, other cond:gt(test.fact_t.d1_k, test.d1_t.d1_k)", - " ├─ExchangeReceiver(Build) 2.00 mpp[tiflash] ", - " │ └─ExchangeSender 2.00 mpp[tiflash] ExchangeType: Broadcast", - " │ └─Selection 2.00 mpp[tiflash] not(isnull(test.d1_t.d1_k))", - " │ └─TableFullScan 2.00 mpp[tiflash] table:d1_t keep order:false", - " └─Selection(Probe) 8.00 mpp[tiflash] not(isnull(test.fact_t.d1_k))", - " └─TableFullScan 8.00 mpp[tiflash] table:fact_t keep order:false" + "StreamAgg 1.00 root funcs:count(1)->Column#11", + "└─TableReader 16.00 root data:ExchangeSender", + " └─ExchangeSender 16.00 mpp[tiflash] ExchangeType: PassThrough", + " └─HashJoin 16.00 mpp[tiflash] CARTESIAN inner join, other cond:gt(test.fact_t.d1_k, test.d1_t.d1_k)", + " ├─ExchangeReceiver(Build) 2.00 mpp[tiflash] ", + " │ └─ExchangeSender 2.00 mpp[tiflash] ExchangeType: Broadcast", + " │ └─Selection 2.00 mpp[tiflash] not(isnull(test.d1_t.d1_k))", + " │ └─TableFullScan 2.00 mpp[tiflash] table:d1_t keep order:false", + " └─Selection(Probe) 8.00 mpp[tiflash] not(isnull(test.fact_t.d1_k))", + " └─TableFullScan 8.00 mpp[tiflash] table:fact_t keep order:false" ] }, { "SQL": "explain format = 'brief' select count(*) from fact_t left join d1_t on fact_t.d1_k > d1_t.d1_k", "Plan": [ - "HashAgg 1.00 root funcs:count(Column#12)->Column#11", - "└─TableReader 1.00 root data:ExchangeSender", - " └─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", - " └─HashAgg 1.00 mpp[tiflash] funcs:count(1)->Column#12", - " └─HashJoin 16.00 mpp[tiflash] CARTESIAN left outer join, other cond:gt(test.fact_t.d1_k, test.d1_t.d1_k)", - " ├─ExchangeReceiver(Build) 2.00 mpp[tiflash] ", - " │ └─ExchangeSender 2.00 mpp[tiflash] ExchangeType: Broadcast", - " │ └─Selection 2.00 mpp[tiflash] not(isnull(test.d1_t.d1_k))", - " │ └─TableFullScan 2.00 mpp[tiflash] table:d1_t keep order:false", - " └─TableFullScan(Probe) 8.00 mpp[tiflash] table:fact_t keep order:false" + "StreamAgg 1.00 root funcs:count(1)->Column#11", + "└─TableReader 16.00 root data:ExchangeSender", + " └─ExchangeSender 16.00 mpp[tiflash] ExchangeType: PassThrough", + " └─HashJoin 16.00 mpp[tiflash] CARTESIAN left outer join, other cond:gt(test.fact_t.d1_k, test.d1_t.d1_k)", + " ├─ExchangeReceiver(Build) 2.00 mpp[tiflash] ", + " │ └─ExchangeSender 2.00 mpp[tiflash] ExchangeType: Broadcast", + " │ └─Selection 2.00 mpp[tiflash] not(isnull(test.d1_t.d1_k))", + " │ └─TableFullScan 2.00 mpp[tiflash] table:d1_t keep order:false", + " └─TableFullScan(Probe) 8.00 mpp[tiflash] table:fact_t keep order:false" ] }, { "SQL": "explain format = 'brief' select count(*) from fact_t right join d1_t on fact_t.d1_k > d1_t.d1_k", "Plan": [ - "HashAgg 1.00 root funcs:count(Column#12)->Column#11", - "└─TableReader 1.00 root data:ExchangeSender", - " └─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", - " └─HashAgg 1.00 mpp[tiflash] funcs:count(1)->Column#12", - " └─HashJoin 16.00 mpp[tiflash] CARTESIAN right outer join, other cond:gt(test.fact_t.d1_k, test.d1_t.d1_k)", - " ├─ExchangeReceiver(Build) 8.00 mpp[tiflash] ", - " │ └─ExchangeSender 8.00 mpp[tiflash] ExchangeType: Broadcast", - " │ └─Selection 8.00 mpp[tiflash] not(isnull(test.fact_t.d1_k))", - " │ └─TableFullScan 8.00 mpp[tiflash] table:fact_t keep order:false", - " └─TableFullScan(Probe) 2.00 mpp[tiflash] table:d1_t keep order:false" + "StreamAgg 1.00 root funcs:count(1)->Column#11", + "└─TableReader 16.00 root data:ExchangeSender", + " └─ExchangeSender 16.00 mpp[tiflash] ExchangeType: PassThrough", + " └─HashJoin 16.00 mpp[tiflash] CARTESIAN right outer join, other cond:gt(test.fact_t.d1_k, test.d1_t.d1_k)", + " ├─ExchangeReceiver(Build) 8.00 mpp[tiflash] ", + " │ └─ExchangeSender 8.00 mpp[tiflash] ExchangeType: Broadcast", + " │ └─Selection 8.00 mpp[tiflash] not(isnull(test.fact_t.d1_k))", + " │ └─TableFullScan 8.00 mpp[tiflash] table:fact_t keep order:false", + " └─TableFullScan(Probe) 2.00 mpp[tiflash] table:d1_t keep order:false" ] }, { @@ -4317,9 +4313,9 @@ { "SQL": "desc format = 'brief' select /*+ read_from_storage(tikv[t], tiflash[t]) */ avg(a) from t", "Plan": [ - "StreamAgg 1.00 root funcs:avg(Column#7, Column#8)->Column#4", - "└─IndexReader 1.00 root index:StreamAgg", - " └─StreamAgg 1.00 cop[tikv] funcs:count(test.t.a)->Column#7, funcs:sum(test.t.a)->Column#8", + "HashAgg 1.00 root funcs:avg(Column#5, Column#6)->Column#4", + "└─IndexReader 1.00 root index:HashAgg", + " └─HashAgg 1.00 cop[tikv] funcs:count(test.t.a)->Column#5, funcs:sum(test.t.a)->Column#6", " └─IndexFullScan 10000.00 cop[tikv] table:t, index:ia(a) keep order:false, stats:pseudo" ], "Warn": [ @@ -4329,9 +4325,9 @@ { "SQL": "desc format = 'brief' select /*+ read_from_storage(tikv[t]) */ avg(a) from t", "Plan": [ - "StreamAgg 1.00 root funcs:avg(Column#7, Column#8)->Column#4", - "└─IndexReader 1.00 root index:StreamAgg", - " └─StreamAgg 1.00 cop[tikv] funcs:count(test.t.a)->Column#7, funcs:sum(test.t.a)->Column#8", + "HashAgg 1.00 root funcs:avg(Column#5, Column#6)->Column#4", + "└─IndexReader 1.00 root index:HashAgg", + " └─HashAgg 1.00 cop[tikv] funcs:count(test.t.a)->Column#5, funcs:sum(test.t.a)->Column#6", " └─IndexFullScan 10000.00 cop[tikv] table:t, index:ia(a) keep order:false, stats:pseudo" ], "Warn": null @@ -4339,9 +4335,9 @@ { "SQL": "desc format = 'brief' select /*+ read_from_storage(tiflash[t]) */ avg(a) from t", "Plan": [ - "StreamAgg 1.00 root funcs:avg(Column#7, Column#8)->Column#4", - "└─IndexReader 1.00 root index:StreamAgg", - " └─StreamAgg 1.00 cop[tikv] funcs:count(test.t.a)->Column#7, funcs:sum(test.t.a)->Column#8", + "HashAgg 1.00 root funcs:avg(Column#5, Column#6)->Column#4", + "└─IndexReader 1.00 root index:HashAgg", + " └─HashAgg 1.00 cop[tikv] funcs:count(test.t.a)->Column#5, funcs:sum(test.t.a)->Column#6", " └─IndexFullScan 10000.00 cop[tikv] table:t, index:ia(a) keep order:false, stats:pseudo" ], "Warn": [ @@ -7080,10 +7076,11 @@ " └─TableReader 8000.00 root data:ExchangeSender", " └─ExchangeSender 8000.00 mpp[tiflash] ExchangeType: PassThrough", " └─Projection 8000.00 mpp[tiflash] Column#5, test.t.id", - " └─HashAgg 8000.00 mpp[tiflash] group by:test.t.id, test.t.name, funcs:count(1)->Column#5, funcs:firstrow(test.t.id)->test.t.id", - " └─ExchangeReceiver 10000.00 mpp[tiflash] ", - " └─ExchangeSender 10000.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.name, collate: utf8mb4_bin], [name: test.t.id, collate: binary]", - " └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo" + " └─HashAgg 8000.00 mpp[tiflash] group by:test.t.id, test.t.name, funcs:sum(Column#7)->Column#5, funcs:firstrow(test.t.id)->test.t.id", + " └─ExchangeReceiver 8000.00 mpp[tiflash] ", + " └─ExchangeSender 8000.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.name, collate: utf8mb4_bin], [name: test.t.id, collate: binary]", + " └─HashAgg 8000.00 mpp[tiflash] group by:test.t.id, test.t.name, funcs:count(1)->Column#7", + " └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo" ] }, { @@ -7093,10 +7090,11 @@ "└─TableReader 8000.00 root data:ExchangeSender", " └─ExchangeSender 8000.00 mpp[tiflash] ExchangeType: PassThrough", " └─Projection 8000.00 mpp[tiflash] Column#5", - " └─HashAgg 8000.00 mpp[tiflash] group by:test.t.name, funcs:count(1)->Column#5", - " └─ExchangeReceiver 10000.00 mpp[tiflash] ", - " └─ExchangeSender 10000.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.name, collate: utf8mb4_bin]", - " └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo" + " └─HashAgg 8000.00 mpp[tiflash] group by:test.t.name, funcs:sum(Column#8)->Column#5", + " └─ExchangeReceiver 8000.00 mpp[tiflash] ", + " └─ExchangeSender 8000.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.name, collate: utf8mb4_bin]", + " └─HashAgg 8000.00 mpp[tiflash] group by:test.t.name, funcs:count(1)->Column#8", + " └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo" ] }, { @@ -7127,10 +7125,11 @@ " │ └─ExchangeSender 7992.00 mpp[tiflash] ExchangeType: Broadcast", " │ └─Projection 7992.00 mpp[tiflash] test.t.id", " │ └─HashAgg 7992.00 mpp[tiflash] group by:test.t.id, funcs:firstrow(test.t.id)->test.t.id", - " │ └─ExchangeReceiver 9990.00 mpp[tiflash] ", - " │ └─ExchangeSender 9990.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.id, collate: binary]", - " │ └─Selection 9990.00 mpp[tiflash] not(isnull(test.t.id))", - " │ └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo", + " │ └─ExchangeReceiver 7992.00 mpp[tiflash] ", + " │ └─ExchangeSender 7992.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.id, collate: binary]", + " │ └─HashAgg 7992.00 mpp[tiflash] group by:test.t.id, ", + " │ └─Selection 9990.00 mpp[tiflash] not(isnull(test.t.id))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo", " └─Selection(Probe) 9990.00 mpp[tiflash] not(isnull(test.t.id))", " └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo" ] @@ -7172,10 +7171,11 @@ " ├─Projection 8000.00 mpp[tiflash] cast(Column#12, bigint(21) BINARY)->Column#12", " │ └─Projection 8000.00 mpp[tiflash] Column#5", " │ └─Projection 8000.00 mpp[tiflash] Column#5, test.t.id", - " │ └─HashAgg 8000.00 mpp[tiflash] group by:test.t.id, test.t.name, funcs:count(1)->Column#5, funcs:firstrow(test.t.id)->test.t.id", - " │ └─ExchangeReceiver 10000.00 mpp[tiflash] ", - " │ └─ExchangeSender 10000.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.name, collate: utf8mb4_bin], [name: test.t.id, collate: binary]", - " │ └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo", + " │ └─HashAgg 8000.00 mpp[tiflash] group by:test.t.id, test.t.name, funcs:sum(Column#19)->Column#5, funcs:firstrow(test.t.id)->test.t.id", + " │ └─ExchangeReceiver 8000.00 mpp[tiflash] ", + " │ └─ExchangeSender 8000.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.name, collate: utf8mb4_bin], [name: test.t.id, collate: binary]", + " │ └─HashAgg 8000.00 mpp[tiflash] group by:test.t.id, test.t.name, funcs:count(1)->Column#19", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo", " └─Projection 10000.00 mpp[tiflash] cast(Column#11, bigint(21) BINARY)->Column#12", " └─Projection 10000.00 mpp[tiflash] plus(test.t.id, 1)->Column#11", " └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo" @@ -7320,11 +7320,12 @@ " └─ExchangeReceiver 2666.67 mpp[tiflash] stream_count: 8", " └─ExchangeSender 2666.67 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t1.c2, collate: binary], stream_count: 8", " └─Projection 2666.67 mpp[tiflash] Column#4, test.t1.c2", - " └─HashAgg 2666.67 mpp[tiflash] group by:test.t1.c1, funcs:count(test.t1.c2)->Column#4, funcs:firstrow(test.t1.c2)->test.t1.c2", - " └─ExchangeReceiver 3333.33 mpp[tiflash] ", - " └─ExchangeSender 3333.33 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t1.c1, collate: binary]", - " └─Selection 3333.33 mpp[tiflash] gt(test.t1.c1, 10)", - " └─TableFullScan 10000.00 mpp[tiflash] table:t1 keep order:false, stats:pseudo" + " └─HashAgg 2666.67 mpp[tiflash] group by:test.t1.c1, funcs:sum(Column#9)->Column#4, funcs:firstrow(Column#10)->test.t1.c2", + " └─ExchangeReceiver 2666.67 mpp[tiflash] ", + " └─ExchangeSender 2666.67 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t1.c1, collate: binary]", + " └─HashAgg 2666.67 mpp[tiflash] group by:test.t1.c1, funcs:count(test.t1.c2)->Column#9, funcs:firstrow(test.t1.c2)->Column#10", + " └─Selection 3333.33 mpp[tiflash] gt(test.t1.c1, 10)", + " └─TableFullScan 10000.00 mpp[tiflash] table:t1 keep order:false, stats:pseudo" ] }, { @@ -7338,11 +7339,12 @@ " └─ExchangeReceiver 2666.67 mpp[tiflash] stream_count: 8", " └─ExchangeSender 2666.67 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t1.c1, collate: binary], stream_count: 8", " └─Projection 2666.67 mpp[tiflash] Column#4, test.t1.c1, test.t1.c2", - " └─HashAgg 2666.67 mpp[tiflash] group by:test.t1.c2, funcs:count(test.t1.c1)->Column#4, funcs:firstrow(test.t1.c1)->test.t1.c1, funcs:firstrow(test.t1.c2)->test.t1.c2", - " └─ExchangeReceiver 3333.33 mpp[tiflash] ", - " └─ExchangeSender 3333.33 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t1.c2, collate: binary]", - " └─Selection 3333.33 mpp[tiflash] gt(test.t1.c2, 10)", - " └─TableFullScan 10000.00 mpp[tiflash] table:t1 keep order:false, stats:pseudo" + " └─HashAgg 2666.67 mpp[tiflash] group by:test.t1.c2, funcs:sum(Column#9)->Column#4, funcs:firstrow(Column#10)->test.t1.c1, funcs:firstrow(test.t1.c2)->test.t1.c2", + " └─ExchangeReceiver 2666.67 mpp[tiflash] ", + " └─ExchangeSender 2666.67 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t1.c2, collate: binary]", + " └─HashAgg 2666.67 mpp[tiflash] group by:test.t1.c2, funcs:count(test.t1.c1)->Column#9, funcs:firstrow(test.t1.c1)->Column#10", + " └─Selection 3333.33 mpp[tiflash] gt(test.t1.c2, 10)", + " └─TableFullScan 10000.00 mpp[tiflash] table:t1 keep order:false, stats:pseudo" ] }, { @@ -7408,11 +7410,12 @@ " └─Window 2666.67 mpp[tiflash] row_number()->Column#6 over(partition by test.t1.c1 order by test.t1.c2 rows between current row and current row)", " └─Sort 2666.67 mpp[tiflash] test.t1.c1, test.t1.c2", " └─Projection 2666.67 mpp[tiflash] Column#4, test.t1.c1, test.t1.c2", - " └─HashAgg 2666.67 mpp[tiflash] group by:test.t1.c1, funcs:count(test.t1.c2)->Column#4, funcs:firstrow(test.t1.c1)->test.t1.c1, funcs:firstrow(test.t1.c2)->test.t1.c2", - " └─ExchangeReceiver 3333.33 mpp[tiflash] ", - " └─ExchangeSender 3333.33 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t1.c1, collate: binary]", - " └─Selection 3333.33 mpp[tiflash] gt(test.t1.c1, 10)", - " └─TableFullScan 10000.00 mpp[tiflash] table:t1 keep order:false, stats:pseudo" + " └─HashAgg 2666.67 mpp[tiflash] group by:test.t1.c1, funcs:sum(Column#9)->Column#4, funcs:firstrow(test.t1.c1)->test.t1.c1, funcs:firstrow(Column#11)->test.t1.c2", + " └─ExchangeReceiver 2666.67 mpp[tiflash] ", + " └─ExchangeSender 2666.67 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t1.c1, collate: binary]", + " └─HashAgg 2666.67 mpp[tiflash] group by:test.t1.c1, funcs:count(test.t1.c2)->Column#9, funcs:firstrow(test.t1.c2)->Column#11", + " └─Selection 3333.33 mpp[tiflash] gt(test.t1.c1, 10)", + " └─TableFullScan 10000.00 mpp[tiflash] table:t1 keep order:false, stats:pseudo" ] } ] diff --git a/planner/core/testdata/join_reorder_suite_out.json b/planner/core/testdata/join_reorder_suite_out.json index 159862538cbe9..e17cbae1f3b3d 100644 --- a/planner/core/testdata/join_reorder_suite_out.json +++ b/planner/core/testdata/join_reorder_suite_out.json @@ -5320,11 +5320,11 @@ "│ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", "└─HashJoin(Probe) 12487.50 root right outer join, equal:[eq(test.t1.a, test.t2.a)]", " ├─TableReader(Build) 9990.00 root data:Selection", - " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t1.a))", - " │ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", " └─TableReader(Probe) 9990.00 root data:Selection", - " └─Selection 9990.00 cop[tikv] not(isnull(test.t2.b))", - " └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo" + " └─Selection 9990.00 cop[tikv] not(isnull(test.t1.a))", + " └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo" ], "Warning": [ "Warning 1815 leading hint is inapplicable, check if the leading hint table is valid", @@ -6333,8 +6333,8 @@ "SQL": "select /*+ straight_join() */ * from t1 join t2 on t1.a=t2.a where t1.a not in (select t3.a from t3)", "Plan": [ "HashJoin 9990.00 root CARTESIAN anti semi join, other cond:eq(test.t1.a, test.t3.a)", - "├─TableReader(Build) 3.00 root data:TableFullScan", - "│ └─TableFullScan 3.00 cop[tikv] table:t3 keep order:false", + "├─IndexReader(Build) 3.00 root index:IndexFullScan", + "│ └─IndexFullScan 3.00 cop[tikv] table:t3, index:a(a) keep order:false", "└─HashJoin(Probe) 12487.50 root inner join, equal:[eq(test.t1.a, test.t2.a)]", " ├─TableReader(Build) 9990.00 root data:Selection", " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", @@ -7394,8 +7394,8 @@ "SQL": "select /*+ leading(t4) */ * from t1 join t2 on t1.a=t2.a join t4 on t1.b = t4.b where t1.a not in (select t3.a from t3)", "Plan": [ "HashJoin 12475.01 root CARTESIAN anti semi join, other cond:eq(test.t1.a, test.t3.a)", - "├─TableReader(Build) 3.00 root data:TableFullScan", - "│ └─TableFullScan 3.00 cop[tikv] table:t3 keep order:false", + "├─IndexReader(Build) 3.00 root index:IndexFullScan", + "│ └─IndexFullScan 3.00 cop[tikv] table:t3, index:a(a) keep order:false", "└─Projection(Probe) 15593.77 root test.t1.a, test.t1.b, test.t2.a, test.t2.b, test.t4.a, test.t4.b", " └─HashJoin 15593.77 root inner join, equal:[eq(test.t1.a, test.t2.a)]", " ├─TableReader(Build) 9990.00 root data:Selection", @@ -7415,8 +7415,8 @@ "SQL": "select /*+ leading(t3@sel_2) */ * from t1 join t2 on t1.a=t2.a where t1.a not in (select t3.a from t3)", "Plan": [ "HashJoin 9990.00 root CARTESIAN anti semi join, other cond:eq(test.t1.a, test.t3.a)", - "├─TableReader(Build) 3.00 root data:TableFullScan", - "│ └─TableFullScan 3.00 cop[tikv] table:t3 keep order:false", + "├─IndexReader(Build) 3.00 root index:IndexFullScan", + "│ └─IndexFullScan 3.00 cop[tikv] table:t3, index:a(a) keep order:false", "└─HashJoin(Probe) 12487.50 root inner join, equal:[eq(test.t1.a, test.t2.a)]", " ├─TableReader(Build) 9990.00 root data:Selection", " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", @@ -7433,8 +7433,8 @@ "SQL": "select /*+ leading(t2, t3@sel_2) */ * from t1 join t2 on t1.a=t2.a where t1.a not in (select t3.a from t3)", "Plan": [ "HashJoin 9990.00 root CARTESIAN anti semi join, other cond:eq(test.t1.a, test.t3.a)", - "├─TableReader(Build) 3.00 root data:TableFullScan", - "│ └─TableFullScan 3.00 cop[tikv] table:t3 keep order:false", + "├─IndexReader(Build) 3.00 root index:IndexFullScan", + "│ └─IndexFullScan 3.00 cop[tikv] table:t3, index:a(a) keep order:false", "└─HashJoin(Probe) 12487.50 root inner join, equal:[eq(test.t1.a, test.t2.a)]", " ├─TableReader(Build) 9990.00 root data:Selection", " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", @@ -7452,8 +7452,8 @@ "SQL": "select /*+ leading(t1, t3@sel_2) */ * from t1 join t2 on t1.a=t2.a where t1.a not in (select t3.a from t3)", "Plan": [ "HashJoin 9990.00 root CARTESIAN anti semi join, other cond:eq(test.t1.a, test.t3.a)", - "├─TableReader(Build) 3.00 root data:TableFullScan", - "│ └─TableFullScan 3.00 cop[tikv] table:t3 keep order:false", + "├─IndexReader(Build) 3.00 root index:IndexFullScan", + "│ └─IndexFullScan 3.00 cop[tikv] table:t3, index:a(a) keep order:false", "└─HashJoin(Probe) 12487.50 root inner join, equal:[eq(test.t1.a, test.t2.a)]", " ├─TableReader(Build) 9990.00 root data:Selection", " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", @@ -7471,8 +7471,8 @@ "SQL": "select /*+ leading(t3@sel_2, t2) */ * from t1 join t2 on t1.a=t2.a where t1.a not in (select t3.a from t3)", "Plan": [ "HashJoin 9990.00 root CARTESIAN anti semi join, other cond:eq(test.t1.a, test.t3.a)", - "├─TableReader(Build) 3.00 root data:TableFullScan", - "│ └─TableFullScan 3.00 cop[tikv] table:t3 keep order:false", + "├─IndexReader(Build) 3.00 root index:IndexFullScan", + "│ └─IndexFullScan 3.00 cop[tikv] table:t3, index:a(a) keep order:false", "└─HashJoin(Probe) 12487.50 root inner join, equal:[eq(test.t1.a, test.t2.a)]", " ├─TableReader(Build) 9990.00 root data:Selection", " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", @@ -7490,8 +7490,8 @@ "SQL": "select /*+ leading(t3@sel_2, t1) */ * from t1 join t2 on t1.a=t2.a where t1.a not in (select t3.a from t3)", "Plan": [ "HashJoin 9990.00 root CARTESIAN anti semi join, other cond:eq(test.t1.a, test.t3.a)", - "├─TableReader(Build) 3.00 root data:TableFullScan", - "│ └─TableFullScan 3.00 cop[tikv] table:t3 keep order:false", + "├─IndexReader(Build) 3.00 root index:IndexFullScan", + "│ └─IndexFullScan 3.00 cop[tikv] table:t3, index:a(a) keep order:false", "└─HashJoin(Probe) 12487.50 root inner join, equal:[eq(test.t1.a, test.t2.a)]", " ├─TableReader(Build) 9990.00 root data:Selection", " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", diff --git a/planner/core/testdata/ordered_result_mode_suite_out.json b/planner/core/testdata/ordered_result_mode_suite_out.json index f6acccadce758..7f6563beaa1a8 100644 --- a/planner/core/testdata/ordered_result_mode_suite_out.json +++ b/planner/core/testdata/ordered_result_mode_suite_out.json @@ -183,12 +183,13 @@ { "Plan": [ "Sort_11 9990.00 root test.t1.a, test.t1.b, test.t1.c, test.t1.d", - "└─HashJoin_23 9990.00 root inner join, equal:[eq(test.t1.a, test.t2.b)]", - " ├─HashAgg_36(Build) 7992.00 root group by:test.t2.b, funcs:firstrow(test.t2.b)->test.t2.b", - " │ └─IndexReader_43 9990.00 root index:IndexFullScan_42", - " │ └─IndexFullScan_42 9990.00 cop[tikv] table:t2, index:b(b) keep order:false, stats:pseudo", - " └─TableReader_47(Probe) 10000.00 root data:TableFullScan_46", - " └─TableFullScan_46 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo" + "└─MergeJoin_13 9990.00 root inner join, left key:test.t1.a, right key:test.t2.b", + " ├─StreamAgg_30(Build) 7992.00 root group by:test.t2.b, funcs:firstrow(test.t2.b)->test.t2.b", + " │ └─IndexReader_31 7992.00 root index:StreamAgg_27", + " │ └─StreamAgg_27 7992.00 cop[tikv] group by:test.t2.b, ", + " │ └─IndexFullScan_29 9990.00 cop[tikv] table:t2, index:b(b) keep order:true, stats:pseudo", + " └─TableReader_26(Probe) 10000.00 root data:TableFullScan_25", + " └─TableFullScan_25 10000.00 cop[tikv] table:t1 keep order:true, stats:pseudo" ] }, { diff --git a/planner/core/testdata/plan_suite_out.json b/planner/core/testdata/plan_suite_out.json index 572f81f7b0b50..43a23f6cf075f 100644 --- a/planner/core/testdata/plan_suite_out.json +++ b/planner/core/testdata/plan_suite_out.json @@ -397,7 +397,7 @@ }, { "SQL": "select * from t where (t.c > 0 and t.c < 2) or (t.c > 4 and t.c < 6) or (t.c > 8 and t.c < 10) or (t.c > 12 and t.c < 14) or (t.c > 16 and t.c < 18)", - "Best": "IndexLookUp(Index(t.c_d_e)[[1,1] [5,5] [9,9] [13,13] [17,17]], Table(t))" + "Best": "TableReader(Table(t)->Sel([or(or(and(gt(test.t.c, 0), lt(test.t.c, 2)), and(gt(test.t.c, 4), lt(test.t.c, 6))), or(and(gt(test.t.c, 8), lt(test.t.c, 10)), or(and(gt(test.t.c, 12), lt(test.t.c, 14)), and(gt(test.t.c, 16), lt(test.t.c, 18)))))]))" }, { "SQL": "select * from t where (t.c > 0 and t.c < 1) or (t.c > 2 and t.c < 3) or (t.c > 4 and t.c < 5) or (t.c > 6 and t.c < 7) or (t.c > 9 and t.c < 10)", @@ -477,7 +477,7 @@ }, { "SQL": "select c from t where t.c = 1 and t.d = 1 order by t.a limit 1", - "Best": "IndexReader(Index(t.c_d_e)[[1 1,1 1]])->TopN([test.t.a],0,1)->Projection" + "Best": "IndexReader(Index(t.c_d_e)[[1 1,1 1]]->TopN([test.t.a],0,1))->TopN([test.t.a],0,1)->Projection" }, { "SQL": "select * from t where t.c = 1 and t.a > 1 order by t.d limit 1", @@ -935,7 +935,7 @@ "Cases": [ { "SQL": "select distinct b from t", - "Best": "TableReader(Table(t))->HashAgg" + "Best": "TableReader(Table(t)->HashAgg)->HashAgg" }, { "SQL": "select count(*) from (select * from t order by b) t group by b", @@ -959,15 +959,15 @@ }, { "SQL": "select sum(e), avg(e + c) from t where c = 1 group by c", - "Best": "IndexReader(Index(t.c_d_e)[[1,1]]->StreamAgg)->StreamAgg" + "Best": "IndexReader(Index(t.c_d_e)[[1,1]])->Projection->StreamAgg" }, { "SQL": "select sum(e), avg(e + c) from t where c = 1 group by e", - "Best": "IndexReader(Index(t.c_d_e)[[1,1]]->HashAgg)->HashAgg" + "Best": "IndexReader(Index(t.c_d_e)[[1,1]])->Projection->HashAgg" }, { "SQL": "select sum(e), avg(b + c) from t where c = 1 and e = 1 group by d", - "Best": "IndexLookUp(Index(t.c_d_e)[[1,1]]->Sel([eq(test.t.e, 1)]), Table(t))->Projection->HashAgg" + "Best": "IndexLookUp(Index(t.c_d_e)[[1,1]]->Sel([eq(test.t.e, 1)]), Table(t))->Projection->Projection->StreamAgg" }, { "SQL": "select sum(e), avg(b + c) from t where c = 1 and b = 1", @@ -975,11 +975,11 @@ }, { "SQL": "select sum(e) as k, avg(b + c) from t where c = 1 and b = 1 and e = 1 group by d order by k", - "Best": "IndexLookUp(Index(t.c_d_e)[[1,1]]->Sel([eq(test.t.e, 1)]), Table(t)->Sel([eq(test.t.b, 1)]))->Projection->HashAgg->Sort" + "Best": "IndexLookUp(Index(t.c_d_e)[[1,1]]->Sel([eq(test.t.e, 1)]), Table(t)->Sel([eq(test.t.b, 1)]))->Projection->Projection->StreamAgg->Sort" }, { "SQL": "select sum(e) as k, avg(b + c) from t where c = 1 and b = 1 and e = 1 group by c order by k", - "Best": "IndexLookUp(Index(t.c_d_e)[[1,1]]->Sel([eq(test.t.e, 1)]), Table(t)->Sel([eq(test.t.b, 1)]))->Projection->HashAgg->Sort" + "Best": "IndexLookUp(Index(t.c_d_e)[[1,1]]->Sel([eq(test.t.e, 1)]), Table(t)->Sel([eq(test.t.b, 1)]))->Projection->Projection->StreamAgg->Sort" }, { "SQL": "select sum(to_base64(e)) from t where c = 1", @@ -987,7 +987,7 @@ }, { "SQL": "select (select count(1) k from t s where s.a = t.a having k != 0) from t", - "Best": "LeftHashJoin{IndexReader(Index(t.f)[[NULL,+inf]])->IndexReader(Index(t.f)[[NULL,+inf]])->Projection}(test.t.a,test.t.a)->Projection" + "Best": "MergeLeftOuterJoin{TableReader(Table(t))->TableReader(Table(t))->Projection}(test.t.a,test.t.a)->Projection" }, { "SQL": "select sum(to_base64(e)) from t group by e,d,c order by c", @@ -1104,7 +1104,7 @@ }, { "SQL": "select b from t where c = 1 or c = 2 or c = 3 or c = 4 or c = 5", - "Best": "IndexLookUp(Index(t.c_d_e)[[1,5]], Table(t))->Projection" + "Best": "TableReader(Table(t)->Sel([or(or(eq(test.t.c, 1), eq(test.t.c, 2)), or(eq(test.t.c, 3), or(eq(test.t.c, 4), eq(test.t.c, 5))))]))->Projection" }, { "SQL": "select a from t where c = 5", @@ -1528,13 +1528,13 @@ ], "Plan": [ "HashJoin 12487.50 root inner join, equal:[eq(test.t.a, test.t.b)]", - "├─UnionScan(Build) 9990.00 root not(isnull(test.t.b))", - "│ └─TableReader 9990.00 root data:Selection", - "│ └─Selection 9990.00 cop[tikv] not(isnull(test.t.b))", - "│ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", - "└─UnionScan(Probe) 10000.00 root ", - " └─TableReader 10000.00 root data:TableFullScan", - " └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo" + "├─UnionScan(Build) 10000.00 root ", + "│ └─TableReader 10000.00 root data:TableFullScan", + "│ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", + "└─UnionScan(Probe) 9990.00 root not(isnull(test.t.b))", + " └─TableReader 9990.00 root data:Selection", + " └─Selection 9990.00 cop[tikv] not(isnull(test.t.b))", + " └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo" ] }, { @@ -1588,7 +1588,7 @@ "Cases": [ { "SQL": "select t1.a, (select count(t2.a) from t t2 where t2.g in (select t3.d from t t3 where t3.c = t1.a)) as agg_col from t t1;", - "Best": "Apply{IndexReader(Index(t.f)[[NULL,+inf]])->IndexHashJoin{IndexReader(Index(t.c_d_e)[[NULL,+inf]]->HashAgg)->HashAgg->IndexReader(Index(t.g)[[NULL,NULL]])}(test.t.d,test.t.g)}->HashAgg" + "Best": "Apply{IndexReader(Index(t.f)[[NULL,+inf]])->IndexJoin{IndexReader(Index(t.c_d_e)[[NULL,+inf]])->HashAgg->IndexReader(Index(t.g)[[NULL,NULL]])}(test.t.d,test.t.g)}->HashAgg" } ] }, @@ -1609,11 +1609,11 @@ }, { "SQL": "select /*+ inl_merge_join(t2) */ t1.a, t2.a from t t1 left join t t2 use index(g_2) on t1.g=t2.g", - "Plan": "IndexMergeJoin{IndexReader(Index(t.g_3)[[NULL,+inf]])->IndexReader(Index(t.g_2)[[NULL,NULL]]->Sel([not(isnull(test.t.g))]))}(test.t.g,test.t.g)" + "Plan": "IndexMergeJoin{IndexReader(Index(t.g_2)[[NULL,+inf]])->IndexReader(Index(t.g_2)[[NULL,NULL]]->Sel([not(isnull(test.t.g))]))}(test.t.g,test.t.g)" }, { "SQL": "select /*+inl_merge_join(t2)*/ t1.a, t2.a from t t1 left join t t2 use index(g_2) on t1.g=t2.g order by t1.a", - "Plan": "IndexMergeJoin{IndexReader(Index(t.g_3)[[NULL,+inf]])->IndexReader(Index(t.g_2)[[NULL,NULL]]->Sel([not(isnull(test.t.g))]))}(test.t.g,test.t.g)->Sort" + "Plan": "IndexMergeJoin{IndexReader(Index(t.g_2)[[NULL,+inf]])->IndexReader(Index(t.g_2)[[NULL,NULL]]->Sel([not(isnull(test.t.g))]))}(test.t.g,test.t.g)->Sort" } ] }, @@ -1995,8 +1995,9 @@ "Plan": [ "HashJoin 80000000.00 root CARTESIAN inner join", "├─HashAgg(Build) 8000.00 root group by:test.t.a, funcs:firstrow(test.t.a)->test.t.a", - "│ └─TableReader 10000.00 root data:TableFullScan", - "│ └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo", + "│ └─TableReader 8000.00 root data:HashAgg", + "│ └─HashAgg 8000.00 cop[tikv] group by:test.t.a, ", + "│ └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo", "└─TableReader(Probe) 10000.00 root data:TableFullScan", " └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" ], @@ -3362,11 +3363,11 @@ "SQL": "select date_format(d,'%Y') as df, sum(a), count(b), count(distinct c) from t group by date_format(d,'%Y')", "Plan": [ "Projection 8000.00 root date_format(test.t.d, %Y)->Column#9, Column#6, cast(Column#13, bigint(21) BINARY)->Column#7, Column#8", - "└─HashAgg 8000.00 root group by:Column#29, funcs:sum(Column#25)->Column#6, funcs:sum(Column#26)->Column#13, funcs:count(Column#27)->Column#8, funcs:firstrow(Column#28)->test.t.d", - " └─Projection 8000.00 root Column#11, cast(Column#12, decimal(20,0) BINARY)->Column#26, test.t.c, test.t.d, date_format(test.t.d, %Y)->Column#29", - " └─HashAgg 8000.00 root group by:Column#23, Column#24, funcs:sum(Column#19)->Column#11, funcs:count(Column#20)->Column#12, funcs:firstrow(Column#21)->test.t.c, funcs:firstrow(Column#22)->test.t.d", - " └─Projection 10000.00 root cast(test.t.a, decimal(10,0) BINARY)->Column#19, test.t.b, test.t.c, test.t.d, date_format(test.t.d, %Y)->Column#23, test.t.c", - " └─TableReader 10000.00 root data:TableFullScan", + "└─HashAgg 8000.00 root group by:Column#23, funcs:sum(Column#19)->Column#6, funcs:sum(Column#20)->Column#13, funcs:count(Column#21)->Column#8, funcs:firstrow(Column#22)->test.t.d", + " └─Projection 8000.00 root Column#11, cast(Column#12, decimal(20,0) BINARY)->Column#20, test.t.c, test.t.d, date_format(test.t.d, %Y)->Column#23", + " └─HashAgg 8000.00 root group by:Column#14, test.t.c, funcs:sum(Column#15)->Column#11, funcs:count(Column#16)->Column#12, funcs:firstrow(test.t.c)->test.t.c, funcs:firstrow(Column#18)->test.t.d", + " └─TableReader 8000.00 root data:HashAgg", + " └─HashAgg 8000.00 cop[tikv] group by:date_format(test.t.d, \"%Y\"), test.t.c, funcs:sum(test.t.a)->Column#15, funcs:count(test.t.b)->Column#16, funcs:firstrow(test.t.d)->Column#18", " └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" ] }, @@ -3376,9 +3377,10 @@ "Projection 8000.00 root test.t.d, test.t.a, cast(Column#10, bigint(21) BINARY)->Column#6, cast(Column#12, bigint(21) BINARY)->Column#7, Column#8", "└─HashAgg 8000.00 root group by:Column#23, Column#24, funcs:sum(Column#18)->Column#10, funcs:sum(Column#19)->Column#12, funcs:count(Column#20)->Column#8, funcs:firstrow(Column#21)->test.t.a, funcs:firstrow(Column#22)->test.t.d", " └─Projection 8000.00 root cast(Column#9, decimal(20,0) BINARY)->Column#18, cast(Column#11, decimal(20,0) BINARY)->Column#19, test.t.c, test.t.a, test.t.d, test.t.d, test.t.a", - " └─HashAgg 8000.00 root group by:test.t.a, test.t.c, test.t.d, funcs:count(1)->Column#9, funcs:count(test.t.b)->Column#11, funcs:firstrow(test.t.c)->test.t.c, funcs:firstrow(test.t.a)->test.t.a, funcs:firstrow(test.t.d)->test.t.d", - " └─TableReader 10000.00 root data:TableFullScan", - " └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" + " └─HashAgg 8000.00 root group by:test.t.a, test.t.c, test.t.d, funcs:count(Column#13)->Column#9, funcs:count(Column#14)->Column#11, funcs:firstrow(test.t.c)->test.t.c, funcs:firstrow(test.t.a)->test.t.a, funcs:firstrow(test.t.d)->test.t.d", + " └─TableReader 8000.00 root data:HashAgg", + " └─HashAgg 8000.00 cop[tikv] group by:test.t.a, test.t.c, test.t.d, funcs:count(1)->Column#13, funcs:count(test.t.b)->Column#14", + " └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" ] }, { @@ -3874,9 +3876,10 @@ "Plan": [ "HashJoin 9990.00 root inner join, equal:[eq(test.t.a, test.t.a)]", "├─HashAgg(Build) 7992.00 root group by:test.t.a, funcs:firstrow(test.t.a)->test.t.a", - "│ └─TableReader 9990.00 root data:Selection", - "│ └─Selection 9990.00 cop[tikv] not(isnull(test.t.a))", - "│ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", + "│ └─TableReader 7992.00 root data:HashAgg", + "│ └─HashAgg 7992.00 cop[tikv] group by:test.t.a, ", + "│ └─Selection 9990.00 cop[tikv] not(isnull(test.t.a))", + "│ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", "└─TableReader(Probe) 9990.00 root data:Selection", " └─Selection 9990.00 cop[tikv] not(isnull(test.t.a))", " └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" @@ -3888,9 +3891,10 @@ "Plan": [ "HashJoin 9990.00 root inner join, equal:[eq(test.t.a, test.t.a)]", "├─HashAgg(Build) 7992.00 root group by:test.t.a, funcs:firstrow(test.t.a)->test.t.a", - "│ └─TableReader 9990.00 root data:Selection", - "│ └─Selection 9990.00 cop[tikv] not(isnull(test.t.a))", - "│ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", + "│ └─TableReader 7992.00 root data:HashAgg", + "│ └─HashAgg 7992.00 cop[tikv] group by:test.t.a, ", + "│ └─Selection 9990.00 cop[tikv] not(isnull(test.t.a))", + "│ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", "└─TableReader(Probe) 9990.00 root data:Selection", " └─Selection 9990.00 cop[tikv] not(isnull(test.t.a))", " └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" @@ -5167,9 +5171,10 @@ "SQL": "SELECT /*+ hash_join_probe(t1) */ * FROM t1 WHERE EXISTS (SELECT /*+ SEMI_JOIN_REWRITE() */ 1 FROM t2 WHERE t2.a = t1.a);", "Plan": [ "HashJoin 10000.00 root inner join, equal:[eq(test.t1.a, test.t2.a)]", - "├─HashAgg(Build) 8000.00 root group by:test.t2.a, funcs:firstrow(test.t2.a)->test.t2.a", - "│ └─TableReader 10000.00 root data:TableFullScan", - "│ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + "├─StreamAgg(Build) 8000.00 root group by:test.t2.a, funcs:firstrow(test.t2.a)->test.t2.a", + "│ └─TableReader 8000.00 root data:StreamAgg", + "│ └─StreamAgg 8000.00 cop[tikv] group by:test.t2.a, ", + "│ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:true, stats:pseudo", "└─TableReader(Probe) 10000.00 root data:TableFullScan", " └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo" ], @@ -5185,9 +5190,10 @@ "HashJoin 10000.00 root inner join, equal:[eq(test.t1.a, test.t2.a)]", "├─TableReader(Build) 10000.00 root data:TableFullScan", "│ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", - "└─HashAgg(Probe) 8000.00 root group by:test.t2.a, funcs:firstrow(test.t2.a)->test.t2.a", - " └─TableReader 10000.00 root data:TableFullScan", - " └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo" + "└─StreamAgg(Probe) 8000.00 root group by:test.t2.a, funcs:firstrow(test.t2.a)->test.t2.a", + " └─TableReader 8000.00 root data:StreamAgg", + " └─StreamAgg 8000.00 cop[tikv] group by:test.t2.a, ", + " └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:true, stats:pseudo" ], "Result": [ "1 1", @@ -5235,9 +5241,10 @@ "HashJoin 10000.00 root inner join, equal:[eq(test.t1.a, test.t2.a)]", "├─TableReader(Build) 10000.00 root data:TableFullScan", "│ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", - "└─HashAgg(Probe) 8000.00 root group by:test.t2.a, funcs:firstrow(test.t2.a)->test.t2.a", - " └─TableReader 10000.00 root data:TableFullScan", - " └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo" + "└─StreamAgg(Probe) 8000.00 root group by:test.t2.a, funcs:firstrow(test.t2.a)->test.t2.a", + " └─TableReader 8000.00 root data:StreamAgg", + " └─StreamAgg 8000.00 cop[tikv] group by:test.t2.a, ", + " └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:true, stats:pseudo" ], "Result": [ "1 1", @@ -5249,9 +5256,10 @@ "SQL": "SELECT /*+ hash_join_build(t2@sel_2) */ * FROM t1 WHERE EXISTS (SELECT /*+ SEMI_JOIN_REWRITE() */ 1 FROM t2 WHERE t2.a = t1.a);", "Plan": [ "HashJoin 10000.00 root inner join, equal:[eq(test.t1.a, test.t2.a)]", - "├─HashAgg(Build) 8000.00 root group by:test.t2.a, funcs:firstrow(test.t2.a)->test.t2.a", - "│ └─TableReader 10000.00 root data:TableFullScan", - "│ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + "├─StreamAgg(Build) 8000.00 root group by:test.t2.a, funcs:firstrow(test.t2.a)->test.t2.a", + "│ └─TableReader 8000.00 root data:StreamAgg", + "│ └─StreamAgg 8000.00 cop[tikv] group by:test.t2.a, ", + "│ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:true, stats:pseudo", "└─TableReader(Probe) 10000.00 root data:TableFullScan", " └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo" ], diff --git a/planner/core/testdata/stats_suite_out.json b/planner/core/testdata/stats_suite_out.json index 2fcc3c6518425..38eb118c4556c 100644 --- a/planner/core/testdata/stats_suite_out.json +++ b/planner/core/testdata/stats_suite_out.json @@ -218,23 +218,23 @@ { "SQL": "select count(1) from t1 left join t2 on t1.a = t2.a group by t1.a, t1.b", "Plan": [ - "HashAgg 4.00 root group by:test.t1.a, test.t1.b, funcs:count(1)->Column#7", - "└─HashJoin 12.00 root left outer join, equal:[eq(test.t1.a, test.t2.a)]", - " ├─TableReader(Build) 4.00 root data:TableFullScan", - " │ └─TableFullScan 4.00 cop[tikv] table:t1 keep order:false", - " └─TableReader(Probe) 9.00 root data:TableFullScan", - " └─TableFullScan 9.00 cop[tikv] table:t2 keep order:false" + "StreamAgg 4.00 root group by:test.t1.a, test.t1.b, funcs:count(1)->Column#7", + "└─IndexJoin 12.00 root left outer join, inner:IndexReader, outer key:test.t1.a, inner key:test.t2.a, equal cond:eq(test.t1.a, test.t2.a)", + " ├─IndexReader(Build) 4.00 root index:IndexFullScan", + " │ └─IndexFullScan 4.00 cop[tikv] table:t1, index:a(a, b) keep order:true", + " └─IndexReader(Probe) 12.00 root index:IndexRangeScan", + " └─IndexRangeScan 12.00 cop[tikv] table:t2, index:a(a, b) range: decided by [eq(test.t2.a, test.t1.a)], keep order:false" ] }, { "SQL": "select count(1) from t1 right join t2 on t1.a = t2.a group by t2.a, t2.b", "Plan": [ - "HashAgg 9.00 root group by:test.t2.a, test.t2.b, funcs:count(1)->Column#7", - "└─HashJoin 12.00 root right outer join, equal:[eq(test.t1.a, test.t2.a)]", - " ├─TableReader(Build) 4.00 root data:TableFullScan", - " │ └─TableFullScan 4.00 cop[tikv] table:t1 keep order:false", - " └─TableReader(Probe) 9.00 root data:TableFullScan", - " └─TableFullScan 9.00 cop[tikv] table:t2 keep order:false" + "StreamAgg 9.00 root group by:test.t2.a, test.t2.b, funcs:count(1)->Column#7", + "└─IndexJoin 12.00 root right outer join, inner:IndexReader, outer key:test.t2.a, inner key:test.t1.a, equal cond:eq(test.t2.a, test.t1.a)", + " ├─IndexReader(Build) 9.00 root index:IndexFullScan", + " │ └─IndexFullScan 9.00 cop[tikv] table:t2, index:a(a, b) keep order:true", + " └─IndexReader(Probe) 12.00 root index:IndexRangeScan", + " └─IndexRangeScan 12.00 cop[tikv] table:t1, index:a(a, b) range: decided by [eq(test.t1.a, test.t2.a)], keep order:false" ] }, { @@ -242,10 +242,10 @@ "Plan": [ "HashAgg 4.00 root group by:test.t1.a, test.t1.b, funcs:count(Column#10)->Column#11", "└─HashJoin 4.00 root left outer semi join, equal:[eq(test.t1.b, test.t2.b)], other cond:gt(test.t2.a, test.t1.a)", - " ├─TableReader(Build) 9.00 root data:TableFullScan", - " │ └─TableFullScan 9.00 cop[tikv] table:t2 keep order:false", - " └─TableReader(Probe) 4.00 root data:TableFullScan", - " └─TableFullScan 4.00 cop[tikv] table:t1 keep order:false" + " ├─IndexReader(Build) 9.00 root index:IndexFullScan", + " │ └─IndexFullScan 9.00 cop[tikv] table:t2, index:a(a, b) keep order:false", + " └─IndexReader(Probe) 4.00 root index:IndexFullScan", + " └─IndexFullScan 4.00 cop[tikv] table:t1, index:a(a, b) keep order:false" ] }, { @@ -253,10 +253,10 @@ "Plan": [ "HashAgg 4.00 root group by:test.t1.a, test.t1.b, funcs:count(Column#10)->Column#11", "└─HashJoin 4.00 root anti left outer semi join, equal:[eq(test.t1.b, test.t2.b)], other cond:gt(test.t2.a, test.t1.a)", - " ├─TableReader(Build) 9.00 root data:TableFullScan", - " │ └─TableFullScan 9.00 cop[tikv] table:t2 keep order:false", - " └─TableReader(Probe) 4.00 root data:TableFullScan", - " └─TableFullScan 4.00 cop[tikv] table:t1 keep order:false" + " ├─IndexReader(Build) 9.00 root index:IndexFullScan", + " │ └─IndexFullScan 9.00 cop[tikv] table:t2, index:a(a, b) keep order:false", + " └─IndexReader(Probe) 4.00 root index:IndexFullScan", + " └─IndexFullScan 4.00 cop[tikv] table:t1, index:a(a, b) keep order:false" ] }, { @@ -276,8 +276,8 @@ "Plan": [ "HashAgg 4.00 root group by:test.t1.a, test.t1.b, funcs:count(Column#5)->Column#6", "└─Window 4.00 root sum(cast(test.t1.a, decimal(10,0) BINARY))->Column#5 over()", - " └─TableReader 4.00 root data:TableFullScan", - " └─TableFullScan 4.00 cop[tikv] table:t1 keep order:false" + " └─IndexReader 4.00 root index:IndexFullScan", + " └─IndexFullScan 4.00 cop[tikv] table:t1, index:a(a, b) keep order:false" ] } ] diff --git a/planner/core/testdata/window_push_down_suite_out.json b/planner/core/testdata/window_push_down_suite_out.json index e4b3aa37c09a6..0b587eb83ce39 100644 --- a/planner/core/testdata/window_push_down_suite_out.json +++ b/planner/core/testdata/window_push_down_suite_out.json @@ -343,11 +343,12 @@ "TableReader_84 8000.00 root data:ExchangeSender_83", "└─ExchangeSender_83 8000.00 mpp[tiflash] ExchangeType: PassThrough", " └─Window_82 8000.00 mpp[tiflash] row_number()->Column#7 over(rows between current row and current row)", - " └─Projection_19 8000.00 mpp[tiflash] Column#5", - " └─HashAgg_12 8000.00 mpp[tiflash] group by:test.employee.deptid, funcs:count(test.employee.empid)->Column#5", - " └─ExchangeReceiver_18 10000.00 mpp[tiflash] ", - " └─ExchangeSender_17 10000.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.employee.deptid, collate: binary]", - " └─TableFullScan_16 10000.00 mpp[tiflash] table:employee keep order:false, stats:pseudo" + " └─Projection_21 8000.00 mpp[tiflash] Column#5", + " └─HashAgg_22 8000.00 mpp[tiflash] group by:test.employee.deptid, funcs:sum(Column#8)->Column#5", + " └─ExchangeReceiver_24 8000.00 mpp[tiflash] ", + " └─ExchangeSender_23 8000.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.employee.deptid, collate: binary]", + " └─HashAgg_13 8000.00 mpp[tiflash] group by:test.employee.deptid, funcs:count(test.employee.empid)->Column#8", + " └─TableFullScan_20 10000.00 mpp[tiflash] table:employee keep order:false, stats:pseudo" ], "Warn": null }, @@ -378,12 +379,13 @@ "TableReader_61 8000.00 root data:ExchangeSender_60", "└─ExchangeSender_60 8000.00 mpp[tiflash] ExchangeType: PassThrough", " └─Window_59 8000.00 mpp[tiflash] row_number()->Column#7 over(partition by test.employee.deptid rows between current row and current row)", - " └─Sort_18 8000.00 mpp[tiflash] test.employee.deptid", - " └─Projection_17 8000.00 mpp[tiflash] Column#5, test.employee.deptid", - " └─HashAgg_12 8000.00 mpp[tiflash] group by:test.employee.deptid, funcs:count(test.employee.empid)->Column#5, funcs:firstrow(test.employee.deptid)->test.employee.deptid", - " └─ExchangeReceiver_16 10000.00 mpp[tiflash] ", - " └─ExchangeSender_15 10000.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.employee.deptid, collate: binary]", - " └─TableFullScan_14 10000.00 mpp[tiflash] table:employee keep order:false, stats:pseudo" + " └─Sort_24 8000.00 mpp[tiflash] test.employee.deptid", + " └─Projection_20 8000.00 mpp[tiflash] Column#5, test.employee.deptid", + " └─HashAgg_21 8000.00 mpp[tiflash] group by:test.employee.deptid, funcs:sum(Column#8)->Column#5, funcs:firstrow(test.employee.deptid)->test.employee.deptid", + " └─ExchangeReceiver_23 8000.00 mpp[tiflash] ", + " └─ExchangeSender_22 8000.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.employee.deptid, collate: binary]", + " └─HashAgg_13 8000.00 mpp[tiflash] group by:test.employee.deptid, funcs:count(test.employee.empid)->Column#8", + " └─TableFullScan_19 10000.00 mpp[tiflash] table:employee keep order:false, stats:pseudo" ], "Warn": null }, @@ -441,15 +443,16 @@ "Plan": [ "TableReader_54 10000.00 root data:ExchangeSender_53", "└─ExchangeSender_53 10000.00 mpp[tiflash] ExchangeType: PassThrough", - " └─Projection_48 10000.00 mpp[tiflash] Column#7", - " └─HashAgg_46 10000.00 mpp[tiflash] group by:Column#6, funcs:count(test.employee.empid)->Column#7", - " └─ExchangeReceiver_32 10000.00 mpp[tiflash] ", - " └─ExchangeSender_31 10000.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: Column#6, collate: binary]", - " └─Window_30 10000.00 mpp[tiflash] row_number()->Column#6 over(partition by test.employee.deptid rows between current row and current row), stream_count: 8", - " └─Sort_21 10000.00 mpp[tiflash] test.employee.deptid, stream_count: 8", - " └─ExchangeReceiver_20 10000.00 mpp[tiflash] stream_count: 8", - " └─ExchangeSender_19 10000.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.employee.deptid, collate: binary], stream_count: 8", - " └─TableFullScan_18 10000.00 mpp[tiflash] table:employee keep order:false, stats:pseudo" + " └─Projection_49 10000.00 mpp[tiflash] Column#7", + " └─HashAgg_50 10000.00 mpp[tiflash] group by:Column#6, funcs:sum(Column#10)->Column#7", + " └─ExchangeReceiver_52 10000.00 mpp[tiflash] ", + " └─ExchangeSender_51 10000.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: Column#6, collate: binary]", + " └─HashAgg_47 10000.00 mpp[tiflash] group by:Column#6, funcs:count(test.employee.empid)->Column#10", + " └─Window_36 10000.00 mpp[tiflash] row_number()->Column#6 over(partition by test.employee.deptid rows between current row and current row), stream_count: 8", + " └─Sort_21 10000.00 mpp[tiflash] test.employee.deptid, stream_count: 8", + " └─ExchangeReceiver_20 10000.00 mpp[tiflash] stream_count: 8", + " └─ExchangeSender_19 10000.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.employee.deptid, collate: binary], stream_count: 8", + " └─TableFullScan_18 10000.00 mpp[tiflash] table:employee keep order:false, stats:pseudo" ], "Warn": null }, diff --git a/planner/core/window_push_down_test.go b/planner/core/window_push_down_test.go index 599fe113ff1c5..0d8538fa19ca0 100644 --- a/planner/core/window_push_down_test.go +++ b/planner/core/window_push_down_test.go @@ -110,6 +110,7 @@ func TestWindowPlanWithOtherOperators(t *testing.T) { dom := domain.GetDomain(tk.Session()) tk.MustExec("use test") + tk.MustExec("set tidb_cost_model_version=2") tk.MustExec("drop table if exists employee") tk.MustExec("create table employee (empid int, deptid int, salary decimal(10,2))") SetTiFlashReplica(t, dom, "test", "employee") diff --git a/planner/optimize.go b/planner/optimize.go index 85a35533ffbf8..747dd5541596a 100644 --- a/planner/optimize.go +++ b/planner/optimize.go @@ -395,6 +395,8 @@ func optimize(ctx context.Context, sctx sessionctx.Context, node ast.Node, is in beginOpt := time.Now() finalPlan, cost, err := core.DoOptimize(ctx, sctx, builder.GetOptFlag(), logic) + // TODO: capture plan replayer here if it matches sql and plan digest + sctx.GetSessionVars().DurationOptimization = time.Since(beginOpt) return finalPlan, names, cost, err } @@ -441,7 +443,11 @@ func buildLogicalPlan(ctx context.Context, sctx sessionctx.Context, node ast.Nod return nil, err } sctx.GetSessionVars().RewritePhaseInfo.DurationRewrite = time.Since(beginRewrite) - sctx.GetSessionVars().StmtCtx.Tables = builder.GetDBTableInfo() + if exec, ok := p.(*core.Execute); ok && exec.PrepStmt != nil { + sctx.GetSessionVars().StmtCtx.Tables = core.GetDBTableInfo(exec.PrepStmt.VisitInfos) + } else { + sctx.GetSessionVars().StmtCtx.Tables = core.GetDBTableInfo(builder.GetVisitInfo()) + } return p, nil } diff --git a/server/BUILD.bazel b/server/BUILD.bazel index abdcf02bdd7a1..56b3741259f81 100644 --- a/server/BUILD.bazel +++ b/server/BUILD.bazel @@ -122,7 +122,7 @@ go_library( go_test( name = "server_test", - timeout = "long", + timeout = "moderate", srcs = [ "column_test.go", "conn_stmt_test.go", diff --git a/server/conn.go b/server/conn.go index 6a466fbffae5f..b319679fdbf14 100644 --- a/server/conn.go +++ b/server/conn.go @@ -1121,8 +1121,8 @@ func (cc *clientConn) Run(ctx context.Context) { startTime := time.Now() err = cc.dispatch(ctx, data) + cc.ctx.GetSessionVars().ClearAlloc(&cc.chunkAlloc, err != nil) cc.chunkAlloc.Reset() - cc.ctx.GetSessionVars().ClearAlloc() if err != nil { cc.audit(plugin.Error) // tell the plugin API there was a dispatch error if terror.ErrorEqual(err, io.EOF) { @@ -1908,7 +1908,7 @@ func (cc *clientConn) handleQuery(ctx context.Context, sql string) (err error) { pointPlans, err = cc.prefetchPointPlanKeys(ctx, stmts) if err != nil { for _, stmt := range stmts { - cc.onExtensionStmtEnd(stmt, err) + cc.onExtensionStmtEnd(stmt, false, err) } return err } @@ -1919,12 +1919,18 @@ func (cc *clientConn) handleQuery(ctx context.Context, sql string) (err error) { } var retryable bool var lastStmt ast.StmtNode + var expiredStmtTaskID uint64 for i, stmt := range stmts { if lastStmt != nil { - cc.onExtensionStmtEnd(lastStmt, nil) + cc.onExtensionStmtEnd(lastStmt, true, nil) } - lastStmt = stmt + + // expiredTaskID is the task ID of the previous statement. When executing a stmt, + // the StmtCtx will be reinit and the TaskID will change. We can compare the StmtCtx.TaskID + // with the previous one to determine whether StmtCtx has been inited for the current stmt. + expiredStmtTaskID = sessVars.StmtCtx.TaskID + if len(pointPlans) > 0 { // Save the point plan in Session, so we don't need to build the point plan again. cc.ctx.SetValue(plannercore.PointPlanKey, plannercore.PointPlanVal{Plan: pointPlans[i]}) @@ -1966,7 +1972,7 @@ func (cc *clientConn) handleQuery(ctx context.Context, sql string) (err error) { } if lastStmt != nil { - cc.onExtensionStmtEnd(lastStmt, err) + cc.onExtensionStmtEnd(lastStmt, sessVars.StmtCtx.TaskID != expiredStmtTaskID, err) } return err diff --git a/server/conn_stmt.go b/server/conn_stmt.go index 26dfd9f38d439..436b2d1721ca0 100644 --- a/server/conn_stmt.go +++ b/server/conn_stmt.go @@ -205,8 +205,13 @@ func (cc *clientConn) handleStmtExecute(ctx context.Context, data []byte) (err e } } + sessVars := cc.ctx.GetSessionVars() + // expiredTaskID is the task ID of the previous statement. When executing a stmt, + // the StmtCtx will be reinit and the TaskID will change. We can compare the StmtCtx.TaskID + // with the previous one to determine whether StmtCtx has been inited for the current stmt. + expiredTaskID := sessVars.StmtCtx.TaskID err = cc.executePlanCacheStmt(ctx, stmt, args, useCursor) - cc.onExtensionBinaryExecuteEnd(stmt, args, err) + cc.onExtensionBinaryExecuteEnd(stmt, args, sessVars.StmtCtx.TaskID != expiredTaskID, err) return err } @@ -302,7 +307,7 @@ const ( func (cc *clientConn) handleStmtFetch(ctx context.Context, data []byte) (err error) { cc.ctx.GetSessionVars().StartTime = time.Now() - cc.ctx.GetSessionVars().ClearAlloc() + cc.ctx.GetSessionVars().ClearAlloc(nil, false) stmtID, fetchSize, err := parseStmtFetchCmd(data) if err != nil { diff --git a/server/conn_test.go b/server/conn_test.go index 7ac9d4f9a6774..84db98f9e1827 100644 --- a/server/conn_test.go +++ b/server/conn_test.go @@ -858,6 +858,7 @@ func TestTiFlashFallback(t *testing.T) { cc.setCtx(&TiDBContext{Session: tk.Session(), stmts: make(map[int]*TiDBStatement)}) tk.MustExec("drop table if exists t") + tk.MustExec("set tidb_cost_model_version=1") tk.MustExec("create table t(a int not null primary key, b int not null)") tk.MustExec("alter table t set tiflash replica 1") tb := external.GetTableByName(t, tk, "test", "t") diff --git a/server/extension.go b/server/extension.go index 9973226cff841..c7cf018eb85a8 100644 --- a/server/extension.go +++ b/server/extension.go @@ -54,7 +54,7 @@ func (cc *clientConn) onExtensionConnEvent(tp extension.ConnEventTp, err error) cc.extensions.OnConnectionEvent(tp, info) } -func (cc *clientConn) onExtensionStmtEnd(node interface{}, err error, args ...expression.Expression) { +func (cc *clientConn) onExtensionStmtEnd(node interface{}, stmtCtxValid bool, err error, args ...expression.Expression) { if !cc.extensions.HasStmtEventListeners() { return } @@ -91,8 +91,8 @@ func (cc *clientConn) onExtensionStmtEnd(node interface{}, err error, args ...ex info.stmtNode = stmt } - if sc := sessVars.StmtCtx; sc != nil && err == nil { - info.sc = sc + if stmtCtxValid { + info.sc = sessVars.StmtCtx } else { info.sc = &stmtctx.StatementContext{} } @@ -112,8 +112,8 @@ func (cc *clientConn) onExtensionSQLParseFailed(sql string, err error) { }) } -func (cc *clientConn) onExtensionBinaryExecuteEnd(prep PreparedStatement, args []expression.Expression, err error) { - cc.onExtensionStmtEnd(prep, err, args...) +func (cc *clientConn) onExtensionBinaryExecuteEnd(prep PreparedStatement, args []expression.Expression, stmtCtxValid bool, err error) { + cc.onExtensionStmtEnd(prep, stmtCtxValid, err, args...) } type stmtEventInfo struct { @@ -191,7 +191,7 @@ func (e *stmtEventInfo) CurrentDB() string { } func (e *stmtEventInfo) AffectedRows() uint64 { - if e.sc == nil { + if e.sc == nil || e.err != nil { return 0 } return e.sc.AffectedRows() diff --git a/server/http_status.go b/server/http_status.go index 20c6a7ae550e0..5f02e1762d4ac 100644 --- a/server/http_status.go +++ b/server/http_status.go @@ -477,7 +477,7 @@ func (s *Server) startStatusServerAndRPCServer(serverMux *http.ServeMux) { logutil.BgLogger().Error("tikv store not etcd background", zap.Error(err)) break } - service := autoid.New(s.statusListener.Addr().String(), etcdAddr, store) + service := autoid.New(s.statusListener.Addr().String(), etcdAddr, store, ebd.TLSConfig()) pb.RegisterAutoIDAllocServer(grpcServer, service) s.autoIDService = service break diff --git a/server/rpc_server.go b/server/rpc_server.go index 893b923d41a77..4dcad639ad4a7 100644 --- a/server/rpc_server.go +++ b/server/rpc_server.go @@ -55,6 +55,12 @@ func NewRPCServer(config *config.Config, dom *domain.Domain, sm util.SessionMana Time: time.Duration(config.Status.GRPCKeepAliveTime) * time.Second, Timeout: time.Duration(config.Status.GRPCKeepAliveTimeout) * time.Second, }), + grpc.KeepaliveEnforcementPolicy(keepalive.EnforcementPolicy{ + // Allow clients send consecutive pings in every 5 seconds. + // The default value of MinTime is 5 minutes, + // which is too long compared with 10 seconds of TiDB's keepalive time. + MinTime: 5 * time.Second, + }), grpc.MaxConcurrentStreams(uint32(config.Status.GRPCConcurrentStreams)), grpc.InitialWindowSize(int32(config.Status.GRPCInitialWindowSize)), grpc.MaxSendMsgSize(config.Status.GRPCMaxSendMsgSize), diff --git a/session/session_test/BUILD.bazel b/session/session_test/BUILD.bazel index 1c20b3b4af125..f0fa774e9f9e3 100644 --- a/session/session_test/BUILD.bazel +++ b/session/session_test/BUILD.bazel @@ -2,7 +2,7 @@ load("@io_bazel_rules_go//go:def.bzl", "go_test") go_test( name = "session_test_test", - timeout = "long", + timeout = "moderate", srcs = [ "main_test.go", "session_test.go", diff --git a/sessionctx/stmtctx/stmtctx.go b/sessionctx/stmtctx/stmtctx.go index bd716e8509938..a02090e490c32 100644 --- a/sessionctx/stmtctx/stmtctx.go +++ b/sessionctx/stmtctx/stmtctx.go @@ -341,6 +341,8 @@ type StatementContext struct { HasFKCascades bool } + // TableStats stores the visited runtime table stats by table id during query + TableStats map[int64]interface{} // useChunkAlloc indicates whether statement use chunk alloc useChunkAlloc bool } diff --git a/sessionctx/variable/BUILD.bazel b/sessionctx/variable/BUILD.bazel index fa4865079e8bf..7c6bcd5330e25 100644 --- a/sessionctx/variable/BUILD.bazel +++ b/sessionctx/variable/BUILD.bazel @@ -85,6 +85,7 @@ go_test( ], embed = [":variable"], flaky = True, + shard_count = 2, deps = [ "//config", "//kv", diff --git a/sessionctx/variable/session.go b/sessionctx/variable/session.go index e046d28fdfc6f..b11b727079630 100644 --- a/sessionctx/variable/session.go +++ b/sessionctx/variable/session.go @@ -96,6 +96,12 @@ type RetryInfo struct { LastRcReadTS uint64 } +// ReuseChunkPool save Alloc object +type ReuseChunkPool struct { + mu sync.Mutex + Alloc chunk.Allocator +} + // Clean does some clean work. func (r *RetryInfo) Clean() { r.autoIncrementIDs.clean() @@ -1292,10 +1298,7 @@ type SessionVars struct { OptPrefixIndexSingleScan bool // ChunkPool Several chunks and columns are cached - ChunkPool struct { - Lock sync.Mutex - Alloc chunk.Allocator - } + ChunkPool ReuseChunkPool // EnableReuseCheck indicates request chunk whether use chunk alloc EnableReuseCheck bool @@ -1304,34 +1307,18 @@ type SessionVars struct { preUseChunkAlloc bool } -// GetNewChunk Attempt to request memory from the chunk pool -// thread safety -func (s *SessionVars) GetNewChunk(fields []*types.FieldType, capacity int) *chunk.Chunk { - //Chunk memory pool is not set - if s.ChunkPool.Alloc == nil { - return chunk.NewChunkWithCapacity(fields, capacity) - } - s.ChunkPool.Lock.Lock() - defer s.ChunkPool.Lock.Unlock() - if s.ChunkPool.Alloc.CheckReuseAllocSize() && (!s.GetUseChunkAlloc()) { - s.StmtCtx.SetUseChunkAlloc() - } - chk := s.ChunkPool.Alloc.Alloc(fields, capacity, capacity) - return chk -} - // GetNewChunkWithCapacity Attempt to request memory from the chunk pool // thread safety -func (s *SessionVars) GetNewChunkWithCapacity(fields []*types.FieldType, capacity int, maxCachesize int) *chunk.Chunk { - if s.ChunkPool.Alloc == nil { +func (s *SessionVars) GetNewChunkWithCapacity(fields []*types.FieldType, capacity int, maxCachesize int, pool chunk.Allocator) *chunk.Chunk { + if pool == nil { return chunk.New(fields, capacity, maxCachesize) } - s.ChunkPool.Lock.Lock() - defer s.ChunkPool.Lock.Unlock() - if s.ChunkPool.Alloc.CheckReuseAllocSize() && (!s.GetUseChunkAlloc()) { + s.ChunkPool.mu.Lock() + defer s.ChunkPool.mu.Unlock() + if pool.CheckReuseAllocSize() && (!s.GetUseChunkAlloc()) { s.StmtCtx.SetUseChunkAlloc() } - chk := s.ChunkPool.Alloc.Alloc(fields, capacity, maxCachesize) + chk := pool.Alloc(fields, capacity, maxCachesize) return chk } @@ -1354,8 +1341,19 @@ func (s *SessionVars) SetAlloc(alloc chunk.Allocator) { } // ClearAlloc indicates stop reuse chunk -func (s *SessionVars) ClearAlloc() { +func (s *SessionVars) ClearAlloc(alloc *chunk.Allocator, b bool) { + if !b { + s.ChunkPool.Alloc = nil + return + } + + // If an error is reported, re-apply for alloc + // Prevent the goroutine left before, affecting the execution of the next sql + // issuse 38918 + s.ChunkPool.mu.Lock() s.ChunkPool.Alloc = nil + s.ChunkPool.mu.Unlock() + *alloc = chunk.NewAllocator() } // GetPreparedStmtByName returns the prepared statement specified by stmtName. @@ -1654,12 +1652,8 @@ func NewSessionVars(hctx HookContext) *SessionVars { ForeignKeyChecks: DefTiDBForeignKeyChecks, HookContext: hctx, EnableReuseCheck: DefTiDBEnableReusechunk, - //useChunkAlloc: DefTiDBUseAlloc, - preUseChunkAlloc: DefTiDBUseAlloc, - ChunkPool: struct { - Lock sync.Mutex - Alloc chunk.Allocator - }{Alloc: nil}, + preUseChunkAlloc: DefTiDBUseAlloc, + ChunkPool: ReuseChunkPool{Alloc: nil}, } vars.KVVars = tikvstore.NewVariables(&vars.Killed) vars.Concurrency = Concurrency{ diff --git a/sessionctx/variable/session_test.go b/sessionctx/variable/session_test.go index 903ecf1bf4fa8..92049902618c2 100644 --- a/sessionctx/variable/session_test.go +++ b/sessionctx/variable/session_test.go @@ -444,10 +444,8 @@ func TestGetReuseChunk(t *testing.T) { require.Nil(t, sessVars.ChunkPool.Alloc) require.False(t, sessVars.GetUseChunkAlloc()) // alloc is nil ,Allocate memory from the system - chk1 := sessVars.GetNewChunk(fieldTypes, 10) + chk1 := sessVars.GetNewChunkWithCapacity(fieldTypes, 10, 10, sessVars.ChunkPool.Alloc) require.NotNil(t, chk1) - chk2 := sessVars.GetNewChunkWithCapacity(fieldTypes, 10, 10) - require.NotNil(t, chk2) chunkReuseMap := make(map[*chunk.Chunk]struct{}, 14) columnReuseMap := make(map[*chunk.Column]struct{}, 14) @@ -461,35 +459,28 @@ func TestGetReuseChunk(t *testing.T) { //tries to apply from the cache initCap := 10 - chk1 = sessVars.GetNewChunk(fieldTypes, initCap) + chk1 = sessVars.GetNewChunkWithCapacity(fieldTypes, initCap, initCap, sessVars.ChunkPool.Alloc) require.NotNil(t, chk1) chunkReuseMap[chk1] = struct{}{} for i := 0; i < chk1.NumCols(); i++ { columnReuseMap[chk1.Column(i)] = struct{}{} } - chk2 = sessVars.GetNewChunkWithCapacity(fieldTypes, initCap, initCap) - require.NotNil(t, chk2) - chunkReuseMap[chk2] = struct{}{} - for i := 0; i < chk2.NumCols(); i++ { - columnReuseMap[chk2.Column(i)] = struct{}{} - } alloc.Reset() - chkres1 := sessVars.GetNewChunk(fieldTypes, 10) + chkres1 := sessVars.GetNewChunkWithCapacity(fieldTypes, 10, 10, sessVars.ChunkPool.Alloc) + require.NotNil(t, chkres1) _, exist := chunkReuseMap[chkres1] require.True(t, exist) for i := 0; i < chkres1.NumCols(); i++ { _, exist := columnReuseMap[chkres1.Column(i)] require.True(t, exist) } - chkres2 := sessVars.GetNewChunkWithCapacity(fieldTypes, 10, 10) - require.NotNil(t, chkres2) - _, exist = chunkReuseMap[chkres2] - require.True(t, exist) - for i := 0; i < chkres2.NumCols(); i++ { - _, exist := columnReuseMap[chkres2.Column(i)] - require.True(t, exist) - } - sessVars.ClearAlloc() + allocpool := variable.ReuseChunkPool{Alloc: alloc} + + sessVars.ClearAlloc(&allocpool.Alloc, false) + require.Equal(t, alloc, allocpool.Alloc) + + sessVars.ClearAlloc(&allocpool.Alloc, true) + require.NotEqual(t, allocpool.Alloc, alloc) require.Nil(t, sessVars.ChunkPool.Alloc) } diff --git a/sessionctx/variable/sysvar.go b/sessionctx/variable/sysvar.go index 580a8deca7b7e..a592e7a7e8831 100644 --- a/sessionctx/variable/sysvar.go +++ b/sessionctx/variable/sysvar.go @@ -746,7 +746,8 @@ var defaultSysVars = []*SysVar{ if floatValue < 0 && floatValue > 0.9 { return "", ErrWrongValueForVar.GenWithStackByArgs(TiDBGOGCTunerThreshold, normalizedValue) } - if globalMemoryLimitTuner < floatValue+0.05 { + // globalMemoryLimitTuner must not be 0. it will be 0 when tidb_server_memory_limit_gc_trigger is not set during startup. + if globalMemoryLimitTuner != 0 && globalMemoryLimitTuner < floatValue+0.05 { return "", errors.New("tidb_gogc_tuner_threshold should be less than tidb_server_memory_limit_gc_trigger - 0.05") } return strconv.FormatFloat(floatValue, 'f', -1, 64), nil @@ -827,7 +828,8 @@ var defaultSysVars = []*SysVar{ if floatValue < 0.51 && floatValue > 1 { // 51% ~ 100% return "", ErrWrongValueForVar.GenWithStackByArgs(TiDBServerMemoryLimitGCTrigger, normalizedValue) } - if floatValue < gogcTunerThreshold+0.05 { + // gogcTunerThreshold must not be 0. it will be 0 when tidb_gogc_tuner_threshold is not set during startup. + if gogcTunerThreshold != 0 && floatValue < gogcTunerThreshold+0.05 { return "", errors.New("tidb_server_memory_limit_gc_trigger should be greater than tidb_gogc_tuner_threshold + 0.05") } @@ -1037,6 +1039,13 @@ var defaultSysVars = []*SysVar{ }, GetGlobal: func(_ context.Context, s *SessionVars) (string, error) { return fmt.Sprintf("%d", MemoryUsageAlarmKeepRecordNum.Load()), nil }}, + {Scope: ScopeGlobal, Name: TiDBEnablePlanReplayerCapture, Value: BoolToOnOff(false), Type: TypeBool, + SetGlobal: func(ctx context.Context, s *SessionVars, val string) error { + EnablePlanReplayerCapture.Store(TiDBOptOn(val)) + return nil + }, GetGlobal: func(ctx context.Context, vars *SessionVars) (string, error) { + return strconv.FormatBool(EnablePlanReplayerCapture.Load()), nil + }}, /* The system variables below have GLOBAL and SESSION scope */ {Scope: ScopeGlobal | ScopeSession, Name: TiDBRowFormatVersion, Value: strconv.Itoa(DefTiDBRowFormatV1), Type: TypeUnsigned, MinValue: 1, MaxValue: 2, SetGlobal: func(_ context.Context, s *SessionVars, val string) error { diff --git a/sessionctx/variable/tidb_vars.go b/sessionctx/variable/tidb_vars.go index 4c261d8def2e4..80c9b41f4cc6e 100644 --- a/sessionctx/variable/tidb_vars.go +++ b/sessionctx/variable/tidb_vars.go @@ -768,6 +768,8 @@ const ( // TiDBEnableExternalTSRead indicates whether to enable read through an external ts TiDBEnableExternalTSRead = "tidb_enable_external_ts_read" + // TiDBEnablePlanReplayerCapture indicates whether to enable plan replayer capture + TiDBEnablePlanReplayerCapture = "tidb_enable_plan_replayer_capture" // TiDBEnableReusechunk indicates whether to enable chunk alloc TiDBEnableReusechunk = "tidb_enable_reuse_chunk" ) @@ -1083,12 +1085,13 @@ const ( DefTiDBServerMemoryLimitGCTrigger = 0.7 DefTiDBEnableGOGCTuner = true // DefTiDBGOGCTunerThreshold is to limit TiDBGOGCTunerThreshold. - DefTiDBGOGCTunerThreshold float64 = 0.6 - DefTiDBOptPrefixIndexSingleScan = true - DefTiDBExternalTS = 0 - DefTiDBEnableExternalTSRead = false - DefTiDBEnableReusechunk = true - DefTiDBUseAlloc = false + DefTiDBGOGCTunerThreshold float64 = 0.6 + DefTiDBOptPrefixIndexSingleScan = true + DefTiDBExternalTS = 0 + DefTiDBEnableExternalTSRead = false + DefTiDBEnableReusechunk = true + DefTiDBUseAlloc = false + DefTiDBEnablePlanReplayerCapture = false ) // Process global variables. @@ -1145,8 +1148,9 @@ var ( // DefTiDBServerMemoryLimit indicates the default value of TiDBServerMemoryLimit(TotalMem * 80%). // It should be a const and shouldn't be modified after tidb is started. - DefTiDBServerMemoryLimit = serverMemoryLimitDefaultValue() - GOGCTunerThreshold = atomic.NewFloat64(DefTiDBGOGCTunerThreshold) + DefTiDBServerMemoryLimit = serverMemoryLimitDefaultValue() + GOGCTunerThreshold = atomic.NewFloat64(DefTiDBGOGCTunerThreshold) + EnablePlanReplayerCapture = atomic.NewBool(DefTiDBEnablePlanReplayerCapture) ) var ( diff --git a/sessionctx/variable/variable_test.go b/sessionctx/variable/variable_test.go index bcf917413de3d..023cc75e7cba7 100644 --- a/sessionctx/variable/variable_test.go +++ b/sessionctx/variable/variable_test.go @@ -436,18 +436,22 @@ func TestDefaultValuesAreSettable(t *testing.T) { for _, sv := range GetSysVars() { if sv.HasSessionScope() && !sv.ReadOnly { val, err := sv.Validate(vars, sv.Value, ScopeSession) - require.Equal(t, val, sv.Value) require.NoError(t, err) + require.Equal(t, val, sv.Value) } if sv.HasGlobalScope() && !sv.ReadOnly { val, err := sv.Validate(vars, sv.Value, ScopeGlobal) - require.Equal(t, val, sv.Value) require.NoError(t, err) + require.Equal(t, val, sv.Value) } } } +func TestLimitBetweenVariable(t *testing.T) { + require.Less(t, DefTiDBGOGCTunerThreshold+0.05, DefTiDBServerMemoryLimitGCTrigger) +} + // TestSysVarNameIsLowerCase tests that no new sysvars are added with uppercase characters. // In MySQL variables are always lowercase, and can be set in a case-insensitive way. func TestSysVarNameIsLowerCase(t *testing.T) { diff --git a/statistics/column.go b/statistics/column.go index 197fc2f5a6794..ccd541f276bcf 100644 --- a/statistics/column.go +++ b/statistics/column.go @@ -141,11 +141,6 @@ func (c *Column) IsInvalid(sctx sessionctx.Context, collPseudo bool) bool { return c.TotalRowCount() == 0 || (!c.IsEssentialStatsLoaded() && c.Histogram.NDV > 0) } -// IsHistNeeded checks if this column needs histogram to be loaded -func (c *Column) IsHistNeeded(collPseudo bool) bool { - return (!collPseudo || !c.NotAccurate()) && c.IsLoadNeeded() -} - func (c *Column) equalRowCount(sctx sessionctx.Context, val types.Datum, encodedVal []byte, realtimeRowCount int64) (float64, error) { if val.IsNull() { return float64(c.NullCount), nil diff --git a/statistics/handle/BUILD.bazel b/statistics/handle/BUILD.bazel index f7ce80fc47937..bcafe4260eb30 100644 --- a/statistics/handle/BUILD.bazel +++ b/statistics/handle/BUILD.bazel @@ -57,7 +57,7 @@ go_library( go_test( name = "handle_test", - timeout = "long", + timeout = "moderate", srcs = [ "ddl_test.go", "dump_test.go", diff --git a/statistics/handle/dump.go b/statistics/handle/dump.go index 123f02b912000..777e3b70058c1 100644 --- a/statistics/handle/dump.go +++ b/statistics/handle/dump.go @@ -168,15 +168,8 @@ func (h *Handle) DumpStatsToJSONBySnapshot(dbName string, tableInfo *model.Table return jsonTbl, nil } -func (h *Handle) tableStatsToJSON(dbName string, tableInfo *model.TableInfo, physicalID int64, snapshot uint64) (*JSONTable, error) { - tbl, err := h.TableStatsFromStorage(tableInfo, physicalID, true, snapshot) - if err != nil || tbl == nil { - return nil, err - } - tbl.Version, tbl.ModifyCount, tbl.Count, err = h.statsMetaByTableIDFromStorage(physicalID, snapshot) - if err != nil { - return nil, err - } +// GenJSONTableFromStats generate jsonTable from tableInfo and stats +func GenJSONTableFromStats(dbName string, tableInfo *model.TableInfo, tbl *statistics.Table) (*JSONTable, error) { jsonTbl := &JSONTable{ DatabaseName: dbName, TableName: tableInfo.Name.L, @@ -185,7 +178,6 @@ func (h *Handle) tableStatsToJSON(dbName string, tableInfo *model.TableInfo, phy Count: tbl.Count, ModifyCount: tbl.ModifyCount, } - for _, col := range tbl.Columns { sc := &stmtctx.StatementContext{TimeZone: time.UTC} hist, err := col.ConvertTo(sc, types.NewFieldType(mysql.TypeBlob)) @@ -202,6 +194,22 @@ func (h *Handle) tableStatsToJSON(dbName string, tableInfo *model.TableInfo, phy return jsonTbl, nil } +func (h *Handle) tableStatsToJSON(dbName string, tableInfo *model.TableInfo, physicalID int64, snapshot uint64) (*JSONTable, error) { + tbl, err := h.TableStatsFromStorage(tableInfo, physicalID, true, snapshot) + if err != nil || tbl == nil { + return nil, err + } + tbl.Version, tbl.ModifyCount, tbl.Count, err = h.statsMetaByTableIDFromStorage(physicalID, snapshot) + if err != nil { + return nil, err + } + jsonTbl, err := GenJSONTableFromStats(dbName, tableInfo, tbl) + if err != nil { + return nil, err + } + return jsonTbl, nil +} + // LoadStatsFromJSON will load statistic from JSONTable, and save it to the storage. func (h *Handle) LoadStatsFromJSON(is infoschema.InfoSchema, jsonTbl *JSONTable) error { table, err := is.TableByName(model.NewCIStr(jsonTbl.DatabaseName), model.NewCIStr(jsonTbl.TableName)) diff --git a/statistics/handle/handle_hist.go b/statistics/handle/handle_hist.go index 1392590130438..4d0128095e071 100644 --- a/statistics/handle/handle_hist.go +++ b/statistics/handle/handle_hist.go @@ -59,6 +59,16 @@ type NeededItemTask struct { // SendLoadRequests send neededColumns requests func (h *Handle) SendLoadRequests(sc *stmtctx.StatementContext, neededHistItems []model.TableItemID, timeout time.Duration) error { remainedItems := h.removeHistLoadedColumns(neededHistItems) + + failpoint.Inject("assertSyncLoadItems", func(val failpoint.Value) { + if sc.OptimizeTracer != nil { + count := val.(int) + if len(remainedItems) != count { + panic("remained items count wrong") + } + } + }) + if len(remainedItems) <= 0 { return nil } @@ -137,10 +147,7 @@ func (h *Handle) removeHistLoadedColumns(neededItems []model.TableItemID) []mode continue } colHist, ok := tbl.Columns[item.ID] - if !ok { - continue - } - if colHist.IsHistNeeded(tbl.Pseudo) { + if ok && colHist.IsStatsInitialized() && !colHist.IsFullLoad() { remainedItems = append(remainedItems, item) } } diff --git a/statistics/handle/handle_hist_test.go b/statistics/handle/handle_hist_test.go index cf9f708345d32..f2b1fa87ba37f 100644 --- a/statistics/handle/handle_hist_test.go +++ b/statistics/handle/handle_hist_test.go @@ -29,6 +29,28 @@ import ( "github.com/stretchr/testify/require" ) +func TestSyncLoadSkipUnAnalyzedItems(t *testing.T) { + store, dom := testkit.CreateMockStoreAndDomain(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + tk.MustExec("drop table if exists t1") + tk.MustExec("create table t(a int)") + tk.MustExec("create table t1(a int)") + h := dom.StatsHandle() + h.SetLease(1) + + // no item would be loaded + require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/statistics/handle/assertSyncLoadItems", `return(0)`)) + tk.MustQuery("trace plan select * from t where a > 10") + failpoint.Disable("github.com/pingcap/tidb/statistics/handle/assertSyncLoadItems") + tk.MustExec("analyze table t1") + // one column would be loaded + require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/statistics/handle/assertSyncLoadItems", `return(1)`)) + tk.MustQuery("trace plan select * from t1 where a > 10") + failpoint.Disable("github.com/pingcap/tidb/statistics/handle/assertSyncLoadItems") +} + func TestConcurrentLoadHist(t *testing.T) { store, dom := testkit.CreateMockStoreAndDomain(t) diff --git a/statistics/handle/handle_test.go b/statistics/handle/handle_test.go index f8f73b933bd9a..28c6379138edd 100644 --- a/statistics/handle/handle_test.go +++ b/statistics/handle/handle_test.go @@ -2264,7 +2264,7 @@ func TestIndexUsageInformation(t *testing.T) { tk.MustQuery("select b from t_idx where b=0") err = do.StatsHandle().DumpIndexUsageToKV() require.NoError(t, err) - tk.MustQuery(querySQL).Check(testkit.Rows( + tk.MustQuery(querySQL).Sort().Check(testkit.Rows( "test t_idx idx_a 3 2", "test t_idx idx_b 2 2", )) diff --git a/statistics/selectivity_test.go b/statistics/selectivity_test.go index 26e79c93cc5fd..f0dad37f7cac8 100644 --- a/statistics/selectivity_test.go +++ b/statistics/selectivity_test.go @@ -299,6 +299,7 @@ func TestStatsVer2(t *testing.T) { store := testkit.CreateMockStore(t) testKit := testkit.NewTestKit(t, store) testKit.MustExec("use test") + testKit.MustExec("set tidb_cost_model_version=2") testKit.MustExec("set tidb_analyze_version=2") testKit.MustExec("drop table if exists tint") diff --git a/statistics/testdata/stats_suite_out.json b/statistics/testdata/stats_suite_out.json index d07b336ccbfbd..a78fafe87ade9 100644 --- a/statistics/testdata/stats_suite_out.json +++ b/statistics/testdata/stats_suite_out.json @@ -170,34 +170,34 @@ " └─TableFullScan_5 8.00 cop[tikv] table:tint keep order:false" ], [ - "IndexLookUp_10 1.00 root ", - "├─IndexRangeScan_8(Build) 1.00 cop[tikv] table:tdouble, index:singular(a) range:[1,1], keep order:false", - "└─TableRowIDScan_9(Probe) 1.00 cop[tikv] table:tdouble keep order:false" + "TableReader_7 1.00 root data:Selection_6", + "└─Selection_6 1.00 cop[tikv] eq(test.tdouble.a, 1)", + " └─TableFullScan_5 8.00 cop[tikv] table:tdouble keep order:false" ], [ - "IndexLookUp_10 1.00 root ", - "├─IndexRangeScan_8(Build) 1.00 cop[tikv] table:tdouble, index:singular(a) range:[4,4], keep order:false", - "└─TableRowIDScan_9(Probe) 1.00 cop[tikv] table:tdouble keep order:false" + "TableReader_7 1.00 root data:Selection_6", + "└─Selection_6 1.00 cop[tikv] eq(test.tdouble.a, 4)", + " └─TableFullScan_5 8.00 cop[tikv] table:tdouble keep order:false" ], [ - "IndexLookUp_10 1.00 root ", - "├─IndexRangeScan_8(Build) 1.00 cop[tikv] table:tdouble, index:singular(a) range:[8,8], keep order:false", - "└─TableRowIDScan_9(Probe) 1.00 cop[tikv] table:tdouble keep order:false" + "TableReader_7 1.00 root data:Selection_6", + "└─Selection_6 1.00 cop[tikv] eq(test.tdouble.a, 8)", + " └─TableFullScan_5 8.00 cop[tikv] table:tdouble keep order:false" ], [ - "IndexLookUp_10 1.00 root ", - "├─IndexRangeScan_8(Build) 1.00 cop[tikv] table:tdecimal, index:singular(a) range:[1.00000000000000000000,1.00000000000000000000], keep order:false", - "└─TableRowIDScan_9(Probe) 1.00 cop[tikv] table:tdecimal keep order:false" + "TableReader_7 1.00 root data:Selection_6", + "└─Selection_6 1.00 cop[tikv] eq(test.tdecimal.a, 1)", + " └─TableFullScan_5 8.00 cop[tikv] table:tdecimal keep order:false" ], [ - "IndexLookUp_10 1.00 root ", - "├─IndexRangeScan_8(Build) 1.00 cop[tikv] table:tdecimal, index:singular(a) range:[4.00000000000000000000,4.00000000000000000000], keep order:false", - "└─TableRowIDScan_9(Probe) 1.00 cop[tikv] table:tdecimal keep order:false" + "TableReader_7 1.00 root data:Selection_6", + "└─Selection_6 1.00 cop[tikv] eq(test.tdecimal.a, 4)", + " └─TableFullScan_5 8.00 cop[tikv] table:tdecimal keep order:false" ], [ - "IndexLookUp_10 1.00 root ", - "├─IndexRangeScan_8(Build) 1.00 cop[tikv] table:tdecimal, index:singular(a) range:[8.00000000000000000000,8.00000000000000000000], keep order:false", - "└─TableRowIDScan_9(Probe) 1.00 cop[tikv] table:tdecimal keep order:false" + "TableReader_7 1.00 root data:Selection_6", + "└─Selection_6 1.00 cop[tikv] eq(test.tdecimal.a, 8)", + " └─TableFullScan_5 8.00 cop[tikv] table:tdecimal keep order:false" ], [ "TableReader_7 1.00 root data:Selection_6", @@ -260,34 +260,34 @@ " └─TableFullScan_5 8.00 cop[tikv] table:tint keep order:false" ], [ - "IndexLookUp_10 1.00 root ", - "├─IndexRangeScan_8(Build) 1.00 cop[tikv] table:tdouble, index:multi(b, c) range:[1 1,1 1], keep order:false", - "└─TableRowIDScan_9(Probe) 1.00 cop[tikv] table:tdouble keep order:false" + "TableReader_7 1.00 root data:Selection_6", + "└─Selection_6 1.00 cop[tikv] eq(test.tdouble.b, 1), eq(test.tdouble.c, 1)", + " └─TableFullScan_5 8.00 cop[tikv] table:tdouble keep order:false" ], [ - "IndexLookUp_10 1.00 root ", - "├─IndexRangeScan_8(Build) 1.00 cop[tikv] table:tdouble, index:multi(b, c) range:[4 4,4 4], keep order:false", - "└─TableRowIDScan_9(Probe) 1.00 cop[tikv] table:tdouble keep order:false" + "TableReader_7 1.00 root data:Selection_6", + "└─Selection_6 1.00 cop[tikv] eq(test.tdouble.b, 4), eq(test.tdouble.c, 4)", + " └─TableFullScan_5 8.00 cop[tikv] table:tdouble keep order:false" ], [ - "IndexLookUp_10 1.00 root ", - "├─IndexRangeScan_8(Build) 1.00 cop[tikv] table:tdouble, index:multi(b, c) range:[8 8,8 8], keep order:false", - "└─TableRowIDScan_9(Probe) 1.00 cop[tikv] table:tdouble keep order:false" + "TableReader_7 1.00 root data:Selection_6", + "└─Selection_6 1.00 cop[tikv] eq(test.tdouble.b, 8), eq(test.tdouble.c, 8)", + " └─TableFullScan_5 8.00 cop[tikv] table:tdouble keep order:false" ], [ - "IndexLookUp_10 1.00 root ", - "├─IndexRangeScan_8(Build) 1.00 cop[tikv] table:tdecimal, index:multi(b, c) range:[1.00000000000000000000 1.00000000000000000000,1.00000000000000000000 1.00000000000000000000], keep order:false", - "└─TableRowIDScan_9(Probe) 1.00 cop[tikv] table:tdecimal keep order:false" + "TableReader_7 1.00 root data:Selection_6", + "└─Selection_6 1.00 cop[tikv] eq(test.tdecimal.b, 1), eq(test.tdecimal.c, 1)", + " └─TableFullScan_5 8.00 cop[tikv] table:tdecimal keep order:false" ], [ - "IndexLookUp_10 1.00 root ", - "├─IndexRangeScan_8(Build) 1.00 cop[tikv] table:tdecimal, index:multi(b, c) range:[4.00000000000000000000 4.00000000000000000000,4.00000000000000000000 4.00000000000000000000], keep order:false", - "└─TableRowIDScan_9(Probe) 1.00 cop[tikv] table:tdecimal keep order:false" + "TableReader_7 1.00 root data:Selection_6", + "└─Selection_6 1.00 cop[tikv] eq(test.tdecimal.b, 4), eq(test.tdecimal.c, 4)", + " └─TableFullScan_5 8.00 cop[tikv] table:tdecimal keep order:false" ], [ - "IndexLookUp_10 1.00 root ", - "├─IndexRangeScan_8(Build) 1.00 cop[tikv] table:tdecimal, index:multi(b, c) range:[8.00000000000000000000 8.00000000000000000000,8.00000000000000000000 8.00000000000000000000], keep order:false", - "└─TableRowIDScan_9(Probe) 1.00 cop[tikv] table:tdecimal keep order:false" + "TableReader_7 1.00 root data:Selection_6", + "└─Selection_6 1.00 cop[tikv] eq(test.tdecimal.b, 8), eq(test.tdecimal.c, 8)", + " └─TableFullScan_5 8.00 cop[tikv] table:tdecimal keep order:false" ], [ "TableReader_7 1.00 root data:Selection_6", @@ -335,34 +335,34 @@ " └─TableFullScan_5 8.00 cop[tikv] table:tint keep order:false" ], [ - "IndexLookUp_10 1.00 root ", - "├─IndexRangeScan_8(Build) 1.00 cop[tikv] table:tdouble, index:multi(b, c) range:[1,1], keep order:false", - "└─TableRowIDScan_9(Probe) 1.00 cop[tikv] table:tdouble keep order:false" + "TableReader_7 1.00 root data:Selection_6", + "└─Selection_6 1.00 cop[tikv] eq(test.tdouble.b, 1)", + " └─TableFullScan_5 8.00 cop[tikv] table:tdouble keep order:false" ], [ - "IndexLookUp_10 1.00 root ", - "├─IndexRangeScan_8(Build) 1.00 cop[tikv] table:tdouble, index:multi(b, c) range:[4,4], keep order:false", - "└─TableRowIDScan_9(Probe) 1.00 cop[tikv] table:tdouble keep order:false" + "TableReader_7 1.00 root data:Selection_6", + "└─Selection_6 1.00 cop[tikv] eq(test.tdouble.b, 4)", + " └─TableFullScan_5 8.00 cop[tikv] table:tdouble keep order:false" ], [ - "IndexLookUp_10 1.00 root ", - "├─IndexRangeScan_8(Build) 1.00 cop[tikv] table:tdouble, index:multi(b, c) range:[8,8], keep order:false", - "└─TableRowIDScan_9(Probe) 1.00 cop[tikv] table:tdouble keep order:false" + "TableReader_7 1.00 root data:Selection_6", + "└─Selection_6 1.00 cop[tikv] eq(test.tdouble.b, 8)", + " └─TableFullScan_5 8.00 cop[tikv] table:tdouble keep order:false" ], [ - "IndexLookUp_10 1.00 root ", - "├─IndexRangeScan_8(Build) 1.00 cop[tikv] table:tdecimal, index:multi(b, c) range:[1.00000000000000000000,1.00000000000000000000], keep order:false", - "└─TableRowIDScan_9(Probe) 1.00 cop[tikv] table:tdecimal keep order:false" + "TableReader_7 1.00 root data:Selection_6", + "└─Selection_6 1.00 cop[tikv] eq(test.tdecimal.b, 1)", + " └─TableFullScan_5 8.00 cop[tikv] table:tdecimal keep order:false" ], [ - "IndexLookUp_10 1.00 root ", - "├─IndexRangeScan_8(Build) 1.00 cop[tikv] table:tdecimal, index:multi(b, c) range:[4.00000000000000000000,4.00000000000000000000], keep order:false", - "└─TableRowIDScan_9(Probe) 1.00 cop[tikv] table:tdecimal keep order:false" + "TableReader_7 1.00 root data:Selection_6", + "└─Selection_6 1.00 cop[tikv] eq(test.tdecimal.b, 4)", + " └─TableFullScan_5 8.00 cop[tikv] table:tdecimal keep order:false" ], [ - "IndexLookUp_10 1.00 root ", - "├─IndexRangeScan_8(Build) 1.00 cop[tikv] table:tdecimal, index:multi(b, c) range:[8.00000000000000000000,8.00000000000000000000], keep order:false", - "└─TableRowIDScan_9(Probe) 1.00 cop[tikv] table:tdecimal keep order:false" + "TableReader_7 1.00 root data:Selection_6", + "└─Selection_6 1.00 cop[tikv] eq(test.tdecimal.b, 8)", + " └─TableFullScan_5 8.00 cop[tikv] table:tdecimal keep order:false" ], [ "TableReader_7 1.00 root data:Selection_6", diff --git a/store/gcworker/gc_worker_test.go b/store/gcworker/gc_worker_test.go index 8d9ae4c9ae377..4fde3dd44cd12 100644 --- a/store/gcworker/gc_worker_test.go +++ b/store/gcworker/gc_worker_test.go @@ -1117,6 +1117,11 @@ func TestRunGCJob(t *testing.T) { pdSafePoint := s.mustGetSafePointFromPd(t) require.Equal(t, safePoint, pdSafePoint) + require.NoError(t, s.gcWorker.saveTime(gcSafePointKey, oracle.GetTimeFromTS(safePoint))) + tikvSafePoint, err := s.gcWorker.loadTime(gcSafePointKey) + require.NoError(t, err) + require.Equal(t, *tikvSafePoint, oracle.GetTimeFromTS(safePoint)) + etcdSafePoint := s.loadEtcdSafePoint(t) require.Equal(t, safePoint, etcdSafePoint) diff --git a/testkit/testkit.go b/testkit/testkit.go index 6952f4c36d484..25479bc862d96 100644 --- a/testkit/testkit.go +++ b/testkit/testkit.go @@ -115,7 +115,6 @@ func (tk *TestKit) Session() session.Session { // MustExec executes a sql statement and asserts nil error. func (tk *TestKit) MustExec(sql string, args ...interface{}) { defer func() { - tk.Session().GetSessionVars().ClearAlloc() if tk.alloc != nil { tk.alloc.Reset() } @@ -138,7 +137,6 @@ func (tk *TestKit) MustExecWithContext(ctx context.Context, sql string, args ... // If expected result is set it asserts the query result equals expected result. func (tk *TestKit) MustQuery(sql string, args ...interface{}) *Result { defer func() { - tk.Session().GetSessionVars().ClearAlloc() if tk.alloc != nil { tk.alloc.Reset() } @@ -271,7 +269,8 @@ func (tk *TestKit) Exec(sql string, args ...interface{}) (sqlexec.RecordSet, err } // ExecWithContext executes a sql statement using the prepared stmt API -func (tk *TestKit) ExecWithContext(ctx context.Context, sql string, args ...interface{}) (sqlexec.RecordSet, error) { +func (tk *TestKit) ExecWithContext(ctx context.Context, sql string, args ...interface{}) (rs sqlexec.RecordSet, err error) { + defer tk.Session().GetSessionVars().ClearAlloc(&tk.alloc, err != nil) if len(args) == 0 { sc := tk.session.GetSessionVars().StmtCtx prevWarns := sc.GetWarnings() @@ -315,7 +314,7 @@ func (tk *TestKit) ExecWithContext(ctx context.Context, sql string, args ...inte } params := expression.Args2Expressions4Test(args...) tk.Session().GetSessionVars().SetAlloc(tk.alloc) - rs, err := tk.session.ExecutePreparedStmt(ctx, stmtID, params) + rs, err = tk.session.ExecutePreparedStmt(ctx, stmtID, params) if err != nil { return rs, errors.Trace(err) } diff --git a/tests/readonlytest/readonly_test.go b/tests/readonlytest/readonly_test.go index 836debae6ac11..654e2542738e5 100644 --- a/tests/readonlytest/readonly_test.go +++ b/tests/readonlytest/readonly_test.go @@ -158,6 +158,11 @@ func TestRestriction(t *testing.T) { require.Error(t, err) require.Equal(t, err.Error(), PriviledgedErrMsg) + // can't do flashback cluster + _, err = s.udb.Exec("flashback cluster to timestamp ''") + require.Error(t, err) + require.Equal(t, err.Error(), ReadOnlyErrMsg) + // can do some Admin stmts _, err = s.udb.Exec("admin show ddl jobs") require.NoError(t, err) diff --git a/tests/realtikvtest/brietest/BUILD.bazel b/tests/realtikvtest/brietest/BUILD.bazel index fc9ff667cfbf6..49ea32406c7d6 100644 --- a/tests/realtikvtest/brietest/BUILD.bazel +++ b/tests/realtikvtest/brietest/BUILD.bazel @@ -2,7 +2,7 @@ load("@io_bazel_rules_go//go:def.bzl", "go_test") go_test( name = "brietest_test", - timeout = "long", + timeout = "moderate", srcs = [ "backup_restore_test.go", "binlog_test.go", @@ -24,6 +24,7 @@ go_test( "@com_github_pingcap_tipb//go-binlog", "@com_github_stretchr_testify//require", "@com_github_tikv_client_go_v2//oracle", + "@com_github_tikv_client_go_v2//util", "@org_golang_google_grpc//:grpc", "@org_uber_go_goleak//:goleak", ], diff --git a/tests/realtikvtest/brietest/flashback_test.go b/tests/realtikvtest/brietest/flashback_test.go index 9c15845a108ad..322359fff411a 100644 --- a/tests/realtikvtest/brietest/flashback_test.go +++ b/tests/realtikvtest/brietest/flashback_test.go @@ -26,6 +26,7 @@ import ( "github.com/pingcap/tidb/tests/realtikvtest" "github.com/stretchr/testify/require" "github.com/tikv/client-go/v2/oracle" + tikvutil "github.com/tikv/client-go/v2/util" ) // MockGC is used to make GC work in the test environment. @@ -42,9 +43,8 @@ func MockGC(tk *testkit.TestKit) (string, string, string, func()) { // disable emulator GC. // Otherwise emulator GC will delete table record as soon as possible after execute drop table ddl. ddlutil.EmulatorGCDisable() - gcTimeFormat := "20060102-15:04:05 -0700 MST" - timeBeforeDrop := time.Now().Add(0 - 48*60*60*time.Second).Format(gcTimeFormat) - timeAfterDrop := time.Now().Add(48 * 60 * 60 * time.Second).Format(gcTimeFormat) + timeBeforeDrop := time.Now().Add(0 - 48*60*60*time.Second).Format(tikvutil.GCTimeFormat) + timeAfterDrop := time.Now().Add(48 * 60 * 60 * time.Second).Format(tikvutil.GCTimeFormat) safePointSQL := `INSERT HIGH_PRIORITY INTO mysql.tidb VALUES ('tikv_gc_safe_point', '%[1]s', '') ON DUPLICATE KEY UPDATE variable_value = '%[1]s'` diff --git a/tests/realtikvtest/sessiontest/BUILD.bazel b/tests/realtikvtest/sessiontest/BUILD.bazel index 17c26103d338c..98f9f2e0b2335 100644 --- a/tests/realtikvtest/sessiontest/BUILD.bazel +++ b/tests/realtikvtest/sessiontest/BUILD.bazel @@ -2,7 +2,7 @@ load("@io_bazel_rules_go//go:def.bzl", "go_test") go_test( name = "sessiontest_test", - timeout = "long", + timeout = "moderate", srcs = [ "main_test.go", "paging_test.go", diff --git a/tests/realtikvtest/statisticstest/BUILD.bazel b/tests/realtikvtest/statisticstest/BUILD.bazel index ace0b42b4f974..1203b0366c08b 100644 --- a/tests/realtikvtest/statisticstest/BUILD.bazel +++ b/tests/realtikvtest/statisticstest/BUILD.bazel @@ -2,7 +2,7 @@ load("@io_bazel_rules_go//go:def.bzl", "go_test") go_test( name = "statisticstest_test", - timeout = "long", + timeout = "moderate", srcs = [ "main_test.go", "statistics_test.go", diff --git a/tests/realtikvtest/statisticstest/main_test.go b/tests/realtikvtest/statisticstest/main_test.go index 8b9cf1e62690c..5fef5b584be04 100644 --- a/tests/realtikvtest/statisticstest/main_test.go +++ b/tests/realtikvtest/statisticstest/main_test.go @@ -26,6 +26,7 @@ func TestMain(m *testing.M) { opts := []goleak.Option{ goleak.IgnoreTopFunction("github.com/golang/glog.(*loggingT).flushDaemon"), goleak.IgnoreTopFunction("github.com/lestrrat-go/httprc.runFetchWorker"), + goleak.IgnoreTopFunction("google.golang.org/grpc.(*ccBalancerWrapper).watcher"), goleak.IgnoreTopFunction("internal/poll.runtime_pollWait"), goleak.IgnoreTopFunction("net/http.(*persistConn).writeLoop"), } diff --git a/tests/realtikvtest/txntest/BUILD.bazel b/tests/realtikvtest/txntest/BUILD.bazel index 3d59b74434fdb..83bc4c549ce1d 100644 --- a/tests/realtikvtest/txntest/BUILD.bazel +++ b/tests/realtikvtest/txntest/BUILD.bazel @@ -2,7 +2,7 @@ load("@io_bazel_rules_go//go:def.bzl", "go_test") go_test( name = "txntest_test", - timeout = "long", + timeout = "moderate", srcs = [ "isolation_test.go", "main_test.go", diff --git a/types/core_time.go b/types/core_time.go index 3a0cd42ed1c3e..aaf3f5711fea9 100644 --- a/types/core_time.go +++ b/types/core_time.go @@ -184,42 +184,6 @@ func (t CoreTime) GoTime(loc *gotime.Location) (gotime.Time, error) { return tm, nil } -// FindZoneTransition check for one Time Zone transition within +/- 4h -// Currently the needed functions are not exported, if gotime.Location.lookup would be exported -// then it would be easy to use that directly -func FindZoneTransition(tIn gotime.Time) (gotime.Time, error) { - // Check most common case first, DST transition on full hour. - // round truncates away from zero! - t2 := tIn.Round(gotime.Hour).Add(-1 * gotime.Hour) - t1 := t2.Add(-1 * gotime.Second) - _, offset1 := t1.Zone() - _, offset2 := t2.Zone() - if offset1 != offset2 { - return t2, nil - } - - // Check if any offset change? - t1 = tIn.Add(-4 * gotime.Hour) - t2 = tIn.Add(4 * gotime.Hour) - _, offset1 = t1.Zone() - _, offset2 = t2.Zone() - if offset1 == offset2 { - return tIn, errors.Trace(ErrWrongValue.GenWithStackByArgs(TimeStr, tIn)) - } - - // Check generic case, like for 'Australia/Lord_Howe' - for t2.After(t1.Add(gotime.Second)) { - t := t1.Add(t2.Sub(t1) / 2).Round(gotime.Second) - _, offset := t.Zone() - if offset == offset1 { - t1 = t - } else { - t2 = t - } - } - return t2, nil -} - // AdjustedGoTime converts Time to GoTime and adjust for invalid DST times // like during the DST change with increased offset, // normally moving to Daylight Saving Time. @@ -230,11 +194,18 @@ func (t CoreTime) AdjustedGoTime(loc *gotime.Location) (gotime.Time, error) { return tm, nil } - tAdj, err2 := FindZoneTransition(tm) - if err2 == nil { - return tAdj, nil + // The converted go time did not map back to the same time, probably it was between a + // daylight saving transition, adjust the time to the closest Zone bound. + start, end := tm.ZoneBounds() + // time zone transitions are normally 1 hour, allow up to 4 hours before returning error + if start.Sub(tm).Abs().Hours() > 4.0 && end.Sub(tm).Abs().Hours() > 4.0 { + return tm, errors.Trace(ErrWrongValue.GenWithStackByArgs(TimeStr, tm)) + } + // use the closest transition time + if tm.Sub(start).Abs() <= tm.Sub(end).Abs() { + return start, nil } - return tm, err + return end, nil } // IsLeapYear returns if it's leap year. diff --git a/types/core_time_test.go b/types/core_time_test.go index 18d21ddaf8563..999a5504f0899 100644 --- a/types/core_time_test.go +++ b/types/core_time_test.go @@ -293,45 +293,6 @@ func TestWeekday(t *testing.T) { } } -func TestFindZoneTransition(t *testing.T) { - tests := []struct { - TZ string - dt string - Expect string - Success bool - }{ - {"Australia/Lord_Howe", "2020-06-29 03:45:00", "", false}, - {"Australia/Lord_Howe", "2020-10-04 02:15:00", "2020-10-04 02:30:00 +11 +1100", true}, - {"Australia/Lord_Howe", "2020-10-04 02:29:59", "2020-10-04 02:30:00 +11 +1100", true}, - {"Australia/Lord_Howe", "2020-10-04 02:29:59.99", "2020-10-04 02:30:00 +11 +1100", true}, - {"Australia/Lord_Howe", "2020-10-04 02:30:00.0001", "2020-10-04 02:30:00 +11 +1100", true}, - {"Australia/Lord_Howe", "2020-10-04 02:30:00", "2020-10-04 02:30:00 +11 +1100", true}, - {"Australia/Lord_Howe", "2020-10-04 02:30:01", "2020-10-04 02:30:00 +11 +1100", true}, - {"Europe/Vilnius", "2020-03-29 03:45:00", "2020-03-29 04:00:00 EEST +0300", true}, - {"Europe/Vilnius", "2020-10-25 03:45:00", "2020-10-25 03:00:00 EET +0200", true}, - {"Europe/Vilnius", "2020-06-29 03:45:00", "", false}, - {"Europe/Amsterdam", "2020-03-29 02:45:00", "2020-03-29 03:00:00 CEST +0200", true}, - {"Europe/Amsterdam", "2020-10-25 02:35:00", "2020-10-25 02:00:00 CET +0100", true}, - {"Europe/Amsterdam", "2020-03-29 02:59:59", "2020-03-29 03:00:00 CEST +0200", true}, - {"Europe/Amsterdam", "2020-03-29 02:59:59.999999999", "2020-03-29 03:00:00 CEST +0200", true}, - {"Europe/Amsterdam", "2020-03-29 03:00:00.000000001", "2020-03-29 03:00:00 CEST +0200", true}, - } - - for _, tt := range tests { - loc, err := time.LoadLocation(tt.TZ) - require.NoError(t, err) - tm, err := time.ParseInLocation("2006-01-02 15:04:05", tt.dt, loc) - require.NoError(t, err) - tp, err := FindZoneTransition(tm) - if !tt.Success { - require.Error(t, err) - } else { - require.NoError(t, err) - require.Equal(t, tt.Expect, tp.Format("2006-01-02 15:04:05.999999999 MST -0700")) - } - } -} - func TestAdjustedGoTime(t *testing.T) { tests := []struct { TZ string @@ -361,9 +322,9 @@ func TestAdjustedGoTime(t *testing.T) { require.NoError(t, err) tp, err := tt.dt.AdjustedGoTime(loc) if !tt.Success { - require.Error(t, err) + require.Error(t, err, tp.Format("2006-01-02 15:04:05.999999999 MST -0700")) } else { - require.NoError(t, err) + require.NoError(t, err, tp.Format("2006-01-02 15:04:05.999999999 MST -0700")) require.Equal(t, tt.Expect, tp.Format("2006-01-02 15:04:05.999999999 MST -0700")) } } diff --git a/util/chunk/alloc.go b/util/chunk/alloc.go index 44fbb126a4989..af3385a644389 100644 --- a/util/chunk/alloc.go +++ b/util/chunk/alloc.go @@ -128,9 +128,9 @@ func (a *allocator) Reset() { a.allocated = a.allocated[:0] //column objects and put them to the column allocator for reuse. - for _, pool := range a.columnAlloc.pool { + for id, pool := range a.columnAlloc.pool { for _, col := range pool.allocColumns { - if (len(pool.freeColumns) < a.columnAlloc.freeColumnsPerType) && (!col.avoidReusing) && (cap(col.data) < MaxCachedLen) { + if (len(pool.freeColumns) < a.columnAlloc.freeColumnsPerType) && checkColumnType(id, col) { col.reset() pool.freeColumns = append(pool.freeColumns, col) } @@ -139,6 +139,27 @@ func (a *allocator) Reset() { } } +// checkColumnType check whether the conditions for entering the corresponding queue are met +// column Reset may change type +func checkColumnType(id int, col *Column) bool { + if col.avoidReusing { + return false + } + + if id == varElemLen { + //Take up too much memory, + if cap(col.data) > MaxCachedLen { + return false + } + return col.elemBuf == nil + } + + if col.elemBuf == nil { + return false + } + return id == cap(col.elemBuf) +} + var _ ColumnAllocator = &poolColumnAllocator{} type poolColumnAllocator struct { diff --git a/util/chunk/alloc_test.go b/util/chunk/alloc_test.go index edad5e3008e77..7c09d818d1222 100644 --- a/util/chunk/alloc_test.go +++ b/util/chunk/alloc_test.go @@ -270,3 +270,22 @@ func TestColumnAllocatorLimit(t *testing.T) { alloc = NewAllocator() require.False(t, alloc.CheckReuseAllocSize()) } + +func TestColumnAllocatorCheck(t *testing.T) { + fieldTypes := []*types.FieldType{ + types.NewFieldTypeBuilder().SetType(mysql.TypeFloat).BuildP(), + types.NewFieldTypeBuilder().SetType(mysql.TypeDatetime).BuildP(), + } + InitChunkAllocSize(10, 20) + alloc := NewAllocator() + for i := 0; i < 4; i++ { + alloc.Alloc(fieldTypes, 5, 10) + } + col := alloc.columnAlloc.NewColumn(types.NewFieldTypeBuilder().SetType(mysql.TypeFloat).BuildP(), 10) + col.Reset(types.ETDatetime) + alloc.Reset() + num := alloc.columnAlloc.pool[getFixedLen(types.NewFieldTypeBuilder().SetType(mysql.TypeFloat).BuildP())].Len() + require.Equal(t, num, 4) + num = alloc.columnAlloc.pool[getFixedLen(types.NewFieldTypeBuilder().SetType(mysql.TypeDatetime).BuildP())].Len() + require.Equal(t, num, 4) +} diff --git a/util/execdetails/execdetails.go b/util/execdetails/execdetails.go index 4c0670e93eb97..2c7103a7901fb 100644 --- a/util/execdetails/execdetails.go +++ b/util/execdetails/execdetails.go @@ -396,6 +396,8 @@ func (crs *CopRuntimeStats) GetActRows() (totalRows int64) { // MergeBasicStats traverses basicCopRuntimeStats in the CopRuntimeStats and collects some useful information. func (crs *CopRuntimeStats) MergeBasicStats() (procTimes []time.Duration, totalTime time.Duration, totalTasks, totalLoops, totalThreads int32) { + crs.Lock() + defer crs.Unlock() procTimes = make([]time.Duration, 0, 32) for _, instanceStats := range crs.stats { for _, stat := range instanceStats { diff --git a/util/ranger/ranger_test.go b/util/ranger/ranger_test.go index 230252bc8489d..522494f447e04 100644 --- a/util/ranger/ranger_test.go +++ b/util/ranger/ranger_test.go @@ -891,6 +891,7 @@ func TestIndexStringIsTrueRange(t *testing.T) { testKit := testkit.NewTestKit(t, store) testKit.MustExec("use test") + testKit.MustExec("set tidb_cost_model_version=2") testKit.MustExec("drop table if exists t0") testKit.MustExec("CREATE TABLE t0(c0 TEXT(10));") testKit.MustExec("INSERT INTO t0(c0) VALUES (1);") diff --git a/util/ranger/testdata/ranger_suite_out.json b/util/ranger/testdata/ranger_suite_out.json index 174731cac0827..e4de18e162c7b 100644 --- a/util/ranger/testdata/ranger_suite_out.json +++ b/util/ranger/testdata/ranger_suite_out.json @@ -32,9 +32,9 @@ { "SQL": "explain format = 'brief' select * from t0 where c0", "Result": [ - "TableReader 0.80 root data:Selection", - "└─Selection 0.80 cop[tikv] test.t0.c0", - " └─TableFullScan 1.00 cop[tikv] table:t0 keep order:false" + "IndexReader 1.00 root index:Selection", + "└─Selection 1.00 cop[tikv] test.t0.c0", + " └─IndexFullScan 1.00 cop[tikv] table:t0, index:i0(c0) keep order:false" ] }, { @@ -56,17 +56,17 @@ { "SQL": "explain format = 'brief' select * from t0 where c0 is true", "Result": [ - "TableReader 0.80 root data:Selection", - "└─Selection 0.80 cop[tikv] istrue(cast(test.t0.c0, double BINARY))", - " └─TableFullScan 1.00 cop[tikv] table:t0 keep order:false" + "IndexReader 1.00 root index:Selection", + "└─Selection 1.00 cop[tikv] istrue(cast(test.t0.c0, double BINARY))", + " └─IndexFullScan 1.00 cop[tikv] table:t0, index:i0(c0) keep order:false" ] }, { "SQL": "explain format = 'brief' select * from t0 where c0 is false", "Result": [ - "TableReader 0.80 root data:Selection", - "└─Selection 0.80 cop[tikv] isfalse(cast(test.t0.c0, double BINARY))", - " └─TableFullScan 1.00 cop[tikv] table:t0 keep order:false" + "IndexReader 1.00 root index:Selection", + "└─Selection 1.00 cop[tikv] isfalse(cast(test.t0.c0, double BINARY))", + " └─IndexFullScan 1.00 cop[tikv] table:t0, index:i0(c0) keep order:false" ] }, { diff --git a/util/security.go b/util/security.go index 648c6c76fa435..ce89633cbad57 100644 --- a/util/security.go +++ b/util/security.go @@ -231,9 +231,21 @@ func NewTLSConfig(opts ...TLSConfigOption) (*tls.Config, error) { return err } + intermediates := x509.NewCertPool() + for _, certBytes := range rawCerts[1:] { + c, err2 := x509.ParseCertificate(certBytes) + if err2 != nil { + return err2 + } + intermediates.AddCert(c) + } + certPoolMu.RLock() defer certPoolMu.RUnlock() - if _, err = cert.Verify(x509.VerifyOptions{Roots: certPool}); err != nil { + if _, err = cert.Verify(x509.VerifyOptions{ + Roots: certPool, + Intermediates: intermediates, + }); err != nil { return errors.Wrap(err, "can't verify certificate, maybe different CA is used") } return nil