github.com/pingcap/tiflow@v0.0.0-20240520035814-5bf52d54e205/cdc/processor/sinkmanager/redo_log_worker.go (about) 1 // Copyright 2022 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 sinkmanager 15 16 import ( 17 "context" 18 "time" 19 20 "github.com/pingcap/errors" 21 "github.com/pingcap/log" 22 "github.com/pingcap/tiflow/cdc/model" 23 "github.com/pingcap/tiflow/cdc/processor/memquota" 24 "github.com/pingcap/tiflow/cdc/processor/sourcemanager" 25 "github.com/pingcap/tiflow/cdc/redo" 26 "github.com/tikv/client-go/v2/oracle" 27 "go.uber.org/zap" 28 ) 29 30 type redoWorker struct { 31 changefeedID model.ChangeFeedID 32 sourceManager *sourcemanager.SourceManager 33 memQuota *memquota.MemQuota 34 redoDMLManager redo.DMLManager 35 } 36 37 func newRedoWorker( 38 changefeedID model.ChangeFeedID, 39 sourceManager *sourcemanager.SourceManager, 40 quota *memquota.MemQuota, 41 redoDMLMgr redo.DMLManager, 42 ) *redoWorker { 43 return &redoWorker{ 44 changefeedID: changefeedID, 45 sourceManager: sourceManager, 46 memQuota: quota, 47 redoDMLManager: redoDMLMgr, 48 } 49 } 50 51 func (w *redoWorker) handleTasks(ctx context.Context, taskChan <-chan *redoTask) error { 52 for { 53 select { 54 case <-ctx.Done(): 55 return ctx.Err() 56 case task := <-taskChan: 57 if err := w.handleTask(ctx, task); err != nil { 58 return errors.Trace(err) 59 } 60 } 61 } 62 } 63 64 func (w *redoWorker) handleTask(ctx context.Context, task *redoTask) (finalErr error) { 65 advancer := newRedoLogAdvancer(task, w.memQuota, requestMemSize, w.redoDMLManager) 66 // The task is finished and some required memory isn't used. 67 defer advancer.cleanup() 68 69 lowerBound, upperBound := validateAndAdjustBound( 70 w.changefeedID, 71 &task.span, 72 task.lowerBound, 73 task.getUpperBound(task.tableSink.getReceivedSorterResolvedTs()), 74 ) 75 advancer.lastPos = lowerBound.Prev() 76 77 allEventCount := 0 78 79 iter := w.sourceManager.FetchByTable(task.span, lowerBound, upperBound, w.memQuota) 80 defer func() { 81 if err := iter.Close(); err != nil { 82 log.Error("redo worker fails to close iterator", 83 zap.String("namespace", w.changefeedID.Namespace), 84 zap.String("changefeed", w.changefeedID.ID), 85 zap.Stringer("span", &task.span), 86 zap.Error(err)) 87 } 88 }() 89 90 defer func() { 91 log.Debug("redo task finished", 92 zap.String("namespace", w.changefeedID.Namespace), 93 zap.String("changefeed", w.changefeedID.ID), 94 zap.Stringer("span", &task.span), 95 zap.Any("lowerBound", lowerBound), 96 zap.Any("upperBound", upperBound), 97 zap.Any("lastPos", advancer.lastPos), 98 zap.Float64("lag", time.Since(oracle.GetTimeFromTS(advancer.lastPos.CommitTs)).Seconds()), 99 zap.Error(finalErr)) 100 101 if finalErr == nil { 102 // Otherwise we can't ensure all events before `lastPos` are emitted. 103 task.callback(advancer.lastPos) 104 } 105 }() 106 107 for advancer.hasEnoughMem() && !task.isCanceled() { 108 e, pos, err := iter.Next(ctx) 109 if err != nil { 110 return errors.Trace(err) 111 } 112 // There is no more data. It means that we finish this scan task. 113 if e == nil { 114 return advancer.finish(ctx, upperBound) 115 } 116 117 allEventCount += 1 118 119 if pos.Valid() { 120 advancer.lastPos = pos 121 } 122 123 advancer.tryMoveToNextTxn(e.CRTs, pos) 124 125 var x []*model.RowChangedEvent 126 var size uint64 127 // NOTICE: The event can be filtered by the event filter. 128 if e.Row != nil { 129 // For all events, we add table replicate ts, so mysql sink can determine safe-mode. 130 e.Row.ReplicatingTs = task.tableSink.replicateTs 131 x, size = handleRowChangedEvents(w.changefeedID, task.span, e) 132 advancer.appendEvents(x, size) 133 } 134 135 err = advancer.tryAdvanceAndAcquireMem( 136 ctx, 137 false, 138 pos.Valid(), 139 ) 140 if err != nil { 141 return errors.Trace(err) 142 } 143 } 144 145 // Even if task is canceled we still call this again, to avoid something 146 // are left and leak forever. 147 return advancer.advance(ctx) 148 }