github.com/pingcap/tiflow@v0.0.0-20240520035814-5bf52d54e205/cdc/redo/manager.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 redo 15 16 import ( 17 "context" 18 "sync" 19 "sync/atomic" 20 "time" 21 22 "github.com/pingcap/failpoint" 23 "github.com/pingcap/log" 24 "github.com/pingcap/tiflow/cdc/model" 25 "github.com/pingcap/tiflow/cdc/processor/tablepb" 26 "github.com/pingcap/tiflow/cdc/redo/common" 27 "github.com/pingcap/tiflow/cdc/redo/writer" 28 "github.com/pingcap/tiflow/cdc/redo/writer/factory" 29 "github.com/pingcap/tiflow/pkg/chann" 30 "github.com/pingcap/tiflow/pkg/config" 31 "github.com/pingcap/tiflow/pkg/errors" 32 "github.com/pingcap/tiflow/pkg/redo" 33 "github.com/pingcap/tiflow/pkg/spanz" 34 "github.com/pingcap/tiflow/pkg/util" 35 "github.com/prometheus/client_golang/prometheus" 36 "go.uber.org/zap" 37 ) 38 39 var ( 40 _ DDLManager = (*ddlManager)(nil) 41 _ DMLManager = (*dmlManager)(nil) 42 ) 43 44 type redoManager interface { 45 util.Runnable 46 47 // Enabled returns whether the manager is enabled 48 Enabled() bool 49 } 50 51 // DDLManager defines an interface that is used to manage ddl logs in owner. 52 type DDLManager interface { 53 redoManager 54 EmitDDLEvent(ctx context.Context, ddl *model.DDLEvent) error 55 UpdateResolvedTs(ctx context.Context, resolvedTs uint64) error 56 GetResolvedTs() model.Ts 57 } 58 59 // NewDisabledDDLManager creates a disabled ddl Manager. 60 func NewDisabledDDLManager() *ddlManager { 61 return &ddlManager{ 62 logManager: &logManager{enabled: false}, 63 } 64 } 65 66 // NewDDLManager creates a new ddl Manager. 67 func NewDDLManager( 68 changefeedID model.ChangeFeedID, 69 cfg *config.ConsistentConfig, ddlStartTs model.Ts, 70 ) *ddlManager { 71 m := newLogManager(changefeedID, cfg, redo.RedoDDLLogFileType) 72 span := spanz.TableIDToComparableSpan(0) 73 m.AddTable(span, ddlStartTs) 74 return &ddlManager{ 75 logManager: m, 76 // The current fakeSpan is meaningless, find a meaningful span in the future. 77 fakeSpan: span, 78 } 79 } 80 81 type ddlManager struct { 82 *logManager 83 fakeSpan tablepb.Span 84 } 85 86 func (m *ddlManager) EmitDDLEvent(ctx context.Context, ddl *model.DDLEvent) error { 87 return m.logManager.emitRedoEvents(ctx, m.fakeSpan, nil, ddl) 88 } 89 90 func (m *ddlManager) UpdateResolvedTs(ctx context.Context, resolvedTs uint64) error { 91 return m.logManager.UpdateResolvedTs(ctx, m.fakeSpan, resolvedTs) 92 } 93 94 func (m *ddlManager) GetResolvedTs() model.Ts { 95 return m.logManager.GetResolvedTs(m.fakeSpan) 96 } 97 98 // DMLManager defines an interface that is used to manage dml logs in processor. 99 type DMLManager interface { 100 redoManager 101 AddTable(span tablepb.Span, startTs uint64) 102 StartTable(span tablepb.Span, startTs uint64) 103 RemoveTable(span tablepb.Span) 104 UpdateResolvedTs(ctx context.Context, span tablepb.Span, resolvedTs uint64) error 105 GetResolvedTs(span tablepb.Span) model.Ts 106 EmitRowChangedEvents( 107 ctx context.Context, 108 span tablepb.Span, 109 releaseRowsMemory func(), 110 rows ...*model.RowChangedEvent, 111 ) error 112 } 113 114 // NewDMLManager creates a new dml Manager. 115 func NewDMLManager(changefeedID model.ChangeFeedID, 116 cfg *config.ConsistentConfig, 117 ) *dmlManager { 118 return &dmlManager{ 119 logManager: newLogManager(changefeedID, cfg, redo.RedoRowLogFileType), 120 } 121 } 122 123 // NewDisabledDMLManager creates a disabled dml Manager. 124 func NewDisabledDMLManager() *dmlManager { 125 return &dmlManager{ 126 logManager: &logManager{enabled: false}, 127 } 128 } 129 130 type dmlManager struct { 131 *logManager 132 } 133 134 // EmitRowChangedEvents emits row changed events to the redo log. 135 func (m *dmlManager) EmitRowChangedEvents( 136 ctx context.Context, 137 span tablepb.Span, 138 releaseRowsMemory func(), 139 rows ...*model.RowChangedEvent, 140 ) error { 141 var events []writer.RedoEvent 142 for _, row := range rows { 143 events = append(events, row) 144 } 145 return m.logManager.emitRedoEvents(ctx, span, releaseRowsMemory, events...) 146 } 147 148 type cacheEvents struct { 149 span tablepb.Span 150 events []writer.RedoEvent 151 resolvedTs model.Ts 152 isResolvedEvent bool 153 154 // releaseMemory is used to track memory usage of the events. 155 releaseMemory func() 156 } 157 158 type statefulRts struct { 159 flushed atomic.Uint64 160 unflushed atomic.Uint64 161 } 162 163 func newStatefulRts(ts model.Ts) (ret statefulRts) { 164 ret.unflushed.Store(ts) 165 ret.flushed.Store(ts) 166 return 167 } 168 169 func (s *statefulRts) getFlushed() model.Ts { 170 return s.flushed.Load() 171 } 172 173 func (s *statefulRts) getUnflushed() model.Ts { 174 return s.unflushed.Load() 175 } 176 177 func (s *statefulRts) checkAndSetUnflushed(unflushed model.Ts) (ok bool) { 178 return util.CompareAndIncrease(&s.unflushed, unflushed) 179 } 180 181 func (s *statefulRts) checkAndSetFlushed(flushed model.Ts) (ok bool) { 182 return util.CompareAndIncrease(&s.flushed, flushed) 183 } 184 185 // logManager manages redo log writer, buffers un-persistent redo logs, calculates 186 // redo log resolved ts. It implements DDLManager and DMLManager interface. 187 type logManager struct { 188 enabled bool 189 cfg *writer.LogWriterConfig 190 writer writer.RedoLogWriter 191 192 rwlock sync.RWMutex 193 // TODO: remove logBuffer and use writer directly after file logWriter is deprecated. 194 logBuffer *chann.DrainableChann[cacheEvents] 195 closed int32 196 197 // rtsMap stores flushed and unflushed resolved timestamps for all tables. 198 // it's just like map[span]*statefulRts. 199 // For a given statefulRts, unflushed is updated in routine bgUpdateLog, 200 // and flushed is updated in flushLog. 201 rtsMap spanz.SyncMap 202 203 flushing int64 204 lastFlushTime time.Time 205 releaseMemoryCbs []func() 206 207 metricWriteLogDuration prometheus.Observer 208 metricFlushLogDuration prometheus.Observer 209 metricTotalRowsCount prometheus.Counter 210 metricRedoWorkerBusyRatio prometheus.Counter 211 } 212 213 func newLogManager( 214 changefeedID model.ChangeFeedID, 215 cfg *config.ConsistentConfig, logType string, 216 ) *logManager { 217 // return a disabled Manager if no consistent config or normal consistent level 218 if cfg == nil || !redo.IsConsistentEnabled(cfg.Level) { 219 return &logManager{enabled: false} 220 } 221 222 return &logManager{ 223 enabled: true, 224 cfg: &writer.LogWriterConfig{ 225 ConsistentConfig: *cfg, 226 LogType: logType, 227 CaptureID: config.GetGlobalServerConfig().AdvertiseAddr, 228 ChangeFeedID: changefeedID, 229 MaxLogSizeInBytes: cfg.MaxLogSize * redo.Megabyte, 230 }, 231 logBuffer: chann.NewAutoDrainChann[cacheEvents](), 232 rtsMap: spanz.SyncMap{}, 233 metricWriteLogDuration: common.RedoWriteLogDurationHistogram. 234 WithLabelValues(changefeedID.Namespace, changefeedID.ID), 235 metricFlushLogDuration: common.RedoFlushLogDurationHistogram. 236 WithLabelValues(changefeedID.Namespace, changefeedID.ID), 237 metricTotalRowsCount: common.RedoTotalRowsCountGauge. 238 WithLabelValues(changefeedID.Namespace, changefeedID.ID), 239 metricRedoWorkerBusyRatio: common.RedoWorkerBusyRatio. 240 WithLabelValues(changefeedID.Namespace, changefeedID.ID), 241 } 242 } 243 244 // Run implements pkg/util.Runnable. 245 func (m *logManager) Run(ctx context.Context, _ ...chan<- error) error { 246 failpoint.Inject("ChangefeedNewRedoManagerError", func() { 247 failpoint.Return(errors.New("changefeed new redo manager injected error")) 248 }) 249 if !m.Enabled() { 250 return nil 251 } 252 253 defer m.close() 254 start := time.Now() 255 w, err := factory.NewRedoLogWriter(ctx, m.cfg) 256 if err != nil { 257 log.Error("redo: failed to create redo log writer", 258 zap.String("namespace", m.cfg.ChangeFeedID.Namespace), 259 zap.String("changefeed", m.cfg.ChangeFeedID.ID), 260 zap.Duration("duration", time.Since(start)), 261 zap.Error(err)) 262 return err 263 } 264 m.writer = w 265 return m.bgUpdateLog(ctx, m.getFlushDuration()) 266 } 267 268 func (m *logManager) getFlushDuration() time.Duration { 269 flushIntervalInMs := m.cfg.FlushIntervalInMs 270 defaultFlushIntervalInMs := redo.DefaultFlushIntervalInMs 271 if m.cfg.LogType == redo.RedoDDLLogFileType { 272 flushIntervalInMs = m.cfg.MetaFlushIntervalInMs 273 defaultFlushIntervalInMs = redo.DefaultMetaFlushIntervalInMs 274 } 275 if flushIntervalInMs < redo.MinFlushIntervalInMs { 276 log.Warn("redo flush interval is too small, use default value", 277 zap.String("namespace", m.cfg.ChangeFeedID.Namespace), 278 zap.String("changefeed", m.cfg.ChangeFeedID.ID), 279 zap.Int("default", defaultFlushIntervalInMs), 280 zap.String("logType", m.cfg.LogType), 281 zap.Int64("interval", flushIntervalInMs)) 282 flushIntervalInMs = int64(defaultFlushIntervalInMs) 283 } 284 return time.Duration(flushIntervalInMs) * time.Millisecond 285 } 286 287 // WaitForReady implements pkg/util.Runnable. 288 func (m *logManager) WaitForReady(_ context.Context) {} 289 290 // Close implements pkg/util.Runnable. 291 func (m *logManager) Close() {} 292 293 // Enabled returns whether this log manager is enabled 294 func (m *logManager) Enabled() bool { 295 return m.enabled 296 } 297 298 // emitRedoEvents sends row changed events to a log buffer, the log buffer 299 // will be consumed by a background goroutine, which converts row changed events 300 // to redo logs and sends to log writer. 301 func (m *logManager) emitRedoEvents( 302 ctx context.Context, 303 span tablepb.Span, 304 releaseRowsMemory func(), 305 events ...writer.RedoEvent, 306 ) error { 307 return m.withLock(func(m *logManager) error { 308 select { 309 case <-ctx.Done(): 310 return errors.Trace(ctx.Err()) 311 case m.logBuffer.In() <- cacheEvents{ 312 span: span, 313 events: events, 314 releaseMemory: releaseRowsMemory, 315 isResolvedEvent: false, 316 }: 317 } 318 return nil 319 }) 320 } 321 322 // StartTable starts a table, which means the table is ready to emit redo events. 323 // Note that this function should only be called once when adding a new table to processor. 324 func (m *logManager) StartTable(span tablepb.Span, resolvedTs uint64) { 325 // advance unflushed resolved ts 326 m.onResolvedTsMsg(span, resolvedTs) 327 328 // advance flushed resolved ts 329 if value, loaded := m.rtsMap.Load(span); loaded { 330 value.(*statefulRts).checkAndSetFlushed(resolvedTs) 331 } 332 } 333 334 // UpdateResolvedTs asynchronously updates resolved ts of a single table. 335 func (m *logManager) UpdateResolvedTs( 336 ctx context.Context, 337 span tablepb.Span, 338 resolvedTs uint64, 339 ) error { 340 return m.withLock(func(m *logManager) error { 341 select { 342 case <-ctx.Done(): 343 return errors.Trace(ctx.Err()) 344 case m.logBuffer.In() <- cacheEvents{ 345 span: span, 346 resolvedTs: resolvedTs, 347 isResolvedEvent: true, 348 }: 349 } 350 return nil 351 }) 352 } 353 354 // GetResolvedTs returns the resolved ts of a table 355 func (m *logManager) GetResolvedTs(span tablepb.Span) model.Ts { 356 if value, ok := m.rtsMap.Load(span); ok { 357 return value.(*statefulRts).getFlushed() 358 } 359 panic("GetResolvedTs is called on an invalid table") 360 } 361 362 // AddTable adds a new table in redo log manager 363 func (m *logManager) AddTable(span tablepb.Span, startTs uint64) { 364 rts := newStatefulRts(startTs) 365 _, loaded := m.rtsMap.LoadOrStore(span, &rts) 366 if loaded { 367 log.Warn("add duplicated table in redo log manager", 368 zap.String("namespace", m.cfg.ChangeFeedID.Namespace), 369 zap.String("changefeed", m.cfg.ChangeFeedID.ID), 370 zap.Stringer("span", &span)) 371 return 372 } 373 } 374 375 // RemoveTable removes a table from redo log manager 376 func (m *logManager) RemoveTable(span tablepb.Span) { 377 if _, ok := m.rtsMap.LoadAndDelete(span); !ok { 378 log.Warn("remove a table not maintained in redo log manager", 379 zap.String("namespace", m.cfg.ChangeFeedID.Namespace), 380 zap.String("changefeed", m.cfg.ChangeFeedID.ID), 381 zap.Stringer("span", &span)) 382 return 383 } 384 } 385 386 func (m *logManager) prepareForFlush() *spanz.HashMap[model.Ts] { 387 tableRtsMap := spanz.NewHashMap[model.Ts]() 388 m.rtsMap.Range(func(span tablepb.Span, value interface{}) bool { 389 rts := value.(*statefulRts) 390 unflushed := rts.getUnflushed() 391 flushed := rts.getFlushed() 392 if unflushed > flushed { 393 flushed = unflushed 394 } 395 tableRtsMap.ReplaceOrInsert(span, flushed) 396 return true 397 }) 398 return tableRtsMap 399 } 400 401 func (m *logManager) postFlush(tableRtsMap *spanz.HashMap[model.Ts]) { 402 tableRtsMap.Range(func(span tablepb.Span, flushed uint64) bool { 403 if value, loaded := m.rtsMap.Load(span); loaded { 404 changed := value.(*statefulRts).checkAndSetFlushed(flushed) 405 if !changed { 406 log.Debug("flush redo with regressed resolved ts", 407 zap.String("namespace", m.cfg.ChangeFeedID.Namespace), 408 zap.String("changefeed", m.cfg.ChangeFeedID.ID), 409 zap.Stringer("span", &span), 410 zap.Uint64("flushed", flushed), 411 zap.Uint64("current", value.(*statefulRts).getFlushed())) 412 } 413 } 414 return true 415 }) 416 } 417 418 func (m *logManager) flushLog( 419 ctx context.Context, handleErr func(err error), workTimeSlice *time.Duration, 420 ) { 421 start := time.Now() 422 defer func() { 423 *workTimeSlice += time.Since(start) 424 }() 425 if !atomic.CompareAndSwapInt64(&m.flushing, 0, 1) { 426 log.Debug("Fail to update flush flag, "+ 427 "the previous flush operation hasn't finished yet", 428 zap.String("namespace", m.cfg.ChangeFeedID.Namespace), 429 zap.String("changefeed", m.cfg.ChangeFeedID.ID)) 430 if time.Since(m.lastFlushTime) > redo.FlushWarnDuration { 431 log.Warn("flushLog blocking too long, the redo manager may be stuck", 432 zap.String("namespace", m.cfg.ChangeFeedID.Namespace), 433 zap.String("changefeed", m.cfg.ChangeFeedID.ID), 434 zap.Duration("duration", time.Since(m.lastFlushTime))) 435 } 436 return 437 } 438 439 m.lastFlushTime = time.Now() 440 releaseMemoryCbs := m.releaseMemoryCbs 441 m.releaseMemoryCbs = make([]func(), 0, 1024) 442 go func() { 443 defer atomic.StoreInt64(&m.flushing, 0) 444 445 tableRtsMap := m.prepareForFlush() 446 log.Debug("Flush redo log", 447 zap.String("namespace", m.cfg.ChangeFeedID.Namespace), 448 zap.String("changefeed", m.cfg.ChangeFeedID.ID), 449 zap.String("logType", m.cfg.LogType), 450 zap.Any("tableRtsMap", tableRtsMap)) 451 err := m.withLock(func(m *logManager) error { 452 return m.writer.FlushLog(ctx) 453 }) 454 for _, releaseMemory := range releaseMemoryCbs { 455 releaseMemory() 456 } 457 m.metricFlushLogDuration.Observe(time.Since(m.lastFlushTime).Seconds()) 458 if err != nil { 459 handleErr(err) 460 return 461 } 462 m.postFlush(tableRtsMap) 463 }() 464 } 465 466 func (m *logManager) handleEvent( 467 ctx context.Context, e cacheEvents, workTimeSlice *time.Duration, 468 ) error { 469 startHandleEvent := time.Now() 470 defer func() { 471 *workTimeSlice += time.Since(startHandleEvent) 472 }() 473 474 if e.isResolvedEvent { 475 m.onResolvedTsMsg(e.span, e.resolvedTs) 476 } else { 477 if e.releaseMemory != nil { 478 m.releaseMemoryCbs = append(m.releaseMemoryCbs, e.releaseMemory) 479 } 480 481 start := time.Now() 482 err := m.writer.WriteEvents(ctx, e.events...) 483 if err != nil { 484 return errors.Trace(err) 485 } 486 writeLogElapse := time.Since(start) 487 log.Debug("redo manager writes rows", 488 zap.String("namespace", m.cfg.ChangeFeedID.Namespace), 489 zap.String("changefeed", m.cfg.ChangeFeedID.ID), 490 zap.Int("rows", len(e.events)), 491 zap.Error(err), 492 zap.Duration("writeLogElapse", writeLogElapse)) 493 m.metricTotalRowsCount.Add(float64(len(e.events))) 494 m.metricWriteLogDuration.Observe(writeLogElapse.Seconds()) 495 } 496 return nil 497 } 498 499 func (m *logManager) onResolvedTsMsg(span tablepb.Span, resolvedTs model.Ts) { 500 // It's possible that the table is removed while redo log is still in writing. 501 if value, loaded := m.rtsMap.Load(span); loaded { 502 value.(*statefulRts).checkAndSetUnflushed(resolvedTs) 503 } 504 } 505 506 func (m *logManager) bgUpdateLog(ctx context.Context, flushDuration time.Duration) error { 507 m.releaseMemoryCbs = make([]func(), 0, 1024) 508 ticker := time.NewTicker(flushDuration) 509 defer ticker.Stop() 510 log.Info("redo manager bgUpdateLog is running", 511 zap.String("namespace", m.cfg.ChangeFeedID.Namespace), 512 zap.String("changefeed", m.cfg.ChangeFeedID.ID), 513 zap.Duration("flushIntervalInMs", flushDuration), 514 zap.Int64("maxLogSize", m.cfg.MaxLogSize), 515 zap.Int("encoderWorkerNum", m.cfg.EncodingWorkerNum), 516 zap.Int("flushWorkerNum", m.cfg.FlushWorkerNum)) 517 518 var err error 519 // logErrCh is used to retrieve errors from log flushing goroutines. 520 // if the channel is full, it's better to block subsequent flushing goroutines. 521 logErrCh := make(chan error, 1) 522 handleErr := func(err error) { logErrCh <- err } 523 524 overseerTicker := time.NewTicker(time.Second * 5) 525 defer overseerTicker.Stop() 526 var workTimeSlice time.Duration 527 startToWork := time.Now() 528 for { 529 select { 530 case <-ctx.Done(): 531 return ctx.Err() 532 case <-ticker.C: 533 m.flushLog(ctx, handleErr, &workTimeSlice) 534 case event, ok := <-m.logBuffer.Out(): 535 if !ok { 536 return nil // channel closed 537 } 538 err = m.handleEvent(ctx, event, &workTimeSlice) 539 case now := <-overseerTicker.C: 540 busyRatio := int(workTimeSlice.Seconds() / now.Sub(startToWork).Seconds() * 1000) 541 m.metricRedoWorkerBusyRatio.Add(float64(busyRatio)) 542 startToWork = now 543 workTimeSlice = 0 544 case err = <-logErrCh: 545 } 546 if err != nil { 547 log.Warn("redo manager writer meets write or flush fail", 548 zap.String("namespace", m.cfg.ChangeFeedID.Namespace), 549 zap.String("changefeed", m.cfg.ChangeFeedID.ID), 550 zap.Error(err)) 551 return err 552 } 553 } 554 } 555 556 func (m *logManager) withLock(action func(m *logManager) error) error { 557 m.rwlock.RLock() 558 defer m.rwlock.RUnlock() 559 if atomic.LoadInt32(&m.closed) != 0 { 560 return errors.ErrRedoWriterStopped.GenWithStack("redo manager is closed") 561 } 562 return action(m) 563 } 564 565 func (m *logManager) close() { 566 m.rwlock.Lock() 567 defer m.rwlock.Unlock() 568 atomic.StoreInt32(&m.closed, 1) 569 570 m.logBuffer.CloseAndDrain() 571 if m.writer != nil { 572 if err := m.writer.Close(); err != nil && errors.Cause(err) != context.Canceled { 573 log.Error("redo manager fails to close writer", 574 zap.String("namespace", m.cfg.ChangeFeedID.Namespace), 575 zap.String("changefeed", m.cfg.ChangeFeedID.ID), 576 zap.Error(err)) 577 } 578 } 579 log.Info("redo manager closed", 580 zap.String("namespace", m.cfg.ChangeFeedID.Namespace), 581 zap.String("changefeed", m.cfg.ChangeFeedID.ID)) 582 }