github.com/pingcap/tiflow@v0.0.0-20240520035814-5bf52d54e205/cdc/processor/sinkmanager/table_sink_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 "github.com/pingcap/errors" 18 "github.com/pingcap/log" 19 "github.com/pingcap/tiflow/cdc/model" 20 "github.com/pingcap/tiflow/cdc/processor/memquota" 21 "github.com/pingcap/tiflow/cdc/processor/sourcemanager/sorter" 22 "go.uber.org/zap" 23 ) 24 25 type tableSinkAdvancer struct { 26 // NOTICE: This task is immutable, so please never modify it. 27 task *sinkTask 28 // splitTxn indicates whether to split the transaction into multiple batches. 29 splitTxn bool 30 // sinkMemQuota is used to acquire memory quota for the table sink. 31 sinkMemQuota *memquota.MemQuota 32 // NOTICE: First time to run the task, we have initialized memory quota for the table. 33 // It is defaultRequestMemSize. 34 availableMem uint64 35 // How much memory we have used. 36 // This is used to calculate how much memory we need to acquire. 37 // Only when usedMem > availableMem we need to acquire memory. 38 usedMem uint64 39 // Used to record the last written position. 40 // We need to use it to update the lower bound of the table sink. 41 lastPos sorter.Position 42 // Buffer the events to be written to the table sink. 43 events []*model.RowChangedEvent 44 45 // Used to record the size of already appended transaction. 46 committedTxnSize uint64 47 // Used to record the latest written transaction commit ts. 48 lastTxnCommitTs uint64 49 50 // Used to record the size of the current transaction. 51 pendingTxnSize uint64 52 // Used to record the current transaction commit ts. 53 currTxnCommitTs uint64 54 } 55 56 func newTableSinkAdvancer( 57 task *sinkTask, 58 splitTxn bool, 59 sinkMemQuota *memquota.MemQuota, 60 availableMem uint64, 61 ) *tableSinkAdvancer { 62 return &tableSinkAdvancer{ 63 task: task, 64 splitTxn: splitTxn, 65 sinkMemQuota: sinkMemQuota, 66 availableMem: availableMem, 67 events: make([]*model.RowChangedEvent, 0, bufferSize), 68 } 69 } 70 71 // advance tries to append the event to the table sink 72 // and advance the table sink. 73 // isLastTime indicates whether this is the last time to call advance. 74 // If it is the last time, and we still have some events in the buffer, 75 // we need to record the memory usage and append the events to the table sink. 76 func (a *tableSinkAdvancer) advance(isLastTime bool) (err error) { 77 // Append the events to the table sink first. 78 if len(a.events) > 0 { 79 if err = a.task.tableSink.appendRowChangedEvents(a.events...); err != nil { 80 return 81 } 82 a.events = a.events[:0] 83 if cap(a.events) > bufferSize { 84 a.events = make([]*model.RowChangedEvent, 0, bufferSize) 85 } 86 } 87 log.Debug("check should advance or not", 88 zap.String("namespace", a.task.tableSink.changefeed.Namespace), 89 zap.String("changefeed", a.task.tableSink.changefeed.ID), 90 zap.Stringer("span", &a.task.span), 91 zap.Bool("splitTxn", a.splitTxn), 92 zap.Uint64("currTxnCommitTs", a.currTxnCommitTs), 93 zap.Uint64("lastTxnCommitTs", a.lastTxnCommitTs), 94 zap.Bool("isLastTime", isLastTime)) 95 96 // Still got the same commit ts. 97 if a.currTxnCommitTs == a.lastPos.CommitTs { 98 // All transactions before currTxnCommitTs are resolved. 99 if a.lastPos.IsCommitFence() { 100 err = advanceTableSink(a.task, a.currTxnCommitTs, 101 a.committedTxnSize+a.pendingTxnSize, a.sinkMemQuota) 102 } else { 103 // This means all events of the current transaction have been fetched, but we can't 104 // ensure whether there are more transaction with the same CommitTs or not. 105 // So we need to advance the table sink with a batchID. It will make sure that 106 // we do not cross the CommitTs boundary. 107 err = advanceTableSinkWithBatchID(a.task, a.currTxnCommitTs, 108 a.committedTxnSize+a.pendingTxnSize, batchID.Load(), a.sinkMemQuota) 109 batchID.Add(1) 110 } 111 112 a.committedTxnSize = 0 113 a.pendingTxnSize = 0 114 } else if a.splitTxn && a.currTxnCommitTs > 0 { 115 // We just got a new commit ts. Because we split the transaction, 116 // we can advance the table sink with the current commit ts. 117 // This will advance some complete transactions before currTxnCommitTs, 118 // and one partial transaction with `batchID`. 119 err = advanceTableSinkWithBatchID(a.task, a.currTxnCommitTs, 120 a.committedTxnSize+a.pendingTxnSize, batchID.Load(), a.sinkMemQuota) 121 122 batchID.Add(1) 123 a.committedTxnSize = 0 124 a.pendingTxnSize = 0 125 } else if !a.splitTxn && a.lastTxnCommitTs > 0 { 126 // We just got a new commit ts. Because we don't split the transaction, 127 // we **only** advance the table sink by the last transaction commit ts. 128 err = advanceTableSink(a.task, a.lastTxnCommitTs, 129 a.committedTxnSize, a.sinkMemQuota) 130 a.committedTxnSize = 0 131 // If it is the last time we call `advance`, but `pendingTxnSize` 132 // hasn't been recorded yet. To avoid losing it, record it manually. 133 if isLastTime && a.pendingTxnSize > 0 { 134 a.sinkMemQuota.Record(a.task.span, 135 model.NewResolvedTs(a.currTxnCommitTs), a.pendingTxnSize) 136 a.pendingTxnSize = 0 137 } 138 } 139 return 140 } 141 142 // lastTimeAdvance only happens when there is no enough memory quota to 143 // acquire, and the task is not finished. 144 // In this case, we need to try to advance the table sink as much as possible. 145 func (a *tableSinkAdvancer) lastTimeAdvance() error { 146 return a.advance(true) 147 } 148 149 // tryAdvanceAndAcquireMem tries to acquire the memory quota and advance the table sink. 150 // allFetched indicates whether all the events have been fetched. Then we 151 // do not need to acquire the memory quota anymore. 152 // txnFinished indicates whether the current transaction has been finished. 153 // If it is finished, it is OK to wait next round task to advance the table sink. 154 // Otherwise, we need to advance the table at least to the current transaction. 155 func (a *tableSinkAdvancer) tryAdvanceAndAcquireMem( 156 allFetched bool, 157 txnFinished bool, 158 ) error { 159 // If used memory size exceeds the required limit, do a force acquire to 160 // make sure the memory quota is not exceeded or leak. 161 // For example, if the memory quota is 100MB, and current usedMem is 90MB, 162 // and availableMem is 100MB, then we can get event from the source manager 163 // but if the event size is 20MB, we just exceed the available memory quota temporarily. 164 // So we need to force acquire the memory quota to make up the difference. 165 exceedAvailableMem := a.availableMem < a.usedMem 166 if exceedAvailableMem { 167 a.sinkMemQuota.ForceAcquire(a.usedMem - a.availableMem) 168 log.Debug("MemoryQuotaTracing: force acquire memory for table sink task", 169 zap.String("namespace", a.task.tableSink.changefeed.Namespace), 170 zap.String("changefeed", a.task.tableSink.changefeed.ID), 171 zap.Stringer("span", &a.task.span), 172 zap.Uint64("memory", a.usedMem-a.availableMem)) 173 a.availableMem = a.usedMem 174 } 175 176 // Do emit in such situations: 177 // 1. we use more memory than we required; 178 // 2. all events are received. 179 // 3. the pending batch size exceeds maxUpdateIntervalSize; 180 if exceedAvailableMem || allFetched || 181 needEmitAndAdvance(a.splitTxn, a.committedTxnSize, a.pendingTxnSize) { 182 if err := a.advance(false); err != nil { 183 return errors.Trace(err) 184 } 185 } 186 187 // All fetched, no need to acquire memory. 188 if allFetched { 189 return nil 190 } 191 192 if a.usedMem >= a.availableMem { 193 // We just finished a transaction, and the memory usage is still high. 194 // We need try to acquire memory for the next transaction. It is possible 195 // we can't acquire memory, but we finish the current transaction. So 196 // we can wait for next round. 197 if txnFinished { 198 if a.sinkMemQuota.TryAcquire(requestMemSize) { 199 a.availableMem += requestMemSize 200 log.Debug("MemoryQuotaTracing: try acquire memory for table sink task", 201 zap.String("namespace", a.task.tableSink.changefeed.Namespace), 202 zap.String("changefeed", a.task.tableSink.changefeed.ID), 203 zap.Stringer("span", &a.task.span), 204 zap.Uint64("memory", requestMemSize)) 205 } 206 } else { 207 // The transaction is not finished and splitTxn is false, we need to 208 // force acquire memory. Because we can't leave rest data 209 // to the next round. 210 if !a.splitTxn { 211 a.sinkMemQuota.ForceAcquire(requestMemSize) 212 a.availableMem += requestMemSize 213 log.Debug("MemoryQuotaTracing: force acquire memory for table sink task", 214 zap.String("namespace", a.task.tableSink.changefeed.Namespace), 215 zap.String("changefeed", a.task.tableSink.changefeed.ID), 216 zap.Stringer("span", &a.task.span), 217 zap.Uint64("memory", requestMemSize)) 218 } else { 219 // NOTE: if splitTxn is true it's not required to force acquire memory. 220 // We can wait for a while because we already flushed some data to 221 // the table sink. 222 if err := a.sinkMemQuota.BlockAcquire(requestMemSize); err != nil { 223 return errors.Trace(err) 224 } 225 a.availableMem += requestMemSize 226 log.Debug("MemoryQuotaTracing: block acquire memory for table sink task", 227 zap.String("namespace", a.task.tableSink.changefeed.Namespace), 228 zap.String("changefeed", a.task.tableSink.changefeed.ID), 229 zap.Stringer("span", &a.task.span), 230 zap.Uint64("memory", requestMemSize)) 231 } 232 } 233 } 234 return nil 235 } 236 237 // tryMoveToNextTxn tries to move to the next transaction. 238 // If the commitTs is different from the current transaction, it means 239 // the current transaction is finished. We need to move to the next transaction. 240 func (a *tableSinkAdvancer) tryMoveToNextTxn(commitTs model.Ts) { 241 if a.currTxnCommitTs != commitTs { 242 // Record the last transaction commitTs and size. 243 a.lastTxnCommitTs = a.currTxnCommitTs 244 a.committedTxnSize += a.pendingTxnSize 245 // Move to the next transaction. 246 a.currTxnCommitTs = commitTs 247 a.pendingTxnSize = 0 248 } 249 } 250 251 // finish finishes the table sink task. 252 // It will move the table sink task to the upperBound position. 253 func (a *tableSinkAdvancer) finish(upperBound sorter.Position) error { 254 a.lastPos = upperBound 255 a.currTxnCommitTs = upperBound.CommitTs 256 a.lastTxnCommitTs = upperBound.CommitTs 257 return a.tryAdvanceAndAcquireMem( 258 true, 259 true, 260 ) 261 } 262 263 // appendEvents appends events to the buffer and record the memory usage. 264 func (a *tableSinkAdvancer) appendEvents(events []*model.RowChangedEvent, size uint64) { 265 a.events = append(a.events, events...) 266 // Record the memory usage. 267 a.usedMem += size 268 // Record the pending transaction size. It means how many events we do 269 // not flush to the table sink. 270 a.pendingTxnSize += size 271 } 272 273 // hasEnoughMem returns whether the table sink task has enough memory to continue. 274 func (a *tableSinkAdvancer) hasEnoughMem() bool { 275 return a.availableMem > a.usedMem 276 } 277 278 // cleanup cleans up the memory usage. 279 // Refund the memory usage if we do not use it. 280 func (a *tableSinkAdvancer) cleanup() { 281 if a.availableMem > a.usedMem { 282 a.sinkMemQuota.Refund(a.availableMem - a.usedMem) 283 log.Debug("MemoryQuotaTracing: refund memory for table sink task", 284 zap.String("namespace", a.task.tableSink.changefeed.Namespace), 285 zap.String("changefeed", a.task.tableSink.changefeed.ID), 286 zap.Stringer("span", &a.task.span), 287 zap.Uint64("memory", a.availableMem-a.usedMem)) 288 } 289 } 290 291 func advanceTableSinkWithBatchID( 292 t *sinkTask, 293 commitTs model.Ts, 294 size uint64, 295 batchID uint64, 296 sinkMemQuota *memquota.MemQuota, 297 ) error { 298 resolvedTs := model.NewResolvedTs(commitTs) 299 resolvedTs.Mode = model.BatchResolvedMode 300 resolvedTs.BatchID = batchID 301 log.Debug("Advance table sink with batch ID", 302 zap.String("namespace", t.tableSink.changefeed.Namespace), 303 zap.String("changefeed", t.tableSink.changefeed.ID), 304 zap.Stringer("span", &t.span), 305 zap.Any("resolvedTs", resolvedTs), 306 zap.Uint64("size", size)) 307 if size > 0 { 308 sinkMemQuota.Record(t.span, resolvedTs, size) 309 } 310 return t.tableSink.updateResolvedTs(resolvedTs) 311 } 312 313 func advanceTableSink( 314 t *sinkTask, 315 commitTs model.Ts, 316 size uint64, 317 sinkMemQuota *memquota.MemQuota, 318 ) error { 319 resolvedTs := model.NewResolvedTs(commitTs) 320 log.Debug("Advance table sink without batch ID", 321 zap.String("namespace", t.tableSink.changefeed.Namespace), 322 zap.String("changefeed", t.tableSink.changefeed.ID), 323 zap.Stringer("span", &t.span), 324 zap.Any("resolvedTs", resolvedTs), 325 zap.Uint64("size", size)) 326 if size > 0 { 327 sinkMemQuota.Record(t.span, resolvedTs, size) 328 } 329 return t.tableSink.updateResolvedTs(resolvedTs) 330 } 331 332 func needEmitAndAdvance(splitTxn bool, committedTxnSize uint64, pendingTxnSize uint64) bool { 333 // If splitTxn is true, we can safely emit all the events in the last transaction 334 // and current transaction. So we use `committedTxnSize+pendingTxnSize`. 335 splitTxnEmitCondition := splitTxn && committedTxnSize+pendingTxnSize >= maxUpdateIntervalSize 336 // If splitTxn is false, we need to emit the events when the size of the 337 // transaction is greater than maxUpdateIntervalSize. 338 // This could help to reduce the overhead of emit and advance too frequently. 339 noSplitTxnEmitCondition := !splitTxn && committedTxnSize >= maxUpdateIntervalSize 340 return splitTxnEmitCondition || 341 noSplitTxnEmitCondition 342 }