github.com/pingcap/tiflow@v0.0.0-20240520035814-5bf52d54e205/cdc/sink/tablesink/progress_tracker.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 tablesink 15 16 import ( 17 "math" 18 "sync" 19 "sync/atomic" 20 "time" 21 22 "github.com/pingcap/log" 23 "github.com/pingcap/tiflow/cdc/model" 24 "github.com/pingcap/tiflow/cdc/processor/tablepb" 25 "go.uber.org/zap" 26 ) 27 28 const ( 29 // waitingInterval is the interval to wait for the all callbacks be called. 30 // It used for closing the table sink. 31 waitingInterval = 100 * time.Millisecond 32 // warnDuration is the duration to warn the progress tracker is not closed. 33 warnDuration = 30 * time.Second 34 // A progressTracker contains several internal fixed-length buffers. 35 // NOTICE: the buffer size must be aligned to 8 bytes. 36 // It shouldn't be too large, otherwise it will consume too much memory. 37 defaultBufferSize uint64 = 4096 38 ) 39 40 // A pendingResolvedTs is received by progressTracker but hasn't been flushed yet. 41 type pendingResolvedTs struct { 42 offset uint64 43 resolvedTs model.ResolvedTs 44 } 45 46 // progressTracker is used to track the progress of the table sink. 47 // 48 // For example, 49 // We have txn1, txn2, resolvedTs2, txn3-1, txn3-2, resolvedTs3, resolvedTs4, resolvedTs5. 50 // txn3-1 and txn3-2 are in the same big txn. 51 // First txn1 and txn2 are written, then the progress can be updated to resolvedTs2. 52 // Then txn3-1 and txn3-2 are written, then the progress can be updated to resolvedTs3. 53 // Next, since no data is being written, we can update to resolvedTs5 in order. 54 // 55 // The core of the algorithm is `pendingEvents`, which is a bit map for all events. 56 // Every event is associated with a `eventID` which is a continuous number. `eventID` 57 // can be regarded as the event's offset in `pendingEvents`. 58 type progressTracker struct { 59 // span is the span of the table sink. 60 span tablepb.Span 61 62 // Internal Buffer size. Modified in tests only. 63 bufferSize uint64 64 65 // Following fields are protected by `mu`. 66 mu sync.Mutex 67 68 // frozen is used to indicate whether the progress tracker is frozen. 69 // It means we do not advance anymore. 70 frozen bool 71 72 // Used to generate the next eventID. 73 nextEventID uint64 74 75 // Every received event is a bit in `pendingEvents`. 76 pendingEvents [][]uint64 77 78 // When old events are flushed the buffer should be released. 79 nextToReleasePos uint64 80 81 // The position that the next event which should be check in `advance`. 82 nextToResolvePos uint64 83 84 resolvedTsCache []pendingResolvedTs 85 86 lastMinResolvedTs model.ResolvedTs 87 88 lastCheckClosed atomic.Int64 89 } 90 91 // newProgressTracker is used to create a new progress tracker. 92 // The last min resolved ts is set to 0. 93 // It means that the table sink has not started yet. 94 func newProgressTracker(span tablepb.Span, bufferSize uint64) *progressTracker { 95 if bufferSize%8 != 0 { 96 panic("bufferSize must be align to 8 bytes") 97 } 98 99 return &progressTracker{ 100 span: span, 101 bufferSize: bufferSize / 8, 102 // It means the start of the table. 103 // It's Ok to use 0 here. 104 // Because sink node only update the checkpoint when it's growing. 105 lastMinResolvedTs: model.NewResolvedTs(0), 106 } 107 } 108 109 // addEvent is used to add the pending event key. `postEventFlush` should be called 110 // when the event has been flushed. 111 func (r *progressTracker) addEvent() (postEventFlush func()) { 112 r.mu.Lock() 113 defer r.mu.Unlock() 114 115 eventID := r.nextEventID 116 bit := eventID % 64 117 r.nextEventID += 1 118 119 bufferCount := len(r.pendingEvents) 120 if bufferCount == 0 || (uint64(len(r.pendingEvents[bufferCount-1])) == r.bufferSize && bit == 0) { 121 // If there is no buffer or the last one is full, we need to allocate a new one. 122 buffer := make([]uint64, 0, r.bufferSize) 123 r.pendingEvents = append(r.pendingEvents, buffer) 124 bufferCount += 1 125 } 126 127 if bit == 0 { 128 // If bit is 0 it means we need to append a new uint64 word for the event. 129 r.pendingEvents[bufferCount-1] = append(r.pendingEvents[bufferCount-1], 0) 130 } 131 lastBuffer := r.pendingEvents[bufferCount-1] 132 133 // Set the corresponding bit to 1. 134 // For example, if the eventID is 3, the bit is 3 % 64 = 3. 135 // 0000000000000000000000000000000000000000000000000000000000000000 -> 136 // 0000000000000000000000000000000000000000000000000000000000001000 137 // When we advance the progress, we can try to find the first 0 bit to indicate the progress. 138 postEventFlush = func() { atomic.AddUint64(&lastBuffer[len(lastBuffer)-1], 1<<bit) } 139 return 140 } 141 142 // addResolvedTs is used to add the pending resolved ts. 143 func (r *progressTracker) addResolvedTs(resolvedTs model.ResolvedTs) { 144 r.mu.Lock() 145 defer r.mu.Unlock() 146 147 // NOTICE: We should **NOT** update the `lastMinResolvedTs` when tracker is frozen. 148 // So there is no need to try to append the resolved ts to `resolvedTsCache`. 149 if r.frozen { 150 return 151 } 152 153 // If there is no event or all events are flushed, we can update the resolved ts directly. 154 if r.nextEventID == 0 || r.nextToResolvePos >= r.nextEventID { 155 // Update the checkpoint ts. 156 r.lastMinResolvedTs = resolvedTs 157 return 158 } 159 160 // Sometimes, if there are no events for a long time and a lot of resolved ts are received, 161 // we can update the last resolved ts directly. 162 tsCacheLen := len(r.resolvedTsCache) 163 if tsCacheLen > 0 { 164 // The offset of the last resolved ts is the last event ID. 165 // It means no event is adding. We can update the resolved ts directly. 166 if r.resolvedTsCache[tsCacheLen-1].offset+1 == r.nextEventID { 167 r.resolvedTsCache[tsCacheLen-1].resolvedTs = resolvedTs 168 return 169 } 170 } 171 172 r.resolvedTsCache = append(r.resolvedTsCache, pendingResolvedTs{ 173 offset: r.nextEventID - 1, 174 resolvedTs: resolvedTs, 175 }) 176 } 177 178 // advance tries to move forward the tracker and returns the latest resolved timestamp. 179 func (r *progressTracker) advance() model.ResolvedTs { 180 r.mu.Lock() 181 defer r.mu.Unlock() 182 183 // `pendingEvents` is like a 3-dimo bit array. To access a given bit in the array, 184 // use `pendingEvents[idx1][idx2][idx3]`. 185 // The first index is used to access the buffer. 186 // The second index is used to access the uint64 in the buffer. 187 // The third index is used to access the bit in the uint64. 188 offset := r.nextToResolvePos - r.nextToReleasePos 189 idx1 := offset / (r.bufferSize * 64) 190 idx2 := offset % (r.bufferSize * 64) / 64 191 idx3 := offset % (r.bufferSize * 64) % 64 192 193 for { 194 if r.nextToResolvePos >= r.nextEventID { 195 // All events are resolved. 196 break 197 } 198 199 currBitMap := atomic.LoadUint64(&r.pendingEvents[idx1][idx2]) 200 if currBitMap == math.MaxUint64 { 201 // Move to the next uint64 word (maybe in the next buffer). 202 idx2 += 1 203 if idx2 >= r.bufferSize { 204 idx2 = 0 205 idx1 += 1 206 } 207 r.nextToResolvePos += 64 - idx3 208 idx3 = 0 209 } else { 210 // Try to find the first 0 bit in the word. 211 for i := idx3; i < 64; i++ { 212 if currBitMap&uint64(1<<i) == 0 { 213 r.nextToResolvePos += i - idx3 214 break 215 } 216 } 217 break 218 } 219 } 220 221 // Try to advance resolved timestamp based on `nextToResolvePos`. 222 if r.nextToResolvePos > 0 { 223 for len(r.resolvedTsCache) > 0 { 224 cached := r.resolvedTsCache[0] 225 if cached.offset <= r.nextToResolvePos-1 { 226 // NOTICE: We should **NOT** update the `lastMinResolvedTs` when tracker is frozen. 227 if !r.frozen { 228 r.lastMinResolvedTs = cached.resolvedTs 229 } 230 r.resolvedTsCache = r.resolvedTsCache[1:] 231 if len(r.resolvedTsCache) == 0 { 232 r.resolvedTsCache = nil 233 } 234 } else { 235 break 236 } 237 } 238 } 239 240 // If a buffer is finished, release it. 241 for r.nextToResolvePos-r.nextToReleasePos >= r.bufferSize*64 { 242 r.nextToReleasePos += r.bufferSize * 64 243 // Use zero value to release the memory. 244 r.pendingEvents[0] = nil 245 r.pendingEvents = r.pendingEvents[1:] 246 if len(r.pendingEvents) == 0 { 247 r.pendingEvents = nil 248 } 249 } 250 251 return r.lastMinResolvedTs 252 } 253 254 // trackingCount returns the number of pending events and resolved timestamps. 255 // Notice: must hold the lock. 256 func (r *progressTracker) trackingCount() int { 257 r.mu.Lock() 258 defer r.mu.Unlock() 259 return int(r.nextEventID - r.nextToResolvePos) 260 } 261 262 // freezeProcess marks we do not advance checkpoint ts anymore. 263 func (r *progressTracker) freezeProcess() { 264 r.mu.Lock() 265 defer r.mu.Unlock() 266 if !r.frozen { 267 r.frozen = true 268 r.lastCheckClosed.Store(time.Now().Unix()) 269 } 270 } 271 272 // close is used to close the progress tracker. 273 func (r *progressTracker) waitClosed(backendDead <-chan struct{}) { 274 waitingTicker := time.NewTicker(waitingInterval) 275 defer waitingTicker.Stop() 276 for { 277 select { 278 case <-backendDead: 279 r.advance() 280 return 281 case <-waitingTicker.C: 282 if r.doCheckClosed() { 283 return 284 } 285 } 286 } 287 } 288 289 func (r *progressTracker) checkClosed(backendDead <-chan struct{}) bool { 290 select { 291 case <-backendDead: 292 r.advance() 293 return true 294 default: 295 return r.doCheckClosed() 296 } 297 } 298 299 func (r *progressTracker) doCheckClosed() bool { 300 resolvedTs := r.advance() 301 trackingCount := r.trackingCount() 302 if trackingCount == 0 { 303 return true 304 } 305 306 now := time.Now().Unix() 307 lastCheck := r.lastCheckClosed.Load() 308 for now > lastCheck+int64(warnDuration.Seconds()) { 309 if r.lastCheckClosed.CompareAndSwap(lastCheck, now) { 310 log.Warn("Close table doesn't return in time, may be stuck", 311 zap.Stringer("span", &r.span), 312 zap.Int("trackingCount", trackingCount), 313 zap.Any("lastMinResolvedTs", resolvedTs)) 314 break 315 } 316 lastCheck = r.lastCheckClosed.Load() 317 } 318 return false 319 }