diff --git a/Makefile b/Makefile index 45a5e33ec782d..a01f8c7e7634d 100644 --- a/Makefile +++ b/Makefile @@ -422,7 +422,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/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/lightning/restore/meta_manager.go b/br/pkg/lightning/restore/meta_manager.go index 659a33c579ef0..2d9875ad56960 100644 --- a/br/pkg/lightning/restore/meta_manager.go +++ b/br/pkg/lightning/restore/meta_manager.go @@ -1186,9 +1186,12 @@ func getGlobalAutoIDAlloc(store kv.Storage, dbID int64, tblInfo *model.TableInfo return nil, errors.New("internal error: dbID should not be 0") } - // We don't need the cache here because we allocate all IDs at once. - // The argument for CustomAutoIncCacheOption is the cache step. step 1 means no cache. - noCache := autoid.CustomAutoIncCacheOption(1) + // We don't need autoid cache here because we allocate all IDs at once. + // The argument for CustomAutoIncCacheOption is the cache step. Step 1 means no cache, + // but step 1 will enable an experimental feature, so we use step 2 here. + // + // See https://github.com/pingcap/tidb/issues/38442 for more details. + noCache := autoid.CustomAutoIncCacheOption(2) tblVer := autoid.AllocOptionTableInfoVersion(tblInfo.Version) hasRowID := common.TableHasAutoRowID(tblInfo) 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/br/tests/lightning_exotic_filenames/data/xfn.etn-schema.sql b/br/tests/lightning_exotic_filenames/data/xfn.etn-schema.sql index e2d94bbdf8f32..d004fa92e0b64 100644 --- a/br/tests/lightning_exotic_filenames/data/xfn.etn-schema.sql +++ b/br/tests/lightning_exotic_filenames/data/xfn.etn-schema.sql @@ -1 +1 @@ -create table `exotic``table````name` (a varchar(6) primary key, b int unique auto_increment) auto_increment=80000; \ No newline at end of file +create table `exotic``table````name` (a varchar(6) primary key /*T![clustered_index] NONCLUSTERED */, b int unique auto_increment) auto_increment=80000; diff --git a/br/tests/lightning_exotic_filenames/data/zwk.zwb-schema.sql b/br/tests/lightning_exotic_filenames/data/zwk.zwb-schema.sql index 449584777c299..d9fae1aad0373 100644 --- a/br/tests/lightning_exotic_filenames/data/zwk.zwb-schema.sql +++ b/br/tests/lightning_exotic_filenames/data/zwk.zwb-schema.sql @@ -1 +1 @@ -create table 中文表(a int primary key); +create table 中文表(a int primary key /*T![clustered_index] NONCLUSTERED */); diff --git a/br/tests/lightning_shard_rowid/data/shard_rowid.shr-schema.sql b/br/tests/lightning_shard_rowid/data/shard_rowid.shr-schema.sql index 312b13c1c1118..d544b7fdb84c1 100644 --- a/br/tests/lightning_shard_rowid/data/shard_rowid.shr-schema.sql +++ b/br/tests/lightning_shard_rowid/data/shard_rowid.shr-schema.sql @@ -3,5 +3,5 @@ CREATE TABLE `test` ( `s1` char(10) NOT NULL, `s2` char(10) NOT NULL, `s3` char(10) DEFAULT NULL, - PRIMARY KEY (`s1`,`s2`) -) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin/*!90000 SHARD_ROW_ID_BITS=3 PRE_SPLIT_REGIONS=3 */; \ No newline at end of file + PRIMARY KEY (`s1`,`s2`) /*T![clustered_index] NONCLUSTERED */ +) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin/*!90000 SHARD_ROW_ID_BITS=3 PRE_SPLIT_REGIONS=3 */; diff --git a/br/tests/lightning_tidb_rowid/data/rowid.non_pk-schema.sql b/br/tests/lightning_tidb_rowid/data/rowid.non_pk-schema.sql index 5b5757644b6dd..52ee2729417a3 100644 --- a/br/tests/lightning_tidb_rowid/data/rowid.non_pk-schema.sql +++ b/br/tests/lightning_tidb_rowid/data/rowid.non_pk-schema.sql @@ -1 +1 @@ -create table non_pk (pk varchar(6) primary key); +create table non_pk (pk varchar(6) primary key /*T![clustered_index] NONCLUSTERED */); diff --git a/br/tests/lightning_tidb_rowid/data/rowid.non_pk_auto_inc-schema.sql b/br/tests/lightning_tidb_rowid/data/rowid.non_pk_auto_inc-schema.sql index a71be02c9e8f1..97aa81838b1bc 100644 --- a/br/tests/lightning_tidb_rowid/data/rowid.non_pk_auto_inc-schema.sql +++ b/br/tests/lightning_tidb_rowid/data/rowid.non_pk_auto_inc-schema.sql @@ -4,6 +4,6 @@ CREATE TABLE `non_pk_auto_inc` ( `pk` char(36) NOT NULL, `id` int(11) NOT NULL AUTO_INCREMENT, - PRIMARY KEY (`pk`), + PRIMARY KEY (`pk`) /*T![clustered_index] NONCLUSTERED */, UNIQUE KEY (`id`) ) ENGINE=InnoDB DEFAULT CHARSET=utf8 COLLATE=utf8_bin; diff --git a/ddl/BUILD.bazel b/ddl/BUILD.bazel index 6d44cd6265bfa..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", 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/domain/domain.go b/domain/domain.go index 0076081e71e5c..cd256c581ab7b 100644 --- a/domain/domain.go +++ b/domain/domain.go @@ -114,6 +114,7 @@ type Domain struct { cancel context.CancelFunc indexUsageSyncLease time.Duration dumpFileGcChecker *dumpFileGcChecker + planReplayerHandle *planReplayerHandle expiredTimeStamp4PC types.Time logBackupAdvancer *daemon.OwnerDaemon @@ -1530,6 +1531,19 @@ func (do *Domain) TelemetryRotateSubWindowLoop(ctx sessionctx.Context) { }() } +// SetupPlanReplayerHandle setup plan replayer handle +func (do *Domain) SetupPlanReplayerHandle(ctx sessionctx.Context) { + do.planReplayerHandle = &planReplayerHandle{ + sctx: ctx, + } + do.dumpFileGcChecker.setupPlanReplayerHandle(do.planReplayerHandle) +} + +// GetPlanReplayerHandle returns plan replayer handle +func (do *Domain) GetPlanReplayerHandle() *planReplayerHandle { + return do.planReplayerHandle +} + // DumpFileGcCheckerLoop creates a goroutine that handles `exit` and `gc`. func (do *Domain) DumpFileGcCheckerLoop() { do.wg.Add(1) diff --git a/domain/plan_replayer.go b/domain/plan_replayer.go index 705225abf2231..0b13500be3a7b 100644 --- a/domain/plan_replayer.go +++ b/domain/plan_replayer.go @@ -15,7 +15,8 @@ package domain import ( - "errors" + "context" + "fmt" "io/ioutil" "os" "path/filepath" @@ -24,8 +25,13 @@ import ( "sync" "time" + "github.com/pingcap/errors" "github.com/pingcap/tidb/config" + "github.com/pingcap/tidb/domain/infosync" + "github.com/pingcap/tidb/kv" + "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/util/logutil" + "github.com/pingcap/tidb/util/sqlexec" "go.uber.org/zap" ) @@ -33,8 +39,9 @@ import ( // For now it is used by `plan replayer` and `trace plan` statement type dumpFileGcChecker struct { sync.Mutex - gcLease time.Duration - paths []string + gcLease time.Duration + paths []string + planReplayerHandle *planReplayerHandle } // GetPlanReplayerDirName returns plan replayer directory path. @@ -44,6 +51,10 @@ func GetPlanReplayerDirName() string { return filepath.Join(tidbLogDir, "replayer") } +func parseType(s string) string { + return strings.Split(s, "_")[0] +} + func parseTime(s string) (time.Time, error) { startIdx := strings.LastIndex(s, "_") if startIdx == -1 { @@ -68,6 +79,10 @@ func (p *dumpFileGcChecker) gcDumpFiles(t time.Duration) { } } +func (p *dumpFileGcChecker) setupPlanReplayerHandle(handle *planReplayerHandle) { + p.planReplayerHandle = handle +} + func (p *dumpFileGcChecker) gcDumpFilesByPath(path string, t time.Duration) { files, err := ioutil.ReadDir(path) if err != nil { @@ -84,6 +99,7 @@ func (p *dumpFileGcChecker) gcDumpFilesByPath(path string, t time.Duration) { logutil.BgLogger().Error("[dumpFileGcChecker] parseTime failed", zap.Error(err), zap.String("filename", fileName)) continue } + isPlanReplayer := parseType(fileName) == "replayer" if !createTime.After(gcTime) { err := os.Remove(filepath.Join(path, f.Name())) if err != nil { @@ -91,6 +107,82 @@ func (p *dumpFileGcChecker) gcDumpFilesByPath(path string, t time.Duration) { continue } logutil.BgLogger().Info("dumpFileGcChecker successful", zap.String("filename", fileName)) + if isPlanReplayer && p.planReplayerHandle != nil { + p.planReplayerHandle.deletePlanReplayerStatus(context.Background(), fileName) + } } } } + +type planReplayerHandle struct { + sync.Mutex + sctx sessionctx.Context +} + +// DeletePlanReplayerStatus delete mysql.plan_replayer_status record +func (h *planReplayerHandle) deletePlanReplayerStatus(ctx context.Context, token string) { + ctx1 := kv.WithInternalSourceType(ctx, kv.InternalTxnStats) + h.Lock() + defer h.Unlock() + exec := h.sctx.(sqlexec.SQLExecutor) + _, err := exec.ExecuteInternal(ctx1, fmt.Sprintf("delete from mysql.plan_replayer_status where token = %v", token)) + if err != nil { + logutil.BgLogger().Warn("delete mysql.plan_replayer_status record failed", zap.String("token", token), zap.Error(err)) + } +} + +// InsertPlanReplayerStatus insert mysql.plan_replayer_status record +func (h *planReplayerHandle) InsertPlanReplayerStatus(ctx context.Context, records []PlanReplayerStatusRecord) { + ctx1 := kv.WithInternalSourceType(ctx, kv.InternalTxnStats) + var instance string + serverInfo, err := infosync.GetServerInfo() + if err != nil { + logutil.BgLogger().Error("failed to get server info", zap.Error(err)) + instance = "unknown" + } else { + instance = fmt.Sprintf("%s:%d", serverInfo.IP, serverInfo.Port) + } + for _, record := range records { + if !record.Internal { + if len(record.FailedReason) > 0 { + h.insertExternalPlanReplayerErrorStatusRecord(ctx1, instance, record) + } else { + h.insertExternalPlanReplayerSuccessStatusRecord(ctx1, instance, record) + } + } + } +} + +func (h *planReplayerHandle) insertExternalPlanReplayerErrorStatusRecord(ctx context.Context, instance string, record PlanReplayerStatusRecord) { + h.Lock() + defer h.Unlock() + exec := h.sctx.(sqlexec.SQLExecutor) + _, err := exec.ExecuteInternal(ctx, fmt.Sprintf( + "insert into mysql.plan_replayer_status (origin_sql, fail_reason, instance) values ('%s','%s','%s')", + record.OriginSQL, record.FailedReason, instance)) + if err != nil { + logutil.BgLogger().Warn("insert mysql.plan_replayer_status record failed", + zap.Error(err)) + } +} + +func (h *planReplayerHandle) insertExternalPlanReplayerSuccessStatusRecord(ctx context.Context, instance string, record PlanReplayerStatusRecord) { + h.Lock() + defer h.Unlock() + exec := h.sctx.(sqlexec.SQLExecutor) + _, err := exec.ExecuteInternal(ctx, fmt.Sprintf( + "insert into mysql.plan_replayer_status (origin_sql, token, instance) values ('%s','%s','%s')", + record.OriginSQL, record.Token, instance)) + if err != nil { + logutil.BgLogger().Warn("insert mysql.plan_replayer_status record failed", + zap.Error(err)) + } +} + +// PlanReplayerStatusRecord indicates record in mysql.plan_replayer_status +type PlanReplayerStatusRecord struct { + Internal bool + OriginSQL string + Token string + FailedReason string +} 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/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/executor_test.go b/executor/executor_test.go index dd03c68fd4d74..7160e74209c6f 100644 --- a/executor/executor_test.go +++ b/executor/executor_test.go @@ -165,6 +165,13 @@ func TestPlanReplayer(t *testing.T) { tk.MustQuery("plan replayer dump explain select * from v1") tk.MustQuery("plan replayer dump explain select * from v2") require.True(t, len(tk.Session().GetSessionVars().LastPlanReplayerToken) > 0) + + // clear the status table and assert + tk.MustExec("delete from mysql.plan_replayer_status") + tk.MustQuery("plan replayer dump explain select * from v2") + token := tk.Session().GetSessionVars().LastPlanReplayerToken + rows := tk.MustQuery(fmt.Sprintf("select * from mysql.plan_replayer_status where token = '%v'", token)).Rows() + require.Len(t, rows, 1) } func TestShow(t *testing.T) { diff --git a/executor/infoschema_cluster_table_test.go b/executor/infoschema_cluster_table_test.go index 38fe23036e313..d2e230c6fd766 100644 --- a/executor/infoschema_cluster_table_test.go +++ b/executor/infoschema_cluster_table_test.go @@ -290,7 +290,7 @@ func TestTableStorageStats(t *testing.T) { "test 2", )) rows := tk.MustQuery("select TABLE_NAME from information_schema.TABLE_STORAGE_STATS where TABLE_SCHEMA = 'mysql';").Rows() - result := 37 + result := 38 require.Len(t, rows, result) // More tests about the privileges. diff --git a/executor/join.go b/executor/join.go index 87781a49d7c50..79f40c05db3a8 100644 --- a/executor/join.go +++ b/executor/join.go @@ -20,7 +20,6 @@ import ( "fmt" "runtime/trace" "strconv" - "sync" "sync/atomic" "time" @@ -70,6 +69,8 @@ type HashJoinExec struct { // closeCh add a lock for closing executor. closeCh chan struct{} + worker util.WaitGroupWrapper + waiter util.WaitGroupWrapper joinType plannercore.JoinType requiredRows int64 @@ -92,9 +93,7 @@ type HashJoinExec struct { prepared bool isOuterJoin bool - // joinWorkerWaitGroup is for sync multiple join workers. - joinWorkerWaitGroup sync.WaitGroup - finished atomic.Value + finished atomic.Bool stats *hashJoinRuntimeStats @@ -154,6 +153,7 @@ func (e *HashJoinExec) Close() error { e.probeChkResourceCh = nil e.joinChkResourceCh = nil terror.Call(e.rowContainer.Close) + e.waiter.Wait() } e.outerMatchedStatus = e.outerMatchedStatus[:0] e.buildSideRows = nil @@ -181,9 +181,10 @@ func (e *HashJoinExec) Open(ctx context.Context) error { e.diskTracker = disk.NewTracker(e.id, -1) e.diskTracker.AttachTo(e.ctx.GetSessionVars().StmtCtx.DiskTracker) + e.worker = util.WaitGroupWrapper{} + e.waiter = util.WaitGroupWrapper{} e.closeCh = make(chan struct{}) e.finished.Store(false) - e.joinWorkerWaitGroup = sync.WaitGroup{} if e.probeTypes == nil { e.probeTypes = retTypes(e.probeSideExec) @@ -205,7 +206,7 @@ func (e *HashJoinExec) Open(ctx context.Context) error { func (e *HashJoinExec) fetchProbeSideChunks(ctx context.Context) { hasWaitedForBuild := false for { - if e.finished.Load().(bool) { + if e.finished.Load() { return } @@ -283,24 +284,24 @@ func (e *HashJoinExec) wait4BuildSide() (emptyBuild bool, err error) { // fetchBuildSideRows fetches all rows from build side executor, and append them // to e.buildSideResult. -func (e *HashJoinExec) fetchBuildSideRows(ctx context.Context, chkCh chan<- *chunk.Chunk, doneCh <-chan struct{}) { +func (e *HashJoinExec) fetchBuildSideRows(ctx context.Context, chkCh chan<- *chunk.Chunk, errCh chan<- error, doneCh <-chan struct{}) { defer close(chkCh) var err error failpoint.Inject("issue30289", func(val failpoint.Value) { if val.(bool) { err = errors.Errorf("issue30289 build return error") - e.buildFinished <- errors.Trace(err) + errCh <- errors.Trace(err) return } }) for { - if e.finished.Load().(bool) { + if e.finished.Load() { return } 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) + errCh <- errors.Trace(err) return } failpoint.Inject("errorFetchBuildSideRowsMockOOMPanic", nil) @@ -357,8 +358,7 @@ func (e *HashJoinExec) initializeForProbe() { func (e *HashJoinExec) fetchAndProbeHashTable(ctx context.Context) { e.initializeForProbe() - e.joinWorkerWaitGroup.Add(1) - go util.WithRecovery(func() { + e.worker.RunWithRecover(func() { defer trace.StartRegion(ctx, "HashJoinProbeSideFetcher").End() e.fetchProbeSideChunks(ctx) }, e.handleProbeSideFetcherPanic) @@ -373,14 +373,13 @@ func (e *HashJoinExec) fetchAndProbeHashTable(ctx context.Context) { } for i := uint(0); i < e.concurrency; i++ { - e.joinWorkerWaitGroup.Add(1) workID := i - go util.WithRecovery(func() { + e.worker.RunWithRecover(func() { defer trace.StartRegion(ctx, "HashJoinWorker").End() e.runJoinWorker(workID, probeKeyColIdx, probeNAKeColIdx) }, e.handleJoinWorkerPanic) } - go util.WithRecovery(e.waitJoinWorkersAndCloseResultChan, nil) + e.waiter.RunWithRecover(e.waitJoinWorkersAndCloseResultChan, nil) } func (e *HashJoinExec) handleProbeSideFetcherPanic(r interface{}) { @@ -390,14 +389,12 @@ func (e *HashJoinExec) handleProbeSideFetcherPanic(r interface{}) { if r != nil { e.joinResultCh <- &hashjoinWorkerResult{err: errors.Errorf("%v", r)} } - e.joinWorkerWaitGroup.Done() } func (e *HashJoinExec) handleJoinWorkerPanic(r interface{}) { if r != nil { e.joinResultCh <- &hashjoinWorkerResult{err: errors.Errorf("%v", r)} } - e.joinWorkerWaitGroup.Done() } // Concurrently handling unmatched rows from the hash table @@ -437,15 +434,14 @@ func (e *HashJoinExec) handleUnmatchedRowsFromHashTable(workerID uint) { } func (e *HashJoinExec) waitJoinWorkersAndCloseResultChan() { - e.joinWorkerWaitGroup.Wait() + e.worker.Wait() if e.useOuterToBuild { // Concurrently handling unmatched rows from the hash table at the tail for i := uint(0); i < e.concurrency; i++ { var workerID = i - e.joinWorkerWaitGroup.Add(1) - go util.WithRecovery(func() { e.handleUnmatchedRowsFromHashTable(workerID) }, e.handleJoinWorkerPanic) + e.worker.RunWithRecover(func() { e.handleUnmatchedRowsFromHashTable(workerID) }, e.handleJoinWorkerPanic) } - e.joinWorkerWaitGroup.Wait() + e.worker.Wait() } close(e.joinResultCh) } @@ -481,7 +477,7 @@ func (e *HashJoinExec) runJoinWorker(workerID uint, probeKeyColIdx, probeNAKeyCo naKeyColIdx: probeNAKeyColIdx, } for ok := true; ok; { - if e.finished.Load().(bool) { + if e.finished.Load() { break } select { @@ -1121,7 +1117,7 @@ func (e *HashJoinExec) Next(ctx context.Context, req *chunk.Chunk) (err error) { for i := uint(0); i < e.concurrency; i++ { e.rowIters = append(e.rowIters, chunk.NewIterator4Slice([]chunk.Row{}).(*chunk.Iterator4Slice)) } - go util.WithRecovery(func() { + e.worker.RunWithRecover(func() { defer trace.StartRegion(ctx, "HashJoinHashTableBuilder").End() e.fetchAndBuildHashTable(ctx) }, e.handleFetchAndBuildHashTablePanic) @@ -1164,10 +1160,10 @@ func (e *HashJoinExec) fetchAndBuildHashTable(ctx context.Context) { buildSideResultCh := make(chan *chunk.Chunk, 1) doneCh := make(chan struct{}) fetchBuildSideRowsOk := make(chan error, 1) - go util.WithRecovery( + e.worker.RunWithRecover( func() { defer trace.StartRegion(ctx, "HashJoinBuildSideFetcher").End() - e.fetchBuildSideRows(ctx, buildSideResultCh, doneCh) + e.fetchBuildSideRows(ctx, buildSideResultCh, fetchBuildSideRowsOk, doneCh) }, func(r interface{}) { if r != nil { @@ -1214,7 +1210,7 @@ func (e *HashJoinExec) buildHashTableForList(buildSideResultCh <-chan *chunk.Chu e.ctx.GetSessionVars().MemTracker.FallbackOldAndSetNewAction(actionSpill) } for chk := range buildSideResultCh { - if e.finished.Load().(bool) { + if e.finished.Load() { return nil } if !e.useOuterToBuild { 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/plan_replayer.go b/executor/plan_replayer.go index 1fcc1d85ff24a..d1a2c7a4c56e1 100644 --- a/executor/plan_replayer.go +++ b/executor/plan_replayer.go @@ -290,6 +290,7 @@ func DumpPlanReplayerInfo(ctx context.Context, sctx sessionctx.Context, sessionVars := task.SessionVars execStmts := task.ExecStmts zw := zip.NewWriter(zf) + records := generateRecords(task) defer func() { err = zw.Close() if err != nil { @@ -298,7 +299,12 @@ func DumpPlanReplayerInfo(ctx context.Context, sctx sessionctx.Context, err = zf.Close() if err != nil { logutil.BgLogger().Error("Closing zip file failed", zap.Error(err), zap.String("filename", fileName)) + for i, record := range records { + record.FailedReason = err.Error() + records[i] = record + } } + domain.GetDomain(sctx).GetPlanReplayerHandle().InsertPlanReplayerStatus(ctx, records) }() // Dump config if err = dumpConfig(zw); err != nil { @@ -367,6 +373,20 @@ func DumpPlanReplayerInfo(ctx context.Context, sctx sessionctx.Context, return dumpExplain(sctx, zw, execStmts, task.Analyze) } +func generateRecords(task *PlanReplayerDumpTask) []domain.PlanReplayerStatusRecord { + records := make([]domain.PlanReplayerStatusRecord, 0) + if len(task.ExecStmts) > 0 { + for _, execStmt := range task.ExecStmts { + records = append(records, domain.PlanReplayerStatusRecord{ + OriginSQL: execStmt.Text(), + Token: task.FileName, + Internal: false, + }) + } + } + return records +} + func dumpConfig(zw *zip.Writer) error { cf, err := zw.Create(configFile) if err != nil { 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/expression/extension.go b/expression/extension.go index 90729c698cd3d..2fca7fdfab6d9 100644 --- a/expression/extension.go +++ b/expression/extension.go @@ -21,9 +21,11 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/tidb/extension" + "github.com/pingcap/tidb/parser/auth" "github.com/pingcap/tidb/parser/mysql" "github.com/pingcap/tidb/privilege" "github.com/pingcap/tidb/sessionctx" + "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tidb/util/sem" @@ -183,6 +185,22 @@ func (b *extensionFuncSig) EvalArgs(row chunk.Row) ([]types.Datum, error) { return result, nil } +func (b *extensionFuncSig) ConnectionInfo() *variable.ConnectionInfo { + return b.ctx.GetSessionVars().ConnectionInfo +} + +func (b *extensionFuncSig) User() *auth.UserIdentity { + return b.ctx.GetSessionVars().User +} + +func (b *extensionFuncSig) ActiveRoles() []*auth.RoleIdentity { + return b.ctx.GetSessionVars().ActiveRoles +} + +func (b *extensionFuncSig) CurrentDB() string { + return b.ctx.GetSessionVars().CurrentDB +} + func init() { extension.RegisterExtensionFunc = registerExtensionFunc extension.RemoveExtensionFunc = removeExtensionFunc diff --git a/extension/function.go b/extension/function.go index 47e727088e0c2..d01519e9b751d 100644 --- a/extension/function.go +++ b/extension/function.go @@ -18,13 +18,19 @@ import ( "context" "github.com/pingcap/errors" + "github.com/pingcap/tidb/parser/auth" + "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/chunk" ) -// FunctionContext is a interface to provide context to the custom function +// FunctionContext is an interface to provide context to the custom function type FunctionContext interface { context.Context + User() *auth.UserIdentity + ActiveRoles() []*auth.RoleIdentity + CurrentDB() string + ConnectionInfo() *variable.ConnectionInfo EvalArgs(row chunk.Row) ([]types.Datum, error) } diff --git a/extension/function_test.go b/extension/function_test.go index a397d1638c6fc..c9521e91edadd 100644 --- a/extension/function_test.go +++ b/extension/function_test.go @@ -24,6 +24,7 @@ import ( "github.com/pingcap/tidb/expression" "github.com/pingcap/tidb/extension" "github.com/pingcap/tidb/parser/auth" + "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/testkit" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/chunk" @@ -68,10 +69,64 @@ var customFunc2 = &extension.FunctionDef{ }, } -func TestInvokeFunc(t *testing.T) { - defer func() { - extension.Reset() - }() +func TestExtensionFuncCtx(t *testing.T) { + defer extension.Reset() + extension.Reset() + + invoked := false + var user *auth.UserIdentity + var currentDB string + var activeRoles []*auth.RoleIdentity + var conn *variable.ConnectionInfo + + require.NoError(t, extension.Register("test", extension.WithCustomFunctions([]*extension.FunctionDef{ + { + Name: "custom_get_ctx", + EvalTp: types.ETString, + EvalStringFunc: func(ctx extension.FunctionContext, row chunk.Row) (string, bool, error) { + require.False(t, invoked) + invoked = true + user = ctx.User() + currentDB = ctx.CurrentDB() + activeRoles = ctx.ActiveRoles() + conn = ctx.ConnectionInfo() + return "done", false, nil + }, + }, + }))) + + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("create user u1@localhost") + tk.MustExec("create role r1") + tk.MustExec("grant r1 to u1@localhost") + tk.MustExec("grant ALL ON test.* to u1@localhost") + + tk1 := testkit.NewTestKit(t, store) + require.NoError(t, tk1.Session().Auth(&auth.UserIdentity{Username: "u1", Hostname: "localhost"}, nil, nil)) + tk1.MustExec("set role r1") + tk1.MustExec("use test") + tk1.Session().GetSessionVars().ConnectionInfo = &variable.ConnectionInfo{ + ConnectionID: 12345, + User: "u1", + } + + tk1.MustQuery("select custom_get_ctx()").Check(testkit.Rows("done")) + + require.True(t, invoked) + require.NotNil(t, user) + require.Equal(t, *tk1.Session().GetSessionVars().User, *user) + require.Equal(t, "test", currentDB) + require.NotNil(t, conn) + require.Equal(t, *tk1.Session().GetSessionVars().ConnectionInfo, *conn) + require.Equal(t, 1, len(activeRoles)) + require.Equal(t, auth.RoleIdentity{Username: "r1", Hostname: "%"}, *activeRoles[0]) +} + +func TestInvokeExtensionFunc(t *testing.T) { + defer extension.Reset() + extension.Reset() extension.Reset() orgFuncList := expression.GetBuiltinList() @@ -99,7 +154,7 @@ func TestInvokeFunc(t *testing.T) { require.EqualError(t, tk2.ExecToErr("select custom_func2(1, 2)"), "[expression:1305]FUNCTION test.custom_func2 does not exist") } -func TestFuncDynamicArgLen(t *testing.T) { +func TestExtensionFuncDynamicArgLen(t *testing.T) { defer extension.Reset() extension.Reset() @@ -142,10 +197,8 @@ func TestFuncDynamicArgLen(t *testing.T) { require.EqualError(t, tk.ExecToErr("select dynamic_arg_func(1, 2)"), expectedErrMsg) } -func TestRegisterFunc(t *testing.T) { - defer func() { - extension.Reset() - }() +func TestRegisterExtensionFunc(t *testing.T) { + defer extension.Reset() // nil func extension.Reset() @@ -213,7 +266,7 @@ func checkFuncList(t *testing.T, orgList []string, customFuncs ...string) { require.Equal(t, checkList, expression.GetBuiltinList()) } -func TestFuncPrivilege(t *testing.T) { +func TestExtensionFuncPrivilege(t *testing.T) { defer func() { extension.Reset() sem.Disable() diff --git a/infoschema/tables_test.go b/infoschema/tables_test.go index 129a98ea921f3..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;") diff --git a/planner/core/cbo_test.go b/planner/core/cbo_test.go index d9c4dba7b676d..340c0e2830162 100644 --- a/planner/core/cbo_test.go +++ b/planner/core/cbo_test.go @@ -435,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/integration_test.go b/planner/core/integration_test.go index 889983c09f0ef..d242233559b7b 100644 --- a/planner/core/integration_test.go +++ b/planner/core/integration_test.go @@ -1321,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\"") @@ -5698,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))") @@ -5727,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)") @@ -7089,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);") @@ -7110,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) } @@ -7826,3 +7830,14 @@ func TestNullConditionForPrefixIndex(t *testing.T) { " └─StreamAgg_9 1.00 cop[tikv] funcs:count(1)->Column#7", " └─IndexRangeScan_17 99.90 cop[tikv] table:t1, index:idx2(c1, c2) range:[\"0xfff\" -inf,\"0xfff\" +inf], keep order:false, stats:pseudo")) } + +func TestAutoIncrementCheckWithCheckConstraint(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec(`CREATE TABLE t ( + id INTEGER NOT NULL AUTO_INCREMENT, + CHECK (id IN (0, 1)), + KEY idx_autoinc_id (id) + )`) +} diff --git a/planner/core/physical_plan_test.go b/planner/core/physical_plan_test.go index dece0b960bf21..36676cb9b1fdc 100644 --- a/planner/core/physical_plan_test.go +++ b/planner/core/physical_plan_test.go @@ -407,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) @@ -443,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 { @@ -1611,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 { @@ -1766,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));`) @@ -2169,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)") @@ -2205,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`) @@ -2279,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)") @@ -2317,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_ver2.go b/planner/core/plan_cost_ver2.go index b717b8580d2b5..0f4c936fa4e46 100644 --- a/planner/core/plan_cost_ver2.go +++ b/planner/core/plan_cost_ver2.go @@ -161,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().Columns) + rowSize := getAvgRowSize(p.stats, p.schema.Columns) netFactor := getTaskNetFactorVer2(p, taskType) concurrency := float64(p.ctx.GetSessionVars().DistSQLScanConcurrency()) @@ -186,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().Columns) + rowSize := getAvgRowSize(p.stats, p.schema.Columns) netFactor := getTaskNetFactorVer2(p, taskType) concurrency := float64(p.ctx.GetSessionVars().DistSQLScanConcurrency()) childType := property.CopSingleReadTaskType diff --git a/planner/core/preprocess.go b/planner/core/preprocess.go index 103f03f7eaf7b..51fa3b365af5d 100644 --- a/planner/core/preprocess.go +++ b/planner/core/preprocess.go @@ -675,6 +675,10 @@ func checkAutoIncrementOp(colDef *ast.ColumnDef, index int) (bool, error) { func isConstraintKeyTp(constraints []*ast.Constraint, colDef *ast.ColumnDef) bool { for _, c := range constraints { + // ignore constraint check + if c.Tp == ast.ConstraintCheck { + continue + } if c.Keys[0].Expr != nil { continue } diff --git a/planner/core/rule_result_reorder_test.go b/planner/core/rule_result_reorder_test.go index 0880f6426c151..09e0f1da35783 100644 --- a/planner/core/rule_result_reorder_test.go +++ b/planner/core/rule_result_reorder_test.go @@ -158,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/testdata/analyze_suite_out.json b/planner/core/testdata/analyze_suite_out.json index 750fc736a7f6e..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", diff --git a/planner/core/testdata/integration_suite_out.json b/planner/core/testdata/integration_suite_out.json index e223d6604e3f1..304d84c59486a 100644 --- a/planner/core/testdata/integration_suite_out.json +++ b/planner/core/testdata/integration_suite_out.json @@ -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" @@ -2621,15 +2620,15 @@ { "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 160879.40 root CARTESIAN left outer semi join", + "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 129653.38 root 1->Column#28", - " └─Apply_22 3.00 129653.08 root CARTESIAN left outer join", - " ├─TableReader_24(Build) 3.00 58.13 root data:TableFullScan_23", - " │ └─TableFullScan_23 3.00 681.92 cop[tikv] table:t 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", @@ -4314,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": [ @@ -4326,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 @@ -4336,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": [ diff --git a/planner/core/testdata/plan_suite_out.json b/planner/core/testdata/plan_suite_out.json index adbd39cf82e7e..43a23f6cf075f 100644 --- a/planner/core/testdata/plan_suite_out.json +++ b/planner/core/testdata/plan_suite_out.json @@ -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", @@ -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" } ] }, @@ -5171,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" ], @@ -5189,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", @@ -5239,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", @@ -5253,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/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_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/session/bootstrap.go b/session/bootstrap.go index aeeb625e25bc0..86ad22b572045 100644 --- a/session/bootstrap.go +++ b/session/bootstrap.go @@ -430,6 +430,16 @@ const ( CreateMDLView = `CREATE OR REPLACE VIEW mysql.tidb_mdl_view as ( select JOB_ID, DB_NAME, TABLE_NAME, QUERY, SESSION_ID, TxnStart, TIDB_DECODE_SQL_DIGESTS(ALL_SQL_DIGESTS, 4096) AS SQL_DIGESTS from information_schema.ddl_jobs, information_schema.CLUSTER_TIDB_TRX, information_schema.CLUSTER_PROCESSLIST where ddl_jobs.STATE = 'running' and find_in_set(ddl_jobs.table_id, CLUSTER_TIDB_TRX.RELATED_TABLE_IDS) and CLUSTER_TIDB_TRX.SESSION_ID=CLUSTER_PROCESSLIST.ID );` + + // CreatePlanReplayerStatusTable is a table about plan replayer status + CreatePlanReplayerStatusTable = `CREATE TABLE IF NOT EXISTS mysql.plan_replayer_status ( + sql_digest VARCHAR(128), + plan_digest VARCHAR(128), + origin_sql TEXT, + token VARCHAR(128), + update_time TIMESTAMP NOT NULL DEFAULT CURRENT_TIMESTAMP ON UPDATE CURRENT_TIMESTAMP, + fail_reason TEXT, + instance VARCHAR(512) NOT NULL comment 'address of the TiDB instance executing the plan replayer job');` ) // bootstrap initiates system DB for a store. @@ -644,11 +654,13 @@ const ( version99 = 99 // version100 converts server-memory-quota to a sysvar version100 = 100 + // version101 add mysql.plan_replayer_status table + version101 = 101 ) // currentBootstrapVersion is defined as a variable, so we can modify its value for testing. // please make sure this is the largest version -var currentBootstrapVersion int64 = version100 +var currentBootstrapVersion int64 = version101 // DDL owner key's expired time is ManagerSessionTTL seconds, we should wait the time and give more time to have a chance to finish it. var internalSQLTimeout = owner.ManagerSessionTTL + 15 @@ -753,6 +765,7 @@ var ( upgradeToVer97, upgradeToVer98, upgradeToVer100, + upgradeToVer101, } ) @@ -1987,6 +2000,13 @@ func upgradeToVer98(s Session, ver int64) { doReentrantDDL(s, "ALTER TABLE mysql.user ADD COLUMN IF NOT EXISTS `Token_issuer` varchar(255)") } +func upgradeToVer101(s Session, ver int64) { + if ver >= version101 { + return + } + doReentrantDDL(s, CreatePlanReplayerStatusTable) +} + func upgradeToVer99Before(s Session, ver int64) bool { if ver >= version99 { return false @@ -2122,6 +2142,8 @@ func doDDLWorks(s Session) { mustExecute(s, CreateAdvisoryLocks) // Create mdl view. mustExecute(s, CreateMDLView) + // Create plan_replayer_status table + mustExecute(s, CreatePlanReplayerStatusTable) } // inTestSuite checks if we are bootstrapping in the context of tests. diff --git a/session/session.go b/session/session.go index d69f7b94b5acb..268c1a9f84092 100644 --- a/session/session.go +++ b/session/session.go @@ -2895,7 +2895,7 @@ func BootstrapSession(store kv.Storage) (*domain.Domain, error) { analyzeConcurrencyQuota := int(config.GetGlobalConfig().Performance.AnalyzePartitionConcurrencyQuota) concurrency := int(config.GetGlobalConfig().Performance.StatsLoadConcurrency) - ses, err := createSessions(store, 7+concurrency+analyzeConcurrencyQuota) + ses, err := createSessions(store, 7) if err != nil { return nil, err } @@ -2969,21 +2969,33 @@ func BootstrapSession(store kv.Storage) (*domain.Domain, error) { }() } + // setup dumpFileGcChecker + dom.SetupPlanReplayerHandle(ses[6]) + dom.DumpFileGcCheckerLoop() + // A sub context for update table stats, and other contexts for concurrent stats loading. cnt := 1 + concurrency + syncStatsCtxs, err := createSessions(store, cnt) + if err != nil { + return nil, err + } subCtxs := make([]sessionctx.Context, cnt) for i := 0; i < cnt; i++ { - subCtxs[i] = sessionctx.Context(ses[6+i]) + subCtxs[i] = sessionctx.Context(syncStatsCtxs[i]) } if err = dom.LoadAndUpdateStatsLoop(subCtxs); err != nil { return nil, err } + + analyzeCtxs, err := createSessions(store, analyzeConcurrencyQuota) + if err != nil { + return nil, err + } subCtxs2 := make([]sessionctx.Context, analyzeConcurrencyQuota) for i := 0; i < analyzeConcurrencyQuota; i++ { - subCtxs2[i] = ses[7+concurrency+i] + subCtxs2[i] = analyzeCtxs[i] } dom.SetupAnalyzeExec(subCtxs2) - dom.DumpFileGcCheckerLoop() dom.LoadSigningCertLoop(cfg.Security.SessionTokenSigningCert, cfg.Security.SessionTokenSigningKey) if raw, ok := store.(kv.EtcdBackend); ok { 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/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/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/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 e4520a855a941..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", 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/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 diff --git a/util/wait_group_wrapper.go b/util/wait_group_wrapper.go index 16c8704920a28..3fb72049f1365 100644 --- a/util/wait_group_wrapper.go +++ b/util/wait_group_wrapper.go @@ -43,7 +43,7 @@ func (w *WaitGroupWrapper) RunWithRecover(exec func(), recoverFn func(r interfac go func() { defer func() { r := recover() - if r != nil && recoverFn != nil { + if recoverFn != nil { recoverFn(r) } w.Done()