github.com/pingcap/ticdc@v0.0.0-20220526033649-485a10ef2652/cdc/puller/sorter/heap_sorter.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 sorter 15 16 import ( 17 "container/heap" 18 "context" 19 "sync" 20 "sync/atomic" 21 "time" 22 23 "github.com/pingcap/errors" 24 "github.com/pingcap/failpoint" 25 "github.com/pingcap/log" 26 "github.com/pingcap/ticdc/cdc/model" 27 "github.com/pingcap/ticdc/pkg/config" 28 cerrors "github.com/pingcap/ticdc/pkg/errors" 29 "github.com/pingcap/ticdc/pkg/util" 30 "github.com/pingcap/ticdc/pkg/workerpool" 31 "go.uber.org/zap" 32 ) 33 34 const ( 35 flushRateLimitPerSecond = 10 36 sortHeapCapacity = 32 37 sortHeapInputChSize = 1024 38 ) 39 40 type flushTask struct { 41 taskID int 42 heapSorterID int 43 reader backEndReader 44 tsLowerBound uint64 45 maxResolvedTs uint64 46 finished chan error 47 dealloc func() error 48 dataSize int64 49 lastTs uint64 // for debugging TODO remove 50 canceller *asyncCanceller 51 52 isEmpty bool // read only field 53 54 deallocLock sync.RWMutex 55 isDeallocated bool // do not access directly 56 backend backEnd // do not access directly 57 } 58 59 func (t *flushTask) markDeallocated() { 60 t.deallocLock.Lock() 61 defer t.deallocLock.Unlock() 62 63 t.backend = nil 64 t.isDeallocated = true 65 } 66 67 func (t *flushTask) GetBackEnd() backEnd { 68 t.deallocLock.RLock() 69 defer t.deallocLock.RUnlock() 70 71 return t.backend 72 } 73 74 type heapSorter struct { 75 id int 76 taskCounter int 77 inputCh chan *model.PolymorphicEvent 78 outputCh chan *flushTask 79 heap sortHeap 80 canceller *asyncCanceller 81 82 poolHandle workerpool.EventHandle 83 internalState *heapSorterInternalState 84 } 85 86 func newHeapSorter(id int, out chan *flushTask) *heapSorter { 87 return &heapSorter{ 88 id: id, 89 inputCh: make(chan *model.PolymorphicEvent, sortHeapInputChSize), 90 outputCh: out, 91 heap: make(sortHeap, 0, sortHeapCapacity), 92 canceller: new(asyncCanceller), 93 } 94 } 95 96 // flush should only be called in the same goroutine where the heap is being written to. 97 func (h *heapSorter) flush(ctx context.Context, maxResolvedTs uint64) error { 98 captureAddr := util.CaptureAddrFromCtx(ctx) 99 changefeedID := util.ChangefeedIDFromCtx(ctx) 100 101 var ( 102 backEnd backEnd 103 lowerBound uint64 104 ) 105 106 if h.heap.Len() > 0 { 107 lowerBound = h.heap[0].entry.CRTs 108 } else { 109 return nil 110 } 111 112 sorterFlushCountHistogram.WithLabelValues(captureAddr, changefeedID).Observe(float64(h.heap.Len())) 113 114 // We check if the heap contains only one entry and that entry is a ResolvedEvent. 115 // As an optimization, when the condition is true, we clear the heap and send an empty flush. 116 // Sending an empty flush saves CPU and potentially IO. 117 // Since when a table is mostly idle or near-idle, most flushes would contain one ResolvedEvent alone, 118 // this optimization will greatly improve performance when (1) total number of table is large, 119 // and (2) most tables do not have many events. 120 if h.heap.Len() == 1 && h.heap[0].entry.RawKV.OpType == model.OpTypeResolved { 121 h.heap.Pop() 122 } 123 124 isEmptyFlush := h.heap.Len() == 0 125 var finishCh chan error 126 if !isEmptyFlush { 127 failpoint.Inject("InjectErrorBackEndAlloc", func() { 128 failpoint.Return(cerrors.ErrUnifiedSorterIOError.Wrap(errors.New("injected alloc error")).FastGenWithCause()) 129 }) 130 131 var err error 132 backEnd, err = pool.alloc(ctx) 133 if err != nil { 134 return errors.Trace(err) 135 } 136 137 finishCh = make(chan error, 1) 138 } 139 140 task := &flushTask{ 141 taskID: h.taskCounter, 142 heapSorterID: h.id, 143 backend: backEnd, 144 tsLowerBound: lowerBound, 145 maxResolvedTs: maxResolvedTs, 146 finished: finishCh, 147 canceller: h.canceller, 148 isEmpty: isEmptyFlush, 149 } 150 h.taskCounter++ 151 152 var oldHeap sortHeap 153 if !isEmptyFlush { 154 task.dealloc = func() error { 155 backEnd := task.GetBackEnd() 156 if backEnd != nil { 157 defer task.markDeallocated() 158 return pool.dealloc(backEnd) 159 } 160 return nil 161 } 162 oldHeap = h.heap 163 h.heap = make(sortHeap, 0, sortHeapCapacity) 164 } else { 165 task.dealloc = func() error { 166 task.markDeallocated() 167 return nil 168 } 169 } 170 failpoint.Inject("sorterDebug", func() { 171 tableID, tableName := util.TableIDFromCtx(ctx) 172 log.Debug("Unified Sorter new flushTask", 173 zap.Int64("table-id", tableID), 174 zap.String("table-name", tableName), 175 zap.Int("heap-id", task.heapSorterID), 176 zap.Uint64("resolvedTs", task.maxResolvedTs)) 177 }) 178 179 if !isEmptyFlush { 180 backEndFinal := backEnd 181 err := heapSorterIOPool.Go(ctx, func() { 182 failpoint.Inject("asyncFlushStartDelay", func() { 183 log.Debug("asyncFlushStartDelay") 184 }) 185 186 h.canceller.EnterAsyncOp() 187 defer h.canceller.FinishAsyncOp() 188 189 if h.canceller.IsCanceled() { 190 if backEndFinal != nil { 191 _ = task.dealloc() 192 } 193 task.finished <- cerrors.ErrAsyncIOCancelled.GenWithStackByArgs() 194 return 195 } 196 197 writer, err := backEnd.writer() 198 if err != nil { 199 if backEndFinal != nil { 200 _ = task.dealloc() 201 } 202 task.finished <- errors.Trace(err) 203 return 204 } 205 206 defer func() { 207 // handle errors (or aborts) gracefully to prevent resource leaking (especially FD's) 208 if writer != nil { 209 _ = writer.flushAndClose() 210 } 211 if backEndFinal != nil { 212 _ = task.dealloc() 213 } 214 close(task.finished) 215 }() 216 217 failpoint.Inject("InjectErrorBackEndWrite", func() { 218 task.finished <- cerrors.ErrUnifiedSorterIOError.Wrap(errors.New("injected write error")).FastGenWithCause() 219 failpoint.Return() 220 }) 221 222 counter := 0 223 for oldHeap.Len() > 0 { 224 failpoint.Inject("asyncFlushInProcessDelay", func() { 225 log.Debug("asyncFlushInProcessDelay") 226 }) 227 // no need to check for cancellation so frequently. 228 if counter%10000 == 0 && h.canceller.IsCanceled() { 229 task.finished <- cerrors.ErrAsyncIOCancelled.GenWithStackByArgs() 230 return 231 } 232 counter++ 233 234 event := heap.Pop(&oldHeap).(*sortItem).entry 235 err := writer.writeNext(event) 236 if err != nil { 237 task.finished <- errors.Trace(err) 238 return 239 } 240 } 241 242 dataSize := writer.dataSize() 243 atomic.StoreInt64(&task.dataSize, int64(dataSize)) 244 eventCount := writer.writtenCount() 245 246 writer1 := writer 247 writer = nil 248 err = writer1.flushAndClose() 249 if err != nil { 250 task.finished <- errors.Trace(err) 251 return 252 } 253 254 backEndFinal = nil 255 256 failpoint.Inject("sorterDebug", func() { 257 tableID, tableName := util.TableIDFromCtx(ctx) 258 log.Debug("Unified Sorter flushTask finished", 259 zap.Int("heap-id", task.heapSorterID), 260 zap.Int64("table-id", tableID), 261 zap.String("table-name", tableName), 262 zap.Uint64("resolvedTs", task.maxResolvedTs), 263 zap.Uint64("data-size", dataSize), 264 zap.Int("size", eventCount)) 265 }) 266 267 task.finished <- nil // DO NOT access `task` beyond this point in this function 268 }) 269 if err != nil { 270 close(task.finished) 271 return errors.Trace(err) 272 } 273 } 274 275 select { 276 case <-ctx.Done(): 277 return ctx.Err() 278 case h.outputCh <- task: 279 } 280 return nil 281 } 282 283 var ( 284 heapSorterPool workerpool.WorkerPool 285 heapSorterIOPool workerpool.AsyncPool 286 poolOnce sync.Once 287 ) 288 289 type heapSorterInternalState struct { 290 maxResolved uint64 291 heapSizeBytesEstimate int64 292 rateCounter int 293 sorterConfig *config.SorterConfig 294 timerMultiplier int 295 } 296 297 func (h *heapSorter) init(ctx context.Context, onError func(err error)) { 298 state := &heapSorterInternalState{ 299 sorterConfig: config.GetGlobalServerConfig().Sorter, 300 } 301 302 poolHandle := heapSorterPool.RegisterEvent(func(ctx context.Context, eventI interface{}) error { 303 event := eventI.(*model.PolymorphicEvent) 304 heap.Push(&h.heap, &sortItem{entry: event}) 305 isResolvedEvent := event.RawKV != nil && event.RawKV.OpType == model.OpTypeResolved 306 307 if isResolvedEvent { 308 if event.RawKV.CRTs < state.maxResolved { 309 log.Panic("ResolvedTs regression, bug?", zap.Uint64("event-resolvedTs", event.RawKV.CRTs), 310 zap.Uint64("max-resolvedTs", state.maxResolved)) 311 } 312 state.maxResolved = event.RawKV.CRTs 313 } 314 315 if event.RawKV.CRTs < state.maxResolved { 316 log.Panic("Bad input to sorter", zap.Uint64("cur-ts", event.RawKV.CRTs), zap.Uint64("maxResolved", state.maxResolved)) 317 } 318 319 // 5 * 8 is for the 5 fields in PolymorphicEvent 320 state.heapSizeBytesEstimate += event.RawKV.ApproximateSize() + 40 321 needFlush := state.heapSizeBytesEstimate >= int64(state.sorterConfig.ChunkSizeLimit) || 322 (isResolvedEvent && state.rateCounter < flushRateLimitPerSecond) 323 324 if needFlush { 325 state.rateCounter++ 326 err := h.flush(ctx, state.maxResolved) 327 if err != nil { 328 return errors.Trace(err) 329 } 330 state.heapSizeBytesEstimate = 0 331 } 332 333 return nil 334 }).SetTimer(ctx, 1*time.Second, func(ctx context.Context) error { 335 state.rateCounter = 0 336 state.timerMultiplier = (state.timerMultiplier + 1) % 5 337 if state.timerMultiplier == 0 && state.rateCounter < flushRateLimitPerSecond { 338 err := h.flush(ctx, state.maxResolved) 339 if err != nil { 340 return errors.Trace(err) 341 } 342 state.heapSizeBytesEstimate = 0 343 } 344 return nil 345 }).OnExit(onError) 346 347 h.poolHandle = poolHandle 348 h.internalState = state 349 } 350 351 // asyncCanceller is a shared object used to cancel async IO operations. 352 // We do not use `context.Context` because (1) selecting on `ctx.Done()` is expensive 353 // especially if the context is shared by many goroutines, and (2) due to the complexity 354 // of managing contexts through the workerpools, using a special shared object seems more reasonable 355 // and readable. 356 type asyncCanceller struct { 357 exitRWLock sync.RWMutex // held when an asynchronous flush is taking place 358 hasExited int32 // this flag should be accessed atomically 359 } 360 361 func (c *asyncCanceller) EnterAsyncOp() { 362 c.exitRWLock.RLock() 363 } 364 365 func (c *asyncCanceller) FinishAsyncOp() { 366 c.exitRWLock.RUnlock() 367 } 368 369 func (c *asyncCanceller) IsCanceled() bool { 370 return atomic.LoadInt32(&c.hasExited) == 1 371 } 372 373 func (c *asyncCanceller) Cancel() { 374 // Sets the flag 375 atomic.StoreInt32(&c.hasExited, 1) 376 377 // By taking the lock, we are making sure that all IO operations that started before setting the flag have finished, 378 // so that by the returning of this function, no more IO operations will finish successfully. 379 // Since IO operations that are NOT successful will clean up themselves, the goroutine in which this 380 // function was called is responsible for releasing files written by only those IO operations that complete BEFORE 381 // this function returns. 382 // In short, we are creating a linearization point here. 383 c.exitRWLock.Lock() 384 defer c.exitRWLock.Unlock() 385 } 386 387 func lazyInitWorkerPool() { 388 poolOnce.Do(func() { 389 sorterConfig := config.GetGlobalServerConfig().Sorter 390 heapSorterPool = workerpool.NewDefaultWorkerPool(sorterConfig.NumWorkerPoolGoroutine) 391 heapSorterIOPool = workerpool.NewDefaultAsyncPool(sorterConfig.NumWorkerPoolGoroutine * 2) 392 }) 393 }