github.com/pingcap/tiflow@v0.0.0-20240520035814-5bf52d54e205/cdc/processor/sinkmanager/redo_log_advancer.go (about) 1 // Copyright 2023 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 19 "github.com/pingcap/errors" 20 "github.com/pingcap/log" 21 "github.com/pingcap/tiflow/cdc/model" 22 "github.com/pingcap/tiflow/cdc/processor/memquota" 23 "github.com/pingcap/tiflow/cdc/processor/sourcemanager/sorter" 24 "github.com/pingcap/tiflow/cdc/redo" 25 "go.uber.org/zap" 26 ) 27 28 type redoLogAdvancer struct { 29 // NOTICE: This task is immutable, so please never modify it. 30 task *redoTask 31 // redoDMLManager is used to write the redo log. 32 redoDMLManager redo.DMLManager 33 // memQuota is used to acquire memory quota for the redo log writer. 34 memQuota *memquota.MemQuota 35 // NOTICE: First time to run the task, we have initialized memory quota for the table. 36 // It is defaultRequestMemSize. 37 availableMem uint64 38 // How much memory we have used. 39 // This is used to calculate how much memory we need to acquire. 40 // Only when usedMem > availableMem we need to acquire memory. 41 usedMem uint64 42 // Used to record the last written position. 43 // We need to use it to update the lower bound of the table sink. 44 lastPos sorter.Position 45 // Buffer the events to be written to the redo log. 46 events []*model.RowChangedEvent 47 48 // emittedCommitTs is used to record the last emitted transaction commit ts. 49 emittedCommitTs uint64 50 // Used to record the latest written transaction commit ts. 51 lastTxnCommitTs uint64 52 53 // pendingTxnSize used to record the size of the uncommitted events. 54 pendingTxnSize uint64 55 // Used to record the current transaction commit ts. 56 currTxnCommitTs uint64 57 } 58 59 func newRedoLogAdvancer( 60 task *redoTask, 61 memQuota *memquota.MemQuota, 62 availableMem uint64, 63 redoDMLManager redo.DMLManager, 64 ) *redoLogAdvancer { 65 return &redoLogAdvancer{ 66 task: task, 67 memQuota: memQuota, 68 availableMem: availableMem, 69 events: make([]*model.RowChangedEvent, 0, bufferSize), 70 redoDMLManager: redoDMLManager, 71 } 72 } 73 74 // advance tries to emit the events to the redo log manager and 75 // advance the resolved ts of the redo log manager. 76 func (a *redoLogAdvancer) advance(ctx context.Context) error { 77 if len(a.events) > 0 { 78 // releaseMem is used to release the memory quota 79 // after the events are written to redo log. 80 // It more like a callback function. 81 var releaseMem func() 82 refundMem := a.pendingTxnSize 83 if refundMem > 0 { 84 releaseMem = func() { 85 a.memQuota.Refund(refundMem) 86 log.Debug("MemoryQuotaTracing: refund memory for redo log task", 87 zap.String("namespace", a.task.tableSink.changefeed.Namespace), 88 zap.String("changefeed", a.task.tableSink.changefeed.ID), 89 zap.Stringer("span", &a.task.span), 90 zap.Uint64("memory", refundMem)) 91 } 92 } 93 if err := a.redoDMLManager.EmitRowChangedEvents(ctx, a.task.span, releaseMem, 94 a.events...); err != nil { 95 return errors.Trace(err) 96 } 97 a.events = a.events[:0] 98 if cap(a.events) > bufferSize { 99 a.events = make([]*model.RowChangedEvent, 0, bufferSize) 100 } 101 a.pendingTxnSize = 0 102 } 103 if a.lastTxnCommitTs > a.emittedCommitTs { 104 if err := a.redoDMLManager.UpdateResolvedTs(ctx, a.task.span, 105 a.lastTxnCommitTs); err != nil { 106 return errors.Trace(err) 107 } 108 log.Debug("update resolved ts to redo", 109 zap.String("namespace", a.task.tableSink.changefeed.Namespace), 110 zap.String("changefeed", a.task.tableSink.changefeed.ID), 111 zap.Stringer("span", &a.task.span), 112 zap.Uint64("resolvedTs", a.lastTxnCommitTs)) 113 a.emittedCommitTs = a.lastTxnCommitTs 114 } 115 116 return nil 117 } 118 119 // tryAdvanceAndAcquireMem tries to acquire the memory quota and advance the redo log manager. 120 // allFetched indicates whether all the events have been fetched. Then we 121 // do not need to acquire the memory quota anymore. 122 // txnFinished indicates whether the current transaction has been finished. 123 // If it is finished, it is OK to wait next round task to advance the table sink. 124 // Otherwise, we need to advance the redo log at least to the current transaction. 125 func (a *redoLogAdvancer) tryAdvanceAndAcquireMem( 126 ctx context.Context, 127 allFetched bool, 128 txnFinished bool, 129 ) error { 130 // If used memory size exceeds the required limit, do a force acquire to 131 // make sure the memory quota is not exceeded or leak. 132 // For example, if the memory quota is 100MB, and current usedMem is 90MB, 133 // and availableMem is 100MB, then we can get event from the source manager 134 // but if the event size is 20MB, we just exceed the available memory quota temporarily. 135 // So we need to force acquire the memory quota to make up the difference. 136 exceedAvailableMem := a.availableMem < a.usedMem 137 if exceedAvailableMem { 138 a.memQuota.ForceAcquire(a.usedMem - a.availableMem) 139 log.Debug("MemoryQuotaTracing: force acquire memory for redo log task", 140 zap.String("namespace", a.task.tableSink.changefeed.Namespace), 141 zap.String("changefeed", a.task.tableSink.changefeed.ID), 142 zap.Stringer("span", &a.task.span), 143 zap.Uint64("memory", a.usedMem-a.availableMem)) 144 a.availableMem = a.usedMem 145 } 146 147 // Do emit in such situations: 148 // 1. we use more memory than we required; 149 // 2. the pending batch size exceeds maxUpdateIntervalSize; 150 // 3. all events are received. 151 if exceedAvailableMem || a.pendingTxnSize >= maxUpdateIntervalSize || allFetched { 152 if err := a.advance( 153 ctx, 154 ); err != nil { 155 return errors.Trace(err) 156 } 157 } 158 159 if allFetched { 160 return nil 161 } 162 163 if a.usedMem >= a.availableMem { 164 if txnFinished { 165 if a.memQuota.TryAcquire(requestMemSize) { 166 a.availableMem += requestMemSize 167 log.Debug("MemoryQuotaTracing: try acquire memory for redo log task", 168 zap.String("namespace", a.task.tableSink.changefeed.Namespace), 169 zap.String("changefeed", a.task.tableSink.changefeed.ID), 170 zap.Stringer("span", &a.task.span), 171 zap.Uint64("memory", requestMemSize)) 172 } 173 } else { 174 // NOTE: it's not required to use `forceAcquire` even if splitTxn is false. 175 // It's because memory will finally be `refund` after redo-logs are written. 176 if err := a.memQuota.BlockAcquire(requestMemSize); err != nil { 177 return errors.Trace(err) 178 } 179 a.availableMem += requestMemSize 180 log.Debug("MemoryQuotaTracing: block acquire memory for redo log task", 181 zap.String("namespace", a.task.tableSink.changefeed.Namespace), 182 zap.String("changefeed", a.task.tableSink.changefeed.ID), 183 zap.Stringer("span", &a.task.span), 184 zap.Uint64("memory", requestMemSize)) 185 } 186 } 187 188 return nil 189 } 190 191 func (a *redoLogAdvancer) finish( 192 ctx context.Context, 193 upperBound sorter.Position, 194 ) error { 195 a.lastPos = upperBound 196 a.lastTxnCommitTs = upperBound.CommitTs 197 err := a.tryAdvanceAndAcquireMem( 198 ctx, 199 true, 200 true, 201 ) 202 return err 203 } 204 205 // tryMoveToNextTxn tries to move to the next transaction. 206 // 1. If the commitTs is different from the current transaction, it means 207 // the current transaction is finished. We need to move to the next transaction. 208 // 2. If current position is a commit fence, it means the current transaction 209 // is finished. We can safely move to the next transaction early. It would be 210 // helpful to advance the redo log manager. 211 func (a *redoLogAdvancer) tryMoveToNextTxn(commitTs model.Ts, pos sorter.Position) { 212 if a.currTxnCommitTs != commitTs { 213 a.lastTxnCommitTs = a.currTxnCommitTs 214 a.currTxnCommitTs = commitTs 215 } 216 217 // If the current position is a commit fence, it means the current transaction 218 // is finished. We can safely move to the next transaction early. 219 // NOTICE: Please do not combine this condition with the previous one. 220 // There is a case that the current position is a commit fence, also 221 // the commitTs is different from the current transaction. 222 // For example: 223 // 1. current commitTs is 10 224 // 2. commitTs is 11 225 // 3. pos is a commit fence (10,11) 226 // In this case, we should move to the next transaction. 227 // The lastTxnCommitTs should be 11, not 10. 228 if pos.IsCommitFence() { 229 a.lastTxnCommitTs = a.currTxnCommitTs 230 } 231 } 232 233 // appendEvents appends events to the buffer and record the memory usage. 234 func (a *redoLogAdvancer) appendEvents(events []*model.RowChangedEvent, size uint64) { 235 a.events = append(a.events, events...) 236 // Record the memory usage. 237 a.usedMem += size 238 // Record the pending transaction size. It means how many events we do 239 // not flush to the redo log manager. 240 a.pendingTxnSize += size 241 } 242 243 // hasEnoughMem returns whether the redo log task has enough memory to continue. 244 func (a *redoLogAdvancer) hasEnoughMem() bool { 245 return a.availableMem > a.usedMem 246 } 247 248 // cleanup cleans up the memory usage. 249 // Refund the memory usage if we do not use it. 250 func (a *redoLogAdvancer) cleanup() { 251 if a.availableMem > a.usedMem { 252 a.memQuota.Refund(a.availableMem - a.usedMem) 253 log.Debug("MemoryQuotaTracing: refund memory for redo log task", 254 zap.String("namespace", a.task.tableSink.changefeed.Namespace), 255 zap.String("changefeed", a.task.tableSink.changefeed.ID), 256 zap.Stringer("span", &a.task.span), 257 zap.Uint64("memory", a.availableMem-a.usedMem)) 258 } 259 }