github.com/pingcap/ticdc@v0.0.0-20220526033649-485a10ef2652/cdc/puller/sorter/merger.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 "math" 20 "strings" 21 "sync" 22 "sync/atomic" 23 "time" 24 25 "github.com/pingcap/errors" 26 "github.com/pingcap/failpoint" 27 "github.com/pingcap/log" 28 "github.com/pingcap/ticdc/cdc/model" 29 cerrors "github.com/pingcap/ticdc/pkg/errors" 30 "github.com/pingcap/ticdc/pkg/notify" 31 "github.com/pingcap/ticdc/pkg/util" 32 "github.com/pingcap/tidb/store/tikv/oracle" 33 "go.uber.org/zap" 34 "golang.org/x/sync/errgroup" 35 ) 36 37 // TODO refactor this into a struct Merger. 38 func runMerger(ctx context.Context, numSorters int, in <-chan *flushTask, out chan *model.PolymorphicEvent, onExit func()) error { 39 captureAddr := util.CaptureAddrFromCtx(ctx) 40 changefeedID := util.ChangefeedIDFromCtx(ctx) 41 42 metricSorterEventCount := sorterEventCount.MustCurryWith(map[string]string{ 43 "capture": captureAddr, 44 "changefeed": changefeedID, 45 }) 46 metricSorterResolvedTsGauge := sorterResolvedTsGauge.WithLabelValues(captureAddr, changefeedID) 47 metricSorterMergerStartTsGauge := sorterMergerStartTsGauge.WithLabelValues(captureAddr, changefeedID) 48 metricSorterMergeCountHistogram := sorterMergeCountHistogram.WithLabelValues(captureAddr, changefeedID) 49 50 lastResolvedTs := make([]uint64, numSorters) 51 minResolvedTs := uint64(0) 52 var workingSet map[*flushTask]struct{} 53 pendingSet := &sync.Map{} 54 55 defer func() { 56 log.Debug("Unified Sorter: merger exiting, cleaning up resources") 57 // cancel pending async IO operations. 58 onExit() 59 cleanUpTask := func(task *flushTask) { 60 select { 61 case err := <-task.finished: 62 _ = printError(err) 63 default: 64 // The task has not finished, so we give up. 65 // It does not matter because: 66 // 1) if the async workerpool has exited, it means the CDC process is exiting, UnifiedSorterCleanUp will 67 // take care of the temp files, 68 // 2) if the async workerpool is not exiting, the unfinished tasks will eventually be executed, 69 // and by that time, since the `onExit` have canceled them, they will not do any IO and clean up themselves. 70 return 71 } 72 73 if task.reader != nil { 74 _ = printError(task.reader.resetAndClose()) 75 task.reader = nil 76 } 77 _ = printError(task.dealloc()) 78 } 79 80 LOOP: 81 for { 82 var task *flushTask 83 select { 84 case task = <-in: 85 default: 86 break LOOP 87 } 88 89 if task == nil { 90 log.Debug("Merger exiting, in-channel is exhausted") 91 break 92 } 93 94 cleanUpTask(task) 95 } 96 97 pendingSet.Range(func(task, _ interface{}) bool { 98 cleanUpTask(task.(*flushTask)) 99 return true 100 }) 101 for task := range workingSet { 102 cleanUpTask(task) 103 } 104 }() 105 106 lastOutputTs := uint64(0) 107 lastOutputResolvedTs := uint64(0) 108 var lastEvent *model.PolymorphicEvent 109 var lastTask *flushTask 110 111 sendResolvedEvent := func(ts uint64) error { 112 lastOutputResolvedTs = ts 113 if ts == 0 { 114 return nil 115 } 116 select { 117 case <-ctx.Done(): 118 return ctx.Err() 119 case out <- model.NewResolvedPolymorphicEvent(0, ts): 120 metricSorterEventCount.WithLabelValues("resolved").Inc() 121 metricSorterResolvedTsGauge.Set(float64(oracle.ExtractPhysical(ts))) 122 return nil 123 } 124 } 125 126 onMinResolvedTsUpdate := func(minResolvedTs /* note the shadowing */ uint64) error { 127 metricSorterMergerStartTsGauge.Set(float64(oracle.ExtractPhysical(minResolvedTs))) 128 workingSet = make(map[*flushTask]struct{}) 129 sortHeap := new(sortHeap) 130 131 // loopErr is used to return an error out of the closure taken by `pendingSet.Range`. 132 var loopErr error 133 // NOTE 1: We can block the closure passed to `pendingSet.Range` WITHOUT worrying about 134 // deadlocks because the closure is NOT called with any lock acquired in the implementation 135 // of Sync.Map. 136 // NOTE 2: It is safe to used `Range` to iterate through the pendingSet, in spite of NOT having 137 // a snapshot consistency because (1) pendingSet is updated first before minResolvedTs is updated, 138 // which guarantees that useful new flushTasks are not missed, and (2) by design, once minResolvedTs is updated, 139 // new flushTasks will satisfy `task.tsLowerBound > minResolvedTs`, and such flushTasks are ignored in 140 // the closure. 141 pendingSet.Range(func(iTask, iCache interface{}) bool { 142 task := iTask.(*flushTask) 143 var cache *model.PolymorphicEvent 144 if iCache != nil { 145 cache = iCache.(*model.PolymorphicEvent) 146 } 147 148 if task.tsLowerBound > minResolvedTs { 149 // the condition above implies that for any event in task.backend, CRTs > minResolvedTs. 150 return true 151 } 152 var event *model.PolymorphicEvent 153 if cache != nil { 154 event = cache 155 } else { 156 select { 157 case <-ctx.Done(): 158 loopErr = ctx.Err() 159 // terminates the loop 160 return false 161 case err := <-task.finished: 162 if err != nil { 163 loopErr = errors.Trace(err) 164 // terminates the loop 165 return false 166 } 167 } 168 169 if task.reader == nil { 170 var err error 171 task.reader, err = task.GetBackEnd().reader() 172 if err != nil { 173 loopErr = errors.Trace(err) 174 // terminates the loop 175 return false 176 } 177 } 178 179 var err error 180 event, err = task.reader.readNext() 181 if err != nil { 182 loopErr = errors.Trace(err) 183 // terminates the loop 184 return false 185 } 186 187 if event == nil { 188 log.Panic("Unexpected end of backEnd data, bug?", 189 zap.Uint64("minResolvedTs", task.maxResolvedTs)) 190 } 191 } 192 193 if event.CRTs > minResolvedTs { 194 pendingSet.Store(task, event) 195 // continues the loop 196 return true 197 } 198 199 pendingSet.Store(task, nil) 200 workingSet[task] = struct{}{} 201 202 heap.Push(sortHeap, &sortItem{ 203 entry: event, 204 data: task, 205 }) 206 return true 207 }) 208 if loopErr != nil { 209 return errors.Trace(loopErr) 210 } 211 212 resolvedTicker := time.NewTicker(1 * time.Second) 213 defer resolvedTicker.Stop() 214 215 retire := func(task *flushTask) error { 216 delete(workingSet, task) 217 cached, ok := pendingSet.Load(task) 218 if !ok { 219 log.Panic("task not found in pendingSet") 220 } 221 222 if cached != nil { 223 return nil 224 } 225 226 nextEvent, err := task.reader.readNext() 227 if err != nil { 228 _ = task.reader.resetAndClose() // prevents fd leak 229 task.reader = nil 230 return errors.Trace(err) 231 } 232 233 if nextEvent == nil { 234 pendingSet.Delete(task) 235 236 err := task.reader.resetAndClose() 237 if err != nil { 238 return errors.Trace(err) 239 } 240 task.reader = nil 241 242 err = task.dealloc() 243 if err != nil { 244 return errors.Trace(err) 245 } 246 } else { 247 pendingSet.Store(task, nextEvent) 248 if nextEvent.CRTs < minResolvedTs { 249 log.Panic("remaining event CRTs too small", 250 zap.Uint64("next-ts", nextEvent.CRTs), 251 zap.Uint64("minResolvedTs", minResolvedTs)) 252 } 253 } 254 return nil 255 } 256 257 failpoint.Inject("sorterDebug", func() { 258 if sortHeap.Len() > 0 { 259 tableID, tableName := util.TableIDFromCtx(ctx) 260 log.Debug("Unified Sorter: start merging", 261 zap.Int64("table-id", tableID), 262 zap.String("table-name", tableName), 263 zap.Uint64("minResolvedTs", minResolvedTs)) 264 } 265 }) 266 267 counter := 0 268 for sortHeap.Len() > 0 { 269 failpoint.Inject("sorterMergeDelay", func() {}) 270 271 item := heap.Pop(sortHeap).(*sortItem) 272 task := item.data.(*flushTask) 273 event := item.entry 274 275 if event.CRTs < task.lastTs { 276 log.Panic("unified sorter: ts regressed in one backEnd, bug?", zap.Uint64("cur-ts", event.CRTs), zap.Uint64("last-ts", task.lastTs)) 277 } 278 task.lastTs = event.CRTs 279 280 if event.RawKV != nil && event.RawKV.OpType != model.OpTypeResolved { 281 if event.CRTs < lastOutputTs { 282 for sortHeap.Len() > 0 { 283 item := heap.Pop(sortHeap).(*sortItem) 284 task := item.data.(*flushTask) 285 event := item.entry 286 log.Debug("dump", zap.Reflect("event", event), zap.Int("heap-id", task.heapSorterID)) 287 } 288 log.Panic("unified sorter: output ts regressed, bug?", 289 zap.Int("counter", counter), 290 zap.Uint64("minResolvedTs", minResolvedTs), 291 zap.Int("cur-heap-id", task.heapSorterID), 292 zap.Int("cur-task-id", task.taskID), 293 zap.Uint64("cur-task-resolved", task.maxResolvedTs), 294 zap.Reflect("cur-event", event), 295 zap.Uint64("cur-ts", event.CRTs), 296 zap.Int("last-heap-id", lastTask.heapSorterID), 297 zap.Int("last-task-id", lastTask.taskID), 298 zap.Uint64("last-task-resolved", task.maxResolvedTs), 299 zap.Reflect("last-event", lastEvent), 300 zap.Uint64("last-ts", lastOutputTs), 301 zap.Int("sort-heap-len", sortHeap.Len())) 302 } 303 304 if event.CRTs <= lastOutputResolvedTs { 305 log.Panic("unified sorter: output ts smaller than resolved ts, bug?", zap.Uint64("minResolvedTs", minResolvedTs), 306 zap.Uint64("lastOutputResolvedTs", lastOutputResolvedTs), zap.Uint64("event-crts", event.CRTs)) 307 } 308 lastOutputTs = event.CRTs 309 lastEvent = event 310 lastTask = task 311 select { 312 case <-ctx.Done(): 313 return ctx.Err() 314 case out <- event: 315 metricSorterEventCount.WithLabelValues("kv").Inc() 316 } 317 } 318 counter += 1 319 320 select { 321 case <-resolvedTicker.C: 322 err := sendResolvedEvent(event.CRTs - 1) 323 if err != nil { 324 return errors.Trace(err) 325 } 326 default: 327 } 328 329 event, err := task.reader.readNext() 330 if err != nil { 331 return errors.Trace(err) 332 } 333 334 if event == nil { 335 // EOF 336 delete(workingSet, task) 337 pendingSet.Delete(task) 338 339 err := task.reader.resetAndClose() 340 if err != nil { 341 return errors.Trace(err) 342 } 343 task.reader = nil 344 345 err = task.dealloc() 346 if err != nil { 347 return errors.Trace(err) 348 } 349 350 continue 351 } 352 353 if event.CRTs > minResolvedTs || (event.CRTs == minResolvedTs && event.RawKV.OpType == model.OpTypeResolved) { 354 // we have processed all events from this task that need to be processed in this merge 355 if event.CRTs > minResolvedTs || event.RawKV.OpType != model.OpTypeResolved { 356 pendingSet.Store(task, event) 357 } 358 err := retire(task) 359 if err != nil { 360 return errors.Trace(err) 361 } 362 continue 363 } 364 365 failpoint.Inject("sorterDebug", func() { 366 if counter%10 == 0 { 367 tableID, tableName := util.TableIDFromCtx(ctx) 368 log.Debug("Merging progress", 369 zap.Int64("table-id", tableID), 370 zap.String("table-name", tableName), 371 zap.Int("counter", counter)) 372 } 373 }) 374 375 heap.Push(sortHeap, &sortItem{ 376 entry: event, 377 data: task, 378 }) 379 } 380 381 if len(workingSet) != 0 { 382 log.Panic("unified sorter: merging ended prematurely, bug?", zap.Uint64("resolvedTs", minResolvedTs)) 383 } 384 385 failpoint.Inject("sorterDebug", func() { 386 if counter > 0 { 387 tableID, tableName := util.TableIDFromCtx(ctx) 388 log.Debug("Unified Sorter: merging ended", 389 zap.Int64("table-id", tableID), 390 zap.String("table-name", tableName), 391 zap.Uint64("resolvedTs", minResolvedTs), zap.Int("count", counter)) 392 } 393 }) 394 err := sendResolvedEvent(minResolvedTs) 395 if err != nil { 396 return errors.Trace(err) 397 } 398 399 if counter > 0 { 400 // ignore empty merges for better visualization of metrics 401 metricSorterMergeCountHistogram.Observe(float64(counter)) 402 } 403 404 return nil 405 } 406 407 resolvedTsNotifier := ¬ify.Notifier{} 408 defer resolvedTsNotifier.Close() 409 errg, ctx := errgroup.WithContext(ctx) 410 411 errg.Go(func() error { 412 for { 413 var task *flushTask 414 select { 415 case <-ctx.Done(): 416 return ctx.Err() 417 case task = <-in: 418 } 419 420 if task == nil { 421 tableID, tableName := util.TableIDFromCtx(ctx) 422 log.Debug("Merger input channel closed, exiting", 423 zap.Int64("table-id", tableID), 424 zap.String("table-name", tableName)) 425 return nil 426 } 427 428 if !task.isEmpty { 429 pendingSet.Store(task, nil) 430 } // otherwise it is an empty flush 431 432 if lastResolvedTs[task.heapSorterID] < task.maxResolvedTs { 433 lastResolvedTs[task.heapSorterID] = task.maxResolvedTs 434 } 435 436 minTemp := uint64(math.MaxUint64) 437 for _, ts := range lastResolvedTs { 438 if minTemp > ts { 439 minTemp = ts 440 } 441 } 442 443 if minTemp > minResolvedTs { 444 atomic.StoreUint64(&minResolvedTs, minTemp) 445 resolvedTsNotifier.Notify() 446 } 447 } 448 }) 449 450 errg.Go(func() error { 451 resolvedTsReceiver, err := resolvedTsNotifier.NewReceiver(time.Second * 1) 452 if err != nil { 453 if cerrors.ErrOperateOnClosedNotifier.Equal(err) { 454 // This won't happen unless `resolvedTsNotifier` has been closed, which is 455 // impossible at this point. 456 log.Panic("unexpected error", zap.Error(err)) 457 } 458 return errors.Trace(err) 459 } 460 461 defer resolvedTsReceiver.Stop() 462 463 var lastResolvedTs uint64 464 for { 465 select { 466 case <-ctx.Done(): 467 return ctx.Err() 468 case <-resolvedTsReceiver.C: 469 curResolvedTs := atomic.LoadUint64(&minResolvedTs) 470 if curResolvedTs > lastResolvedTs { 471 err := onMinResolvedTsUpdate(curResolvedTs) 472 if err != nil { 473 return errors.Trace(err) 474 } 475 } else if curResolvedTs < lastResolvedTs { 476 log.Panic("resolved-ts regressed in sorter", 477 zap.Uint64("cur-resolved-ts", curResolvedTs), 478 zap.Uint64("last-resolved-ts", lastResolvedTs)) 479 } 480 } 481 } 482 }) 483 484 return errg.Wait() 485 } 486 487 func mergerCleanUp(in <-chan *flushTask) { 488 for task := range in { 489 select { 490 case err := <-task.finished: 491 _ = printError(err) 492 default: 493 break 494 } 495 496 if task.reader != nil { 497 _ = printError(task.reader.resetAndClose()) 498 } 499 _ = printError(task.dealloc()) 500 } 501 } 502 503 // printError is a helper for tracing errors on function returns 504 func printError(err error) error { 505 if err != nil && errors.Cause(err) != context.Canceled && 506 errors.Cause(err) != context.DeadlineExceeded && 507 !strings.Contains(err.Error(), "context canceled") && 508 !strings.Contains(err.Error(), "context deadline exceeded") && 509 cerrors.ErrAsyncIOCancelled.NotEqual(errors.Cause(err)) { 510 511 log.Warn("Unified Sorter: Error detected", zap.Error(err), zap.Stack("stack")) 512 } 513 return err 514 }