github.com/pingcap/tiflow@v0.0.0-20240520035814-5bf52d54e205/dm/syncer/checkpoint_flush_worker.go (about) 1 // Copyright 2021 PingCAP, Inc. 2 // 3 // Licensed under the Apache License, Version 2.0 (the "License"); 4 // you may not use this file except in compliance with the License. 5 // You may obtain a copy of the License at 6 // 7 // http://www.apache.org/licenses/LICENSE-2.0 8 // 9 // Unless required by applicable law or agreed to in writing, software 10 // distributed under the License is distributed on an "AS IS" BASIS, 11 // See the License for the specific language governing permissions and 12 // limitations under the License. 13 14 package syncer 15 16 import ( 17 "fmt" 18 19 "github.com/pingcap/errors" 20 "github.com/pingcap/failpoint" 21 "github.com/pingcap/tidb/pkg/util/filter" 22 tcontext "github.com/pingcap/tiflow/dm/pkg/context" 23 "github.com/pingcap/tiflow/dm/pkg/terror" 24 "go.uber.org/atomic" 25 "go.uber.org/zap" 26 ) 27 28 type checkpointFlushTask struct { 29 snapshotInfo *SnapshotInfo 30 // extra sharding ddl sqls 31 exceptTables []*filter.Table 32 shardMetaSQLs []string 33 shardMetaArgs [][]interface{} 34 // async flush job 35 asyncflushJob *job 36 // error chan for sync flush 37 syncFlushErrCh chan error 38 } 39 40 type checkpointFlushWorker struct { 41 input chan *checkpointFlushTask 42 cp CheckPoint 43 execError *atomic.Error 44 afterFlushFn func(task *checkpointFlushTask) error 45 updateJobMetricsFn func(bool, string, *job) 46 } 47 48 // Add add a new flush checkpoint job. 49 func (w *checkpointFlushWorker) Add(msg *checkpointFlushTask) { 50 w.input <- msg 51 } 52 53 // Run read flush tasks from input and execute one by one. 54 func (w *checkpointFlushWorker) Run(ctx *tcontext.Context) { 55 for task := range w.input { 56 isAsyncFlush := task.asyncflushJob != nil 57 58 // for async checkpoint flush, it waits all worker finish execute flush job 59 // for sync checkpoint flush, it skips waiting here because 60 // we waits all worker finish execute flush job before adding flushCPTask into flush worker 61 if isAsyncFlush { 62 task.asyncflushJob.flushWg.Wait() 63 64 w.updateJobMetricsFn(true, adminQueueName, task.asyncflushJob) 65 ctx.L().Info("async flush checkpoint snapshot job has been processed by dml worker, about to flush checkpoint snapshot", zap.Int64("job sequence", task.asyncflushJob.flushSeq), zap.Int("snapshot_id", task.snapshotInfo.id)) 66 } else { 67 ctx.L().Info("about to sync flush checkpoint snapshot", zap.Int("snapshot_id", task.snapshotInfo.id)) 68 } 69 70 flushLogMsg := "sync flush" 71 if isAsyncFlush { 72 flushLogMsg = "async flush" 73 } 74 75 err := w.execError.Load() 76 // TODO: for now, if any error occurred (including user canceled), checkpoint won't be updated. But if we have put 77 // optimistic shard info, DM-master may resolved the optimistic lock and let other worker execute DDL. So after this 78 // worker resume, it can not execute the DML/DDL in old binlog because of downstream table structure mismatching. 79 // We should find a way to (compensating) implement a transaction containing interaction with both etcd and SQL. 80 if err != nil && (terror.ErrDBExecuteFailed.Equal(err) || terror.ErrDBUnExpect.Equal(err)) { 81 ctx.L().Warn(fmt.Sprintf("error detected when executing SQL job, skip %s checkpoint and shutdown checkpointFlushWorker", flushLogMsg), 82 zap.Stringer("globalPos", task.snapshotInfo.globalPos), 83 zap.Error(err)) 84 85 if !isAsyncFlush { 86 task.syncFlushErrCh <- nil 87 } 88 return 89 } 90 91 err = w.cp.FlushPointsExcept(ctx, task.snapshotInfo.id, task.exceptTables, task.shardMetaSQLs, task.shardMetaArgs) 92 failpoint.Inject("AsyncCheckpointFlushThrowError", func() { 93 if isAsyncFlush { 94 ctx.L().Warn("async checkpoint flush error triggered", zap.String("failpoint", "AsyncCheckpointFlushThrowError")) 95 err = errors.New("async checkpoint flush throw error") 96 } 97 }) 98 99 if err != nil { 100 ctx.L().Warn(fmt.Sprintf("%s checkpoint snapshot failed, ignore this error", flushLogMsg), zap.Any("flushCpTask", task), zap.Error(err)) 101 // async flush error will be skipped here but sync flush error will raised up 102 if !isAsyncFlush { 103 task.syncFlushErrCh <- err 104 } 105 continue 106 } 107 108 ctx.L().Info(fmt.Sprintf("%s checkpoint snapshot successfully", flushLogMsg), zap.Int("snapshot_id", task.snapshotInfo.id), 109 zap.Stringer("pos", task.snapshotInfo.globalPos)) 110 if err = w.afterFlushFn(task); err != nil { 111 ctx.L().Warn(fmt.Sprintf("%s post-process(afterFlushFn) failed", flushLogMsg), zap.Error(err)) 112 } 113 114 // async flush error will be skipped here but sync flush error will raised up 115 if !isAsyncFlush { 116 task.syncFlushErrCh <- err 117 } 118 } 119 } 120 121 // Close wait all pending job finish and stop this worker. 122 func (w *checkpointFlushWorker) Close() { 123 close(w.input) 124 }