From 2565d072825b47e33e4db55c52d5da44874b3979 Mon Sep 17 00:00:00 2001 From: Lynn Date: Fri, 6 Jan 2023 01:43:53 +0800 Subject: [PATCH 01/25] ddl: support dist-reorg --- ddl/attributes_sql_test.go | 4 + ddl/backfilling.go | 304 +++++++++++++++--- ddl/column.go | 7 +- ddl/db_integration_test.go | 20 ++ ddl/ddl.go | 53 ++- ddl/ddl_test.go | 5 + ddl/ddl_worker.go | 24 +- ddl/ddl_worker_util.go | 156 +++++++++ ddl/ddl_workerpool.go | 87 +++-- ddl/ddl_workerpool_test.go | 28 +- ddl/delete_range.go | 2 +- ddl/index.go | 135 +++++++- ddl/job_table.go | 212 +++++++++++- ddl/job_table_test.go | 99 +++++- ddl/reorg.go | 3 +- go.mod | 3 +- go.sum | 11 +- parser/model/ddl.go | 4 +- .../featuretag/distributereorg/default.go | 2 +- util/gpool/spmc/spmcpool.go | 4 +- 20 files changed, 1035 insertions(+), 128 deletions(-) create mode 100644 ddl/ddl_worker_util.go diff --git a/ddl/attributes_sql_test.go b/ddl/attributes_sql_test.go index 95f881e6fb3fe..0b7ae03fa5f4a 100644 --- a/ddl/attributes_sql_test.go +++ b/ddl/attributes_sql_test.go @@ -73,6 +73,8 @@ func TestAlterTableAttributes(t *testing.T) { // without equal tk.MustExec(`alter table alter_t attributes " merge_option=allow ";`) tk.MustExec(`alter table alter_t attributes " merge_option=allow , key=value ";`) + + tk.MustExec("drop table alter_t") } func TestAlterTablePartitionAttributes(t *testing.T) { @@ -134,6 +136,8 @@ PARTITION BY RANGE (c) ( require.Len(t, rows4, 1) require.NotEqual(t, rows3[0][3], rows4[0][3]) require.NotEqual(t, rows[0][3], rows4[0][3]) + + tk.MustExec("drop table alter_p") } func TestTruncateTable(t *testing.T) { diff --git a/ddl/backfilling.go b/ddl/backfilling.go index a7c23a545208e..64eb19e9ebb56 100644 --- a/ddl/backfilling.go +++ b/ddl/backfilling.go @@ -67,7 +67,7 @@ const ( minGenTaskBatch = 1024 minDistTaskCnt = 16 retrySQLTimes = 3 - retrySQLInterval = 500 * time.Millisecond + retrySQLInterval = 300 * time.Millisecond ) func (bT backfillerType) String() string { @@ -122,7 +122,7 @@ func GetOracleTimeWithStartTS(se *session) (time.Time, error) { return oracle.GetTimeFromTS(txn.StartTS()).UTC(), nil } -// GetOracleTime returns the current time from TS. +// GetOracleTime returns the current time from TS without txn. func GetOracleTime(store kv.Storage) (time.Time, error) { currentVer, err := store.CurrentVersion(kv.GlobalTxnScope) if err != nil { @@ -189,7 +189,70 @@ func GetLeaseGoTime(currTime time.Time, lease time.Duration) types.Time { // Instead, it is divided into batches, each time a kv transaction completes the backfilling // of a partial batch. +// backfillTaskContext is the context of the batch adding indices or updating column values. +// After finishing the batch adding indices or updating column values, result in backfillTaskContext will be merged into backfillResult. +type backfillTaskContext struct { + nextKey kv.Key + done bool + addedCount int + scanCount int + warnings map[errors.ErrorID]*terror.Error + warningsCount map[errors.ErrorID]int64 + finishTS uint64 +} + +type backfillWorkerContext struct { + currID int + mu sync.Mutex + sessCtxs []sessionctx.Context + backfillWorkers []*backfillWorker +} + +type newBackfillerFunc func(bfCtx *backfillCtx) (bf backfiller, err error) + +func newBackfillWorkerContext(d *ddl, schemaName string, tbl table.Table, workerCnt int, reorgTp model.ReorgType, bfFunc newBackfillerFunc) (*backfillWorkerContext, error) { + if workerCnt <= 0 { + return nil, nil + } + + bws, err := d.backfillCtxPool.batchGet(workerCnt) + if err != nil || len(bws) == 0 { + logutil.BgLogger().Debug("[ddl] no backfill worker available now", zap.Error(err)) + return nil, errors.Trace(err) + } + seCtxs := make([]sessionctx.Context, 0, len(bws)) + for i := 0; i < len(bws); i++ { + se, err := d.sessPool.get() + if err != nil { + logutil.BgLogger().Fatal("[ddl] dispatch backfill jobs loop get session failed, it should not happen, please try restart TiDB", zap.Error(err)) + } + sess := newSession(se) + bfCtx := newBackfillCtx(d.ddlCtx, 0, sess, reorgTp, schemaName, tbl) + bf, err := bfFunc(bfCtx) + if err != nil { + logutil.BgLogger().Fatal("[ddl] dispatch backfill jobs loop new add index worker failed, it should not happen, please try restart TiDB", zap.Error(err)) + return nil, errors.Trace(err) + } + seCtxs = append(seCtxs, se) + bws[i].backfiller = bf + } + return &backfillWorkerContext{backfillWorkers: bws, sessCtxs: seCtxs}, nil +} + +func (bwCtx *backfillWorkerContext) GetContext() *backfillWorker { + bwCtx.mu.Lock() + offset := bwCtx.currID % len(bwCtx.backfillWorkers) + bw := bwCtx.backfillWorkers[offset] + bwStr := fmt.Sprintf("%#v", bw) + logutil.BgLogger().Warn("[ddl] GetContext", zap.Int("workers", len(bwCtx.backfillWorkers)), zap.Int("currID", bwCtx.currID), zap.Int("offset", offset), + zap.String("bw", bwStr)) + bwCtx.currID++ + bwCtx.mu.Unlock() + return bw +} + type backfillCtx struct { + id int *ddlCtx reorgTp model.ReorgType sessCtx sessionctx.Context @@ -198,9 +261,13 @@ type backfillCtx struct { batchCnt int } -func newBackfillCtx(ctx *ddlCtx, sessCtx sessionctx.Context, reorgTp model.ReorgType, +func newBackfillCtx(ctx *ddlCtx, id int, sessCtx sessionctx.Context, reorgTp model.ReorgType, schemaName string, tbl table.Table) *backfillCtx { + if id == 0 { + id = int(backfillContextID.Add(1)) + } return &backfillCtx{ + id: id, ddlCtx: ctx, sessCtx: sessCtx, reorgTp: reorgTp, @@ -220,6 +287,130 @@ type backfiller interface { String() string } +type backfilWorkerManager struct { + bwCtx *backfillWorkerContext + wg util.WaitGroupWrapper + exitCh chan struct{} +} + +func newBackfilWorkerManager(bwCtx *backfillWorkerContext) *backfilWorkerManager { + return &backfilWorkerManager{ + bwCtx: bwCtx, + exitCh: make(chan struct{}), + } +} + +func handleTask(task *reorgBackfillTask, _ int, bfWorker *backfillWorker) *backfillResult { + // To prevent different workers from using the same session. + // TODO: backfillWorkerPool is global, and bfWorkers is used in this function, we'd better do something make worker and job's ID can be matched. + bfWorker.runTask(task) + ret := <-bfWorker.resultCh + if dbterror.ErrDDLJobNotFound.Equal(ret.err) { + logutil.BgLogger().Info("the backfill job instance ID or lease is changed", zap.Error(ret.err)) + ret.err = nil + } + + return ret +} + +// backfillJob2Task builds reorg task. +func (dc *ddlCtx) backfillJob2Task(t table.Table, bfJob *BackfillJob) (*reorgBackfillTask, error) { + pt := t.(table.PhysicalTable) + if tbl, ok := t.(table.PartitionedTable); ok { + pt = tbl.GetPartition(bfJob.Meta.PhysicalTableID) + if pt == nil { + return nil, dbterror.ErrCancelledDDLJob.GenWithStack("Can not find partition id %d for table %d", bfJob.Meta.PhysicalTableID, t.Meta().ID) + } + } + endKey := bfJob.EndKey + // TODO: Check reorgInfo.mergingTmpIdx + endK, err := getRangeEndKey(dc.jobContext(bfJob.JobID), dc.store, bfJob.Meta.Priority, pt.RecordPrefix(), bfJob.StartKey, endKey) + if err != nil { + logutil.BgLogger().Info("[ddl] convert backfill job to task, get reverse key failed", zap.String("backfill job", bfJob.AbbrStr()), zap.Error(err)) + } else { + logutil.BgLogger().Info("[ddl] convert backfill job to task, change end key", zap.String("backfill job", + bfJob.AbbrStr()), zap.String("current key", hex.EncodeToString(bfJob.StartKey)), zap.Bool("end include", bfJob.Meta.EndInclude), + zap.String("end key", hex.EncodeToString(endKey)), zap.String("current end key", hex.EncodeToString(endK))) + endKey = endK + } + + return &reorgBackfillTask{ + bfJob: bfJob, + physicalTable: pt, + // TODO: Remove these fields after remove the old logic. + sqlQuery: bfJob.Meta.Query, + startKey: bfJob.StartKey, + endKey: endKey, + endInclude: bfJob.Meta.EndInclude, + priority: bfJob.Meta.Priority}, nil +} + +func GetTasks(d *ddlCtx, sess *session, tbl table.Table, runningJobID int64, concurrency int) ([]*reorgBackfillTask, error) { + // TODO: At present, only add index is processed. In the future, different elements need to be distinguished. + var err error + var bJobs []*BackfillJob + for i := 0; i < retrySQLTimes; i++ { + bJobs, err = GetAndMarkBackfillJobsForOneEle(sess, concurrency, runningJobID, d.uuid, InstanceLease) + if err != nil { + // TODO: test: if all tidbs can't get the unmark backfill job(a tidb mark a backfill job, other tidbs returned, then the tidb can't handle this job.) + if dbterror.ErrDDLJobNotFound.Equal(err) { + logutil.BgLogger().Info("no backfill job, handle backfill task finished") + return nil, err + } + if kv.ErrWriteConflict.Equal(err) { + logutil.BgLogger().Info("GetAndMarkBackfillJobsForOneEle failed", zap.Error(err)) + time.Sleep(retrySQLInterval) + continue + } + } + + tasks := make([]*reorgBackfillTask, 0, len(bJobs)) + for _, bJ := range bJobs { + task, err := d.backfillJob2Task(tbl, bJ) + if err != nil { + return nil, err + } + tasks = append(tasks, task) + } + return tasks, nil + } + + return nil, err +} + +func (bwm *backfilWorkerManager) waitFinalResult(resultCh <-chan *backfillResult) { + bwm.wg.Run(func() { + for { + select { + case result, ok := <-resultCh: + logutil.BgLogger().Warn("=========================== handle backfill task failed", zap.Bool("ok", ok)) + if result.err != nil { + logutil.BgLogger().Warn("handle backfill task failed", zap.Error(result.err)) + return + } + if !ok { + return + } + case <-bwm.exitCh: + logutil.BgLogger().Warn("=========================== exit xxx") + return + } + } + }) +} + +func (bwm *backfilWorkerManager) close(d *ddl) { + close(bwm.exitCh) + bwm.wg.Wait() + + for _, s := range bwm.bwCtx.sessCtxs { + d.sessPool.put(s) + } + for _, w := range bwm.bwCtx.backfillWorkers { + d.backfillCtxPool.put(w) + } +} + type backfillResult struct { taskID int addedCount int @@ -228,17 +419,6 @@ type backfillResult struct { err error } -// backfillTaskContext is the context of the batch adding indices or updating column values. -// After finishing the batch adding indices or updating column values, result in backfillTaskContext will be merged into backfillResult. -type backfillTaskContext struct { - nextKey kv.Key - done bool - addedCount int - scanCount int - warnings map[errors.ErrorID]*terror.Error - warningsCount map[errors.ErrorID]int64 -} - type reorgBackfillTask struct { bfJob *BackfillJob physicalTable table.PhysicalTable @@ -288,7 +468,6 @@ func mergeBackfillCtxToResult(taskCtx *backfillTaskContext, result *backfillResu } type backfillWorker struct { - id int backfiller taskCh chan *reorgBackfillTask resultCh chan *backfillResult @@ -296,11 +475,10 @@ type backfillWorker struct { cancel func() } -func newBackfillWorker(ctx context.Context, id int, bf backfiller) *backfillWorker { +func newBackfillWorker(ctx context.Context, bf backfiller) *backfillWorker { bfCtx, cancel := context.WithCancel(ctx) return &backfillWorker{ backfiller: bf, - id: id, taskCh: make(chan *reorgBackfillTask, 1), resultCh: make(chan *backfillResult, 1), ctx: bfCtx, @@ -325,10 +503,10 @@ func (w *backfillWorker) finishJob(bfJob *BackfillJob) error { } func (w *backfillWorker) String() string { - if w.backfiller == nil { - return fmt.Sprintf("worker %d", w.id) + if w.backfiller == nil || w.GetCtx() == nil { + return "" } - return fmt.Sprintf("worker %d, tp %s", w.id, w.backfiller.String()) + return fmt.Sprintf("backfill-worker %d, tp %s", w.GetCtx().id, w.backfiller.String()) } func (w *backfillWorker) Close() { @@ -396,7 +574,7 @@ func (w *backfillWorker) handleBackfillTask(d *ddlCtx, task *reorgBackfillTask, if num := result.scanCount - lastLogCount; num >= 90000 { lastLogCount = result.scanCount - logutil.BgLogger().Info("[ddl] backfill worker back fill index", + logutil.BgLogger().Info("[ddl] backfill worker back fill index", zap.Stringer("worker", w), zap.Int("addedCount", result.addedCount), zap.Int("scanCount", result.scanCount), zap.String("next key", hex.EncodeToString(taskCtx.nextKey)), zap.Float64("speed(rows/s)", float64(num)/time.Since(lastLogTime).Seconds())) @@ -423,18 +601,52 @@ func (w *backfillWorker) handleBackfillTask(d *ddlCtx, task *reorgBackfillTask, } } logutil.BgLogger().Info("[ddl] backfill worker finish task", - zap.Stringer("worker", w), - zap.Stringer("task", task), + zap.Stringer("worker", w), zap.Stringer("task", task), zap.Int("added count", result.addedCount), zap.Int("scan count", result.scanCount), zap.String("next key", hex.EncodeToString(result.nextKey)), - zap.String("take time", time.Since(startTime).String())) + zap.Stringer("take time", time.Since(startTime))) if ResultCounterForTest != nil && result.err == nil { ResultCounterForTest.Add(1) } return result } +func (w *backfillWorker) runTask(task *reorgBackfillTask) { + logutil.BgLogger().Info("[ddl] backfill worker start", zap.Stringer("worker", w)) + w.GetCtx().setDDLLabelForTopSQL(task.jobID, task.sqlQuery) + + logutil.BgLogger().Debug("[ddl] backfill worker got task", zap.Int("workerID", w.GetCtx().id), zap.String("task", task.String())) + failpoint.Inject("mockBackfillRunErr", func() { + if w.GetCtx().id == 0 { + result := &backfillResult{taskID: task.id, addedCount: 0, nextKey: nil, err: errors.Errorf("mock backfill error")} + w.resultCh <- result + } + }) + failpoint.Inject("mockHighLoadForAddIndex", func() { + sqlPrefixes := []string{"alter"} + topsql.MockHighCPULoad(task.sqlQuery, sqlPrefixes, 5) + }) + failpoint.Inject("mockBackfillSlow", func() { + time.Sleep(100 * time.Millisecond) + }) + + // Change the batch size dynamically. + w.GetCtx().batchCnt = int(variable.GetDDLReorgBatchSize()) + result := w.handleBackfillTask(w.GetCtx().ddlCtx, task, w.backfiller) + w.resultCh <- result + if result.err != nil { + logutil.BgLogger().Info("[ddl] backfill worker exit on error", + zap.Stringer("worker", w), zap.Error(result.err)) + return + } + traceID := task.bfJob.JobID + 100 + finish := injectSpan(traceID, fmt.Sprintf("handle-job-%d-task-%d", task.bfJob.JobID, task.bfJob.ID)) + task.bfJob.RowCount = int64(result.addedCount) + w.finishJob(task.bfJob) + finish() +} + func (w *backfillWorker) run(d *ddlCtx, bf backfiller, job *model.Job) { logutil.BgLogger().Info("[ddl] backfill worker start", zap.Stringer("worker", w)) var curTaskID int @@ -443,22 +655,20 @@ func (w *backfillWorker) run(d *ddlCtx, bf backfiller, job *model.Job) { }, false) for { if util.HasCancelled(w.ctx) { - logutil.BgLogger().Info("[ddl] backfill worker exit on context done", - zap.Stringer("worker", w), zap.Int("workerID", w.id)) + logutil.BgLogger().Info("[ddl] backfill worker exit on context done", zap.Stringer("worker", w)) return } task, more := <-w.taskCh if !more { - logutil.BgLogger().Info("[ddl] backfill worker exit", - zap.Stringer("worker", w), zap.Int("workerID", w.id)) + logutil.BgLogger().Info("[ddl] backfill worker exit", zap.Stringer("worker", w)) return } curTaskID = task.id d.setDDLLabelForTopSQL(job.ID, job.Query) - logutil.BgLogger().Debug("[ddl] backfill worker got task", zap.Int("workerID", w.id), zap.String("task", task.String())) + logutil.BgLogger().Debug("[ddl] backfill worker got task", zap.Int("workerID", w.GetCtx().id), zap.String("task", task.String())) failpoint.Inject("mockBackfillRunErr", func() { - if w.id == 0 { + if w.GetCtx().id == 0 { result := &backfillResult{taskID: task.id, addedCount: 0, nextKey: nil, err: errors.Errorf("mock backfill error")} w.resultCh <- result failpoint.Continue() @@ -480,7 +690,7 @@ func (w *backfillWorker) run(d *ddlCtx, bf backfiller, job *model.Job) { w.resultCh <- result if result.err != nil { logutil.BgLogger().Info("[ddl] backfill worker exit on error", - zap.Stringer("worker", w), zap.Int("workerID", w.id), zap.Error(result.err)) + zap.Stringer("worker", w), zap.Error(result.err)) return } } @@ -629,6 +839,8 @@ func getBatchTasks(t table.Table, reorgInfo *reorgInfo, kvRanges []kv.KeyRange, } // Build reorg tasks. job := reorgInfo.Job + //nolint:forcetypeassert + phyTbl := t.(table.PhysicalTable) jobCtx := reorgInfo.d.jobContext(reorgInfo.Job.ID) for i, keyRange := range kvRanges { startKey := keyRange.StartKey @@ -648,8 +860,6 @@ func getBatchTasks(t table.Table, reorgInfo *reorgInfo, kvRanges []kv.KeyRange, endKey = prefix.PrefixNext() } - //nolint:forcetypeassert - phyTbl := t.(table.PhysicalTable) task := &reorgBackfillTask{ id: i, jobID: reorgInfo.Job.ID, @@ -834,8 +1044,8 @@ func (b *backfillScheduler) adjustWorkerSize() error { ) switch b.tp { case typeAddIndexWorker: - backfillCtx := newBackfillCtx(reorgInfo.d, sessCtx, reorgInfo.ReorgMeta.ReorgTp, job.SchemaName, b.tbl) - idxWorker, err := newAddIndexWorker(b.decodeColMap, i, b.tbl, backfillCtx, + backfillCtx := newBackfillCtx(reorgInfo.d, i, sessCtx, reorgInfo.ReorgMeta.ReorgTp, job.SchemaName, b.tbl) + idxWorker, err := newAddIndexWorker(b.decodeColMap, b.tbl, backfillCtx, jc, job.ID, reorgInfo.currElement.ID, reorgInfo.currElement.TypeKey) if err != nil { if b.canSkipError(err) { @@ -844,22 +1054,22 @@ func (b *backfillScheduler) adjustWorkerSize() error { return err } idxWorker.copReqSenderPool = b.copReqSenderPool - runner = newBackfillWorker(jc.ddlJobCtx, i, idxWorker) + runner = newBackfillWorker(jc.ddlJobCtx, idxWorker) worker = idxWorker case typeAddIndexMergeTmpWorker: - backfillCtx := newBackfillCtx(reorgInfo.d, sessCtx, reorgInfo.ReorgMeta.ReorgTp, job.SchemaName, b.tbl) + backfillCtx := newBackfillCtx(reorgInfo.d, i, sessCtx, reorgInfo.ReorgMeta.ReorgTp, job.SchemaName, b.tbl) tmpIdxWorker := newMergeTempIndexWorker(backfillCtx, i, b.tbl, reorgInfo.currElement.ID, jc) - runner = newBackfillWorker(jc.ddlJobCtx, i, tmpIdxWorker) + runner = newBackfillWorker(jc.ddlJobCtx, tmpIdxWorker) worker = tmpIdxWorker case typeUpdateColumnWorker: // Setting InCreateOrAlterStmt tells the difference between SELECT casting and ALTER COLUMN casting. sessCtx.GetSessionVars().StmtCtx.InCreateOrAlterStmt = true - updateWorker := newUpdateColumnWorker(sessCtx, b.tbl, b.decodeColMap, reorgInfo, jc) - runner = newBackfillWorker(jc.ddlJobCtx, i, updateWorker) + updateWorker := newUpdateColumnWorker(sessCtx, i, b.tbl, b.decodeColMap, reorgInfo, jc) + runner = newBackfillWorker(jc.ddlJobCtx, updateWorker) worker = updateWorker case typeCleanUpIndexWorker: - idxWorker := newCleanUpIndexWorker(sessCtx, b.tbl, b.decodeColMap, reorgInfo, jc) - runner = newBackfillWorker(jc.ddlJobCtx, i, idxWorker) + idxWorker := newCleanUpIndexWorker(sessCtx, i, b.tbl, b.decodeColMap, reorgInfo, jc) + runner = newBackfillWorker(jc.ddlJobCtx, idxWorker) worker = idxWorker default: return errors.New("unknown backfill type") @@ -1060,6 +1270,7 @@ func addBatchBackfillJobs(sess *session, bfWorkerType backfillerType, reorgInfo JobMeta: &model.JobMeta{ SchemaID: reorgInfo.Job.SchemaID, TableID: reorgInfo.Job.TableID, + Type: reorgInfo.Job.Type, Query: reorgInfo.Job.Query, }, } @@ -1085,7 +1296,7 @@ func addBatchBackfillJobs(sess *session, bfWorkerType backfillerType, reorgInfo return nil } -func (*ddlCtx) splitTableToBackfillJobs(sess *session, reorgInfo *reorgInfo, pTbl table.PhysicalTable, isUnique bool, +func (dc *ddlCtx) splitTableToBackfillJobs(sess *session, reorgInfo *reorgInfo, pTbl table.PhysicalTable, isUnique bool, bfWorkerType backfillerType, startKey kv.Key, currBackfillJobID int64) error { endKey := reorgInfo.EndKey isFirstOps := true @@ -1106,7 +1317,8 @@ func (*ddlCtx) splitTableToBackfillJobs(sess *session, reorgInfo *reorgInfo, pTb isFirstOps = false remains := kvRanges[len(batchTasks):] - // TODO: After adding backfillCh do asyncNotify(dc.backfillJobCh). + asyncNotify(dc.backfillJobCh) + dc.asyncNotifyWorker(dc.backfillJobCh, addingBackfillJob, bJobs[0].ID, "backfill_job") logutil.BgLogger().Info("[ddl] split backfill jobs to the backfill table", zap.Int("batchTasksCnt", len(batchTasks)), zap.Int("totalRegionCnt", len(kvRanges)), @@ -1134,6 +1346,7 @@ func (*ddlCtx) splitTableToBackfillJobs(sess *session, reorgInfo *reorgInfo, pTb } func (dc *ddlCtx) controlWritePhysicalTableRecord(sess *session, t table.PhysicalTable, bfWorkerType backfillerType, reorgInfo *reorgInfo) error { + defer injectSpan(reorgInfo.Job.ID, "control-write-records")() startKey, endKey := reorgInfo.StartKey, reorgInfo.EndKey if startKey == nil && endKey == nil { return nil @@ -1162,6 +1375,8 @@ func (dc *ddlCtx) controlWritePhysicalTableRecord(sess *session, t table.Physica idxInfo := model.FindIndexInfoByID(t.Meta().Indices, reorgInfo.currElement.ID) isUnique = idxInfo.Unique } + logutil.BgLogger().Info("[ddl] control write physical table record ----------------- 00", + zap.Int64("jobID", reorgInfo.Job.ID), zap.Reflect("maxBfJob", maxBfJob)) err = dc.splitTableToBackfillJobs(sess, reorgInfo, t, isUnique, bfWorkerType, startKey, currBackfillJobID) if err != nil { return errors.Trace(err) @@ -1247,6 +1462,7 @@ func checkAndHandleInterruptedBackfillJobs(sess *session, jobID, currEleID int64 return nil } + logutil.BgLogger().Info(fmt.Sprintf("finish ***************************** xx sql:%#v", bJobs[0])) for i := 0; i < retrySQLTimes; i++ { err = MoveBackfillJobsToHistoryTable(sess, bJobs[0]) if err == nil { diff --git a/ddl/column.go b/ddl/column.go index 25ce1f81b9557..89b866285c7ec 100644 --- a/ddl/column.go +++ b/ddl/column.go @@ -1116,8 +1116,7 @@ func (w *worker) updateCurrentElement(t table.Table, reorgInfo *reorgInfo) error err := reorgInfo.UpdateReorgMeta(reorgInfo.StartKey, w.sessPool) logutil.BgLogger().Info("[ddl] update column and indexes", zap.Int64("job ID", reorgInfo.Job.ID), - zap.ByteString("element type", reorgInfo.currElement.TypeKey), - zap.Int64("element ID", reorgInfo.currElement.ID), + zap.Stringer("element", reorgInfo.currElement), zap.String("start key", hex.EncodeToString(reorgInfo.StartKey)), zap.String("end key", hex.EncodeToString(reorgInfo.EndKey))) if err != nil { @@ -1147,7 +1146,7 @@ type updateColumnWorker struct { jobContext *JobContext } -func newUpdateColumnWorker(sessCtx sessionctx.Context, t table.PhysicalTable, decodeColMap map[int64]decoder.Column, reorgInfo *reorgInfo, jc *JobContext) *updateColumnWorker { +func newUpdateColumnWorker(sessCtx sessionctx.Context, id int, t table.PhysicalTable, decodeColMap map[int64]decoder.Column, reorgInfo *reorgInfo, jc *JobContext) *updateColumnWorker { if !bytes.Equal(reorgInfo.currElement.TypeKey, meta.ColumnElementKey) { logutil.BgLogger().Error("Element type for updateColumnWorker incorrect", zap.String("jobQuery", reorgInfo.Query), zap.String("reorgInfo", reorgInfo.String())) @@ -1163,7 +1162,7 @@ func newUpdateColumnWorker(sessCtx sessionctx.Context, t table.PhysicalTable, de } rowDecoder := decoder.NewRowDecoder(t, t.WritableCols(), decodeColMap) return &updateColumnWorker{ - backfillCtx: newBackfillCtx(reorgInfo.d, sessCtx, reorgInfo.ReorgMeta.ReorgTp, reorgInfo.SchemaName, t), + backfillCtx: newBackfillCtx(reorgInfo.d, id, sessCtx, reorgInfo.ReorgMeta.ReorgTp, reorgInfo.SchemaName, t), oldColInfo: oldCol, newColInfo: newCol, metricCounter: metrics.BackfillTotalCounter.WithLabelValues(metrics.GenerateReorgLabel("update_col_rate", reorgInfo.SchemaName, t.Meta().Name.String())), diff --git a/ddl/db_integration_test.go b/ddl/db_integration_test.go index 1d482f8cecada..213a23536a2d3 100644 --- a/ddl/db_integration_test.go +++ b/ddl/db_integration_test.go @@ -4311,3 +4311,23 @@ func TestRegexpFunctionsGeneratedColumn(t *testing.T) { tk.MustExec("drop table if exists reg_like") } + +func TestAddIndexX(t *testing.T) { + // TODO: store := testkit.CreateMockStore(t, mockstore.WithDDLChecker()) + store := testkit.CreateMockStore(t) + + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + + tk.MustExec("create table test_add_index(a int, b int not null default '0')") + tk.MustExec("insert into test_add_index values(1, 1),(20,20),(300,300),(4000,4000),(50000,50000),(123456,123456),(1234567,1234567),(12345678,12345678)") + tk.MustExec("split table test_add_index BETWEEN (0) AND (10000000) REGIONS 7;") + tk.MustExec("alter table test_add_index add index idx(b)") + tk.MustExec("admin check table test_add_index") + tk.MustQuery("select count(1) from mysql.tidb_ddl_backfill").Check(testkit.Rows("0")) + tk.MustQuery("select id, ele_id, ele_key, type, state from mysql.tidb_ddl_backfill_history").Check(testkit.Rows("1 1 _idx_ 0 6")) + tk.MustExec("alter table test_add_index add unique index idx1(b)") + tk.MustExec("admin check table test_add_index") + tk.MustQuery("select count(1) from mysql.tidb_ddl_backfill").Check(testkit.Rows("0")) + tk.MustQuery("select id, ele_id, ele_key, type, state from mysql.tidb_ddl_backfill_history").Check(testkit.Rows("1 1 _idx_ 0 6", "1 2 _idx_ 0 6")) +} diff --git a/ddl/ddl.go b/ddl/ddl.go index 224f07739355d..89a0433164c15 100644 --- a/ddl/ddl.go +++ b/ddl/ddl.go @@ -60,6 +60,7 @@ import ( "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tidb/util/dbterror" "github.com/pingcap/tidb/util/gcutil" + "github.com/pingcap/tidb/util/gpool/spmc" "github.com/pingcap/tidb/util/logutil" "github.com/pingcap/tidb/util/mathutil" "github.com/pingcap/tidb/util/sqlexec" @@ -83,8 +84,9 @@ const ( batchAddingJobs = 10 - reorgWorkerCnt = 10 - generalWorkerCnt = 1 + reorgWorkerCnt = 10 + generalWorkerCnt = 1 + backfillWorkerCnt = 32 // checkFlagIndexInJobArgs is the recoverCheckFlag index used in RecoverTable/RecoverSchema job arg list. checkFlagIndexInJobArgs = 1 @@ -272,6 +274,8 @@ type ddl struct { // used in the concurrency ddl. reorgWorkerPool *workerPool generalDDLWorkerPool *workerPool + backfillCtxPool *backfillCtxPool + backfillWorkerPool *spmc.Pool[*reorgBackfillTask, *backfillResult, int, *backfillWorker, *backfillWorkerContext] // get notification if any DDL coming. ddlJobCh chan struct{} } @@ -327,6 +331,8 @@ type ddlCtx struct { statsHandle *handle.Handle tableLockCkr util.DeadTableLockChecker etcdCli *clientv3.Client + // backfillJobCh gets notification if any backfill jobs coming. + backfillJobCh chan struct{} *waitSchemaSyncedController *schemaVersionManager @@ -343,6 +349,12 @@ type ddlCtx struct { // reorgCtxMap maps job ID to reorg context. reorgCtxMap map[int64]*reorgCtx } + // backfillCtx is used for backfill workers. + backfillCtx struct { + sync.RWMutex + jobCtxMap map[int64]*JobContext + backfillCtxMap map[int64]struct{} + } jobCtx struct { sync.RWMutex @@ -428,15 +440,15 @@ func (dc *ddlCtx) setDDLLabelForTopSQL(jobID int64, jobQuery string) { ctx.setDDLLabelForTopSQL(jobQuery) } -func (dc *ddlCtx) setDDLSourceForDiagnosis(job *model.Job) { +func (dc *ddlCtx) setDDLSourceForDiagnosis(jobID int64, jobType model.ActionType) { dc.jobCtx.Lock() defer dc.jobCtx.Unlock() - ctx, exists := dc.jobCtx.jobCtxMap[job.ID] + ctx, exists := dc.jobCtx.jobCtxMap[jobID] if !exists { ctx = NewJobContext() - dc.jobCtx.jobCtxMap[job.ID] = ctx + dc.jobCtx.jobCtxMap[jobID] = ctx } - ctx.setDDLLabelForDiagnosis(job) + ctx.setDDLLabelForDiagnosis(jobType) } func (dc *ddlCtx) getResourceGroupTaggerForTopSQL(jobID int64) tikvrpc.ResourceGroupTagger { @@ -673,6 +685,20 @@ func (d *ddl) prepareWorkers4ConcurrencyDDL() { d.wg.Run(d.startDispatchLoop) } +func (d *ddl) prepareBackfillWorkers() { + workerFactory := func() func() (pools.Resource, error) { + return func() (pools.Resource, error) { + bk := newBackfillWorker(context.Background(), nil) + metrics.DDLCounter.WithLabelValues(fmt.Sprintf("%s_backfill_worker", metrics.CreateDDL)).Inc() + return bk, nil + } + } + d.backfillCtxPool = newBackfillContextPool(pools.NewResourcePool(workerFactory(), backfillWorkerCnt, backfillWorkerCnt, 0)) + d.backfillWorkerPool = spmc.NewSPMCPool[*reorgBackfillTask, *backfillResult, int, *backfillWorker, *backfillWorkerContext]("backfill", int32(backfillWorkerCnt)) + d.backfillJobCh = make(chan struct{}, 1) + d.wg.Run(d.startDispatchBackfillJobsLoop) +} + // Start implements DDL.Start interface. func (d *ddl) Start(ctxPool *pools.ResourcePool) error { logutil.BgLogger().Info("[ddl] start DDL", zap.String("ID", d.uuid), zap.Bool("runWorker", config.GetGlobalConfig().Instance.TiDBEnableDDL.Load())) @@ -690,6 +716,7 @@ func (d *ddl) Start(ctxPool *pools.ResourcePool) error { d.delRangeMgr = d.newDeleteRangeManager(ctxPool == nil) d.prepareWorkers4ConcurrencyDDL() + d.prepareBackfillWorkers() if config.TableLockEnabled() { d.wg.Add(1) @@ -774,6 +801,12 @@ func (d *ddl) close() { if d.generalDDLWorkerPool != nil { d.generalDDLWorkerPool.close() } + if d.backfillCtxPool != nil { + d.backfillCtxPool.close() + } + if d.backfillWorkerPool != nil { + d.backfillWorkerPool.ReleaseAndWait() + } // d.delRangeMgr using sessions from d.sessPool. // Put it before d.sessPool.close to reduce the time spent by d.sessPool.close. @@ -889,15 +922,15 @@ func getJobCheckInterval(job *model.Job, i int) (time.Duration, bool) { } } -func (d *ddl) asyncNotifyWorker(job *model.Job) { +func (d *ddlCtx) asyncNotifyWorker(ch chan struct{}, etcdPath string, jobID int64, jobType string) { // If the workers don't run, we needn't notify workers. if !config.GetGlobalConfig().Instance.TiDBEnableDDL.Load() { return } if d.isOwner() { - asyncNotify(d.ddlJobCh) + asyncNotify(ch) } else { - d.asyncNotifyByEtcd(addingDDLJobConcurrent, job) + d.asyncNotifyByEtcd(etcdPath, jobID, jobType) } } @@ -966,7 +999,7 @@ func (d *ddl) DoDDLJob(ctx sessionctx.Context, job *model.Job) error { sessVars.StmtCtx.IsDDLJobInQueue = true // Notice worker that we push a new job and wait the job done. - d.asyncNotifyWorker(job) + d.asyncNotifyWorker(d.ddlJobCh, addingDDLJobConcurrent, job.ID, job.Type.String()) logutil.BgLogger().Info("[ddl] start DDL job", zap.String("job", job.String()), zap.String("query", job.Query)) var historyJob *model.Job diff --git a/ddl/ddl_test.go b/ddl/ddl_test.go index 6b210d2445c26..dc19436afd9b9 100644 --- a/ddl/ddl_test.go +++ b/ddl/ddl_test.go @@ -58,6 +58,11 @@ var JobNeedGCForTest = jobNeedGC // NewSession is only used for test. var NewSession = newSession +// GetDDLCtx returns ddlCtx for test. +func GetDDLCtx(d DDL) *ddlCtx { + return d.(*ddl).ddlCtx +} + // GetMaxRowID is used for test. func GetMaxRowID(store kv.Storage, priority int, t table.Table, startHandle, endHandle kv.Key) (kv.Key, error) { return getRangeEndKey(NewJobContext(), store, priority, t.RecordPrefix(), startHandle, endHandle) diff --git a/ddl/ddl_worker.go b/ddl/ddl_worker.go index e6fdafd2f62fb..04c0095d376a7 100644 --- a/ddl/ddl_worker.go +++ b/ddl/ddl_worker.go @@ -52,10 +52,14 @@ import ( var ( // ddlWorkerID is used for generating the next DDL worker ID. ddlWorkerID = atomicutil.NewInt32(0) + // backfillContextID is used for generating the next backfill context ID. + backfillContextID = atomicutil.NewInt32(0) // WaitTimeWhenErrorOccurred is waiting interval when processing DDL jobs encounter errors. WaitTimeWhenErrorOccurred = int64(1 * time.Second) mockDDLErrOnce = int64(0) + // TestNotifyBeginTxnCh is used for if the txn is begining in runInTxn. + TestNotifyBeginTxnCh = make(chan struct{}) ) // GetWaitTimeWhenErrorOccurred return waiting interval when processing DDL jobs encounter errors. @@ -162,18 +166,18 @@ func (w *worker) Close() { logutil.Logger(w.logCtx).Info("[ddl] DDL worker closed", zap.Duration("take time", time.Since(startTime))) } -func (d *ddl) asyncNotifyByEtcd(addingDDLJobKey string, job *model.Job) { +func (d *ddlCtx) asyncNotifyByEtcd(etcdPath string, jobID int64, jobType string) { if d.etcdCli == nil { return } - jobID := strconv.FormatInt(job.ID, 10) + jobIDStr := strconv.FormatInt(jobID, 10) timeStart := time.Now() - err := util.PutKVToEtcd(d.ctx, d.etcdCli, 1, addingDDLJobKey, jobID) + err := util.PutKVToEtcd(d.ctx, d.etcdCli, 1, etcdPath, jobIDStr) if err != nil { - logutil.BgLogger().Info("[ddl] notify handling DDL job failed", zap.String("jobID", jobID), zap.Error(err)) + logutil.BgLogger().Info("[ddl] notify handling DDL job failed", zap.String("jobID", jobIDStr), zap.Error(err)) } - metrics.DDLWorkerHistogram.WithLabelValues(metrics.WorkerNotifyDDLJob, job.Type.String(), metrics.RetLabel(err)).Observe(time.Since(timeStart).Seconds()) + metrics.DDLWorkerHistogram.WithLabelValues(metrics.WorkerNotifyDDLJob, jobType, metrics.RetLabel(err)).Observe(time.Since(timeStart).Seconds()) } func asyncNotify(ch chan struct{}) { @@ -673,18 +677,18 @@ var DDLBackfillers = map[model.ActionType]string{ model.ActionDropIndex: "drop_index", } -func getDDLRequestSource(job *model.Job) string { - if tp, ok := DDLBackfillers[job.Type]; ok { +func getDDLRequestSource(jobType model.ActionType) string { + if tp, ok := DDLBackfillers[jobType]; ok { return kv.InternalTxnBackfillDDLPrefix + tp } return kv.InternalTxnDDL } -func (w *JobContext) setDDLLabelForDiagnosis(job *model.Job) { +func (w *JobContext) setDDLLabelForDiagnosis(jobType model.ActionType) { if w.tp != "" { return } - w.tp = getDDLRequestSource(job) + w.tp = getDDLRequestSource(jobType) w.ddlJobCtx = kv.WithInternalSourceType(w.ddlJobCtx, w.ddlJobSourceType()) } @@ -736,7 +740,7 @@ func (w *worker) HandleDDLJobTable(d *ddlCtx, job *model.Job) (int64, error) { txn.SetDiskFullOpt(kvrpcpb.DiskFullOpt_NotAllowedOnFull) } w.setDDLLabelForTopSQL(job.ID, job.Query) - w.setDDLSourceForDiagnosis(job) + w.setDDLSourceForDiagnosis(job.ID, job.Type) jobContext := w.jobContext(job.ID) if tagger := w.getResourceGroupTaggerForTopSQL(job.ID); tagger != nil { txn.SetOption(kv.ResourceGroupTagger, tagger) diff --git a/ddl/ddl_worker_util.go b/ddl/ddl_worker_util.go new file mode 100644 index 0000000000000..07cb6aec69b66 --- /dev/null +++ b/ddl/ddl_worker_util.go @@ -0,0 +1,156 @@ +// 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" + "context" + "fmt" + "math" + "strings" + "time" + + "github.com/pingcap/tidb/util/generic" + "github.com/pingcap/tidb/util/logutil" + minitrace "github.com/tikv/minitrace-go" + "github.com/tikv/minitrace-go/jaeger" + "go.uber.org/zap" + "golang.org/x/exp/slices" +) + +var timeDetails = generic.NewSyncMap[int64, *spanCtx](10) + +type spanCtx struct { + ctx context.Context + root minitrace.TraceHandle +} + +func injectSpan(jobID int64, event string) func() { + if sctx, ok := timeDetails.Load(jobID); ok { + hd := minitrace.StartSpan(sctx.ctx, event) + return func() { + hd.Finish() + } + } + return func() {} +} + +func initializeTrace(jobID int64) { + ctx, root := minitrace.StartRootSpan(context.Background(), + "add-index-worker", uint64(jobID), 0, nil) + timeDetails.Store(jobID, &spanCtx{ + ctx: ctx, + root: root, + }) +} + +func collectTrace(jobID int64) string { + if sctx, ok := timeDetails.Load(jobID); ok { + rootTrace, _ := sctx.root.Collect() + analyzed := analyzeTrace(rootTrace) + if len(rootTrace.Spans) < 1000 { + reportTrace(rootTrace) + } + timeDetails.Delete(jobID) + return analyzed + } + return "" +} + +const batchSize = 512 + +func reportTrace(rootTrace minitrace.Trace) { + buf := bytes.NewBuffer(make([]uint8, 0, 4096)) + for _, subTrace := range splitTraces(rootTrace) { + buf.Reset() + trace := jaeger.MiniSpansToJaegerTrace("add-index", subTrace) + err := jaeger.ThriftCompactEncode(buf, trace) + if err != nil { + logutil.BgLogger().Warn("cannot collectTrace", zap.Error(err)) + return + } + err = jaeger.Send(buf.Bytes(), "127.0.0.1:6831") + if err != nil { + logutil.BgLogger().Warn("cannot collectTrace", zap.Error(err)) + return + } + } +} + +func splitTraces(trace minitrace.Trace) []minitrace.Trace { + var traces []minitrace.Trace + for len(trace.Spans) > batchSize { + traces = append(traces, minitrace.Trace{ + TraceID: trace.TraceID, + Spans: trace.Spans[:batchSize], + }) + trace.Spans = trace.Spans[batchSize:] + } + traces = append(traces, minitrace.Trace{ + TraceID: trace.TraceID, + Spans: trace.Spans, + }) + return traces +} + +func analyzeTrace(trace minitrace.Trace) string { + groupByEvent := make(map[string][]*minitrace.Span, 16) + for i, span := range trace.Spans { + spans := groupByEvent[span.Event] + if len(spans) == 0 { + groupByEvent[span.Event] = []*minitrace.Span{&trace.Spans[i]} + } else { + groupByEvent[span.Event] = append(spans, &trace.Spans[i]) + } + } + orderedEvents := make([]string, 0, len(groupByEvent)) + for event := range groupByEvent { + orderedEvents = append(orderedEvents, event) + } + slices.Sort(orderedEvents) + var sb strings.Builder + sb.WriteString("{") + for i := 0; i < len(orderedEvents); i++ { + spans := groupByEvent[orderedEvents[i]] + sum := uint64(0) + min := uint64(math.MaxUint64) + max := uint64(0) + for _, span := range spans { + dur := span.DurationNs + sum += dur + if dur < min { + min = dur + } + if dur > max { + max = dur + } + } + avg := sum / uint64(len(spans)) + sb.WriteString(orderedEvents[i]) + sb.WriteString(":") + if len(spans) < 20 { + sb.WriteString(fmt.Sprintf("%f", time.Duration(sum).Seconds())) + } else { + sb.WriteString(fmt.Sprintf(`{sum: %f, min: %f, max: %f, avg: %f}`, + time.Duration(sum).Seconds(), time.Duration(min).Seconds(), + time.Duration(max).Seconds(), time.Duration(avg).Seconds())) + } + if i != len(orderedEvents)-1 { + sb.WriteString(", ") + } + } + sb.WriteString("}") + return sb.String() +} diff --git a/ddl/ddl_workerpool.go b/ddl/ddl_workerpool.go index de709b6faeb3b..ece51a333d62b 100644 --- a/ddl/ddl_workerpool.go +++ b/ddl/ddl_workerpool.go @@ -88,38 +88,38 @@ func (wp *workerPool) tp() jobType { return wp.t } -// backfilWorkerPool is used to new backfill worker. -type backfilWorkerPool struct { +// backfillCtxPool is used to new backfill context. +type backfillCtxPool struct { exit atomic.Bool resPool *pools.ResourcePool } -func newBackfillWorkerPool(resPool *pools.ResourcePool) *backfilWorkerPool { - return &backfilWorkerPool{ +func newBackfillContextPool(resPool *pools.ResourcePool) *backfillCtxPool { + return &backfillCtxPool{ exit: *atomic.NewBool(false), resPool: resPool, } } // setCapacity changes the capacity of the pool. -// A setCapacity of 0 is equivalent to closing the backfilWorkerPool. -func (bwp *backfilWorkerPool) setCapacity(capacity int) error { - return bwp.resPool.SetCapacity(capacity) +// A setCapacity of 0 is equivalent to closing the backfillCtxPool. +func (bcp *backfillCtxPool) setCapacity(capacity int) error { + return bcp.resPool.SetCapacity(capacity) } -// get gets backfilWorkerPool from context resource pool. -// Please remember to call put after you finished using backfilWorkerPool. -func (bwp *backfilWorkerPool) get() (*backfillWorker, error) { - if bwp.resPool == nil { +// get gets backfillCtxPool from context resource pool. +// Please remember to call put after you finished using backfillCtxPool. +func (bcp *backfillCtxPool) get() (*backfillWorker, error) { + if bcp.resPool == nil { return nil, nil } - if bwp.exit.Load() { + if bcp.exit.Load() { return nil, errors.Errorf("backfill worker pool is closed") } - // no need to protect bwp.resPool - resource, err := bwp.resPool.TryGet() + // no need to protect bcp.resPool + resource, err := bcp.resPool.TryGet() if err != nil { return nil, errors.Trace(err) } @@ -131,24 +131,65 @@ func (bwp *backfilWorkerPool) get() (*backfillWorker, error) { return worker, nil } +// batchGet gets a batch backfillWorkers from context resource pool. +// Please remember to call batchPut after you finished using backfillWorkerPool. +func (bcp *backfillCtxPool) batchGet(cnt int) ([]*backfillWorker, error) { + if bcp.resPool == nil { + return nil, nil + } + + if bcp.exit.Load() { + return nil, errors.Errorf("backfill worker pool is closed") + } + + workers := make([]*backfillWorker, 0, cnt) + for i := 0; i < cnt; i++ { + // no need to protect bcp.resPool + res, err := bcp.resPool.TryGet() + if err != nil { + return nil, errors.Trace(err) + } + if res == nil { + return workers, nil + } + worker := res.(*backfillWorker) + workers = append(workers, worker) + } + + return workers, nil +} + +// batchPut returns workerPool to context resource pool. +func (bcp *backfillCtxPool) batchPut(wks []*backfillWorker) { + if bcp.resPool == nil || bcp.exit.Load() { + return + } + + // No need to protect bcp.resPool, even the bcp.resPool is closed, the ctx still need to + // put into resPool, because when resPool is closing, it will wait all the ctx returns, then resPool finish closing. + for _, wk := range wks { + bcp.resPool.Put(wk) + } +} + // put returns workerPool to context resource pool. -func (bwp *backfilWorkerPool) put(wk *backfillWorker) { - if bwp.resPool == nil || bwp.exit.Load() { +func (bcp *backfillCtxPool) put(wk *backfillWorker) { + if bcp.resPool == nil || bcp.exit.Load() { return } - // No need to protect bwp.resPool, even the bwp.resPool is closed, the ctx still need to + // No need to protect bcp.resPool, even the bcp.resPool is closed, the ctx still need to // put into resPool, because when resPool is closing, it will wait all the ctx returns, then resPool finish closing. - bwp.resPool.Put(wk) + bcp.resPool.Put(wk) } -// close clean up the backfilWorkerPool. -func (bwp *backfilWorkerPool) close() { +// close clean up the backfillCtxPool. +func (bcp *backfillCtxPool) close() { // Prevent closing resPool twice. - if bwp.resPool == nil || bwp.exit.Load() { + if bcp.resPool == nil || bcp.exit.Load() { return } - bwp.exit.Store(true) + bcp.exit.Store(true) logutil.BgLogger().Info("[ddl] closing workerPool") - bwp.resPool.Close() + bcp.resPool.Close() } diff --git a/ddl/ddl_workerpool_test.go b/ddl/ddl_workerpool_test.go index 123d05abb1d86..df39d463c23f9 100644 --- a/ddl/ddl_workerpool_test.go +++ b/ddl/ddl_workerpool_test.go @@ -37,36 +37,38 @@ func TestDDLWorkerPool(t *testing.T) { func TestBackfillWorkerPool(t *testing.T) { f := func() func() (pools.Resource, error) { return func() (pools.Resource, error) { - wk := newBackfillWorker(context.Background(), 1, nil) + wk := newBackfillWorker(context.Background(), nil) return wk, nil } } - pool := newBackfillWorkerPool(pools.NewResourcePool(f(), 1, 2, 0)) - bwp, err := pool.get() + pool := newBackfillContextPool(pools.NewResourcePool(f(), 1, 2, 0)) + bc, err := pool.get() require.NoError(t, err) - require.Equal(t, 1, bwp.id) + require.NotNil(t, bc) + require.Nil(t, bc.backfiller) // test it to reach the capacity - bwp1, err := pool.get() + bc1, err := pool.get() require.NoError(t, err) - require.Nil(t, bwp1) + require.Nil(t, bc1) // test setCapacity err = pool.setCapacity(2) require.NoError(t, err) - bwp1, err = pool.get() + bc1, err = pool.get() require.NoError(t, err) - require.Equal(t, 1, bwp1.id) - pool.put(bwp) - pool.put(bwp1) + require.NotNil(t, bc) + require.Nil(t, bc.backfiller) + pool.put(bc) + pool.put(bc1) // test close pool.close() pool.close() require.Equal(t, true, pool.exit.Load()) - pool.put(bwp1) + pool.put(bc1) - bwp, err = pool.get() + bc, err = pool.get() require.Error(t, err) require.Equal(t, "backfill worker pool is closed", err.Error()) - require.Nil(t, bwp) + require.Nil(t, bc) } diff --git a/ddl/delete_range.go b/ddl/delete_range.go index 669ff286ea9dd..899bb1bb7143e 100644 --- a/ddl/delete_range.go +++ b/ddl/delete_range.go @@ -266,7 +266,7 @@ func insertJobIntoDeleteRangeTable(ctx context.Context, sctx sessionctx.Context, return errors.Trace(err) } - ctx = kv.WithInternalSourceType(ctx, getDDLRequestSource(job)) + ctx = kv.WithInternalSourceType(ctx, getDDLRequestSource(job.Type)) s := sctx.(sqlexec.SQLExecutor) switch job.Type { case model.ActionDropSchema: diff --git a/ddl/index.go b/ddl/index.go index f4e5ca8381ace..f2e39542303cd 100644 --- a/ddl/index.go +++ b/ddl/index.go @@ -610,6 +610,7 @@ func (w *worker) onCreateIndex(d *ddlCtx, t *meta.Meta, job *model.Job, isPK boo return ver, err } logutil.BgLogger().Info("[ddl] run add index job", zap.String("job", job.String()), zap.Reflect("indexInfo", indexInfo)) + initializeTrace(job.ID) } originalState := indexInfo.State switch indexInfo.State { @@ -655,6 +656,7 @@ func (w *worker) onCreateIndex(d *ddlCtx, t *meta.Meta, job *model.Job, isPK boo job.SnapshotVer = 0 job.SchemaState = model.StateWriteReorganization case model.StateWriteReorganization: + defer injectSpan(job.ID, "write-reorg")() // reorganization -> public tbl, err := getTable(d.store, schemaID, tblInfo) if err != nil { @@ -686,6 +688,9 @@ func (w *worker) onCreateIndex(d *ddlCtx, t *meta.Meta, job *model.Job, isPK boo job.Args = []interface{}{indexInfo.ID, false /*if exists*/, getPartitionIDs(tbl.Meta())} // Finish this job. job.FinishTableJob(model.JobStateDone, model.StatePublic, ver, tblInfo) + details := collectTrace(job.ID) + logutil.BgLogger().Info("[ddl] &&&&&&&&&&&&&&&&&&&&&&&&&&&&&&&&-------------------------- finish add index job", zap.String("job", job.String()), + zap.String("time details", details)) if job.ReorgMeta.ReorgTp == model.ReorgTypeLitMerge { ingest.LitBackCtxMgr.Unregister(job.ID) } @@ -796,7 +801,7 @@ func doReorgWorkForCreateIndexMultiSchema(w *worker, d *ddlCtx, t *meta.Meta, jo return true, ver, err } -func doReorgWorkForCreateIndex(w *worker, d *ddlCtx, t *meta.Meta, job *model.Job, +func doReorgWorkForCreateIndexCopy(w *worker, d *ddlCtx, t *meta.Meta, job *model.Job, tbl table.Table, indexInfo *model.IndexInfo) (done bool, ver int64, err error) { bfProcess := pickBackfillType(w, job) if !bfProcess.NeedMergeProcess() { @@ -879,6 +884,60 @@ func doReorgWorkForCreateIndex(w *worker, d *ddlCtx, t *meta.Meta, job *model.Jo } } +func doReorgWorkForCreateIndex(w *worker, d *ddlCtx, t *meta.Meta, job *model.Job, + tbl table.Table, indexInfo *model.IndexInfo) (done bool, ver int64, err error) { + bfProcess := pickBackfillType(w, job) + if !bfProcess.NeedMergeProcess() { + return runReorgJobAndHandleErr(w, d, t, job, tbl, indexInfo, false) + } + switch indexInfo.BackfillState { + case model.BackfillStateRunning: + logutil.BgLogger().Info("[ddl] index backfill state running", + zap.Int64("job ID", job.ID), zap.String("table", tbl.Meta().Name.O), + zap.Bool("ingest mode", bfProcess == model.ReorgTypeLitMerge), + zap.String("index", indexInfo.Name.O)) + switch bfProcess { + case model.ReorgTypeLitMerge: + done, ver, err = runReorgJobAndHandleErr(w, d, t, job, tbl, indexInfo, false) + if err != nil { + // TODO: + err = tryFallbackToTxnMerge(job, err) + return false, ver, errors.Trace(err) + } + if !done { + return false, ver, nil + } + case model.ReorgTypeTxnMerge: + done, ver, err = runReorgJobAndHandleErr(w, d, t, job, tbl, indexInfo, false) + if err != nil || !done { + return false, ver, errors.Trace(err) + } + } + indexInfo.BackfillState = model.BackfillStateReadyToMerge + ver, err = updateVersionAndTableInfo(d, t, job, tbl.Meta(), true) + return false, ver, errors.Trace(err) + case model.BackfillStateReadyToMerge: + logutil.BgLogger().Info("[ddl] index backfill state ready to merge", zap.Int64("job ID", job.ID), + zap.String("table", tbl.Meta().Name.O), zap.String("index", indexInfo.Name.O)) + indexInfo.BackfillState = model.BackfillStateMerging + if bfProcess == model.ReorgTypeLitMerge { + ingest.LitBackCtxMgr.Unregister(job.ID) + } + job.SnapshotVer = 0 // Reset the snapshot version for merge index reorg. + ver, err = updateVersionAndTableInfo(d, t, job, tbl.Meta(), true) + return false, ver, errors.Trace(err) + case model.BackfillStateMerging: + done, ver, err = runReorgJobAndHandleErr(w, d, t, job, tbl, indexInfo, true) + if !done { + return false, ver, err + } + indexInfo.BackfillState = model.BackfillStateInapplicable // Prevent double-write on this index. + return true, ver, nil + default: + return false, 0, dbterror.ErrInvalidDDLState.GenWithStackByArgs("backfill", indexInfo.BackfillState) + } +} + func runReorgJobAndHandleErr(w *worker, d *ddlCtx, t *meta.Meta, job *model.Job, tbl table.Table, indexInfo *model.IndexInfo, mergingTmpIdx bool) (done bool, ver int64, err error) { elements := []*meta.Element{{ID: indexInfo.ID, TypeKey: meta.IndexElementKey}} @@ -1219,7 +1278,7 @@ type addIndexWorker struct { distinctCheckFlags []bool } -func newAddIndexWorker(decodeColMap map[int64]decoder.Column, id int, t table.PhysicalTable, bfCtx *backfillCtx, jc *JobContext, jobID, eleID int64, eleTypeKey []byte) (*addIndexWorker, error) { +func newAddIndexWorker(decodeColMap map[int64]decoder.Column, t table.PhysicalTable, bfCtx *backfillCtx, jc *JobContext, jobID, eleID int64, eleTypeKey []byte) (*addIndexWorker, error) { if !bytes.Equal(eleTypeKey, meta.IndexElementKey) { logutil.BgLogger().Error("Element type for addIndexWorker incorrect", zap.String("jobQuery", jc.cacheSQL), zap.Int64("job ID", jobID), zap.ByteString("element type", eleTypeKey), zap.Int64("element ID", eleID)) @@ -1239,7 +1298,7 @@ func newAddIndexWorker(decodeColMap map[int64]decoder.Column, id int, t table.Ph if err != nil { return nil, errors.Trace(err) } - lwCtx, err = ei.NewWriterCtx(id) + lwCtx, err = ei.NewWriterCtx(bfCtx.id) if err != nil { return nil, err } @@ -1323,6 +1382,21 @@ func (w *baseIndexWorker) GetCtx() *backfillCtx { return w.backfillCtx } +func newBaseIndexWorkerContext(d *ddl, sess *session, schemaName model.CIStr, tbl table.Table, workerCnt int, bfJob *BackfillJob, jobCtx *JobContext) (*backfillWorkerContext, error) { + //nolint:forcetypeassert + phyTbl := tbl.(table.PhysicalTable) + return newBackfillWorkerContext(d, schemaName.O, tbl, workerCnt, bfJob.Meta.ReorgTp, + func(bfCtx *backfillCtx) (backfiller, error) { + decodeColMap, err := makeupDecodeColMap(sess, schemaName, phyTbl) + if err != nil { + logutil.BgLogger().Debug("[ddl] make up decode col map failed", zap.Error(err)) + return nil, errors.Trace(err) + } + bf, err1 := newAddIndexWorker(decodeColMap, phyTbl, bfCtx, jobCtx, bfJob.JobID, bfJob.EleID, bfJob.EleKey) + return bf, err1 + }) +} + // mockNotOwnerErrOnce uses to make sure `notOwnerErr` only mock error once. var mockNotOwnerErrOnce uint32 @@ -1381,7 +1455,7 @@ func (w *baseIndexWorker) getNextKey(taskRange reorgBackfillTask, taskDone bool) if !taskDone { // The task is not done. So we need to pick the last processed entry's handle and add one. lastHandle := w.idxRecords[len(w.idxRecords)-1].handle - recordKey := tablecodec.EncodeRecordKey(w.table.RecordPrefix(), lastHandle) + recordKey := tablecodec.EncodeRecordKey(taskRange.physicalTable.RecordPrefix(), lastHandle) return recordKey.Next() } if taskRange.endInclude { @@ -1457,7 +1531,7 @@ func (w *baseIndexWorker) fetchRowColVals(txn kv.Transaction, taskRange reorgBac taskDone = true } - logutil.BgLogger().Debug("[ddl] txn fetches handle info", zap.Uint64("txnStartTS", txn.StartTS()), + logutil.BgLogger().Debug("[ddl] txn fetches handle info", zap.Stringer("worker", w), zap.Uint64("txnStartTS", txn.StartTS()), zap.String("taskRange", taskRange.String()), zap.Duration("takeTime", time.Since(startTime))) return w.idxRecords, w.getNextKey(taskRange, taskDone), taskDone, errors.Trace(err) } @@ -1578,6 +1652,7 @@ func (w *addIndexWorker) BackfillDataInTxn(handleRange reorgBackfillTask) (taskC jobID := handleRange.getJobID() ctx := kv.WithInternalSourceType(context.Background(), w.jobContext.ddlJobSourceType()) errInTxn = kv.RunInNewTxn(ctx, w.sessCtx.GetStore(), true, func(ctx context.Context, txn kv.Transaction) (err error) { + taskCtx.finishTS = txn.StartTS() taskCtx.addedCount = 0 taskCtx.scanCount = 0 txn.SetOption(kv.Priority, handleRange.priority) @@ -1761,8 +1836,7 @@ func (w *worker) updateReorgInfo(t table.PartitionedTable, reorg *reorgInfo) (bo err = reorg.UpdateReorgMeta(reorg.StartKey, w.sessPool) logutil.BgLogger().Info("[ddl] job update reorgInfo", zap.Int64("jobID", reorg.Job.ID), - zap.ByteString("elementType", reorg.currElement.TypeKey), - zap.Int64("elementID", reorg.currElement.ID), + zap.Stringer("element", reorg.currElement), zap.Int64("partitionTableID", pid), zap.String("startKey", hex.EncodeToString(reorg.StartKey)), zap.String("endKey", hex.EncodeToString(reorg.EndKey)), zap.Error(err)) @@ -1820,7 +1894,7 @@ type cleanUpIndexWorker struct { baseIndexWorker } -func newCleanUpIndexWorker(sessCtx sessionctx.Context, t table.PhysicalTable, decodeColMap map[int64]decoder.Column, reorgInfo *reorgInfo, jc *JobContext) *cleanUpIndexWorker { +func newCleanUpIndexWorker(sessCtx sessionctx.Context, id int, t table.PhysicalTable, decodeColMap map[int64]decoder.Column, reorgInfo *reorgInfo, jc *JobContext) *cleanUpIndexWorker { indexes := make([]table.Index, 0, len(t.Indices())) rowDecoder := decoder.NewRowDecoder(t, t.WritableCols(), decodeColMap) for _, index := range t.Indices() { @@ -1830,7 +1904,7 @@ func newCleanUpIndexWorker(sessCtx sessionctx.Context, t table.PhysicalTable, de } return &cleanUpIndexWorker{ baseIndexWorker: baseIndexWorker{ - backfillCtx: newBackfillCtx(reorgInfo.d, sessCtx, reorgInfo.ReorgMeta.ReorgTp, reorgInfo.SchemaName, t), + backfillCtx: newBackfillCtx(reorgInfo.d, id, sessCtx, reorgInfo.ReorgMeta.ReorgTp, reorgInfo.SchemaName, t), indexes: indexes, rowDecoder: rowDecoder, defaultVals: make([]types.Datum, len(t.WritableCols())), @@ -1953,6 +2027,49 @@ func (w *worker) updateReorgInfoForPartitions(t table.PartitionedTable, reorg *r return false, errors.Trace(err) } +func runBackfillJobsWithLightning(d *ddl, sess *session, bfJob *BackfillJob, jobCtx *JobContext) { + // TODO: Consider redo it. + bc, ok := ingest.LitBackCtxMgr.Load(bfJob.JobID) + if ok && bc.Done() { + logutil.BgLogger().Warn("[ddl] lightning loaded") + return + } + var err error + logutil.BgLogger().Warn(fmt.Sprintf("------------------------------------ bf:%#v", bfJob)) + bc, err = ingest.LitBackCtxMgr.Register(d.ctx, bfJob.Meta.IsUnique, bfJob.JobID, bfJob.Meta.SQLMode) + if err != nil { + // TODO: tryFallbackToTxnMerge + logutil.BgLogger().Warn("[ddl] lightning register error", zap.Error(err)) + return + } + + logutil.BgLogger().Warn("00 ******** load backfill job and run reorg jobs start", zap.Int64("job id", bfJob.JobID)) + tbl, err := runBackfillJobs(d, sess, bfJob, jobCtx) + if err != nil { + logutil.BgLogger().Warn("[ddl] runBackfillJobs error", zap.Error(err)) + ingest.LitBackCtxMgr.Unregister(bfJob.JobID) + // TODO: tryFallbackToTxnMerge + return + } + + finish := injectSpan(bfJob.JobID, "finish-import") + err = bc.FinishImport(bfJob.EleID, bfJob.Meta.IsUnique, tbl) + if err != nil { + if kv.ErrKeyExists.Equal(err) { + logutil.BgLogger().Warn("[ddl] import index duplicate key, convert job to rollback", zap.String("job", bfJob.AbbrStr()), zap.Error(err)) + // TODO: convertAddIdxJob2RollbackJob + } else { + logutil.BgLogger().Warn("[ddl] lightning import error", zap.Error(err)) + // TODO: tryFallbackToTxnMerge + } + ingest.LitBackCtxMgr.Unregister(bfJob.JobID) + return + } + ingest.LitBackCtxMgr.Unregister(bfJob.ID) + finish() + bc.SetDone() +} + // changingIndex is used to store the index that need to be changed during modifying column. type changingIndex struct { IndexInfo *model.IndexInfo diff --git a/ddl/job_table.go b/ddl/job_table.go index 740bb5c0b7da1..6b2716925ae5e 100644 --- a/ddl/job_table.go +++ b/ddl/job_table.go @@ -26,12 +26,16 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/failpoint" "github.com/pingcap/kvproto/pkg/kvrpcpb" + "github.com/pingcap/tidb/ddl/ingest" + ddlutil "github.com/pingcap/tidb/ddl/util" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/meta" "github.com/pingcap/tidb/metrics" "github.com/pingcap/tidb/parser/model" "github.com/pingcap/tidb/sessionctx/variable" + "github.com/pingcap/tidb/table" "github.com/pingcap/tidb/util/dbterror" + "github.com/pingcap/tidb/util/gpool/spmc" "github.com/pingcap/tidb/util/logutil" clientv3 "go.etcd.io/etcd/client/v3" "go.uber.org/zap" @@ -40,6 +44,7 @@ import ( var ( addingDDLJobConcurrent = "/tidb/ddl/add_ddl_job_general" + addingBackfillJob = "/tidb/ddl/add_backfill_job" ) func (dc *ddlCtx) insertRunningDDLJobMap(id int64) { @@ -309,6 +314,182 @@ func (d *ddl) markJobProcessing(sess *session, job *model.Job) error { return errors.Trace(err) } +func (d *ddl) startDispatchBackfillJobsLoop() { + isDistReorg := variable.DDLEnableDistributeReorg.Load() + if !isDistReorg { + return + } + d.backfillCtx.jobCtxMap = make(map[int64]*JobContext) + d.backfillCtx.backfillCtxMap = make(map[int64]struct{}) + + logutil.BgLogger().Warn("------------------------------- start backfill jobs loop") + + var notifyBackfillJobByEtcdCh clientv3.WatchChan + if d.etcdCli != nil { + notifyBackfillJobByEtcdCh = d.etcdCli.Watch(d.ctx, addingBackfillJob) + } + ticker := time.NewTicker(300 * time.Millisecond) + defer ticker.Stop() + for { + if isChanClosed(d.ctx.Done()) { + return + } + select { + case <-d.backfillJobCh: + case <-ticker.C: + case _, ok := <-notifyBackfillJobByEtcdCh: + if !ok { + logutil.BgLogger().Warn("[ddl] start backfill worker watch channel closed", zap.String("watch key", addingBackfillJob)) + notifyBackfillJobByEtcdCh = d.etcdCli.Watch(d.ctx, addingBackfillJob) + time.Sleep(time.Second) + continue + } + case <-d.ctx.Done(): + return + } + d.loadBackfillJobAndRun() + } +} + +func (d *ddl) getTableByTxn(store kv.Storage, schemaID, tableID int64) (*model.DBInfo, table.Table, error) { + var tbl table.Table + var dbInfo *model.DBInfo + err := kv.RunInNewTxn(d.ctx, store, false, func(ctx context.Context, txn kv.Transaction) error { + t := meta.NewMeta(txn) + var err1 error + dbInfo, err1 = t.GetDatabase(schemaID) + if err1 != nil { + return errors.Trace(err1) + } + tblInfo, err1 := getTableInfo(t, tableID, schemaID) + if err1 != nil { + return errors.Trace(err1) + } + tbl, err1 = getTable(store, schemaID, tblInfo) + return errors.Trace(err1) + }) + return dbInfo, tbl, err +} + +func (d *ddl) loadBackfillJobAndRun() { + se, err := d.sessPool.get() + defer d.sessPool.put(se) + if err != nil { + logutil.BgLogger().Fatal("dispatch backfill jobs loop get session failed, it should not happen, please try restart TiDB", zap.Error(err)) + } + sess := newSession(se) + + if err := ddlutil.LoadDDLReorgVars(context.Background(), sess); err != nil { + logutil.BgLogger().Error("[ddl] load DDL reorganization variable failed", zap.Error(err)) + } + + d.backfillCtx.Lock() + jobCtxMapLen := len(d.backfillCtx.jobCtxMap) + runningJobIDs := make([]int64, 0, jobCtxMapLen) + if jobCtxMapLen >= reorgWorkerCnt { + logutil.BgLogger().Warn("00 ******** load backfill job and run reorg jos is more than limit", zap.Int("limit", reorgWorkerCnt)) + d.backfillCtx.Unlock() + return + } else { + for id := range d.backfillCtx.jobCtxMap { + runningJobIDs = append(runningJobIDs, id) + } + } + d.backfillCtx.Unlock() + + // TODO: Add ele info to distinguish backfill jobs. + bJobs, err := GetBackfillJobsForOneEle(sess, 1, runningJobIDs, InstanceLease) + bJobCnt := len(bJobs) + if bJobCnt == 0 || err != nil { + if err != nil { + logutil.BgLogger().Warn("[ddl] get backfill jobs met error", zap.Error(err)) + } + return + } + + bJob := bJobs[0] + d.backfillCtx.Lock() + jobCtx, ok := d.backfillCtx.jobCtxMap[bJob.JobID] + if !ok { + d.setDDLLabelForTopSQL(bJob.JobID, bJob.Meta.Query) + d.setDDLSourceForDiagnosis(bJob.JobID, bJob.Meta.Type) + jobCtx = d.jobContext(bJob.JobID) + d.backfillCtx.jobCtxMap[bJob.JobID] = jobCtx + d.backfillCtx.Unlock() + } else { + logutil.BgLogger().Warn("00 ******** load backfill job and run reorg jobs exit", zap.Int64("job id", bJob.JobID)) + d.backfillCtx.Unlock() + return + } + + d.wg.Run(func() { + defer func() { + d.backfillCtx.Lock() + delete(d.backfillCtx.jobCtxMap, bJob.JobID) + d.backfillCtx.Unlock() + logutil.BgLogger().Warn("00 ******** load backfill job and run reorg jobs finished", zap.Int64("job id", bJob.JobID)) + }() + traceID := bJob.ID + 100 + initializeTrace(traceID) + + if bJob.Meta.ReorgTp == model.ReorgTypeLitMerge { + if !ingest.LitInitialized { + logutil.BgLogger().Warn("[ddl] we can't do LitMerg", + zap.Bool("LitInitialized", ingest.LitInitialized), zap.String("bJob", bJob.AbbrStr())) + return + } + logutil.BgLogger().Info("[ddl] &&&&&&&&&&&&&&&&&&&&&&&&&&&&&&&&&&&&&&&&&&&&&&&& litMerge") + logutil.BgLogger().Info("run backfill job with LitMerge", zap.String("bJob", bJob.AbbrStr())) + runBackfillJobsWithLightning(d, sess, bJob, jobCtx) + } else { + logutil.BgLogger().Info("[ddl] &&&&&&&&&&&&&&&&&&&&&&&&&&&&&&&&&&&&&&&&&&&&&&&& txnMerge") + logutil.BgLogger().Info("run backfill job with TxnMerge", zap.String("bJob", bJob.AbbrStr())) + runBackfillJobs(d, sess, bJob, jobCtx) + } + + err = syncBackfillHistoryJobs(sess, d.uuid, bJob) + if err != nil { + logutil.BgLogger().Warn("[ddl] syncBackfillHistoryJobs error", zap.Error(err)) + } + details := collectTrace(bJob.JobID) + logutil.BgLogger().Info("[ddl] &&&&&&&&&&&&&&&&&&&&&&&&&&&&&&&&-------------------------- finish backfill jobs", + zap.Int64("job ID", bJob.JobID), zap.String("time details", details)) + }) +} + +func runBackfillJobs(d *ddl, sess *session, bJob *BackfillJob, jobCtx *JobContext) (table.Table, error) { + logutil.BgLogger().Warn("00 ******** run backfill jobs", zap.Int64("job id", bJob.JobID)) + dbInfo, tbl, err := d.getTableByTxn(d.store, bJob.Meta.SchemaID, bJob.Meta.TableID) + if err != nil { + logutil.BgLogger().Warn("[ddl] runBackfillJobs gets table failed", zap.String("bfJob", bJob.AbbrStr()), zap.Error(err)) + return nil, err + } + + workerCnt := int(variable.GetDDLReorgWorkerCounter()) + // TODO: Different worker using different newBackfillerFunc. + bwCtx, err := newBaseIndexWorkerContext(d, sess, dbInfo.Name, tbl, workerCnt, bJob, jobCtx) + if err != nil { + logutil.BgLogger().Info("[ddl] new add index worker context failed", zap.Error(err)) + return nil, errors.Trace(err) + } + bwMgr := newBackfilWorkerManager(bwCtx) + d.backfillWorkerPool.SetConsumerFunc(func(task *reorgBackfillTask, _ int, bfWorker *backfillWorker) *backfillResult { + return handleTask(task, 0, bfWorker) + }) + proFunc := func() ([]*reorgBackfillTask, error) { + return GetTasks(d.ddlCtx, sess, tbl, bJob.JobID, workerCnt*2) + } + // add new task + resultCh, control := d.backfillWorkerPool.AddProduceBySlice(proFunc, 0, bwCtx, spmc.WithConcurrency(workerCnt)) + bwMgr.waitFinalResult(resultCh) + + // waiting task finishing + control.Wait() + bwMgr.close(d) + + return tbl, nil +} + const ( addDDLJobSQL = "insert into mysql.tidb_ddl_job(job_id, reorg, schema_ids, table_ids, job_meta, type, processing) values" updateDDLJobSQL = "update mysql.tidb_ddl_job set job_meta = %s where job_id = %d" @@ -396,7 +577,7 @@ func updateDDLJob2Table(sctx *session, job *model.Job, updateRawArgs bool) error // getDDLReorgHandle gets DDL reorg handle. func getDDLReorgHandle(sess *session, job *model.Job) (element *meta.Element, startKey, endKey kv.Key, physicalTableID int64, err error) { sql := fmt.Sprintf("select ele_id, ele_type, start_key, end_key, physical_id from mysql.tidb_ddl_reorg where job_id = %d", job.ID) - ctx := kv.WithInternalSourceType(context.Background(), getDDLRequestSource(job)) + ctx := kv.WithInternalSourceType(context.Background(), getDDLRequestSource(job.Type)) rows, err := sess.execute(ctx, sql, "get_handle") if err != nil { return nil, nil, nil, 0, err @@ -496,6 +677,14 @@ func getJobsBySQL(sess *session, tbl, condition string) ([]*model.Job, error) { return jobs, nil } +func syncBackfillHistoryJobs(sess *session, uuid string, backfillJob *BackfillJob) error { + sql := fmt.Sprintf("update mysql.%s set state = %d where ddl_job_id = %d and ele_id = %d and ele_key = '%s' and exec_id = '%s' limit 1;", + BackfillHistoryTable, model.JobStateSynced, backfillJob.JobID, backfillJob.EleID, backfillJob.EleKey, uuid) + logutil.BgLogger().Warn("update ***************************** " + fmt.Sprintf("sql:%v, sess:%#v", sql, sess)) + _, err := sess.execute(context.Background(), sql, "sync_backfill_history_job") + return err +} + func generateInsertBackfillJobSQL(tableName string, backfillJobs []*BackfillJob) (string, error) { sqlPrefix := fmt.Sprintf("insert into mysql.%s(id, ddl_job_id, ele_id, ele_key, store_id, type, exec_id, exec_lease, state, curr_key, start_key, end_key, start_ts, finish_ts, row_count, backfill_meta) values", tableName) var sql string @@ -548,6 +737,7 @@ func AddBackfillJobs(sess *session, backfillJobs []*BackfillJob) error { return err } _, err = sess.execute(context.Background(), sql, label) + logutil.BgLogger().Warn("insert ***************************** " + fmt.Sprintf("sql:%v, sess:%#v", sql, sess)) return errors.Trace(err) }) } @@ -557,6 +747,18 @@ func runInTxn(se *session, f func(*session) error) (err error) { if err != nil { return err } + failpoint.Inject("NotifyBeginTxnCh", func(val failpoint.Value) { + //nolint:forcetypeassert + v := val.(int) + if v == 1 { + mockDDLErrOnce = 1 + TestNotifyBeginTxnCh <- struct{}{} + } else if v == 2 && mockDDLErrOnce == 1 { + <-TestNotifyBeginTxnCh + mockDDLErrOnce = 0 + } + }) + err = f(se) if err != nil { se.rollback() @@ -691,6 +893,7 @@ func getUnsyncedInstanceIDs(sess *session, jobID int64, label string) ([]string, InstanceID := row.GetString(1) InstanceIDs = append(InstanceIDs, InstanceID) } + logutil.BgLogger().Info(fmt.Sprintf("get unsynced exec ID ***************************** 00 lable:%s, sql:%s, insttanceIDs:%s", sql, label, InstanceIDs)) return InstanceIDs, nil } @@ -726,6 +929,7 @@ func GetBackfillJobs(sess *session, tblName, condition string, label string) ([] } bJobs = append(bJobs, &bfJob) } + logutil.BgLogger().Warn("get ***************************** " + fmt.Sprintf("sql:%v, label:%#v, bJobs:%v", condition, label, bJobs)) return bJobs, nil } @@ -734,6 +938,7 @@ func GetBackfillJobs(sess *session, tblName, condition string, label string) ([] func RemoveBackfillJob(sess *session, isOneEle bool, backfillJob *BackfillJob) error { sql := fmt.Sprintf("delete from mysql.tidb_ddl_backfill where ddl_job_id = %d and ele_id = %d and ele_key = '%s'", backfillJob.JobID, backfillJob.EleID, backfillJob.EleKey) + logutil.BgLogger().Info(fmt.Sprintf("remove ***************************** xx sql:%#v", backfillJob)) if !isOneEle { sql += fmt.Sprintf(" and id = %d", backfillJob.ID) } @@ -746,8 +951,9 @@ func updateBackfillJob(sess *session, tableName string, backfillJob *BackfillJob if err != nil { return err } - sql := fmt.Sprintf("update mysql.%s set exec_id = '%s', exec_lease = '%s', state = %d, backfill_meta = '%s' where ddl_job_id = %d and ele_id = %d and ele_key = '%s' and id = %d", - tableName, backfillJob.InstanceID, backfillJob.InstanceLease, backfillJob.State, mate, backfillJob.JobID, backfillJob.EleID, backfillJob.EleKey, backfillJob.ID) + sql := fmt.Sprintf("update mysql.%s set exec_id = '%s', exec_lease = '%s', state = %d, curr_key = '%s', row_count = %d, backfill_meta = '%s' where ddl_job_id = %d and ele_id = %d and ele_key = '%s' and id = %d", + tableName, backfillJob.InstanceID, backfillJob.InstanceLease, backfillJob.State, backfillJob.CurrKey, backfillJob.RowCount, mate, backfillJob.JobID, backfillJob.EleID, backfillJob.EleKey, backfillJob.ID) _, err = sess.execute(context.Background(), sql, label) + logutil.BgLogger().Warn("update ***************************** " + fmt.Sprintf("sql:%v, label:%#v, err:%v", sql, label, err)) return err } diff --git a/ddl/job_table_test.go b/ddl/job_table_test.go index d869dcecc2c0e..8351b9998f3c6 100644 --- a/ddl/job_table_test.go +++ b/ddl/job_table_test.go @@ -17,6 +17,8 @@ package ddl_test import ( "context" "fmt" + "strconv" + "strings" "sync" "testing" "time" @@ -26,6 +28,7 @@ import ( "github.com/pingcap/tidb/meta" "github.com/pingcap/tidb/parser/model" "github.com/pingcap/tidb/sessionctx" + "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/sessiontxn" "github.com/pingcap/tidb/testkit" "github.com/pingcap/tidb/types" @@ -266,9 +269,9 @@ func TestSimpleExecBackfillJobs(t *testing.T) { // Test some backfill jobs, add backfill jobs to the table. cnt := 2 bjTestCases := make([]*ddl.BackfillJob, 0, cnt*3) - bJobs1 := makeAddIdxBackfillJobs(1, 2, jobID1, eleID1, cnt, "alter table add index idx(a)") - bJobs2 := makeAddIdxBackfillJobs(1, 2, jobID2, eleID2, cnt, "alter table add index idx(b)") - bJobs3 := makeAddIdxBackfillJobs(1, 2, jobID2, eleID3, cnt, "alter table add index idx(c)") + bJobs1 := makeAddIdxBackfillJobs(1, 2, jobID1, eleID1, cnt, "alter table t add index idx(a)") + bJobs2 := makeAddIdxBackfillJobs(1, 2, jobID2, eleID2, cnt, "alter table t add index idx(b)") + bJobs3 := makeAddIdxBackfillJobs(1, 2, jobID2, eleID3, cnt, "alter table t add index idx(c)") bjTestCases = append(bjTestCases, bJobs1...) bjTestCases = append(bjTestCases, bJobs2...) bjTestCases = append(bjTestCases, bJobs3...) @@ -540,3 +543,93 @@ func TestSimpleExecBackfillJobs(t *testing.T) { // 6 jobID1 eleID1 JobStateNone // 7 jobID1 eleID1 JobStateNone } + +func TestGetTasks(t *testing.T) { + // TODO: update the variable of `enableDistReorg` + isDistReorg := variable.DDLEnableDistributeReorg.Load() + variable.DDLEnableDistributeReorg.Store(false) + defer func() { variable.DDLEnableDistributeReorg.Store(isDistReorg) }() + + store, dom := testkit.CreateMockStoreAndDomain(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + se := ddl.NewSession(tk.Session()) + d := dom.DDL() + + jobID1 := int64(1) + eleID1 := int64(11) + uuid := d.GetID() + cnt := 3 + instanceLease := ddl.InstanceLease + bJobsTestCases := makeAddIdxBackfillJobs(1, 2, jobID1, eleID1, cnt, "alter table t add index idx(a)") + err := ddl.AddBackfillJobs(se, bJobsTestCases) + require.NoError(t, err) + + var wg util.WaitGroupWrapper + // Mock GetAndMarkBackfillJobsForOneEle gets a writing conflict error. + // Step 1: se1 begins txn1. + // Step 2: se2 begins txn2. + // Step 3: execute txn1 and txn2, then txn1 or txn2 returns a writing conflict error. + var err1 error + ch := make(chan struct{}, 1) + wg.Run(func() { + require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/ddl/NotifyBeginTxnCh", `return(1)`)) + ch <- struct{}{} + var bJobs []*ddl.BackfillJob + bJobs, err = ddl.GetAndMarkBackfillJobsForOneEle(se, 1, jobID1, uuid, instanceLease) + require.Len(t, bJobs, 1) + }) + <-ch + defer func() { require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/ddl/NotifyBeginTxnCh")) }() + wg.Run(func() { + tk1 := testkit.NewTestKit(t, store) + tk1.MustExec("use test") + se1 := ddl.NewSession(tk1.Session()) + require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/ddl/NotifyBeginTxnCh", `return(2)`)) + var bJobs1 []*ddl.BackfillJob + bJobs1, err1 = ddl.GetAndMarkBackfillJobsForOneEle(se1, 1, jobID1, uuid, instanceLease) + require.Len(t, bJobs1, 1) + }) + wg.Wait() + if err == nil { + require.NotNil(t, err1) + require.True(t, strings.Contains(err1.Error(), "[kv:9007]Write conflict")) + } else { + require.Nil(t, err1) + require.True(t, strings.Contains(err.Error(), "[kv:9007]Write conflict")) + } + + // get tbl + tk.MustExec("create table t(a int, b int)") + var tableID int64 + rs := tk.MustQuery("select TIDB_TABLE_ID from information_schema.tables where table_name='t' and table_schema='test';") + tableIDi, err := strconv.Atoi(rs.Rows()[0][0].(string)) + require.Nil(t, err) + tableID = int64(tableIDi) + tbl := testGetTable(t, dom, tableID) + require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/ddl/NotifyBeginTxnCh", `return(0)`)) + // Mock GetAndMarkBackfillJobsForOneEle gets a writing conflict error, but getTasks is successful. + // Step 1: se1 begins txn1. + // Step 2: se2 begins txn2. + // Step 3: execute txn1 and txn2, then txn1 or txn2 returns a writing conflict error. + // Step 4: se2 begin txn3. + // Step 5: getTasks(txn3) executes successfully. + wg.Run(func() { + require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/ddl/NotifyBeginTxnCh", `return(1)`)) + ch <- struct{}{} + bJobs, err := ddl.GetTasks(ddl.GetDDLCtx(d), se, tbl, jobID1, 1) + require.Nil(t, err) + require.Len(t, bJobs, 1) + }) + <-ch + wg.Run(func() { + tk1 := testkit.NewTestKit(t, store) + tk1.MustExec("use test") + se1 := ddl.NewSession(tk1.Session()) + require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/ddl/NotifyBeginTxnCh", `return(2)`)) + bJobs1, err1 := ddl.GetTasks(ddl.GetDDLCtx(d), se1, tbl, jobID1, 1) + require.Nil(t, err1) + require.Len(t, bJobs1, 1) + }) + wg.Wait() +} diff --git a/ddl/reorg.go b/ddl/reorg.go index 7912560499344..f6ca9ec9be83e 100644 --- a/ddl/reorg.go +++ b/ddl/reorg.go @@ -277,8 +277,7 @@ func (w *worker) runReorgJob(rh *reorgHandler, reorgInfo *reorgInfo, tblInfo *mo err := updateDDLReorgStartHandle(rh.s, job, currentElement, doneKey) logutil.BgLogger().Info("[ddl] run reorg job wait timeout", zap.Duration("wait time", waitTimeout), - zap.ByteString("element type", currentElement.TypeKey), - zap.Int64("element ID", currentElement.ID), + zap.Stringer("element", currentElement), zap.Int64("total added row count", rowCount), zap.String("done key", hex.EncodeToString(doneKey)), zap.Error(err)) diff --git a/go.mod b/go.mod index d87619f7dc67c..21f091b05d6d2 100644 --- a/go.mod +++ b/go.mod @@ -63,7 +63,7 @@ require ( github.com/mgechev/revive v1.2.4 github.com/ngaut/pools v0.0.0-20180318154953-b7bc8c42aac7 github.com/nishanths/predeclared v0.2.2 - github.com/opentracing/basictracer-go v1.0.0 + github.com/opentracing/basictracer-go v1.1.0 github.com/opentracing/opentracing-go v1.2.0 github.com/phayes/freeport v0.0.0-20180830031419-95f893ade6f2 github.com/pingcap/badger v1.5.1-0.20221229114011-ddffaa0fff7a @@ -90,6 +90,7 @@ require ( github.com/tdakkota/asciicheck v0.1.1 github.com/tiancaiamao/appdash v0.0.0-20181126055449-889f96f722a2 github.com/tikv/client-go/v2 v2.0.4-0.20221226080148-018c59dbd837 + github.com/tikv/minitrace-go v0.0.0-20220923091513-8e6316bb4097 github.com/tikv/pd/client v0.0.0-20221031025758-80f0d8ca4d07 github.com/timakin/bodyclose v0.0.0-20210704033933-f49887972144 github.com/twmb/murmur3 v1.1.3 diff --git a/go.sum b/go.sum index efeb0a0fb5537..6cfac2760729d 100644 --- a/go.sum +++ b/go.sum @@ -739,8 +739,9 @@ github.com/onsi/gomega v1.7.1/go.mod h1:XdKZgCCFLUoM/7CFJVPcG8C1xQ1AJ0vpAezJrB7J github.com/onsi/gomega v1.10.1/go.mod h1:iN09h71vgCQne3DLsj+A5owkum+a2tYe+TOCB1ybHNo= github.com/onsi/gomega v1.18.1 h1:M1GfJqGRrBrrGGsbxzV5dqM2U2ApXefZCQpkukxYRLE= github.com/opentracing-contrib/go-stdlib v0.0.0-20170113013457-1de4cc2120e7/go.mod h1:PLldrQSroqzH70Xl+1DQcGnefIbqsKR7UDaiux3zV+w= -github.com/opentracing/basictracer-go v1.0.0 h1:YyUAhaEfjoWXclZVJ9sGoNct7j4TVk7lZWlQw5UXuoo= github.com/opentracing/basictracer-go v1.0.0/go.mod h1:QfBfYuafItcjQuMwinw9GhYKwFXS9KnPs5lxoYwgW74= +github.com/opentracing/basictracer-go v1.1.0 h1:Oa1fTSBvAl8pa3U+IJYqrKm0NALwH9OsgwOqDv4xJW0= +github.com/opentracing/basictracer-go v1.1.0/go.mod h1:V2HZueSJEp879yv285Aap1BS69fQMD+MNP1mRs6mBQc= github.com/opentracing/opentracing-go v1.0.2/go.mod h1:UkNAQd3GIcIGf0SeVgPpRdFStlNbqXla1AfSYxPUl2o= github.com/opentracing/opentracing-go v1.1.0/go.mod h1:UkNAQd3GIcIGf0SeVgPpRdFStlNbqXla1AfSYxPUl2o= github.com/opentracing/opentracing-go v1.2.0 h1:uEJPy/1a5RIPAJ0Ov+OIO8OxWu77jEv+1B0VhjKrZUs= @@ -758,6 +759,7 @@ github.com/peterbourgon/g2s v0.0.0-20170223122336-d4e7ad98afea/go.mod h1:1VcHEd3 github.com/petermattis/goid v0.0.0-20170504144140-0ded85884ba5/go.mod h1:jvVRKCrJTQWu0XVbaOlby/2lO20uSCHEMzzplHXte1o= github.com/phayes/freeport v0.0.0-20180830031419-95f893ade6f2 h1:JhzVVoYvbOACxoUmOs6V/G4D5nPVUW73rKvXxP4XUJc= github.com/phayes/freeport v0.0.0-20180830031419-95f893ade6f2/go.mod h1:iIss55rKnNBTvrwdmkUpLnDpZoAHvWaiq5+iMmen4AE= +github.com/philhofer/fwd v1.1.1/go.mod h1:gk3iGcWd9+svBvR0sR+KPcfE+RNWozjowpeBVG3ZVNU= github.com/pierrec/lz4 v2.6.0+incompatible/go.mod h1:pdkljMzZIN41W+lC3N2tnIh5sFi+IEE17M5jbnwPHcY= github.com/pierrec/lz4 v2.6.1+incompatible h1:9UY3+iC23yxF0UfGaYrGplQ+79Rg+h/q9FV9ix19jjM= github.com/pierrec/lz4 v2.6.1+incompatible/go.mod h1:pdkljMzZIN41W+lC3N2tnIh5sFi+IEE17M5jbnwPHcY= @@ -879,6 +881,7 @@ github.com/shurcooL/sanitized_anchor_name v1.0.0/go.mod h1:1NzhyTcUVG4SuEtjjoZeV github.com/shurcooL/vfsgen v0.0.0-20180711163814-62bca832be04/go.mod h1:TrYk7fJVaAttu97ZZKrO9UbRa8izdowaMIZcxYMbVaw= github.com/shurcooL/vfsgen v0.0.0-20181202132449-6a9ea43bcacd h1:ug7PpSOB5RBPK1Kg6qskGBoP3Vnj/aNYFTznWvlkGo0= github.com/shurcooL/vfsgen v0.0.0-20181202132449-6a9ea43bcacd/go.mod h1:TrYk7fJVaAttu97ZZKrO9UbRa8izdowaMIZcxYMbVaw= +github.com/silentred/gid v1.0.0/go.mod h1:DMQPn66uY+3ed7rWfzOVET7VbDBAhjz+6AmmlixUK08= github.com/sirupsen/logrus v1.2.0/go.mod h1:LxeOpSwHxABJmUn/MG1IvRgCAasNZTLOkJPxbbu5VWo= github.com/sirupsen/logrus v1.4.2/go.mod h1:tLMulIdttU9McNUspp0xgXVQah82FyeX6MwdIuYE2rE= github.com/sirupsen/logrus v1.6.0/go.mod h1:7uNnSEd1DgxDLC74fIahvMZmmYsHGZGEOFrfsX/uA88= @@ -935,10 +938,13 @@ github.com/tiancaiamao/gp v0.0.0-20221221095600-1a473d1f9b4b h1:4RNtqw1/tW67qP9f github.com/tiancaiamao/gp v0.0.0-20221221095600-1a473d1f9b4b/go.mod h1:h4xBhSNtOeEosLJ4P7JyKXX7Cabg7AVkWCK5gV2vOrM= github.com/tikv/client-go/v2 v2.0.4-0.20221226080148-018c59dbd837 h1:m6glgBGCIds9QURbk8Mn+8mjLKDcv6nWrNwYh92fydQ= github.com/tikv/client-go/v2 v2.0.4-0.20221226080148-018c59dbd837/go.mod h1:ptS8K+VBrEH2gIS3JxaiFSSLfDDyuS2xcdLozOtBWBw= +github.com/tikv/minitrace-go v0.0.0-20220923091513-8e6316bb4097 h1:nvIrUVo5YJZMsCn6yTxrpgrokIo/wug5N/nL5mc7v50= +github.com/tikv/minitrace-go v0.0.0-20220923091513-8e6316bb4097/go.mod h1:ukJr0BfYeYbO3n15LAV2Dp4jvFpIPF2g14NU227ZTLY= github.com/tikv/pd/client v0.0.0-20221031025758-80f0d8ca4d07 h1:ckPpxKcl75mO2N6a4cJXiZH43hvcHPpqc9dh1TmH1nc= github.com/tikv/pd/client v0.0.0-20221031025758-80f0d8ca4d07/go.mod h1:CipBxPfxPUME+BImx9MUYXCnAVLS3VJUr3mnSJwh40A= github.com/timakin/bodyclose v0.0.0-20210704033933-f49887972144 h1:kl4KhGNsJIbDHS9/4U9yQo1UcPQM0kOMJHn29EoH/Ro= github.com/timakin/bodyclose v0.0.0-20210704033933-f49887972144/go.mod h1:Qimiffbc6q9tBWlVV6x0P9sat/ao1xEkREYPPj9hphk= +github.com/tinylib/msgp v1.1.5/go.mod h1:eQsjooMTnV42mHu917E26IogZ2930nFyBQdofk10Udg= github.com/tklauser/go-sysconf v0.3.9/go.mod h1:11DU/5sG7UexIrp/O6g35hrWzu0JxlwQ3LSFUzyeuhs= github.com/tklauser/go-sysconf v0.3.10 h1:IJ1AZGZRWbY8T5Vfk04D9WOA5WSejdflXxP03OUqALw= github.com/tklauser/go-sysconf v0.3.10/go.mod h1:C8XykCvCb+Gn0oNCWPIlcb0RuglQTYaQ2hGm7jmxEFk= @@ -948,6 +954,7 @@ github.com/tklauser/numcpus v0.4.0/go.mod h1:1+UI3pD8NW14VMwdgJNJ1ESk2UnwhAnz5hM github.com/tmc/grpc-websocket-proxy v0.0.0-20190109142713-0ad062ec5ee5/go.mod h1:ncp9v5uamzpCO7NfCPTXjqaC+bZgJeR0sMTm6dMHP7U= github.com/tmc/grpc-websocket-proxy v0.0.0-20201229170055-e5319fda7802 h1:uruHq4dN7GR16kFc5fp3d1RIYzJW5onx8Ybykw2YQFA= github.com/tmc/grpc-websocket-proxy v0.0.0-20201229170055-e5319fda7802/go.mod h1:ncp9v5uamzpCO7NfCPTXjqaC+bZgJeR0sMTm6dMHP7U= +github.com/ttacon/chalk v0.0.0-20160626202418-22c06c80ed31/go.mod h1:onvgF043R+lC5RZ8IT9rBXDaEDnpnw/Cl+HFiw+v/7Q= github.com/twmb/murmur3 v1.1.3 h1:D83U0XYKcHRYwYIpBKf3Pks91Z0Byda/9SJ8B6EMRcA= github.com/twmb/murmur3 v1.1.3/go.mod h1:Qq/R7NUyOfr65zD+6Q5IHKsJLwP7exErjN6lyyq3OSQ= github.com/uber/jaeger-client-go v2.22.1+incompatible h1:NHcubEkVbahf9t3p75TOCR83gdUHXjRJvjoBh1yACsM= @@ -1165,6 +1172,7 @@ golang.org/x/net v0.0.0-20200222125558-5a598a2470a0/go.mod h1:z5CRVTTTmAJ677TzLL golang.org/x/net v0.0.0-20200226121028-0de0cce0169b/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= golang.org/x/net v0.0.0-20200301022130-244492dfa37a/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= golang.org/x/net v0.0.0-20200324143707-d3edc9973b7e/go.mod h1:qpuaurCH72eLCgpAm/N6yyVIVM9cpaDIP3A8BGJEC5A= +golang.org/x/net v0.0.0-20200421231249-e086a090c8fd/go.mod h1:qpuaurCH72eLCgpAm/N6yyVIVM9cpaDIP3A8BGJEC5A= golang.org/x/net v0.0.0-20200501053045-e0ff5e5a1de5/go.mod h1:qpuaurCH72eLCgpAm/N6yyVIVM9cpaDIP3A8BGJEC5A= golang.org/x/net v0.0.0-20200506145744-7e3656a0809f/go.mod h1:qpuaurCH72eLCgpAm/N6yyVIVM9cpaDIP3A8BGJEC5A= golang.org/x/net v0.0.0-20200513185701-a91f0712d120/go.mod h1:qpuaurCH72eLCgpAm/N6yyVIVM9cpaDIP3A8BGJEC5A= @@ -1401,6 +1409,7 @@ golang.org/x/tools v0.0.0-20200729194436-6467de6f59a7/go.mod h1:njjCfa9FT2d7l9Bc golang.org/x/tools v0.0.0-20200804011535-6c149bb5ef0d/go.mod h1:njjCfa9FT2d7l9Bc6FUM5FLjQPp3cFF28FI3qnDFljA= golang.org/x/tools v0.0.0-20200825202427-b303f430e36d/go.mod h1:njjCfa9FT2d7l9Bc6FUM5FLjQPp3cFF28FI3qnDFljA= golang.org/x/tools v0.0.0-20200904185747-39188db58858/go.mod h1:Cj7w3i3Rnn0Xh82ur9kSqwfTHTeVxaDqrfMjpcNT6bE= +golang.org/x/tools v0.0.0-20201022035929-9cf592e881e9/go.mod h1:emZCQorbCU4vsT4fOWvOPXz4eW1wZW4PmDk9uLelYpA= golang.org/x/tools v0.0.0-20201110124207-079ba7bd75cd/go.mod h1:emZCQorbCU4vsT4fOWvOPXz4eW1wZW4PmDk9uLelYpA= golang.org/x/tools v0.0.0-20201125231158-b5590deeca9b/go.mod h1:emZCQorbCU4vsT4fOWvOPXz4eW1wZW4PmDk9uLelYpA= golang.org/x/tools v0.0.0-20201201161351-ac6f37ff4c2a/go.mod h1:emZCQorbCU4vsT4fOWvOPXz4eW1wZW4PmDk9uLelYpA= diff --git a/parser/model/ddl.go b/parser/model/ddl.go index 46c1e65477d99..1417b5390d6ef 100644 --- a/parser/model/ddl.go +++ b/parser/model/ddl.go @@ -418,7 +418,9 @@ func (sub *SubJob) FromProxyJob(proxyJob *Job, ver int64) { type JobMeta struct { SchemaID int64 `json:"schema_id"` TableID int64 `json:"table_id"` - // Query string of the ddl job. + // Type is the DDL job's type. + Type ActionType `json:"job_type"` + // Query is the DDL job's SQL string. Query string `json:"query"` // Priority is only used to set the operation priority of adding indices. Priority int `json:"priority"` diff --git a/sessionctx/variable/featuretag/distributereorg/default.go b/sessionctx/variable/featuretag/distributereorg/default.go index 910629adde825..9f9c81967ba15 100644 --- a/sessionctx/variable/featuretag/distributereorg/default.go +++ b/sessionctx/variable/featuretag/distributereorg/default.go @@ -17,4 +17,4 @@ package distributereorg // TiDBEnableDistributeReorg is a feature tag -const TiDBEnableDistributeReorg bool = false +const TiDBEnableDistributeReorg bool = true diff --git a/util/gpool/spmc/spmcpool.go b/util/gpool/spmc/spmcpool.go index b69c7a05e0eca..c61ee4ff39407 100644 --- a/util/gpool/spmc/spmcpool.go +++ b/util/gpool/spmc/spmcpool.go @@ -55,7 +55,7 @@ type Pool[T any, U any, C any, CT any, TF pooltask.Context[CT]] struct { } // NewSPMCPool create a single producer, multiple consumer goroutine pool. -func NewSPMCPool[T any, U any, C any, CT any, TF pooltask.Context[CT]](name string, size int32, options ...Option) (*Pool[T, U, C, CT, TF], error) { +func NewSPMCPool[T any, U any, C any, CT any, TF pooltask.Context[CT]](name string, size int32, options ...Option) *Pool[T, U, C, CT, TF] { opts := loadOptions(options...) if expiry := opts.ExpiryDuration; expiry <= 0 { opts.ExpiryDuration = gpool.DefaultCleanIntervalTime @@ -79,7 +79,7 @@ func NewSPMCPool[T any, U any, C any, CT any, TF pooltask.Context[CT]](name stri result.cond = sync.NewCond(result.lock) // Start a goroutine to clean up expired workers periodically. go result.purgePeriodically() - return result, nil + return result } // purgePeriodically clears expired workers periodically which runs in an individual goroutine, as a scavenger. From 4de286165fa73aa9b6321658f892a3cb46386431 Mon Sep 17 00:00:00 2001 From: Lynn Date: Fri, 6 Jan 2023 17:10:02 +0800 Subject: [PATCH 02/25] ddl: pass tests and update --- ddl/backfilling.go | 318 ++++++++++------------------------- ddl/db_integration_test.go | 5 + ddl/ddl.go | 49 +++++- ddl/ddl_workerpool.go | 16 +- ddl/ddl_workerpool_test.go | 20 ++- ddl/dist_backfilling.go | 203 ++++++++++++++++++++++ ddl/failtest/fail_db_test.go | 4 + ddl/index.go | 37 ++-- ddl/index_merge_tmp_test.go | 11 +- ddl/job_table.go | 147 ++++++++-------- ddl/job_table_test.go | 19 ++- ddl/reorg.go | 2 +- metrics/metrics.go | 17 +- parser/model/ddl.go | 8 +- 14 files changed, 503 insertions(+), 353 deletions(-) create mode 100644 ddl/dist_backfilling.go diff --git a/ddl/backfilling.go b/ddl/backfilling.go index 64eb19e9ebb56..dedd3c02cacef 100644 --- a/ddl/backfilling.go +++ b/ddl/backfilling.go @@ -65,7 +65,7 @@ const ( updateInstanceLease = 25 * time.Second genTaskBatch = 4096 minGenTaskBatch = 1024 - minDistTaskCnt = 16 + minDistTaskCnt = 32 retrySQLTimes = 3 retrySQLInterval = 300 * time.Millisecond ) @@ -201,56 +201,6 @@ type backfillTaskContext struct { finishTS uint64 } -type backfillWorkerContext struct { - currID int - mu sync.Mutex - sessCtxs []sessionctx.Context - backfillWorkers []*backfillWorker -} - -type newBackfillerFunc func(bfCtx *backfillCtx) (bf backfiller, err error) - -func newBackfillWorkerContext(d *ddl, schemaName string, tbl table.Table, workerCnt int, reorgTp model.ReorgType, bfFunc newBackfillerFunc) (*backfillWorkerContext, error) { - if workerCnt <= 0 { - return nil, nil - } - - bws, err := d.backfillCtxPool.batchGet(workerCnt) - if err != nil || len(bws) == 0 { - logutil.BgLogger().Debug("[ddl] no backfill worker available now", zap.Error(err)) - return nil, errors.Trace(err) - } - seCtxs := make([]sessionctx.Context, 0, len(bws)) - for i := 0; i < len(bws); i++ { - se, err := d.sessPool.get() - if err != nil { - logutil.BgLogger().Fatal("[ddl] dispatch backfill jobs loop get session failed, it should not happen, please try restart TiDB", zap.Error(err)) - } - sess := newSession(se) - bfCtx := newBackfillCtx(d.ddlCtx, 0, sess, reorgTp, schemaName, tbl) - bf, err := bfFunc(bfCtx) - if err != nil { - logutil.BgLogger().Fatal("[ddl] dispatch backfill jobs loop new add index worker failed, it should not happen, please try restart TiDB", zap.Error(err)) - return nil, errors.Trace(err) - } - seCtxs = append(seCtxs, se) - bws[i].backfiller = bf - } - return &backfillWorkerContext{backfillWorkers: bws, sessCtxs: seCtxs}, nil -} - -func (bwCtx *backfillWorkerContext) GetContext() *backfillWorker { - bwCtx.mu.Lock() - offset := bwCtx.currID % len(bwCtx.backfillWorkers) - bw := bwCtx.backfillWorkers[offset] - bwStr := fmt.Sprintf("%#v", bw) - logutil.BgLogger().Warn("[ddl] GetContext", zap.Int("workers", len(bwCtx.backfillWorkers)), zap.Int("currID", bwCtx.currID), zap.Int("offset", offset), - zap.String("bw", bwStr)) - bwCtx.currID++ - bwCtx.mu.Unlock() - return bw -} - type backfillCtx struct { id int *ddlCtx @@ -287,134 +237,11 @@ type backfiller interface { String() string } -type backfilWorkerManager struct { - bwCtx *backfillWorkerContext - wg util.WaitGroupWrapper - exitCh chan struct{} -} - -func newBackfilWorkerManager(bwCtx *backfillWorkerContext) *backfilWorkerManager { - return &backfilWorkerManager{ - bwCtx: bwCtx, - exitCh: make(chan struct{}), - } -} - -func handleTask(task *reorgBackfillTask, _ int, bfWorker *backfillWorker) *backfillResult { - // To prevent different workers from using the same session. - // TODO: backfillWorkerPool is global, and bfWorkers is used in this function, we'd better do something make worker and job's ID can be matched. - bfWorker.runTask(task) - ret := <-bfWorker.resultCh - if dbterror.ErrDDLJobNotFound.Equal(ret.err) { - logutil.BgLogger().Info("the backfill job instance ID or lease is changed", zap.Error(ret.err)) - ret.err = nil - } - - return ret -} - -// backfillJob2Task builds reorg task. -func (dc *ddlCtx) backfillJob2Task(t table.Table, bfJob *BackfillJob) (*reorgBackfillTask, error) { - pt := t.(table.PhysicalTable) - if tbl, ok := t.(table.PartitionedTable); ok { - pt = tbl.GetPartition(bfJob.Meta.PhysicalTableID) - if pt == nil { - return nil, dbterror.ErrCancelledDDLJob.GenWithStack("Can not find partition id %d for table %d", bfJob.Meta.PhysicalTableID, t.Meta().ID) - } - } - endKey := bfJob.EndKey - // TODO: Check reorgInfo.mergingTmpIdx - endK, err := getRangeEndKey(dc.jobContext(bfJob.JobID), dc.store, bfJob.Meta.Priority, pt.RecordPrefix(), bfJob.StartKey, endKey) - if err != nil { - logutil.BgLogger().Info("[ddl] convert backfill job to task, get reverse key failed", zap.String("backfill job", bfJob.AbbrStr()), zap.Error(err)) - } else { - logutil.BgLogger().Info("[ddl] convert backfill job to task, change end key", zap.String("backfill job", - bfJob.AbbrStr()), zap.String("current key", hex.EncodeToString(bfJob.StartKey)), zap.Bool("end include", bfJob.Meta.EndInclude), - zap.String("end key", hex.EncodeToString(endKey)), zap.String("current end key", hex.EncodeToString(endK))) - endKey = endK - } - - return &reorgBackfillTask{ - bfJob: bfJob, - physicalTable: pt, - // TODO: Remove these fields after remove the old logic. - sqlQuery: bfJob.Meta.Query, - startKey: bfJob.StartKey, - endKey: endKey, - endInclude: bfJob.Meta.EndInclude, - priority: bfJob.Meta.Priority}, nil -} - -func GetTasks(d *ddlCtx, sess *session, tbl table.Table, runningJobID int64, concurrency int) ([]*reorgBackfillTask, error) { - // TODO: At present, only add index is processed. In the future, different elements need to be distinguished. - var err error - var bJobs []*BackfillJob - for i := 0; i < retrySQLTimes; i++ { - bJobs, err = GetAndMarkBackfillJobsForOneEle(sess, concurrency, runningJobID, d.uuid, InstanceLease) - if err != nil { - // TODO: test: if all tidbs can't get the unmark backfill job(a tidb mark a backfill job, other tidbs returned, then the tidb can't handle this job.) - if dbterror.ErrDDLJobNotFound.Equal(err) { - logutil.BgLogger().Info("no backfill job, handle backfill task finished") - return nil, err - } - if kv.ErrWriteConflict.Equal(err) { - logutil.BgLogger().Info("GetAndMarkBackfillJobsForOneEle failed", zap.Error(err)) - time.Sleep(retrySQLInterval) - continue - } - } - - tasks := make([]*reorgBackfillTask, 0, len(bJobs)) - for _, bJ := range bJobs { - task, err := d.backfillJob2Task(tbl, bJ) - if err != nil { - return nil, err - } - tasks = append(tasks, task) - } - return tasks, nil - } - - return nil, err -} - -func (bwm *backfilWorkerManager) waitFinalResult(resultCh <-chan *backfillResult) { - bwm.wg.Run(func() { - for { - select { - case result, ok := <-resultCh: - logutil.BgLogger().Warn("=========================== handle backfill task failed", zap.Bool("ok", ok)) - if result.err != nil { - logutil.BgLogger().Warn("handle backfill task failed", zap.Error(result.err)) - return - } - if !ok { - return - } - case <-bwm.exitCh: - logutil.BgLogger().Warn("=========================== exit xxx") - return - } - } - }) -} - -func (bwm *backfilWorkerManager) close(d *ddl) { - close(bwm.exitCh) - bwm.wg.Wait() - - for _, s := range bwm.bwCtx.sessCtxs { - d.sessPool.put(s) - } - for _, w := range bwm.bwCtx.backfillWorkers { - d.backfillCtxPool.put(w) - } -} - type backfillResult struct { taskID int addedCount int scanCount int + unsyncErr error nextKey kv.Key err error } @@ -498,7 +325,6 @@ func (w *backfillWorker) updateLease(execID string, bfJob *BackfillJob, nextKey } func (w *backfillWorker) finishJob(bfJob *BackfillJob) error { - bfJob.State = model.JobStateDone return w.backfiller.FinishTask(bfJob) } @@ -612,15 +438,18 @@ func (w *backfillWorker) handleBackfillTask(d *ddlCtx, task *reorgBackfillTask, return result } -func (w *backfillWorker) runTask(task *reorgBackfillTask) { - logutil.BgLogger().Info("[ddl] backfill worker start", zap.Stringer("worker", w)) - w.GetCtx().setDDLLabelForTopSQL(task.jobID, task.sqlQuery) +func (w *backfillWorker) runTask(task *reorgBackfillTask) (result *backfillResult) { + logutil.BgLogger().Info("[ddl] backfill worker start", zap.Stringer("worker", w), zap.String("task", task.String())) + defer util.Recover(metrics.LabelDDL, "backfillWorker.runTask", func() { + result = &backfillResult{taskID: task.id, err: dbterror.ErrReorgPanic} + }, false) + defer w.GetCtx().setDDLLabelForTopSQL(task.jobID, task.sqlQuery) - logutil.BgLogger().Debug("[ddl] backfill worker got task", zap.Int("workerID", w.GetCtx().id), zap.String("task", task.String())) failpoint.Inject("mockBackfillRunErr", func() { if w.GetCtx().id == 0 { result := &backfillResult{taskID: task.id, addedCount: 0, nextKey: nil, err: errors.Errorf("mock backfill error")} - w.resultCh <- result + logutil.BgLogger().Info("OOOOOOOOOOOOOOOOOOOOOOOOOOOOOOOOOOOOOOOOOOOOOOOOOOOOOOOOOOOOOOOOOOOO") + failpoint.Return(result) } }) failpoint.Inject("mockHighLoadForAddIndex", func() { @@ -633,18 +462,29 @@ func (w *backfillWorker) runTask(task *reorgBackfillTask) { // Change the batch size dynamically. w.GetCtx().batchCnt = int(variable.GetDDLReorgBatchSize()) - result := w.handleBackfillTask(w.GetCtx().ddlCtx, task, w.backfiller) - w.resultCh <- result + result = w.handleBackfillTask(w.GetCtx().ddlCtx, task, w.backfiller) + finish := injectSpan(task.bfJob.JobID+100, fmt.Sprintf("handle-job-%d-task-%d", task.bfJob.JobID, task.bfJob.ID)) + defer finish() + task.bfJob.RowCount = int64(result.addedCount) if result.err != nil { - logutil.BgLogger().Info("[ddl] backfill worker exit on error", - zap.Stringer("worker", w), zap.Error(result.err)) - return + logutil.BgLogger().Warn("[ddl] backfill worker runTask failed", + zap.Stringer("worker", w), zap.String("backfillJob", task.bfJob.AbbrStr()), zap.Error(result.err)) + if dbterror.ErrDDLJobNotFound.Equal(result.err) { + result.err = nil + return result + } + task.bfJob.State = model.JobStateCancelled + task.bfJob.Meta.Error = toTError(result.err) + if err := w.finishJob(task.bfJob); err != nil { + logutil.BgLogger().Info("[ddl] backfill worker runTask, finishJob failed", + zap.Stringer("worker", w), zap.String("backfillJob", task.bfJob.AbbrStr()), zap.Error(err)) + result.err = err + } + } else { + task.bfJob.State = model.JobStateDone + result.err = w.finishJob(task.bfJob) } - traceID := task.bfJob.JobID + 100 - finish := injectSpan(traceID, fmt.Sprintf("handle-job-%d-task-%d", task.bfJob.JobID, task.bfJob.ID)) - task.bfJob.RowCount = int64(result.addedCount) - w.finishJob(task.bfJob) - finish() + return result } func (w *backfillWorker) run(d *ddlCtx, bf backfiller, job *model.Job) { @@ -1317,8 +1157,7 @@ func (dc *ddlCtx) splitTableToBackfillJobs(sess *session, reorgInfo *reorgInfo, isFirstOps = false remains := kvRanges[len(batchTasks):] - asyncNotify(dc.backfillJobCh) - dc.asyncNotifyWorker(dc.backfillJobCh, addingBackfillJob, bJobs[0].ID, "backfill_job") + dc.asyncNotifyWorker(dc.backfillJobCh, addingBackfillJob, reorgInfo.Job.ID, "backfill_job") logutil.BgLogger().Info("[ddl] split backfill jobs to the backfill table", zap.Int("batchTasksCnt", len(batchTasks)), zap.Int("totalRegionCnt", len(kvRanges)), @@ -1352,16 +1191,17 @@ func (dc *ddlCtx) controlWritePhysicalTableRecord(sess *session, t table.Physica return nil } - if err := dc.isReorgRunnable(reorgInfo.Job.ID); err != nil { + ddlJobID := reorgInfo.Job.ID + if err := dc.isReorgRunnable(ddlJobID); err != nil { return errors.Trace(err) } currBackfillJobID := int64(1) - err := checkAndHandleInterruptedBackfillJobs(sess, reorgInfo.Job.ID, reorgInfo.currElement.ID, reorgInfo.currElement.TypeKey) + err := checkAndHandleInterruptedBackfillJobs(sess, ddlJobID, reorgInfo.currElement.ID, reorgInfo.currElement.TypeKey) if err != nil { return errors.Trace(err) } - maxBfJob, err := GetMaxBackfillJob(sess, reorgInfo.Job.ID, reorgInfo.currElement.ID, reorgInfo.currElement.TypeKey) + maxBfJob, err := GetMaxBackfillJob(sess, ddlJobID, reorgInfo.currElement.ID, reorgInfo.currElement.TypeKey) if err != nil { return errors.Trace(err) } @@ -1376,50 +1216,52 @@ func (dc *ddlCtx) controlWritePhysicalTableRecord(sess *session, t table.Physica isUnique = idxInfo.Unique } logutil.BgLogger().Info("[ddl] control write physical table record ----------------- 00", - zap.Int64("jobID", reorgInfo.Job.ID), zap.Reflect("maxBfJob", maxBfJob)) + zap.Int64("ddlJobID", ddlJobID), zap.Reflect("maxBfJob", maxBfJob)) err = dc.splitTableToBackfillJobs(sess, reorgInfo, t, isUnique, bfWorkerType, startKey, currBackfillJobID) if err != nil { return errors.Trace(err) } var backfillJobFinished bool - jobID := reorgInfo.Job.ID + currEle := reorgInfo.currElement ticker := time.NewTicker(300 * time.Millisecond) defer ticker.Stop() for { - if err := dc.isReorgRunnable(reorgInfo.Job.ID); err != nil { + if err := dc.isReorgRunnable(ddlJobID); err != nil { return errors.Trace(err) } select { case <-ticker.C: if !backfillJobFinished { - err := checkAndHandleInterruptedBackfillJobs(sess, jobID, reorgInfo.currElement.ID, reorgInfo.currElement.TypeKey) + err := checkAndHandleInterruptedBackfillJobs(sess, ddlJobID, currEle.ID, currEle.TypeKey) if err != nil { - logutil.BgLogger().Warn("[ddl] finish interrupted backfill jobs", zap.Int64("job ID", jobID), zap.Error(err)) + logutil.BgLogger().Warn("[ddl] finish interrupted backfill jobs", zap.Int64("job ID", ddlJobID), zap.Error(err)) return errors.Trace(err) } - bfJob, err := getBackfillJobWithRetry(sess, BackfillTable, jobID, reorgInfo.currElement.ID, reorgInfo.currElement.TypeKey, false) + bfJob, err := getBackfillJobWithRetry(sess, BackfillTable, ddlJobID, currEle.ID, currEle.TypeKey, false) if err != nil { - logutil.BgLogger().Info("[ddl] getBackfillJobWithRetry failed", zap.Int64("job ID", jobID), zap.Error(err)) + logutil.BgLogger().Info("[ddl] getBackfillJobWithRetry failed", zap.Int64("job ID", ddlJobID), zap.Error(err)) return errors.Trace(err) } + logutil.BgLogger().Info("[ddl] control write physical table record ---------------------------------- 11", zap.Bool("bfJobIsNil", bfJob == nil)) if bfJob == nil { backfillJobFinished = true - logutil.BgLogger().Info("[ddl] finish backfill jobs", zap.Int64("job ID", jobID)) + logutil.BgLogger().Info("[ddl] finish backfill jobs", zap.Int64("job ID", ddlJobID)) } } if backfillJobFinished { // TODO: Consider whether these backfill jobs are always out of sync. - isSynced, err := checkJobIsSynced(sess, jobID) + isSynced, err := checkJobIsFinished(sess, ddlJobID) if err != nil { - logutil.BgLogger().Warn("[ddl] checkJobIsSynced failed", zap.Int64("job ID", jobID), zap.Error(err)) + logutil.BgLogger().Warn("[ddl] checkJobIsFinished failed", zap.Int64("job ID", ddlJobID), zap.Error(err)) return errors.Trace(err) } + logutil.BgLogger().Info("[ddl] control write physical table record ---------------------------------- 22", zap.Bool("isSynced", isSynced)) if isSynced { - logutil.BgLogger().Info("[ddl] sync backfill jobs", zap.Int64("job ID", jobID)) - return nil + logutil.BgLogger().Info("[ddl] finish backfill jobs and put them to history", zap.Int64("job ID", ddlJobID)) + return GetBackfillErr(sess, ddlJobID, currEle.ID, currEle.TypeKey) } } case <-dc.ctx.Done(): @@ -1428,11 +1270,11 @@ func (dc *ddlCtx) controlWritePhysicalTableRecord(sess *session, t table.Physica } } -func checkJobIsSynced(sess *session, jobID int64) (bool, error) { +func checkJobIsFinished(sess *session, ddlJobID int64) (bool, error) { var err error var unsyncedInstanceIDs []string for i := 0; i < retrySQLTimes; i++ { - unsyncedInstanceIDs, err = getUnsyncedInstanceIDs(sess, jobID, "check_backfill_history_job_sync") + unsyncedInstanceIDs, err = getUnsyncedInstanceIDs(sess, ddlJobID, "check_backfill_history_job_sync") if err == nil && len(unsyncedInstanceIDs) == 0 { return true, nil } @@ -1445,10 +1287,36 @@ func checkJobIsSynced(sess *session, jobID int64) (bool, error) { return false, errors.Trace(err) } -func checkAndHandleInterruptedBackfillJobs(sess *session, jobID, currEleID int64, currEleKey []byte) (err error) { +func GetBackfillErr(sess *session, ddlJobID, currEleID int64, currEleKey []byte) error { + var err error + var metas []*model.BackfillMeta + for i := 0; i < retrySQLTimes; i++ { + metas, err = GetBackfillMetas(sess, BackfillHistoryTable, fmt.Sprintf("ddl_job_id = %d and ele_id = %d and ele_key = %s", + ddlJobID, currEleID, wrapKey2String(currEleKey)), "get_backfill_job_metas") + if err == nil { + for _, m := range metas { + if m.Error != nil { + logutil.BgLogger().Info("[ddl] control write physical table record ---------------------------------- 33", + zap.Int64("job ID", ddlJobID), zap.Error(m.Error)) + return m.Error + } + } + logutil.BgLogger().Info("[ddl] control write physical table record ---------------------------------- 33", + zap.Int64("job ID", ddlJobID)) + return nil + } + + logutil.BgLogger().Info("[ddl] GetBackfillMetas failed in checkJobIsSynced", zap.Int("tryTimes", i), zap.Error(err)) + time.Sleep(retrySQLInterval) + } + + return errors.Trace(err) +} + +func checkAndHandleInterruptedBackfillJobs(sess *session, ddlJobID, currEleID int64, currEleKey []byte) (err error) { var bJobs []*BackfillJob for i := 0; i < retrySQLTimes; i++ { - bJobs, err = GetInterruptedBackfillJobsForOneEle(sess, jobID, currEleID, currEleKey) + bJobs, err = GetInterruptedBackfillJobsForOneEle(sess, ddlJobID, currEleID, currEleKey) if err == nil { break } @@ -1466,7 +1334,7 @@ func checkAndHandleInterruptedBackfillJobs(sess *session, jobID, currEleID int64 for i := 0; i < retrySQLTimes; i++ { err = MoveBackfillJobsToHistoryTable(sess, bJobs[0]) if err == nil { - return errors.Errorf(bJobs[0].Meta.ErrMsg) + return bJobs[0].Meta.Error } logutil.BgLogger().Info("[ddl] MoveBackfillJobsToHistoryTable failed", zap.Error(err)) time.Sleep(retrySQLInterval) @@ -1474,14 +1342,14 @@ func checkAndHandleInterruptedBackfillJobs(sess *session, jobID, currEleID int64 return errors.Trace(err) } -func checkBackfillJobCount(sess *session, jobID, currEleID int64, currEleKey []byte) (backfillJobCnt int, err error) { - err = checkAndHandleInterruptedBackfillJobs(sess, jobID, currEleID, currEleKey) +func checkBackfillJobCount(sess *session, ddlJobID, currEleID int64, currEleKey []byte) (backfillJobCnt int, err error) { + err = checkAndHandleInterruptedBackfillJobs(sess, ddlJobID, currEleID, currEleKey) if err != nil { return 0, errors.Trace(err) } - backfillJobCnt, err = GetBackfillJobCount(sess, BackfillTable, fmt.Sprintf("ddl_job_id = %d and ele_id = %d and ele_key = '%s'", - jobID, currEleID, currEleKey), "check_backfill_job_count") + backfillJobCnt, err = GetBackfillJobCount(sess, BackfillTable, fmt.Sprintf("ddl_job_id = %d and ele_id = %d and ele_key = %s", + ddlJobID, currEleID, wrapKey2String(currEleKey)), "check_backfill_job_count") if err != nil { return 0, errors.Trace(err) } @@ -1489,7 +1357,7 @@ func checkBackfillJobCount(sess *session, jobID, currEleID int64, currEleKey []b return backfillJobCnt, nil } -func getBackfillJobWithRetry(sess *session, tableName string, jobID, currEleID int64, currEleKey []byte, isDesc bool) (*BackfillJob, error) { +func getBackfillJobWithRetry(sess *session, tableName string, ddlJobID, currEleID int64, currEleKey []byte, isDesc bool) (*BackfillJob, error) { var err error var bJobs []*BackfillJob descStr := "" @@ -1497,8 +1365,8 @@ func getBackfillJobWithRetry(sess *session, tableName string, jobID, currEleID i descStr = "order by id desc" } for i := 0; i < retrySQLTimes; i++ { - bJobs, err = GetBackfillJobs(sess, tableName, fmt.Sprintf("ddl_job_id = %d and ele_id = %d and ele_key = '%s' %s limit 1", - jobID, currEleID, currEleKey, descStr), "check_backfill_job_state") + bJobs, err = GetBackfillJobs(sess, tableName, fmt.Sprintf("ddl_job_id = %d and ele_id = %d and ele_key = %s %s limit 1", + ddlJobID, currEleID, wrapKey2String(currEleKey), descStr), "check_backfill_job_state") if err != nil { logutil.BgLogger().Warn("[ddl] GetBackfillJobs failed", zap.Error(err)) continue @@ -1513,12 +1381,12 @@ func getBackfillJobWithRetry(sess *session, tableName string, jobID, currEleID i } // GetMaxBackfillJob gets the max backfill job in BackfillTable and BackfillHistoryTable. -func GetMaxBackfillJob(sess *session, jobID, currEleID int64, currEleKey []byte) (*BackfillJob, error) { - bfJob, err := getBackfillJobWithRetry(sess, BackfillTable, jobID, currEleID, currEleKey, true) +func GetMaxBackfillJob(sess *session, ddlJobID, currEleID int64, currEleKey []byte) (*BackfillJob, error) { + bfJob, err := getBackfillJobWithRetry(sess, BackfillTable, ddlJobID, currEleID, currEleKey, true) if err != nil { return nil, errors.Trace(err) } - hJob, err := getBackfillJobWithRetry(sess, BackfillHistoryTable, jobID, currEleID, currEleKey, true) + hJob, err := getBackfillJobWithRetry(sess, BackfillHistoryTable, ddlJobID, currEleID, currEleKey, true) if err != nil { return nil, errors.Trace(err) } @@ -1544,8 +1412,8 @@ func MoveBackfillJobsToHistoryTable(sessCtx sessionctx.Context, bfJob *BackfillJ return runInTxn(sess, func(se *session) error { // TODO: Consider batch by batch update backfill jobs and insert backfill history jobs. - bJobs, err := GetBackfillJobs(sess, BackfillTable, fmt.Sprintf("ddl_job_id = %d and ele_id = %d and ele_key = '%s'", - bfJob.JobID, bfJob.EleID, bfJob.EleKey), "update_backfill_job") + bJobs, err := GetBackfillJobs(sess, BackfillTable, fmt.Sprintf("ddl_job_id = %d and ele_id = %d and ele_key = %s", + bfJob.JobID, bfJob.EleID, wrapKey2String(bfJob.EleKey)), "update_backfill_job") if err != nil { return errors.Trace(err) } diff --git a/ddl/db_integration_test.go b/ddl/db_integration_test.go index 213a23536a2d3..d7055cfa23cd9 100644 --- a/ddl/db_integration_test.go +++ b/ddl/db_integration_test.go @@ -1338,6 +1338,9 @@ func TestAddIndexAfterAddColumn(t *testing.T) { tk.MustExec("insert into test_add_index_after_add_col values(1, 2),(2,2)") tk.MustExec("alter table test_add_index_after_add_col add column c int not null default '0'") sql := "alter table test_add_index_after_add_col add unique index cc(c) " + limit := variable.GetDDLErrorCountLimit() + variable.SetDDLErrorCountLimit(3) + defer variable.SetDDLErrorCountLimit(limit) tk.MustGetErrCode(sql, errno.ErrDupEntry) sql = "alter table test_add_index_after_add_col add index idx_test(f1,f2,f3,f4,f5,f6,f7,f8,f9,f10,f11,f12,f13,f14,f15,f16,f17);" tk.MustGetErrCode(sql, errno.ErrTooManyKeyParts) @@ -4322,10 +4325,12 @@ func TestAddIndexX(t *testing.T) { tk.MustExec("create table test_add_index(a int, b int not null default '0')") tk.MustExec("insert into test_add_index values(1, 1),(20,20),(300,300),(4000,4000),(50000,50000),(123456,123456),(1234567,1234567),(12345678,12345678)") tk.MustExec("split table test_add_index BETWEEN (0) AND (10000000) REGIONS 7;") + fmt.Println("****************************************************************************** 00") tk.MustExec("alter table test_add_index add index idx(b)") tk.MustExec("admin check table test_add_index") tk.MustQuery("select count(1) from mysql.tidb_ddl_backfill").Check(testkit.Rows("0")) tk.MustQuery("select id, ele_id, ele_key, type, state from mysql.tidb_ddl_backfill_history").Check(testkit.Rows("1 1 _idx_ 0 6")) + fmt.Println("****************************************************************************** 11") tk.MustExec("alter table test_add_index add unique index idx1(b)") tk.MustExec("admin check table test_add_index") tk.MustQuery("select count(1) from mysql.tidb_ddl_backfill").Check(testkit.Rows("0")) diff --git a/ddl/ddl.go b/ddl/ddl.go index 89a0433164c15..486023feaf2b3 100644 --- a/ddl/ddl.go +++ b/ddl/ddl.go @@ -476,27 +476,66 @@ func (dc *ddlCtx) jobContext(jobID int64) *JobContext { return NewJobContext() } +func (dc *ddlCtx) removeBackfillCtxJobCtx(jobID int64) { + dc.backfillCtx.Lock() + delete(dc.backfillCtx.jobCtxMap, jobID) + dc.backfillCtx.Unlock() +} + +func (dc *ddlCtx) backfillCtxJobIDs() []int64 { + dc.backfillCtx.Lock() + defer dc.backfillCtx.Unlock() + + runningJobIDs := make([]int64, 0, len(dc.backfillCtx.jobCtxMap)) + for id := range dc.backfillCtx.jobCtxMap { + runningJobIDs = append(runningJobIDs, id) + } + return runningJobIDs +} + +func (dc *ddlCtx) setBackfillCtxJobContext(jobID int64, jobQuery string, jobType model.ActionType) (*JobContext, bool) { + dc.backfillCtx.Lock() + defer dc.backfillCtx.Unlock() + + jobCtx, existent := dc.backfillCtx.jobCtxMap[jobID] + if !existent { + dc.setDDLLabelForTopSQL(jobID, jobQuery) + dc.setDDLSourceForDiagnosis(jobID, jobType) + jobCtx = dc.jobContext(jobID) + dc.backfillCtx.jobCtxMap[jobID] = jobCtx + } + return jobCtx, existent +} + func (dc *ddlCtx) getReorgCtx(jobID int64) *reorgCtx { dc.reorgCtx.RLock() defer dc.reorgCtx.RUnlock() return dc.reorgCtx.reorgCtxMap[jobID] } -func (dc *ddlCtx) newReorgCtx(r *reorgInfo) *reorgCtx { +func (dc *ddlCtx) newReorgCtx(jobID int64, startKey []byte, currElement *meta.Element, rowCount int64) *reorgCtx { rc := &reorgCtx{} rc.doneCh = make(chan error, 1) // initial reorgCtx - rc.setRowCount(r.Job.GetRowCount()) - rc.setNextKey(r.StartKey) - rc.setCurrentElement(r.currElement) + rc.setRowCount(rowCount) + rc.setNextKey(startKey) + rc.setCurrentElement(currElement) rc.mu.warnings = make(map[errors.ErrorID]*terror.Error) rc.mu.warningsCount = make(map[errors.ErrorID]int64) dc.reorgCtx.Lock() defer dc.reorgCtx.Unlock() - dc.reorgCtx.reorgCtxMap[r.Job.ID] = rc + dc.reorgCtx.reorgCtxMap[jobID] = rc return rc } +func (dc *ddlCtx) setReorgCtxForBackfill(bfJob *BackfillJob) { + rc := dc.getReorgCtx(bfJob.JobID) + if rc == nil { + ele := &meta.Element{ID: bfJob.EleID, TypeKey: bfJob.EleKey} + dc.newReorgCtx(bfJob.JobID, bfJob.StartKey, ele, bfJob.RowCount) + } +} + func (dc *ddlCtx) removeReorgCtx(job *model.Job) { dc.reorgCtx.Lock() defer dc.reorgCtx.Unlock() diff --git a/ddl/ddl_workerpool.go b/ddl/ddl_workerpool.go index ece51a333d62b..3ed4d8f499fa9 100644 --- a/ddl/ddl_workerpool.go +++ b/ddl/ddl_workerpool.go @@ -150,6 +150,9 @@ func (bcp *backfillCtxPool) batchGet(cnt int) ([]*backfillWorker, error) { return nil, errors.Trace(err) } if res == nil { + if len(workers) == 0 { + return nil, nil + } return workers, nil } worker := res.(*backfillWorker) @@ -159,19 +162,6 @@ func (bcp *backfillCtxPool) batchGet(cnt int) ([]*backfillWorker, error) { return workers, nil } -// batchPut returns workerPool to context resource pool. -func (bcp *backfillCtxPool) batchPut(wks []*backfillWorker) { - if bcp.resPool == nil || bcp.exit.Load() { - return - } - - // No need to protect bcp.resPool, even the bcp.resPool is closed, the ctx still need to - // put into resPool, because when resPool is closing, it will wait all the ctx returns, then resPool finish closing. - for _, wk := range wks { - bcp.resPool.Put(wk) - } -} - // put returns workerPool to context resource pool. func (bcp *backfillCtxPool) put(wk *backfillWorker) { if bcp.resPool == nil || bcp.exit.Load() { diff --git a/ddl/ddl_workerpool_test.go b/ddl/ddl_workerpool_test.go index df39d463c23f9..8f86e816507d5 100644 --- a/ddl/ddl_workerpool_test.go +++ b/ddl/ddl_workerpool_test.go @@ -41,18 +41,26 @@ func TestBackfillWorkerPool(t *testing.T) { return wk, nil } } - pool := newBackfillContextPool(pools.NewResourcePool(f(), 1, 2, 0)) + pool := newBackfillContextPool(pools.NewResourcePool(f(), 3, 4, 0)) bc, err := pool.get() require.NoError(t, err) require.NotNil(t, bc) require.Nil(t, bc.backfiller) + bcs, err := pool.batchGet(3) + require.NoError(t, err) + require.Len(t, bcs, 2) // test it to reach the capacity bc1, err := pool.get() require.NoError(t, err) require.Nil(t, bc1) + bcs1, err := pool.batchGet(1) + require.NoError(t, err) + require.Nil(t, bcs1) // test setCapacity - err = pool.setCapacity(2) + err = pool.setCapacity(5) + require.Equal(t, "capacity 5 is out of range", err.Error()) + err = pool.setCapacity(4) require.NoError(t, err) bc1, err = pool.get() require.NoError(t, err) @@ -60,6 +68,9 @@ func TestBackfillWorkerPool(t *testing.T) { require.Nil(t, bc.backfiller) pool.put(bc) pool.put(bc1) + for _, bc := range bcs { + pool.put(bc) + } // test close pool.close() @@ -71,4 +82,9 @@ func TestBackfillWorkerPool(t *testing.T) { require.Error(t, err) require.Equal(t, "backfill worker pool is closed", err.Error()) require.Nil(t, bc) + + bcs, err = pool.batchGet(1) + require.Error(t, err) + require.Equal(t, "backfill worker pool is closed", err.Error()) + require.Nil(t, bcs) } diff --git a/ddl/dist_backfilling.go b/ddl/dist_backfilling.go new file mode 100644 index 0000000000000..f94371e0a1ff6 --- /dev/null +++ b/ddl/dist_backfilling.go @@ -0,0 +1,203 @@ +// Copyright 2023 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package ddl + +import ( + "encoding/hex" + "sync" + "time" + + "github.com/pingcap/errors" + "github.com/pingcap/tidb/kv" + "github.com/pingcap/tidb/parser/model" + "github.com/pingcap/tidb/resourcemanager/pooltask" + "github.com/pingcap/tidb/sessionctx" + "github.com/pingcap/tidb/table" + "github.com/pingcap/tidb/util" + "github.com/pingcap/tidb/util/dbterror" + "github.com/pingcap/tidb/util/logutil" + "go.uber.org/zap" +) + +type backfillWorkerContext struct { + currID int + mu sync.Mutex + sessCtxs []sessionctx.Context + backfillWorkers []*backfillWorker +} + +type newBackfillerFunc func(bfCtx *backfillCtx) (bf backfiller, err error) + +func newBackfillWorkerContext(d *ddl, schemaName string, tbl table.Table, workerCnt int, reorgTp model.ReorgType, + bfFunc newBackfillerFunc) (*backfillWorkerContext, error) { + if workerCnt <= 0 { + return nil, nil + } + + bCtxs, err := d.backfillCtxPool.batchGet(workerCnt) + if err != nil || len(bCtxs) == 0 { + logutil.BgLogger().Debug("[ddl] no backfill context available now", zap.Int("backfillCtx", len(bCtxs)), zap.Error(err)) + return nil, errors.Trace(err) + } + seCtxs := make([]sessionctx.Context, 0, len(bCtxs)) + for i := 0; i < len(bCtxs); i++ { + se, err := d.sessPool.get() + if err != nil { + logutil.BgLogger().Error("[ddl] new backfill worker context, get a session failed", zap.Error(err)) + return nil, errors.Trace(err) + } + sess := newSession(se) + bfCtx := newBackfillCtx(d.ddlCtx, 0, sess, reorgTp, schemaName, tbl) + bf, err := bfFunc(bfCtx) + if err != nil { + logutil.BgLogger().Error("[ddl] new backfill worker context, do bfFunc failed", zap.Error(err)) + return nil, errors.Trace(err) + } + seCtxs = append(seCtxs, se) + bCtxs[i].backfiller = bf + } + return &backfillWorkerContext{backfillWorkers: bCtxs, sessCtxs: seCtxs}, nil +} + +func (bwCtx *backfillWorkerContext) GetContext() *backfillWorker { + bwCtx.mu.Lock() + // TODO: Special considerations are required if the number of consumers we get from the backfillWorkerPool is increased. + offset := bwCtx.currID % len(bwCtx.backfillWorkers) + // To prevent different workers from using the same session. + bw := bwCtx.backfillWorkers[offset] + logutil.BgLogger().Info("[ddl] backfill worker get context", zap.Int("workerCount", len(bwCtx.backfillWorkers)), + zap.Int("currID", bwCtx.currID), zap.Int("offset", offset), zap.Stringer("backfill worker", bw)) + bwCtx.currID++ + bwCtx.mu.Unlock() + return bw +} + +type backfilWorkerManager struct { + bwCtx *backfillWorkerContext + wg util.WaitGroupWrapper + unsyncErr error + exitCh chan struct{} +} + +func newBackfilWorkerManager(bwCtx *backfillWorkerContext) *backfilWorkerManager { + return &backfilWorkerManager{ + bwCtx: bwCtx, + exitCh: make(chan struct{}), + } +} + +func (bwm *backfilWorkerManager) waitFinalResult(resultCh <-chan *backfillResult, + tControl pooltask.TaskController[*reorgBackfillTask, *backfillResult, int, *backfillWorker, *backfillWorkerContext]) { + bwm.wg.Run(func() { + for { + select { + case result, ok := <-resultCh: + logutil.BgLogger().Warn("=========================== handle backfill task failed", + zap.Bool("ok", ok)) + if !ok { + return + } + if result.err != nil { + logutil.BgLogger().Warn("handle backfill task failed", zap.Error(result.err)) + bwm.unsyncErr = result.err + // TODO: After spmc supports, exit this tControl related goroutines in spmc pool. + return + } + case <-bwm.exitCh: + logutil.BgLogger().Warn("=========================== exit xxx") + return + } + } + }) +} + +func (bwm *backfilWorkerManager) close(d *ddl) error { + close(bwm.exitCh) + bwm.wg.Wait() + + for _, s := range bwm.bwCtx.sessCtxs { + d.sessPool.put(s) + } + for _, w := range bwm.bwCtx.backfillWorkers { + d.backfillCtxPool.put(w) + } + + return bwm.unsyncErr +} + +// backfillJob2Task builds reorg task. +func (dc *ddlCtx) backfillJob2Task(t table.Table, bfJob *BackfillJob) (*reorgBackfillTask, error) { + pt := t.(table.PhysicalTable) + if tbl, ok := t.(table.PartitionedTable); ok { + pt = tbl.GetPartition(bfJob.Meta.PhysicalTableID) + if pt == nil { + return nil, dbterror.ErrCancelledDDLJob.GenWithStack("Can not find partition id %d for table %d", bfJob.Meta.PhysicalTableID, t.Meta().ID) + } + } + endKey := bfJob.EndKey + // TODO: Check reorgInfo.mergingTmpIdx + endK, err := getRangeEndKey(dc.jobContext(bfJob.JobID), dc.store, bfJob.Meta.Priority, pt.RecordPrefix(), bfJob.StartKey, endKey) + if err != nil { + logutil.BgLogger().Info("[ddl] convert backfill job to task, get reverse key failed", zap.String("backfill job", bfJob.AbbrStr()), zap.Error(err)) + } else { + logutil.BgLogger().Info("[ddl] convert backfill job to task, change end key", zap.String("backfill job", + bfJob.AbbrStr()), zap.String("current key", hex.EncodeToString(bfJob.StartKey)), zap.Bool("end include", bfJob.Meta.EndInclude), + zap.String("end key", hex.EncodeToString(endKey)), zap.String("current end key", hex.EncodeToString(endK))) + endKey = endK + } + + return &reorgBackfillTask{ + bfJob: bfJob, + physicalTable: pt, + // TODO: Remove these fields after remove the old logic. + sqlQuery: bfJob.Meta.Query, + startKey: bfJob.StartKey, + endKey: endKey, + endInclude: bfJob.Meta.EndInclude, + priority: bfJob.Meta.Priority}, nil +} + +func GetTasks(d *ddlCtx, sess *session, tbl table.Table, runningJobID int64, concurrency int) ([]*reorgBackfillTask, error) { + // TODO: At present, only add index is processed. In the future, different elements need to be distinguished. + var err error + var bJobs []*BackfillJob + for i := 0; i < retrySQLTimes; i++ { + bJobs, err = GetAndMarkBackfillJobsForOneEle(sess, concurrency, runningJobID, d.uuid, InstanceLease) + if err != nil { + // TODO: add test: if all tidbs can't get the unmark backfill job(a tidb mark a backfill job, other tidbs returned, then the tidb can't handle this job.) + if dbterror.ErrDDLJobNotFound.Equal(err) { + logutil.BgLogger().Info("no backfill job, handle backfill task finished") + return nil, err + } + if kv.ErrWriteConflict.Equal(err) { + logutil.BgLogger().Info("GetAndMarkBackfillJobsForOneEle failed", zap.Error(err)) + time.Sleep(retrySQLInterval) + continue + } + } + + tasks := make([]*reorgBackfillTask, 0, len(bJobs)) + for _, bJ := range bJobs { + task, err := d.backfillJob2Task(tbl, bJ) + if err != nil { + return nil, err + } + tasks = append(tasks, task) + } + return tasks, nil + } + + return nil, err +} diff --git a/ddl/failtest/fail_db_test.go b/ddl/failtest/fail_db_test.go index 4a938e5fd2ad4..6f33f6b2107dd 100644 --- a/ddl/failtest/fail_db_test.go +++ b/ddl/failtest/fail_db_test.go @@ -333,6 +333,10 @@ func TestRunDDLJobPanicDisableClusteredIndex(t *testing.T) { } func testAddIndexWorkerNum(t *testing.T, s *failedSuite, test func(*testkit.TestKit)) { + if variable.DDLEnableDistributeReorg.Load() { + t.Skip("dist reorg didn't support checkBackfillWorkerNum, skip this test") + } + tk := testkit.NewTestKit(t, s.store) tk.MustExec("create database if not exists test_db") tk.MustExec("use test_db") diff --git a/ddl/index.go b/ddl/index.go index f2e39542303cd..256ec6cd0c4bd 100644 --- a/ddl/index.go +++ b/ddl/index.go @@ -667,7 +667,13 @@ func (w *worker) onCreateIndex(d *ddlCtx, t *meta.Meta, job *model.Job, isPK boo if job.MultiSchemaInfo != nil { done, ver, err = doReorgWorkForCreateIndexMultiSchema(w, d, t, job, tbl, indexInfo) } else { - done, ver, err = doReorgWorkForCreateIndex(w, d, t, job, tbl, indexInfo) + // TODO: Support typeAddIndexMergeTmpWorker and partitionTable. + isDistReorg := variable.DDLEnableDistributeReorg.Load() + if isDistReorg { + done, ver, err = doReorgWorkForCreateIndexWithDistReorg(w, d, t, job, tbl, indexInfo) + } else { + done, ver, err = doReorgWorkForCreateIndex(w, d, t, job, tbl, indexInfo) + } } if !done { return ver, err @@ -801,7 +807,7 @@ func doReorgWorkForCreateIndexMultiSchema(w *worker, d *ddlCtx, t *meta.Meta, jo return true, ver, err } -func doReorgWorkForCreateIndexCopy(w *worker, d *ddlCtx, t *meta.Meta, job *model.Job, +func doReorgWorkForCreateIndex(w *worker, d *ddlCtx, t *meta.Meta, job *model.Job, tbl table.Table, indexInfo *model.IndexInfo) (done bool, ver int64, err error) { bfProcess := pickBackfillType(w, job) if !bfProcess.NeedMergeProcess() { @@ -884,7 +890,7 @@ func doReorgWorkForCreateIndexCopy(w *worker, d *ddlCtx, t *meta.Meta, job *mode } } -func doReorgWorkForCreateIndex(w *worker, d *ddlCtx, t *meta.Meta, job *model.Job, +func doReorgWorkForCreateIndexWithDistReorg(w *worker, d *ddlCtx, t *meta.Meta, job *model.Job, tbl table.Table, indexInfo *model.IndexInfo) (done bool, ver int64, err error) { bfProcess := pickBackfillType(w, job) if !bfProcess.NeedMergeProcess() { @@ -899,8 +905,9 @@ func doReorgWorkForCreateIndex(w *worker, d *ddlCtx, t *meta.Meta, job *model.Jo switch bfProcess { case model.ReorgTypeLitMerge: done, ver, err = runReorgJobAndHandleErr(w, d, t, job, tbl, indexInfo, false) + logutil.BgLogger().Warn("============================================ finish LitMerge", + zap.Bool("done", done), zap.Error(err)) if err != nil { - // TODO: err = tryFallbackToTxnMerge(job, err) return false, ver, errors.Trace(err) } @@ -909,6 +916,8 @@ func doReorgWorkForCreateIndex(w *worker, d *ddlCtx, t *meta.Meta, job *model.Jo } case model.ReorgTypeTxnMerge: done, ver, err = runReorgJobAndHandleErr(w, d, t, job, tbl, indexInfo, false) + logutil.BgLogger().Warn("============================================ finish TxnMerge", + zap.Bool("done", done), zap.Error(err)) if err != nil || !done { return false, ver, errors.Trace(err) } @@ -959,6 +968,7 @@ func runReorgJobAndHandleErr(w *worker, d *ddlCtx, t *meta.Meta, job *model.Job, }, false) return w.addTableIndex(tbl, reorgInfo) }) + logutil.BgLogger().Warn("============================================ run reorg job", zap.Error(err)) if err != nil { if dbterror.ErrWaitReorgTimeout.Equal(err) { // if timeout, we should return, check for the owner and re-wait job done. @@ -1382,14 +1392,15 @@ func (w *baseIndexWorker) GetCtx() *backfillCtx { return w.backfillCtx } -func newBaseIndexWorkerContext(d *ddl, sess *session, schemaName model.CIStr, tbl table.Table, workerCnt int, bfJob *BackfillJob, jobCtx *JobContext) (*backfillWorkerContext, error) { +func newAddIndexWorkerContext(d *ddl, sess *session, schemaName model.CIStr, tbl table.Table, workerCnt int, + bfJob *BackfillJob, jobCtx *JobContext) (*backfillWorkerContext, error) { //nolint:forcetypeassert phyTbl := tbl.(table.PhysicalTable) return newBackfillWorkerContext(d, schemaName.O, tbl, workerCnt, bfJob.Meta.ReorgTp, func(bfCtx *backfillCtx) (backfiller, error) { decodeColMap, err := makeupDecodeColMap(sess, schemaName, phyTbl) if err != nil { - logutil.BgLogger().Debug("[ddl] make up decode col map failed", zap.Error(err)) + logutil.BgLogger().Info("[ddl] make up decode column map failed", zap.Error(err)) return nil, errors.Trace(err) } bf, err1 := newAddIndexWorker(decodeColMap, phyTbl, bfCtx, jobCtx, bfJob.JobID, bfJob.EleID, bfJob.EleKey) @@ -1677,6 +1688,7 @@ func (w *addIndexWorker) BackfillDataInTxn(handleRange reorgBackfillTask) (taskC } taskCtx.nextKey = nextKey taskCtx.done = taskDone + logutil.BgLogger().Info("OOOOOOOOOOOOOOOOOOOOOOOOOOOOOOOOOOOOOOOOOOOOOOOOOOOOOOOOOOOOOOOOOOOO", zap.Bool("done", taskDone)) err = w.batchCheckUniqueKey(txn, idxRecords) if err != nil { @@ -1773,7 +1785,7 @@ func (w *worker) addTableIndex(t table.Table, reorgInfo *reorgInfo) error { phyTbl := t.(table.PhysicalTable) // TODO: Support typeAddIndexMergeTmpWorker and partitionTable. isDistReorg := variable.DDLEnableDistributeReorg.Load() - if isDistReorg && !reorgInfo.mergingTmpIdx { + if isDistReorg && !reorgInfo.mergingTmpIdx && reorgInfo.Job.Type == model.ActionAddIndex { sCtx, err := w.sessPool.get() if err != nil { return errors.Trace(err) @@ -2027,12 +2039,12 @@ func (w *worker) updateReorgInfoForPartitions(t table.PartitionedTable, reorg *r return false, errors.Trace(err) } -func runBackfillJobsWithLightning(d *ddl, sess *session, bfJob *BackfillJob, jobCtx *JobContext) { +func runBackfillJobsWithLightning(d *ddl, sess *session, bfJob *BackfillJob, jobCtx *JobContext) error { // TODO: Consider redo it. bc, ok := ingest.LitBackCtxMgr.Load(bfJob.JobID) if ok && bc.Done() { logutil.BgLogger().Warn("[ddl] lightning loaded") - return + return errors.New(ingest.LitErrGetBackendFail) } var err error logutil.BgLogger().Warn(fmt.Sprintf("------------------------------------ bf:%#v", bfJob)) @@ -2040,7 +2052,7 @@ func runBackfillJobsWithLightning(d *ddl, sess *session, bfJob *BackfillJob, job if err != nil { // TODO: tryFallbackToTxnMerge logutil.BgLogger().Warn("[ddl] lightning register error", zap.Error(err)) - return + return err } logutil.BgLogger().Warn("00 ******** load backfill job and run reorg jobs start", zap.Int64("job id", bfJob.JobID)) @@ -2049,7 +2061,7 @@ func runBackfillJobsWithLightning(d *ddl, sess *session, bfJob *BackfillJob, job logutil.BgLogger().Warn("[ddl] runBackfillJobs error", zap.Error(err)) ingest.LitBackCtxMgr.Unregister(bfJob.JobID) // TODO: tryFallbackToTxnMerge - return + return err } finish := injectSpan(bfJob.JobID, "finish-import") @@ -2063,11 +2075,12 @@ func runBackfillJobsWithLightning(d *ddl, sess *session, bfJob *BackfillJob, job // TODO: tryFallbackToTxnMerge } ingest.LitBackCtxMgr.Unregister(bfJob.JobID) - return + return err } ingest.LitBackCtxMgr.Unregister(bfJob.ID) finish() bc.SetDone() + return nil } // changingIndex is used to store the index that need to be changed during modifying column. diff --git a/ddl/index_merge_tmp_test.go b/ddl/index_merge_tmp_test.go index 6027c5cee215f..569afca194311 100644 --- a/ddl/index_merge_tmp_test.go +++ b/ddl/index_merge_tmp_test.go @@ -16,7 +16,6 @@ package ddl_test import ( "testing" - "time" "github.com/pingcap/tidb/ddl" "github.com/pingcap/tidb/ddl/ingest" @@ -396,6 +395,7 @@ func TestAddIndexMergeConflictWithPessimistic(t *testing.T) { callback := &ddl.TestDDLCallback{Do: dom} runPessimisticTxn := false + doRollbackCh := make(chan struct{}, 1) callback.OnJobRunBeforeExported = func(job *model.Job) { if t.Failed() { return @@ -418,6 +418,7 @@ func TestAddIndexMergeConflictWithPessimistic(t *testing.T) { assert.NoError(t, err) _, err = tk2.Exec("update t set a = 3 where id = 1;") assert.NoError(t, err) + doRollbackCh <- struct{}{} } } dom.DDL().SetHook(callback) @@ -426,13 +427,7 @@ func TestAddIndexMergeConflictWithPessimistic(t *testing.T) { tk.MustExec("alter table t add index idx(a);") afterCommit <- struct{}{} }() - timer := time.NewTimer(300 * time.Millisecond) - select { - case <-timer.C: - break - case <-afterCommit: - require.Fail(t, "should be blocked by the pessimistic txn") - } + <-doRollbackCh tk2.MustExec("rollback;") <-afterCommit dom.DDL().SetHook(originHook) diff --git a/ddl/job_table.go b/ddl/job_table.go index 6b2716925ae5e..2e9082cf75471 100644 --- a/ddl/job_table.go +++ b/ddl/job_table.go @@ -34,6 +34,7 @@ import ( "github.com/pingcap/tidb/parser/model" "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/table" + tidbutil "github.com/pingcap/tidb/util" "github.com/pingcap/tidb/util/dbterror" "github.com/pingcap/tidb/util/gpool/spmc" "github.com/pingcap/tidb/util/logutil" @@ -328,7 +329,7 @@ func (d *ddl) startDispatchBackfillJobsLoop() { if d.etcdCli != nil { notifyBackfillJobByEtcdCh = d.etcdCli.Watch(d.ctx, addingBackfillJob) } - ticker := time.NewTicker(300 * time.Millisecond) + ticker := time.NewTicker(500 * time.Millisecond) defer ticker.Stop() for { if isChanClosed(d.ctx.Done()) { @@ -383,50 +384,37 @@ func (d *ddl) loadBackfillJobAndRun() { logutil.BgLogger().Error("[ddl] load DDL reorganization variable failed", zap.Error(err)) } - d.backfillCtx.Lock() - jobCtxMapLen := len(d.backfillCtx.jobCtxMap) - runningJobIDs := make([]int64, 0, jobCtxMapLen) - if jobCtxMapLen >= reorgWorkerCnt { + runningJobIDs := d.backfillCtxJobIDs() + if len(runningJobIDs) >= reorgWorkerCnt { logutil.BgLogger().Warn("00 ******** load backfill job and run reorg jos is more than limit", zap.Int("limit", reorgWorkerCnt)) d.backfillCtx.Unlock() return - } else { - for id := range d.backfillCtx.jobCtxMap { - runningJobIDs = append(runningJobIDs, id) - } } - d.backfillCtx.Unlock() // TODO: Add ele info to distinguish backfill jobs. bJobs, err := GetBackfillJobsForOneEle(sess, 1, runningJobIDs, InstanceLease) bJobCnt := len(bJobs) if bJobCnt == 0 || err != nil { if err != nil { - logutil.BgLogger().Warn("[ddl] get backfill jobs met error", zap.Error(err)) + logutil.BgLogger().Warn("[ddl] get backfill jobs failed", zap.Error(err)) + } else { + logutil.BgLogger().Debug("[ddl] get no backfill job") } return } bJob := bJobs[0] - d.backfillCtx.Lock() - jobCtx, ok := d.backfillCtx.jobCtxMap[bJob.JobID] - if !ok { - d.setDDLLabelForTopSQL(bJob.JobID, bJob.Meta.Query) - d.setDDLSourceForDiagnosis(bJob.JobID, bJob.Meta.Type) - jobCtx = d.jobContext(bJob.JobID) - d.backfillCtx.jobCtxMap[bJob.JobID] = jobCtx - d.backfillCtx.Unlock() - } else { - logutil.BgLogger().Warn("00 ******** load backfill job and run reorg jobs exit", zap.Int64("job id", bJob.JobID)) - d.backfillCtx.Unlock() + jobCtx, existent := d.setBackfillCtxJobContext(bJob.JobID, bJob.Meta.Query, bJob.Meta.Type) + if existent { + logutil.BgLogger().Warn("[ddl] get the type of backfill job is running", zap.String("backfill job", bJob.AbbrStr())) return } - + // TODO: Adjust how the non-owner uses ReorgCtx. + d.setReorgCtxForBackfill(bJob) d.wg.Run(func() { defer func() { - d.backfillCtx.Lock() - delete(d.backfillCtx.jobCtxMap, bJob.JobID) - d.backfillCtx.Unlock() + d.removeBackfillCtxJobCtx(bJob.JobID) + tidbutil.Recover(metrics.LabelBackfillWorker, fmt.Sprintf("runBackfillJobs"), nil, false) logutil.BgLogger().Warn("00 ******** load backfill job and run reorg jobs finished", zap.Int64("job id", bJob.JobID)) }() traceID := bJob.ID + 100 @@ -438,18 +426,18 @@ func (d *ddl) loadBackfillJobAndRun() { zap.Bool("LitInitialized", ingest.LitInitialized), zap.String("bJob", bJob.AbbrStr())) return } - logutil.BgLogger().Info("[ddl] &&&&&&&&&&&&&&&&&&&&&&&&&&&&&&&&&&&&&&&&&&&&&&&& litMerge") - logutil.BgLogger().Info("run backfill job with LitMerge", zap.String("bJob", bJob.AbbrStr())) - runBackfillJobsWithLightning(d, sess, bJob, jobCtx) + logutil.BgLogger().Info("[ddl] run backfill job with LitMerge", zap.String("bJob", bJob.AbbrStr())) + err = runBackfillJobsWithLightning(d, sess, bJob, jobCtx) } else { - logutil.BgLogger().Info("[ddl] &&&&&&&&&&&&&&&&&&&&&&&&&&&&&&&&&&&&&&&&&&&&&&&& txnMerge") - logutil.BgLogger().Info("run backfill job with TxnMerge", zap.String("bJob", bJob.AbbrStr())) - runBackfillJobs(d, sess, bJob, jobCtx) + logutil.BgLogger().Info("[ddl] run backfill job with TxnMerge", zap.String("bJob", bJob.AbbrStr())) + _, err = runBackfillJobs(d, sess, bJob, jobCtx) } - err = syncBackfillHistoryJobs(sess, d.uuid, bJob) + if err == nil { + err = syncBackfillHistoryJobs(sess, d.uuid, bJob) + } if err != nil { - logutil.BgLogger().Warn("[ddl] syncBackfillHistoryJobs error", zap.Error(err)) + logutil.BgLogger().Warn("[ddl] run backfill job failed", zap.Error(err)) } details := collectTrace(bJob.JobID) logutil.BgLogger().Info("[ddl] &&&&&&&&&&&&&&&&&&&&&&&&&&&&&&&&-------------------------- finish backfill jobs", @@ -458,7 +446,6 @@ func (d *ddl) loadBackfillJobAndRun() { } func runBackfillJobs(d *ddl, sess *session, bJob *BackfillJob, jobCtx *JobContext) (table.Table, error) { - logutil.BgLogger().Warn("00 ******** run backfill jobs", zap.Int64("job id", bJob.JobID)) dbInfo, tbl, err := d.getTableByTxn(d.store, bJob.Meta.SchemaID, bJob.Meta.TableID) if err != nil { logutil.BgLogger().Warn("[ddl] runBackfillJobs gets table failed", zap.String("bfJob", bJob.AbbrStr()), zap.Error(err)) @@ -467,27 +454,27 @@ func runBackfillJobs(d *ddl, sess *session, bJob *BackfillJob, jobCtx *JobContex workerCnt := int(variable.GetDDLReorgWorkerCounter()) // TODO: Different worker using different newBackfillerFunc. - bwCtx, err := newBaseIndexWorkerContext(d, sess, dbInfo.Name, tbl, workerCnt, bJob, jobCtx) - if err != nil { - logutil.BgLogger().Info("[ddl] new add index worker context failed", zap.Error(err)) + workerCtx, err := newAddIndexWorkerContext(d, sess, dbInfo.Name, tbl, workerCnt, bJob, jobCtx) + if err != nil || workerCtx == nil { + logutil.BgLogger().Info("[ddl] new adding index worker context failed", zap.Reflect("workerCtx", workerCtx), zap.Error(err)) return nil, errors.Trace(err) } - bwMgr := newBackfilWorkerManager(bwCtx) + bwMgr := newBackfilWorkerManager(workerCtx) d.backfillWorkerPool.SetConsumerFunc(func(task *reorgBackfillTask, _ int, bfWorker *backfillWorker) *backfillResult { - return handleTask(task, 0, bfWorker) + return bfWorker.runTask(task) }) proFunc := func() ([]*reorgBackfillTask, error) { return GetTasks(d.ddlCtx, sess, tbl, bJob.JobID, workerCnt*2) } // add new task - resultCh, control := d.backfillWorkerPool.AddProduceBySlice(proFunc, 0, bwCtx, spmc.WithConcurrency(workerCnt)) - bwMgr.waitFinalResult(resultCh) + resultCh, control := d.backfillWorkerPool.AddProduceBySlice(proFunc, 0, workerCtx, spmc.WithConcurrency(workerCnt)) + bwMgr.waitFinalResult(resultCh, control) // waiting task finishing control.Wait() - bwMgr.close(d) + err = bwMgr.close(d) - return tbl, nil + return tbl, err } const ( @@ -678,33 +665,35 @@ func getJobsBySQL(sess *session, tbl, condition string) ([]*model.Job, error) { } func syncBackfillHistoryJobs(sess *session, uuid string, backfillJob *BackfillJob) error { - sql := fmt.Sprintf("update mysql.%s set state = %d where ddl_job_id = %d and ele_id = %d and ele_key = '%s' and exec_id = '%s' limit 1;", - BackfillHistoryTable, model.JobStateSynced, backfillJob.JobID, backfillJob.EleID, backfillJob.EleKey, uuid) + sql := fmt.Sprintf("update mysql.%s set state = %d where ddl_job_id = %d and ele_id = %d and ele_key = %s and exec_id = '%s' limit 1;", + BackfillHistoryTable, model.JobStateSynced, backfillJob.JobID, backfillJob.EleID, wrapKey2String(backfillJob.EleKey), uuid) logutil.BgLogger().Warn("update ***************************** " + fmt.Sprintf("sql:%v, sess:%#v", sql, sess)) _, err := sess.execute(context.Background(), sql, "sync_backfill_history_job") return err } func generateInsertBackfillJobSQL(tableName string, backfillJobs []*BackfillJob) (string, error) { - sqlPrefix := fmt.Sprintf("insert into mysql.%s(id, ddl_job_id, ele_id, ele_key, store_id, type, exec_id, exec_lease, state, curr_key, start_key, end_key, start_ts, finish_ts, row_count, backfill_meta) values", tableName) - var sql string + sqlBuilder := strings.Builder{} + sqlBuilder.WriteString("insert into mysql.") + sqlBuilder.WriteString(tableName) + sqlBuilder.WriteString("(id, ddl_job_id, ele_id, ele_key, store_id, type, exec_id, exec_lease, state, curr_key, start_key, end_key, start_ts, finish_ts, row_count, backfill_meta) values") + jobs := "" for i, bj := range backfillJobs { mateByte, err := bj.Meta.Encode() if err != nil { return "", errors.Trace(err) } - if i == 0 { - sql = sqlPrefix + fmt.Sprintf("(%d, %d, %d, '%s', %d, %d, '%s', '%s', %d, '%s', '%s', '%s', %d, %d, %d, '%s')", - bj.ID, bj.JobID, bj.EleID, bj.EleKey, bj.StoreID, bj.Tp, bj.InstanceID, bj.InstanceLease, bj.State, - bj.CurrKey, bj.StartKey, bj.EndKey, bj.StartTS, bj.FinishTS, bj.RowCount, mateByte) - continue + if i != 0 { + sqlBuilder.WriteString(", ") } - sql += fmt.Sprintf(", (%d, %d, %d, '%s', %d, %d, '%s', '%s', %d, '%s', '%s', '%s', %d, %d, %d, '%s')", - bj.ID, bj.JobID, bj.EleID, bj.EleKey, bj.StoreID, bj.Tp, bj.InstanceID, bj.InstanceLease, bj.State, - bj.CurrKey, bj.StartKey, bj.EndKey, bj.StartTS, bj.FinishTS, bj.RowCount, mateByte) + sqlBuilder.WriteString(fmt.Sprintf("(%d, %d, %d, %s, %d, %d, '%s', '%s', %d, %s, %s, %s, %d, %d, %d, %s)", + bj.ID, bj.JobID, bj.EleID, wrapKey2String(bj.EleKey), bj.StoreID, bj.Tp, bj.InstanceID, bj.InstanceLease, bj.State, wrapKey2String(bj.CurrKey), + wrapKey2String(bj.StartKey), wrapKey2String(bj.EndKey), bj.StartTS, bj.FinishTS, bj.RowCount, wrapKey2String(mateByte))) + jobs += fmt.Sprintf("job:%#v; ", bj.AbbrStr()) } - return sql, nil + logutil.BgLogger().Warn("insert ***************************** " + fmt.Sprintf("sql:%v", jobs)) + return sqlBuilder.String(), nil } // AddBackfillHistoryJob adds the backfill jobs to the tidb_ddl_backfill_history table. @@ -737,7 +726,6 @@ func AddBackfillJobs(sess *session, backfillJobs []*BackfillJob) error { return err } _, err = sess.execute(context.Background(), sql, label) - logutil.BgLogger().Warn("insert ***************************** " + fmt.Sprintf("sql:%v, sess:%#v", sql, sess)) return errors.Trace(err) }) } @@ -860,8 +848,8 @@ func GetAndMarkBackfillJobsForOneEle(sess *session, batch int, jobID int64, uuid // GetInterruptedBackfillJobsForOneEle gets the interrupted backfill jobs in the tblName table that contains only one element. func GetInterruptedBackfillJobsForOneEle(sess *session, jobID, eleID int64, eleKey []byte) ([]*BackfillJob, error) { - bJobs, err := GetBackfillJobs(sess, BackfillTable, fmt.Sprintf("ddl_job_id = %d and ele_id = %d and ele_key = '%s' and (state = %d or state = %d)", - jobID, eleID, eleKey, model.JobStateRollingback, model.JobStateCancelling), "get_interrupt_backfill_job") + bJobs, err := GetBackfillJobs(sess, BackfillTable, fmt.Sprintf("ddl_job_id = %d and ele_id = %d and ele_key = '%s' and state = %d", + jobID, eleID, eleKey, model.JobStateCancelled), "get_interrupt_backfill_job") if err != nil || len(bJobs) == 0 { return nil, err } @@ -881,9 +869,31 @@ func GetBackfillJobCount(sess *session, tblName, condition string, label string) return int(rows[0].GetInt64(0)), nil } +func GetBackfillMetas(sess *session, tblName, condition string, label string) ([]*model.BackfillMeta, error) { + rows, err := sess.execute(context.Background(), fmt.Sprintf("select backfill_meta from mysql.%s where %s", tblName, condition), label) + if err != nil { + return nil, errors.Trace(err) + } + if len(rows) == 0 { + return nil, dbterror.ErrDDLJobNotFound.FastGenByArgs(fmt.Sprintf("get wrong result cnt:%d", len(rows))) + } + + metas := make([]*model.BackfillMeta, 0, len(rows)) + for _, r := range rows { + meta := &model.BackfillMeta{} + err = meta.Decode(r.GetBytes(0)) + if err != nil { + return nil, errors.Trace(err) + } + metas = append(metas, meta) + } + + return metas, nil +} + func getUnsyncedInstanceIDs(sess *session, jobID int64, label string) ([]string, error) { - sql := fmt.Sprintf("select sum(state = %d) as tmp, exec_id from mysql.tidb_ddl_backfill_history where ddl_job_id = %d group by exec_id having tmp = 0;", - model.JobStateSynced, jobID) + sql := fmt.Sprintf("select sum((state=%d) + (state=%d)) as tmp, exec_id from mysql.tidb_ddl_backfill_history where ddl_job_id = %d group by exec_id having tmp = 0;", + model.JobStateSynced, model.JobStateCancelled, jobID) rows, err := sess.execute(context.Background(), sql, label) if err != nil { return nil, errors.Trace(err) @@ -893,7 +903,7 @@ func getUnsyncedInstanceIDs(sess *session, jobID int64, label string) ([]string, InstanceID := row.GetString(1) InstanceIDs = append(InstanceIDs, InstanceID) } - logutil.BgLogger().Info(fmt.Sprintf("get unsynced exec ID ***************************** 00 lable:%s, sql:%s, insttanceIDs:%s", sql, label, InstanceIDs)) + logutil.BgLogger().Info(fmt.Sprintf("get unfinished exec ID ***************************** 00 lable:%s, sql:%s, instanceIDs:%s", sql, label, InstanceIDs)) return InstanceIDs, nil } @@ -936,8 +946,8 @@ func GetBackfillJobs(sess *session, tblName, condition string, label string) ([] // RemoveBackfillJob removes the backfill jobs from the tidb_ddl_backfill table. // If isOneEle is true, removes all jobs with backfillJob's ddl_job_id, ele_id and ele_key. Otherwise, removes the backfillJob. func RemoveBackfillJob(sess *session, isOneEle bool, backfillJob *BackfillJob) error { - sql := fmt.Sprintf("delete from mysql.tidb_ddl_backfill where ddl_job_id = %d and ele_id = %d and ele_key = '%s'", - backfillJob.JobID, backfillJob.EleID, backfillJob.EleKey) + sql := fmt.Sprintf("delete from mysql.tidb_ddl_backfill where ddl_job_id = %d and ele_id = %d and ele_key = %s", + backfillJob.JobID, backfillJob.EleID, wrapKey2String(backfillJob.EleKey)) logutil.BgLogger().Info(fmt.Sprintf("remove ***************************** xx sql:%#v", backfillJob)) if !isOneEle { sql += fmt.Sprintf(" and id = %d", backfillJob.ID) @@ -951,9 +961,10 @@ func updateBackfillJob(sess *session, tableName string, backfillJob *BackfillJob if err != nil { return err } - sql := fmt.Sprintf("update mysql.%s set exec_id = '%s', exec_lease = '%s', state = %d, curr_key = '%s', row_count = %d, backfill_meta = '%s' where ddl_job_id = %d and ele_id = %d and ele_key = '%s' and id = %d", - tableName, backfillJob.InstanceID, backfillJob.InstanceLease, backfillJob.State, backfillJob.CurrKey, backfillJob.RowCount, mate, backfillJob.JobID, backfillJob.EleID, backfillJob.EleKey, backfillJob.ID) + sql := fmt.Sprintf("update mysql.%s set exec_id = '%s', exec_lease = '%s', state = %d, curr_key = %s, row_count = %d, backfill_meta = %s where ddl_job_id = %d and ele_id = %d and ele_key = %s and id = %d", + tableName, backfillJob.InstanceID, backfillJob.InstanceLease, backfillJob.State, wrapKey2String(backfillJob.CurrKey), backfillJob.RowCount, wrapKey2String(mate), + backfillJob.JobID, backfillJob.EleID, wrapKey2String(backfillJob.EleKey), backfillJob.ID) _, err = sess.execute(context.Background(), sql, label) - logutil.BgLogger().Warn("update ***************************** " + fmt.Sprintf("sql:%v, label:%#v, err:%v", sql, label, err)) + logutil.BgLogger().Warn("update ***************************** " + fmt.Sprintf("bfJob:%s, sql:%v, label:%#v, err:%v", backfillJob.AbbrStr(), sql, label, err)) return err } diff --git a/ddl/job_table_test.go b/ddl/job_table_test.go index 8351b9998f3c6..a901afc0934ba 100644 --- a/ddl/job_table_test.go +++ b/ddl/job_table_test.go @@ -255,6 +255,7 @@ func TestSimpleExecBackfillJobs(t *testing.T) { uuid := d.GetID() eleKey := meta.IndexElementKey instanceLease := ddl.InstanceLease + // test no backfill job bJobs, err := ddl.GetBackfillJobsForOneEle(se, 1, []int64{jobID1, jobID2}, instanceLease) require.NoError(t, err) @@ -401,9 +402,9 @@ func TestSimpleExecBackfillJobs(t *testing.T) { bJobs1[0].State = model.JobStateRollingback bJobs1[0].ID = 2 bJobs1[0].InstanceID = uuid - bJobs1[1].State = model.JobStateCancelling + bJobs1[1].State = model.JobStateCancelled bJobs1[1].ID = 3 - bJobs1[1].Meta.ErrMsg = "errMsg" + bJobs1[1].Meta.Error = dbterror.ErrCancelledDDLJob err = ddl.AddBackfillJobs(se, bJobs1) require.NoError(t, err) // ID jobID eleID state @@ -415,20 +416,24 @@ func TestSimpleExecBackfillJobs(t *testing.T) { // 0 jobID2 eleID3 JobStateNone // 1 jobID2 eleID3 JobStateNone // 2 jobID1 eleID1 JobStateRollingback - // 3 jobID1 eleID1 JobStateCancelling + // 3 jobID1 eleID1 JobStateCancelled bjob, err = ddl.GetMaxBackfillJob(se, jobID1, eleID1, eleKey) require.NoError(t, err) require.Equal(t, bJobs1[1], bjob) bJobs, err = ddl.GetInterruptedBackfillJobsForOneEle(se, jobID1, eleID1, eleKey) require.NoError(t, err) - require.Len(t, bJobs, 2) - equalBackfillJob(t, bJobs1[0], bJobs[0], types.ZeroTime) - equalBackfillJob(t, bJobs1[1], bJobs[1], types.ZeroTime) + require.Len(t, bJobs, 1) + equalBackfillJob(t, bJobs1[1], bJobs[0], types.ZeroTime) // test the BackfillJob's AbbrStr require.Equal(t, fmt.Sprintf("ID:2, JobID:1, EleID:11, Type:add index, State:rollingback, InstanceID:%s, InstanceLease:0000-00-00 00:00:00", uuid), bJobs1[0].AbbrStr()) - require.Equal(t, "ID:3, JobID:1, EleID:11, Type:add index, State:cancelling, InstanceID:, InstanceLease:0000-00-00 00:00:00", bJobs1[1].AbbrStr()) + require.Equal(t, "ID:3, JobID:1, EleID:11, Type:add index, State:cancelled, InstanceID:, InstanceLease:0000-00-00 00:00:00", bJobs1[1].AbbrStr()) require.Equal(t, "ID:0, JobID:2, EleID:33, Type:add index, State:none, InstanceID:, InstanceLease:0000-00-00 00:00:00", bJobs3[0].AbbrStr()) require.Equal(t, "ID:1, JobID:2, EleID:33, Type:add index, State:none, InstanceID:, InstanceLease:0000-00-00 00:00:00", bJobs3[1].AbbrStr()) + // test GetBackfillMetas + bfErr := ddl.GetBackfillErr(se, jobID1, eleID1, eleKey) + require.Error(t, bfErr, dbterror.ErrCancelledDDLJob) + bfErr = ddl.GetBackfillErr(se, jobID2, eleID2, eleKey) + require.NoError(t, bfErr) bJobs1[0].State = model.JobStateNone bJobs1[0].ID = 5 diff --git a/ddl/reorg.go b/ddl/reorg.go index f6ca9ec9be83e..8511ca34c14d5 100644 --- a/ddl/reorg.go +++ b/ddl/reorg.go @@ -206,7 +206,7 @@ func (w *worker) runReorgJob(rh *reorgHandler, reorgInfo *reorgInfo, tblInfo *mo if job.IsCancelling() { return dbterror.ErrCancelledDDLJob } - rc = w.newReorgCtx(reorgInfo) + rc = w.newReorgCtx(reorgInfo.Job.ID, reorgInfo.StartKey, reorgInfo.currElement, reorgInfo.Job.GetRowCount()) w.wg.Add(1) go func() { defer w.wg.Done() diff --git a/metrics/metrics.go b/metrics/metrics.go index 889f4c5996481..b5ad088a18e06 100644 --- a/metrics/metrics.go +++ b/metrics/metrics.go @@ -46,14 +46,15 @@ var ( // metrics labels. const ( - LabelSession = "session" - LabelDomain = "domain" - LabelDDLOwner = "ddl-owner" - LabelDDL = "ddl" - LabelDDLWorker = "ddl-worker" - LabelDDLSyncer = "ddl-syncer" - LabelGCWorker = "gcworker" - LabelAnalyze = "analyze" + LabelSession = "session" + LabelDomain = "domain" + LabelDDLOwner = "ddl-owner" + LabelDDL = "ddl" + LabelDDLWorker = "ddl-worker" + LabelBackfillWorker = "backfill-worker" + LabelDDLSyncer = "ddl-syncer" + LabelGCWorker = "gcworker" + LabelAnalyze = "analyze" LabelBatchRecvLoop = "batch-recv-loop" LabelBatchSendLoop = "batch-send-loop" diff --git a/parser/model/ddl.go b/parser/model/ddl.go index 1417b5390d6ef..fc866ac6fa9d9 100644 --- a/parser/model/ddl.go +++ b/parser/model/ddl.go @@ -428,10 +428,10 @@ type JobMeta struct { // BackfillMeta is meta info of the backfill job. type BackfillMeta struct { - PhysicalTableID int64 `json:"physical_table_id"` - IsUnique bool `json:"is_unique"` - EndInclude bool `json:"end_include"` - ErrMsg string `json:"err_msg"` + PhysicalTableID int64 `json:"physical_table_id"` + IsUnique bool `json:"is_unique"` + EndInclude bool `json:"end_include"` + Error *terror.Error `json:"err"` SQLMode mysql.SQLMode `json:"sql_mode"` Warnings map[errors.ErrorID]*terror.Error `json:"warnings"` From b0712f097c9212e062483e76b60afe299f1793b6 Mon Sep 17 00:00:00 2001 From: Lynn Date: Thu, 12 Jan 2023 15:32:50 +0800 Subject: [PATCH 03/25] tiny update --- ddl/backfilling.go | 10 --------- ddl/db_integration_test.go | 22 ------------------- ddl/dist_backfilling.go | 3 --- ddl/index.go | 22 ++----------------- ddl/job_table.go | 19 ++-------------- .../featuretag/distributereorg/default.go | 1 + 6 files changed, 5 insertions(+), 72 deletions(-) diff --git a/ddl/backfilling.go b/ddl/backfilling.go index dedd3c02cacef..64e7d848f8257 100644 --- a/ddl/backfilling.go +++ b/ddl/backfilling.go @@ -448,7 +448,6 @@ func (w *backfillWorker) runTask(task *reorgBackfillTask) (result *backfillResul failpoint.Inject("mockBackfillRunErr", func() { if w.GetCtx().id == 0 { result := &backfillResult{taskID: task.id, addedCount: 0, nextKey: nil, err: errors.Errorf("mock backfill error")} - logutil.BgLogger().Info("OOOOOOOOOOOOOOOOOOOOOOOOOOOOOOOOOOOOOOOOOOOOOOOOOOOOOOOOOOOOOOOOOOOO") failpoint.Return(result) } }) @@ -1215,8 +1214,6 @@ func (dc *ddlCtx) controlWritePhysicalTableRecord(sess *session, t table.Physica idxInfo := model.FindIndexInfoByID(t.Meta().Indices, reorgInfo.currElement.ID) isUnique = idxInfo.Unique } - logutil.BgLogger().Info("[ddl] control write physical table record ----------------- 00", - zap.Int64("ddlJobID", ddlJobID), zap.Reflect("maxBfJob", maxBfJob)) err = dc.splitTableToBackfillJobs(sess, reorgInfo, t, isUnique, bfWorkerType, startKey, currBackfillJobID) if err != nil { return errors.Trace(err) @@ -1245,7 +1242,6 @@ func (dc *ddlCtx) controlWritePhysicalTableRecord(sess *session, t table.Physica logutil.BgLogger().Info("[ddl] getBackfillJobWithRetry failed", zap.Int64("job ID", ddlJobID), zap.Error(err)) return errors.Trace(err) } - logutil.BgLogger().Info("[ddl] control write physical table record ---------------------------------- 11", zap.Bool("bfJobIsNil", bfJob == nil)) if bfJob == nil { backfillJobFinished = true logutil.BgLogger().Info("[ddl] finish backfill jobs", zap.Int64("job ID", ddlJobID)) @@ -1258,7 +1254,6 @@ func (dc *ddlCtx) controlWritePhysicalTableRecord(sess *session, t table.Physica logutil.BgLogger().Warn("[ddl] checkJobIsFinished failed", zap.Int64("job ID", ddlJobID), zap.Error(err)) return errors.Trace(err) } - logutil.BgLogger().Info("[ddl] control write physical table record ---------------------------------- 22", zap.Bool("isSynced", isSynced)) if isSynced { logutil.BgLogger().Info("[ddl] finish backfill jobs and put them to history", zap.Int64("job ID", ddlJobID)) return GetBackfillErr(sess, ddlJobID, currEle.ID, currEle.TypeKey) @@ -1296,13 +1291,9 @@ func GetBackfillErr(sess *session, ddlJobID, currEleID int64, currEleKey []byte) if err == nil { for _, m := range metas { if m.Error != nil { - logutil.BgLogger().Info("[ddl] control write physical table record ---------------------------------- 33", - zap.Int64("job ID", ddlJobID), zap.Error(m.Error)) return m.Error } } - logutil.BgLogger().Info("[ddl] control write physical table record ---------------------------------- 33", - zap.Int64("job ID", ddlJobID)) return nil } @@ -1330,7 +1321,6 @@ func checkAndHandleInterruptedBackfillJobs(sess *session, ddlJobID, currEleID in return nil } - logutil.BgLogger().Info(fmt.Sprintf("finish ***************************** xx sql:%#v", bJobs[0])) for i := 0; i < retrySQLTimes; i++ { err = MoveBackfillJobsToHistoryTable(sess, bJobs[0]) if err == nil { diff --git a/ddl/db_integration_test.go b/ddl/db_integration_test.go index d7055cfa23cd9..f3c805efd5e4b 100644 --- a/ddl/db_integration_test.go +++ b/ddl/db_integration_test.go @@ -4314,25 +4314,3 @@ func TestRegexpFunctionsGeneratedColumn(t *testing.T) { tk.MustExec("drop table if exists reg_like") } - -func TestAddIndexX(t *testing.T) { - // TODO: store := testkit.CreateMockStore(t, mockstore.WithDDLChecker()) - store := testkit.CreateMockStore(t) - - tk := testkit.NewTestKit(t, store) - tk.MustExec("use test") - - tk.MustExec("create table test_add_index(a int, b int not null default '0')") - tk.MustExec("insert into test_add_index values(1, 1),(20,20),(300,300),(4000,4000),(50000,50000),(123456,123456),(1234567,1234567),(12345678,12345678)") - tk.MustExec("split table test_add_index BETWEEN (0) AND (10000000) REGIONS 7;") - fmt.Println("****************************************************************************** 00") - tk.MustExec("alter table test_add_index add index idx(b)") - tk.MustExec("admin check table test_add_index") - tk.MustQuery("select count(1) from mysql.tidb_ddl_backfill").Check(testkit.Rows("0")) - tk.MustQuery("select id, ele_id, ele_key, type, state from mysql.tidb_ddl_backfill_history").Check(testkit.Rows("1 1 _idx_ 0 6")) - fmt.Println("****************************************************************************** 11") - tk.MustExec("alter table test_add_index add unique index idx1(b)") - tk.MustExec("admin check table test_add_index") - tk.MustQuery("select count(1) from mysql.tidb_ddl_backfill").Check(testkit.Rows("0")) - tk.MustQuery("select id, ele_id, ele_key, type, state from mysql.tidb_ddl_backfill_history").Check(testkit.Rows("1 1 _idx_ 0 6", "1 2 _idx_ 0 6")) -} diff --git a/ddl/dist_backfilling.go b/ddl/dist_backfilling.go index f94371e0a1ff6..10c3ef899acdf 100644 --- a/ddl/dist_backfilling.go +++ b/ddl/dist_backfilling.go @@ -104,8 +104,6 @@ func (bwm *backfilWorkerManager) waitFinalResult(resultCh <-chan *backfillResult for { select { case result, ok := <-resultCh: - logutil.BgLogger().Warn("=========================== handle backfill task failed", - zap.Bool("ok", ok)) if !ok { return } @@ -116,7 +114,6 @@ func (bwm *backfilWorkerManager) waitFinalResult(resultCh <-chan *backfillResult return } case <-bwm.exitCh: - logutil.BgLogger().Warn("=========================== exit xxx") return } } diff --git a/ddl/index.go b/ddl/index.go index 256ec6cd0c4bd..765a04febf296 100644 --- a/ddl/index.go +++ b/ddl/index.go @@ -695,8 +695,6 @@ func (w *worker) onCreateIndex(d *ddlCtx, t *meta.Meta, job *model.Job, isPK boo // Finish this job. job.FinishTableJob(model.JobStateDone, model.StatePublic, ver, tblInfo) details := collectTrace(job.ID) - logutil.BgLogger().Info("[ddl] &&&&&&&&&&&&&&&&&&&&&&&&&&&&&&&&-------------------------- finish add index job", zap.String("job", job.String()), - zap.String("time details", details)) if job.ReorgMeta.ReorgTp == model.ReorgTypeLitMerge { ingest.LitBackCtxMgr.Unregister(job.ID) } @@ -905,10 +903,8 @@ func doReorgWorkForCreateIndexWithDistReorg(w *worker, d *ddlCtx, t *meta.Meta, switch bfProcess { case model.ReorgTypeLitMerge: done, ver, err = runReorgJobAndHandleErr(w, d, t, job, tbl, indexInfo, false) - logutil.BgLogger().Warn("============================================ finish LitMerge", - zap.Bool("done", done), zap.Error(err)) if err != nil { - err = tryFallbackToTxnMerge(job, err) + logutil.BgLogger().Warn("[ddl] dist lightning import error", zap.Error(err)) return false, ver, errors.Trace(err) } if !done { @@ -916,8 +912,6 @@ func doReorgWorkForCreateIndexWithDistReorg(w *worker, d *ddlCtx, t *meta.Meta, } case model.ReorgTypeTxnMerge: done, ver, err = runReorgJobAndHandleErr(w, d, t, job, tbl, indexInfo, false) - logutil.BgLogger().Warn("============================================ finish TxnMerge", - zap.Bool("done", done), zap.Error(err)) if err != nil || !done { return false, ver, errors.Trace(err) } @@ -968,7 +962,6 @@ func runReorgJobAndHandleErr(w *worker, d *ddlCtx, t *meta.Meta, job *model.Job, }, false) return w.addTableIndex(tbl, reorgInfo) }) - logutil.BgLogger().Warn("============================================ run reorg job", zap.Error(err)) if err != nil { if dbterror.ErrWaitReorgTimeout.Equal(err) { // if timeout, we should return, check for the owner and re-wait job done. @@ -1688,7 +1681,6 @@ func (w *addIndexWorker) BackfillDataInTxn(handleRange reorgBackfillTask) (taskC } taskCtx.nextKey = nextKey taskCtx.done = taskDone - logutil.BgLogger().Info("OOOOOOOOOOOOOOOOOOOOOOOOOOOOOOOOOOOOOOOOOOOOOOOOOOOOOOOOOOOOOOOOOOOO", zap.Bool("done", taskDone)) err = w.batchCheckUniqueKey(txn, idxRecords) if err != nil { @@ -2047,33 +2039,23 @@ func runBackfillJobsWithLightning(d *ddl, sess *session, bfJob *BackfillJob, job return errors.New(ingest.LitErrGetBackendFail) } var err error - logutil.BgLogger().Warn(fmt.Sprintf("------------------------------------ bf:%#v", bfJob)) bc, err = ingest.LitBackCtxMgr.Register(d.ctx, bfJob.Meta.IsUnique, bfJob.JobID, bfJob.Meta.SQLMode) if err != nil { - // TODO: tryFallbackToTxnMerge logutil.BgLogger().Warn("[ddl] lightning register error", zap.Error(err)) return err } - logutil.BgLogger().Warn("00 ******** load backfill job and run reorg jobs start", zap.Int64("job id", bfJob.JobID)) tbl, err := runBackfillJobs(d, sess, bfJob, jobCtx) if err != nil { logutil.BgLogger().Warn("[ddl] runBackfillJobs error", zap.Error(err)) ingest.LitBackCtxMgr.Unregister(bfJob.JobID) - // TODO: tryFallbackToTxnMerge return err } finish := injectSpan(bfJob.JobID, "finish-import") err = bc.FinishImport(bfJob.EleID, bfJob.Meta.IsUnique, tbl) if err != nil { - if kv.ErrKeyExists.Equal(err) { - logutil.BgLogger().Warn("[ddl] import index duplicate key, convert job to rollback", zap.String("job", bfJob.AbbrStr()), zap.Error(err)) - // TODO: convertAddIdxJob2RollbackJob - } else { - logutil.BgLogger().Warn("[ddl] lightning import error", zap.Error(err)) - // TODO: tryFallbackToTxnMerge - } + logutil.BgLogger().Warn("[ddl] lightning import error", zap.String("first backfill job", bfJob.AbbrStr()), zap.Error(err)) ingest.LitBackCtxMgr.Unregister(bfJob.JobID) return err } diff --git a/ddl/job_table.go b/ddl/job_table.go index 2e9082cf75471..f819064be9cf9 100644 --- a/ddl/job_table.go +++ b/ddl/job_table.go @@ -323,8 +323,6 @@ func (d *ddl) startDispatchBackfillJobsLoop() { d.backfillCtx.jobCtxMap = make(map[int64]*JobContext) d.backfillCtx.backfillCtxMap = make(map[int64]struct{}) - logutil.BgLogger().Warn("------------------------------- start backfill jobs loop") - var notifyBackfillJobByEtcdCh clientv3.WatchChan if d.etcdCli != nil { notifyBackfillJobByEtcdCh = d.etcdCli.Watch(d.ctx, addingBackfillJob) @@ -386,7 +384,6 @@ func (d *ddl) loadBackfillJobAndRun() { runningJobIDs := d.backfillCtxJobIDs() if len(runningJobIDs) >= reorgWorkerCnt { - logutil.BgLogger().Warn("00 ******** load backfill job and run reorg jos is more than limit", zap.Int("limit", reorgWorkerCnt)) d.backfillCtx.Unlock() return } @@ -415,10 +412,7 @@ func (d *ddl) loadBackfillJobAndRun() { defer func() { d.removeBackfillCtxJobCtx(bJob.JobID) tidbutil.Recover(metrics.LabelBackfillWorker, fmt.Sprintf("runBackfillJobs"), nil, false) - logutil.BgLogger().Warn("00 ******** load backfill job and run reorg jobs finished", zap.Int64("job id", bJob.JobID)) }() - traceID := bJob.ID + 100 - initializeTrace(traceID) if bJob.Meta.ReorgTp == model.ReorgTypeLitMerge { if !ingest.LitInitialized { @@ -439,9 +433,6 @@ func (d *ddl) loadBackfillJobAndRun() { if err != nil { logutil.BgLogger().Warn("[ddl] run backfill job failed", zap.Error(err)) } - details := collectTrace(bJob.JobID) - logutil.BgLogger().Info("[ddl] &&&&&&&&&&&&&&&&&&&&&&&&&&&&&&&&-------------------------- finish backfill jobs", - zap.Int64("job ID", bJob.JobID), zap.String("time details", details)) }) } @@ -667,7 +658,6 @@ func getJobsBySQL(sess *session, tbl, condition string) ([]*model.Job, error) { func syncBackfillHistoryJobs(sess *session, uuid string, backfillJob *BackfillJob) error { sql := fmt.Sprintf("update mysql.%s set state = %d where ddl_job_id = %d and ele_id = %d and ele_key = %s and exec_id = '%s' limit 1;", BackfillHistoryTable, model.JobStateSynced, backfillJob.JobID, backfillJob.EleID, wrapKey2String(backfillJob.EleKey), uuid) - logutil.BgLogger().Warn("update ***************************** " + fmt.Sprintf("sql:%v, sess:%#v", sql, sess)) _, err := sess.execute(context.Background(), sql, "sync_backfill_history_job") return err } @@ -692,7 +682,6 @@ func generateInsertBackfillJobSQL(tableName string, backfillJobs []*BackfillJob) wrapKey2String(bj.StartKey), wrapKey2String(bj.EndKey), bj.StartTS, bj.FinishTS, bj.RowCount, wrapKey2String(mateByte))) jobs += fmt.Sprintf("job:%#v; ", bj.AbbrStr()) } - logutil.BgLogger().Warn("insert ***************************** " + fmt.Sprintf("sql:%v", jobs)) return sqlBuilder.String(), nil } @@ -903,7 +892,6 @@ func getUnsyncedInstanceIDs(sess *session, jobID int64, label string) ([]string, InstanceID := row.GetString(1) InstanceIDs = append(InstanceIDs, InstanceID) } - logutil.BgLogger().Info(fmt.Sprintf("get unfinished exec ID ***************************** 00 lable:%s, sql:%s, instanceIDs:%s", sql, label, InstanceIDs)) return InstanceIDs, nil } @@ -939,7 +927,6 @@ func GetBackfillJobs(sess *session, tblName, condition string, label string) ([] } bJobs = append(bJobs, &bfJob) } - logutil.BgLogger().Warn("get ***************************** " + fmt.Sprintf("sql:%v, label:%#v, bJobs:%v", condition, label, bJobs)) return bJobs, nil } @@ -948,7 +935,6 @@ func GetBackfillJobs(sess *session, tblName, condition string, label string) ([] func RemoveBackfillJob(sess *session, isOneEle bool, backfillJob *BackfillJob) error { sql := fmt.Sprintf("delete from mysql.tidb_ddl_backfill where ddl_job_id = %d and ele_id = %d and ele_key = %s", backfillJob.JobID, backfillJob.EleID, wrapKey2String(backfillJob.EleKey)) - logutil.BgLogger().Info(fmt.Sprintf("remove ***************************** xx sql:%#v", backfillJob)) if !isOneEle { sql += fmt.Sprintf(" and id = %d", backfillJob.ID) } @@ -962,9 +948,8 @@ func updateBackfillJob(sess *session, tableName string, backfillJob *BackfillJob return err } sql := fmt.Sprintf("update mysql.%s set exec_id = '%s', exec_lease = '%s', state = %d, curr_key = %s, row_count = %d, backfill_meta = %s where ddl_job_id = %d and ele_id = %d and ele_key = %s and id = %d", - tableName, backfillJob.InstanceID, backfillJob.InstanceLease, backfillJob.State, wrapKey2String(backfillJob.CurrKey), backfillJob.RowCount, wrapKey2String(mate), - backfillJob.JobID, backfillJob.EleID, wrapKey2String(backfillJob.EleKey), backfillJob.ID) + tableName, backfillJob.InstanceID, backfillJob.InstanceLease, backfillJob.State, wrapKey2String(backfillJob.CurrKey), backfillJob.RowCount, + wrapKey2String(mate), backfillJob.JobID, backfillJob.EleID, wrapKey2String(backfillJob.EleKey), backfillJob.ID) _, err = sess.execute(context.Background(), sql, label) - logutil.BgLogger().Warn("update ***************************** " + fmt.Sprintf("bfJob:%s, sql:%v, label:%#v, err:%v", backfillJob.AbbrStr(), sql, label, err)) return err } diff --git a/sessionctx/variable/featuretag/distributereorg/default.go b/sessionctx/variable/featuretag/distributereorg/default.go index 9f9c81967ba15..b5d2f59b26a1b 100644 --- a/sessionctx/variable/featuretag/distributereorg/default.go +++ b/sessionctx/variable/featuretag/distributereorg/default.go @@ -17,4 +17,5 @@ package distributereorg // TiDBEnableDistributeReorg is a feature tag +// TODO: Before merge this PR, change this value to false. const TiDBEnableDistributeReorg bool = true From c64b193115f472384e85d316106da0ba75db1c57 Mon Sep 17 00:00:00 2001 From: Lynn Date: Thu, 12 Jan 2023 17:16:50 +0800 Subject: [PATCH 04/25] ddl, parser: remove usless file and tiny update --- DEPS.bzl | 42 +++++++++- ddl/BUILD.bazel | 4 + ddl/backfilling.go | 4 +- ddl/ddl.go | 6 +- ddl/ddl_worker.go | 2 +- ddl/ddl_worker_util.go | 156 ------------------------------------ ddl/dist_backfilling.go | 1 + ddl/index.go | 4 - ddl/job_table.go | 3 +- go.mod | 1 - go.sum | 7 -- parser/model/BUILD.bazel | 1 + parser/model/ddl_test.go | 3 +- util/gpool/spmc/BUILD.bazel | 1 - util/gpool/spmc/spmcpool.go | 10 +-- 15 files changed, 60 insertions(+), 185 deletions(-) delete mode 100644 ddl/ddl_worker_util.go diff --git a/DEPS.bzl b/DEPS.bzl index 442749f514a5e..153f2326a5d01 100644 --- a/DEPS.bzl +++ b/DEPS.bzl @@ -2812,8 +2812,8 @@ def go_deps(): name = "com_github_opentracing_basictracer_go", build_file_proto_mode = "disable_global", importpath = "github.com/opentracing/basictracer-go", - sum = "h1:YyUAhaEfjoWXclZVJ9sGoNct7j4TVk7lZWlQw5UXuoo=", - version = "v1.0.0", + sum = "h1:Oa1fTSBvAl8pa3U+IJYqrKm0NALwH9OsgwOqDv4xJW0=", + version = "v1.1.0", ) go_repository( name = "com_github_opentracing_contrib_go_stdlib", @@ -2917,6 +2917,14 @@ def go_deps(): sum = "h1:JhzVVoYvbOACxoUmOs6V/G4D5nPVUW73rKvXxP4XUJc=", version = "v0.0.0-20180830031419-95f893ade6f2", ) + go_repository( + name = "com_github_philhofer_fwd", + build_file_proto_mode = "disable", + importpath = "github.com/philhofer/fwd", + sum = "h1:GdGcTjf5RNAxwS4QLsiMzJYj5KEvPJD3Abr261yRQXQ=", + version = "v1.1.1", + ) + go_repository( name = "com_github_pierrec_lz4", build_file_proto_mode = "disable_global", @@ -3360,6 +3368,14 @@ def go_deps(): sum = "h1:ug7PpSOB5RBPK1Kg6qskGBoP3Vnj/aNYFTznWvlkGo0=", version = "v0.0.0-20181202132449-6a9ea43bcacd", ) + go_repository( + name = "com_github_silentred_gid", + build_file_proto_mode = "disable", + importpath = "github.com/silentred/gid", + sum = "h1:JdsH8McqPUeY8IN4C0gxENnJG2zysvh+/xDJWhPvGVQ=", + version = "v1.0.0", + ) + go_repository( name = "com_github_sirupsen_logrus", build_file_proto_mode = "disable_global", @@ -3585,6 +3601,14 @@ def go_deps(): sum = "h1:RI6bs9TDIIJ96N0lR5uZoGO8QNot4qS/1l+Mobx0InM=", version = "v2.0.5-0.20230110071533-f313ddf58d73", ) + go_repository( + name = "com_github_tikv_minitrace_go", + build_file_proto_mode = "disable", + importpath = "github.com/tikv/minitrace-go", + sum = "h1:nvIrUVo5YJZMsCn6yTxrpgrokIo/wug5N/nL5mc7v50=", + version = "v0.0.0-20220923091513-8e6316bb4097", + ) + go_repository( name = "com_github_tikv_pd_client", build_file_proto_mode = "disable_global", @@ -3614,6 +3638,13 @@ def go_deps(): sum = "h1:phZCcypL/vtx6cGxObJgWZ5wexZF5SXFPLOM+ru0e/M=", version = "v0.1.0", ) + go_repository( + name = "com_github_tinylib_msgp", + build_file_proto_mode = "disable", + importpath = "github.com/tinylib/msgp", + sum = "h1:2gXmtWueD2HefZHQe1QOy9HVzmFrLOVvsXwXBQ0ayy0=", + version = "v1.1.5", + ) go_repository( name = "com_github_tklauser_go_sysconf", @@ -3650,6 +3681,13 @@ def go_deps(): sum = "h1:NowYhSdyE/1zwK9QCLeRb6USWdoif80Ie+v+yU8u1Zw=", version = "v2.5.1", ) + go_repository( + name = "com_github_ttacon_chalk", + build_file_proto_mode = "disable", + importpath = "github.com/ttacon/chalk", + sum = "h1:OXcKh35JaYsGMRzpvFkLv/MEyPuL49CThT1pZ8aSml4=", + version = "v0.0.0-20160626202418-22c06c80ed31", + ) go_repository( name = "com_github_twmb_murmur3", diff --git a/ddl/BUILD.bazel b/ddl/BUILD.bazel index a382cbea6bc79..ec9cf72e6b740 100644 --- a/ddl/BUILD.bazel +++ b/ddl/BUILD.bazel @@ -24,6 +24,7 @@ go_library( "ddl_workerpool.go", "delete_range.go", "delete_range_util.go", + "dist_backfilling.go", "foreign_key.go", "generated_column.go", "index.go", @@ -80,6 +81,8 @@ go_library( "//parser/opcode", "//parser/terror", "//parser/types", + "//resourcemanager/pooltask", + "//resourcemanager/util", "//sessionctx", "//sessionctx/binloginfo", "//sessionctx/stmtctx", @@ -105,6 +108,7 @@ go_library( "//util/filter", "//util/gcutil", "//util/generic", + "//util/gpool/spmc", "//util/hack", "//util/logutil", "//util/mathutil", diff --git a/ddl/backfilling.go b/ddl/backfilling.go index c04559685f2c2..c8bedf85dc5cd 100644 --- a/ddl/backfilling.go +++ b/ddl/backfilling.go @@ -462,8 +462,6 @@ func (w *backfillWorker) runTask(task *reorgBackfillTask) (result *backfillResul // Change the batch size dynamically. w.GetCtx().batchCnt = int(variable.GetDDLReorgBatchSize()) result = w.handleBackfillTask(w.GetCtx().ddlCtx, task, w.backfiller) - finish := injectSpan(task.bfJob.JobID+100, fmt.Sprintf("handle-job-%d-task-%d", task.bfJob.JobID, task.bfJob.ID)) - defer finish() task.bfJob.RowCount = int64(result.addedCount) if result.err != nil { logutil.BgLogger().Warn("[ddl] backfill worker runTask failed", @@ -1186,7 +1184,6 @@ func (dc *ddlCtx) splitTableToBackfillJobs(sess *session, reorgInfo *reorgInfo, } func (dc *ddlCtx) controlWritePhysicalTableRecord(sess *session, t table.PhysicalTable, bfWorkerType backfillerType, reorgInfo *reorgInfo) error { - defer injectSpan(reorgInfo.Job.ID, "control-write-records")() startKey, endKey := reorgInfo.StartKey, reorgInfo.EndKey if startKey == nil && endKey == nil { return nil @@ -1284,6 +1281,7 @@ func checkJobIsFinished(sess *session, ddlJobID int64) (bool, error) { return false, errors.Trace(err) } +// GetBackfillErr gets the error in backfill job. func GetBackfillErr(sess *session, ddlJobID, currEleID int64, currEleKey []byte) error { var err error var metas []*model.BackfillMeta diff --git a/ddl/ddl.go b/ddl/ddl.go index eee5da7b335f4..e23b4d4921e56 100644 --- a/ddl/ddl.go +++ b/ddl/ddl.go @@ -973,15 +973,15 @@ func getJobCheckInterval(job *model.Job, i int) (time.Duration, bool) { } } -func (d *ddlCtx) asyncNotifyWorker(ch chan struct{}, etcdPath string, jobID int64, jobType string) { +func (dc *ddlCtx) asyncNotifyWorker(ch chan struct{}, etcdPath string, jobID int64, jobType string) { // If the workers don't run, we needn't notify workers. if !config.GetGlobalConfig().Instance.TiDBEnableDDL.Load() { return } - if d.isOwner() { + if dc.isOwner() { asyncNotify(ch) } else { - d.asyncNotifyByEtcd(etcdPath, jobID, jobType) + dc.asyncNotifyByEtcd(etcdPath, jobID, jobType) } } diff --git a/ddl/ddl_worker.go b/ddl/ddl_worker.go index 5764b2cc8822c..4ba5a6c636640 100644 --- a/ddl/ddl_worker.go +++ b/ddl/ddl_worker.go @@ -58,7 +58,7 @@ var ( WaitTimeWhenErrorOccurred = int64(1 * time.Second) mockDDLErrOnce = int64(0) - // TestNotifyBeginTxnCh is used for if the txn is begining in runInTxn. + // TestNotifyBeginTxnCh is used for if the txn is beginning in runInTxn. TestNotifyBeginTxnCh = make(chan struct{}) ) diff --git a/ddl/ddl_worker_util.go b/ddl/ddl_worker_util.go deleted file mode 100644 index 07cb6aec69b66..0000000000000 --- a/ddl/ddl_worker_util.go +++ /dev/null @@ -1,156 +0,0 @@ -// 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" - "context" - "fmt" - "math" - "strings" - "time" - - "github.com/pingcap/tidb/util/generic" - "github.com/pingcap/tidb/util/logutil" - minitrace "github.com/tikv/minitrace-go" - "github.com/tikv/minitrace-go/jaeger" - "go.uber.org/zap" - "golang.org/x/exp/slices" -) - -var timeDetails = generic.NewSyncMap[int64, *spanCtx](10) - -type spanCtx struct { - ctx context.Context - root minitrace.TraceHandle -} - -func injectSpan(jobID int64, event string) func() { - if sctx, ok := timeDetails.Load(jobID); ok { - hd := minitrace.StartSpan(sctx.ctx, event) - return func() { - hd.Finish() - } - } - return func() {} -} - -func initializeTrace(jobID int64) { - ctx, root := minitrace.StartRootSpan(context.Background(), - "add-index-worker", uint64(jobID), 0, nil) - timeDetails.Store(jobID, &spanCtx{ - ctx: ctx, - root: root, - }) -} - -func collectTrace(jobID int64) string { - if sctx, ok := timeDetails.Load(jobID); ok { - rootTrace, _ := sctx.root.Collect() - analyzed := analyzeTrace(rootTrace) - if len(rootTrace.Spans) < 1000 { - reportTrace(rootTrace) - } - timeDetails.Delete(jobID) - return analyzed - } - return "" -} - -const batchSize = 512 - -func reportTrace(rootTrace minitrace.Trace) { - buf := bytes.NewBuffer(make([]uint8, 0, 4096)) - for _, subTrace := range splitTraces(rootTrace) { - buf.Reset() - trace := jaeger.MiniSpansToJaegerTrace("add-index", subTrace) - err := jaeger.ThriftCompactEncode(buf, trace) - if err != nil { - logutil.BgLogger().Warn("cannot collectTrace", zap.Error(err)) - return - } - err = jaeger.Send(buf.Bytes(), "127.0.0.1:6831") - if err != nil { - logutil.BgLogger().Warn("cannot collectTrace", zap.Error(err)) - return - } - } -} - -func splitTraces(trace minitrace.Trace) []minitrace.Trace { - var traces []minitrace.Trace - for len(trace.Spans) > batchSize { - traces = append(traces, minitrace.Trace{ - TraceID: trace.TraceID, - Spans: trace.Spans[:batchSize], - }) - trace.Spans = trace.Spans[batchSize:] - } - traces = append(traces, minitrace.Trace{ - TraceID: trace.TraceID, - Spans: trace.Spans, - }) - return traces -} - -func analyzeTrace(trace minitrace.Trace) string { - groupByEvent := make(map[string][]*minitrace.Span, 16) - for i, span := range trace.Spans { - spans := groupByEvent[span.Event] - if len(spans) == 0 { - groupByEvent[span.Event] = []*minitrace.Span{&trace.Spans[i]} - } else { - groupByEvent[span.Event] = append(spans, &trace.Spans[i]) - } - } - orderedEvents := make([]string, 0, len(groupByEvent)) - for event := range groupByEvent { - orderedEvents = append(orderedEvents, event) - } - slices.Sort(orderedEvents) - var sb strings.Builder - sb.WriteString("{") - for i := 0; i < len(orderedEvents); i++ { - spans := groupByEvent[orderedEvents[i]] - sum := uint64(0) - min := uint64(math.MaxUint64) - max := uint64(0) - for _, span := range spans { - dur := span.DurationNs - sum += dur - if dur < min { - min = dur - } - if dur > max { - max = dur - } - } - avg := sum / uint64(len(spans)) - sb.WriteString(orderedEvents[i]) - sb.WriteString(":") - if len(spans) < 20 { - sb.WriteString(fmt.Sprintf("%f", time.Duration(sum).Seconds())) - } else { - sb.WriteString(fmt.Sprintf(`{sum: %f, min: %f, max: %f, avg: %f}`, - time.Duration(sum).Seconds(), time.Duration(min).Seconds(), - time.Duration(max).Seconds(), time.Duration(avg).Seconds())) - } - if i != len(orderedEvents)-1 { - sb.WriteString(", ") - } - } - sb.WriteString("}") - return sb.String() -} diff --git a/ddl/dist_backfilling.go b/ddl/dist_backfilling.go index 10c3ef899acdf..c8f5cc284c5fd 100644 --- a/ddl/dist_backfilling.go +++ b/ddl/dist_backfilling.go @@ -166,6 +166,7 @@ func (dc *ddlCtx) backfillJob2Task(t table.Table, bfJob *BackfillJob) (*reorgBac priority: bfJob.Meta.Priority}, nil } +// GetTasks gets the backfill tasks associated with the non-runningJobID. func GetTasks(d *ddlCtx, sess *session, tbl table.Table, runningJobID int64, concurrency int) ([]*reorgBackfillTask, error) { // TODO: At present, only add index is processed. In the future, different elements need to be distinguished. var err error diff --git a/ddl/index.go b/ddl/index.go index fca63c908b819..ef761c3be76e8 100644 --- a/ddl/index.go +++ b/ddl/index.go @@ -610,7 +610,6 @@ func (w *worker) onCreateIndex(d *ddlCtx, t *meta.Meta, job *model.Job, isPK boo return ver, err } logutil.BgLogger().Info("[ddl] run add index job", zap.String("job", job.String()), zap.Reflect("indexInfo", indexInfo)) - initializeTrace(job.ID) } originalState := indexInfo.State switch indexInfo.State { @@ -656,7 +655,6 @@ func (w *worker) onCreateIndex(d *ddlCtx, t *meta.Meta, job *model.Job, isPK boo job.SnapshotVer = 0 job.SchemaState = model.StateWriteReorganization case model.StateWriteReorganization: - defer injectSpan(job.ID, "write-reorg")() // reorganization -> public tbl, err := getTable(d.store, schemaID, tblInfo) if err != nil { @@ -2059,7 +2057,6 @@ func runBackfillJobsWithLightning(d *ddl, sess *session, bfJob *BackfillJob, job return err } - finish := injectSpan(bfJob.JobID, "finish-import") err = bc.FinishImport(bfJob.EleID, bfJob.Meta.IsUnique, tbl) if err != nil { logutil.BgLogger().Warn("[ddl] lightning import error", zap.String("first backfill job", bfJob.AbbrStr()), zap.Error(err)) @@ -2067,7 +2064,6 @@ func runBackfillJobsWithLightning(d *ddl, sess *session, bfJob *BackfillJob, job return err } ingest.LitBackCtxMgr.Unregister(bfJob.ID) - finish() bc.SetDone() return nil } diff --git a/ddl/job_table.go b/ddl/job_table.go index c13aa52a7b144..e973324caf195 100644 --- a/ddl/job_table.go +++ b/ddl/job_table.go @@ -411,7 +411,7 @@ func (d *ddl) loadBackfillJobAndRun() { d.wg.Run(func() { defer func() { d.removeBackfillCtxJobCtx(bJob.JobID) - tidbutil.Recover(metrics.LabelBackfillWorker, fmt.Sprintf("runBackfillJobs"), nil, false) + tidbutil.Recover(metrics.LabelBackfillWorker, "runBackfillJobs", nil, false) }() if bJob.Meta.ReorgTp == model.ReorgTypeLitMerge { @@ -847,6 +847,7 @@ func GetBackfillJobCount(sess *session, tblName, condition string, label string) return int(rows[0].GetInt64(0)), nil } +// GetBackfillMetas gets the backfill metas in the tblName table according to condition. func GetBackfillMetas(sess *session, tblName, condition string, label string) ([]*model.BackfillMeta, error) { rows, err := sess.execute(context.Background(), fmt.Sprintf("select backfill_meta from mysql.%s where %s", tblName, condition), label) if err != nil { diff --git a/go.mod b/go.mod index 28e5be51ea9cb..16d9caa5374b5 100644 --- a/go.mod +++ b/go.mod @@ -91,7 +91,6 @@ require ( github.com/tdakkota/asciicheck v0.1.1 github.com/tiancaiamao/appdash v0.0.0-20181126055449-889f96f722a2 github.com/tikv/client-go/v2 v2.0.5-0.20230110071533-f313ddf58d73 - github.com/tikv/minitrace-go v0.0.0-20220923091513-8e6316bb4097 github.com/tikv/pd/client v0.0.0-20221031025758-80f0d8ca4d07 github.com/timakin/bodyclose v0.0.0-20210704033933-f49887972144 github.com/twmb/murmur3 v1.1.3 diff --git a/go.sum b/go.sum index 4a8ac463e71b4..2d040266085c6 100644 --- a/go.sum +++ b/go.sum @@ -761,7 +761,6 @@ github.com/petermattis/goid v0.0.0-20170504144140-0ded85884ba5 h1:rUMC+oZ89Om6l9 github.com/petermattis/goid v0.0.0-20170504144140-0ded85884ba5/go.mod h1:jvVRKCrJTQWu0XVbaOlby/2lO20uSCHEMzzplHXte1o= github.com/phayes/freeport v0.0.0-20180830031419-95f893ade6f2 h1:JhzVVoYvbOACxoUmOs6V/G4D5nPVUW73rKvXxP4XUJc= github.com/phayes/freeport v0.0.0-20180830031419-95f893ade6f2/go.mod h1:iIss55rKnNBTvrwdmkUpLnDpZoAHvWaiq5+iMmen4AE= -github.com/philhofer/fwd v1.1.1/go.mod h1:gk3iGcWd9+svBvR0sR+KPcfE+RNWozjowpeBVG3ZVNU= github.com/pierrec/lz4 v2.6.0+incompatible/go.mod h1:pdkljMzZIN41W+lC3N2tnIh5sFi+IEE17M5jbnwPHcY= github.com/pierrec/lz4 v2.6.1+incompatible h1:9UY3+iC23yxF0UfGaYrGplQ+79Rg+h/q9FV9ix19jjM= github.com/pierrec/lz4 v2.6.1+incompatible/go.mod h1:pdkljMzZIN41W+lC3N2tnIh5sFi+IEE17M5jbnwPHcY= @@ -884,7 +883,6 @@ github.com/shurcooL/sanitized_anchor_name v1.0.0/go.mod h1:1NzhyTcUVG4SuEtjjoZeV github.com/shurcooL/vfsgen v0.0.0-20180711163814-62bca832be04/go.mod h1:TrYk7fJVaAttu97ZZKrO9UbRa8izdowaMIZcxYMbVaw= github.com/shurcooL/vfsgen v0.0.0-20181202132449-6a9ea43bcacd h1:ug7PpSOB5RBPK1Kg6qskGBoP3Vnj/aNYFTznWvlkGo0= github.com/shurcooL/vfsgen v0.0.0-20181202132449-6a9ea43bcacd/go.mod h1:TrYk7fJVaAttu97ZZKrO9UbRa8izdowaMIZcxYMbVaw= -github.com/silentred/gid v1.0.0/go.mod h1:DMQPn66uY+3ed7rWfzOVET7VbDBAhjz+6AmmlixUK08= github.com/sirupsen/logrus v1.2.0/go.mod h1:LxeOpSwHxABJmUn/MG1IvRgCAasNZTLOkJPxbbu5VWo= github.com/sirupsen/logrus v1.4.2/go.mod h1:tLMulIdttU9McNUspp0xgXVQah82FyeX6MwdIuYE2rE= github.com/sirupsen/logrus v1.6.0/go.mod h1:7uNnSEd1DgxDLC74fIahvMZmmYsHGZGEOFrfsX/uA88= @@ -941,13 +939,10 @@ github.com/tiancaiamao/gp v0.0.0-20221230034425-4025bc8a4d4a h1:J/YdBZ46WKpXsxsW github.com/tiancaiamao/gp v0.0.0-20221230034425-4025bc8a4d4a/go.mod h1:h4xBhSNtOeEosLJ4P7JyKXX7Cabg7AVkWCK5gV2vOrM= github.com/tikv/client-go/v2 v2.0.5-0.20230110071533-f313ddf58d73 h1:RI6bs9TDIIJ96N0lR5uZoGO8QNot4qS/1l+Mobx0InM= github.com/tikv/client-go/v2 v2.0.5-0.20230110071533-f313ddf58d73/go.mod h1:dO/2a/xi/EO3eVv9xN5G1VFtd/hythzgTeeCbW5SWuI= -github.com/tikv/minitrace-go v0.0.0-20220923091513-8e6316bb4097 h1:nvIrUVo5YJZMsCn6yTxrpgrokIo/wug5N/nL5mc7v50= -github.com/tikv/minitrace-go v0.0.0-20220923091513-8e6316bb4097/go.mod h1:ukJr0BfYeYbO3n15LAV2Dp4jvFpIPF2g14NU227ZTLY= github.com/tikv/pd/client v0.0.0-20221031025758-80f0d8ca4d07 h1:ckPpxKcl75mO2N6a4cJXiZH43hvcHPpqc9dh1TmH1nc= github.com/tikv/pd/client v0.0.0-20221031025758-80f0d8ca4d07/go.mod h1:CipBxPfxPUME+BImx9MUYXCnAVLS3VJUr3mnSJwh40A= github.com/timakin/bodyclose v0.0.0-20210704033933-f49887972144 h1:kl4KhGNsJIbDHS9/4U9yQo1UcPQM0kOMJHn29EoH/Ro= github.com/timakin/bodyclose v0.0.0-20210704033933-f49887972144/go.mod h1:Qimiffbc6q9tBWlVV6x0P9sat/ao1xEkREYPPj9hphk= -github.com/tinylib/msgp v1.1.5/go.mod h1:eQsjooMTnV42mHu917E26IogZ2930nFyBQdofk10Udg= github.com/tklauser/go-sysconf v0.3.9/go.mod h1:11DU/5sG7UexIrp/O6g35hrWzu0JxlwQ3LSFUzyeuhs= github.com/tklauser/go-sysconf v0.3.10 h1:IJ1AZGZRWbY8T5Vfk04D9WOA5WSejdflXxP03OUqALw= github.com/tklauser/go-sysconf v0.3.10/go.mod h1:C8XykCvCb+Gn0oNCWPIlcb0RuglQTYaQ2hGm7jmxEFk= @@ -957,7 +952,6 @@ github.com/tklauser/numcpus v0.4.0/go.mod h1:1+UI3pD8NW14VMwdgJNJ1ESk2UnwhAnz5hM github.com/tmc/grpc-websocket-proxy v0.0.0-20190109142713-0ad062ec5ee5/go.mod h1:ncp9v5uamzpCO7NfCPTXjqaC+bZgJeR0sMTm6dMHP7U= github.com/tmc/grpc-websocket-proxy v0.0.0-20201229170055-e5319fda7802 h1:uruHq4dN7GR16kFc5fp3d1RIYzJW5onx8Ybykw2YQFA= github.com/tmc/grpc-websocket-proxy v0.0.0-20201229170055-e5319fda7802/go.mod h1:ncp9v5uamzpCO7NfCPTXjqaC+bZgJeR0sMTm6dMHP7U= -github.com/ttacon/chalk v0.0.0-20160626202418-22c06c80ed31/go.mod h1:onvgF043R+lC5RZ8IT9rBXDaEDnpnw/Cl+HFiw+v/7Q= github.com/twmb/murmur3 v1.1.3 h1:D83U0XYKcHRYwYIpBKf3Pks91Z0Byda/9SJ8B6EMRcA= github.com/twmb/murmur3 v1.1.3/go.mod h1:Qq/R7NUyOfr65zD+6Q5IHKsJLwP7exErjN6lyyq3OSQ= github.com/uber/jaeger-client-go v2.22.1+incompatible h1:NHcubEkVbahf9t3p75TOCR83gdUHXjRJvjoBh1yACsM= @@ -1412,7 +1406,6 @@ golang.org/x/tools v0.0.0-20200729194436-6467de6f59a7/go.mod h1:njjCfa9FT2d7l9Bc golang.org/x/tools v0.0.0-20200804011535-6c149bb5ef0d/go.mod h1:njjCfa9FT2d7l9Bc6FUM5FLjQPp3cFF28FI3qnDFljA= golang.org/x/tools v0.0.0-20200825202427-b303f430e36d/go.mod h1:njjCfa9FT2d7l9Bc6FUM5FLjQPp3cFF28FI3qnDFljA= golang.org/x/tools v0.0.0-20200904185747-39188db58858/go.mod h1:Cj7w3i3Rnn0Xh82ur9kSqwfTHTeVxaDqrfMjpcNT6bE= -golang.org/x/tools v0.0.0-20201022035929-9cf592e881e9/go.mod h1:emZCQorbCU4vsT4fOWvOPXz4eW1wZW4PmDk9uLelYpA= golang.org/x/tools v0.0.0-20201110124207-079ba7bd75cd/go.mod h1:emZCQorbCU4vsT4fOWvOPXz4eW1wZW4PmDk9uLelYpA= golang.org/x/tools v0.0.0-20201125231158-b5590deeca9b/go.mod h1:emZCQorbCU4vsT4fOWvOPXz4eW1wZW4PmDk9uLelYpA= golang.org/x/tools v0.0.0-20201201161351-ac6f37ff4c2a/go.mod h1:emZCQorbCU4vsT4fOWvOPXz4eW1wZW4PmDk9uLelYpA= diff --git a/parser/model/BUILD.bazel b/parser/model/BUILD.bazel index ccf52aa687713..08f707fef56f5 100644 --- a/parser/model/BUILD.bazel +++ b/parser/model/BUILD.bazel @@ -32,6 +32,7 @@ go_test( deps = [ "//parser/charset", "//parser/mysql", + "//parser/terror", "//parser/types", "@com_github_stretchr_testify//require", ], diff --git a/parser/model/ddl_test.go b/parser/model/ddl_test.go index d67b6ac91175a..7ea82606363ce 100644 --- a/parser/model/ddl_test.go +++ b/parser/model/ddl_test.go @@ -18,6 +18,7 @@ import ( "unsafe" "github.com/pingcap/tidb/parser/model" + "github.com/pingcap/tidb/parser/terror" "github.com/stretchr/testify/require" ) @@ -61,7 +62,7 @@ func TestBackfillMetaCodec(t *testing.T) { } bm := &model.BackfillMeta{ EndInclude: true, - ErrMsg: "has a err", + Error: terror.ErrResultUndetermined, JobMeta: jm, } bmBytes, err := bm.Encode() diff --git a/util/gpool/spmc/BUILD.bazel b/util/gpool/spmc/BUILD.bazel index 1c951a219fb20..b3fb100373a7a 100644 --- a/util/gpool/spmc/BUILD.bazel +++ b/util/gpool/spmc/BUILD.bazel @@ -11,7 +11,6 @@ go_library( importpath = "github.com/pingcap/tidb/util/gpool/spmc", visibility = ["//visibility:public"], deps = [ - "//resourcemanager", "//resourcemanager/pooltask", "//resourcemanager/util", "//util/gpool", diff --git a/util/gpool/spmc/spmcpool.go b/util/gpool/spmc/spmcpool.go index 0f81d86448828..78e0a3b214870 100644 --- a/util/gpool/spmc/spmcpool.go +++ b/util/gpool/spmc/spmcpool.go @@ -21,7 +21,6 @@ import ( "time" "github.com/pingcap/log" - "github.com/pingcap/tidb/resourcemanager" "github.com/pingcap/tidb/resourcemanager/pooltask" "github.com/pingcap/tidb/resourcemanager/util" "github.com/pingcap/tidb/util/gpool" @@ -79,10 +78,11 @@ func NewSPMCPool[T any, U any, C any, CT any, TF pooltask.Context[CT]](name stri result.capacity.Add(size) result.workers = newWorkerLoopQueue[T, U, C, CT, TF](int(size)) result.cond = sync.NewCond(result.lock) - err := resourcemanager.GlobalResourceManager.Register(result, name, component) - if err != nil { - return nil, err - } + // TODO: wait https://github.com/pingcap/tidb/pull/40547 + // err := resourcemanager.GlobalResourceManager.Register(result, name, component) + // if err != nil { + // return nil, err + // } // Start a goroutine to clean up expired workers periodically. go result.purgePeriodically() return result, nil From e813a444122b478a9d134c209ae0fd135db44bcb Mon Sep 17 00:00:00 2001 From: Lynn Date: Fri, 13 Jan 2023 11:13:16 +0800 Subject: [PATCH 05/25] util/gpool/spmc, ddl: tiny update and init sess ctx --- ddl/backfilling.go | 34 +++++++++++++++++++++------- ddl/constant.go | 4 ++-- ddl/dist_backfilling.go | 44 +++++++++++++++++++++++++------------ ddl/index.go | 2 +- ddl/job_table.go | 9 ++++++-- ddl/job_table_test.go | 34 ++++++++++++++++++---------- util/gpool/spmc/BUILD.bazel | 1 + util/gpool/spmc/spmcpool.go | 10 ++++----- 8 files changed, 94 insertions(+), 44 deletions(-) diff --git a/ddl/backfilling.go b/ddl/backfilling.go index c8bedf85dc5cd..a22ea5c48abe7 100644 --- a/ddl/backfilling.go +++ b/ddl/backfilling.go @@ -776,11 +776,16 @@ func makeupDecodeColMap(sessCtx sessionctx.Context, dbName model.CIStr, t table. return decodeColMap, nil } -func setSessCtxLocation(sctx sessionctx.Context, info *reorgInfo) error { +func setSessCtxLocation(sctx sessionctx.Context, tzLocation *model.TimeZoneLocation) error { // It is set to SystemLocation to be compatible with nil LocationInfo. - *sctx.GetSessionVars().TimeZone = *timeutil.SystemLocation() - if info.ReorgMeta.Location != nil { - loc, err := info.ReorgMeta.Location.GetLocation() + tz := *timeutil.SystemLocation() + if sctx.GetSessionVars().TimeZone == nil { + sctx.GetSessionVars().TimeZone = &tz + } else { + *sctx.GetSessionVars().TimeZone = tz + } + if tzLocation != nil { + loc, err := tzLocation.GetLocation() if err != nil { return errors.Trace(err) } @@ -829,15 +834,21 @@ func newBackfillScheduler(ctx context.Context, info *reorgInfo, sessPool *sessio func (b *backfillScheduler) newSessCtx() (sessionctx.Context, error) { reorgInfo := b.reorgInfo sessCtx := newContext(reorgInfo.d.store) + if err := initSessCtx(sessCtx, reorgInfo.ReorgMeta.SQLMode, reorgInfo.ReorgMeta.Location); err != nil { + return nil, errors.Trace(err) + } + return sessCtx, nil +} + +func initSessCtx(sessCtx sessionctx.Context, sqlMode mysql.SQLMode, tzLocation *model.TimeZoneLocation) error { 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) + if err := setSessCtxLocation(sessCtx, tzLocation); err != nil { + return errors.Trace(err) } sessCtx.GetSessionVars().StmtCtx.BadNullAsWarning = !sqlMode.HasStrictMode() sessCtx.GetSessionVars().StmtCtx.TruncateAsWarning = !sqlMode.HasStrictMode() @@ -846,7 +857,7 @@ func (b *backfillScheduler) newSessCtx() (sessionctx.Context, error) { sessCtx.GetSessionVars().StmtCtx.DividedByZeroAsWarning = !sqlMode.HasStrictMode() sessCtx.GetSessionVars().StmtCtx.IgnoreZeroInDate = !sqlMode.HasStrictMode() || sqlMode.HasAllowInvalidDatesMode() sessCtx.GetSessionVars().StmtCtx.NoZeroDate = sqlMode.HasStrictMode() - return sessCtx, nil + return nil } func (b *backfillScheduler) setMaxWorkerSize(maxSize int) { @@ -1097,6 +1108,7 @@ func addBatchBackfillJobs(sess *session, bfWorkerType backfillerType, reorgInfo if notDistTask { instanceID = reorgInfo.d.uuid } + // TODO: Adjust the number of ranges(region) for each task. for _, task := range batchTasks { bm := &model.BackfillMeta{ @@ -1194,6 +1206,12 @@ func (dc *ddlCtx) controlWritePhysicalTableRecord(sess *session, t table.Physica return errors.Trace(err) } + defaultSQLMode := sess.GetSessionVars().SQLMode + defer func() { + sess.GetSessionVars().SQLMode = defaultSQLMode + }() + // Make timestamp type can be inserted ZeroTimestamp. + sess.GetSessionVars().SQLMode = mysql.ModeNone currBackfillJobID := int64(1) err := checkAndHandleInterruptedBackfillJobs(sess, ddlJobID, reorgInfo.currElement.ID, reorgInfo.currElement.TypeKey) if err != nil { diff --git a/ddl/constant.go b/ddl/constant.go index 3fe6bf4a04ee6..f9de82e2e6dad 100644 --- a/ddl/constant.go +++ b/ddl/constant.go @@ -58,7 +58,7 @@ const ( store_id bigint, type int, exec_id blob default null, - exec_lease Time, + exec_lease timestamp, state int, curr_key blob, start_key blob, @@ -77,7 +77,7 @@ const ( store_id bigint, type int, exec_id blob default null, - exec_lease Time, + exec_lease timestamp, state int, curr_key blob, start_key blob, diff --git a/ddl/dist_backfilling.go b/ddl/dist_backfilling.go index c8f5cc284c5fd..b85a53340b559 100644 --- a/ddl/dist_backfilling.go +++ b/ddl/dist_backfilling.go @@ -40,7 +40,7 @@ type backfillWorkerContext struct { type newBackfillerFunc func(bfCtx *backfillCtx) (bf backfiller, err error) -func newBackfillWorkerContext(d *ddl, schemaName string, tbl table.Table, workerCnt int, reorgTp model.ReorgType, +func newBackfillWorkerContext(d *ddl, schemaName string, tbl table.Table, workerCnt int, bfMeta *model.BackfillMeta, bfFunc newBackfillerFunc) (*backfillWorkerContext, error) { if workerCnt <= 0 { return nil, nil @@ -51,24 +51,36 @@ func newBackfillWorkerContext(d *ddl, schemaName string, tbl table.Table, worker logutil.BgLogger().Debug("[ddl] no backfill context available now", zap.Int("backfillCtx", len(bCtxs)), zap.Error(err)) return nil, errors.Trace(err) } - seCtxs := make([]sessionctx.Context, 0, len(bCtxs)) + bwCtx := &backfillWorkerContext{backfillWorkers: bCtxs, sessCtxs: make([]sessionctx.Context, 0, len(bCtxs))} + defer func() { + if err != nil { + bwCtx.close(d) + } + }() + for i := 0; i < len(bCtxs); i++ { - se, err := d.sessPool.get() + var se sessionctx.Context + se, err = d.sessPool.get() if err != nil { logutil.BgLogger().Error("[ddl] new backfill worker context, get a session failed", zap.Error(err)) return nil, errors.Trace(err) } - sess := newSession(se) - bfCtx := newBackfillCtx(d.ddlCtx, 0, sess, reorgTp, schemaName, tbl) - bf, err := bfFunc(bfCtx) + err = initSessCtx(se, bfMeta.SQLMode, bfMeta.Location) + if err != nil { + logutil.BgLogger().Error("[ddl] new backfill worker context, init the session ctx failed", zap.Error(err)) + return nil, errors.Trace(err) + } + bfCtx := newBackfillCtx(d.ddlCtx, 0, se, bfMeta.ReorgTp, schemaName, tbl) + var bf backfiller + bf, err = bfFunc(bfCtx) if err != nil { logutil.BgLogger().Error("[ddl] new backfill worker context, do bfFunc failed", zap.Error(err)) return nil, errors.Trace(err) } - seCtxs = append(seCtxs, se) + bwCtx.sessCtxs = append(bwCtx.sessCtxs, se) bCtxs[i].backfiller = bf } - return &backfillWorkerContext{backfillWorkers: bCtxs, sessCtxs: seCtxs}, nil + return bwCtx, nil } func (bwCtx *backfillWorkerContext) GetContext() *backfillWorker { @@ -84,6 +96,15 @@ func (bwCtx *backfillWorkerContext) GetContext() *backfillWorker { return bw } +func (bwCtx *backfillWorkerContext) close(d *ddl) { + for _, s := range bwCtx.sessCtxs { + d.sessPool.put(s) + } + for _, w := range bwCtx.backfillWorkers { + d.backfillCtxPool.put(w) + } +} + type backfilWorkerManager struct { bwCtx *backfillWorkerContext wg util.WaitGroupWrapper @@ -124,12 +145,7 @@ func (bwm *backfilWorkerManager) close(d *ddl) error { close(bwm.exitCh) bwm.wg.Wait() - for _, s := range bwm.bwCtx.sessCtxs { - d.sessPool.put(s) - } - for _, w := range bwm.bwCtx.backfillWorkers { - d.backfillCtxPool.put(w) - } + bwm.bwCtx.close(d) return bwm.unsyncErr } diff --git a/ddl/index.go b/ddl/index.go index 26bf4a2917d89..1ddf2ce04d8f3 100644 --- a/ddl/index.go +++ b/ddl/index.go @@ -1387,7 +1387,7 @@ func newAddIndexWorkerContext(d *ddl, sess *session, schemaName model.CIStr, tbl bfJob *BackfillJob, jobCtx *JobContext) (*backfillWorkerContext, error) { //nolint:forcetypeassert phyTbl := tbl.(table.PhysicalTable) - return newBackfillWorkerContext(d, schemaName.O, tbl, workerCnt, bfJob.Meta.ReorgTp, + return newBackfillWorkerContext(d, schemaName.O, tbl, workerCnt, bfJob.Meta, func(bfCtx *backfillCtx) (backfiller, error) { decodeColMap, err := makeupDecodeColMap(sess, schemaName, phyTbl) if err != nil { diff --git a/ddl/job_table.go b/ddl/job_table.go index e973324caf195..1b613f3324814 100644 --- a/ddl/job_table.go +++ b/ddl/job_table.go @@ -34,6 +34,7 @@ import ( "github.com/pingcap/tidb/parser/model" "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/table" + "github.com/pingcap/tidb/types" tidbutil "github.com/pingcap/tidb/util" "github.com/pingcap/tidb/util/dbterror" "github.com/pingcap/tidb/util/gpool/spmc" @@ -693,6 +694,7 @@ func generateInsertBackfillJobSQL(tableName string, backfillJobs []*BackfillJob) if i != 0 { sqlBuilder.WriteString(", ") } + bj.InstanceLease.SetFsp(0) sqlBuilder.WriteString(fmt.Sprintf("(%d, %d, %d, %s, %d, %d, '%s', '%s', %d, %s, %s, %s, %d, %d, %d, %s)", bj.ID, bj.JobID, bj.EleID, wrapKey2String(bj.EleKey), bj.StoreID, bj.Tp, bj.InstanceID, bj.InstanceLease, bj.State, wrapKey2String(bj.CurrKey), wrapKey2String(bj.StartKey), wrapKey2String(bj.EndKey), bj.StartTS, bj.FinishTS, bj.RowCount, wrapKey2String(mateByte))) @@ -757,10 +759,11 @@ func GetBackfillJobsForOneEle(s *session, batch int, excludedJobIDs []int64, lea if err != nil { return err } + leaseStr := currTime.Add(-lease).Format(types.TimeFormat) bJobs, err = GetBackfillJobs(se, BackfillTable, fmt.Sprintf("(exec_ID = '' or exec_lease < '%v') %s order by ddl_job_id, ele_key, ele_id limit %d", - currTime.Add(-lease), eJobIDsBuilder.String(), batch), "get_backfill_job") + leaseStr, eJobIDsBuilder.String(), batch), "get_backfill_job") return err }) if err != nil || len(bJobs) == 0 { @@ -789,10 +792,11 @@ func GetAndMarkBackfillJobsForOneEle(s *session, batch int, jobID int64, uuid st if err != nil { return err } + leaseStr := currTime.Add(-lease).Format(types.TimeFormat) bJobs, err = GetBackfillJobs(se, BackfillTable, fmt.Sprintf("(exec_ID = '' or exec_lease < '%v') and ddl_job_id = %d order by ddl_job_id, ele_key, ele_id limit %d", - currTime.Add(-lease), jobID, batch), "get_mark_backfill_job") + leaseStr, jobID, batch), "get_mark_backfill_job") if err != nil { return err } @@ -937,6 +941,7 @@ func updateBackfillJob(sess *session, tableName string, backfillJob *BackfillJob if err != nil { return err } + backfillJob.InstanceLease.SetFsp(0) sql := fmt.Sprintf("update mysql.%s set exec_id = '%s', exec_lease = '%s', state = %d, curr_key = %s, row_count = %d, backfill_meta = %s where ddl_job_id = %d and ele_id = %d and ele_key = %s and id = %d", tableName, backfillJob.InstanceID, backfillJob.InstanceLease, backfillJob.State, wrapKey2String(backfillJob.CurrKey), backfillJob.RowCount, wrapKey2String(mate), backfillJob.JobID, backfillJob.EleID, wrapKey2String(backfillJob.EleKey), backfillJob.ID) diff --git a/ddl/job_table_test.go b/ddl/job_table_test.go index a901afc0934ba..d9223161a17f9 100644 --- a/ddl/job_table_test.go +++ b/ddl/job_table_test.go @@ -27,6 +27,7 @@ import ( "github.com/pingcap/tidb/ddl" "github.com/pingcap/tidb/meta" "github.com/pingcap/tidb/parser/model" + "github.com/pingcap/tidb/parser/mysql" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/sessiontxn" @@ -198,15 +199,16 @@ func makeAddIdxBackfillJobs(schemaID, tblID, jobID, eleID int64, cnt int, query }, } bj := &ddl.BackfillJob{ - ID: int64(i), - JobID: jobID, - EleID: eleID, - EleKey: meta.IndexElementKey, - State: model.JobStateNone, - CurrKey: sKey, - StartKey: sKey, - EndKey: eKey, - Meta: bm, + ID: int64(i), + JobID: jobID, + EleID: eleID, + EleKey: meta.IndexElementKey, + State: model.JobStateNone, + InstanceLease: types.ZeroTimestamp, + CurrKey: sKey, + StartKey: sKey, + EndKey: eKey, + Meta: bm, } bJobs = append(bJobs, bj) } @@ -263,8 +265,7 @@ func TestSimpleExecBackfillJobs(t *testing.T) { bJobs, err = ddl.GetAndMarkBackfillJobsForOneEle(se, 1, jobID1, uuid, instanceLease) require.EqualError(t, err, dbterror.ErrDDLJobNotFound.FastGen("get zero backfill job").Error()) require.Nil(t, bJobs) - allCnt, err := ddl.GetBackfillJobCount(se, ddl.BackfillTable, fmt.Sprintf("ddl_job_id = %d and ele_id = %d and ele_key = '%s'", - jobID1, eleID2, meta.IndexElementKey), "check_backfill_job_count") + allCnt, err := ddl.GetBackfillJobCount(se, ddl.BackfillTable, getIdxConditionStr(jobID1, eleID2), "check_backfill_job_count") require.NoError(t, err) require.Equal(t, allCnt, 0) // Test some backfill jobs, add backfill jobs to the table. @@ -277,6 +278,9 @@ func TestSimpleExecBackfillJobs(t *testing.T) { bjTestCases = append(bjTestCases, bJobs2...) bjTestCases = append(bjTestCases, bJobs3...) err = ddl.AddBackfillJobs(se, bjTestCases) + require.Equal(t, err.Error(), "[table:1292]Incorrect timestamp value: '0000-00-00 00:00:00' for column 'exec_lease' at row 1") + tk.Session().GetSessionVars().SQLMode = mysql.ModeNone + err = ddl.AddBackfillJobs(se, bjTestCases) // ID jobID eleID InstanceID // ------------------------------------- // 0 jobID1 eleID1 uuid @@ -423,12 +427,17 @@ func TestSimpleExecBackfillJobs(t *testing.T) { bJobs, err = ddl.GetInterruptedBackfillJobsForOneEle(se, jobID1, eleID1, eleKey) require.NoError(t, err) require.Len(t, bJobs, 1) - equalBackfillJob(t, bJobs1[1], bJobs[0], types.ZeroTime) + equalBackfillJob(t, bJobs1[1], bJobs[0], types.ZeroTimestamp) // test the BackfillJob's AbbrStr require.Equal(t, fmt.Sprintf("ID:2, JobID:1, EleID:11, Type:add index, State:rollingback, InstanceID:%s, InstanceLease:0000-00-00 00:00:00", uuid), bJobs1[0].AbbrStr()) require.Equal(t, "ID:3, JobID:1, EleID:11, Type:add index, State:cancelled, InstanceID:, InstanceLease:0000-00-00 00:00:00", bJobs1[1].AbbrStr()) require.Equal(t, "ID:0, JobID:2, EleID:33, Type:add index, State:none, InstanceID:, InstanceLease:0000-00-00 00:00:00", bJobs3[0].AbbrStr()) require.Equal(t, "ID:1, JobID:2, EleID:33, Type:add index, State:none, InstanceID:, InstanceLease:0000-00-00 00:00:00", bJobs3[1].AbbrStr()) + // test select tidb_ddl_backfill + tk.MustQuery(fmt.Sprintf("select exec_id, exec_lease from mysql.tidb_ddl_backfill where id = %d and %s", bJobs1[0].ID, getIdxConditionStr(jobID1, eleID1))). + Check(testkit.Rows(fmt.Sprintf("%s 0000-00-00 00:00:00", uuid))) + tk.MustQuery(fmt.Sprintf("select exec_id, exec_lease from mysql.tidb_ddl_backfill where id = %d and %s", bJobs1[1].ID, getIdxConditionStr(jobID1, eleID1))). + Check(testkit.Rows(" 0000-00-00 00:00:00")) // test GetBackfillMetas bfErr := ddl.GetBackfillErr(se, jobID1, eleID1, eleKey) require.Error(t, bfErr, dbterror.ErrCancelledDDLJob) @@ -559,6 +568,7 @@ func TestGetTasks(t *testing.T) { tk := testkit.NewTestKit(t, store) tk.MustExec("use test") se := ddl.NewSession(tk.Session()) + se.GetSessionVars().SQLMode = mysql.ModeNone d := dom.DDL() jobID1 := int64(1) diff --git a/util/gpool/spmc/BUILD.bazel b/util/gpool/spmc/BUILD.bazel index b3fb100373a7a..1c951a219fb20 100644 --- a/util/gpool/spmc/BUILD.bazel +++ b/util/gpool/spmc/BUILD.bazel @@ -11,6 +11,7 @@ go_library( importpath = "github.com/pingcap/tidb/util/gpool/spmc", visibility = ["//visibility:public"], deps = [ + "//resourcemanager", "//resourcemanager/pooltask", "//resourcemanager/util", "//util/gpool", diff --git a/util/gpool/spmc/spmcpool.go b/util/gpool/spmc/spmcpool.go index c99f6b34515bd..b8cecb289c0e5 100644 --- a/util/gpool/spmc/spmcpool.go +++ b/util/gpool/spmc/spmcpool.go @@ -21,6 +21,7 @@ import ( "time" "github.com/pingcap/log" + "github.com/pingcap/tidb/resourcemanager" "github.com/pingcap/tidb/resourcemanager/pooltask" "github.com/pingcap/tidb/resourcemanager/util" "github.com/pingcap/tidb/util/gpool" @@ -78,11 +79,10 @@ func NewSPMCPool[T any, U any, C any, CT any, TF pooltask.Context[CT]](name stri result.capacity.Add(size) result.workers = newWorkerLoopQueue[T, U, C, CT, TF](int(size)) result.cond = sync.NewCond(result.lock) - // TODO: wait https://github.com/pingcap/tidb/pull/40547 - // err := resourcemanager.GlobalResourceManager.Register(result, name, component) - // if err != nil { - // return nil, err - // } + err := resourcemanager.GlobalResourceManager.Register(result, name, component) + if err != nil { + return nil, err + } // Start a goroutine to clean up expired workers periodically. go result.purgePeriodically() return result, nil From edf840ac25cd2a78b9b0059be7f7dfad52f46e26 Mon Sep 17 00:00:00 2001 From: Weizhen Wang Date: Tue, 17 Jan 2023 10:49:55 +0800 Subject: [PATCH 06/25] add distributereorg feature flag Signed-off-by: Weizhen Wang --- Makefile | 4 ++++ sessionctx/variable/featuretag/distributereorg/default.go | 5 ++--- .../variable/featuretag/distributereorg/non_default.go | 2 +- 3 files changed, 7 insertions(+), 4 deletions(-) diff --git a/Makefile b/Makefile index b138bfbbd0f04..eef8f621fba14 100644 --- a/Makefile +++ b/Makefile @@ -408,6 +408,10 @@ bazel_coverage_test: failpoint-enable bazel_ci_prepare --build_event_json_file=bazel_1.json --@io_bazel_rules_go//go/config:cover_format=go_cover --define gotags=deadlock \ -- //... -//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_1.json --@io_bazel_rules_go//go/config:cover_format=go_cover --define gotags=deadlock,distributereorg \ + -- //... -//cmd/... -//tests/graceshutdown/... \ + -//tests/globalkilltest/... -//tests/readonlytest/... -//br/pkg/task:task_test -//tests/realtikvtest/... bazel_build: bazel_ci_prepare mkdir -p bin diff --git a/sessionctx/variable/featuretag/distributereorg/default.go b/sessionctx/variable/featuretag/distributereorg/default.go index b5d2f59b26a1b..6594d3ff726a4 100644 --- a/sessionctx/variable/featuretag/distributereorg/default.go +++ b/sessionctx/variable/featuretag/distributereorg/default.go @@ -12,10 +12,9 @@ // See the License for the specific language governing permissions and // limitations under the License. -//go:build !featuretag +//go:build !distributereorg package distributereorg // TiDBEnableDistributeReorg is a feature tag -// TODO: Before merge this PR, change this value to false. -const TiDBEnableDistributeReorg bool = true +const TiDBEnableDistributeReorg bool = false diff --git a/sessionctx/variable/featuretag/distributereorg/non_default.go b/sessionctx/variable/featuretag/distributereorg/non_default.go index f6286ba5b3409..9530be9c617d1 100644 --- a/sessionctx/variable/featuretag/distributereorg/non_default.go +++ b/sessionctx/variable/featuretag/distributereorg/non_default.go @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -//go:build featuretag +//go:build distributereorg package distributereorg From 114a957bfda06ec4d76cda95112281a057df4917 Mon Sep 17 00:00:00 2001 From: Weizhen Wang Date: Tue, 17 Jan 2023 15:41:30 +0800 Subject: [PATCH 07/25] *: fix "pool is existed" test Signed-off-by: Weizhen Wang --- br/pkg/mock/BUILD.bazel | 1 + br/pkg/mock/mock_cluster.go | 2 ++ br/pkg/restore/BUILD.bazel | 2 ++ br/pkg/restore/db_test.go | 5 +++++ executor/analyzetest/BUILD.bazel | 1 + executor/analyzetest/analyze_test.go | 2 ++ resourcemanager/util/shard_pool_map.go | 13 ++++++++++++- statistics/handle/BUILD.bazel | 1 + statistics/handle/update_test.go | 2 ++ testkit/BUILD.bazel | 1 + testkit/mockstore.go | 3 +++ 11 files changed, 32 insertions(+), 1 deletion(-) diff --git a/br/pkg/mock/BUILD.bazel b/br/pkg/mock/BUILD.bazel index d7bb24d4d3e04..dd64bf916be9a 100644 --- a/br/pkg/mock/BUILD.bazel +++ b/br/pkg/mock/BUILD.bazel @@ -27,6 +27,7 @@ go_library( "//parser", "//parser/ast", "//parser/model", + "//resourcemanager", "//server", "//session", "//store/mockstore", diff --git a/br/pkg/mock/mock_cluster.go b/br/pkg/mock/mock_cluster.go index 41caff5345aa6..681406138ac2a 100644 --- a/br/pkg/mock/mock_cluster.go +++ b/br/pkg/mock/mock_cluster.go @@ -18,6 +18,7 @@ import ( "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/domain" "github.com/pingcap/tidb/kv" + "github.com/pingcap/tidb/resourcemanager" "github.com/pingcap/tidb/server" "github.com/pingcap/tidb/session" "github.com/pingcap/tidb/store/mockstore" @@ -123,6 +124,7 @@ func (mock *Cluster) Stop() { _ = mock.HttpServer.Close() } view.Stop() + resourcemanager.GlobalResourceManager.Reset() } type configOverrider func(*mysql.Config) diff --git a/br/pkg/restore/BUILD.bazel b/br/pkg/restore/BUILD.bazel index 7d0071047d8e8..e3272bfcca7a1 100644 --- a/br/pkg/restore/BUILD.bazel +++ b/br/pkg/restore/BUILD.bazel @@ -145,6 +145,8 @@ go_test( "//parser/model", "//parser/mysql", "//parser/types", + "//resourcemanager", + "//resourcemanager/util", "//sessionctx/stmtctx", "//store/pdtypes", "//tablecodec", diff --git a/br/pkg/restore/db_test.go b/br/pkg/restore/db_test.go index 3a5416501e4df..2d7d932104dea 100644 --- a/br/pkg/restore/db_test.go +++ b/br/pkg/restore/db_test.go @@ -23,6 +23,8 @@ import ( "github.com/pingcap/tidb/parser/model" "github.com/pingcap/tidb/parser/mysql" "github.com/pingcap/tidb/parser/types" + "github.com/pingcap/tidb/resourcemanager" + "github.com/pingcap/tidb/resourcemanager/util" "github.com/pingcap/tidb/testkit" "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" @@ -36,6 +38,7 @@ type testRestoreSchemaSuite struct { } func createRestoreSchemaSuite(t *testing.T) *testRestoreSchemaSuite { + util.InTest.Store(true) var err error s := new(testRestoreSchemaSuite) s.mockGlue = &gluetidb.MockGlue{} @@ -47,6 +50,8 @@ func createRestoreSchemaSuite(t *testing.T) *testRestoreSchemaSuite { require.NoError(t, s.mock.Start()) t.Cleanup(func() { s.mock.Stop() + resourcemanager.GlobalResourceManager.Reset() + util.InTest.Store(false) }) return s } diff --git a/executor/analyzetest/BUILD.bazel b/executor/analyzetest/BUILD.bazel index 3112abe57c00f..0090ec36aa0f4 100644 --- a/executor/analyzetest/BUILD.bazel +++ b/executor/analyzetest/BUILD.bazel @@ -20,6 +20,7 @@ go_test( "//parser/mysql", "//parser/terror", "//planner/core", + "//resourcemanager", "//session", "//sessionctx", "//sessionctx/variable", diff --git a/executor/analyzetest/analyze_test.go b/executor/analyzetest/analyze_test.go index 843200fea6cf9..bc93d3e36ec20 100644 --- a/executor/analyzetest/analyze_test.go +++ b/executor/analyzetest/analyze_test.go @@ -35,6 +35,7 @@ import ( "github.com/pingcap/tidb/parser/mysql" "github.com/pingcap/tidb/parser/terror" "github.com/pingcap/tidb/planner/core" + "github.com/pingcap/tidb/resourcemanager" "github.com/pingcap/tidb/session" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/sessionctx/variable" @@ -2104,6 +2105,7 @@ func TestAnalyzeColumnsAfterAnalyzeAll(t *testing.T) { "test t b 0 0 2 1 3 4 0", "test t b 0 1 3 1 6 6 0")) tk.MustQuery(fmt.Sprintf("select hist_id from mysql.stats_histograms where version = (select version from mysql.stats_meta where table_id = %d)", tblID)).Check(testkit.Rows("2")) + resourcemanager.GlobalResourceManager.Reset() }(val) } } diff --git a/resourcemanager/util/shard_pool_map.go b/resourcemanager/util/shard_pool_map.go index 371365af031e1..7efd663c3154f 100644 --- a/resourcemanager/util/shard_pool_map.go +++ b/resourcemanager/util/shard_pool_map.go @@ -15,11 +15,22 @@ package util import ( + "flag" "sync" + "sync/atomic" "github.com/pingcap/errors" ) +// InTest is a flag to indicate whether the code is running in test. +var InTest atomic.Bool + +func init() { + if flag.Lookup("test.v") != nil { + InTest.Store(true) + } +} + const shard = 8 func hash(key string) int { @@ -69,7 +80,7 @@ func newPoolMap() poolMap { func (p *poolMap) Add(key string, pool *PoolContainer) error { p.mu.Lock() defer p.mu.Unlock() - if _, contain := p.poolMap[key]; contain { + if _, contain := p.poolMap[key]; contain && !InTest.Load() { return errors.New("pool is already exist") } p.poolMap[key] = pool diff --git a/statistics/handle/BUILD.bazel b/statistics/handle/BUILD.bazel index d52847495d539..fbdd84243acaf 100644 --- a/statistics/handle/BUILD.bazel +++ b/statistics/handle/BUILD.bazel @@ -81,6 +81,7 @@ go_test( "//metrics", "//parser/model", "//parser/mysql", + "//resourcemanager", "//session", "//sessionctx/stmtctx", "//sessionctx/variable", diff --git a/statistics/handle/update_test.go b/statistics/handle/update_test.go index 1e2f9fb228442..a53dcf65dc114 100644 --- a/statistics/handle/update_test.go +++ b/statistics/handle/update_test.go @@ -25,6 +25,7 @@ import ( "github.com/pingcap/tidb/metrics" "github.com/pingcap/tidb/parser/model" "github.com/pingcap/tidb/parser/mysql" + "github.com/pingcap/tidb/resourcemanager" "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/statistics" @@ -2351,6 +2352,7 @@ func TestCollectPredicateColumnsFromExecute(t *testing.T) { require.True(t, rows[0][4].(string) != "") require.True(t, rows[0][5].(string) == "") } + resourcemanager.GlobalResourceManager.Reset() }(val) } } diff --git a/testkit/BUILD.bazel b/testkit/BUILD.bazel index c28ef0614eb04..f811c1087b526 100644 --- a/testkit/BUILD.bazel +++ b/testkit/BUILD.bazel @@ -22,6 +22,7 @@ go_library( "//parser/terror", "//planner/core", "//resourcemanager", + "//resourcemanager/util", "//session", "//session/txninfo", "//sessionctx/variable", diff --git a/testkit/mockstore.go b/testkit/mockstore.go index 9756d5bb65804..6a84899191ff8 100644 --- a/testkit/mockstore.go +++ b/testkit/mockstore.go @@ -25,6 +25,7 @@ import ( "github.com/pingcap/tidb/domain" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/resourcemanager" + "github.com/pingcap/tidb/resourcemanager/util" "github.com/pingcap/tidb/session" "github.com/pingcap/tidb/store/driver" "github.com/pingcap/tidb/store/mockstore" @@ -82,6 +83,7 @@ func bootstrap(t testing.TB, store kv.Storage, lease time.Duration) *domain.Doma session.DisableStats4Test() domain.DisablePlanReplayerBackgroundJob4Test() domain.DisableDumpHistoricalStats4Test() + util.InTest.Store(true) dom, err := session.BootstrapSession(store) require.NoError(t, err) @@ -92,6 +94,7 @@ func bootstrap(t testing.TB, store kv.Storage, lease time.Duration) *domain.Doma err := store.Close() require.NoError(t, err) view.Stop() + util.InTest.Store(false) resourcemanager.GlobalResourceManager.Reset() }) return dom From 9b5025ca95e7aa2ea0c623792216d97dae3a0b89 Mon Sep 17 00:00:00 2001 From: Lynn Date: Tue, 17 Jan 2023 23:08:25 +0800 Subject: [PATCH 08/25] ddl, br: don't run backfill jobs when tidb_enable_ddl is enable and set InTest true --- br/pkg/mock/mock_cluster.go | 3 +++ ddl/backfilling.go | 3 --- ddl/ddl.go | 9 +++++---- ddl/index.go | 4 ++-- ddl/job_table.go | 2 +- 5 files changed, 11 insertions(+), 10 deletions(-) diff --git a/br/pkg/mock/mock_cluster.go b/br/pkg/mock/mock_cluster.go index 681406138ac2a..a24715555bb02 100644 --- a/br/pkg/mock/mock_cluster.go +++ b/br/pkg/mock/mock_cluster.go @@ -19,6 +19,7 @@ import ( "github.com/pingcap/tidb/domain" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/resourcemanager" + "github.com/pingcap/tidb/resourcemanager/util" "github.com/pingcap/tidb/server" "github.com/pingcap/tidb/session" "github.com/pingcap/tidb/store/mockstore" @@ -73,6 +74,7 @@ func NewCluster() (*Cluster, error) { session.SetSchemaLease(0) session.DisableStats4Test() + util.InTest.Store(true) dom, err := session.BootstrapSession(storage) if err != nil { return nil, errors.Trace(err) @@ -124,6 +126,7 @@ func (mock *Cluster) Stop() { _ = mock.HttpServer.Close() } view.Stop() + util.InTest.Store(true) resourcemanager.GlobalResourceManager.Reset() } diff --git a/ddl/backfilling.go b/ddl/backfilling.go index a22ea5c48abe7..a3a65918cecdf 100644 --- a/ddl/backfilling.go +++ b/ddl/backfilling.go @@ -329,9 +329,6 @@ func (w *backfillWorker) finishJob(bfJob *BackfillJob) error { } func (w *backfillWorker) String() string { - if w.backfiller == nil || w.GetCtx() == nil { - return "" - } return fmt.Sprintf("backfill-worker %d, tp %s", w.GetCtx().id, w.backfiller.String()) } diff --git a/ddl/ddl.go b/ddl/ddl.go index 4f9686e0c143d..cb4a5173f8b4a 100644 --- a/ddl/ddl.go +++ b/ddl/ddl.go @@ -765,21 +765,22 @@ func (d *ddl) Start(ctxPool *pools.ResourcePool) error { d.delRangeMgr = d.newDeleteRangeManager(ctxPool == nil) d.prepareWorkers4ConcurrencyDDL() - if err := d.prepareBackfillWorkers(); err != nil { - return err - } if config.TableLockEnabled() { d.wg.Add(1) go d.startCleanDeadTableLock() } - // If tidb_enable_ddl is true, we need campaign owner and do DDL job. + // If tidb_enable_ddl is true, we need campaign owner and do DDL jobs. Besides, we also can do backfill jobs. // Otherwise, we needn't do that. if config.GetGlobalConfig().Instance.TiDBEnableDDL.Load() { if err := d.EnableDDL(); err != nil { return err } + + if err := d.prepareBackfillWorkers(); err != nil { + return err + } } variable.RegisterStatistics(d) diff --git a/ddl/index.go b/ddl/index.go index 1ddf2ce04d8f3..8d7ead0704d94 100644 --- a/ddl/index.go +++ b/ddl/index.go @@ -1383,13 +1383,13 @@ func (w *baseIndexWorker) GetCtx() *backfillCtx { return w.backfillCtx } -func newAddIndexWorkerContext(d *ddl, sess *session, schemaName model.CIStr, tbl table.Table, workerCnt int, +func newAddIndexWorkerContext(d *ddl, schemaName model.CIStr, tbl table.Table, workerCnt int, bfJob *BackfillJob, jobCtx *JobContext) (*backfillWorkerContext, error) { //nolint:forcetypeassert phyTbl := tbl.(table.PhysicalTable) return newBackfillWorkerContext(d, schemaName.O, tbl, workerCnt, bfJob.Meta, func(bfCtx *backfillCtx) (backfiller, error) { - decodeColMap, err := makeupDecodeColMap(sess, schemaName, phyTbl) + decodeColMap, err := makeupDecodeColMap(bfCtx.sessCtx, schemaName, phyTbl) if err != nil { logutil.BgLogger().Info("[ddl] make up decode column map failed", zap.Error(err)) return nil, errors.Trace(err) diff --git a/ddl/job_table.go b/ddl/job_table.go index 1b613f3324814..51917e5fc9e49 100644 --- a/ddl/job_table.go +++ b/ddl/job_table.go @@ -446,7 +446,7 @@ func runBackfillJobs(d *ddl, sess *session, bJob *BackfillJob, jobCtx *JobContex workerCnt := int(variable.GetDDLReorgWorkerCounter()) // TODO: Different worker using different newBackfillerFunc. - workerCtx, err := newAddIndexWorkerContext(d, sess, dbInfo.Name, tbl, workerCnt, bJob, jobCtx) + workerCtx, err := newAddIndexWorkerContext(d, dbInfo.Name, tbl, workerCnt, bJob, jobCtx) if err != nil || workerCtx == nil { logutil.BgLogger().Info("[ddl] new adding index worker context failed", zap.Reflect("workerCtx", workerCtx), zap.Error(err)) return nil, errors.Trace(err) From 641c14dfba3c59eb98e8e42572353e1bae923556 Mon Sep 17 00:00:00 2001 From: Lynn Date: Wed, 18 Jan 2023 00:21:41 +0800 Subject: [PATCH 09/25] ddl, br: move some codes to dist_* files and make bazel happy --- br/pkg/mock/BUILD.bazel | 1 + ddl/BUILD.bazel | 1 + ddl/backfilling.go | 261 +-------------------------------- ddl/db_test.go | 6 +- ddl/ddl.go | 19 ++- ddl/dist_backfilling.go | 37 ++++- ddl/dist_owner.go | 296 ++++++++++++++++++++++++++++++++++++++ ddl/index.go | 4 +- ddl/job_table.go | 84 ++++------- ddl/modify_column_test.go | 17 ++- ddl/reorg.go | 6 +- 11 files changed, 405 insertions(+), 327 deletions(-) create mode 100644 ddl/dist_owner.go diff --git a/br/pkg/mock/BUILD.bazel b/br/pkg/mock/BUILD.bazel index dd64bf916be9a..474fb81c2081f 100644 --- a/br/pkg/mock/BUILD.bazel +++ b/br/pkg/mock/BUILD.bazel @@ -28,6 +28,7 @@ go_library( "//parser/ast", "//parser/model", "//resourcemanager", + "//resourcemanager/util", "//server", "//session", "//store/mockstore", diff --git a/ddl/BUILD.bazel b/ddl/BUILD.bazel index 6bbb148f33eff..f21eab76e5a3c 100644 --- a/ddl/BUILD.bazel +++ b/ddl/BUILD.bazel @@ -25,6 +25,7 @@ go_library( "delete_range.go", "delete_range_util.go", "dist_backfilling.go", + "dist_owner.go", "foreign_key.go", "generated_column.go", "index.go", diff --git a/ddl/backfilling.go b/ddl/backfilling.go index a3a65918cecdf..17922993bbc2f 100644 --- a/ddl/backfilling.go +++ b/ddl/backfilling.go @@ -364,12 +364,13 @@ func (w *backfillWorker) handleBackfillTask(d *ddlCtx, task *reorgBackfillTask, jobID := task.getJobID() rc := d.getReorgCtx(jobID) + isDistReorg := task.bfJob != nil for { // Give job chance to be canceled, if we not check it here, // if there is panic in bf.BackfillDataInTxn we will never cancel the job. // Because reorgRecordTask may run a long time, // we should check whether this ddl job is still runnable. - err := d.isReorgRunnable(jobID) + err := d.isReorgRunnable(jobID, isDistReorg) if err != nil { result.err = err return result @@ -409,7 +410,7 @@ func (w *backfillWorker) handleBackfillTask(d *ddlCtx, task *reorgBackfillTask, break } - if task.bfJob != nil { + if isDistReorg { // TODO: Adjust the updating lease frequency by batch processing time carefully. if time.Since(batchStartTime) < updateInstanceLease { continue @@ -624,7 +625,7 @@ func (dc *ddlCtx) sendTasksAndWait(scheduler *backfillScheduler, totalAddedCount nextKey, taskAddedCount, err := waitTaskResults(scheduler, batchTasks, totalAddedCount) elapsedTime := time.Since(startTime) if err == nil { - err = dc.isReorgRunnable(reorgInfo.Job.ID) + err = dc.isReorgRunnable(reorgInfo.Job.ID, false) } // Update the reorg handle that has been processed. @@ -1008,7 +1009,7 @@ func (dc *ddlCtx) writePhysicalTableRecord(sessPool *sessionPool, t table.Physic return errors.Trace(err) } - if err := dc.isReorgRunnable(reorgInfo.Job.ID); err != nil { + if err := dc.isReorgRunnable(reorgInfo.Job.ID, false); err != nil { return errors.Trace(err) } if startKey == nil && endKey == nil { @@ -1192,258 +1193,6 @@ func (dc *ddlCtx) splitTableToBackfillJobs(sess *session, reorgInfo *reorgInfo, return nil } -func (dc *ddlCtx) controlWritePhysicalTableRecord(sess *session, t table.PhysicalTable, bfWorkerType backfillerType, reorgInfo *reorgInfo) error { - startKey, endKey := reorgInfo.StartKey, reorgInfo.EndKey - if startKey == nil && endKey == nil { - return nil - } - - ddlJobID := reorgInfo.Job.ID - if err := dc.isReorgRunnable(ddlJobID); err != nil { - return errors.Trace(err) - } - - defaultSQLMode := sess.GetSessionVars().SQLMode - defer func() { - sess.GetSessionVars().SQLMode = defaultSQLMode - }() - // Make timestamp type can be inserted ZeroTimestamp. - sess.GetSessionVars().SQLMode = mysql.ModeNone - currBackfillJobID := int64(1) - err := checkAndHandleInterruptedBackfillJobs(sess, ddlJobID, reorgInfo.currElement.ID, reorgInfo.currElement.TypeKey) - if err != nil { - return errors.Trace(err) - } - maxBfJob, err := GetMaxBackfillJob(sess, ddlJobID, reorgInfo.currElement.ID, reorgInfo.currElement.TypeKey) - if err != nil { - return errors.Trace(err) - } - if maxBfJob != nil { - startKey = maxBfJob.EndKey - currBackfillJobID = maxBfJob.ID + 1 - } - - var isUnique bool - if bfWorkerType == typeAddIndexWorker { - idxInfo := model.FindIndexInfoByID(t.Meta().Indices, reorgInfo.currElement.ID) - isUnique = idxInfo.Unique - } - err = dc.splitTableToBackfillJobs(sess, reorgInfo, t, isUnique, bfWorkerType, startKey, currBackfillJobID) - if err != nil { - return errors.Trace(err) - } - - var backfillJobFinished bool - currEle := reorgInfo.currElement - ticker := time.NewTicker(300 * time.Millisecond) - defer ticker.Stop() - for { - if err := dc.isReorgRunnable(ddlJobID); err != nil { - return errors.Trace(err) - } - - select { - case <-ticker.C: - if !backfillJobFinished { - err := checkAndHandleInterruptedBackfillJobs(sess, ddlJobID, currEle.ID, currEle.TypeKey) - if err != nil { - logutil.BgLogger().Warn("[ddl] finish interrupted backfill jobs", zap.Int64("job ID", ddlJobID), zap.Error(err)) - return errors.Trace(err) - } - - bfJob, err := getBackfillJobWithRetry(sess, BackfillTable, ddlJobID, currEle.ID, currEle.TypeKey, false) - if err != nil { - logutil.BgLogger().Info("[ddl] getBackfillJobWithRetry failed", zap.Int64("job ID", ddlJobID), zap.Error(err)) - return errors.Trace(err) - } - if bfJob == nil { - backfillJobFinished = true - logutil.BgLogger().Info("[ddl] finish backfill jobs", zap.Int64("job ID", ddlJobID)) - } - } - if backfillJobFinished { - // TODO: Consider whether these backfill jobs are always out of sync. - isSynced, err := checkJobIsFinished(sess, ddlJobID) - if err != nil { - logutil.BgLogger().Warn("[ddl] checkJobIsFinished failed", zap.Int64("job ID", ddlJobID), zap.Error(err)) - return errors.Trace(err) - } - if isSynced { - logutil.BgLogger().Info("[ddl] finish backfill jobs and put them to history", zap.Int64("job ID", ddlJobID)) - return GetBackfillErr(sess, ddlJobID, currEle.ID, currEle.TypeKey) - } - } - case <-dc.ctx.Done(): - return dc.ctx.Err() - } - } -} - -func checkJobIsFinished(sess *session, ddlJobID int64) (bool, error) { - var err error - var unsyncedInstanceIDs []string - for i := 0; i < retrySQLTimes; i++ { - unsyncedInstanceIDs, err = getUnsyncedInstanceIDs(sess, ddlJobID, "check_backfill_history_job_sync") - if err == nil && len(unsyncedInstanceIDs) == 0 { - return true, nil - } - - logutil.BgLogger().Info("[ddl] checkJobIsSynced failed", - zap.Strings("unsyncedInstanceIDs", unsyncedInstanceIDs), zap.Int("tryTimes", i), zap.Error(err)) - time.Sleep(retrySQLInterval) - } - - return false, errors.Trace(err) -} - -// GetBackfillErr gets the error in backfill job. -func GetBackfillErr(sess *session, ddlJobID, currEleID int64, currEleKey []byte) error { - var err error - var metas []*model.BackfillMeta - for i := 0; i < retrySQLTimes; i++ { - metas, err = GetBackfillMetas(sess, BackfillHistoryTable, fmt.Sprintf("ddl_job_id = %d and ele_id = %d and ele_key = %s", - ddlJobID, currEleID, wrapKey2String(currEleKey)), "get_backfill_job_metas") - if err == nil { - for _, m := range metas { - if m.Error != nil { - return m.Error - } - } - return nil - } - - logutil.BgLogger().Info("[ddl] GetBackfillMetas failed in checkJobIsSynced", zap.Int("tryTimes", i), zap.Error(err)) - time.Sleep(retrySQLInterval) - } - - return errors.Trace(err) -} - -func checkAndHandleInterruptedBackfillJobs(sess *session, ddlJobID, currEleID int64, currEleKey []byte) (err error) { - var bJobs []*BackfillJob - for i := 0; i < retrySQLTimes; i++ { - bJobs, err = GetInterruptedBackfillJobsForOneEle(sess, ddlJobID, currEleID, currEleKey) - if err == nil { - break - } - logutil.BgLogger().Info("[ddl] getInterruptedBackfillJobsForOneEle failed", zap.Error(err)) - time.Sleep(retrySQLInterval) - } - if err != nil { - return errors.Trace(err) - } - if len(bJobs) == 0 { - return nil - } - - for i := 0; i < retrySQLTimes; i++ { - err = MoveBackfillJobsToHistoryTable(sess, bJobs[0]) - if err == nil { - return bJobs[0].Meta.Error - } - logutil.BgLogger().Info("[ddl] MoveBackfillJobsToHistoryTable failed", zap.Error(err)) - time.Sleep(retrySQLInterval) - } - return errors.Trace(err) -} - -func checkBackfillJobCount(sess *session, ddlJobID, currEleID int64, currEleKey []byte) (backfillJobCnt int, err error) { - err = checkAndHandleInterruptedBackfillJobs(sess, ddlJobID, currEleID, currEleKey) - if err != nil { - return 0, errors.Trace(err) - } - - backfillJobCnt, err = GetBackfillJobCount(sess, BackfillTable, fmt.Sprintf("ddl_job_id = %d and ele_id = %d and ele_key = %s", - ddlJobID, currEleID, wrapKey2String(currEleKey)), "check_backfill_job_count") - if err != nil { - return 0, errors.Trace(err) - } - - return backfillJobCnt, nil -} - -func getBackfillJobWithRetry(sess *session, tableName string, ddlJobID, currEleID int64, currEleKey []byte, isDesc bool) (*BackfillJob, error) { - var err error - var bJobs []*BackfillJob - descStr := "" - if isDesc { - descStr = "order by id desc" - } - for i := 0; i < retrySQLTimes; i++ { - bJobs, err = GetBackfillJobs(sess, tableName, fmt.Sprintf("ddl_job_id = %d and ele_id = %d and ele_key = %s %s limit 1", - ddlJobID, currEleID, wrapKey2String(currEleKey), descStr), "check_backfill_job_state") - if err != nil { - logutil.BgLogger().Warn("[ddl] GetBackfillJobs failed", zap.Error(err)) - continue - } - - if len(bJobs) != 0 { - return bJobs[0], nil - } - break - } - return nil, errors.Trace(err) -} - -// GetMaxBackfillJob gets the max backfill job in BackfillTable and BackfillHistoryTable. -func GetMaxBackfillJob(sess *session, ddlJobID, currEleID int64, currEleKey []byte) (*BackfillJob, error) { - bfJob, err := getBackfillJobWithRetry(sess, BackfillTable, ddlJobID, currEleID, currEleKey, true) - if err != nil { - return nil, errors.Trace(err) - } - hJob, err := getBackfillJobWithRetry(sess, BackfillHistoryTable, ddlJobID, currEleID, currEleKey, true) - if err != nil { - return nil, errors.Trace(err) - } - - if bfJob == nil { - return hJob, nil - } - if hJob == nil { - return bfJob, nil - } - if bfJob.ID > hJob.ID { - return bfJob, nil - } - return hJob, nil -} - -// MoveBackfillJobsToHistoryTable moves backfill table jobs to the backfill history table. -func MoveBackfillJobsToHistoryTable(sctx sessionctx.Context, bfJob *BackfillJob) error { - s, ok := sctx.(*session) - if !ok { - return errors.Errorf("sess ctx:%#v convert session failed", sctx) - } - - return s.runInTxn(func(se *session) error { - // TODO: Consider batch by batch update backfill jobs and insert backfill history jobs. - bJobs, err := GetBackfillJobs(se, BackfillTable, fmt.Sprintf("ddl_job_id = %d and ele_id = %d and ele_key = %s", - bfJob.JobID, bfJob.EleID, wrapKey2String(bfJob.EleKey)), "update_backfill_job") - if err != nil { - return errors.Trace(err) - } - if len(bJobs) == 0 { - return nil - } - - txn, err := se.txn() - if err != nil { - return errors.Trace(err) - } - startTS := txn.StartTS() - err = RemoveBackfillJob(se, true, bJobs[0]) - if err == nil { - for _, bj := range bJobs { - bj.State = model.JobStateCancelled - bj.FinishTS = startTS - } - err = AddBackfillHistoryJob(se, bJobs) - } - logutil.BgLogger().Info("[ddl] move backfill jobs to history table", zap.Int("job count", len(bJobs))) - return errors.Trace(err) - }) -} - // recordIterFunc is used for low-level record iteration. type recordIterFunc func(h kv.Handle, rowKey kv.Key, rawRecord []byte) (more bool, err error) diff --git a/ddl/db_test.go b/ddl/db_test.go index 46cfe301ec4f4..e4324e71fee7c 100644 --- a/ddl/db_test.go +++ b/ddl/db_test.go @@ -990,7 +990,11 @@ func TestAddIndexFailOnCaseWhenCanExit(t *testing.T) { tk.MustExec("drop table if exists t") tk.MustExec("create table t(a int, b int)") tk.MustExec("insert into t values(1, 1)") - tk.MustGetErrMsg("alter table t add index idx(b)", "[ddl:-1]DDL job rollback, error msg: job.ErrCount:1, mock unknown type: ast.whenClause.") + if variable.DDLEnableDistributeReorg.Load() { + tk.MustGetErrMsg("alter table t add index idx(b)", "[ddl:-1]job.ErrCount:0, mock unknown type: ast.whenClause.") + } else { + tk.MustGetErrMsg("alter table t add index idx(b)", "[ddl:-1]DDL job rollback, error msg: job.ErrCount:1, mock unknown type: ast.whenClause.") + } tk.MustExec("drop table if exists t") } diff --git a/ddl/ddl.go b/ddl/ddl.go index cb4a5173f8b4a..c7f0a74d98c03 100644 --- a/ddl/ddl.go +++ b/ddl/ddl.go @@ -348,11 +348,7 @@ type ddlCtx struct { // It holds the running DDL jobs ID. runningJobIDs []string // reorgCtx is used for reorganization. - reorgCtx struct { - sync.RWMutex - // reorgCtxMap maps job ID to reorg context. - reorgCtxMap map[int64]*reorgCtx - } + reorgCtx reorgContexts // backfillCtx is used for backfill workers. backfillCtx struct { sync.RWMutex @@ -511,6 +507,19 @@ func (dc *ddlCtx) setBackfillCtxJobContext(jobID int64, jobQuery string, jobType return jobCtx, existent } +type reorgContexts struct { + sync.RWMutex + // reorgCtxMap maps job ID to reorg context. + reorgCtxMap map[int64]*reorgCtx +} + +func getReorgCtx(reorgCtxs *reorgContexts, jobID int64) *reorgCtx { + reorgCtxs.RLock() + defer reorgCtxs.RUnlock() + return reorgCtxs.reorgCtxMap[jobID] +} + +// TODO: Using getReorgCtx instead of dc.getReorgCtx. func (dc *ddlCtx) getReorgCtx(jobID int64) *reorgCtx { dc.reorgCtx.RLock() defer dc.reorgCtx.RUnlock() diff --git a/ddl/dist_backfilling.go b/ddl/dist_backfilling.go index b85a53340b559..9e27635078767 100644 --- a/ddl/dist_backfilling.go +++ b/ddl/dist_backfilling.go @@ -24,9 +24,11 @@ import ( "github.com/pingcap/tidb/parser/model" "github.com/pingcap/tidb/resourcemanager/pooltask" "github.com/pingcap/tidb/sessionctx" + "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/table" "github.com/pingcap/tidb/util" "github.com/pingcap/tidb/util/dbterror" + "github.com/pingcap/tidb/util/gpool/spmc" "github.com/pingcap/tidb/util/logutil" "go.uber.org/zap" ) @@ -96,6 +98,39 @@ func (bwCtx *backfillWorkerContext) GetContext() *backfillWorker { return bw } +func runBackfillJobs(d *ddl, sess *session, bJob *BackfillJob, jobCtx *JobContext) (table.Table, error) { + dbInfo, tbl, err := d.getTableByTxn(d.store, bJob.Meta.SchemaID, bJob.Meta.TableID) + if err != nil { + logutil.BgLogger().Warn("[ddl] runBackfillJobs gets table failed", zap.String("bfJob", bJob.AbbrStr()), zap.Error(err)) + return nil, err + } + + workerCnt := int(variable.GetDDLReorgWorkerCounter()) + // TODO: Different worker using different newBackfillerFunc. + workerCtx, err := newAddIndexWorkerContext(d, dbInfo.Name, tbl, workerCnt, bJob, jobCtx) + if err != nil || workerCtx == nil { + logutil.BgLogger().Info("[ddl] new adding index worker context failed", zap.Reflect("workerCtx", workerCtx), zap.Error(err)) + return nil, errors.Trace(err) + } + bwMgr := newBackfilWorkerManager(workerCtx) + d.backfillWorkerPool.SetConsumerFunc(func(task *reorgBackfillTask, _ int, bfWorker *backfillWorker) *backfillResult { + return bfWorker.runTask(task) + }) + proFunc := func() ([]*reorgBackfillTask, error) { + // TODO: After BackfillJob replaces reorgBackfillTask, use backfiller's GetTasks instead of it. + return GetTasks(d.ddlCtx, sess, tbl, bJob.JobID, workerCnt*2) + } + // add new task + resultCh, control := d.backfillWorkerPool.AddProduceBySlice(proFunc, 0, workerCtx, spmc.WithConcurrency(workerCnt)) + bwMgr.waitFinalResult(resultCh, control) + + // waiting task finishing + control.Wait() + err = bwMgr.close(d) + + return tbl, err +} + func (bwCtx *backfillWorkerContext) close(d *ddl) { for _, s := range bwCtx.sessCtxs { d.sessPool.put(s) @@ -131,7 +166,7 @@ func (bwm *backfilWorkerManager) waitFinalResult(resultCh <-chan *backfillResult if result.err != nil { logutil.BgLogger().Warn("handle backfill task failed", zap.Error(result.err)) bwm.unsyncErr = result.err - // TODO: After spmc supports, exit this tControl related goroutines in spmc pool. + tControl.Stop() return } case <-bwm.exitCh: diff --git a/ddl/dist_owner.go b/ddl/dist_owner.go new file mode 100644 index 0000000000000..7fb34c0938774 --- /dev/null +++ b/ddl/dist_owner.go @@ -0,0 +1,296 @@ +// Copyright 2023 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package ddl + +import ( + "context" + "fmt" + "time" + + "github.com/pingcap/errors" + "github.com/pingcap/tidb/meta" + "github.com/pingcap/tidb/parser/model" + "github.com/pingcap/tidb/parser/mysql" + "github.com/pingcap/tidb/sessionctx" + "github.com/pingcap/tidb/table" + "github.com/pingcap/tidb/util/dbterror" + "github.com/pingcap/tidb/util/logutil" + "go.uber.org/zap" +) + +func (dc *ddlCtx) controlWritePhysicalTableRecord(sess *session, t table.PhysicalTable, bfWorkerType backfillerType, reorgInfo *reorgInfo) error { + startKey, endKey := reorgInfo.StartKey, reorgInfo.EndKey + if startKey == nil && endKey == nil { + return nil + } + + ddlJobID := reorgInfo.Job.ID + if err := dc.isReorgRunnable(ddlJobID, true); err != nil { + return errors.Trace(err) + } + + currEle := reorgInfo.currElement + defaultSQLMode := sess.GetSessionVars().SQLMode + defer func() { + sess.GetSessionVars().SQLMode = defaultSQLMode + }() + // Make timestamp type can be inserted ZeroTimestamp. + sess.GetSessionVars().SQLMode = mysql.ModeNone + currBackfillJobID := int64(1) + err := checkAndHandleInterruptedBackfillJobs(sess, ddlJobID, currEle.ID, currEle.TypeKey) + if err != nil { + return errors.Trace(err) + } + maxBfJob, err := GetMaxBackfillJob(sess, ddlJobID, currEle.ID, currEle.TypeKey) + if err != nil { + return errors.Trace(err) + } + if maxBfJob != nil { + startKey = maxBfJob.EndKey + currBackfillJobID = maxBfJob.ID + 1 + } + + var isUnique bool + if bfWorkerType == typeAddIndexWorker { + idxInfo := model.FindIndexInfoByID(t.Meta().Indices, currEle.ID) + isUnique = idxInfo.Unique + } + err = dc.splitTableToBackfillJobs(sess, reorgInfo, t, isUnique, bfWorkerType, startKey, currBackfillJobID) + if err != nil { + return errors.Trace(err) + } + dc.ctx.Done() + return checkReorgJobFinished(dc.ctx, sess, &dc.reorgCtx, ddlJobID, currEle) +} + +func checkReorgJobFinished(ctx context.Context, sess *session, reorgCtxs *reorgContexts, ddlJobID int64, currEle *meta.Element) error { + var times int64 + var bfJob *BackfillJob + var backfillJobFinished bool + ticker := time.NewTicker(300 * time.Millisecond) + defer ticker.Stop() + for { + if getReorgCtx(reorgCtxs, ddlJobID).isReorgCanceled() { + // Job is cancelled. So it can't be done. + return dbterror.ErrCancelledDDLJob + } + + select { + case <-ticker.C: + // Print this log every 5 min. + if times%1000 == 0 { + logutil.BgLogger().Info("[ddl] check all backfill jobs is finished", + zap.Int64("job ID", ddlJobID), zap.Bool("isFinished", backfillJobFinished), zap.Reflect("bfJob", bfJob)) + } + times++ + if !backfillJobFinished { + err := checkAndHandleInterruptedBackfillJobs(sess, ddlJobID, currEle.ID, currEle.TypeKey) + if err != nil { + logutil.BgLogger().Warn("[ddl] finish interrupted backfill jobs", zap.Int64("job ID", ddlJobID), zap.Error(err)) + return errors.Trace(err) + } + + bfJob, err = getBackfillJobWithRetry(sess, BackfillTable, ddlJobID, currEle.ID, currEle.TypeKey, false) + if err != nil { + logutil.BgLogger().Info("[ddl] getBackfillJobWithRetry failed", zap.Int64("job ID", ddlJobID), zap.Error(err)) + return errors.Trace(err) + } + if bfJob == nil { + backfillJobFinished = true + logutil.BgLogger().Info("[ddl] finish all backfill jobs", zap.Int64("job ID", ddlJobID)) + } + } + if backfillJobFinished { + // TODO: Consider whether these backfill jobs are always out of sync. + isSynced, err := checkJobIsFinished(sess, ddlJobID) + if err != nil { + logutil.BgLogger().Warn("[ddl] checkJobIsFinished failed", zap.Int64("job ID", ddlJobID), zap.Error(err)) + return errors.Trace(err) + } + if isSynced { + logutil.BgLogger().Info("[ddl] finish all backfill jobs and put them to history", zap.Int64("job ID", ddlJobID)) + return GetBackfillErr(sess, ddlJobID, currEle.ID, currEle.TypeKey) + } + } + case <-ctx.Done(): + return ctx.Err() + } + } +} + +func checkJobIsFinished(sess *session, ddlJobID int64) (bool, error) { + var err error + var unsyncedInstanceIDs []string + for i := 0; i < retrySQLTimes; i++ { + unsyncedInstanceIDs, err = getUnsyncedInstanceIDs(sess, ddlJobID, "check_backfill_history_job_sync") + if err == nil && len(unsyncedInstanceIDs) == 0 { + return true, nil + } + + logutil.BgLogger().Info("[ddl] checkJobIsSynced failed", + zap.Strings("unsyncedInstanceIDs", unsyncedInstanceIDs), zap.Int("tryTimes", i), zap.Error(err)) + time.Sleep(retrySQLInterval) + } + + return false, errors.Trace(err) +} + +// GetBackfillErr gets the error in backfill job. +func GetBackfillErr(sess *session, ddlJobID, currEleID int64, currEleKey []byte) error { + var err error + var metas []*model.BackfillMeta + for i := 0; i < retrySQLTimes; i++ { + metas, err = GetBackfillMetas(sess, BackfillHistoryTable, fmt.Sprintf("ddl_job_id = %d and ele_id = %d and ele_key = %s", + ddlJobID, currEleID, wrapKey2String(currEleKey)), "get_backfill_job_metas") + if err == nil { + for _, m := range metas { + if m.Error != nil { + return m.Error + } + } + return nil + } + + logutil.BgLogger().Info("[ddl] GetBackfillMetas failed in checkJobIsSynced", zap.Int("tryTimes", i), zap.Error(err)) + time.Sleep(retrySQLInterval) + } + + return errors.Trace(err) +} + +func checkAndHandleInterruptedBackfillJobs(sess *session, ddlJobID, currEleID int64, currEleKey []byte) (err error) { + var bJobs []*BackfillJob + for i := 0; i < retrySQLTimes; i++ { + bJobs, err = GetInterruptedBackfillJobsForOneEle(sess, ddlJobID, currEleID, currEleKey) + if err == nil { + break + } + logutil.BgLogger().Info("[ddl] getInterruptedBackfillJobsForOneEle failed", zap.Error(err)) + time.Sleep(retrySQLInterval) + } + if err != nil { + return errors.Trace(err) + } + if len(bJobs) == 0 { + return nil + } + + for i := 0; i < retrySQLTimes; i++ { + err = MoveBackfillJobsToHistoryTable(sess, bJobs[0]) + if err == nil { + return bJobs[0].Meta.Error + } + logutil.BgLogger().Info("[ddl] MoveBackfillJobsToHistoryTable failed", zap.Error(err)) + time.Sleep(retrySQLInterval) + } + return errors.Trace(err) +} + +func checkBackfillJobCount(sess *session, ddlJobID, currEleID int64, currEleKey []byte) (backfillJobCnt int, err error) { + err = checkAndHandleInterruptedBackfillJobs(sess, ddlJobID, currEleID, currEleKey) + if err != nil { + return 0, errors.Trace(err) + } + + backfillJobCnt, err = GetBackfillJobCount(sess, BackfillTable, fmt.Sprintf("ddl_job_id = %d and ele_id = %d and ele_key = %s", + ddlJobID, currEleID, wrapKey2String(currEleKey)), "check_backfill_job_count") + if err != nil { + return 0, errors.Trace(err) + } + + return backfillJobCnt, nil +} + +func getBackfillJobWithRetry(sess *session, tableName string, ddlJobID, currEleID int64, currEleKey []byte, isDesc bool) (*BackfillJob, error) { + var err error + var bJobs []*BackfillJob + descStr := "" + if isDesc { + descStr = "order by id desc" + } + for i := 0; i < retrySQLTimes; i++ { + bJobs, err = GetBackfillJobs(sess, tableName, fmt.Sprintf("ddl_job_id = %d and ele_id = %d and ele_key = %s %s limit 1", + ddlJobID, currEleID, wrapKey2String(currEleKey), descStr), "check_backfill_job_state") + if err != nil { + logutil.BgLogger().Warn("[ddl] GetBackfillJobs failed", zap.Error(err)) + continue + } + + if len(bJobs) != 0 { + return bJobs[0], nil + } + break + } + return nil, errors.Trace(err) +} + +// GetMaxBackfillJob gets the max backfill job in BackfillTable and BackfillHistoryTable. +func GetMaxBackfillJob(sess *session, ddlJobID, currEleID int64, currEleKey []byte) (*BackfillJob, error) { + bfJob, err := getBackfillJobWithRetry(sess, BackfillTable, ddlJobID, currEleID, currEleKey, true) + if err != nil { + return nil, errors.Trace(err) + } + hJob, err := getBackfillJobWithRetry(sess, BackfillHistoryTable, ddlJobID, currEleID, currEleKey, true) + if err != nil { + return nil, errors.Trace(err) + } + + if bfJob == nil { + return hJob, nil + } + if hJob == nil { + return bfJob, nil + } + if bfJob.ID > hJob.ID { + return bfJob, nil + } + return hJob, nil +} + +// MoveBackfillJobsToHistoryTable moves backfill table jobs to the backfill history table. +func MoveBackfillJobsToHistoryTable(sctx sessionctx.Context, bfJob *BackfillJob) error { + s, ok := sctx.(*session) + if !ok { + return errors.Errorf("sess ctx:%#v convert session failed", sctx) + } + + return s.runInTxn(func(se *session) error { + // TODO: Consider batch by batch update backfill jobs and insert backfill history jobs. + bJobs, err := GetBackfillJobs(se, BackfillTable, fmt.Sprintf("ddl_job_id = %d and ele_id = %d and ele_key = %s", + bfJob.JobID, bfJob.EleID, wrapKey2String(bfJob.EleKey)), "update_backfill_job") + if err != nil { + return errors.Trace(err) + } + if len(bJobs) == 0 { + return nil + } + + txn, err := se.txn() + if err != nil { + return errors.Trace(err) + } + startTS := txn.StartTS() + err = RemoveBackfillJob(se, true, bJobs[0]) + if err == nil { + for _, bj := range bJobs { + bj.State = model.JobStateCancelled + bj.FinishTS = startTS + } + err = AddBackfillHistoryJob(se, bJobs) + } + logutil.BgLogger().Info("[ddl] move backfill jobs to history table", zap.Int("job count", len(bJobs))) + return errors.Trace(err) + }) +} diff --git a/ddl/index.go b/ddl/index.go index 8d7ead0704d94..f41ecbb46c7ba 100644 --- a/ddl/index.go +++ b/ddl/index.go @@ -970,7 +970,9 @@ func runReorgJobAndHandleErr(w *worker, d *ddlCtx, t *meta.Meta, job *model.Job, // if timeout, we should return, check for the owner and re-wait job done. return false, ver, nil } - if kv.ErrKeyExists.Equal(err) || dbterror.ErrCancelledDDLJob.Equal(err) || dbterror.ErrCantDecodeRecord.Equal(err) { + if kv.ErrKeyExists.Equal(err) || dbterror.ErrCancelledDDLJob.Equal(err) || dbterror.ErrCantDecodeRecord.Equal(err) || + // TODO: Remove this check make it can be retry. Related test is TestModifyColumnReorgInfo. + variable.DDLEnableDistributeReorg.Load() { logutil.BgLogger().Warn("[ddl] run add index job failed, convert job to rollback", zap.String("job", job.String()), zap.Error(err)) ver, err = convertAddIdxJob2RollbackJob(d, t, job, tbl.Meta(), indexInfo, err) if err1 := rh.RemoveDDLReorgHandle(job, reorgInfo.elements); err1 != nil { diff --git a/ddl/job_table.go b/ddl/job_table.go index 51917e5fc9e49..3b898bcf2c21f 100644 --- a/ddl/job_table.go +++ b/ddl/job_table.go @@ -27,7 +27,6 @@ import ( "github.com/pingcap/failpoint" "github.com/pingcap/kvproto/pkg/kvrpcpb" "github.com/pingcap/tidb/ddl/ingest" - ddlutil "github.com/pingcap/tidb/ddl/util" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/meta" "github.com/pingcap/tidb/metrics" @@ -37,7 +36,6 @@ import ( "github.com/pingcap/tidb/types" tidbutil "github.com/pingcap/tidb/util" "github.com/pingcap/tidb/util/dbterror" - "github.com/pingcap/tidb/util/gpool/spmc" "github.com/pingcap/tidb/util/logutil" clientv3 "go.etcd.io/etcd/client/v3" "go.uber.org/zap" @@ -317,10 +315,6 @@ func (d *ddl) markJobProcessing(sess *session, job *model.Job) error { } func (d *ddl) startDispatchBackfillJobsLoop() { - isDistReorg := variable.DDLEnableDistributeReorg.Load() - if !isDistReorg { - return - } d.backfillCtx.jobCtxMap = make(map[int64]*JobContext) d.backfillCtx.backfillCtxMap = make(map[int64]struct{}) @@ -372,6 +366,10 @@ func (d *ddl) getTableByTxn(store kv.Storage, schemaID, tableID int64) (*model.D } func (d *ddl) loadBackfillJobAndRun() { + isDistReorg := variable.DDLEnableDistributeReorg.Load() + if !isDistReorg { + return + } se, err := d.sessPool.get() defer d.sessPool.put(se) if err != nil { @@ -379,10 +377,6 @@ func (d *ddl) loadBackfillJobAndRun() { } sess := newSession(se) - if err := ddlutil.LoadDDLReorgVars(context.Background(), sess); err != nil { - logutil.BgLogger().Error("[ddl] load DDL reorganization variable failed", zap.Error(err)) - } - runningJobIDs := d.backfillCtxJobIDs() if len(runningJobIDs) >= reorgWorkerCnt { d.backfillCtx.Unlock() @@ -394,9 +388,9 @@ func (d *ddl) loadBackfillJobAndRun() { bJobCnt := len(bJobs) if bJobCnt == 0 || err != nil { if err != nil { - logutil.BgLogger().Warn("[ddl] get backfill jobs failed", zap.Error(err)) + logutil.BgLogger().Warn("[ddl] get backfill jobs failed in a instance", zap.Error(err)) } else { - logutil.BgLogger().Debug("[ddl] get no backfill job") + logutil.BgLogger().Debug("[ddl] get no backfill job in a instance") } return } @@ -404,7 +398,7 @@ func (d *ddl) loadBackfillJobAndRun() { bJob := bJobs[0] jobCtx, existent := d.setBackfillCtxJobContext(bJob.JobID, bJob.Meta.Query, bJob.Meta.Type) if existent { - logutil.BgLogger().Warn("[ddl] get the type of backfill job is running", zap.String("backfill job", bJob.AbbrStr())) + logutil.BgLogger().Warn("[ddl] get the type of backfill job is running in a instance", zap.String("backfill job", bJob.AbbrStr())) return } // TODO: Adjust how the non-owner uses ReorgCtx. @@ -417,57 +411,22 @@ func (d *ddl) loadBackfillJobAndRun() { if bJob.Meta.ReorgTp == model.ReorgTypeLitMerge { if !ingest.LitInitialized { - logutil.BgLogger().Warn("[ddl] we can't do LitMerg", + logutil.BgLogger().Warn("[ddl] we can't do ingest in a instance", zap.Bool("LitInitialized", ingest.LitInitialized), zap.String("bJob", bJob.AbbrStr())) return } - logutil.BgLogger().Info("[ddl] run backfill job with LitMerge", zap.String("bJob", bJob.AbbrStr())) + logutil.BgLogger().Info("[ddl] run backfill jobs with ingest in a instance", zap.String("bJob", bJob.AbbrStr())) err = runBackfillJobsWithLightning(d, sess, bJob, jobCtx) } else { - logutil.BgLogger().Info("[ddl] run backfill job with TxnMerge", zap.String("bJob", bJob.AbbrStr())) + logutil.BgLogger().Info("[ddl] run backfill jobs with txn-merge in a instance", zap.String("bJob", bJob.AbbrStr())) _, err = runBackfillJobs(d, sess, bJob, jobCtx) } if err == nil { err = syncBackfillHistoryJobs(sess, d.uuid, bJob) } - if err != nil { - logutil.BgLogger().Warn("[ddl] run backfill job failed", zap.Error(err)) - } - }) -} - -func runBackfillJobs(d *ddl, sess *session, bJob *BackfillJob, jobCtx *JobContext) (table.Table, error) { - dbInfo, tbl, err := d.getTableByTxn(d.store, bJob.Meta.SchemaID, bJob.Meta.TableID) - if err != nil { - logutil.BgLogger().Warn("[ddl] runBackfillJobs gets table failed", zap.String("bfJob", bJob.AbbrStr()), zap.Error(err)) - return nil, err - } - - workerCnt := int(variable.GetDDLReorgWorkerCounter()) - // TODO: Different worker using different newBackfillerFunc. - workerCtx, err := newAddIndexWorkerContext(d, dbInfo.Name, tbl, workerCnt, bJob, jobCtx) - if err != nil || workerCtx == nil { - logutil.BgLogger().Info("[ddl] new adding index worker context failed", zap.Reflect("workerCtx", workerCtx), zap.Error(err)) - return nil, errors.Trace(err) - } - bwMgr := newBackfilWorkerManager(workerCtx) - d.backfillWorkerPool.SetConsumerFunc(func(task *reorgBackfillTask, _ int, bfWorker *backfillWorker) *backfillResult { - return bfWorker.runTask(task) + logutil.BgLogger().Warn("[ddl] run backfill jobs finished in a instance", zap.Stringer("reorg type", bJob.Meta.ReorgTp), zap.Error(err)) }) - proFunc := func() ([]*reorgBackfillTask, error) { - // TODO: After BackfillJob replaces reorgBackfillTask, use backfiller's GetTasks instead of it. - return GetTasks(d.ddlCtx, sess, tbl, bJob.JobID, workerCnt*2) - } - // add new task - resultCh, control := d.backfillWorkerPool.AddProduceBySlice(proFunc, 0, workerCtx, spmc.WithConcurrency(workerCnt)) - bwMgr.waitFinalResult(resultCh, control) - - // waiting task finishing - control.Wait() - err = bwMgr.close(d) - - return tbl, err } const ( @@ -830,12 +789,21 @@ func GetAndMarkBackfillJobsForOneEle(s *session, batch int, jobID int64, uuid st // GetInterruptedBackfillJobsForOneEle gets the interrupted backfill jobs in the tblName table that contains only one element. func GetInterruptedBackfillJobsForOneEle(sess *session, jobID, eleID int64, eleKey []byte) ([]*BackfillJob, error) { - bJobs, err := GetBackfillJobs(sess, BackfillTable, fmt.Sprintf("ddl_job_id = %d and ele_id = %d and ele_key = '%s' and state = %d", - jobID, eleID, eleKey, model.JobStateCancelled), "get_interrupt_backfill_job") - if err != nil || len(bJobs) == 0 { - return nil, err - } - return bJobs, nil + condition := fmt.Sprintf("ddl_job_id = %d and ele_id = %d and ele_key = '%s' and state = %d limit 1", + jobID, eleID, eleKey, model.JobStateCancelled) + var bJobs []*BackfillJob + err := sess.runInTxn(func(se *session) error { + var err1 error + bJobs, err1 = GetBackfillJobs(sess, BackfillTable, condition, "get_interrupt_backfill_job") + if err1 != nil || len(bJobs) != 0 { + return err1 + } + + bJobs, err1 = GetBackfillJobs(sess, BackfillHistoryTable, condition, "get_interrupt_backfill_job") + return err1 + }) + + return bJobs, err } // GetBackfillJobCount gets the number of rows in the tblName table according to condition. diff --git a/ddl/modify_column_test.go b/ddl/modify_column_test.go index 6eb8e633be007..43dac1b2437b6 100644 --- a/ddl/modify_column_test.go +++ b/ddl/modify_column_test.go @@ -53,8 +53,11 @@ func TestModifyColumnReorgInfo(t *testing.T) { originalTimeout := ddl.ReorgWaitTimeout ddl.ReorgWaitTimeout = 10 * time.Millisecond + limit := variable.GetDDLErrorCountLimit() + variable.SetDDLErrorCountLimit(5) defer func() { ddl.ReorgWaitTimeout = originalTimeout + variable.SetDDLErrorCountLimit(limit) }() tk := testkit.NewTestKit(t, store) tk.MustExec("use test") @@ -159,10 +162,16 @@ func TestModifyColumnReorgInfo(t *testing.T) { // Test encountering a "notOwnerErr" error which caused the processing backfill job to exit halfway. // During the period, the old TiDB version(do not exist the element information) is upgraded to the new TiDB version. require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/ddl/MockGetIndexRecordErr", `return("addIdxNotOwnerErr")`)) - tk.MustExec("alter table t1 add index idx2(c1)") - expectedElements = []*meta.Element{ - {ID: 7, TypeKey: meta.IndexElementKey}} - checkReorgHandle(elements, expectedElements) + // TODO: Remove this check after "err" isn't nil in runReorgJobAndHandleErr. + if variable.DDLEnableDistributeReorg.Load() { + err = tk.ExecToErr("alter table t1 add index idx2(c1)") + require.EqualError(t, err, "[ddl:8201]TiDB server is not a DDL owner") + } else { + tk.MustExec("alter table t1 add index idx2(c1)") + expectedElements = []*meta.Element{ + {ID: 7, TypeKey: meta.IndexElementKey}} + checkReorgHandle(elements, expectedElements) + } tk.MustExec("admin check table t1") require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/ddl/MockGetIndexRecordErr")) } diff --git a/ddl/reorg.go b/ddl/reorg.go index ed7a4ae8cdd6a..3eaa8080f1f73 100644 --- a/ddl/reorg.go +++ b/ddl/reorg.go @@ -338,7 +338,7 @@ func getTableTotalCount(w *worker, tblInfo *model.TableInfo) int64 { return rows[0].GetInt64(0) } -func (dc *ddlCtx) isReorgRunnable(jobID int64) error { +func (dc *ddlCtx) isReorgRunnable(jobID int64, isDistReorg bool) error { if isChanClosed(dc.ctx.Done()) { // Worker is closed. So it can't do the reorganization. return dbterror.ErrInvalidWorker.GenWithStack("worker is closed") @@ -349,6 +349,10 @@ func (dc *ddlCtx) isReorgRunnable(jobID int64) error { return dbterror.ErrCancelledDDLJob } + // If isDistReorg is true, we needn't check if it is owner. + if isDistReorg { + return nil + } if !dc.isOwner() { // If it's not the owner, we will try later, so here just returns an error. logutil.BgLogger().Info("[ddl] DDL is not the DDL owner", zap.String("ID", dc.uuid)) From 6d2d1e8e3735d7a03dd5e99fdd1b9edcfd0ab527 Mon Sep 17 00:00:00 2001 From: Lynn Date: Wed, 18 Jan 2023 22:15:05 +0800 Subject: [PATCH 10/25] ddl: address comments and tiny update --- ddl/dist_backfilling.go | 2 +- ddl/dist_owner.go | 2 +- ddl/index.go | 1 - ddl/job_table.go | 57 +++++++++++++++++------------------------ ddl/job_table_test.go | 17 ++++++------ 5 files changed, 33 insertions(+), 46 deletions(-) diff --git a/ddl/dist_backfilling.go b/ddl/dist_backfilling.go index 9e27635078767..32f88f9adec61 100644 --- a/ddl/dist_backfilling.go +++ b/ddl/dist_backfilling.go @@ -118,7 +118,7 @@ func runBackfillJobs(d *ddl, sess *session, bJob *BackfillJob, jobCtx *JobContex }) proFunc := func() ([]*reorgBackfillTask, error) { // TODO: After BackfillJob replaces reorgBackfillTask, use backfiller's GetTasks instead of it. - return GetTasks(d.ddlCtx, sess, tbl, bJob.JobID, workerCnt*2) + return GetTasks(d.ddlCtx, sess, tbl, bJob.JobID, workerCnt+5) } // add new task resultCh, control := d.backfillWorkerPool.AddProduceBySlice(proFunc, 0, workerCtx, spmc.WithConcurrency(workerCnt)) diff --git a/ddl/dist_owner.go b/ddl/dist_owner.go index 7fb34c0938774..9ff3249116867 100644 --- a/ddl/dist_owner.go +++ b/ddl/dist_owner.go @@ -89,12 +89,12 @@ func checkReorgJobFinished(ctx context.Context, sess *session, reorgCtxs *reorgC select { case <-ticker.C: + times++ // Print this log every 5 min. if times%1000 == 0 { logutil.BgLogger().Info("[ddl] check all backfill jobs is finished", zap.Int64("job ID", ddlJobID), zap.Bool("isFinished", backfillJobFinished), zap.Reflect("bfJob", bfJob)) } - times++ if !backfillJobFinished { err := checkAndHandleInterruptedBackfillJobs(sess, ddlJobID, currEle.ID, currEle.TypeKey) if err != nil { diff --git a/ddl/index.go b/ddl/index.go index cfc15bb8617a3..8867512321404 100644 --- a/ddl/index.go +++ b/ddl/index.go @@ -2050,7 +2050,6 @@ func runBackfillJobsWithLightning(d *ddl, sess *session, bfJob *BackfillJob, job // TODO: Consider redo it. bc, ok := ingest.LitBackCtxMgr.Load(bfJob.JobID) if ok && bc.Done() { - logutil.BgLogger().Warn("[ddl] lightning loaded") return errors.New(ingest.LitErrGetBackendFail) } var err error diff --git a/ddl/job_table.go b/ddl/job_table.go index 3b898bcf2c21f..5169e0bf02ee5 100644 --- a/ddl/job_table.go +++ b/ddl/job_table.go @@ -379,53 +379,51 @@ func (d *ddl) loadBackfillJobAndRun() { runningJobIDs := d.backfillCtxJobIDs() if len(runningJobIDs) >= reorgWorkerCnt { - d.backfillCtx.Unlock() return } // TODO: Add ele info to distinguish backfill jobs. - bJobs, err := GetBackfillJobsForOneEle(sess, 1, runningJobIDs, InstanceLease) - bJobCnt := len(bJobs) - if bJobCnt == 0 || err != nil { + // Get a Backfill job to get the reorg info like element info, schema ID and so on. + bfJob, err := GetBackfillJobForOneEle(sess, runningJobIDs, InstanceLease) + if err != nil || bfJob == nil { if err != nil { - logutil.BgLogger().Warn("[ddl] get backfill jobs failed in a instance", zap.Error(err)) + logutil.BgLogger().Warn("[ddl] get backfill jobs failed in this instance", zap.Error(err)) } else { - logutil.BgLogger().Debug("[ddl] get no backfill job in a instance") + logutil.BgLogger().Debug("[ddl] get no backfill job in this instance") } return } - bJob := bJobs[0] - jobCtx, existent := d.setBackfillCtxJobContext(bJob.JobID, bJob.Meta.Query, bJob.Meta.Type) + jobCtx, existent := d.setBackfillCtxJobContext(bfJob.JobID, bfJob.Meta.Query, bfJob.Meta.Type) if existent { - logutil.BgLogger().Warn("[ddl] get the type of backfill job is running in a instance", zap.String("backfill job", bJob.AbbrStr())) + logutil.BgLogger().Warn("[ddl] get the type of backfill job is running in this instance", zap.String("backfill job", bfJob.AbbrStr())) return } // TODO: Adjust how the non-owner uses ReorgCtx. - d.setReorgCtxForBackfill(bJob) + d.setReorgCtxForBackfill(bfJob) d.wg.Run(func() { defer func() { - d.removeBackfillCtxJobCtx(bJob.JobID) + d.removeBackfillCtxJobCtx(bfJob.JobID) tidbutil.Recover(metrics.LabelBackfillWorker, "runBackfillJobs", nil, false) }() - if bJob.Meta.ReorgTp == model.ReorgTypeLitMerge { + if bfJob.Meta.ReorgTp == model.ReorgTypeLitMerge { if !ingest.LitInitialized { - logutil.BgLogger().Warn("[ddl] we can't do ingest in a instance", - zap.Bool("LitInitialized", ingest.LitInitialized), zap.String("bJob", bJob.AbbrStr())) + logutil.BgLogger().Warn("[ddl] we can't do ingest in this instance", + zap.Bool("LitInitialized", ingest.LitInitialized), zap.String("bfJob", bfJob.AbbrStr())) return } - logutil.BgLogger().Info("[ddl] run backfill jobs with ingest in a instance", zap.String("bJob", bJob.AbbrStr())) - err = runBackfillJobsWithLightning(d, sess, bJob, jobCtx) + logutil.BgLogger().Info("[ddl] run backfill jobs with ingest in this instance", zap.String("bfJob", bfJob.AbbrStr())) + err = runBackfillJobsWithLightning(d, sess, bfJob, jobCtx) } else { - logutil.BgLogger().Info("[ddl] run backfill jobs with txn-merge in a instance", zap.String("bJob", bJob.AbbrStr())) - _, err = runBackfillJobs(d, sess, bJob, jobCtx) + logutil.BgLogger().Info("[ddl] run backfill jobs with txn-merge in this instance", zap.String("bfJob", bfJob.AbbrStr())) + _, err = runBackfillJobs(d, sess, bfJob, jobCtx) } if err == nil { - err = syncBackfillHistoryJobs(sess, d.uuid, bJob) + err = syncBackfillHistoryJobs(sess, d.uuid, bfJob) } - logutil.BgLogger().Warn("[ddl] run backfill jobs finished in a instance", zap.Stringer("reorg type", bJob.Meta.ReorgTp), zap.Error(err)) + logutil.BgLogger().Warn("[ddl] run backfill jobs finished in this instance", zap.Stringer("reorg type", bfJob.Meta.ReorgTp), zap.Error(err)) }) } @@ -696,8 +694,8 @@ func AddBackfillJobs(s *session, backfillJobs []*BackfillJob) error { }) } -// GetBackfillJobsForOneEle batch gets the backfill jobs in the tblName table that contains only one element. -func GetBackfillJobsForOneEle(s *session, batch int, excludedJobIDs []int64, lease time.Duration) ([]*BackfillJob, error) { +// GetBackfillJobForOneEle gets the backfill jobs in the tblName table that contains only one element. +func GetBackfillJobForOneEle(s *session, excludedJobIDs []int64, lease time.Duration) (*BackfillJob, error) { eJobIDsBuilder := strings.Builder{} for i, id := range excludedJobIDs { if i == 0 { @@ -721,24 +719,15 @@ func GetBackfillJobsForOneEle(s *session, batch int, excludedJobIDs []int64, lea leaseStr := currTime.Add(-lease).Format(types.TimeFormat) bJobs, err = GetBackfillJobs(se, BackfillTable, - fmt.Sprintf("(exec_ID = '' or exec_lease < '%v') %s order by ddl_job_id, ele_key, ele_id limit %d", - leaseStr, eJobIDsBuilder.String(), batch), "get_backfill_job") + fmt.Sprintf("(exec_ID = '' or exec_lease < '%v') %s order by ddl_job_id, ele_key, ele_id limit 1", + leaseStr, eJobIDsBuilder.String()), "get_backfill_job") return err }) if err != nil || len(bJobs) == 0 { return nil, err } - validLen := 1 - firstJobID, firstEleID, firstEleKey := bJobs[0].JobID, bJobs[0].EleID, bJobs[0].EleKey - for i := 1; i < len(bJobs); i++ { - if bJobs[i].JobID != firstJobID || bJobs[i].EleID != firstEleID || !bytes.Equal(bJobs[i].EleKey, firstEleKey) { - break - } - validLen++ - } - - return bJobs[:validLen], nil + return bJobs[0], nil } // GetAndMarkBackfillJobsForOneEle batch gets the backfill jobs in the tblName table that contains only one element, diff --git a/ddl/job_table_test.go b/ddl/job_table_test.go index d9223161a17f9..60423c73e6dd9 100644 --- a/ddl/job_table_test.go +++ b/ddl/job_table_test.go @@ -259,10 +259,10 @@ func TestSimpleExecBackfillJobs(t *testing.T) { instanceLease := ddl.InstanceLease // test no backfill job - bJobs, err := ddl.GetBackfillJobsForOneEle(se, 1, []int64{jobID1, jobID2}, instanceLease) + bJob, err := ddl.GetBackfillJobForOneEle(se, []int64{jobID1, jobID2}, instanceLease) require.NoError(t, err) - require.Nil(t, bJobs) - bJobs, err = ddl.GetAndMarkBackfillJobsForOneEle(se, 1, jobID1, uuid, instanceLease) + require.Nil(t, bJob) + bJobs, err := ddl.GetAndMarkBackfillJobsForOneEle(se, 1, jobID1, uuid, instanceLease) require.EqualError(t, err, dbterror.ErrDDLJobNotFound.FastGen("get zero backfill job").Error()) require.Nil(t, bJobs) allCnt, err := ddl.GetBackfillJobCount(se, ddl.BackfillTable, getIdxConditionStr(jobID1, eleID2), "check_backfill_job_count") @@ -291,14 +291,13 @@ func TestSimpleExecBackfillJobs(t *testing.T) { // 1 jobID2 eleID3 "" require.NoError(t, err) // test get some backfill jobs - bJobs, err = ddl.GetBackfillJobsForOneEle(se, 1, []int64{jobID2 - 1, jobID2 + 1}, instanceLease) + bJob, err = ddl.GetBackfillJobForOneEle(se, []int64{jobID2 - 1, jobID2 + 1}, instanceLease) require.NoError(t, err) - require.Len(t, bJobs, 1) expectJob := bjTestCases[2] - if expectJob.ID != bJobs[0].ID { + if expectJob.ID != bJob.ID { expectJob = bjTestCases[3] } - require.Equal(t, expectJob, bJobs[0]) + require.Equal(t, expectJob, bJob) previousTime, err := ddl.GetOracleTimeWithStartTS(se) require.EqualError(t, err, "[kv:8024]invalid transaction") readInTxn(se, func(sessionctx.Context) { @@ -386,7 +385,7 @@ func TestSimpleExecBackfillJobs(t *testing.T) { require.Nil(t, bjob) bJobs, err = ddl.GetInterruptedBackfillJobsForOneEle(se, jobID1, eleID1, eleKey) require.NoError(t, err) - require.Nil(t, bJobs) + require.Len(t, bJobs, 0) err = ddl.AddBackfillJobs(se, bjTestCases) require.NoError(t, err) // ID jobID eleID @@ -402,7 +401,7 @@ func TestSimpleExecBackfillJobs(t *testing.T) { require.Equal(t, bJobs2[1], bjob) bJobs, err = ddl.GetInterruptedBackfillJobsForOneEle(se, jobID1, eleID1, eleKey) require.NoError(t, err) - require.Nil(t, bJobs) + require.Len(t, bJobs, 0) bJobs1[0].State = model.JobStateRollingback bJobs1[0].ID = 2 bJobs1[0].InstanceID = uuid From 81e8c5b75a4c7e2957b588966e037646ec10e769 Mon Sep 17 00:00:00 2001 From: Lynn Date: Thu, 19 Jan 2023 20:00:46 +0800 Subject: [PATCH 11/25] ddl: make merge happy --- ddl/index.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/ddl/index.go b/ddl/index.go index f765e7a7fba89..df500cf70ac83 100644 --- a/ddl/index.go +++ b/ddl/index.go @@ -1328,7 +1328,7 @@ func newAddIndexWorker(decodeColMap map[int64]decoder.Column, t table.PhysicalTa if err != nil { return nil, errors.Trace(err) } - lwCtx, err = ei.NewWriterCtx(bfCtx.id) + lwCtx, err = ei.NewWriterCtx(bfCtx.id, indexInfo.Unique) if err != nil { return nil, err } From bbae32b1837d9727e1203bf817a6798b639bfc41 Mon Sep 17 00:00:00 2001 From: Lynn Date: Fri, 20 Jan 2023 11:06:51 +0800 Subject: [PATCH 12/25] ddl: make TestAddIndexMergeConflictWithPessimistic correct and stable --- ddl/ddl_test.go | 10 ++++++++++ ddl/dist_owner.go | 4 +++- ddl/index_merge_tmp_test.go | 16 +++++++++++++--- 3 files changed, 26 insertions(+), 4 deletions(-) diff --git a/ddl/ddl_test.go b/ddl/ddl_test.go index dc19436afd9b9..ee17dde7e8bbd 100644 --- a/ddl/ddl_test.go +++ b/ddl/ddl_test.go @@ -63,6 +63,16 @@ func GetDDLCtx(d DDL) *ddlCtx { return d.(*ddl).ddlCtx } +// SetCheckInterval sets checkBackfillJobFinishInterval, it's for test. +func SetCheckInterval(interval time.Duration) { + checkBackfillJobFinishInterval = interval +} + +// GetCheckInterval gets checkBackfillJobFinishInterval, it's for test. +func GetCheckInterval() time.Duration { + return checkBackfillJobFinishInterval +} + // GetMaxRowID is used for test. func GetMaxRowID(store kv.Storage, priority int, t table.Table, startHandle, endHandle kv.Key) (kv.Key, error) { return getRangeEndKey(NewJobContext(), store, priority, t.RecordPrefix(), startHandle, endHandle) diff --git a/ddl/dist_owner.go b/ddl/dist_owner.go index 9ff3249116867..b04e871987639 100644 --- a/ddl/dist_owner.go +++ b/ddl/dist_owner.go @@ -30,6 +30,8 @@ import ( "go.uber.org/zap" ) +var checkBackfillJobFinishInterval = 300 * time.Millisecond + func (dc *ddlCtx) controlWritePhysicalTableRecord(sess *session, t table.PhysicalTable, bfWorkerType backfillerType, reorgInfo *reorgInfo) error { startKey, endKey := reorgInfo.StartKey, reorgInfo.EndKey if startKey == nil && endKey == nil { @@ -79,7 +81,7 @@ func checkReorgJobFinished(ctx context.Context, sess *session, reorgCtxs *reorgC var times int64 var bfJob *BackfillJob var backfillJobFinished bool - ticker := time.NewTicker(300 * time.Millisecond) + ticker := time.NewTicker(checkBackfillJobFinishInterval) defer ticker.Stop() for { if getReorgCtx(reorgCtxs, ddlJobID).isReorgCanceled() { diff --git a/ddl/index_merge_tmp_test.go b/ddl/index_merge_tmp_test.go index ae44e0544dc51..dba1eb9c5d1b8 100644 --- a/ddl/index_merge_tmp_test.go +++ b/ddl/index_merge_tmp_test.go @@ -16,6 +16,7 @@ package ddl_test import ( "testing" + "time" "github.com/pingcap/failpoint" "github.com/pingcap/tidb/ddl" @@ -478,6 +479,11 @@ func TestAddIndexMergeConflictWithPessimistic(t *testing.T) { tk.MustExec(`CREATE TABLE t (id int primary key, a int);`) tk.MustExec(`INSERT INTO t VALUES (1, 1);`) + // Make shorten the conversion time from ReorgTypeLitMerge to BackfillStateReadyToMerge. + interval := ddl.GetCheckInterval() + ddl.SetCheckInterval(50 * time.Millisecond) + defer ddl.SetCheckInterval(interval) + // Force onCreateIndex use the txn-merge process. ingest.LitInitialized = false tk.MustExec("set @@global.tidb_ddl_enable_fast_reorg = 1;") @@ -487,7 +493,6 @@ func TestAddIndexMergeConflictWithPessimistic(t *testing.T) { callback := &ddl.TestDDLCallback{Do: dom} runPessimisticTxn := false - doRollbackCh := make(chan struct{}, 1) callback.OnJobRunBeforeExported = func(job *model.Job) { if t.Failed() { return @@ -510,7 +515,6 @@ func TestAddIndexMergeConflictWithPessimistic(t *testing.T) { assert.NoError(t, err) _, err = tk2.Exec("update t set a = 3 where id = 1;") assert.NoError(t, err) - doRollbackCh <- struct{}{} } } dom.DDL().SetHook(callback) @@ -519,7 +523,13 @@ func TestAddIndexMergeConflictWithPessimistic(t *testing.T) { tk.MustExec("alter table t add index idx(a);") afterCommit <- struct{}{} }() - <-doRollbackCh + timer := time.NewTimer(300 * time.Millisecond) + select { + case <-timer.C: + break + case <-afterCommit: + require.Fail(t, "should be blocked by the pessimistic txn") + } tk2.MustExec("rollback;") <-afterCommit dom.DDL().SetHook(originHook) From 618b4ba7788f2012c653ea16833b516093ac3234 Mon Sep 17 00:00:00 2001 From: Lynn Date: Sat, 21 Jan 2023 00:30:14 +0800 Subject: [PATCH 13/25] *: make InTest worker and tiny update --- br/pkg/mock/mock_cluster.go | 5 --- br/pkg/restore/db_test.go | 5 --- ddl/ddl.go | 3 +- ddl/dist_owner.go | 2 +- ddl/job_table.go | 27 +++++-------- ddl/job_table_test.go | 56 +++++++++++++------------- resourcemanager/util/shard_pool_map.go | 14 ++++++- 7 files changed, 53 insertions(+), 59 deletions(-) diff --git a/br/pkg/mock/mock_cluster.go b/br/pkg/mock/mock_cluster.go index a24715555bb02..41caff5345aa6 100644 --- a/br/pkg/mock/mock_cluster.go +++ b/br/pkg/mock/mock_cluster.go @@ -18,8 +18,6 @@ import ( "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/domain" "github.com/pingcap/tidb/kv" - "github.com/pingcap/tidb/resourcemanager" - "github.com/pingcap/tidb/resourcemanager/util" "github.com/pingcap/tidb/server" "github.com/pingcap/tidb/session" "github.com/pingcap/tidb/store/mockstore" @@ -74,7 +72,6 @@ func NewCluster() (*Cluster, error) { session.SetSchemaLease(0) session.DisableStats4Test() - util.InTest.Store(true) dom, err := session.BootstrapSession(storage) if err != nil { return nil, errors.Trace(err) @@ -126,8 +123,6 @@ func (mock *Cluster) Stop() { _ = mock.HttpServer.Close() } view.Stop() - util.InTest.Store(true) - resourcemanager.GlobalResourceManager.Reset() } type configOverrider func(*mysql.Config) diff --git a/br/pkg/restore/db_test.go b/br/pkg/restore/db_test.go index 2d7d932104dea..3a5416501e4df 100644 --- a/br/pkg/restore/db_test.go +++ b/br/pkg/restore/db_test.go @@ -23,8 +23,6 @@ import ( "github.com/pingcap/tidb/parser/model" "github.com/pingcap/tidb/parser/mysql" "github.com/pingcap/tidb/parser/types" - "github.com/pingcap/tidb/resourcemanager" - "github.com/pingcap/tidb/resourcemanager/util" "github.com/pingcap/tidb/testkit" "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" @@ -38,7 +36,6 @@ type testRestoreSchemaSuite struct { } func createRestoreSchemaSuite(t *testing.T) *testRestoreSchemaSuite { - util.InTest.Store(true) var err error s := new(testRestoreSchemaSuite) s.mockGlue = &gluetidb.MockGlue{} @@ -50,8 +47,6 @@ func createRestoreSchemaSuite(t *testing.T) *testRestoreSchemaSuite { require.NoError(t, s.mock.Start()) t.Cleanup(func() { s.mock.Stop() - resourcemanager.GlobalResourceManager.Reset() - util.InTest.Store(false) }) return s } diff --git a/ddl/ddl.go b/ddl/ddl.go index c7f0a74d98c03..1e53878456799 100644 --- a/ddl/ddl.go +++ b/ddl/ddl.go @@ -65,6 +65,7 @@ import ( "github.com/pingcap/tidb/util/logutil" "github.com/pingcap/tidb/util/mathutil" "github.com/pingcap/tidb/util/sqlexec" + "github.com/pingcap/tidb/util/syncutil" "github.com/tikv/client-go/v2/tikvrpc" clientv3 "go.etcd.io/etcd/client/v3" atomicutil "go.uber.org/atomic" @@ -351,7 +352,7 @@ type ddlCtx struct { reorgCtx reorgContexts // backfillCtx is used for backfill workers. backfillCtx struct { - sync.RWMutex + syncutil.RWMutex jobCtxMap map[int64]*JobContext backfillCtxMap map[int64]struct{} } diff --git a/ddl/dist_owner.go b/ddl/dist_owner.go index b04e871987639..2642f7fd9ebb7 100644 --- a/ddl/dist_owner.go +++ b/ddl/dist_owner.go @@ -175,7 +175,7 @@ func GetBackfillErr(sess *session, ddlJobID, currEleID int64, currEleKey []byte) func checkAndHandleInterruptedBackfillJobs(sess *session, ddlJobID, currEleID int64, currEleKey []byte) (err error) { var bJobs []*BackfillJob for i := 0; i < retrySQLTimes; i++ { - bJobs, err = GetInterruptedBackfillJobsForOneEle(sess, ddlJobID, currEleID, currEleKey) + bJobs, err = GetInterruptedBackfillJobForOneEle(sess, ddlJobID, currEleID, currEleKey) if err == nil { break } diff --git a/ddl/job_table.go b/ddl/job_table.go index 5169e0bf02ee5..f54acb789d788 100644 --- a/ddl/job_table.go +++ b/ddl/job_table.go @@ -631,7 +631,7 @@ func getJobsBySQL(sess *session, tbl, condition string) ([]*model.Job, error) { func syncBackfillHistoryJobs(sess *session, uuid string, backfillJob *BackfillJob) error { sql := fmt.Sprintf("update mysql.%s set state = %d where ddl_job_id = %d and ele_id = %d and ele_key = %s and exec_id = '%s' limit 1;", - BackfillHistoryTable, model.JobStateSynced, backfillJob.JobID, backfillJob.EleID, wrapKey2String(backfillJob.EleKey), uuid) + BackfillHistoryTable, model.JobStateSynced, backfillJob.JobID, backfillJob.EleID, (backfillJob.EleKey), uuid) _, err := sess.execute(context.Background(), sql, "sync_backfill_history_job") return err } @@ -776,23 +776,14 @@ func GetAndMarkBackfillJobsForOneEle(s *session, batch int, jobID int64, uuid st return bJobs[:validLen], err } -// GetInterruptedBackfillJobsForOneEle gets the interrupted backfill jobs in the tblName table that contains only one element. -func GetInterruptedBackfillJobsForOneEle(sess *session, jobID, eleID int64, eleKey []byte) ([]*BackfillJob, error) { - condition := fmt.Sprintf("ddl_job_id = %d and ele_id = %d and ele_key = '%s' and state = %d limit 1", - jobID, eleID, eleKey, model.JobStateCancelled) - var bJobs []*BackfillJob - err := sess.runInTxn(func(se *session) error { - var err1 error - bJobs, err1 = GetBackfillJobs(sess, BackfillTable, condition, "get_interrupt_backfill_job") - if err1 != nil || len(bJobs) != 0 { - return err1 - } - - bJobs, err1 = GetBackfillJobs(sess, BackfillHistoryTable, condition, "get_interrupt_backfill_job") - return err1 - }) - - return bJobs, err +// GetInterruptedBackfillJobForOneEle gets a interrupted backfill job in the tblName table that contains only one element. +func GetInterruptedBackfillJobForOneEle(sess *session, jobID, eleID int64, eleKey []byte) ([]*BackfillJob, error) { + bJobs, err := GetBackfillJobs(sess, BackfillHistoryTable, fmt.Sprintf("ddl_job_id = %d and ele_id = %d and ele_key = %s and state = %d limit 1", + jobID, eleID, wrapKey2String(eleKey), model.JobStateCancelled), "get_interrupt_backfill_job") + if err != nil || len(bJobs) == 0 { + return nil, err + } + return bJobs, nil } // GetBackfillJobCount gets the number of rows in the tblName table according to condition. diff --git a/ddl/job_table_test.go b/ddl/job_table_test.go index 60423c73e6dd9..105220c1d0c0f 100644 --- a/ddl/job_table_test.go +++ b/ddl/job_table_test.go @@ -228,8 +228,8 @@ func equalBackfillJob(t *testing.T, a, b *ddl.BackfillJob, lessTime types.Time) } func getIdxConditionStr(jobID, eleID int64) string { - return fmt.Sprintf("ddl_job_id = %d and ele_id = %d and ele_key = '%s'", - jobID, eleID, meta.IndexElementKey) + return fmt.Sprintf("ddl_job_id = %d and ele_id = %d and ele_key = %s", + jobID, eleID, wrapKey2String(meta.IndexElementKey)) } func readInTxn(se sessionctx.Context, f func(sessionctx.Context)) (err error) { @@ -379,13 +379,10 @@ func TestSimpleExecBackfillJobs(t *testing.T) { require.Len(t, bJobs, 1) require.Equal(t, bJobs[0].FinishTS, uint64(0)) - // test GetMaxBackfillJob and GetInterruptedBackfillJobsForOneEle + // test GetMaxBackfillJob bjob, err := ddl.GetMaxBackfillJob(se, bJobs3[0].JobID, bJobs3[0].EleID, eleKey) require.NoError(t, err) require.Nil(t, bjob) - bJobs, err = ddl.GetInterruptedBackfillJobsForOneEle(se, jobID1, eleID1, eleKey) - require.NoError(t, err) - require.Len(t, bJobs, 0) err = ddl.AddBackfillJobs(se, bjTestCases) require.NoError(t, err) // ID jobID eleID @@ -399,9 +396,6 @@ func TestSimpleExecBackfillJobs(t *testing.T) { bjob, err = ddl.GetMaxBackfillJob(se, jobID2, eleID2, eleKey) require.NoError(t, err) require.Equal(t, bJobs2[1], bjob) - bJobs, err = ddl.GetInterruptedBackfillJobsForOneEle(se, jobID1, eleID1, eleKey) - require.NoError(t, err) - require.Len(t, bJobs, 0) bJobs1[0].State = model.JobStateRollingback bJobs1[0].ID = 2 bJobs1[0].InstanceID = uuid @@ -423,10 +417,6 @@ func TestSimpleExecBackfillJobs(t *testing.T) { bjob, err = ddl.GetMaxBackfillJob(se, jobID1, eleID1, eleKey) require.NoError(t, err) require.Equal(t, bJobs1[1], bjob) - bJobs, err = ddl.GetInterruptedBackfillJobsForOneEle(se, jobID1, eleID1, eleKey) - require.NoError(t, err) - require.Len(t, bJobs, 1) - equalBackfillJob(t, bJobs1[1], bJobs[0], types.ZeroTimestamp) // test the BackfillJob's AbbrStr require.Equal(t, fmt.Sprintf("ID:2, JobID:1, EleID:11, Type:add index, State:rollingback, InstanceID:%s, InstanceLease:0000-00-00 00:00:00", uuid), bJobs1[0].AbbrStr()) require.Equal(t, "ID:3, JobID:1, EleID:11, Type:add index, State:cancelled, InstanceID:, InstanceLease:0000-00-00 00:00:00", bJobs1[1].AbbrStr()) @@ -458,7 +448,7 @@ func TestSimpleExecBackfillJobs(t *testing.T) { // 0 jobID2 eleID3 JobStateNone // 1 jobID2 eleID3 JobStateNone // 2 jobID1 eleID1 JobStateRollingback - // 3 jobID1 eleID1 JobStateCancelling + // 3 jobID1 eleID1 JobStateCancelled // // BackfillHistoryTable // ID jobID eleID state @@ -481,7 +471,7 @@ func TestSimpleExecBackfillJobs(t *testing.T) { // 0 jobID2 eleID3 JobStateNone // 1 jobID2 eleID3 JobStateNone // 2 jobID1 eleID1 JobStateRollingback - // 3 jobID1 eleID1 JobStateCancelling + // 3 jobID1 eleID1 JobStateCancelled // 6 jobID1 eleID1 JobStateNone // 7 jobID1 eleID1 JobStateNone // @@ -494,7 +484,7 @@ func TestSimpleExecBackfillJobs(t *testing.T) { require.NoError(t, err) require.Equal(t, bJobs1[1], bjob) - // test MoveBackfillJobsToHistoryTable + // test MoveBackfillJobsToHistoryTable and GetInterruptedBackfillJobForOneEle allCnt, err = ddl.GetBackfillJobCount(se, ddl.BackfillTable, getIdxConditionStr(jobID2, eleID3), "test_get_bj") require.NoError(t, err) require.Equal(t, allCnt, 2) @@ -514,7 +504,7 @@ func TestSimpleExecBackfillJobs(t *testing.T) { // 0 jobID2 eleID2 JobStateNone // 1 jobID2 eleID2 JobStateNone // 2 jobID1 eleID1 JobStateRollingback - // 3 jobID1 eleID1 JobStateCancelling + // 3 jobID1 eleID1 JobStateCancelled // 6 jobID1 eleID1 JobStateNone // 7 jobID1 eleID1 JobStateNone // @@ -523,8 +513,11 @@ func TestSimpleExecBackfillJobs(t *testing.T) { // -------------------------------- // 5 jobID1 eleID1 JobStateNone // 4 jobID1 eleID1 JobStateNone - // 0 jobID2 eleID3 JobStateNone - // 1 jobID2 eleID3 JobStateNone + // 0 jobID2 eleID3 JobStateCancelled + // 1 jobID2 eleID3 JobStateCancelled + bJobs, err = ddl.GetInterruptedBackfillJobForOneEle(se, jobID1, eleID1, eleKey) + require.NoError(t, err) + require.Len(t, bJobs, 0) allCnt, err = ddl.GetBackfillJobCount(se, ddl.BackfillTable, getIdxConditionStr(jobID1, eleID1), "test_get_bj") require.NoError(t, err) require.Equal(t, allCnt, 6) @@ -536,6 +529,15 @@ func TestSimpleExecBackfillJobs(t *testing.T) { allCnt, err = ddl.GetBackfillJobCount(se, ddl.BackfillHistoryTable, getIdxConditionStr(jobID1, eleID1), "test_get_bj") require.NoError(t, err) require.Equal(t, allCnt, 8) + bJobs, err = ddl.GetInterruptedBackfillJobForOneEle(se, jobID2, eleID3, eleKey) + require.NoError(t, err) + require.Len(t, bJobs, 1) + expectJob = bJobs3[0] + if expectJob.ID != bJob.ID { + expectJob = bJobs3[1] + } + expectJob.State = model.JobStateCancelled + equalBackfillJob(t, bJobs3[0], bJobs[0], types.ZeroTimestamp) // BackfillTable // ID jobID eleID state // -------------------------------- @@ -547,14 +549,14 @@ func TestSimpleExecBackfillJobs(t *testing.T) { // -------------------------------- // 5 jobID1 eleID1 JobStateNone // 4 jobID1 eleID1 JobStateNone - // 0 jobID2 eleID3 JobStateNone - // 1 jobID2 eleID3 JobStateNone - // 0 jobID1 eleID1 JobStateNone - // 1 jobID1 eleID1 JobStateNone - // 2 jobID1 eleID1 JobStateRollingback - // 3 jobID1 eleID1 JobStateCancelling - // 6 jobID1 eleID1 JobStateNone - // 7 jobID1 eleID1 JobStateNone + // 0 jobID2 eleID3 JobStateCancelled + // 1 jobID2 eleID3 JobStateCancelled + // 0 jobID1 eleID1 JobStateCancelled + // 1 jobID1 eleID1 JobStateCancelled + // 2 jobID1 eleID1 JobStateCancelled + // 3 jobID1 eleID1 JobStateCancelled + // 6 jobID1 eleID1 JobStateCancelled + // 7 jobID1 eleID1 JobStateCancelled } func TestGetTasks(t *testing.T) { diff --git a/resourcemanager/util/shard_pool_map.go b/resourcemanager/util/shard_pool_map.go index 7efd663c3154f..d6a4d11d8b48d 100644 --- a/resourcemanager/util/shard_pool_map.go +++ b/resourcemanager/util/shard_pool_map.go @@ -15,7 +15,8 @@ package util import ( - "flag" + "os" + "strings" "sync" "sync/atomic" @@ -25,8 +26,17 @@ import ( // InTest is a flag to indicate whether the code is running in test. var InTest atomic.Bool +func isInTests() bool { + for _, arg := range os.Args { + if strings.HasPrefix(arg, "-test.v=") { + return true + } + } + return false +} + func init() { - if flag.Lookup("test.v") != nil { + if isInTests() { InTest.Store(true) } } From 36617f410246d04ff454cb51c7a4ed4568107542 Mon Sep 17 00:00:00 2001 From: Lynn Date: Mon, 30 Jan 2023 15:23:58 +0800 Subject: [PATCH 14/25] *: make bazel happy, tiny update and fix primary key issue --- ddl/backfilling.go | 5 +++++ ddl/ddl_test.go | 10 ---------- ddl/dist_backfilling.go | 17 ++++++++++++++--- ddl/dist_owner.go | 23 +++++++++++++++++++++-- ddl/index.go | 21 +++++++++------------ ddl/indexmergetest/merge_test.go | 6 +++--- ddl/job_table.go | 4 ++-- executor/analyzetest/BUILD.bazel | 1 - executor/analyzetest/analyze_test.go | 2 -- parser/model/ddl.go | 10 ++++++++++ statistics/handle/BUILD.bazel | 1 - statistics/handle/update_test.go | 2 -- 12 files changed, 64 insertions(+), 38 deletions(-) diff --git a/ddl/backfilling.go b/ddl/backfilling.go index 427bc2b82d7dd..9141f97a694d3 100644 --- a/ddl/backfilling.go +++ b/ddl/backfilling.go @@ -839,6 +839,11 @@ func (b *backfillScheduler) newSessCtx() (sessionctx.Context, error) { } func initSessCtx(sessCtx sessionctx.Context, sqlMode mysql.SQLMode, tzLocation *model.TimeZoneLocation) error { + // Unify the TimeZone settings in newContext. + if sessCtx.GetSessionVars().StmtCtx.TimeZone == nil { + tz := *time.UTC + sessCtx.GetSessionVars().StmtCtx.TimeZone = &tz + } sessCtx.GetSessionVars().StmtCtx.IsDDLJobInQueue = true // Set the row encode format version. rowFormat := variable.GetDDLReorgRowFormat() diff --git a/ddl/ddl_test.go b/ddl/ddl_test.go index ee17dde7e8bbd..dc19436afd9b9 100644 --- a/ddl/ddl_test.go +++ b/ddl/ddl_test.go @@ -63,16 +63,6 @@ func GetDDLCtx(d DDL) *ddlCtx { return d.(*ddl).ddlCtx } -// SetCheckInterval sets checkBackfillJobFinishInterval, it's for test. -func SetCheckInterval(interval time.Duration) { - checkBackfillJobFinishInterval = interval -} - -// GetCheckInterval gets checkBackfillJobFinishInterval, it's for test. -func GetCheckInterval() time.Duration { - return checkBackfillJobFinishInterval -} - // GetMaxRowID is used for test. func GetMaxRowID(store kv.Storage, priority int, t table.Table, startHandle, endHandle kv.Key) (kv.Key, error) { return getRangeEndKey(NewJobContext(), store, priority, t.RecordPrefix(), startHandle, endHandle) diff --git a/ddl/dist_backfilling.go b/ddl/dist_backfilling.go index 32f88f9adec61..8b698f9a69e36 100644 --- a/ddl/dist_backfilling.go +++ b/ddl/dist_backfilling.go @@ -20,6 +20,7 @@ import ( "time" "github.com/pingcap/errors" + "github.com/pingcap/tidb/ddl/ingest" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/parser/model" "github.com/pingcap/tidb/resourcemanager/pooltask" @@ -98,7 +99,7 @@ func (bwCtx *backfillWorkerContext) GetContext() *backfillWorker { return bw } -func runBackfillJobs(d *ddl, sess *session, bJob *BackfillJob, jobCtx *JobContext) (table.Table, error) { +func runBackfillJobs(d *ddl, ingestBackendCtx *ingest.BackendContext, sess *session, bJob *BackfillJob, jobCtx *JobContext) (table.Table, error) { dbInfo, tbl, err := d.getTableByTxn(d.store, bJob.Meta.SchemaID, bJob.Meta.TableID) if err != nil { logutil.BgLogger().Warn("[ddl] runBackfillJobs gets table failed", zap.String("bfJob", bJob.AbbrStr()), zap.Error(err)) @@ -122,7 +123,7 @@ func runBackfillJobs(d *ddl, sess *session, bJob *BackfillJob, jobCtx *JobContex } // add new task resultCh, control := d.backfillWorkerPool.AddProduceBySlice(proFunc, 0, workerCtx, spmc.WithConcurrency(workerCnt)) - bwMgr.waitFinalResult(resultCh, control) + bwMgr.waitFinalResult(resultCh, ingestBackendCtx, workerCnt, bJob.EleID, control) // waiting task finishing control.Wait() @@ -154,9 +155,10 @@ func newBackfilWorkerManager(bwCtx *backfillWorkerContext) *backfilWorkerManager } } -func (bwm *backfilWorkerManager) waitFinalResult(resultCh <-chan *backfillResult, +func (bwm *backfilWorkerManager) waitFinalResult(resultCh <-chan *backfillResult, ingestBackendCtx *ingest.BackendContext, workerCnt int, eleID int64, tControl pooltask.TaskController[*reorgBackfillTask, *backfillResult, int, *backfillWorker, *backfillWorkerContext]) { bwm.wg.Run(func() { + i := 0 for { select { case result, ok := <-resultCh: @@ -169,6 +171,15 @@ func (bwm *backfilWorkerManager) waitFinalResult(resultCh <-chan *backfillResult tControl.Stop() return } + + if ingestBackendCtx != nil && i%workerCnt == 0 { + err := ingestBackendCtx.Flush(eleID) + if err != nil { + bwm.unsyncErr = err + return + } + } + i++ case <-bwm.exitCh: return } diff --git a/ddl/dist_owner.go b/ddl/dist_owner.go index 2642f7fd9ebb7..78dc9073d57e5 100644 --- a/ddl/dist_owner.go +++ b/ddl/dist_owner.go @@ -24,13 +24,32 @@ import ( "github.com/pingcap/tidb/parser/model" "github.com/pingcap/tidb/parser/mysql" "github.com/pingcap/tidb/sessionctx" + "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/table" "github.com/pingcap/tidb/util/dbterror" "github.com/pingcap/tidb/util/logutil" "go.uber.org/zap" ) -var checkBackfillJobFinishInterval = 300 * time.Millisecond +// CheckBackfillJobFinishInterval is export for test. +var CheckBackfillJobFinishInterval = 300 * time.Millisecond + +func initDistReorg(reorgMeta *model.DDLReorgMeta, tbl table.Table) { + if reorgMeta.IsDistReorg != model.DistReorgNone { + return + } + + isDistReorg := variable.DDLEnableDistributeReorg.Load() + // TODO: Support partitionTable. + if _, ok := tbl.(table.PartitionedTable); ok { + isDistReorg = false + } + if isDistReorg { + reorgMeta.IsDistReorg = model.DistReorgTrue + } else { + reorgMeta.IsDistReorg = model.DistReorgFalse + } +} func (dc *ddlCtx) controlWritePhysicalTableRecord(sess *session, t table.PhysicalTable, bfWorkerType backfillerType, reorgInfo *reorgInfo) error { startKey, endKey := reorgInfo.StartKey, reorgInfo.EndKey @@ -81,7 +100,7 @@ func checkReorgJobFinished(ctx context.Context, sess *session, reorgCtxs *reorgC var times int64 var bfJob *BackfillJob var backfillJobFinished bool - ticker := time.NewTicker(checkBackfillJobFinishInterval) + ticker := time.NewTicker(CheckBackfillJobFinishInterval) defer ticker.Stop() for { if getReorgCtx(reorgCtxs, ddlJobID).isReorgCanceled() { diff --git a/ddl/index.go b/ddl/index.go index e066cc5a919a5..64ba5ad677765 100644 --- a/ddl/index.go +++ b/ddl/index.go @@ -667,9 +667,8 @@ func (w *worker) onCreateIndex(d *ddlCtx, t *meta.Meta, job *model.Job, isPK boo if job.MultiSchemaInfo != nil { done, ver, err = doReorgWorkForCreateIndexMultiSchema(w, d, t, job, tbl, indexInfo) } else { - // TODO: Support typeAddIndexMergeTmpWorker and partitionTable. - isDistReorg := variable.DDLEnableDistributeReorg.Load() - if isDistReorg { + initDistReorg(job.ReorgMeta, tbl) + if job.ReorgMeta.IsDistReorg == model.DistReorgTrue { done, ver, err = doReorgWorkForCreateIndexWithDistReorg(w, d, t, job, tbl, indexInfo) } else { done, ver, err = doReorgWorkForCreateIndex(w, d, t, job, tbl, indexInfo) @@ -927,9 +926,6 @@ func doReorgWorkForCreateIndexWithDistReorg(w *worker, d *ddlCtx, t *meta.Meta, logutil.BgLogger().Info("[ddl] index backfill state ready to merge", zap.Int64("job ID", job.ID), zap.String("table", tbl.Meta().Name.O), zap.String("index", indexInfo.Name.O)) indexInfo.BackfillState = model.BackfillStateMerging - if bfProcess == model.ReorgTypeLitMerge { - ingest.LitBackCtxMgr.Unregister(job.ID) - } job.SnapshotVer = 0 // Reset the snapshot version for merge index reorg. ver, err = updateVersionAndTableInfo(d, t, job, tbl.Meta(), true) return false, ver, errors.Trace(err) @@ -1367,8 +1363,8 @@ func (w *baseIndexWorker) UpdateTask(bfJob *BackfillJob) error { s := newSession(w.backfillCtx.sessCtx) return s.runInTxn(func(se *session) error { - jobs, err := GetBackfillJobs(se, BackfillTable, fmt.Sprintf("ddl_job_id = %d and ele_id = %d and ele_key = '%s' and id = %d", - bfJob.JobID, bfJob.EleID, bfJob.EleKey, bfJob.ID), "update_backfill_task") + jobs, err := GetBackfillJobs(se, BackfillTable, fmt.Sprintf("ddl_job_id = %d and ele_id = %d and ele_key = %s and id = %d", + bfJob.JobID, bfJob.EleID, wrapKey2String(bfJob.EleKey), bfJob.ID), "update_backfill_task") if err != nil { return err } @@ -1822,9 +1818,9 @@ func (w *worker) addTableIndex(t table.Table, reorgInfo *reorgInfo) error { } else { //nolint:forcetypeassert phyTbl := t.(table.PhysicalTable) - // TODO: Support typeAddIndexMergeTmpWorker and partitionTable. - isDistReorg := variable.DDLEnableDistributeReorg.Load() - if isDistReorg && !reorgInfo.mergingTmpIdx && reorgInfo.Job.Type == model.ActionAddIndex { + // TODO: Support typeAddIndexMergeTmpWorker. + isDistReorg := reorgInfo.Job.ReorgMeta.IsDistReorg == model.DistReorgTrue + if isDistReorg && !reorgInfo.mergingTmpIdx { sCtx, err := w.sessPool.get() if err != nil { return errors.Trace(err) @@ -2091,13 +2087,14 @@ func runBackfillJobsWithLightning(d *ddl, sess *session, bfJob *BackfillJob, job return err } - tbl, err := runBackfillJobs(d, sess, bfJob, jobCtx) + tbl, err := runBackfillJobs(d, bc, sess, bfJob, jobCtx) if err != nil { logutil.BgLogger().Warn("[ddl] runBackfillJobs error", zap.Error(err)) ingest.LitBackCtxMgr.Unregister(bfJob.JobID) return err } + bc.EngMgr.ResetWorkers(bc, bfJob.JobID, bfJob.EleID) err = bc.FinishImport(bfJob.EleID, bfJob.Meta.IsUnique, tbl) if err != nil { logutil.BgLogger().Warn("[ddl] lightning import error", zap.String("first backfill job", bfJob.AbbrStr()), zap.Error(err)) diff --git a/ddl/indexmergetest/merge_test.go b/ddl/indexmergetest/merge_test.go index 05b8c6b66a8c9..12b1255acf615 100644 --- a/ddl/indexmergetest/merge_test.go +++ b/ddl/indexmergetest/merge_test.go @@ -470,9 +470,9 @@ func TestAddIndexMergeConflictWithPessimistic(t *testing.T) { tk.MustExec(`INSERT INTO t VALUES (1, 1);`) // Make shorten the conversion time from ReorgTypeLitMerge to BackfillStateReadyToMerge. - interval := ddl.GetCheckInterval() - ddl.SetCheckInterval(50 * time.Millisecond) - defer ddl.SetCheckInterval(interval) + interval := ddl.CheckBackfillJobFinishInterval + ddl.CheckBackfillJobFinishInterval = 50 * time.Millisecond + defer func() { ddl.CheckBackfillJobFinishInterval = interval }() // Force onCreateIndex use the txn-merge process. ingest.LitInitialized = false diff --git a/ddl/job_table.go b/ddl/job_table.go index c3a745249c8e6..61c4dff5e6dff 100644 --- a/ddl/job_table.go +++ b/ddl/job_table.go @@ -416,7 +416,7 @@ func (d *ddl) loadBackfillJobAndRun() { err = runBackfillJobsWithLightning(d, sess, bfJob, jobCtx) } else { logutil.BgLogger().Info("[ddl] run backfill jobs with txn-merge in this instance", zap.String("bfJob", bfJob.AbbrStr())) - _, err = runBackfillJobs(d, sess, bfJob, jobCtx) + _, err = runBackfillJobs(d, nil, sess, bfJob, jobCtx) } if err == nil { @@ -630,7 +630,7 @@ func getJobsBySQL(sess *session, tbl, condition string) ([]*model.Job, error) { func syncBackfillHistoryJobs(sess *session, uuid string, backfillJob *BackfillJob) error { sql := fmt.Sprintf("update mysql.%s set state = %d where ddl_job_id = %d and ele_id = %d and ele_key = %s and exec_id = '%s' limit 1;", - BackfillHistoryTable, model.JobStateSynced, backfillJob.JobID, backfillJob.EleID, (backfillJob.EleKey), uuid) + BackfillHistoryTable, model.JobStateSynced, backfillJob.JobID, backfillJob.EleID, wrapKey2String(backfillJob.EleKey), uuid) _, err := sess.execute(context.Background(), sql, "sync_backfill_history_job") return err } diff --git a/executor/analyzetest/BUILD.bazel b/executor/analyzetest/BUILD.bazel index 0090ec36aa0f4..3112abe57c00f 100644 --- a/executor/analyzetest/BUILD.bazel +++ b/executor/analyzetest/BUILD.bazel @@ -20,7 +20,6 @@ go_test( "//parser/mysql", "//parser/terror", "//planner/core", - "//resourcemanager", "//session", "//sessionctx", "//sessionctx/variable", diff --git a/executor/analyzetest/analyze_test.go b/executor/analyzetest/analyze_test.go index bc93d3e36ec20..843200fea6cf9 100644 --- a/executor/analyzetest/analyze_test.go +++ b/executor/analyzetest/analyze_test.go @@ -35,7 +35,6 @@ import ( "github.com/pingcap/tidb/parser/mysql" "github.com/pingcap/tidb/parser/terror" "github.com/pingcap/tidb/planner/core" - "github.com/pingcap/tidb/resourcemanager" "github.com/pingcap/tidb/session" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/sessionctx/variable" @@ -2105,7 +2104,6 @@ func TestAnalyzeColumnsAfterAnalyzeAll(t *testing.T) { "test t b 0 0 2 1 3 4 0", "test t b 0 1 3 1 6 6 0")) tk.MustQuery(fmt.Sprintf("select hist_id from mysql.stats_histograms where version = (select version from mysql.stats_meta where table_id = %d)", tblID)).Check(testkit.Rows("2")) - resourcemanager.GlobalResourceManager.Reset() }(val) } } diff --git a/parser/model/ddl.go b/parser/model/ddl.go index 9f679aa7383a9..7a6b9d7b8fd1e 100644 --- a/parser/model/ddl.go +++ b/parser/model/ddl.go @@ -226,6 +226,15 @@ func (h *HistoryInfo) Clean() { h.MultipleTableInfos = nil } +const ( + // DistReorgFalse means it's not distributed reorganization. + DistReorgFalse int8 = -1 + // DistReorgNone means it's not be set. + DistReorgNone int8 = 0 + // DistReorgTrue means it is distributed reorganization. + DistReorgTrue int8 = 1 +) + // DDLReorgMeta is meta info of DDL reorganization. type DDLReorgMeta struct { // EndHandle is the last handle of the adding indices table. @@ -237,6 +246,7 @@ type DDLReorgMeta struct { WarningsCount map[errors.ErrorID]int64 `json:"warnings_count"` Location *TimeZoneLocation `json:"location"` ReorgTp ReorgType `json:"reorg_tp"` + IsDistReorg int8 `json:"is_dist_reorg"` } // ReorgType indicates which process is used for the data reorganization. diff --git a/statistics/handle/BUILD.bazel b/statistics/handle/BUILD.bazel index fbdd84243acaf..d52847495d539 100644 --- a/statistics/handle/BUILD.bazel +++ b/statistics/handle/BUILD.bazel @@ -81,7 +81,6 @@ go_test( "//metrics", "//parser/model", "//parser/mysql", - "//resourcemanager", "//session", "//sessionctx/stmtctx", "//sessionctx/variable", diff --git a/statistics/handle/update_test.go b/statistics/handle/update_test.go index a53dcf65dc114..1e2f9fb228442 100644 --- a/statistics/handle/update_test.go +++ b/statistics/handle/update_test.go @@ -25,7 +25,6 @@ import ( "github.com/pingcap/tidb/metrics" "github.com/pingcap/tidb/parser/model" "github.com/pingcap/tidb/parser/mysql" - "github.com/pingcap/tidb/resourcemanager" "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/statistics" @@ -2352,7 +2351,6 @@ func TestCollectPredicateColumnsFromExecute(t *testing.T) { require.True(t, rows[0][4].(string) != "") require.True(t, rows[0][5].(string) == "") } - resourcemanager.GlobalResourceManager.Reset() }(val) } } From bef0fb8b7a5ec975aa34c7308869d02228095430 Mon Sep 17 00:00:00 2001 From: Lynn Date: Tue, 31 Jan 2023 12:38:04 +0800 Subject: [PATCH 15/25] ddl/ingest: fix #40881 --- ddl/ingest/engine.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/ddl/ingest/engine.go b/ddl/ingest/engine.go index 0c9409bf7657e..24779ef9d7718 100644 --- a/ddl/ingest/engine.go +++ b/ddl/ingest/engine.go @@ -203,7 +203,7 @@ func (ei *engineInfo) newWriterContext(workerID int, unique bool) (*WriterContex func (ei *engineInfo) closeWriters() error { var firstErr error - for wid := range ei.writerCache.Keys() { + for _, wid := range ei.writerCache.Keys() { if w, ok := ei.writerCache.Load(wid); ok { _, err := w.Close(ei.ctx) if err != nil { From 9f2149e4770508446971c5dff435cda067398197 Mon Sep 17 00:00:00 2001 From: Lynn Date: Tue, 31 Jan 2023 19:55:57 +0800 Subject: [PATCH 16/25] ddl: address comments and make test stable --- ddl/BUILD.bazel | 1 + ddl/backfilling.go | 1 - ddl/db_integration_test.go | 3 --- ddl/ddl.go | 1 + ddl/main_test.go | 2 ++ resourcemanager/util/shard_pool_map.go | 2 +- 6 files changed, 5 insertions(+), 5 deletions(-) diff --git a/ddl/BUILD.bazel b/ddl/BUILD.bazel index a4be758fa46b1..a7f6f84471925 100644 --- a/ddl/BUILD.bazel +++ b/ddl/BUILD.bazel @@ -237,6 +237,7 @@ go_test( "//parser/terror", "//parser/types", "//planner/core", + "//resourcemanager/util", "//server", "//session", "//sessionctx", diff --git a/ddl/backfilling.go b/ddl/backfilling.go index 9141f97a694d3..962951dd679e5 100644 --- a/ddl/backfilling.go +++ b/ddl/backfilling.go @@ -241,7 +241,6 @@ type backfillResult struct { taskID int addedCount int scanCount int - unsyncErr error nextKey kv.Key err error } diff --git a/ddl/db_integration_test.go b/ddl/db_integration_test.go index da66c00133419..16d3d7a2478e1 100644 --- a/ddl/db_integration_test.go +++ b/ddl/db_integration_test.go @@ -1338,9 +1338,6 @@ func TestAddIndexAfterAddColumn(t *testing.T) { tk.MustExec("insert into test_add_index_after_add_col values(1, 2),(2,2)") tk.MustExec("alter table test_add_index_after_add_col add column c int not null default '0'") sql := "alter table test_add_index_after_add_col add unique index cc(c) " - limit := variable.GetDDLErrorCountLimit() - variable.SetDDLErrorCountLimit(3) - defer variable.SetDDLErrorCountLimit(limit) tk.MustGetErrCode(sql, errno.ErrDupEntry) sql = "alter table test_add_index_after_add_col add index idx_test(f1,f2,f3,f4,f5,f6,f7,f8,f9,f10,f11,f12,f13,f14,f15,f16,f17);" tk.MustGetErrCode(sql, errno.ErrTooManyKeyParts) diff --git a/ddl/ddl.go b/ddl/ddl.go index f1866a8f0b266..e5316eb87e37b 100644 --- a/ddl/ddl.go +++ b/ddl/ddl.go @@ -786,6 +786,7 @@ func (d *ddl) Start(ctxPool *pools.ResourcePool) error { return err } + // TODO: Currently, it is only processed during initialization and is expected to be added to EnableDDL later. if err := d.prepareBackfillWorkers(); err != nil { return err } diff --git a/ddl/main_test.go b/ddl/main_test.go index 6a8642ae34380..e83df6b6b063b 100644 --- a/ddl/main_test.go +++ b/ddl/main_test.go @@ -26,6 +26,7 @@ import ( "github.com/pingcap/tidb/domain" "github.com/pingcap/tidb/domain/infosync" "github.com/pingcap/tidb/meta/autoid" + "github.com/pingcap/tidb/resourcemanager/util" "github.com/pingcap/tidb/testkit/testsetup" "github.com/tikv/client-go/v2/tikv" "go.uber.org/goleak" @@ -41,6 +42,7 @@ func TestMain(m *testing.M) { autoid.SetStep(5000) ddl.ReorgWaitTimeout = 30 * time.Millisecond ddl.RunInGoTest = true + util.InTest.Store(true) ddl.SetBatchInsertDeleteRangeSize(2) config.UpdateGlobal(func(conf *config.Config) { diff --git a/resourcemanager/util/shard_pool_map.go b/resourcemanager/util/shard_pool_map.go index d6a4d11d8b48d..585a9f241fc56 100644 --- a/resourcemanager/util/shard_pool_map.go +++ b/resourcemanager/util/shard_pool_map.go @@ -28,7 +28,7 @@ var InTest atomic.Bool func isInTests() bool { for _, arg := range os.Args { - if strings.HasPrefix(arg, "-test.v=") { + if strings.HasPrefix(arg, "-test.v=") || strings.HasPrefix(arg, "-test.run=") { return true } } From bf25e1b4c7ab0f98e8fc686e602aff2ac2a12759 Mon Sep 17 00:00:00 2001 From: Lynn Date: Wed, 1 Feb 2023 22:40:28 +0800 Subject: [PATCH 17/25] ddl, parser: fix TestAddIndexIngestAdjustBackfillWorkerCountFail and address comments --- ddl/backfilling.go | 12 ++--- ddl/ddl.go | 13 +++-- ddl/ddl_worker.go | 3 +- ddl/dist_backfilling.go | 54 ++++++++++++------- ddl/dist_owner.go | 16 +++--- ddl/index.go | 31 +++++------ ddl/job_table.go | 11 ++-- parser/model/ddl.go | 11 +--- tests/realtikvtest/addindextest/BUILD.bazel | 1 + .../addindextest/integration_test.go | 4 ++ 10 files changed, 82 insertions(+), 74 deletions(-) diff --git a/ddl/backfilling.go b/ddl/backfilling.go index 962951dd679e5..f4fcd96cd1897 100644 --- a/ddl/backfilling.go +++ b/ddl/backfilling.go @@ -66,7 +66,7 @@ const ( genTaskBatch = 4096 minGenTaskBatch = 1024 minDistTaskCnt = 32 - retrySQLTimes = 3 + retrySQLTimes = 10 retrySQLInterval = 300 * time.Millisecond ) @@ -900,7 +900,7 @@ func (b *backfillScheduler) adjustWorkerSize() error { idxWorker, err := newAddIndexWorker(b.decodeColMap, b.tbl, backfillCtx, jc, job.ID, reorgInfo.currElement.ID, reorgInfo.currElement.TypeKey) if err != nil { - if b.canSkipError(err) { + if canSkipError(b.reorgInfo.ID, len(b.workers), err) { continue } return err @@ -967,14 +967,14 @@ func (b *backfillScheduler) initCopReqSenderPool() { b.copReqSenderPool = newCopReqSenderPool(b.ctx, copCtx, sessCtx.GetStore()) } -func (b *backfillScheduler) canSkipError(err error) bool { - if len(b.workers) > 0 { +func canSkipError(jobID int64, workerCnt int, err error) bool { + if workerCnt > 0 { // The error can be skipped because the rest workers can handle the tasks. return true } logutil.BgLogger().Warn("[ddl] create add index backfill worker failed", - zap.Int("current worker count", len(b.workers)), - zap.Int64("job ID", b.reorgInfo.ID), zap.Error(err)) + zap.Int("current worker count", workerCnt), + zap.Int64("job ID", jobID), zap.Error(err)) return false } diff --git a/ddl/ddl.go b/ddl/ddl.go index e5316eb87e37b..a9e0f4d96478c 100644 --- a/ddl/ddl.go +++ b/ddl/ddl.go @@ -737,14 +737,12 @@ func (d *ddl) prepareWorkers4ConcurrencyDDL() { } func (d *ddl) prepareBackfillWorkers() error { - workerFactory := func() func() (pools.Resource, error) { - return func() (pools.Resource, error) { - bk := newBackfillWorker(context.Background(), nil) - metrics.DDLCounter.WithLabelValues(fmt.Sprintf("%s_backfill_worker", metrics.CreateDDL)).Inc() - return bk, nil - } + workerFactory := func() (pools.Resource, error) { + bk := newBackfillWorker(context.Background(), nil) + metrics.DDLCounter.WithLabelValues(fmt.Sprintf("%s_backfill_worker", metrics.CreateDDL)).Inc() + return bk, nil } - d.backfillCtxPool = newBackfillContextPool(pools.NewResourcePool(workerFactory(), backfillWorkerCnt, backfillWorkerCnt, 0)) + d.backfillCtxPool = newBackfillContextPool(pools.NewResourcePool(workerFactory, backfillWorkerCnt, backfillWorkerCnt, 0)) var err error d.backfillWorkerPool, err = spmc.NewSPMCPool[*reorgBackfillTask, *backfillResult, int, *backfillWorker, *backfillWorkerContext]("backfill", int32(backfillWorkerCnt), rmutil.DDL) @@ -985,6 +983,7 @@ func getJobCheckInterval(job *model.Job, i int) (time.Duration, bool) { func (dc *ddlCtx) asyncNotifyWorker(ch chan struct{}, etcdPath string, jobID int64, jobType string) { // If the workers don't run, we needn't notify workers. + // TODO: It does not affect informing the backfill worker. if !config.GetGlobalConfig().Instance.TiDBEnableDDL.Load() { return } diff --git a/ddl/ddl_worker.go b/ddl/ddl_worker.go index 4ba5a6c636640..68b67836216d9 100644 --- a/ddl/ddl_worker.go +++ b/ddl/ddl_worker.go @@ -175,7 +175,8 @@ func (d *ddlCtx) asyncNotifyByEtcd(etcdPath string, jobID int64, jobType string) timeStart := time.Now() err := util.PutKVToEtcd(d.ctx, d.etcdCli, 1, etcdPath, jobIDStr) if err != nil { - logutil.BgLogger().Info("[ddl] notify handling DDL job failed", zap.String("jobID", jobIDStr), zap.Error(err)) + logutil.BgLogger().Info("[ddl] notify handling DDL job failed", + zap.String("etcdPath", etcdPath), zap.Int64("jobID", jobID), zap.String("type", jobType), zap.Error(err)) } metrics.DDLWorkerHistogram.WithLabelValues(metrics.WorkerNotifyDDLJob, jobType, metrics.RetLabel(err)).Observe(time.Since(timeStart).Seconds()) } diff --git a/ddl/dist_backfilling.go b/ddl/dist_backfilling.go index 8b698f9a69e36..d5fea60e46be3 100644 --- a/ddl/dist_backfilling.go +++ b/ddl/dist_backfilling.go @@ -43,45 +43,52 @@ type backfillWorkerContext struct { type newBackfillerFunc func(bfCtx *backfillCtx) (bf backfiller, err error) -func newBackfillWorkerContext(d *ddl, schemaName string, tbl table.Table, workerCnt int, bfMeta *model.BackfillMeta, +func newBackfillWorkerContext(d *ddl, schemaName string, tbl table.Table, workerCnt int, jobID int64, bfMeta *model.BackfillMeta, bfFunc newBackfillerFunc) (*backfillWorkerContext, error) { if workerCnt <= 0 { return nil, nil } - bCtxs, err := d.backfillCtxPool.batchGet(workerCnt) - if err != nil || len(bCtxs) == 0 { - logutil.BgLogger().Debug("[ddl] no backfill context available now", zap.Int("backfillCtx", len(bCtxs)), zap.Error(err)) - return nil, errors.Trace(err) - } - bwCtx := &backfillWorkerContext{backfillWorkers: bCtxs, sessCtxs: make([]sessionctx.Context, 0, len(bCtxs))} + bwCtx := &backfillWorkerContext{backfillWorkers: make([]*backfillWorker, 0, workerCnt), sessCtxs: make([]sessionctx.Context, 0, workerCnt)} + var err error defer func() { if err != nil { bwCtx.close(d) } }() - for i := 0; i < len(bCtxs); i++ { + for i := 0; i < workerCnt; i++ { var se sessionctx.Context se, err = d.sessPool.get() if err != nil { - logutil.BgLogger().Error("[ddl] new backfill worker context, get a session failed", zap.Error(err)) + logutil.BgLogger().Error("[ddl] new backfill worker context, get a session failed", zap.Int64("jobID", jobID), zap.Error(err)) return nil, errors.Trace(err) } + bwCtx.sessCtxs = append(bwCtx.sessCtxs, se) err = initSessCtx(se, bfMeta.SQLMode, bfMeta.Location) if err != nil { - logutil.BgLogger().Error("[ddl] new backfill worker context, init the session ctx failed", zap.Error(err)) + logutil.BgLogger().Error("[ddl] new backfill worker context, init the session ctx failed", zap.Int64("jobID", jobID), zap.Error(err)) return nil, errors.Trace(err) } - bfCtx := newBackfillCtx(d.ddlCtx, 0, se, bfMeta.ReorgTp, schemaName, tbl) + var bf backfiller - bf, err = bfFunc(bfCtx) + bf, err = bfFunc(newBackfillCtx(d.ddlCtx, 0, se, bfMeta.ReorgTp, schemaName, tbl)) if err != nil { - logutil.BgLogger().Error("[ddl] new backfill worker context, do bfFunc failed", zap.Error(err)) + if canSkipError(jobID, len(bwCtx.backfillWorkers), err) { + err = nil + continue + } + logutil.BgLogger().Error("[ddl] new backfill worker context, do bfFunc failed", zap.Int64("jobID", jobID), zap.Error(err)) return nil, errors.Trace(err) } - bwCtx.sessCtxs = append(bwCtx.sessCtxs, se) - bCtxs[i].backfiller = bf + bCtx, err := d.backfillCtxPool.get() + if err != nil || bCtx == nil { + logutil.BgLogger().Info("[ddl] new backfill worker context, get backfill context failed", zap.Int64("jobID", jobID), zap.Error(err)) + err = nil + break + } + bCtx.backfiller = bf + bwCtx.backfillWorkers = append(bwCtx.backfillWorkers, bCtx) } return bwCtx, nil } @@ -99,12 +106,19 @@ func (bwCtx *backfillWorkerContext) GetContext() *backfillWorker { return bw } -func runBackfillJobs(d *ddl, ingestBackendCtx *ingest.BackendContext, sess *session, bJob *BackfillJob, jobCtx *JobContext) (table.Table, error) { +func runBackfillJobs(d *ddl, ingestBackendCtx *ingest.BackendContext, bJob *BackfillJob, jobCtx *JobContext) (table.Table, error) { dbInfo, tbl, err := d.getTableByTxn(d.store, bJob.Meta.SchemaID, bJob.Meta.TableID) if err != nil { logutil.BgLogger().Warn("[ddl] runBackfillJobs gets table failed", zap.String("bfJob", bJob.AbbrStr()), zap.Error(err)) return nil, err } + se, err := d.sessPool.get() + if err != nil { + logutil.BgLogger().Warn("[ddl] run backfill jobs get session failed", zap.Error(err)) + return nil, err + } + defer d.sessPool.put(se) + sess := newSession(se) workerCnt := int(variable.GetDDLReorgWorkerCounter()) // TODO: Different worker using different newBackfillerFunc. @@ -113,17 +127,19 @@ func runBackfillJobs(d *ddl, ingestBackendCtx *ingest.BackendContext, sess *sess logutil.BgLogger().Info("[ddl] new adding index worker context failed", zap.Reflect("workerCtx", workerCtx), zap.Error(err)) return nil, errors.Trace(err) } + workerCnt = len(workerCtx.backfillWorkers) bwMgr := newBackfilWorkerManager(workerCtx) d.backfillWorkerPool.SetConsumerFunc(func(task *reorgBackfillTask, _ int, bfWorker *backfillWorker) *backfillResult { return bfWorker.runTask(task) }) + proFunc := func() ([]*reorgBackfillTask, error) { // TODO: After BackfillJob replaces reorgBackfillTask, use backfiller's GetTasks instead of it. return GetTasks(d.ddlCtx, sess, tbl, bJob.JobID, workerCnt+5) } // add new task resultCh, control := d.backfillWorkerPool.AddProduceBySlice(proFunc, 0, workerCtx, spmc.WithConcurrency(workerCnt)) - bwMgr.waitFinalResult(resultCh, ingestBackendCtx, workerCnt, bJob.EleID, control) + bwMgr.waitFinalResult(resultCh, ingestBackendCtx, bJob.EleID, control) // waiting task finishing control.Wait() @@ -155,10 +171,12 @@ func newBackfilWorkerManager(bwCtx *backfillWorkerContext) *backfilWorkerManager } } -func (bwm *backfilWorkerManager) waitFinalResult(resultCh <-chan *backfillResult, ingestBackendCtx *ingest.BackendContext, workerCnt int, eleID int64, +func (bwm *backfilWorkerManager) waitFinalResult(resultCh <-chan *backfillResult, ingestBackendCtx *ingest.BackendContext, eleID int64, tControl pooltask.TaskController[*reorgBackfillTask, *backfillResult, int, *backfillWorker, *backfillWorkerContext]) { bwm.wg.Run(func() { i := 0 + workerCnt := len(bwm.bwCtx.backfillWorkers) + for { select { case result, ok := <-resultCh: diff --git a/ddl/dist_owner.go b/ddl/dist_owner.go index 78dc9073d57e5..52938bdebc6eb 100644 --- a/ddl/dist_owner.go +++ b/ddl/dist_owner.go @@ -20,6 +20,7 @@ import ( "time" "github.com/pingcap/errors" + "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/meta" "github.com/pingcap/tidb/parser/model" "github.com/pingcap/tidb/parser/mysql" @@ -34,9 +35,10 @@ import ( // CheckBackfillJobFinishInterval is export for test. var CheckBackfillJobFinishInterval = 300 * time.Millisecond -func initDistReorg(reorgMeta *model.DDLReorgMeta, tbl table.Table) { - if reorgMeta.IsDistReorg != model.DistReorgNone { - return +func initDistReorg(reorgMeta *model.DDLReorgMeta, store kv.Storage, schemaID int64, tblInfo *model.TableInfo) error { + tbl, err := getTable(store, schemaID, tblInfo) + if err != nil { + return errors.Trace(err) } isDistReorg := variable.DDLEnableDistributeReorg.Load() @@ -44,11 +46,8 @@ func initDistReorg(reorgMeta *model.DDLReorgMeta, tbl table.Table) { if _, ok := tbl.(table.PartitionedTable); ok { isDistReorg = false } - if isDistReorg { - reorgMeta.IsDistReorg = model.DistReorgTrue - } else { - reorgMeta.IsDistReorg = model.DistReorgFalse - } + reorgMeta.IsDistReorg = isDistReorg + return nil } func (dc *ddlCtx) controlWritePhysicalTableRecord(sess *session, t table.PhysicalTable, bfWorkerType backfillerType, reorgInfo *reorgInfo) error { @@ -92,7 +91,6 @@ func (dc *ddlCtx) controlWritePhysicalTableRecord(sess *session, t table.Physica if err != nil { return errors.Trace(err) } - dc.ctx.Done() return checkReorgJobFinished(dc.ctx, sess, &dc.reorgCtx, ddlJobID, currEle) } diff --git a/ddl/index.go b/ddl/index.go index 64ba5ad677765..312d7fe2cf305 100644 --- a/ddl/index.go +++ b/ddl/index.go @@ -41,7 +41,6 @@ import ( "github.com/pingcap/tidb/parser/mysql" "github.com/pingcap/tidb/parser/terror" "github.com/pingcap/tidb/sessionctx" - "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/table" "github.com/pingcap/tidb/table/tables" "github.com/pingcap/tidb/tablecodec" @@ -656,6 +655,12 @@ func (w *worker) onCreateIndex(d *ddlCtx, t *meta.Meta, job *model.Job, isPK boo // Initialize SnapshotVer to 0 for later reorganization check. job.SnapshotVer = 0 job.SchemaState = model.StateWriteReorganization + + if job.MultiSchemaInfo == nil { + if err := initDistReorg(job.ReorgMeta, d.store, schemaID, tblInfo); err != nil { + return ver, errors.Trace(err) + } + } case model.StateWriteReorganization: // reorganization -> public tbl, err := getTable(d.store, schemaID, tblInfo) @@ -667,8 +672,7 @@ func (w *worker) onCreateIndex(d *ddlCtx, t *meta.Meta, job *model.Job, isPK boo if job.MultiSchemaInfo != nil { done, ver, err = doReorgWorkForCreateIndexMultiSchema(w, d, t, job, tbl, indexInfo) } else { - initDistReorg(job.ReorgMeta, tbl) - if job.ReorgMeta.IsDistReorg == model.DistReorgTrue { + if job.ReorgMeta.IsDistReorg { done, ver, err = doReorgWorkForCreateIndexWithDistReorg(w, d, t, job, tbl, indexInfo) } else { done, ver, err = doReorgWorkForCreateIndex(w, d, t, job, tbl, indexInfo) @@ -991,7 +995,7 @@ func runReorgJobAndHandleErr(w *worker, d *ddlCtx, t *meta.Meta, job *model.Job, } if kv.ErrKeyExists.Equal(err) || dbterror.ErrCancelledDDLJob.Equal(err) || dbterror.ErrCantDecodeRecord.Equal(err) || // TODO: Remove this check make it can be retry. Related test is TestModifyColumnReorgInfo. - variable.DDLEnableDistributeReorg.Load() { + job.ReorgMeta.IsDistReorg { logutil.BgLogger().Warn("[ddl] run add index job failed, convert job to rollback", zap.String("job", job.String()), zap.Error(err)) ver, err = convertAddIdxJob2RollbackJob(d, t, job, tbl.Meta(), indexInfo, err) if err1 := rh.RemoveDDLReorgHandle(job, reorgInfo.elements); err1 != nil { @@ -1408,11 +1412,11 @@ func newAddIndexWorkerContext(d *ddl, schemaName model.CIStr, tbl table.Table, w bfJob *BackfillJob, jobCtx *JobContext) (*backfillWorkerContext, error) { //nolint:forcetypeassert phyTbl := tbl.(table.PhysicalTable) - return newBackfillWorkerContext(d, schemaName.O, tbl, workerCnt, bfJob.Meta, + return newBackfillWorkerContext(d, schemaName.O, tbl, workerCnt, bfJob.JobID, bfJob.Meta, func(bfCtx *backfillCtx) (backfiller, error) { decodeColMap, err := makeupDecodeColMap(bfCtx.sessCtx, schemaName, phyTbl) if err != nil { - logutil.BgLogger().Info("[ddl] make up decode column map failed", zap.Error(err)) + logutil.BgLogger().Error("[ddl] make up decode column map failed", zap.Error(err)) return nil, errors.Trace(err) } bf, err1 := newAddIndexWorker(decodeColMap, phyTbl, bfCtx, jobCtx, bfJob.JobID, bfJob.EleID, bfJob.EleKey) @@ -1819,8 +1823,7 @@ func (w *worker) addTableIndex(t table.Table, reorgInfo *reorgInfo) error { //nolint:forcetypeassert phyTbl := t.(table.PhysicalTable) // TODO: Support typeAddIndexMergeTmpWorker. - isDistReorg := reorgInfo.Job.ReorgMeta.IsDistReorg == model.DistReorgTrue - if isDistReorg && !reorgInfo.mergingTmpIdx { + if reorgInfo.Job.ReorgMeta.IsDistReorg && !reorgInfo.mergingTmpIdx { sCtx, err := w.sessPool.get() if err != nil { return errors.Trace(err) @@ -2074,20 +2077,14 @@ func (w *worker) updateReorgInfoForPartitions(t table.PartitionedTable, reorg *r return false, errors.Trace(err) } -func runBackfillJobsWithLightning(d *ddl, sess *session, bfJob *BackfillJob, jobCtx *JobContext) error { - // TODO: Consider redo it. - bc, ok := ingest.LitBackCtxMgr.Load(bfJob.JobID) - if ok && bc.Done() { - return errors.New(ingest.LitErrGetBackendFail) - } - var err error - bc, err = ingest.LitBackCtxMgr.Register(d.ctx, bfJob.Meta.IsUnique, bfJob.JobID, bfJob.Meta.SQLMode) +func runBackfillJobsWithLightning(d *ddl, bfJob *BackfillJob, jobCtx *JobContext) error { + bc, err := ingest.LitBackCtxMgr.Register(d.ctx, bfJob.Meta.IsUnique, bfJob.JobID, bfJob.Meta.SQLMode) if err != nil { logutil.BgLogger().Warn("[ddl] lightning register error", zap.Error(err)) return err } - tbl, err := runBackfillJobs(d, bc, sess, bfJob, jobCtx) + tbl, err := runBackfillJobs(d, bc, bfJob, jobCtx) if err != nil { logutil.BgLogger().Warn("[ddl] runBackfillJobs error", zap.Error(err)) ingest.LitBackCtxMgr.Unregister(bfJob.JobID) diff --git a/ddl/job_table.go b/ddl/job_table.go index 61c4dff5e6dff..752e3974ae777 100644 --- a/ddl/job_table.go +++ b/ddl/job_table.go @@ -370,10 +370,10 @@ func (d *ddl) loadBackfillJobAndRun() { return } se, err := d.sessPool.get() - defer d.sessPool.put(se) if err != nil { logutil.BgLogger().Fatal("dispatch backfill jobs loop get session failed, it should not happen, please try restart TiDB", zap.Error(err)) } + defer d.sessPool.put(se) sess := newSession(se) runningJobIDs := d.backfillCtxJobIDs() @@ -413,16 +413,16 @@ func (d *ddl) loadBackfillJobAndRun() { return } logutil.BgLogger().Info("[ddl] run backfill jobs with ingest in this instance", zap.String("bfJob", bfJob.AbbrStr())) - err = runBackfillJobsWithLightning(d, sess, bfJob, jobCtx) + err = runBackfillJobsWithLightning(d, bfJob, jobCtx) } else { logutil.BgLogger().Info("[ddl] run backfill jobs with txn-merge in this instance", zap.String("bfJob", bfJob.AbbrStr())) - _, err = runBackfillJobs(d, nil, sess, bfJob, jobCtx) + _, err = runBackfillJobs(d, nil, bfJob, jobCtx) } if err == nil { err = syncBackfillHistoryJobs(sess, d.uuid, bfJob) } - logutil.BgLogger().Warn("[ddl] run backfill jobs finished in this instance", zap.Stringer("reorg type", bfJob.Meta.ReorgTp), zap.Error(err)) + logutil.BgLogger().Info("[ddl] run backfill jobs finished in this instance", zap.Stringer("reorg type", bfJob.Meta.ReorgTp), zap.Error(err)) }) } @@ -775,7 +775,7 @@ func GetAndMarkBackfillJobsForOneEle(s *session, batch int, jobID int64, uuid st return bJobs[:validLen], err } -// GetInterruptedBackfillJobForOneEle gets a interrupted backfill job in the tblName table that contains only one element. +// GetInterruptedBackfillJobForOneEle gets an interrupted backfill job that contains only one element. func GetInterruptedBackfillJobForOneEle(sess *session, jobID, eleID int64, eleKey []byte) ([]*BackfillJob, error) { bJobs, err := GetBackfillJobs(sess, BackfillHistoryTable, fmt.Sprintf("ddl_job_id = %d and ele_id = %d and ele_key = %s and state = %d limit 1", jobID, eleID, wrapKey2String(eleKey), model.JobStateCancelled), "get_interrupt_backfill_job") @@ -888,7 +888,6 @@ func updateBackfillJob(sess *session, tableName string, backfillJob *BackfillJob if err != nil { return err } - backfillJob.InstanceLease.SetFsp(0) sql := fmt.Sprintf("update mysql.%s set exec_id = '%s', exec_lease = '%s', state = %d, curr_key = %s, row_count = %d, backfill_meta = %s where ddl_job_id = %d and ele_id = %d and ele_key = %s and id = %d", tableName, backfillJob.InstanceID, backfillJob.InstanceLease, backfillJob.State, wrapKey2String(backfillJob.CurrKey), backfillJob.RowCount, wrapKey2String(mate), backfillJob.JobID, backfillJob.EleID, wrapKey2String(backfillJob.EleKey), backfillJob.ID) diff --git a/parser/model/ddl.go b/parser/model/ddl.go index 7a6b9d7b8fd1e..7ca33341258c5 100644 --- a/parser/model/ddl.go +++ b/parser/model/ddl.go @@ -226,15 +226,6 @@ func (h *HistoryInfo) Clean() { h.MultipleTableInfos = nil } -const ( - // DistReorgFalse means it's not distributed reorganization. - DistReorgFalse int8 = -1 - // DistReorgNone means it's not be set. - DistReorgNone int8 = 0 - // DistReorgTrue means it is distributed reorganization. - DistReorgTrue int8 = 1 -) - // DDLReorgMeta is meta info of DDL reorganization. type DDLReorgMeta struct { // EndHandle is the last handle of the adding indices table. @@ -246,7 +237,7 @@ type DDLReorgMeta struct { WarningsCount map[errors.ErrorID]int64 `json:"warnings_count"` Location *TimeZoneLocation `json:"location"` ReorgTp ReorgType `json:"reorg_tp"` - IsDistReorg int8 `json:"is_dist_reorg"` + IsDistReorg bool `json:"is_dist_reorg"` } // ReorgType indicates which process is used for the data reorganization. diff --git a/tests/realtikvtest/addindextest/BUILD.bazel b/tests/realtikvtest/addindextest/BUILD.bazel index a79f2a15f8ca7..573fb1f531abc 100644 --- a/tests/realtikvtest/addindextest/BUILD.bazel +++ b/tests/realtikvtest/addindextest/BUILD.bazel @@ -41,6 +41,7 @@ go_test( "//domain", "//errno", "//parser/model", + "//sessionctx/variable", "//testkit", "//tests/realtikvtest", "@com_github_pingcap_failpoint//:failpoint", diff --git a/tests/realtikvtest/addindextest/integration_test.go b/tests/realtikvtest/addindextest/integration_test.go index ed1e4e6c85dcd..e5c147d3337a3 100644 --- a/tests/realtikvtest/addindextest/integration_test.go +++ b/tests/realtikvtest/addindextest/integration_test.go @@ -29,6 +29,7 @@ import ( "github.com/pingcap/tidb/domain" "github.com/pingcap/tidb/errno" "github.com/pingcap/tidb/parser/model" + "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/testkit" "github.com/pingcap/tidb/tests/realtikvtest" "github.com/stretchr/testify/assert" @@ -184,6 +185,9 @@ func TestIngestMVIndexOnPartitionTable(t *testing.T) { } func TestAddIndexIngestAdjustBackfillWorker(t *testing.T) { + if variable.DDLEnableDistributeReorg.Load() { + t.Skip("dist reorg didn't support checkBackfillWorkerNum, skip this test") + } store := realtikvtest.CreateMockStoreAndSetup(t) tk := testkit.NewTestKit(t, store) tk.MustExec("drop database if exists addindexlit;") From 563165eb3745098c56433d3ab036f297f1f558cc Mon Sep 17 00:00:00 2001 From: Lynn Date: Thu, 2 Feb 2023 00:09:37 +0800 Subject: [PATCH 18/25] ddl: make bazel happy --- ddl/BUILD.bazel | 1 - ddl/dist_backfilling.go | 3 ++- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/ddl/BUILD.bazel b/ddl/BUILD.bazel index 033c99bd008d5..a117b052d0651 100644 --- a/ddl/BUILD.bazel +++ b/ddl/BUILD.bazel @@ -238,7 +238,6 @@ go_test( "//parser/terror", "//parser/types", "//planner/core", - "//resourcemanager/util", "//server", "//session", "//sessionctx", diff --git a/ddl/dist_backfilling.go b/ddl/dist_backfilling.go index d5fea60e46be3..9238427391078 100644 --- a/ddl/dist_backfilling.go +++ b/ddl/dist_backfilling.go @@ -81,7 +81,8 @@ func newBackfillWorkerContext(d *ddl, schemaName string, tbl table.Table, worker logutil.BgLogger().Error("[ddl] new backfill worker context, do bfFunc failed", zap.Int64("jobID", jobID), zap.Error(err)) return nil, errors.Trace(err) } - bCtx, err := d.backfillCtxPool.get() + var bCtx *backfillWorker + bCtx, err = d.backfillCtxPool.get() if err != nil || bCtx == nil { logutil.BgLogger().Info("[ddl] new backfill worker context, get backfill context failed", zap.Int64("jobID", jobID), zap.Error(err)) err = nil From 90d573c5f372adc9133e0cd161bac7674ff5f7af Mon Sep 17 00:00:00 2001 From: Lynn Date: Thu, 2 Feb 2023 09:06:05 +0800 Subject: [PATCH 19/25] *: add intest tag --- Makefile | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/Makefile b/Makefile index 5b9f66e03f2cd..cb6186bf69c5d 100644 --- a/Makefile +++ b/Makefile @@ -415,7 +415,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_1.json --@io_bazel_rules_go//go/config:cover_format=go_cover --define gotags=deadlock,distributereorg \ + --build_event_json_file=bazel_1.json --@io_bazel_rules_go//go/config:cover_format=go_cover --define gotags=deadlock,intest,distributereorg \ -- //... -//cmd/... -//tests/graceshutdown/... \ -//tests/globalkilltest/... -//tests/readonlytest/... -//br/pkg/task:task_test -//tests/realtikvtest/... From 35ec0d5f911c83eccaa2012e396afbbae7a5bee4 Mon Sep 17 00:00:00 2001 From: Lynn Date: Thu, 2 Feb 2023 13:34:01 +0800 Subject: [PATCH 20/25] ddl: make test faster --- ddl/backfilling.go | 6 ++++-- ddl/dist_backfilling.go | 2 +- ddl/dist_owner.go | 8 ++++---- ddl/main_test.go | 2 ++ 4 files changed, 11 insertions(+), 7 deletions(-) diff --git a/ddl/backfilling.go b/ddl/backfilling.go index f4fcd96cd1897..4455801b7cc6d 100644 --- a/ddl/backfilling.go +++ b/ddl/backfilling.go @@ -67,9 +67,11 @@ const ( minGenTaskBatch = 1024 minDistTaskCnt = 32 retrySQLTimes = 10 - retrySQLInterval = 300 * time.Millisecond ) +// RetrySQLInterval is export for test. +var RetrySQLInterval = 300 * time.Millisecond + func (bT backfillerType) String() string { switch bT { case typeAddIndexWorker: @@ -1190,7 +1192,7 @@ func (dc *ddlCtx) splitTableToBackfillJobs(sess *session, reorgInfo *reorgInfo, if bJobCnt < minGenTaskBatch { break } - time.Sleep(retrySQLInterval) + time.Sleep(RetrySQLInterval) } startKey = remains[0].StartKey } diff --git a/ddl/dist_backfilling.go b/ddl/dist_backfilling.go index 9238427391078..79b1250d0e98c 100644 --- a/ddl/dist_backfilling.go +++ b/ddl/dist_backfilling.go @@ -262,7 +262,7 @@ func GetTasks(d *ddlCtx, sess *session, tbl table.Table, runningJobID int64, con } if kv.ErrWriteConflict.Equal(err) { logutil.BgLogger().Info("GetAndMarkBackfillJobsForOneEle failed", zap.Error(err)) - time.Sleep(retrySQLInterval) + time.Sleep(RetrySQLInterval) continue } } diff --git a/ddl/dist_owner.go b/ddl/dist_owner.go index 52938bdebc6eb..d10491fd0da3e 100644 --- a/ddl/dist_owner.go +++ b/ddl/dist_owner.go @@ -160,7 +160,7 @@ func checkJobIsFinished(sess *session, ddlJobID int64) (bool, error) { logutil.BgLogger().Info("[ddl] checkJobIsSynced failed", zap.Strings("unsyncedInstanceIDs", unsyncedInstanceIDs), zap.Int("tryTimes", i), zap.Error(err)) - time.Sleep(retrySQLInterval) + time.Sleep(RetrySQLInterval) } return false, errors.Trace(err) @@ -183,7 +183,7 @@ func GetBackfillErr(sess *session, ddlJobID, currEleID int64, currEleKey []byte) } logutil.BgLogger().Info("[ddl] GetBackfillMetas failed in checkJobIsSynced", zap.Int("tryTimes", i), zap.Error(err)) - time.Sleep(retrySQLInterval) + time.Sleep(RetrySQLInterval) } return errors.Trace(err) @@ -197,7 +197,7 @@ func checkAndHandleInterruptedBackfillJobs(sess *session, ddlJobID, currEleID in break } logutil.BgLogger().Info("[ddl] getInterruptedBackfillJobsForOneEle failed", zap.Error(err)) - time.Sleep(retrySQLInterval) + time.Sleep(RetrySQLInterval) } if err != nil { return errors.Trace(err) @@ -212,7 +212,7 @@ func checkAndHandleInterruptedBackfillJobs(sess *session, ddlJobID, currEleID in return bJobs[0].Meta.Error } logutil.BgLogger().Info("[ddl] MoveBackfillJobsToHistoryTable failed", zap.Error(err)) - time.Sleep(retrySQLInterval) + time.Sleep(RetrySQLInterval) } return errors.Trace(err) } diff --git a/ddl/main_test.go b/ddl/main_test.go index 6a8642ae34380..b6cb17a1146bb 100644 --- a/ddl/main_test.go +++ b/ddl/main_test.go @@ -40,6 +40,8 @@ func TestMain(m *testing.M) { autoid.SetStep(5000) ddl.ReorgWaitTimeout = 30 * time.Millisecond + ddl.RetrySQLInterval = 30 * time.Millisecond + ddl.CheckBackfillJobFinishInterval = 50 * time.Millisecond ddl.RunInGoTest = true ddl.SetBatchInsertDeleteRangeSize(2) From 9873b3e2bf9373f04efc615e2d04b12b36f7e0b4 Mon Sep 17 00:00:00 2001 From: Lynn Date: Thu, 2 Feb 2023 19:36:25 +0800 Subject: [PATCH 21/25] *: update gomod --- go.mod | 2 +- go.sum | 4 +--- 2 files changed, 2 insertions(+), 4 deletions(-) diff --git a/go.mod b/go.mod index 5c3a136bab0ad..94def97907765 100644 --- a/go.mod +++ b/go.mod @@ -64,7 +64,7 @@ require ( github.com/mgechev/revive v1.2.5 github.com/ngaut/pools v0.0.0-20180318154953-b7bc8c42aac7 github.com/nishanths/predeclared v0.2.2 - github.com/opentracing/basictracer-go v1.1.0 + github.com/opentracing/basictracer-go v1.0.0 github.com/opentracing/opentracing-go v1.2.0 github.com/phayes/freeport v0.0.0-20180830031419-95f893ade6f2 github.com/pingcap/badger v1.5.1-0.20230103063557-828f39b09b6d diff --git a/go.sum b/go.sum index c764c347d89b1..c5964185fa6c5 100644 --- a/go.sum +++ b/go.sum @@ -1099,9 +1099,8 @@ github.com/onsi/gomega v1.7.1/go.mod h1:XdKZgCCFLUoM/7CFJVPcG8C1xQ1AJ0vpAezJrB7J github.com/onsi/gomega v1.10.1/go.mod h1:iN09h71vgCQne3DLsj+A5owkum+a2tYe+TOCB1ybHNo= github.com/onsi/gomega v1.23.0 h1:/oxKu9c2HVap+F3PfKort2Hw5DEU+HGlW8n+tguWsys= github.com/opentracing-contrib/go-stdlib v0.0.0-20170113013457-1de4cc2120e7/go.mod h1:PLldrQSroqzH70Xl+1DQcGnefIbqsKR7UDaiux3zV+w= +github.com/opentracing/basictracer-go v1.0.0 h1:YyUAhaEfjoWXclZVJ9sGoNct7j4TVk7lZWlQw5UXuoo= github.com/opentracing/basictracer-go v1.0.0/go.mod h1:QfBfYuafItcjQuMwinw9GhYKwFXS9KnPs5lxoYwgW74= -github.com/opentracing/basictracer-go v1.1.0 h1:Oa1fTSBvAl8pa3U+IJYqrKm0NALwH9OsgwOqDv4xJW0= -github.com/opentracing/basictracer-go v1.1.0/go.mod h1:V2HZueSJEp879yv285Aap1BS69fQMD+MNP1mRs6mBQc= github.com/opentracing/opentracing-go v1.0.2/go.mod h1:UkNAQd3GIcIGf0SeVgPpRdFStlNbqXla1AfSYxPUl2o= github.com/opentracing/opentracing-go v1.1.0/go.mod h1:UkNAQd3GIcIGf0SeVgPpRdFStlNbqXla1AfSYxPUl2o= github.com/opentracing/opentracing-go v1.2.0 h1:uEJPy/1a5RIPAJ0Ov+OIO8OxWu77jEv+1B0VhjKrZUs= @@ -1546,7 +1545,6 @@ golang.org/x/net v0.0.0-20200222125558-5a598a2470a0/go.mod h1:z5CRVTTTmAJ677TzLL golang.org/x/net v0.0.0-20200226121028-0de0cce0169b/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= golang.org/x/net v0.0.0-20200301022130-244492dfa37a/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= golang.org/x/net v0.0.0-20200324143707-d3edc9973b7e/go.mod h1:qpuaurCH72eLCgpAm/N6yyVIVM9cpaDIP3A8BGJEC5A= -golang.org/x/net v0.0.0-20200421231249-e086a090c8fd/go.mod h1:qpuaurCH72eLCgpAm/N6yyVIVM9cpaDIP3A8BGJEC5A= golang.org/x/net v0.0.0-20200501053045-e0ff5e5a1de5/go.mod h1:qpuaurCH72eLCgpAm/N6yyVIVM9cpaDIP3A8BGJEC5A= golang.org/x/net v0.0.0-20200506145744-7e3656a0809f/go.mod h1:qpuaurCH72eLCgpAm/N6yyVIVM9cpaDIP3A8BGJEC5A= golang.org/x/net v0.0.0-20200513185701-a91f0712d120/go.mod h1:qpuaurCH72eLCgpAm/N6yyVIVM9cpaDIP3A8BGJEC5A= From 63ff39967153d9d677a80ab01a13a76207a0415c Mon Sep 17 00:00:00 2001 From: Lynn Date: Thu, 2 Feb 2023 20:39:11 +0800 Subject: [PATCH 22/25] *: update deps.bzl --- DEPS.bzl | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/DEPS.bzl b/DEPS.bzl index 84dbefb846ae7..3722acb64ed9e 100644 --- a/DEPS.bzl +++ b/DEPS.bzl @@ -3168,8 +3168,8 @@ def go_deps(): name = "com_github_opentracing_basictracer_go", build_file_proto_mode = "disable_global", importpath = "github.com/opentracing/basictracer-go", - sum = "h1:Oa1fTSBvAl8pa3U+IJYqrKm0NALwH9OsgwOqDv4xJW0=", - version = "v1.1.0", + sum = "h1:YyUAhaEfjoWXclZVJ9sGoNct7j4TVk7lZWlQw5UXuoo=", + version = "v1.0.0", ) go_repository( name = "com_github_opentracing_contrib_go_stdlib", From 1e0403cb4cc56ddd0d83aa8c174c735e39d39032 Mon Sep 17 00:00:00 2001 From: Lynn Date: Mon, 6 Feb 2023 11:26:50 +0800 Subject: [PATCH 23/25] *: update .bazelrc --- .bazelrc | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/.bazelrc b/.bazelrc index 61356f086fda5..bf7b293218127 100644 --- a/.bazelrc +++ b/.bazelrc @@ -1,4 +1,4 @@ -startup --host_jvm_args=-Xmx8g +startup --host_jvm_args=-Xmx4g startup --unlimit_coredumps run:ci --color=yes @@ -19,7 +19,7 @@ build:race --config=ci build:race --@io_bazel_rules_go//go/config:race --test_env=GORACE=halt_on_error=1 --test_sharding_strategy=disabled test --test_env=TZ=Asia/Shanghai -test --test_output=errors --test_summary=testcase +test --test_output=streamed --test_summary=testcase test:ci --color=yes test:ci --verbose_failures --test_verbose_timeout_warnings test:ci --test_env=GO_TEST_WRAP_TESTV=1 From 8d02ea696c3a52e5d6be3e3586cd55ce484d98cf Mon Sep 17 00:00:00 2001 From: Lynn Date: Mon, 6 Feb 2023 12:02:35 +0800 Subject: [PATCH 24/25] *: revert --test_output --- .bazelrc | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.bazelrc b/.bazelrc index bf7b293218127..11c7c8f9ad2ba 100644 --- a/.bazelrc +++ b/.bazelrc @@ -19,7 +19,7 @@ build:race --config=ci build:race --@io_bazel_rules_go//go/config:race --test_env=GORACE=halt_on_error=1 --test_sharding_strategy=disabled test --test_env=TZ=Asia/Shanghai -test --test_output=streamed --test_summary=testcase +test --test_output=errors --test_summary=testcase test:ci --color=yes test:ci --verbose_failures --test_verbose_timeout_warnings test:ci --test_env=GO_TEST_WRAP_TESTV=1 From 90096d0dcb3cab5f6d5f33f614fab789583f1e01 Mon Sep 17 00:00:00 2001 From: Lynn Date: Mon, 6 Feb 2023 13:20:08 +0800 Subject: [PATCH 25/25] *: update DEPS.bzl --- DEPS.bzl | 22 ---------------------- 1 file changed, 22 deletions(-) diff --git a/DEPS.bzl b/DEPS.bzl index 58fc8fd2337bc..ccc74d30966da 100644 --- a/DEPS.bzl +++ b/DEPS.bzl @@ -3774,14 +3774,6 @@ def go_deps(): sum = "h1:ug7PpSOB5RBPK1Kg6qskGBoP3Vnj/aNYFTznWvlkGo0=", version = "v0.0.0-20181202132449-6a9ea43bcacd", ) - go_repository( - name = "com_github_silentred_gid", - build_file_proto_mode = "disable", - importpath = "github.com/silentred/gid", - sum = "h1:JdsH8McqPUeY8IN4C0gxENnJG2zysvh+/xDJWhPvGVQ=", - version = "v1.0.0", - ) - go_repository( name = "com_github_sirupsen_logrus", build_file_proto_mode = "disable_global", @@ -4103,13 +4095,6 @@ def go_deps(): sum = "h1:phZCcypL/vtx6cGxObJgWZ5wexZF5SXFPLOM+ru0e/M=", version = "v0.1.0", ) - go_repository( - name = "com_github_tinylib_msgp", - build_file_proto_mode = "disable", - importpath = "github.com/tinylib/msgp", - sum = "h1:2gXmtWueD2HefZHQe1QOy9HVzmFrLOVvsXwXBQ0ayy0=", - version = "v1.1.5", - ) go_repository( name = "com_github_tklauser_go_sysconf", @@ -4146,13 +4131,6 @@ def go_deps(): sum = "h1:NowYhSdyE/1zwK9QCLeRb6USWdoif80Ie+v+yU8u1Zw=", version = "v2.5.1", ) - go_repository( - name = "com_github_ttacon_chalk", - build_file_proto_mode = "disable", - importpath = "github.com/ttacon/chalk", - sum = "h1:OXcKh35JaYsGMRzpvFkLv/MEyPuL49CThT1pZ8aSml4=", - version = "v0.0.0-20160626202418-22c06c80ed31", - ) go_repository( name = "com_github_twmb_murmur3",