From 7294c3712811c54204ff42588561539f78eeee77 Mon Sep 17 00:00:00 2001 From: Neil Shen Date: Wed, 4 Jan 2023 12:30:20 +0800 Subject: [PATCH] processor(ticdc): replace tableID with Span in sourcemanager and sinkmanager (#7978) ref pingcap/tiflow#7720 --- cdc/processor/pipeline/puller.go | 4 +- cdc/processor/processor.go | 74 +++---- cdc/processor/sinkmanager/manager.go | 150 +++++++------- cdc/processor/sinkmanager/manager_test.go | 93 ++++----- cdc/processor/sinkmanager/mem_quota.go | 38 ++-- cdc/processor/sinkmanager/mem_quota_test.go | 57 +++--- cdc/processor/sinkmanager/redo_cache.go | 24 ++- cdc/processor/sinkmanager/redo_cache_test.go | 6 +- cdc/processor/sinkmanager/redo_log_worker.go | 31 ++- .../sinkmanager/table_progress_heap.go | 4 +- .../sinkmanager/table_progress_heap_test.go | 13 +- .../sinkmanager/table_sink_worker.go | 42 ++-- .../sinkmanager/table_sink_worker_test.go | 185 +++++++++--------- .../sinkmanager/table_sink_wrapper.go | 21 +- .../sinkmanager/table_sink_wrapper_test.go | 20 +- cdc/processor/sinkmanager/tasks.go | 6 +- cdc/processor/sourcemanager/engine/engine.go | 17 +- .../engine/memory/event_sorter.go | 68 ++++--- .../engine/memory/event_sorter_test.go | 11 +- .../sourcemanager/engine/mock/engine_mock.go | 59 +++--- .../engine/pebble/event_sorter.go | 136 +++++++------ .../engine/pebble/event_sorter_test.go | 43 ++-- cdc/processor/sourcemanager/manager.go | 51 ++--- .../sourcemanager/puller/puller_wrapper.go | 18 +- cdc/sinkv2/eventsink/mq/worker.go | 4 +- cdc/sinkv2/eventsink/mq/worker_test.go | 2 +- pkg/config/debug.go | 7 +- pkg/spanz/sync_map.go | 64 ++++++ 28 files changed, 678 insertions(+), 570 deletions(-) create mode 100644 pkg/spanz/sync_map.go diff --git a/cdc/processor/pipeline/puller.go b/cdc/processor/pipeline/puller.go index d71a87b475f..8bb746a03ed 100644 --- a/cdc/processor/pipeline/puller.go +++ b/cdc/processor/pipeline/puller.go @@ -40,13 +40,13 @@ type pullerNode struct { } func newPullerNode( - tableID tablepb.Span, + span tablepb.Span, startTs model.Ts, tableName string, changefeed model.ChangeFeedID, ) *pullerNode { return &pullerNode{ - span: tableID, + span: span, startTs: startTs, tableName: tableName, changefeed: changefeed, diff --git a/cdc/processor/processor.go b/cdc/processor/processor.go index 85cafdc4049..1e4f1cdf074 100644 --- a/cdc/processor/processor.go +++ b/cdc/processor/processor.go @@ -149,7 +149,7 @@ func (p *processor) AddTableSpan( var alreadyExist bool var state tablepb.TableState if p.pullBasedSinking { - state, alreadyExist = p.sinkManager.GetTableState(span.TableID) + state, alreadyExist = p.sinkManager.GetTableState(span) } else { table, ok := p.tableSpans.Get(span) if ok { @@ -176,7 +176,7 @@ func (p *processor) AddTableSpan( // be stopped on original capture already, it's safe to start replicating data now. if !isPrepare { if p.pullBasedSinking { - if err := p.sinkManager.StartTable(span.TableID, startTs); err != nil { + if err := p.sinkManager.StartTable(span, startTs); err != nil { return false, errors.Trace(err) } } else { @@ -221,12 +221,12 @@ func (p *processor) AddTableSpan( if p.pullBasedSinking { p.sinkManager.AddTable( - span.TableID, startTs, p.changefeed.Info.TargetTs) + span, startTs, p.changefeed.Info.TargetTs) if p.redoManager.Enabled() { p.redoManager.AddTable(span, startTs) } p.sourceManager.AddTable( - ctx.(cdcContext.Context), span.TableID, p.getTableName(ctx, span.TableID), startTs) + ctx.(cdcContext.Context), span, p.getTableName(ctx, span.TableID), startTs) } else { table, err := p.createTablePipeline( ctx.(cdcContext.Context), span, &model.TableReplicaInfo{StartTs: startTs}) @@ -246,7 +246,7 @@ func (p *processor) RemoveTableSpan(span tablepb.Span) bool { } if p.pullBasedSinking { - _, exist := p.sinkManager.GetTableState(span.TableID) + _, exist := p.sinkManager.GetTableState(span) if !exist { log.Warn("Table which will be deleted is not found", zap.String("capture", p.captureInfo.ID), @@ -255,7 +255,7 @@ func (p *processor) RemoveTableSpan(span tablepb.Span) bool { zap.Stringer("span", &span)) return true } - p.sinkManager.AsyncStopTable(span.TableID) + p.sinkManager.AsyncStopTable(span) return true } table, ok := p.tableSpans.Get(span) @@ -297,9 +297,9 @@ func (p *processor) IsAddTableSpanFinished(span tablepb.Span, isPrepare bool) bo done := func() bool { var alreadyExist bool if p.pullBasedSinking { - state, alreadyExist = p.sinkManager.GetTableState(span.TableID) + state, alreadyExist = p.sinkManager.GetTableState(span) if alreadyExist { - stats := p.sinkManager.GetTableStats(span.TableID) + stats := p.sinkManager.GetTableStats(span) tableResolvedTs = stats.ResolvedTs tableCheckpointTs = stats.CheckpointTs } @@ -372,9 +372,9 @@ func (p *processor) IsRemoveTableSpanFinished(span tablepb.Span) (model.Ts, bool var state tablepb.TableState var tableCheckpointTs uint64 if p.pullBasedSinking { - state, alreadyExist = p.sinkManager.GetTableState(span.TableID) + state, alreadyExist = p.sinkManager.GetTableState(span) if alreadyExist { - stats := p.sinkManager.GetTableStats(span.TableID) + stats := p.sinkManager.GetTableStats(span) tableCheckpointTs = stats.CheckpointTs } } else { @@ -407,12 +407,12 @@ func (p *processor) IsRemoveTableSpanFinished(span tablepb.Span) (model.Ts, bool } if p.pullBasedSinking { - stats := p.sinkManager.GetTableStats(span.TableID) + stats := p.sinkManager.GetTableStats(span) if p.redoManager.Enabled() { p.redoManager.RemoveTable(span) } - p.sinkManager.RemoveTable(span.TableID) - p.sourceManager.RemoveTable(span.TableID) + p.sinkManager.RemoveTable(span) + p.sourceManager.RemoveTable(span) log.Info("table removed", zap.String("captureID", p.captureInfo.ID), zap.String("namespace", p.changefeedID.Namespace), @@ -441,7 +441,7 @@ func (p *processor) IsRemoveTableSpanFinished(span tablepb.Span) (model.Ts, bool // GetTableSpanCount implements TableExecutor interface. func (p *processor) GetTableSpanCount() int { if p.pullBasedSinking { - return len(p.sinkManager.GetAllCurrentTableIDs()) + return len(p.sinkManager.GetAllCurrentTableSpans()) } return p.tableSpans.Len() } @@ -454,7 +454,7 @@ func (p *processor) GetCheckpoint() (checkpointTs, resolvedTs model.Ts) { // GetTableSpanStatus implements TableExecutor interface func (p *processor) GetTableSpanStatus(span tablepb.Span, collectStat bool) tablepb.TableStatus { if p.pullBasedSinking { - state, exist := p.sinkManager.GetTableState(span.TableID) + state, exist := p.sinkManager.GetTableState(span) if !exist { return tablepb.TableStatus{ TableID: span.TableID, @@ -462,10 +462,10 @@ func (p *processor) GetTableSpanStatus(span tablepb.Span, collectStat bool) tabl State: tablepb.TableStateAbsent, } } - sinkStats := p.sinkManager.GetTableStats(span.TableID) + sinkStats := p.sinkManager.GetTableStats(span) stats := tablepb.Stats{} if collectStat { - stats = p.getStatsFromSourceManagerAndSinkManager(span.TableID, sinkStats) + stats = p.getStatsFromSourceManagerAndSinkManager(span, sinkStats) } return tablepb.TableStatus{ TableID: span.TableID, @@ -502,8 +502,10 @@ func (p *processor) GetTableSpanStatus(span tablepb.Span, collectStat bool) tabl } } -func (p *processor) getStatsFromSourceManagerAndSinkManager(tableID model.TableID, sinkStats sinkmanager.TableStats) tablepb.Stats { - pullerStats := p.sourceManager.GetTablePullerStats(tableID) +func (p *processor) getStatsFromSourceManagerAndSinkManager( + span tablepb.Span, sinkStats sinkmanager.TableStats, +) tablepb.Stats { + pullerStats := p.sourceManager.GetTablePullerStats(span) now, _ := p.upstream.PDClock.CurrentTime() stats := tablepb.Stats{ @@ -526,7 +528,7 @@ func (p *processor) getStatsFromSourceManagerAndSinkManager(tableID model.TableI }, } - sortStats := p.sourceManager.GetTableSorterStats(tableID) + sortStats := p.sourceManager.GetTableSorterStats(span) stats.StageCheckpoints["sorter-ingress"] = tablepb.Checkpoint{ CheckpointTs: sortStats.ReceivedMaxCommitTs, ResolvedTs: sortStats.ReceivedMaxResolvedTs, @@ -1068,21 +1070,21 @@ func (p *processor) handlePosition(currentTs int64) { minCheckpointTs := minResolvedTs minCheckpointTableID := int64(0) if p.pullBasedSinking { - tableIDs := p.sinkManager.GetAllCurrentTableIDs() - for _, tableID := range tableIDs { - stats := p.sinkManager.GetTableStats(tableID) + spans := p.sinkManager.GetAllCurrentTableSpans() + for _, span := range spans { + stats := p.sinkManager.GetTableStats(span) log.Debug("sink manager gets table stats", zap.String("namespace", p.changefeedID.Namespace), zap.String("changefeed", p.changefeedID.ID), - zap.Int64("tableID", tableID), + zap.Stringer("span", &span), zap.Any("stats", stats)) if stats.ResolvedTs < minResolvedTs { minResolvedTs = stats.ResolvedTs - minResolvedTableID = tableID + minResolvedTableID = span.TableID } if stats.CheckpointTs < minCheckpointTs { minCheckpointTs = stats.CheckpointTs - minCheckpointTableID = tableID + minCheckpointTableID = span.TableID } } } else { @@ -1225,8 +1227,8 @@ func (p *processor) removeTable(table tablepb.TablePipeline, span tablepb.Span) p.redoManager.RemoveTable(span) } if p.pullBasedSinking { - p.sinkManager.RemoveTable(span.TableID) - p.sourceManager.RemoveTable(span.TableID) + p.sinkManager.RemoveTable(span) + p.sourceManager.RemoveTable(span) } else { table.Cancel() table.Wait() @@ -1264,8 +1266,8 @@ func (p *processor) doGCSchemaStorage() { func (p *processor) refreshMetrics() { if p.pullBasedSinking { - tables := p.sinkManager.GetAllCurrentTableIDs() - p.metricSyncTableNumGauge.Set(float64(len(tables))) + tableSpans := p.sinkManager.GetAllCurrentTableSpans() + p.metricSyncTableNumGauge.Set(float64(len(tableSpans))) sortEngineReceivedEvents := p.sourceManager.ReceivedEvents() tableSinksReceivedEvents := p.sinkManager.ReceivedEvents() p.metricRemainKVEventGauge.Set(float64(sortEngineReceivedEvents - tableSinksReceivedEvents)) @@ -1443,13 +1445,13 @@ func (p *processor) cleanupMetrics() { func (p *processor) WriteDebugInfo(w io.Writer) error { fmt.Fprintf(w, "%+v\n", *p.changefeed) if p.pullBasedSinking { - tables := p.sinkManager.GetAllCurrentTableIDs() - for _, tableID := range tables { - state, _ := p.sinkManager.GetTableState(tableID) - stats := p.sinkManager.GetTableStats(tableID) + spans := p.sinkManager.GetAllCurrentTableSpans() + for _, span := range spans { + state, _ := p.sinkManager.GetTableState(span) + stats := p.sinkManager.GetTableStats(span) // TODO: add table name. - fmt.Fprintf(w, "tableID: %d, resolvedTs: %d, checkpointTs: %d, state: %s\n", - tableID, stats.ResolvedTs, stats.CheckpointTs, state) + fmt.Fprintf(w, "span: %s, resolvedTs: %d, checkpointTs: %d, state: %s\n", + &span, stats.ResolvedTs, stats.CheckpointTs, state) } } else { p.tableSpans.Ascend(func(span tablepb.Span, tablePipeline tablepb.TablePipeline) bool { diff --git a/cdc/processor/sinkmanager/manager.go b/cdc/processor/sinkmanager/manager.go index 15607d73f3c..14e2d5997c7 100644 --- a/cdc/processor/sinkmanager/manager.go +++ b/cdc/processor/sinkmanager/manager.go @@ -83,7 +83,7 @@ type SinkManager struct { // sinkFactory used to create table sink. sinkFactory *factory.SinkFactory // tableSinks is a map from tableID to tableSink. - tableSinks sync.Map + tableSinks spanz.SyncMap // lastBarrierTs is the last barrier ts. lastBarrierTs atomic.Uint64 @@ -271,15 +271,14 @@ func (m *SinkManager) backgroundGC() { zap.String("changefeed", m.changefeedID.ID)) return case <-ticker.C: - tableSinks := make(map[model.TableID]*tableSinkWrapper) - m.tableSinks.Range(func(key, value any) bool { - tableID := key.(model.TableID) + tableSinks := make(map[spanz.HashableSpan]*tableSinkWrapper) + m.tableSinks.Range(func(key tablepb.Span, value any) bool { wrapper := value.(*tableSinkWrapper) - tableSinks[tableID] = wrapper + tableSinks[spanz.ToHashableSpan(key)] = wrapper return true }) - for tableID, sink := range tableSinks { + for hspan, sink := range tableSinks { if time.Since(sink.lastCleanTime) < cleanTableInterval { continue } @@ -294,11 +293,12 @@ func (m *SinkManager) backgroundGC() { continue } - if err := m.sourceManager.CleanByTable(tableID, cleanPos); err != nil { + span := hspan.ToSpan() + if err := m.sourceManager.CleanByTable(span, cleanPos); err != nil { log.Error("Failed to clean table in sort engine", zap.String("namespace", m.changefeedID.Namespace), zap.String("changefeed", m.changefeedID.ID), - zap.Int64("tableID", tableID), + zap.Stringer("span", &span), zap.Error(err)) select { case m.errChan <- err: @@ -308,7 +308,7 @@ func (m *SinkManager) backgroundGC() { log.Debug("table stale data has been cleaned", zap.String("namespace", m.changefeedID.Namespace), zap.String("changefeed", m.changefeedID.ID), - zap.Int64("tableID", tableID), + zap.Stringer("span", &span), zap.Any("upperBound", cleanPos)) } sink.lastCleanTime = time.Now() @@ -345,14 +345,14 @@ func (m *SinkManager) generateSinkTasks() error { // Collect some table progresses. for len(tables) < sinkWorkerNum && m.sinkProgressHeap.len() > 0 { slowestTableProgress := m.sinkProgressHeap.pop() - tableID := slowestTableProgress.tableID + span := slowestTableProgress.span - value, ok := m.tableSinks.Load(tableID) + value, ok := m.tableSinks.Load(span) if !ok { log.Info("Table sink not found, probably already removed", zap.String("namespace", m.changefeedID.Namespace), zap.String("changefeed", m.changefeedID.ID), - zap.Int64("tableID", tableID)) + zap.Stringer("span", &span)) // Maybe the table sink is removed by the processor.(Scheduled the table to other nodes.) // So we do **not** need add it back to the heap. continue @@ -371,7 +371,7 @@ func (m *SinkManager) generateSinkTasks() error { log.Info("Table sink is not replicating, skip it", zap.String("namespace", m.changefeedID.Namespace), zap.String("changefeed", m.changefeedID.ID), - zap.Int64("tableID", tableID), + zap.Stringer("span", &span), zap.String("tableState", tableState.String())) continue } @@ -401,17 +401,17 @@ func (m *SinkManager) generateSinkTasks() error { log.Debug("MemoryQuotaTracing: try acquire memory for table sink task", zap.String("namespace", m.changefeedID.Namespace), zap.String("changefeed", m.changefeedID.ID), - zap.Int64("tableID", tableSink.tableID), + zap.Stringer("span", &tableSink.span), zap.Uint64("memory", requestMemSize)) t := &sinkTask{ - tableID: tableSink.tableID, + span: tableSink.span, lowerBound: lowerBound, getUpperBound: getUpperBound, tableSink: tableSink, callback: func(lastWrittenPos engine.Position) { p := &progress{ - tableID: tableSink.tableID, + span: tableSink.span, nextLowerBoundPos: lastWrittenPos.Next(), version: slowestTableProgress.version, } @@ -432,7 +432,7 @@ func (m *SinkManager) generateSinkTasks() error { log.Debug("Generate sink task", zap.String("namespace", m.changefeedID.Namespace), zap.String("changefeed", m.changefeedID.ID), - zap.Int64("tableID", tableSink.tableID), + zap.Stringer("span", &tableSink.span), zap.Any("lowerBound", lowerBound), zap.Any("currentUpperBound", upperBound)) default: @@ -440,7 +440,7 @@ func (m *SinkManager) generateSinkTasks() error { log.Debug("MemoryQuotaTracing: refund memory for table sink task", zap.String("namespace", m.changefeedID.Namespace), zap.String("changefeed", m.changefeedID.ID), - zap.Int64("tableID", tableSink.tableID), + zap.Stringer("span", &tableSink.span), zap.Uint64("memory", requestMemSize)) break LOOP } @@ -483,14 +483,14 @@ func (m *SinkManager) generateRedoTasks() error { for len(tables) < redoWorkerNum && m.redoProgressHeap.len() > 0 { slowestTableProgress := m.redoProgressHeap.pop() - tableID := slowestTableProgress.tableID + span := slowestTableProgress.span - value, ok := m.tableSinks.Load(tableID) + value, ok := m.tableSinks.Load(span) if !ok { log.Info("Table sink not found, probably already removed", zap.String("namespace", m.changefeedID.Namespace), zap.String("changefeed", m.changefeedID.ID), - zap.Int64("tableID", tableID)) + zap.Stringer("span", &span)) // Maybe the table sink is removed by the processor.(Scheduled the table to other nodes.) // So we do **not** need add it back to the heap. continue @@ -509,7 +509,7 @@ func (m *SinkManager) generateRedoTasks() error { log.Info("Table sink is not replicating, skip it", zap.String("namespace", m.changefeedID.Namespace), zap.String("changefeed", m.changefeedID.ID), - zap.Int64("tableID", tableID), + zap.Stringer("span", &span), zap.String("tableState", tableState.String())) continue } @@ -539,17 +539,17 @@ func (m *SinkManager) generateRedoTasks() error { log.Debug("MemoryQuotaTracing: try acquire memory for redo log task", zap.String("namespace", m.changefeedID.Namespace), zap.String("changefeed", m.changefeedID.ID), - zap.Int64("tableID", tableSink.tableID), + zap.Stringer("span", &tableSink.span), zap.Uint64("memory", requestMemSize)) t := &redoTask{ - tableID: tableSink.tableID, + span: tableSink.span, lowerBound: lowerBound, getUpperBound: getUpperBound, tableSink: tableSink, callback: func(lastWrittenPos engine.Position) { p := &progress{ - tableID: tableSink.tableID, + span: tableSink.span, nextLowerBoundPos: lastWrittenPos.Next(), version: slowestTableProgress.version, } @@ -570,7 +570,7 @@ func (m *SinkManager) generateRedoTasks() error { log.Debug("Generate redo task", zap.String("namespace", m.changefeedID.Namespace), zap.String("changefeed", m.changefeedID.ID), - zap.Int64("tableID", tableSink.tableID), + zap.Stringer("span", &tableSink.span), zap.Any("lowerBound", lowerBound), zap.Any("currentUpperBound", upperBound)) default: @@ -578,7 +578,7 @@ func (m *SinkManager) generateRedoTasks() error { log.Debug("MemoryQuotaTracing: refund memory for redo log task", zap.String("namespace", m.changefeedID.Namespace), zap.String("changefeed", m.changefeedID.ID), - zap.Int64("tableID", tableSink.tableID), + zap.Stringer("span", &tableSink.span), zap.Uint64("memory", requestMemSize)) break LOOP } @@ -610,14 +610,14 @@ func (m *SinkManager) generateRedoTasks() error { // UpdateReceivedSorterResolvedTs updates the received sorter resolved ts for the table. // NOTE: it's still possible to be called during m.Close is in calling, so Close should // take care of this. -func (m *SinkManager) UpdateReceivedSorterResolvedTs(tableID model.TableID, ts model.Ts) { - tableSink, ok := m.tableSinks.Load(tableID) +func (m *SinkManager) UpdateReceivedSorterResolvedTs(span tablepb.Span, ts model.Ts) { + tableSink, ok := m.tableSinks.Load(span) if !ok { // It's possible that the table is in removing. log.Debug("Table sink not found when updating resolved ts", zap.String("namespace", m.changefeedID.Namespace), zap.String("changefeed", m.changefeedID.ID), - zap.Int64("tableID", tableID)) + zap.Stringer("span", &span)) return } tableSink.(*tableSinkWrapper).updateReceivedSorterResolvedTs(ts) @@ -636,47 +636,46 @@ func (m *SinkManager) UpdateBarrierTs(ts model.Ts) { } // AddTable adds a table(TableSink) to the sink manager. -func (m *SinkManager) AddTable(tableID model.TableID, startTs model.Ts, targetTs model.Ts) { +func (m *SinkManager) AddTable(span tablepb.Span, startTs model.Ts, targetTs model.Ts) { sinkWrapper := newTableSinkWrapper( m.changefeedID, - tableID, - m.sinkFactory.CreateTableSink( - m.changefeedID, spanz.TableIDToComparableSpan(tableID), m.metricsTableSinkTotalRows), + span, + m.sinkFactory.CreateTableSink(m.changefeedID, span, m.metricsTableSinkTotalRows), tablepb.TableStatePreparing, startTs, targetTs, ) - _, loaded := m.tableSinks.LoadOrStore(tableID, sinkWrapper) + _, loaded := m.tableSinks.LoadOrStore(span, sinkWrapper) if loaded { log.Panic("Add an exists table sink", zap.String("namespace", m.changefeedID.Namespace), zap.String("changefeed", m.changefeedID.ID), - zap.Int64("tableID", tableID)) + zap.Stringer("span", &span)) return } - m.memQuota.addTable(tableID) + m.memQuota.addTable(span) log.Info("Add table sink", zap.String("namespace", m.changefeedID.Namespace), zap.String("changefeed", m.changefeedID.ID), - zap.Int64("tableID", tableID), + zap.Stringer("span", &span), zap.Uint64("startTs", startTs), zap.Uint64("version", sinkWrapper.version)) } // StartTable sets the table(TableSink) state to replicating. -func (m *SinkManager) StartTable(tableID model.TableID, startTs model.Ts) error { +func (m *SinkManager) StartTable(span tablepb.Span, startTs model.Ts) error { log.Info("Start table sink", zap.String("namespace", m.changefeedID.Namespace), zap.String("changefeed", m.changefeedID.ID), - zap.Int64("tableID", tableID), + zap.Stringer("span", &span), zap.Uint64("startTs", startTs), ) - tableSink, ok := m.tableSinks.Load(tableID) + tableSink, ok := m.tableSinks.Load(span) if !ok { log.Panic("Table sink not found when starting table stats", zap.String("namespace", m.changefeedID.Namespace), zap.String("changefeed", m.changefeedID.ID), - zap.Int64("tableID", tableID)) + zap.Stringer("span", &span)) } backoffBaseDelayInMs := int64(100) totalRetryDuration := 10 * time.Second @@ -690,7 +689,7 @@ func (m *SinkManager) StartTable(tableID model.TableID, startTs model.Ts) error log.Debug("Set replicate ts", zap.String("namespace", m.changefeedID.Namespace), zap.String("changefeed", m.changefeedID.ID), - zap.Int64("tableID", tableID), + zap.Stringer("span", &span), zap.Uint64("replicateTs", replicateTs), ) return nil @@ -702,13 +701,13 @@ func (m *SinkManager) StartTable(tableID model.TableID, startTs model.Ts) error } tableSink.(*tableSinkWrapper).start(startTs, replicateTs) m.sinkProgressHeap.push(&progress{ - tableID: tableID, + span: span, nextLowerBoundPos: engine.Position{StartTs: 0, CommitTs: startTs + 1}, version: tableSink.(*tableSinkWrapper).version, }) if m.redoManager != nil { m.redoProgressHeap.push(&progress{ - tableID: tableID, + span: span, nextLowerBoundPos: engine.Position{StartTs: 0, CommitTs: startTs + 1}, version: tableSink.(*tableSinkWrapper).version, }) @@ -717,104 +716,103 @@ func (m *SinkManager) StartTable(tableID model.TableID, startTs model.Ts) error } // AsyncStopTable sets the table(TableSink) state to stopped. -func (m *SinkManager) AsyncStopTable(tableID model.TableID) { +func (m *SinkManager) AsyncStopTable(span tablepb.Span) { m.wg.Add(1) go func() { defer m.wg.Done() log.Info("Async stop table sink", zap.String("namespace", m.changefeedID.Namespace), zap.String("changefeed", m.changefeedID.ID), - zap.Int64("tableID", tableID), + zap.Stringer("span", &span), ) - tableSink, ok := m.tableSinks.Load(tableID) + tableSink, ok := m.tableSinks.Load(span) if !ok { log.Panic("Table sink not found when removing table", zap.String("namespace", m.changefeedID.Namespace), zap.String("changefeed", m.changefeedID.ID), - zap.Int64("tableID", tableID)) + zap.Stringer("span", &span)) } tableSink.(*tableSinkWrapper).close(m.ctx) - cleanedBytes := m.memQuota.clean(tableID) + cleanedBytes := m.memQuota.clean(span) log.Debug("MemoryQuotaTracing: Clean up memory quota for table sink task when removing table", zap.String("namespace", m.changefeedID.Namespace), zap.String("changefeed", m.changefeedID.ID), - zap.Int64("tableID", tableID), + zap.Stringer("span", &span), zap.Uint64("memory", cleanedBytes), ) log.Info("Table sink closed asynchronously", zap.String("namespace", m.changefeedID.Namespace), zap.String("changefeed", m.changefeedID.ID), - zap.Int64("tableID", tableID), + zap.Stringer("span", &span), ) }() } // RemoveTable removes a table(TableSink) from the sink manager. -func (m *SinkManager) RemoveTable(tableID model.TableID) { +func (m *SinkManager) RemoveTable(span tablepb.Span) { // NOTICE: It is safe to only remove the table sink from the map. // Because if we found the table sink is closed, we will not add it back to the heap. // Also, no need to GC the SortEngine. Because the SortEngine also removes this table. - value, exists := m.tableSinks.LoadAndDelete(tableID) + value, exists := m.tableSinks.LoadAndDelete(span) if !exists { log.Panic("Remove an unexist table sink", zap.String("namespace", m.changefeedID.Namespace), zap.String("changefeed", m.changefeedID.ID), - zap.Int64("tableID", tableID)) + zap.Stringer("span", &span)) } sink := value.(*tableSinkWrapper) log.Info("Remove table sink successfully", zap.String("namespace", m.changefeedID.Namespace), zap.String("changefeed", m.changefeedID.ID), - zap.Int64("tableID", tableID), + zap.Stringer("span", &span), zap.Uint64("checkpointTs", sink.getCheckpointTs().Ts)) if m.eventCache != nil { - m.eventCache.removeTable(tableID) + m.eventCache.removeTable(span) } } -// GetAllCurrentTableIDs returns all the table IDs in the sink manager. -func (m *SinkManager) GetAllCurrentTableIDs() []model.TableID { - var tableIDs []model.TableID - m.tableSinks.Range(func(key, value interface{}) bool { - tableIDs = append(tableIDs, key.(model.TableID)) +// GetAllCurrentTableSpans returns all spans in the sink manager. +func (m *SinkManager) GetAllCurrentTableSpans() []tablepb.Span { + var spans []tablepb.Span + m.tableSinks.Range(func(key tablepb.Span, value interface{}) bool { + spans = append(spans, key) return true }) - return tableIDs + return spans } // GetTableState returns the table(TableSink) state. -func (m *SinkManager) GetTableState(tableID model.TableID) (tablepb.TableState, bool) { - tableSink, ok := m.tableSinks.Load(tableID) +func (m *SinkManager) GetTableState(span tablepb.Span) (tablepb.TableState, bool) { + tableSink, ok := m.tableSinks.Load(span) if !ok { log.Debug("Table sink not found when getting table state", zap.String("namespace", m.changefeedID.Namespace), zap.String("changefeed", m.changefeedID.ID), - zap.Int64("tableID", tableID)) + zap.Stringer("span", &span)) return tablepb.TableStateAbsent, false } return tableSink.(*tableSinkWrapper).getState(), true } // GetTableStats returns the state of the table. -func (m *SinkManager) GetTableStats(tableID model.TableID) TableStats { - value, ok := m.tableSinks.Load(tableID) +func (m *SinkManager) GetTableStats(span tablepb.Span) TableStats { + value, ok := m.tableSinks.Load(span) if !ok { log.Panic("Table sink not found when getting table stats", zap.String("namespace", m.changefeedID.Namespace), zap.String("changefeed", m.changefeedID.ID), - zap.Int64("tableID", tableID)) + zap.Stringer("span", &span)) } tableSink := value.(*tableSinkWrapper) checkpointTs := tableSink.getCheckpointTs() - m.memQuota.release(tableID, checkpointTs) - + m.memQuota.release(span, checkpointTs) var resolvedTs model.Ts // If redo log is enabled, we have to use redo log's resolved ts to calculate processor's min resolved ts. if m.redoManager != nil { - resolvedTs = m.redoManager.GetResolvedTs(spanz.TableIDToComparableSpan(tableID)) + resolvedTs = m.redoManager.GetResolvedTs(span) } else { - resolvedTs = tableSink.getReceivedSorterResolvedTs() + resolvedTs = m.sourceManager.GetTableResolvedTs(span) } return TableStats{ @@ -829,7 +827,7 @@ func (m *SinkManager) GetTableStats(tableID model.TableID) TableStats { // ReceivedEvents returns the number of events received by all table sinks. func (m *SinkManager) ReceivedEvents() int64 { totalReceivedEvents := int64(0) - m.tableSinks.Range(func(_, value interface{}) bool { + m.tableSinks.Range(func(_ tablepb.Span, value interface{}) bool { totalReceivedEvents += value.(*tableSinkWrapper).getReceivedEventCount() return true }) @@ -851,11 +849,11 @@ func (m *SinkManager) Close() error { if err != nil { return errors.Trace(err) } - m.tableSinks.Range(func(key, value interface{}) bool { + m.tableSinks.Range(func(_ tablepb.Span, value interface{}) bool { sink := value.(*tableSinkWrapper) sink.close(m.ctx) if m.eventCache != nil { - m.eventCache.removeTable(sink.tableID) + m.eventCache.removeTable(sink.span) } return true }) diff --git a/cdc/processor/sinkmanager/manager_test.go b/cdc/processor/sinkmanager/manager_test.go index 597f2f30536..40a43c8fb9c 100644 --- a/cdc/processor/sinkmanager/manager_test.go +++ b/cdc/processor/sinkmanager/manager_test.go @@ -26,6 +26,7 @@ import ( "github.com/pingcap/tiflow/cdc/processor/sourcemanager/engine/memory" "github.com/pingcap/tiflow/cdc/processor/tablepb" "github.com/pingcap/tiflow/pkg/config" + "github.com/pingcap/tiflow/pkg/spanz" "github.com/pingcap/tiflow/pkg/upstream" "github.com/prometheus/client_golang/prometheus" "github.com/stretchr/testify/require" @@ -77,9 +78,9 @@ func getChangefeedInfo() *model.ChangeFeedInfo { func addTableAndAddEventsToSortEngine( t *testing.T, engine engine.SortEngine, - tableID model.TableID, + span tablepb.Span, ) { - engine.AddTable(tableID) + engine.AddTable(span) events := []*model.PolymorphicEvent{ { StartTs: 1, @@ -89,7 +90,7 @@ func addTableAndAddEventsToSortEngine( StartTs: 1, CRTs: 1, }, - Row: genRowChangedEvent(1, 1, tableID), + Row: genRowChangedEvent(1, 1, span), }, { StartTs: 1, @@ -99,7 +100,7 @@ func addTableAndAddEventsToSortEngine( StartTs: 1, CRTs: 2, }, - Row: genRowChangedEvent(1, 2, tableID), + Row: genRowChangedEvent(1, 2, span), }, { StartTs: 1, @@ -109,7 +110,7 @@ func addTableAndAddEventsToSortEngine( StartTs: 1, CRTs: 3, }, - Row: genRowChangedEvent(1, 3, tableID), + Row: genRowChangedEvent(1, 3, span), }, { StartTs: 2, @@ -119,7 +120,7 @@ func addTableAndAddEventsToSortEngine( StartTs: 2, CRTs: 4, }, - Row: genRowChangedEvent(2, 4, tableID), + Row: genRowChangedEvent(2, 4, span), }, { CRTs: 4, @@ -130,7 +131,7 @@ func addTableAndAddEventsToSortEngine( }, } for _, event := range events { - err := engine.Add(tableID, event) + err := engine.Add(span, event) require.NoError(t, err) } } @@ -147,18 +148,18 @@ func TestAddTable(t *testing.T) { err := manager.Close() require.NoError(t, err) }() - tableID := model.TableID(1) - manager.AddTable(tableID, 1, 100) - tableSink, ok := manager.tableSinks.Load(tableID) + span := spanz.TableIDToComparableSpan(1) + manager.AddTable(span, 1, 100) + tableSink, ok := manager.tableSinks.Load(span) require.True(t, ok) require.NotNil(t, tableSink) require.Equal(t, 0, manager.sinkProgressHeap.len(), "Not started table shout not in progress heap") - err := manager.StartTable(tableID, 1) + err := manager.StartTable(span, 1) require.NoError(t, err) require.Equal(t, uint64(0x7ffffffffffbffff), tableSink.(*tableSinkWrapper).replicateTs) progress := manager.sinkProgressHeap.pop() - require.Equal(t, tableID, progress.tableID) + require.Equal(t, span, progress.span) require.Equal(t, uint64(0), progress.nextLowerBoundPos.StartTs) require.Equal(t, uint64(2), progress.nextLowerBoundPos.CommitTs) } @@ -175,32 +176,32 @@ func TestRemoveTable(t *testing.T) { err := manager.Close() require.NoError(t, err) }() - tableID := model.TableID(1) - manager.AddTable(tableID, 1, 100) - tableSink, ok := manager.tableSinks.Load(tableID) + span := spanz.TableIDToComparableSpan(1) + manager.AddTable(span, 1, 100) + tableSink, ok := manager.tableSinks.Load(span) require.True(t, ok) require.NotNil(t, tableSink) - err := manager.StartTable(tableID, 0) + err := manager.StartTable(span, 0) require.NoError(t, err) - addTableAndAddEventsToSortEngine(t, e, tableID) + addTableAndAddEventsToSortEngine(t, e, span) manager.UpdateBarrierTs(4) - manager.UpdateReceivedSorterResolvedTs(tableID, 5) + manager.UpdateReceivedSorterResolvedTs(span, 5) // Check all the events are sent to sink and record the memory usage. require.Eventually(t, func() bool { return manager.memQuota.getUsedBytes() == 872 }, 5*time.Second, 10*time.Millisecond) - manager.AsyncStopTable(tableID) + manager.AsyncStopTable(span) require.Eventually(t, func() bool { - state, ok := manager.GetTableState(tableID) + state, ok := manager.GetTableState(span) require.True(t, ok) return state == tablepb.TableStateStopped }, 5*time.Second, 10*time.Millisecond) - manager.RemoveTable(tableID) + manager.RemoveTable(span) - _, ok = manager.tableSinks.Load(tableID) + _, ok = manager.tableSinks.Load(span) require.False(t, ok) require.Equal(t, uint64(0), manager.memQuota.getUsedBytes(), "After remove table, the memory usage should be 0.") } @@ -235,16 +236,16 @@ func TestGenerateTableSinkTaskWithBarrierTs(t *testing.T) { err := manager.Close() require.NoError(t, err) }() - tableID := model.TableID(1) - manager.AddTable(tableID, 1, 100) - addTableAndAddEventsToSortEngine(t, e, tableID) + span := spanz.TableIDToComparableSpan(1) + manager.AddTable(span, 1, 100) + addTableAndAddEventsToSortEngine(t, e, span) manager.UpdateBarrierTs(4) - manager.UpdateReceivedSorterResolvedTs(tableID, 5) - err := manager.StartTable(tableID, 0) + manager.UpdateReceivedSorterResolvedTs(span, 5) + err := manager.StartTable(span, 0) require.NoError(t, err) require.Eventually(t, func() bool { - tableSink, ok := manager.tableSinks.Load(tableID) + tableSink, ok := manager.tableSinks.Load(span) require.True(t, ok) checkpointTS := tableSink.(*tableSinkWrapper).getCheckpointTs() return checkpointTS.ResolvedMark() == 4 @@ -263,18 +264,18 @@ func TestGenerateTableSinkTaskWithResolvedTs(t *testing.T) { err := manager.Close() require.NoError(t, err) }() - tableID := model.TableID(1) - manager.AddTable(tableID, 1, 100) - addTableAndAddEventsToSortEngine(t, e, tableID) + span := spanz.TableIDToComparableSpan(1) + manager.AddTable(span, 1, 100) + addTableAndAddEventsToSortEngine(t, e, span) // This would happen when the table just added to this node and redo log is enabled. // So there is possibility that the resolved ts is smaller than the global barrier ts. manager.UpdateBarrierTs(4) - manager.UpdateReceivedSorterResolvedTs(tableID, 3) - err := manager.StartTable(tableID, 0) + manager.UpdateReceivedSorterResolvedTs(span, 3) + err := manager.StartTable(span, 0) require.NoError(t, err) require.Eventually(t, func() bool { - tableSink, ok := manager.tableSinks.Load(tableID) + tableSink, ok := manager.tableSinks.Load(span) require.True(t, ok) checkpointTS := tableSink.(*tableSinkWrapper).getCheckpointTs() return checkpointTS.ResolvedMark() == 3 @@ -293,17 +294,17 @@ func TestGetTableStatsToReleaseMemQuota(t *testing.T) { err := manager.Close() require.NoError(t, err) }() - tableID := model.TableID(1) - manager.AddTable(tableID, 1, 100) - addTableAndAddEventsToSortEngine(t, e, tableID) + span := spanz.TableIDToComparableSpan(1) + manager.AddTable(span, 1, 100) + addTableAndAddEventsToSortEngine(t, e, span) manager.UpdateBarrierTs(4) - manager.UpdateReceivedSorterResolvedTs(tableID, 5) - err := manager.StartTable(tableID, 0) + manager.UpdateReceivedSorterResolvedTs(span, 5) + err := manager.StartTable(span, 0) require.NoError(t, err) require.Eventually(t, func() bool { - s := manager.GetTableStats(tableID) + s := manager.GetTableStats(span) return manager.memQuota.getUsedBytes() == 0 && s.CheckpointTs == 4 }, 5*time.Second, 10*time.Millisecond) } @@ -316,14 +317,14 @@ func TestDoNotGenerateTableSinkTaskWhenTableIsNotReplicating(t *testing.T) { changefeedInfo := getChangefeedInfo() manager, e := createManagerWithMemEngine(t, ctx, model.DefaultChangeFeedID("1"), changefeedInfo, make(chan error, 1)) - tableID := model.TableID(1) - manager.AddTable(tableID, 1, 100) - addTableAndAddEventsToSortEngine(t, e, tableID) + span := spanz.TableIDToComparableSpan(1) + manager.AddTable(span, 1, 100) + addTableAndAddEventsToSortEngine(t, e, span) manager.UpdateBarrierTs(4) - manager.UpdateReceivedSorterResolvedTs(tableID, 5) + manager.UpdateReceivedSorterResolvedTs(span, 5) require.Equal(t, uint64(0), manager.memQuota.getUsedBytes()) - tableSink, ok := manager.tableSinks.Load(tableID) + tableSink, ok := manager.tableSinks.Load(span) require.True(t, ok) require.NotNil(t, tableSink) require.Equal(t, uint64(1), tableSink.(*tableSinkWrapper).getCheckpointTs().Ts) @@ -354,5 +355,5 @@ func TestUpdateReceivedSorterResolvedTsOfNonExistTable(t *testing.T) { changefeedInfo := getChangefeedInfo() manager, _ := createManagerWithMemEngine(t, ctx, model.DefaultChangeFeedID("1"), changefeedInfo, make(chan error, 1)) - manager.UpdateReceivedSorterResolvedTs(model.TableID(1), 1) + manager.UpdateReceivedSorterResolvedTs(spanz.TableIDToComparableSpan(1), 1) } diff --git a/cdc/processor/sinkmanager/mem_quota.go b/cdc/processor/sinkmanager/mem_quota.go index c885bf1113b..fd5ae2914bb 100644 --- a/cdc/processor/sinkmanager/mem_quota.go +++ b/cdc/processor/sinkmanager/mem_quota.go @@ -20,7 +20,9 @@ import ( "github.com/pingcap/log" "github.com/pingcap/tiflow/cdc/model" + "github.com/pingcap/tiflow/cdc/processor/tablepb" cerrors "github.com/pingcap/tiflow/pkg/errors" + "github.com/pingcap/tiflow/pkg/spanz" "github.com/prometheus/client_golang/prometheus" "go.uber.org/zap" ) @@ -43,7 +45,7 @@ type memQuota struct { // usedBytes is the memory usage of one changefeed. usedBytes uint64 // tableMemory is the memory usage of each table. - tableMemory map[model.TableID][]*memConsumeRecord + tableMemory *spanz.Map[[]*memConsumeRecord] // isClosed is used to indicate whether the mem quota is closed. isClosed atomic.Bool @@ -56,7 +58,7 @@ func newMemQuota(changefeedID model.ChangeFeedID, totalBytes uint64) *memQuota { changefeedID: changefeedID, totalBytes: totalBytes, usedBytes: 0, - tableMemory: make(map[model.TableID][]*memConsumeRecord), + tableMemory: spanz.NewMap[[]*memConsumeRecord](), metricTotal: MemoryQuota.WithLabelValues(changefeedID.Namespace, changefeedID.ID, "total"), metricUsed: MemoryQuota.WithLabelValues(changefeedID.Namespace, changefeedID.ID, "used"), } @@ -127,20 +129,20 @@ func (m *memQuota) refund(nBytes uint64) { } } -func (m *memQuota) addTable(tableID model.TableID) { +func (m *memQuota) addTable(span tablepb.Span) { m.mu.Lock() defer m.mu.Unlock() - m.tableMemory[tableID] = make([]*memConsumeRecord, 0, 2) + m.tableMemory.ReplaceOrInsert(span, make([]*memConsumeRecord, 0, 2)) } // record records the memory usage of a table. -func (m *memQuota) record(tableID model.TableID, resolved model.ResolvedTs, nBytes uint64) { +func (m *memQuota) record(span tablepb.Span, resolved model.ResolvedTs, nBytes uint64) { if nBytes == 0 { return } m.mu.Lock() defer m.mu.Unlock() - if _, ok := m.tableMemory[tableID]; !ok { + if _, ok := m.tableMemory.Get(span); !ok { // Can't find the table record, the table must be removed. if m.usedBytes < nBytes { log.Panic("memQuota.refund fail", @@ -153,25 +155,25 @@ func (m *memQuota) record(tableID model.TableID, resolved model.ResolvedTs, nByt } return } - m.tableMemory[tableID] = append(m.tableMemory[tableID], &memConsumeRecord{ + m.tableMemory.ReplaceOrInsert(span, append(m.tableMemory.GetV(span), &memConsumeRecord{ resolvedTs: resolved, size: nBytes, - }) + })) } // release try to use resolvedTs to release the memory quota. // Because we append records in order, we can use binary search to find the first record // that is greater than resolvedTs, and release the memory quota of the records before it. -func (m *memQuota) release(tableID model.TableID, resolved model.ResolvedTs) { +func (m *memQuota) release(span tablepb.Span, resolved model.ResolvedTs) { m.mu.Lock() defer m.mu.Unlock() - if _, ok := m.tableMemory[tableID]; !ok { + if _, ok := m.tableMemory.Get(span); !ok { // This can happen when // 1. the table has no data and never been recorded. // 2. the table is in async removing. return } - records := m.tableMemory[tableID] + records := m.tableMemory.GetV(span) i := sort.Search(len(records), func(i int) bool { return records[i].resolvedTs.Greater(resolved) }) @@ -179,7 +181,7 @@ func (m *memQuota) release(tableID model.TableID, resolved model.ResolvedTs) { for j := 0; j < i; j++ { toRelease += records[j].size } - m.tableMemory[tableID] = records[i:] + m.tableMemory.ReplaceOrInsert(span, records[i:]) if toRelease == 0 { return } @@ -196,26 +198,26 @@ func (m *memQuota) release(tableID model.TableID, resolved model.ResolvedTs) { // clean all records of the table. // Return the cleaned memory quota. -func (m *memQuota) clean(tableID model.TableID) uint64 { +func (m *memQuota) clean(span tablepb.Span) uint64 { m.mu.Lock() defer m.mu.Unlock() - if _, ok := m.tableMemory[tableID]; !ok { + if _, ok := m.tableMemory.Get(span); !ok { // This can happen when the table has no data and never been recorded. log.Warn("Table consumed memory records not found", zap.String("namespace", m.changefeedID.Namespace), zap.String("changefeed", m.changefeedID.ID), - zap.Int64("tableID", tableID)) + zap.Stringer("span", &span)) return 0 } cleaned := uint64(0) - records := m.tableMemory[tableID] + records := m.tableMemory.GetV(span) for _, record := range records { cleaned += record.size } m.usedBytes -= cleaned m.metricUsed.Set(float64(m.usedBytes)) - delete(m.tableMemory, tableID) + m.tableMemory.Delete(span) if m.usedBytes < m.totalBytes { m.blockAcquireCond.Broadcast() } @@ -227,7 +229,7 @@ func (m *memQuota) close() { m.mu.Lock() defer m.mu.Unlock() // NOTE: m.usedBytes is not reset, because refund can still be called after closed. - m.tableMemory = make(map[model.TableID][]*memConsumeRecord) + m.tableMemory = spanz.NewMap[[]*memConsumeRecord]() m.metricUsed.Set(float64(0)) m.isClosed.Store(true) m.blockAcquireCond.Broadcast() diff --git a/cdc/processor/sinkmanager/mem_quota_test.go b/cdc/processor/sinkmanager/mem_quota_test.go index baa5e27b526..1e8a8a48b52 100644 --- a/cdc/processor/sinkmanager/mem_quota_test.go +++ b/cdc/processor/sinkmanager/mem_quota_test.go @@ -19,6 +19,7 @@ import ( "github.com/pingcap/tiflow/cdc/model" cerrors "github.com/pingcap/tiflow/pkg/errors" + "github.com/pingcap/tiflow/pkg/spanz" "github.com/stretchr/testify/require" ) @@ -52,8 +53,10 @@ func TestMemQuotaBlockAcquire(t *testing.T) { defer m.close() err := m.blockAcquire(100) require.NoError(t, err) - m.record(1, model.NewResolvedTs(1), 50) - m.record(1, model.NewResolvedTs(2), 50) + + span := spanz.TableIDToComparableSpan(1) + m.record(span, model.NewResolvedTs(1), 50) + m.record(span, model.NewResolvedTs(2), 50) var wg sync.WaitGroup wg.Add(1) @@ -68,8 +71,8 @@ func TestMemQuotaBlockAcquire(t *testing.T) { err := m.blockAcquire(50) require.NoError(t, err) }() - m.release(1, model.NewResolvedTs(1)) - m.release(1, model.NewResolvedTs(2)) + m.release(span, model.NewResolvedTs(1)) + m.release(span, model.NewResolvedTs(2)) wg.Wait() } @@ -80,7 +83,8 @@ func TestMemQuotaClose(t *testing.T) { err := m.blockAcquire(100) require.NoError(t, err) - m.record(1, model.NewResolvedTs(2), 100) + span := spanz.TableIDToComparableSpan(1) + m.record(span, model.NewResolvedTs(2), 100) var wg sync.WaitGroup wg.Add(1) @@ -112,7 +116,7 @@ func TestMemQuotaClose(t *testing.T) { wg.Add(1) go func() { defer wg.Done() - m.release(1, model.NewResolvedTs(2)) + m.release(span, model.NewResolvedTs(2)) }() m.close() wg.Wait() @@ -158,27 +162,28 @@ func TestMemQuotaRecordAndRelease(t *testing.T) { m := newMemQuota(model.DefaultChangeFeedID("1"), 300) defer m.close() - m.addTable(1) + span := spanz.TableIDToComparableSpan(1) + m.addTable(span) require.True(t, m.tryAcquire(100)) - m.record(1, model.NewResolvedTs(100), 100) + m.record(span, model.NewResolvedTs(100), 100) require.True(t, m.tryAcquire(100)) - m.record(1, model.NewResolvedTs(200), 100) + m.record(span, model.NewResolvedTs(200), 100) require.True(t, m.tryAcquire(100)) - m.record(1, model.NewResolvedTs(300), 100) + m.record(span, model.NewResolvedTs(300), 100) require.False(t, m.tryAcquire(1)) require.False(t, m.hasAvailable(1)) // release the memory of resolvedTs 100 - m.release(1, model.NewResolvedTs(101)) + m.release(span, model.NewResolvedTs(101)) require.True(t, m.hasAvailable(100)) // release the memory of resolvedTs 200 - m.release(1, model.NewResolvedTs(201)) + m.release(span, model.NewResolvedTs(201)) require.True(t, m.hasAvailable(200)) // release the memory of resolvedTs 300 - m.release(1, model.NewResolvedTs(301)) + m.release(span, model.NewResolvedTs(301)) require.True(t, m.hasAvailable(300)) // release the memory of resolvedTs 300 again - m.release(1, model.NewResolvedTs(301)) + m.release(span, model.NewResolvedTs(301)) require.True(t, m.hasAvailable(300)) } @@ -187,7 +192,8 @@ func TestMemQuotaRecordAndReleaseWithBatchID(t *testing.T) { m := newMemQuota(model.DefaultChangeFeedID("1"), 300) defer m.close() - m.addTable(1) + span := spanz.TableIDToComparableSpan(1) + m.addTable(span) require.True(t, m.tryAcquire(100)) resolvedTs := model.ResolvedTs{ @@ -195,21 +201,21 @@ func TestMemQuotaRecordAndReleaseWithBatchID(t *testing.T) { Ts: 100, BatchID: 1, } - m.record(1, resolvedTs, 100) + m.record(span, resolvedTs, 100) resolvedTs = model.ResolvedTs{ Mode: model.BatchResolvedMode, Ts: 100, BatchID: 2, } require.True(t, m.tryAcquire(100)) - m.record(1, resolvedTs, 100) + m.record(span, resolvedTs, 100) resolvedTs = model.ResolvedTs{ Mode: model.BatchResolvedMode, Ts: 100, BatchID: 3, } require.True(t, m.tryAcquire(100)) - m.record(1, resolvedTs, 100) + m.record(span, resolvedTs, 100) require.False(t, m.tryAcquire(1)) require.False(t, m.hasAvailable(1)) @@ -219,10 +225,10 @@ func TestMemQuotaRecordAndReleaseWithBatchID(t *testing.T) { Ts: 100, BatchID: 2, } - m.release(1, resolvedTs) + m.release(span, resolvedTs) require.True(t, m.hasAvailable(200)) // release the memory of resolvedTs 101 - m.release(1, model.NewResolvedTs(101)) + m.release(span, model.NewResolvedTs(101)) require.True(t, m.hasAvailable(300)) } @@ -231,19 +237,20 @@ func TestMemQuotaRecordAndClean(t *testing.T) { m := newMemQuota(model.DefaultChangeFeedID("1"), 300) defer m.close() - m.addTable(1) + span := spanz.TableIDToComparableSpan(1) + m.addTable(span) require.True(t, m.tryAcquire(100)) - m.record(1, model.NewResolvedTs(100), 100) + m.record(span, model.NewResolvedTs(100), 100) require.True(t, m.tryAcquire(100)) - m.record(1, model.NewResolvedTs(200), 100) + m.record(span, model.NewResolvedTs(200), 100) require.True(t, m.tryAcquire(100)) - m.record(1, model.NewResolvedTs(300), 100) + m.record(span, model.NewResolvedTs(300), 100) require.False(t, m.tryAcquire(1)) require.False(t, m.hasAvailable(1)) // clean the all memory. - cleanedBytes := m.clean(1) + cleanedBytes := m.clean(span) require.Equal(t, uint64(300), cleanedBytes) require.True(t, m.hasAvailable(100)) } diff --git a/cdc/processor/sinkmanager/redo_cache.go b/cdc/processor/sinkmanager/redo_cache.go index 98e9392b38c..2743030699e 100644 --- a/cdc/processor/sinkmanager/redo_cache.go +++ b/cdc/processor/sinkmanager/redo_cache.go @@ -20,6 +20,8 @@ import ( "github.com/pingcap/tiflow/cdc/model" "github.com/pingcap/tiflow/cdc/processor/sourcemanager/engine" + "github.com/pingcap/tiflow/cdc/processor/tablepb" + "github.com/pingcap/tiflow/pkg/spanz" "github.com/prometheus/client_golang/prometheus" ) @@ -29,7 +31,7 @@ type redoEventCache struct { allocated uint64 // atomically shared in several goroutines. mu sync.Mutex - tables map[model.TableID]*eventAppender + tables map[spanz.HashableSpan]*eventAppender metricRedoEventCache prometheus.Gauge } @@ -69,15 +71,15 @@ func newRedoEventCache(changefeedID model.ChangeFeedID, capacity uint64) *redoEv return &redoEventCache{ capacity: capacity, allocated: 0, - tables: make(map[model.TableID]*eventAppender), + tables: make(map[spanz.HashableSpan]*eventAppender), metricRedoEventCache: RedoEventCache.WithLabelValues(changefeedID.Namespace, changefeedID.ID), } } -func (r *redoEventCache) removeTable(tableID model.TableID) { +func (r *redoEventCache) removeTable(span tablepb.Span) { r.mu.Lock() - item, exists := r.tables[tableID] + item, exists := r.tables[spanz.ToHashableSpan(span)] defer r.mu.Unlock() if exists { item.mu.Lock() @@ -90,22 +92,24 @@ func (r *redoEventCache) removeTable(tableID model.TableID) { item.sizes = nil item.pushCounts = nil item.mu.Unlock() - delete(r.tables, tableID) + delete(r.tables, spanz.ToHashableSpan(span)) } } -func (r *redoEventCache) maybeCreateAppender(tableID model.TableID, lowerBound engine.Position) *eventAppender { +func (r *redoEventCache) maybeCreateAppender( + span tablepb.Span, lowerBound engine.Position, +) *eventAppender { r.mu.Lock() defer r.mu.Unlock() - item, exists := r.tables[tableID] + item, exists := r.tables[spanz.ToHashableSpan(span)] if !exists { item = &eventAppender{ capacity: r.capacity, cache: r, lowerBound: lowerBound, } - r.tables[tableID] = item + r.tables[spanz.ToHashableSpan(span)] = item return item } @@ -125,10 +129,10 @@ func (r *redoEventCache) maybeCreateAppender(tableID model.TableID, lowerBound e return item } -func (r *redoEventCache) getAppender(tableID model.TableID) *eventAppender { +func (r *redoEventCache) getAppender(span tablepb.Span) *eventAppender { r.mu.Lock() defer r.mu.Unlock() - return r.tables[tableID] + return r.tables[spanz.ToHashableSpan(span)] } func (e *eventAppender) pop(lowerBound, upperBound engine.Position) (res popResult) { diff --git a/cdc/processor/sinkmanager/redo_cache_test.go b/cdc/processor/sinkmanager/redo_cache_test.go index 87f911dab7f..a583d3c56cd 100644 --- a/cdc/processor/sinkmanager/redo_cache_test.go +++ b/cdc/processor/sinkmanager/redo_cache_test.go @@ -18,6 +18,7 @@ import ( "github.com/pingcap/tiflow/cdc/model" "github.com/pingcap/tiflow/cdc/processor/sourcemanager/engine" + "github.com/pingcap/tiflow/pkg/spanz" "github.com/stretchr/testify/require" ) @@ -27,7 +28,8 @@ func TestRedoEventCache(t *testing.T) { var broken uint64 var popRes popResult - appender := cache.maybeCreateAppender(3, engine.Position{StartTs: 1, CommitTs: 2}) + span := spanz.TableIDToComparableSpan(3) + appender := cache.maybeCreateAppender(span, engine.Position{StartTs: 1, CommitTs: 2}) appender.push(&model.RowChangedEvent{StartTs: 1, CommitTs: 2}, 100, engine.Position{}) appender.push(&model.RowChangedEvent{StartTs: 1, CommitTs: 2}, 200, engine.Position{}) @@ -68,7 +70,7 @@ func TestRedoEventCache(t *testing.T) { require.Equal(t, 0, len(appender.events)) require.True(t, appender.broken) - appender = cache.maybeCreateAppender(3, engine.Position{StartTs: 11, CommitTs: 12}) + appender = cache.maybeCreateAppender(span, engine.Position{StartTs: 11, CommitTs: 12}) require.False(t, appender.broken) require.Equal(t, uint64(0), appender.upperBound.StartTs) require.Equal(t, uint64(0), appender.upperBound.CommitTs) diff --git a/cdc/processor/sinkmanager/redo_log_worker.go b/cdc/processor/sinkmanager/redo_log_worker.go index dd9aa2e6f5e..8bcca04d039 100644 --- a/cdc/processor/sinkmanager/redo_log_worker.go +++ b/cdc/processor/sinkmanager/redo_log_worker.go @@ -22,7 +22,6 @@ import ( "github.com/pingcap/tiflow/cdc/processor/sourcemanager" "github.com/pingcap/tiflow/cdc/processor/sourcemanager/engine" "github.com/pingcap/tiflow/cdc/redo" - "github.com/pingcap/tiflow/pkg/spanz" "go.uber.org/zap" ) @@ -75,13 +74,13 @@ func (w *redoWorker) handleTask(ctx context.Context, task *redoTask) (finalErr e log.Panic("redo task upperbound must be a ResolvedTs", zap.String("namespace", w.changefeedID.Namespace), zap.String("changefeed", w.changefeedID.ID), - zap.Int64("tableID", task.tableID), + zap.Stringer("span", &task.span), zap.Any("upperBound", upperBound)) } var cache *eventAppender if w.eventCache != nil { - cache = w.eventCache.maybeCreateAppender(task.tableID, task.lowerBound) + cache = w.eventCache.maybeCreateAppender(task.span, task.lowerBound) } // Events are pushed into redoEventCache if possible. Otherwise, their memory will @@ -112,12 +111,11 @@ func (w *redoWorker) handleTask(ctx context.Context, task *redoTask) (finalErr e log.Debug("MemoryQuotaTracing: refund memory for redo log task", zap.String("namespace", w.changefeedID.Namespace), zap.String("changefeed", w.changefeedID.ID), - zap.Int64("tableID", task.tableID), + zap.Stringer("span", &task.span), zap.Uint64("memory", refundMem)) } } - err := w.redoManager.EmitRowChangedEvents( - ctx, spanz.TableIDToComparableSpan(task.tableID), releaseMem, rows...) + err := w.redoManager.EmitRowChangedEvents(ctx, task.span, releaseMem, rows...) if err != nil { return errors.Trace(err) } @@ -127,14 +125,13 @@ func (w *redoWorker) handleTask(ctx context.Context, task *redoTask) (finalErr e cachedSize = 0 } if lastTxnCommitTs > emitedCommitTs { - if err := w.redoManager.UpdateResolvedTs( - ctx, spanz.TableIDToComparableSpan(task.tableID), lastTxnCommitTs); err != nil { + if err := w.redoManager.UpdateResolvedTs(ctx, task.span, lastTxnCommitTs); err != nil { return errors.Trace(err) } log.Debug("update resolved ts to redo", zap.String("namespace", w.changefeedID.Namespace), zap.String("changefeed", w.changefeedID.ID), - zap.Int64("tableID", task.tableID), + zap.Stringer("span", &task.span), zap.Uint64("resolvedTs", lastTxnCommitTs)) emitedCommitTs = lastTxnCommitTs } @@ -160,7 +157,7 @@ func (w *redoWorker) handleTask(ctx context.Context, task *redoTask) (finalErr e log.Debug("MemoryQuotaTracing: force acquire memory for redo log task", zap.String("namespace", w.changefeedID.Namespace), zap.String("changefeed", w.changefeedID.ID), - zap.Int64("tableID", task.tableID), + zap.Stringer("span", &task.span), zap.Uint64("memory", usedMemSize-availableMemSize)) availableMemSize = usedMemSize } @@ -175,7 +172,7 @@ func (w *redoWorker) handleTask(ctx context.Context, task *redoTask) (finalErr e log.Debug("MemoryQuotaTracing: try acquire memory for redo log task", zap.String("namespace", w.changefeedID.Namespace), zap.String("changefeed", w.changefeedID.ID), - zap.Int64("tableID", task.tableID), + zap.Stringer("span", &task.span), zap.Uint64("memory", requestMemSize)) } } else { @@ -189,7 +186,7 @@ func (w *redoWorker) handleTask(ctx context.Context, task *redoTask) (finalErr e log.Debug("MemoryQuotaTracing: block acquire memory for redo log task", zap.String("namespace", w.changefeedID.Namespace), zap.String("changefeed", w.changefeedID.ID), - zap.Int64("tableID", task.tableID), + zap.Stringer("span", &task.span), zap.Uint64("memory", requestMemSize)) } } @@ -197,7 +194,7 @@ func (w *redoWorker) handleTask(ctx context.Context, task *redoTask) (finalErr e return nil } - iter := w.sourceManager.FetchByTable(task.tableID, task.lowerBound, upperBound) + iter := w.sourceManager.FetchByTable(task.span, task.lowerBound, upperBound) allEventCount := 0 defer func() { task.tableSink.updateReceivedSorterCommitTs(lastTxnCommitTs) @@ -208,14 +205,14 @@ func (w *redoWorker) handleTask(ctx context.Context, task *redoTask) (finalErr e log.Error("redo worker fails to close iterator", zap.String("namespace", w.changefeedID.Namespace), zap.String("changefeed", w.changefeedID.ID), - zap.Int64("tableID", task.tableID), + zap.Stringer("span", &task.span), zap.Error(err)) } log.Debug("redo task finished", zap.String("namespace", w.changefeedID.Namespace), zap.String("changefeed", w.changefeedID.ID), - zap.Int64("tableID", task.tableID), + zap.Stringer("span", &task.span), zap.Any("lowerBound", task.lowerBound), zap.Any("upperBound", upperBound), zap.Any("lastPos", lastPos)) @@ -230,7 +227,7 @@ func (w *redoWorker) handleTask(ctx context.Context, task *redoTask) (finalErr e log.Debug("MemoryQuotaTracing: refund memory for redo log task", zap.String("namespace", w.changefeedID.Namespace), zap.String("changefeed", w.changefeedID.ID), - zap.Int64("tableID", task.tableID), + zap.Stringer("span", &task.span), zap.Uint64("memory", availableMemSize-usedMemSize)) } }() @@ -269,7 +266,7 @@ func (w *redoWorker) handleTask(ctx context.Context, task *redoTask) (finalErr e if e.Row != nil { // For all rows, we add table replicate ts, so mysql sink can determine safe-mode. e.Row.ReplicatingTs = task.tableSink.replicateTs - x, size, err = convertRowChangedEvents(w.changefeedID, task.tableID, w.enableOldValue, e) + x, size, err = convertRowChangedEvents(w.changefeedID, task.span, w.enableOldValue, e) if err != nil { return errors.Trace(err) } diff --git a/cdc/processor/sinkmanager/table_progress_heap.go b/cdc/processor/sinkmanager/table_progress_heap.go index 455086ca747..f078d9448bc 100644 --- a/cdc/processor/sinkmanager/table_progress_heap.go +++ b/cdc/processor/sinkmanager/table_progress_heap.go @@ -17,13 +17,13 @@ import ( "container/heap" "sync" - "github.com/pingcap/tiflow/cdc/model" "github.com/pingcap/tiflow/cdc/processor/sourcemanager/engine" + "github.com/pingcap/tiflow/cdc/processor/tablepb" ) // progress is the fetch progress of a table. type progress struct { - tableID model.TableID + span tablepb.Span nextLowerBoundPos engine.Position version uint64 } diff --git a/cdc/processor/sinkmanager/table_progress_heap_test.go b/cdc/processor/sinkmanager/table_progress_heap_test.go index 2b8affdbde7..f6c0e678c27 100644 --- a/cdc/processor/sinkmanager/table_progress_heap_test.go +++ b/cdc/processor/sinkmanager/table_progress_heap_test.go @@ -17,6 +17,7 @@ import ( "testing" "github.com/pingcap/tiflow/cdc/processor/sourcemanager/engine" + "github.com/pingcap/tiflow/pkg/spanz" "github.com/stretchr/testify/require" ) @@ -25,21 +26,21 @@ func TestTableProgresses(t *testing.T) { p := newTableProgresses() p.push(&progress{ - tableID: 1, + span: spanz.TableIDToComparableSpan(1), nextLowerBoundPos: engine.Position{ StartTs: 1, CommitTs: 2, }, }) p.push(&progress{ - tableID: 3, + span: spanz.TableIDToComparableSpan(3), nextLowerBoundPos: engine.Position{ StartTs: 2, CommitTs: 2, }, }) p.push(&progress{ - tableID: 2, + span: spanz.TableIDToComparableSpan(2), nextLowerBoundPos: engine.Position{ StartTs: 2, CommitTs: 3, @@ -49,11 +50,11 @@ func TestTableProgresses(t *testing.T) { require.Equal(t, p.len(), 3) pg := p.pop() - require.Equal(t, int64(1), pg.tableID, "table1 is the slowest table") + require.Equal(t, spanz.TableIDToComparableSpan(1), pg.span, "table1 is the slowest table") pg = p.pop() - require.Equal(t, int64(3), pg.tableID, "table2 is the slowest table") + require.Equal(t, spanz.TableIDToComparableSpan(3), pg.span, "table2 is the slowest table") pg = p.pop() - require.Equal(t, int64(2), pg.tableID, "table3 is the slowest table") + require.Equal(t, spanz.TableIDToComparableSpan(2), pg.span, "table3 is the slowest table") require.Equal(t, p.len(), 0, "all tables are popped") } diff --git a/cdc/processor/sinkmanager/table_sink_worker.go b/cdc/processor/sinkmanager/table_sink_worker.go index 886c65caa47..ef74f26d512 100644 --- a/cdc/processor/sinkmanager/table_sink_worker.go +++ b/cdc/processor/sinkmanager/table_sink_worker.go @@ -84,7 +84,7 @@ func (w *sinkWorker) handleTask(ctx context.Context, task *sinkTask) (finalErr e log.Panic("sink task upperbound must be a ResolvedTs", zap.String("namespace", w.changefeedID.Namespace), zap.String("changefeed", w.changefeedID.ID), - zap.Int64("tableID", task.tableID), + zap.Stringer("span", &task.span), zap.Any("upperBound", upperBound)) } @@ -119,7 +119,7 @@ func (w *sinkWorker) handleTask(ctx context.Context, task *sinkTask) (finalErr e log.Debug("MemoryQuotaTracing: refund memory for table sink task", zap.String("namespace", w.changefeedID.Namespace), zap.String("changefeed", w.changefeedID.ID), - zap.Int64("tableID", task.tableID), + zap.Stringer("span", &task.span), zap.Uint64("memory", availableMem-usedMem)) task.callback(lowerBound.Prev()) return nil @@ -142,7 +142,7 @@ func (w *sinkWorker) handleTask(ctx context.Context, task *sinkTask) (finalErr e log.Debug("check should advance or not", zap.String("namespace", w.changefeedID.Namespace), zap.String("changefeed", w.changefeedID.ID), - zap.Int64("tableID", task.tableID), + zap.Stringer("span", &task.span), zap.Bool("splitTxn", w.splitTxn), zap.Uint64("currTxnCommitTs", currTxnCommitTs), zap.Uint64("lastTxnCommitTs", lastTxnCommitTs), @@ -172,7 +172,7 @@ func (w *sinkWorker) handleTask(ctx context.Context, task *sinkTask) (finalErr e // It's the last time we call `doEmitAndAdvance`, but `pendingTxnSize` // hasn't been recorded yet. To avoid losing it, record it manually. if isLastTime && pendingTxnSize > 0 { - w.memQuota.record(task.tableID, model.NewResolvedTs(currTxnCommitTs), pendingTxnSize) + w.memQuota.record(task.span, model.NewResolvedTs(currTxnCommitTs), pendingTxnSize) pendingTxnSize = 0 } } @@ -197,7 +197,7 @@ func (w *sinkWorker) handleTask(ctx context.Context, task *sinkTask) (finalErr e log.Debug("MemoryQuotaTracing: force acquire memory for table sink task", zap.String("namespace", w.changefeedID.Namespace), zap.String("changefeed", w.changefeedID.ID), - zap.Int64("tableID", task.tableID), + zap.Stringer("span", &task.span), zap.Uint64("memory", usedMem-availableMem)) availableMem = usedMem } @@ -212,7 +212,7 @@ func (w *sinkWorker) handleTask(ctx context.Context, task *sinkTask) (finalErr e log.Debug("MemoryQuotaTracing: try acquire memory for table sink task", zap.String("namespace", w.changefeedID.Namespace), zap.String("changefeed", w.changefeedID.ID), - zap.Int64("tableID", task.tableID), + zap.Stringer("span", &task.span), zap.Uint64("memory", requestMemSize)) } } else { @@ -222,7 +222,7 @@ func (w *sinkWorker) handleTask(ctx context.Context, task *sinkTask) (finalErr e log.Debug("MemoryQuotaTracing: force acquire memory for table sink task", zap.String("namespace", w.changefeedID.Namespace), zap.String("changefeed", w.changefeedID.ID), - zap.Int64("tableID", task.tableID), + zap.Stringer("span", &task.span), zap.Uint64("memory", requestMemSize)) } else { // NOTE: if splitTxn is true it's not required to force acquire memory. @@ -233,7 +233,7 @@ func (w *sinkWorker) handleTask(ctx context.Context, task *sinkTask) (finalErr e log.Debug("MemoryQuotaTracing: block acquire memory for table sink task", zap.String("namespace", w.changefeedID.Namespace), zap.String("changefeed", w.changefeedID.ID), - zap.Int64("tableID", task.tableID), + zap.Stringer("span", &task.span), zap.Uint64("memory", requestMemSize)) } } @@ -244,7 +244,7 @@ func (w *sinkWorker) handleTask(ctx context.Context, task *sinkTask) (finalErr e // lowerBound and upperBound are both closed intervals. allEventSize := uint64(0) allEventCount := 0 - iter := w.sourceManager.FetchByTable(task.tableID, lowerBound, upperBound) + iter := w.sourceManager.FetchByTable(task.span, lowerBound, upperBound) defer func() { w.metricRedoEventCacheMiss.Add(float64(allEventSize)) task.tableSink.receivedEventCount.Add(int64(allEventCount)) @@ -264,14 +264,14 @@ func (w *sinkWorker) handleTask(ctx context.Context, task *sinkTask) (finalErr e log.Error("Sink worker fails to close iterator", zap.String("namespace", w.changefeedID.Namespace), zap.String("changefeed", w.changefeedID.ID), - zap.Int64("tableID", task.tableID), + zap.Stringer("span", &task.span), zap.Error(err)) } log.Debug("Sink task finished", zap.String("namespace", w.changefeedID.Namespace), zap.String("changefeed", w.changefeedID.ID), - zap.Int64("tableID", task.tableID), + zap.Stringer("span", &task.span), zap.Any("lowerBound", lowerBound), zap.Any("upperBound", upperBound), zap.Bool("splitTxn", w.splitTxn), @@ -288,7 +288,7 @@ func (w *sinkWorker) handleTask(ctx context.Context, task *sinkTask) (finalErr e log.Debug("MemoryQuotaTracing: refund memory for table sink task", zap.String("namespace", w.changefeedID.Namespace), zap.String("changefeed", w.changefeedID.ID), - zap.Int64("tableID", task.tableID), + zap.Stringer("span", &task.span), zap.Uint64("memory", availableMem-usedMem)) } }() @@ -322,7 +322,7 @@ func (w *sinkWorker) handleTask(ctx context.Context, task *sinkTask) (finalErr e if e.Row != nil { // For all rows, we add table replicate ts, so mysql sink can determine safe-mode. e.Row.ReplicatingTs = task.tableSink.replicateTs - x, size, err := convertRowChangedEvents(w.changefeedID, task.tableID, w.enableOldValue, e) + x, size, err := convertRowChangedEvents(w.changefeedID, task.span, w.enableOldValue, e) if err != nil { return err } @@ -348,7 +348,7 @@ func (w *sinkWorker) fetchFromCache( newLowerBound := *lowerBound newUpperBound := *upperBound - cache := w.eventCache.getAppender(task.tableID) + cache := w.eventCache.getAppender(task.span) if cache == nil { return } @@ -383,13 +383,13 @@ func (w *sinkWorker) fetchFromCache( } } // NOTE: the recorded size can be not accurate, but let it be. - w.memQuota.record(task.tableID, resolvedTs, popRes.releaseSize) + w.memQuota.record(task.span, resolvedTs, popRes.releaseSize) if err = task.tableSink.updateResolvedTs(resolvedTs); err == nil { } log.Debug("Advance table sink", zap.String("namespace", w.changefeedID.Namespace), zap.String("changefeed", w.changefeedID.ID), - zap.Int64("tableID", task.tableID), + zap.Stringer("span", &task.span), zap.Any("resolvedTs", resolvedTs)) } else { newUpperBound = popRes.boundary.Prev() @@ -398,7 +398,7 @@ func (w *sinkWorker) fetchFromCache( log.Debug("fetchFromCache is performed", zap.String("namespace", w.changefeedID.Namespace), zap.String("changefeed", w.changefeedID.ID), - zap.Int64("tableID", task.tableID), + zap.Stringer("span", &task.span), zap.Bool("success", popRes.success), zap.Int("eventsLen", len(popRes.events)), zap.Bool("cacheDrained", cacheDrained), @@ -418,11 +418,11 @@ func (w *sinkWorker) advanceTableSinkWithBatchID(t *sinkTask, commitTs model.Ts, log.Debug("Advance table sink with batch ID", zap.String("namespace", w.changefeedID.Namespace), zap.String("changefeed", w.changefeedID.ID), - zap.Int64("tableID", t.tableID), + zap.Stringer("span", &t.span), zap.Any("resolvedTs", resolvedTs), zap.Uint64("size", size)) if size > 0 { - w.memQuota.record(t.tableID, resolvedTs, size) + w.memQuota.record(t.span, resolvedTs, size) } return t.tableSink.updateResolvedTs(resolvedTs) } @@ -432,11 +432,11 @@ func (w *sinkWorker) advanceTableSink(t *sinkTask, commitTs model.Ts, size uint6 log.Debug("Advance table sink without batch ID", zap.String("namespace", w.changefeedID.Namespace), zap.String("changefeed", w.changefeedID.ID), - zap.Int64("tableID", t.tableID), + zap.Stringer("span", &t.span), zap.Any("resolvedTs", resolvedTs), zap.Uint64("size", size)) if size > 0 { - w.memQuota.record(t.tableID, resolvedTs, size) + w.memQuota.record(t.span, resolvedTs, size) } return t.tableSink.updateResolvedTs(resolvedTs) } diff --git a/cdc/processor/sinkmanager/table_sink_worker_test.go b/cdc/processor/sinkmanager/table_sink_worker_test.go index f1ab768aaed..e57bc1ab4ca 100644 --- a/cdc/processor/sinkmanager/table_sink_worker_test.go +++ b/cdc/processor/sinkmanager/table_sink_worker_test.go @@ -24,6 +24,7 @@ import ( "github.com/pingcap/tiflow/cdc/processor/sourcemanager" "github.com/pingcap/tiflow/cdc/processor/sourcemanager/engine" "github.com/pingcap/tiflow/cdc/processor/sourcemanager/engine/memory" + "github.com/pingcap/tiflow/cdc/processor/tablepb" cerrors "github.com/pingcap/tiflow/pkg/errors" "github.com/pingcap/tiflow/pkg/spanz" "github.com/pingcap/tiflow/pkg/upstream" @@ -35,7 +36,7 @@ func createWorker( changefeedID model.ChangeFeedID, memQuota uint64, splitTxn bool, - tableIDs ...model.TableID, + spans ...tablepb.Span, ) (*sinkWorker, engine.SortEngine) { sortEngine := memory.New(context.Background()) sm := sourcemanager.New(changefeedID, upstream.NewUpstream4Test(&mockPD{}), @@ -44,8 +45,8 @@ func createWorker( // To avoid refund or release panics. quota := newMemQuota(changefeedID, memQuota+1024*1024*1024) quota.forceAcquire(1024 * 1024 * 1024) - for _, tableID := range tableIDs { - quota.addTable(tableID) + for _, span := range spans { + quota.addTable(span) } return newSinkWorker(changefeedID, sm, quota, nil, splitTxn, false), sortEngine @@ -53,10 +54,13 @@ func createWorker( // nolint:unparam // It is ok to use the same tableID in test. -func addEventsToSortEngine(t *testing.T, events []*model.PolymorphicEvent, sortEngine engine.SortEngine, tableID model.TableID) { - sortEngine.AddTable(tableID) +func addEventsToSortEngine( + t *testing.T, events []*model.PolymorphicEvent, + sortEngine engine.SortEngine, span tablepb.Span, +) { + sortEngine.AddTable(span) for _, event := range events { - err := sortEngine.Add(tableID, event) + err := sortEngine.Add(span, event) require.NoError(t, err) } } @@ -64,14 +68,14 @@ func addEventsToSortEngine(t *testing.T, events []*model.PolymorphicEvent, sortE // It is ok to use the same tableID in test. // //nolint:unparam -func genRowChangedEvent(startTs, commitTs uint64, tableID model.TableID) *model.RowChangedEvent { +func genRowChangedEvent(startTs, commitTs uint64, span tablepb.Span) *model.RowChangedEvent { return &model.RowChangedEvent{ StartTs: startTs, CommitTs: commitTs, Table: &model.TableName{ Schema: "table", Table: "table", - TableID: tableID, + TableID: span.TableID, IsPartition: false, }, Columns: []*model.Column{ @@ -104,7 +108,7 @@ func (suite *workerSuite) TestHandleTaskWithSplitTxnAndGotSomeFilteredEvents() { suite.T().Skip("need to be fixed") changefeedID := model.DefaultChangeFeedID("1") - tableID := model.TableID(1) + span := spanz.TableIDToComparableSpan(1) ctx, cancel := context.WithCancel(context.Background()) // Only for three events. @@ -120,7 +124,7 @@ func (suite *workerSuite) TestHandleTaskWithSplitTxnAndGotSomeFilteredEvents() { StartTs: 1, CRTs: 1, }, - Row: genRowChangedEvent(1, 1, tableID), + Row: genRowChangedEvent(1, 1, span), }, // This event will be filtered, so its Row will be nil. { @@ -151,7 +155,7 @@ func (suite *workerSuite) TestHandleTaskWithSplitTxnAndGotSomeFilteredEvents() { StartTs: 1, CRTs: 2, }, - Row: genRowChangedEvent(1, 2, tableID), + Row: genRowChangedEvent(1, 2, span), }, { StartTs: 1, @@ -161,7 +165,7 @@ func (suite *workerSuite) TestHandleTaskWithSplitTxnAndGotSomeFilteredEvents() { StartTs: 1, CRTs: 3, }, - Row: genRowChangedEvent(1, 3, tableID), + Row: genRowChangedEvent(1, 3, span), }, { CRTs: 4, @@ -173,7 +177,7 @@ func (suite *workerSuite) TestHandleTaskWithSplitTxnAndGotSomeFilteredEvents() { } w, e := createWorker(changefeedID, eventSize, true) - addEventsToSortEngine(suite.T(), events, e, tableID) + addEventsToSortEngine(suite.T(), events, e, span) taskChan := make(chan *sinkTask) var wg sync.WaitGroup @@ -184,8 +188,7 @@ func (suite *workerSuite) TestHandleTaskWithSplitTxnAndGotSomeFilteredEvents() { require.Equal(suite.T(), context.Canceled, err) }() - wrapper, sink := createTableSinkWrapper( - changefeedID, spanz.TableIDToComparableSpan(tableID)) + wrapper, sink := createTableSinkWrapper(changefeedID, span) lowerBoundPos := engine.Position{ StartTs: 0, CommitTs: 1, @@ -208,7 +211,7 @@ func (suite *workerSuite) TestHandleTaskWithSplitTxnAndGotSomeFilteredEvents() { cancel() } taskChan <- &sinkTask{ - tableID: tableID, + span: span, lowerBound: lowerBoundPos, getUpperBound: upperBoundGetter, tableSink: wrapper, @@ -222,7 +225,7 @@ func (suite *workerSuite) TestHandleTaskWithSplitTxnAndGotSomeFilteredEvents() { // Test the case that the worker will stop when no memory quota and meet the txn boundary. func (suite *workerSuite) TestHandleTaskWithSplitTxnAndAbortWhenNoMemAndOneTxnFinished() { changefeedID := model.DefaultChangeFeedID("1") - tableID := model.TableID(1) + span := spanz.TableIDToComparableSpan(1) ctx, cancel := context.WithCancel(context.Background()) // Only for three events. @@ -238,7 +241,7 @@ func (suite *workerSuite) TestHandleTaskWithSplitTxnAndAbortWhenNoMemAndOneTxnFi StartTs: 1, CRTs: 1, }, - Row: genRowChangedEvent(1, 1, tableID), + Row: genRowChangedEvent(1, 1, span), }, { StartTs: 1, @@ -248,7 +251,7 @@ func (suite *workerSuite) TestHandleTaskWithSplitTxnAndAbortWhenNoMemAndOneTxnFi StartTs: 1, CRTs: 2, }, - Row: genRowChangedEvent(1, 2, tableID), + Row: genRowChangedEvent(1, 2, span), }, { StartTs: 1, @@ -258,7 +261,7 @@ func (suite *workerSuite) TestHandleTaskWithSplitTxnAndAbortWhenNoMemAndOneTxnFi StartTs: 1, CRTs: 3, }, - Row: genRowChangedEvent(1, 3, tableID), + Row: genRowChangedEvent(1, 3, span), }, { StartTs: 2, @@ -268,7 +271,7 @@ func (suite *workerSuite) TestHandleTaskWithSplitTxnAndAbortWhenNoMemAndOneTxnFi StartTs: 2, CRTs: 4, }, - Row: genRowChangedEvent(2, 4, tableID), + Row: genRowChangedEvent(2, 4, span), }, { CRTs: 4, @@ -279,8 +282,8 @@ func (suite *workerSuite) TestHandleTaskWithSplitTxnAndAbortWhenNoMemAndOneTxnFi }, } - w, e := createWorker(changefeedID, eventSize, true, tableID) - addEventsToSortEngine(suite.T(), events, e, tableID) + w, e := createWorker(changefeedID, eventSize, true, span) + addEventsToSortEngine(suite.T(), events, e, span) taskChan := make(chan *sinkTask) var wg sync.WaitGroup @@ -291,8 +294,7 @@ func (suite *workerSuite) TestHandleTaskWithSplitTxnAndAbortWhenNoMemAndOneTxnFi require.Equal(suite.T(), context.Canceled, err) }() - wrapper, sink := createTableSinkWrapper( - changefeedID, spanz.TableIDToComparableSpan(tableID)) + wrapper, sink := createTableSinkWrapper(changefeedID, span) lowerBoundPos := engine.Position{ StartTs: 0, CommitTs: 1, @@ -315,7 +317,7 @@ func (suite *workerSuite) TestHandleTaskWithSplitTxnAndAbortWhenNoMemAndOneTxnFi cancel() } taskChan <- &sinkTask{ - tableID: tableID, + span: span, lowerBound: lowerBoundPos, getUpperBound: upperBoundGetter, tableSink: wrapper, @@ -329,7 +331,7 @@ func (suite *workerSuite) TestHandleTaskWithSplitTxnAndAbortWhenNoMemAndOneTxnFi // Test the case that worker will block when no memory quota until the mem quota is aborted. func (suite *workerSuite) TestHandleTaskWithSplitTxnAndAbortWhenNoMemAndBlocked() { changefeedID := model.DefaultChangeFeedID("1") - tableID := model.TableID(1) + span := spanz.TableIDToComparableSpan(1) ctx, cancel := context.WithCancel(context.Background()) // Only for three events. @@ -345,7 +347,7 @@ func (suite *workerSuite) TestHandleTaskWithSplitTxnAndAbortWhenNoMemAndBlocked( StartTs: 1, CRTs: 10, }, - Row: genRowChangedEvent(1, 10, tableID), + Row: genRowChangedEvent(1, 10, span), }, { StartTs: 1, @@ -355,7 +357,7 @@ func (suite *workerSuite) TestHandleTaskWithSplitTxnAndAbortWhenNoMemAndBlocked( StartTs: 1, CRTs: 10, }, - Row: genRowChangedEvent(1, 10, tableID), + Row: genRowChangedEvent(1, 10, span), }, { StartTs: 1, @@ -365,7 +367,7 @@ func (suite *workerSuite) TestHandleTaskWithSplitTxnAndAbortWhenNoMemAndBlocked( StartTs: 1, CRTs: 10, }, - Row: genRowChangedEvent(1, 10, tableID), + Row: genRowChangedEvent(1, 10, span), }, { StartTs: 1, @@ -375,7 +377,7 @@ func (suite *workerSuite) TestHandleTaskWithSplitTxnAndAbortWhenNoMemAndBlocked( StartTs: 1, CRTs: 10, }, - Row: genRowChangedEvent(1, 10, tableID), + Row: genRowChangedEvent(1, 10, span), }, { CRTs: 14, @@ -385,8 +387,8 @@ func (suite *workerSuite) TestHandleTaskWithSplitTxnAndAbortWhenNoMemAndBlocked( }, }, } - w, e := createWorker(changefeedID, eventSize, true, tableID) - addEventsToSortEngine(suite.T(), events, e, tableID) + w, e := createWorker(changefeedID, eventSize, true, span) + addEventsToSortEngine(suite.T(), events, e, span) taskChan := make(chan *sinkTask) var wg sync.WaitGroup @@ -397,8 +399,7 @@ func (suite *workerSuite) TestHandleTaskWithSplitTxnAndAbortWhenNoMemAndBlocked( require.ErrorIs(suite.T(), err, cerrors.ErrFlowControllerAborted) }() - wrapper, sink := createTableSinkWrapper( - changefeedID, spanz.TableIDToComparableSpan(tableID)) + wrapper, sink := createTableSinkWrapper(changefeedID, span) lowerBoundPos := engine.Position{ StartTs: 0, CommitTs: 1, @@ -416,7 +417,7 @@ func (suite *workerSuite) TestHandleTaskWithSplitTxnAndAbortWhenNoMemAndBlocked( }, lastWritePos) } taskChan <- &sinkTask{ - tableID: tableID, + span: span, lowerBound: lowerBoundPos, getUpperBound: upperBoundGetter, tableSink: wrapper, @@ -436,7 +437,7 @@ func (suite *workerSuite) TestHandleTaskWithSplitTxnAndAbortWhenNoMemAndBlocked( // Test the case that worker will advance the table sink only when it reaches the batch size. func (suite *workerSuite) TestHandleTaskWithSplitTxnAndOnlyAdvanceTableSinkWhenReachOneBatchSize() { changefeedID := model.DefaultChangeFeedID("1") - tableID := model.TableID(1) + span := spanz.TableIDToComparableSpan(1) ctx, cancel := context.WithCancel(context.Background()) // For five events. @@ -452,7 +453,7 @@ func (suite *workerSuite) TestHandleTaskWithSplitTxnAndOnlyAdvanceTableSinkWhenR StartTs: 1, CRTs: 2, }, - Row: genRowChangedEvent(1, 2, tableID), + Row: genRowChangedEvent(1, 2, span), }, { StartTs: 1, @@ -462,7 +463,7 @@ func (suite *workerSuite) TestHandleTaskWithSplitTxnAndOnlyAdvanceTableSinkWhenR StartTs: 1, CRTs: 2, }, - Row: genRowChangedEvent(1, 2, tableID), + Row: genRowChangedEvent(1, 2, span), }, { StartTs: 1, @@ -472,7 +473,7 @@ func (suite *workerSuite) TestHandleTaskWithSplitTxnAndOnlyAdvanceTableSinkWhenR StartTs: 1, CRTs: 2, }, - Row: genRowChangedEvent(1, 2, tableID), + Row: genRowChangedEvent(1, 2, span), }, { StartTs: 1, @@ -482,7 +483,7 @@ func (suite *workerSuite) TestHandleTaskWithSplitTxnAndOnlyAdvanceTableSinkWhenR StartTs: 1, CRTs: 2, }, - Row: genRowChangedEvent(1, 2, tableID), + Row: genRowChangedEvent(1, 2, span), }, { StartTs: 1, @@ -492,7 +493,7 @@ func (suite *workerSuite) TestHandleTaskWithSplitTxnAndOnlyAdvanceTableSinkWhenR StartTs: 1, CRTs: 2, }, - Row: genRowChangedEvent(1, 2, tableID), + Row: genRowChangedEvent(1, 2, span), }, { StartTs: 1, @@ -502,7 +503,7 @@ func (suite *workerSuite) TestHandleTaskWithSplitTxnAndOnlyAdvanceTableSinkWhenR StartTs: 1, CRTs: 3, }, - Row: genRowChangedEvent(1, 3, tableID), + Row: genRowChangedEvent(1, 3, span), }, { CRTs: 4, @@ -512,8 +513,8 @@ func (suite *workerSuite) TestHandleTaskWithSplitTxnAndOnlyAdvanceTableSinkWhenR }, }, } - w, e := createWorker(changefeedID, eventSize, true, tableID) - addEventsToSortEngine(suite.T(), events, e, tableID) + w, e := createWorker(changefeedID, eventSize, true, span) + addEventsToSortEngine(suite.T(), events, e, span) taskChan := make(chan *sinkTask) var wg sync.WaitGroup @@ -524,8 +525,7 @@ func (suite *workerSuite) TestHandleTaskWithSplitTxnAndOnlyAdvanceTableSinkWhenR require.ErrorIs(suite.T(), err, context.Canceled) }() - wrapper, sink := createTableSinkWrapper( - changefeedID, spanz.TableIDToComparableSpan(tableID)) + wrapper, sink := createTableSinkWrapper(changefeedID, span) lowerBoundPos := engine.Position{ StartTs: 0, CommitTs: 1, @@ -548,7 +548,7 @@ func (suite *workerSuite) TestHandleTaskWithSplitTxnAndOnlyAdvanceTableSinkWhenR cancel() } taskChan <- &sinkTask{ - tableID: tableID, + span: span, lowerBound: lowerBoundPos, getUpperBound: upperBoundGetter, tableSink: wrapper, @@ -563,7 +563,7 @@ func (suite *workerSuite) TestHandleTaskWithSplitTxnAndOnlyAdvanceTableSinkWhenR // Test the case that the worker will force consume only one Txn when the memory quota is not enough. func (suite *workerSuite) TestHandleTaskWithoutSplitTxnAndAbortWhenNoMemAndForceConsume() { changefeedID := model.DefaultChangeFeedID("1") - tableID := model.TableID(1) + span := spanz.TableIDToComparableSpan(1) ctx, cancel := context.WithCancel(context.Background()) // Only for three events. @@ -579,7 +579,7 @@ func (suite *workerSuite) TestHandleTaskWithoutSplitTxnAndAbortWhenNoMemAndForce StartTs: 1, CRTs: 2, }, - Row: genRowChangedEvent(1, 2, tableID), + Row: genRowChangedEvent(1, 2, span), }, { StartTs: 1, @@ -589,7 +589,7 @@ func (suite *workerSuite) TestHandleTaskWithoutSplitTxnAndAbortWhenNoMemAndForce StartTs: 1, CRTs: 2, }, - Row: genRowChangedEvent(1, 2, tableID), + Row: genRowChangedEvent(1, 2, span), }, { StartTs: 1, @@ -599,7 +599,7 @@ func (suite *workerSuite) TestHandleTaskWithoutSplitTxnAndAbortWhenNoMemAndForce StartTs: 1, CRTs: 2, }, - Row: genRowChangedEvent(1, 2, tableID), + Row: genRowChangedEvent(1, 2, span), }, { StartTs: 1, @@ -609,7 +609,7 @@ func (suite *workerSuite) TestHandleTaskWithoutSplitTxnAndAbortWhenNoMemAndForce StartTs: 1, CRTs: 2, }, - Row: genRowChangedEvent(1, 2, tableID), + Row: genRowChangedEvent(1, 2, span), }, { StartTs: 1, @@ -619,7 +619,7 @@ func (suite *workerSuite) TestHandleTaskWithoutSplitTxnAndAbortWhenNoMemAndForce StartTs: 1, CRTs: 2, }, - Row: genRowChangedEvent(1, 2, tableID), + Row: genRowChangedEvent(1, 2, span), }, { StartTs: 1, @@ -629,7 +629,7 @@ func (suite *workerSuite) TestHandleTaskWithoutSplitTxnAndAbortWhenNoMemAndForce StartTs: 1, CRTs: 4, }, - Row: genRowChangedEvent(1, 4, tableID), + Row: genRowChangedEvent(1, 4, span), }, { CRTs: 5, @@ -639,9 +639,9 @@ func (suite *workerSuite) TestHandleTaskWithoutSplitTxnAndAbortWhenNoMemAndForce }, }, } - w, e := createWorker(changefeedID, eventSize, false, tableID) + w, e := createWorker(changefeedID, eventSize, false, span) w.splitTxn = false - addEventsToSortEngine(suite.T(), events, e, tableID) + addEventsToSortEngine(suite.T(), events, e, span) taskChan := make(chan *sinkTask) var wg sync.WaitGroup @@ -652,8 +652,7 @@ func (suite *workerSuite) TestHandleTaskWithoutSplitTxnAndAbortWhenNoMemAndForce require.Equal(suite.T(), context.Canceled, err) }() - wrapper, sink := createTableSinkWrapper( - changefeedID, spanz.TableIDToComparableSpan(tableID)) + wrapper, sink := createTableSinkWrapper(changefeedID, span) lowerBoundPos := engine.Position{ StartTs: 0, CommitTs: 1, @@ -676,7 +675,7 @@ func (suite *workerSuite) TestHandleTaskWithoutSplitTxnAndAbortWhenNoMemAndForce cancel() } taskChan <- &sinkTask{ - tableID: tableID, + span: span, lowerBound: lowerBoundPos, getUpperBound: upperBoundGetter, tableSink: wrapper, @@ -692,7 +691,7 @@ func (suite *workerSuite) TestHandleTaskWithoutSplitTxnOnlyAdvanceTableSinkWhenR suite.T().Skip("need to be fixed") changefeedID := model.DefaultChangeFeedID("1") - tableID := model.TableID(1) + span := spanz.TableIDToComparableSpan(1) ctx, cancel := context.WithCancel(context.Background()) // Only for three events. @@ -708,7 +707,7 @@ func (suite *workerSuite) TestHandleTaskWithoutSplitTxnOnlyAdvanceTableSinkWhenR StartTs: 1, CRTs: 1, }, - Row: genRowChangedEvent(1, 1, tableID), + Row: genRowChangedEvent(1, 1, span), }, { StartTs: 1, @@ -718,7 +717,7 @@ func (suite *workerSuite) TestHandleTaskWithoutSplitTxnOnlyAdvanceTableSinkWhenR StartTs: 1, CRTs: 2, }, - Row: genRowChangedEvent(1, 2, tableID), + Row: genRowChangedEvent(1, 2, span), }, { StartTs: 1, @@ -728,7 +727,7 @@ func (suite *workerSuite) TestHandleTaskWithoutSplitTxnOnlyAdvanceTableSinkWhenR StartTs: 1, CRTs: 3, }, - Row: genRowChangedEvent(1, 3, tableID), + Row: genRowChangedEvent(1, 3, span), }, { StartTs: 1, @@ -738,7 +737,7 @@ func (suite *workerSuite) TestHandleTaskWithoutSplitTxnOnlyAdvanceTableSinkWhenR StartTs: 1, CRTs: 3, }, - Row: genRowChangedEvent(1, 3, tableID), + Row: genRowChangedEvent(1, 3, span), }, { StartTs: 1, @@ -748,7 +747,7 @@ func (suite *workerSuite) TestHandleTaskWithoutSplitTxnOnlyAdvanceTableSinkWhenR StartTs: 1, CRTs: 3, }, - Row: genRowChangedEvent(1, 3, tableID), + Row: genRowChangedEvent(1, 3, span), }, { StartTs: 1, @@ -758,7 +757,7 @@ func (suite *workerSuite) TestHandleTaskWithoutSplitTxnOnlyAdvanceTableSinkWhenR StartTs: 1, CRTs: 4, }, - Row: genRowChangedEvent(1, 4, tableID), + Row: genRowChangedEvent(1, 4, span), }, { CRTs: 5, @@ -769,7 +768,7 @@ func (suite *workerSuite) TestHandleTaskWithoutSplitTxnOnlyAdvanceTableSinkWhenR }, } w, e := createWorker(changefeedID, eventSize, false) - addEventsToSortEngine(suite.T(), events, e, tableID) + addEventsToSortEngine(suite.T(), events, e, span) taskChan := make(chan *sinkTask) var wg sync.WaitGroup @@ -780,8 +779,7 @@ func (suite *workerSuite) TestHandleTaskWithoutSplitTxnOnlyAdvanceTableSinkWhenR require.Equal(suite.T(), context.Canceled, err) }() - wrapper, sink := createTableSinkWrapper( - changefeedID, spanz.TableIDToComparableSpan(tableID)) + wrapper, sink := createTableSinkWrapper(changefeedID, span) lowerBoundPos := engine.Position{ StartTs: 0, CommitTs: 1, @@ -804,7 +802,7 @@ func (suite *workerSuite) TestHandleTaskWithoutSplitTxnOnlyAdvanceTableSinkWhenR cancel() } taskChan <- &sinkTask{ - tableID: tableID, + span: span, lowerBound: lowerBoundPos, getUpperBound: upperBoundGetter, tableSink: wrapper, @@ -819,7 +817,7 @@ func (suite *workerSuite) TestHandleTaskWithoutSplitTxnOnlyAdvanceTableSinkWhenR // Test the case that the worker will advance the table sink only when meet the new commit ts. func (suite *workerSuite) TestHandleTaskWithSplitTxnAndDoNotAdvanceTableUntilMeetNewCommitTs() { changefeedID := model.DefaultChangeFeedID("1") - tableID := model.TableID(1) + span := spanz.TableIDToComparableSpan(1) ctx, cancel := context.WithCancel(context.Background()) // Only for three events. @@ -835,7 +833,7 @@ func (suite *workerSuite) TestHandleTaskWithSplitTxnAndDoNotAdvanceTableUntilMee StartTs: 1, CRTs: 1, }, - Row: genRowChangedEvent(1, 1, tableID), + Row: genRowChangedEvent(1, 1, span), }, // Although the commit ts is 2, the event is not sent to sink because the commit ts is not changed. { @@ -846,7 +844,7 @@ func (suite *workerSuite) TestHandleTaskWithSplitTxnAndDoNotAdvanceTableUntilMee StartTs: 1, CRTs: 2, }, - Row: genRowChangedEvent(1, 2, tableID), + Row: genRowChangedEvent(1, 2, span), }, { StartTs: 1, @@ -856,7 +854,7 @@ func (suite *workerSuite) TestHandleTaskWithSplitTxnAndDoNotAdvanceTableUntilMee StartTs: 1, CRTs: 2, }, - Row: genRowChangedEvent(1, 2, tableID), + Row: genRowChangedEvent(1, 2, span), }, // We will block at this event. { @@ -867,7 +865,7 @@ func (suite *workerSuite) TestHandleTaskWithSplitTxnAndDoNotAdvanceTableUntilMee StartTs: 1, CRTs: 3, }, - Row: genRowChangedEvent(1, 2, tableID), + Row: genRowChangedEvent(1, 2, span), }, { StartTs: 1, @@ -877,7 +875,7 @@ func (suite *workerSuite) TestHandleTaskWithSplitTxnAndDoNotAdvanceTableUntilMee StartTs: 1, CRTs: 3, }, - Row: genRowChangedEvent(1, 3, tableID), + Row: genRowChangedEvent(1, 3, span), }, { CRTs: 4, @@ -887,8 +885,8 @@ func (suite *workerSuite) TestHandleTaskWithSplitTxnAndDoNotAdvanceTableUntilMee }, }, } - w, e := createWorker(changefeedID, eventSize, true, tableID) - addEventsToSortEngine(suite.T(), events, e, tableID) + w, e := createWorker(changefeedID, eventSize, true, span) + addEventsToSortEngine(suite.T(), events, e, span) taskChan := make(chan *sinkTask) var wg sync.WaitGroup @@ -899,8 +897,7 @@ func (suite *workerSuite) TestHandleTaskWithSplitTxnAndDoNotAdvanceTableUntilMee require.ErrorIs(suite.T(), err, context.Canceled) }() - wrapper, sink := createTableSinkWrapper( - changefeedID, spanz.TableIDToComparableSpan(tableID)) + wrapper, sink := createTableSinkWrapper(changefeedID, span) lowerBoundPos := engine.Position{ StartTs: 0, CommitTs: 1, @@ -922,7 +919,7 @@ func (suite *workerSuite) TestHandleTaskWithSplitTxnAndDoNotAdvanceTableUntilMee }, lastWritePos.Next()) } taskChan <- &sinkTask{ - tableID: tableID, + span: span, lowerBound: lowerBoundPos, getUpperBound: upperBoundGetter, tableSink: wrapper, @@ -946,7 +943,7 @@ func (suite *workerSuite) TestHandleTaskWithSplitTxnAndDoNotAdvanceTableUntilMee // Test the case that the worker will advance the table sink only when task is finished. func (suite *workerSuite) TestHandleTaskWithSplitTxnAndAdvanceTableUntilTaskIsFinished() { changefeedID := model.DefaultChangeFeedID("1") - tableID := model.TableID(1) + span := spanz.TableIDToComparableSpan(1) ctx, cancel := context.WithCancel(context.Background()) // Only for three events. @@ -962,7 +959,7 @@ func (suite *workerSuite) TestHandleTaskWithSplitTxnAndAdvanceTableUntilTaskIsFi StartTs: 1, CRTs: 1, }, - Row: genRowChangedEvent(1, 1, tableID), + Row: genRowChangedEvent(1, 1, span), }, { CRTs: 4, @@ -972,8 +969,8 @@ func (suite *workerSuite) TestHandleTaskWithSplitTxnAndAdvanceTableUntilTaskIsFi }, }, } - w, e := createWorker(changefeedID, eventSize, true, tableID) - addEventsToSortEngine(suite.T(), events, e, tableID) + w, e := createWorker(changefeedID, eventSize, true, span) + addEventsToSortEngine(suite.T(), events, e, span) taskChan := make(chan *sinkTask) var wg sync.WaitGroup @@ -984,8 +981,7 @@ func (suite *workerSuite) TestHandleTaskWithSplitTxnAndAdvanceTableUntilTaskIsFi require.ErrorIs(suite.T(), err, context.Canceled) }() - wrapper, sink := createTableSinkWrapper( - changefeedID, spanz.TableIDToComparableSpan(tableID)) + wrapper, sink := createTableSinkWrapper(changefeedID, span) lowerBoundPos := engine.Position{ StartTs: 0, CommitTs: 1, @@ -1007,7 +1003,7 @@ func (suite *workerSuite) TestHandleTaskWithSplitTxnAndAdvanceTableUntilTaskIsFi }, lastWritePos.Next()) } taskChan <- &sinkTask{ - tableID: tableID, + span: span, lowerBound: lowerBoundPos, getUpperBound: upperBoundGetter, tableSink: wrapper, @@ -1028,7 +1024,7 @@ func (suite *workerSuite) TestHandleTaskWithSplitTxnAndAdvanceTableUntilTaskIsFi // Test the case that the worker will advance the table sink directly when there are no events. func (suite *workerSuite) TestHandleTaskWithSplitTxnAndAdvanceTableIfNoWorkload() { changefeedID := model.DefaultChangeFeedID("1") - tableID := model.TableID(1) + span := spanz.TableIDToComparableSpan(1) ctx, cancel := context.WithCancel(context.Background()) // Only for three events. @@ -1044,8 +1040,8 @@ func (suite *workerSuite) TestHandleTaskWithSplitTxnAndAdvanceTableIfNoWorkload( }, }, } - w, e := createWorker(changefeedID, eventSize, true, tableID) - addEventsToSortEngine(suite.T(), events, e, tableID) + w, e := createWorker(changefeedID, eventSize, true, span) + addEventsToSortEngine(suite.T(), events, e, span) taskChan := make(chan *sinkTask) var wg sync.WaitGroup @@ -1056,8 +1052,7 @@ func (suite *workerSuite) TestHandleTaskWithSplitTxnAndAdvanceTableIfNoWorkload( require.ErrorIs(suite.T(), err, context.Canceled) }() - wrapper, _ := createTableSinkWrapper( - changefeedID, spanz.TableIDToComparableSpan(tableID)) + wrapper, _ := createTableSinkWrapper(changefeedID, span) lowerBoundPos := engine.Position{ StartTs: 0, CommitTs: 1, @@ -1079,7 +1074,7 @@ func (suite *workerSuite) TestHandleTaskWithSplitTxnAndAdvanceTableIfNoWorkload( }, lastWritePos.Next()) } taskChan <- &sinkTask{ - tableID: tableID, + span: span, lowerBound: lowerBoundPos, getUpperBound: upperBoundGetter, tableSink: wrapper, diff --git a/cdc/processor/sinkmanager/table_sink_wrapper.go b/cdc/processor/sinkmanager/table_sink_wrapper.go index d601646b6c7..0a0e8b612e5 100644 --- a/cdc/processor/sinkmanager/table_sink_wrapper.go +++ b/cdc/processor/sinkmanager/table_sink_wrapper.go @@ -42,7 +42,7 @@ type tableSinkWrapper struct { // changefeed used for logging. changefeed model.ChangeFeedID // tableID used for logging. - tableID model.TableID + span tablepb.Span // tableSink is the underlying sink. tableSink sinkv2.TableSink // state used to control the lifecycle of the table. @@ -90,7 +90,7 @@ func newRangeEventCount(pos engine.Position, events int) rangeEventCount { func newTableSinkWrapper( changefeed model.ChangeFeedID, - tableID model.TableID, + span tablepb.Span, tableSink sinkv2.TableSink, state tablepb.TableState, startTs model.Ts, @@ -99,7 +99,7 @@ func newTableSinkWrapper( res := &tableSinkWrapper{ version: atomic.AddUint64(&version, 1), changefeed: changefeed, - tableID: tableID, + span: span, tableSink: tableSink, state: &state, startTs: startTs, @@ -115,7 +115,7 @@ func (t *tableSinkWrapper) start(startTs model.Ts, replicateTs model.Ts) { log.Panic("The table sink has already started", zap.String("namespace", t.changefeed.Namespace), zap.String("changefeed", t.changefeed.ID), - zap.Int64("tableID", t.tableID), + zap.Stringer("span", &t.span), zap.Uint64("startTs", startTs), zap.Uint64("replicateTs", replicateTs), zap.Uint64("oldReplicateTs", t.replicateTs), @@ -124,7 +124,7 @@ func (t *tableSinkWrapper) start(startTs model.Ts, replicateTs model.Ts) { log.Info("Sink is started", zap.String("namespace", t.changefeed.Namespace), zap.String("changefeed", t.changefeed.ID), - zap.Int64("tableID", t.tableID), + zap.Stringer("span", &t.span), zap.Uint64("startTs", startTs), zap.Uint64("replicateTs", replicateTs), ) @@ -205,7 +205,7 @@ func (t *tableSinkWrapper) close(ctx context.Context) { defer t.state.Store(tablepb.TableStateStopped) t.tableSink.Close(ctx) log.Info("Sink is closed", - zap.Int64("tableID", t.tableID), + zap.Stringer("span", &t.span), zap.String("namespace", t.changefeed.Namespace), zap.String("changefeed", t.changefeed.ID)) } @@ -264,7 +264,10 @@ func (t *tableSinkWrapper) cleanRangeEventCounts(upperBound engine.Position, min // convertRowChangedEvents uses to convert RowChangedEvents to TableSinkRowChangedEvents. // It will deal with the old value compatibility. -func convertRowChangedEvents(changefeed model.ChangeFeedID, tableID model.TableID, enableOldValue bool, events ...*model.PolymorphicEvent) ([]*model.RowChangedEvent, uint64, error) { +func convertRowChangedEvents( + changefeed model.ChangeFeedID, span tablepb.Span, enableOldValue bool, + events ...*model.PolymorphicEvent, +) ([]*model.RowChangedEvent, uint64, error) { size := 0 rowChangedEvents := make([]*model.RowChangedEvent, 0, len(events)) for _, e := range events { @@ -272,7 +275,7 @@ func convertRowChangedEvents(changefeed model.ChangeFeedID, tableID model.TableI log.Warn("skip emit nil event", zap.String("namespace", changefeed.Namespace), zap.String("changefeed", changefeed.ID), - zap.Int64("tableID", tableID), + zap.Stringer("span", &span), zap.Any("event", e)) continue } @@ -284,7 +287,7 @@ func convertRowChangedEvents(changefeed model.ChangeFeedID, tableID model.TableI // Just ignore these row changed events. if colLen == 0 && preColLen == 0 { log.Warn("skip emit empty row event", - zap.Int64("tableID", tableID), + zap.Stringer("span", &span), zap.String("namespace", changefeed.Namespace), zap.String("changefeed", changefeed.ID), zap.Any("event", e)) diff --git a/cdc/processor/sinkmanager/table_sink_wrapper_test.go b/cdc/processor/sinkmanager/table_sink_wrapper_test.go index 0662c310c87..3a05d0cec3c 100644 --- a/cdc/processor/sinkmanager/table_sink_wrapper_test.go +++ b/cdc/processor/sinkmanager/table_sink_wrapper_test.go @@ -74,7 +74,7 @@ func createTableSinkWrapper( sink, &eventsink.RowChangeEventAppender{}, prometheus.NewCounter(prometheus.CounterOpts{})) wrapper := newTableSinkWrapper( changefeedID, - span.TableID, + span, innerTableSink, tableState, 0, @@ -108,9 +108,9 @@ func TestConvertNilRowChangedEvents(t *testing.T) { events := []*model.PolymorphicEvent{nil} changefeedID := model.DefaultChangeFeedID("1") - tableID := model.TableID(1) + span := spanz.TableIDToComparableSpan(1) enableOldVlaue := false - result, size, err := convertRowChangedEvents(changefeedID, tableID, enableOldVlaue, events...) + result, size, err := convertRowChangedEvents(changefeedID, span, enableOldVlaue, events...) require.NoError(t, err) require.Equal(t, 0, len(result)) require.Equal(t, uint64(0), size) @@ -130,9 +130,9 @@ func TestConvertEmptyRowChangedEvents(t *testing.T) { }, } changefeedID := model.DefaultChangeFeedID("1") - tableID := model.TableID(1) + span := spanz.TableIDToComparableSpan(1) enableOldValue := false - result, size, err := convertRowChangedEvents(changefeedID, tableID, enableOldValue, events...) + result, size, err := convertRowChangedEvents(changefeedID, span, enableOldValue, events...) require.NoError(t, err) require.Equal(t, 0, len(result)) require.Equal(t, uint64(0), size) @@ -182,9 +182,9 @@ func TestConvertRowChangedEventsWhenEnableOldValue(t *testing.T) { }, } changefeedID := model.DefaultChangeFeedID("1") - tableID := model.TableID(1) + span := spanz.TableIDToComparableSpan(1) enableOldValue := true - result, size, err := convertRowChangedEvents(changefeedID, tableID, enableOldValue, events...) + result, size, err := convertRowChangedEvents(changefeedID, span, enableOldValue, events...) require.NoError(t, err) require.Equal(t, 1, len(result)) require.Equal(t, uint64(216), size) @@ -235,9 +235,9 @@ func TestConvertRowChangedEventsWhenDisableOldValue(t *testing.T) { }, } changefeedID := model.DefaultChangeFeedID("1") - tableID := model.TableID(1) + span := spanz.TableIDToComparableSpan(1) enableOldValue := false - result, size, err := convertRowChangedEvents(changefeedID, tableID, enableOldValue, events...) + result, size, err := convertRowChangedEvents(changefeedID, span, enableOldValue, events...) require.NoError(t, err) require.Equal(t, 2, len(result)) require.Equal(t, uint64(216), size) @@ -283,7 +283,7 @@ func TestConvertRowChangedEventsWhenDisableOldValue(t *testing.T) { }, }, } - result, size, err = convertRowChangedEvents(changefeedID, tableID, enableOldValue, events...) + result, size, err = convertRowChangedEvents(changefeedID, span, enableOldValue, events...) require.NoError(t, err) require.Equal(t, 1, len(result)) require.Equal(t, uint64(216), size) diff --git a/cdc/processor/sinkmanager/tasks.go b/cdc/processor/sinkmanager/tasks.go index a2966c7a6ed..c1e123d2b8f 100644 --- a/cdc/processor/sinkmanager/tasks.go +++ b/cdc/processor/sinkmanager/tasks.go @@ -14,8 +14,8 @@ package sinkmanager import ( - "github.com/pingcap/tiflow/cdc/model" "github.com/pingcap/tiflow/cdc/processor/sourcemanager/engine" + "github.com/pingcap/tiflow/cdc/processor/tablepb" ) const ( @@ -43,7 +43,7 @@ type isCanceled func() bool // sinkTask is a task for a table sink. // It only considers how to control the table sink. type sinkTask struct { - tableID model.TableID + span tablepb.Span // lowerBound indicates the lower bound of the task. // It is a closed interval. lowerBound engine.Position @@ -58,7 +58,7 @@ type sinkTask struct { // redoTask is a task for the redo log. type redoTask struct { - tableID model.TableID + span tablepb.Span lowerBound engine.Position getUpperBound upperBoundGetter tableSink *tableSinkWrapper diff --git a/cdc/processor/sourcemanager/engine/engine.go b/cdc/processor/sourcemanager/engine/engine.go index 44ae671e01a..a875dcdf612 100644 --- a/cdc/processor/sourcemanager/engine/engine.go +++ b/cdc/processor/sourcemanager/engine/engine.go @@ -15,6 +15,7 @@ package engine import ( "github.com/pingcap/tiflow/cdc/model" + "github.com/pingcap/tiflow/cdc/processor/tablepb" ) // SortEngine is a storage engine to store and sort CDC events. @@ -26,29 +27,29 @@ type SortEngine interface { IsTableBased() bool // AddTable adds the table into the engine. - AddTable(tableID model.TableID) + AddTable(span tablepb.Span) // RemoveTable removes the table from the engine. - RemoveTable(tableID model.TableID) + RemoveTable(span tablepb.Span) // Add adds the given events into the sort engine. // // NOTE: it's an asynchronous interface. To get the notification of when // events are available for fetching, OnResolve is what you want. - Add(tableID model.TableID, events ...*model.PolymorphicEvent) error + Add(span tablepb.Span, events ...*model.PolymorphicEvent) error // GetResolvedTs gets resolved timestamp of the given table. - GetResolvedTs(tableID model.TableID) model.Ts + GetResolvedTs(span tablepb.Span) model.Ts // OnResolve pushes action into SortEngine's hook list, which // will be called after any events are resolved. - OnResolve(action func(model.TableID, model.Ts)) + OnResolve(action func(tablepb.Span, model.Ts)) // FetchByTable creates an iterator to fetch events from the given table. // lowerBound is inclusive and only resolved events can be retrieved. // // NOTE: FetchByTable is always available even if IsTableBased returns false. - FetchByTable(tableID model.TableID, lowerBound, upperBound Position) EventIterator + FetchByTable(span tablepb.Span, lowerBound, upperBound Position) EventIterator // FetchAllTables creates an iterator to fetch events from all tables. // lowerBound is inclusive and only resolved events can be retrieved. @@ -61,7 +62,7 @@ type SortEngine interface { // The SortEngine instance can GC them later. // // NOTE: CleanByTable is always available even if IsTableBased returns false. - CleanByTable(tableID model.TableID, upperBound Position) error + CleanByTable(span tablepb.Span, upperBound Position) error // CleanAllTables tells the engine events of all tables in the given range // (unlimited, upperBound] are committed and not necessary any more. @@ -71,7 +72,7 @@ type SortEngine interface { CleanAllTables(upperBound Position) error // GetStatsByTable gets the statistics of the given table. - GetStatsByTable(tableID model.TableID) TableStats + GetStatsByTable(span tablepb.Span) TableStats // ReceivedEvents returns the number of events received by the sort engine. ReceivedEvents() int64 diff --git a/cdc/processor/sourcemanager/engine/memory/event_sorter.go b/cdc/processor/sourcemanager/engine/memory/event_sorter.go index e76af1729d3..d6a8f270348 100644 --- a/cdc/processor/sourcemanager/engine/memory/event_sorter.go +++ b/cdc/processor/sourcemanager/engine/memory/event_sorter.go @@ -22,6 +22,8 @@ import ( "github.com/pingcap/log" "github.com/pingcap/tiflow/cdc/model" "github.com/pingcap/tiflow/cdc/processor/sourcemanager/engine" + "github.com/pingcap/tiflow/cdc/processor/tablepb" + "github.com/pingcap/tiflow/pkg/spanz" "go.uber.org/zap" ) @@ -32,11 +34,11 @@ var ( // EventSorter accepts out-of-order raw kv entries and output sorted entries. type EventSorter struct { - // Just like map[model.TableID]*tableSorter. - tables sync.Map + // Just like map[tablepb.Span]*tableSorter. + tables spanz.SyncMap mu sync.RWMutex - onResolves []func(model.TableID, model.Ts) + onResolves []func(tablepb.Span, model.Ts) } // EventIter implements sorter.EventIterator. @@ -56,24 +58,24 @@ func (s *EventSorter) IsTableBased() bool { } // AddTable implements engine.SortEngine. -func (s *EventSorter) AddTable(tableID model.TableID) { - if _, exists := s.tables.LoadOrStore(tableID, &tableSorter{}); exists { - log.Panic("add an exist table", zap.Int64("tableID", tableID)) +func (s *EventSorter) AddTable(span tablepb.Span) { + if _, exists := s.tables.LoadOrStore(span, &tableSorter{}); exists { + log.Panic("add an exist table", zap.Stringer("span", &span)) } } // RemoveTable implements engine.SortEngine. -func (s *EventSorter) RemoveTable(tableID model.TableID) { - if _, exists := s.tables.LoadAndDelete(tableID); !exists { - log.Panic("remove an unexist table", zap.Int64("tableID", tableID)) +func (s *EventSorter) RemoveTable(span tablepb.Span) { + if _, exists := s.tables.LoadAndDelete(span); !exists { + log.Panic("remove an unexist table", zap.Stringer("span", &span)) } } // Add implements engine.SortEngine. -func (s *EventSorter) Add(tableID model.TableID, events ...*model.PolymorphicEvent) (err error) { - value, exists := s.tables.Load(tableID) +func (s *EventSorter) Add(span tablepb.Span, events ...*model.PolymorphicEvent) (err error) { + value, exists := s.tables.Load(span) if !exists { - log.Panic("add events into an unexist table", zap.Int64("tableID", tableID)) + log.Panic("add events into an unexist table", zap.Stringer("span", &span)) } resolvedTs, hasNewResolved := value.(*tableSorter).add(events...) @@ -81,37 +83,39 @@ func (s *EventSorter) Add(tableID model.TableID, events ...*model.PolymorphicEve s.mu.RLock() defer s.mu.RUnlock() for _, onResolve := range s.onResolves { - onResolve(tableID, resolvedTs) + onResolve(span, resolvedTs) } } return nil } // GetResolvedTs implements engine.SortEngine. -func (s *EventSorter) GetResolvedTs(tableID model.TableID) model.Ts { - value, exists := s.tables.Load(tableID) +func (s *EventSorter) GetResolvedTs(span tablepb.Span) model.Ts { + value, exists := s.tables.Load(span) if !exists { - log.Panic("get resolved ts from an unexist table", zap.Int64("tableID", tableID)) + log.Panic("get resolved ts from an unexist table", zap.Stringer("span", &span)) } return value.(*tableSorter).getResolvedTs() } // OnResolve implements engine.SortEngine. -func (s *EventSorter) OnResolve(action func(model.TableID, model.Ts)) { +func (s *EventSorter) OnResolve(action func(tablepb.Span, model.Ts)) { s.mu.Lock() defer s.mu.Unlock() s.onResolves = append(s.onResolves, action) } // FetchByTable implements engine.SortEngine. -func (s *EventSorter) FetchByTable(tableID model.TableID, lowerBound, upperBound engine.Position) engine.EventIterator { - value, exists := s.tables.Load(tableID) +func (s *EventSorter) FetchByTable( + span tablepb.Span, lowerBound, upperBound engine.Position, +) engine.EventIterator { + value, exists := s.tables.Load(span) if !exists { - log.Panic("fetch events from an unexist table", zap.Int64("tableID", tableID)) + log.Panic("fetch events from an unexist table", zap.Stringer("span", &span)) } - return value.(*tableSorter).fetch(tableID, lowerBound, upperBound) + return value.(*tableSorter).fetch(span, lowerBound, upperBound) } // FetchAllTables implements engine.SortEngine. @@ -121,13 +125,13 @@ func (s *EventSorter) FetchAllTables(lowerBound engine.Position) engine.EventIte } // CleanByTable implements engine.SortEngine. -func (s *EventSorter) CleanByTable(tableID model.TableID, upperBound engine.Position) error { - value, exists := s.tables.Load(tableID) +func (s *EventSorter) CleanByTable(span tablepb.Span, upperBound engine.Position) error { + value, exists := s.tables.Load(span) if !exists { - log.Panic("clean an unexist table", zap.Int64("tableID", tableID)) + log.Panic("clean an unexist table", zap.Stringer("span", &span)) } - value.(*tableSorter).clean(tableID, upperBound) + value.(*tableSorter).clean(span, upperBound) return nil } @@ -138,7 +142,7 @@ func (s *EventSorter) CleanAllTables(upperBound engine.Position) error { } // GetStatsByTable implements engine.SortEngine. -func (s *EventSorter) GetStatsByTable(tableID model.TableID) engine.TableStats { +func (s *EventSorter) GetStatsByTable(span tablepb.Span) engine.TableStats { log.Panic("GetStatsByTable should never be called") return engine.TableStats{} } @@ -151,7 +155,7 @@ func (s *EventSorter) ReceivedEvents() int64 { // Close implements engine.SortEngine. func (s *EventSorter) Close() error { - s.tables = sync.Map{} + s.tables = spanz.SyncMap{} return nil } @@ -228,13 +232,15 @@ func (s *tableSorter) getResolvedTs() model.Ts { return *s.resolvedTs } -func (s *tableSorter) fetch(tableID model.TableID, lowerBound, upperBound engine.Position) engine.EventIterator { +func (s *tableSorter) fetch( + span tablepb.Span, lowerBound, upperBound engine.Position, +) engine.EventIterator { s.mu.RLock() defer s.mu.RUnlock() iter := &EventIter{} if s.resolvedTs == nil || upperBound.CommitTs > *s.resolvedTs { - log.Panic("fetch unresolved events", zap.Int64("tableID", tableID)) + log.Panic("fetch unresolved events", zap.Stringer("span", &span)) } startIdx := sort.Search(len(s.resolved), func(idx int) bool { @@ -251,11 +257,11 @@ func (s *tableSorter) fetch(tableID model.TableID, lowerBound, upperBound engine return iter } -func (s *tableSorter) clean(tableID model.TableID, upperBound engine.Position) { +func (s *tableSorter) clean(span tablepb.Span, upperBound engine.Position) { s.mu.Lock() defer s.mu.Unlock() if s.resolvedTs == nil || upperBound.CommitTs > *s.resolvedTs { - log.Panic("clean unresolved events", zap.Int64("tableID", tableID)) + log.Panic("clean unresolved events", zap.Stringer("span", &span)) } startIdx := sort.Search(len(s.resolved), func(idx int) bool { diff --git a/cdc/processor/sourcemanager/engine/memory/event_sorter_test.go b/cdc/processor/sourcemanager/engine/memory/event_sorter_test.go index 490b47ad5bc..2c59a4d84c0 100644 --- a/cdc/processor/sourcemanager/engine/memory/event_sorter_test.go +++ b/cdc/processor/sourcemanager/engine/memory/event_sorter_test.go @@ -19,6 +19,7 @@ import ( "github.com/pingcap/tiflow/cdc/model" "github.com/pingcap/tiflow/cdc/processor/sourcemanager/engine" + "github.com/pingcap/tiflow/pkg/spanz" "github.com/stretchr/testify/require" ) @@ -90,15 +91,17 @@ func TestEventSorter(t *testing.T) { }, } + span := spanz.TableIDToComparableSpan(1) es := New(context.Background()) - es.AddTable(1) + es.AddTable(span) var nextToFetch engine.Position for _, tc := range testCases { for _, entry := range tc.input { - es.Add(1, model.NewPolymorphicEvent(entry)) + es.Add(span, model.NewPolymorphicEvent(entry)) } - es.Add(1, model.NewResolvedPolymorphicEvent(0, tc.resolvedTs)) - iter := es.FetchByTable(1, nextToFetch, engine.Position{CommitTs: tc.resolvedTs, StartTs: tc.resolvedTs}) + es.Add(span, model.NewResolvedPolymorphicEvent(0, tc.resolvedTs)) + iter := es.FetchByTable( + span, nextToFetch, engine.Position{CommitTs: tc.resolvedTs, StartTs: tc.resolvedTs}) for _, expect := range tc.expect { event, pos, _ := iter.Next() require.NotNil(t, event) diff --git a/cdc/processor/sourcemanager/engine/mock/engine_mock.go b/cdc/processor/sourcemanager/engine/mock/engine_mock.go index d50e11ed4dd..7982f85b70d 100644 --- a/cdc/processor/sourcemanager/engine/mock/engine_mock.go +++ b/cdc/processor/sourcemanager/engine/mock/engine_mock.go @@ -10,6 +10,7 @@ import ( gomock "github.com/golang/mock/gomock" model "github.com/pingcap/tiflow/cdc/model" engine "github.com/pingcap/tiflow/cdc/processor/sourcemanager/engine" + tablepb "github.com/pingcap/tiflow/cdc/processor/tablepb" ) // MockSortEngine is a mock of SortEngine interface. @@ -36,9 +37,9 @@ func (m *MockSortEngine) EXPECT() *MockSortEngineMockRecorder { } // Add mocks base method. -func (m *MockSortEngine) Add(tableID model.TableID, events ...*model.PolymorphicEvent) error { +func (m *MockSortEngine) Add(span tablepb.Span, events ...*model.PolymorphicEvent) error { m.ctrl.T.Helper() - varargs := []interface{}{tableID} + varargs := []interface{}{span} for _, a := range events { varargs = append(varargs, a) } @@ -48,22 +49,22 @@ func (m *MockSortEngine) Add(tableID model.TableID, events ...*model.Polymorphic } // Add indicates an expected call of Add. -func (mr *MockSortEngineMockRecorder) Add(tableID interface{}, events ...interface{}) *gomock.Call { +func (mr *MockSortEngineMockRecorder) Add(span interface{}, events ...interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() - varargs := append([]interface{}{tableID}, events...) + varargs := append([]interface{}{span}, events...) return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Add", reflect.TypeOf((*MockSortEngine)(nil).Add), varargs...) } // AddTable mocks base method. -func (m *MockSortEngine) AddTable(tableID model.TableID) { +func (m *MockSortEngine) AddTable(span tablepb.Span) { m.ctrl.T.Helper() - m.ctrl.Call(m, "AddTable", tableID) + m.ctrl.Call(m, "AddTable", span) } // AddTable indicates an expected call of AddTable. -func (mr *MockSortEngineMockRecorder) AddTable(tableID interface{}) *gomock.Call { +func (mr *MockSortEngineMockRecorder) AddTable(span interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "AddTable", reflect.TypeOf((*MockSortEngine)(nil).AddTable), tableID) + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "AddTable", reflect.TypeOf((*MockSortEngine)(nil).AddTable), span) } // CleanAllTables mocks base method. @@ -81,17 +82,17 @@ func (mr *MockSortEngineMockRecorder) CleanAllTables(upperBound interface{}) *go } // CleanByTable mocks base method. -func (m *MockSortEngine) CleanByTable(tableID model.TableID, upperBound engine.Position) error { +func (m *MockSortEngine) CleanByTable(span tablepb.Span, upperBound engine.Position) error { m.ctrl.T.Helper() - ret := m.ctrl.Call(m, "CleanByTable", tableID, upperBound) + ret := m.ctrl.Call(m, "CleanByTable", span, upperBound) ret0, _ := ret[0].(error) return ret0 } // CleanByTable indicates an expected call of CleanByTable. -func (mr *MockSortEngineMockRecorder) CleanByTable(tableID, upperBound interface{}) *gomock.Call { +func (mr *MockSortEngineMockRecorder) CleanByTable(span, upperBound interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "CleanByTable", reflect.TypeOf((*MockSortEngine)(nil).CleanByTable), tableID, upperBound) + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "CleanByTable", reflect.TypeOf((*MockSortEngine)(nil).CleanByTable), span, upperBound) } // Close mocks base method. @@ -123,45 +124,45 @@ func (mr *MockSortEngineMockRecorder) FetchAllTables(lowerBound interface{}) *go } // FetchByTable mocks base method. -func (m *MockSortEngine) FetchByTable(tableID model.TableID, lowerBound, upperBound engine.Position) engine.EventIterator { +func (m *MockSortEngine) FetchByTable(span tablepb.Span, lowerBound, upperBound engine.Position) engine.EventIterator { m.ctrl.T.Helper() - ret := m.ctrl.Call(m, "FetchByTable", tableID, lowerBound, upperBound) + ret := m.ctrl.Call(m, "FetchByTable", span, lowerBound, upperBound) ret0, _ := ret[0].(engine.EventIterator) return ret0 } // FetchByTable indicates an expected call of FetchByTable. -func (mr *MockSortEngineMockRecorder) FetchByTable(tableID, lowerBound, upperBound interface{}) *gomock.Call { +func (mr *MockSortEngineMockRecorder) FetchByTable(span, lowerBound, upperBound interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "FetchByTable", reflect.TypeOf((*MockSortEngine)(nil).FetchByTable), tableID, lowerBound, upperBound) + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "FetchByTable", reflect.TypeOf((*MockSortEngine)(nil).FetchByTable), span, lowerBound, upperBound) } // GetResolvedTs mocks base method. -func (m *MockSortEngine) GetResolvedTs(tableID model.TableID) model.Ts { +func (m *MockSortEngine) GetResolvedTs(span tablepb.Span) model.Ts { m.ctrl.T.Helper() - ret := m.ctrl.Call(m, "GetResolvedTs", tableID) + ret := m.ctrl.Call(m, "GetResolvedTs", span) ret0, _ := ret[0].(model.Ts) return ret0 } // GetResolvedTs indicates an expected call of GetResolvedTs. -func (mr *MockSortEngineMockRecorder) GetResolvedTs(tableID interface{}) *gomock.Call { +func (mr *MockSortEngineMockRecorder) GetResolvedTs(span interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "GetResolvedTs", reflect.TypeOf((*MockSortEngine)(nil).GetResolvedTs), tableID) + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "GetResolvedTs", reflect.TypeOf((*MockSortEngine)(nil).GetResolvedTs), span) } // GetStatsByTable mocks base method. -func (m *MockSortEngine) GetStatsByTable(tableID model.TableID) engine.TableStats { +func (m *MockSortEngine) GetStatsByTable(span tablepb.Span) engine.TableStats { m.ctrl.T.Helper() - ret := m.ctrl.Call(m, "GetStatsByTable", tableID) + ret := m.ctrl.Call(m, "GetStatsByTable", span) ret0, _ := ret[0].(engine.TableStats) return ret0 } // GetStatsByTable indicates an expected call of GetStatsByTable. -func (mr *MockSortEngineMockRecorder) GetStatsByTable(tableID interface{}) *gomock.Call { +func (mr *MockSortEngineMockRecorder) GetStatsByTable(span interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "GetStatsByTable", reflect.TypeOf((*MockSortEngine)(nil).GetStatsByTable), tableID) + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "GetStatsByTable", reflect.TypeOf((*MockSortEngine)(nil).GetStatsByTable), span) } // IsTableBased mocks base method. @@ -179,7 +180,7 @@ func (mr *MockSortEngineMockRecorder) IsTableBased() *gomock.Call { } // OnResolve mocks base method. -func (m *MockSortEngine) OnResolve(action func(model.TableID, model.Ts)) { +func (m *MockSortEngine) OnResolve(action func(tablepb.Span, model.Ts)) { m.ctrl.T.Helper() m.ctrl.Call(m, "OnResolve", action) } @@ -205,15 +206,15 @@ func (mr *MockSortEngineMockRecorder) ReceivedEvents() *gomock.Call { } // RemoveTable mocks base method. -func (m *MockSortEngine) RemoveTable(tableID model.TableID) { +func (m *MockSortEngine) RemoveTable(span tablepb.Span) { m.ctrl.T.Helper() - m.ctrl.Call(m, "RemoveTable", tableID) + m.ctrl.Call(m, "RemoveTable", span) } // RemoveTable indicates an expected call of RemoveTable. -func (mr *MockSortEngineMockRecorder) RemoveTable(tableID interface{}) *gomock.Call { +func (mr *MockSortEngineMockRecorder) RemoveTable(span interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "RemoveTable", reflect.TypeOf((*MockSortEngine)(nil).RemoveTable), tableID) + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "RemoveTable", reflect.TypeOf((*MockSortEngine)(nil).RemoveTable), span) } // MockEventIterator is a mock of EventIterator interface. diff --git a/cdc/processor/sourcemanager/engine/pebble/event_sorter.go b/cdc/processor/sourcemanager/engine/pebble/event_sorter.go index 54f033424bd..b9ce502674a 100644 --- a/cdc/processor/sourcemanager/engine/pebble/event_sorter.go +++ b/cdc/processor/sourcemanager/engine/pebble/event_sorter.go @@ -27,8 +27,10 @@ import ( "github.com/pingcap/tiflow/cdc/model" "github.com/pingcap/tiflow/cdc/processor/sourcemanager/engine" "github.com/pingcap/tiflow/cdc/processor/sourcemanager/engine/pebble/encoding" + "github.com/pingcap/tiflow/cdc/processor/tablepb" metrics "github.com/pingcap/tiflow/cdc/sorter/db" "github.com/pingcap/tiflow/pkg/chann" + "github.com/pingcap/tiflow/pkg/spanz" "go.uber.org/zap" ) @@ -41,7 +43,6 @@ var ( type EventSorter struct { // Read-only fields. changefeedID model.ChangeFeedID - uniqueID uint32 dbs []*pebble.DB channs []*chann.Chann[eventWithTableID] serde encoding.MsgPackGenSerde @@ -53,8 +54,8 @@ type EventSorter struct { // Following fields are protected by mu. mu sync.RWMutex isClosed bool - onResolves []func(model.TableID, model.Ts) - tables map[model.TableID]*tableState + onResolves []func(tablepb.Span, model.Ts) + tables *spanz.Map[*tableState] } // EventIter implements sorter.EventIterator. @@ -75,11 +76,10 @@ func New(ID model.ChangeFeedID, dbs []*pebble.DB) *EventSorter { eventSorter := &EventSorter{ changefeedID: ID, - uniqueID: genUniqueID(), dbs: dbs, channs: channs, closed: make(chan struct{}), - tables: make(map[model.TableID]*tableState), + tables: spanz.NewMap[*tableState](), } for i := range eventSorter.dbs { @@ -111,52 +111,55 @@ func (s *EventSorter) IsTableBased() bool { } // AddTable implements engine.SortEngine. -func (s *EventSorter) AddTable(tableID model.TableID) { +func (s *EventSorter) AddTable(span tablepb.Span) { s.mu.Lock() - if _, exists := s.tables[tableID]; exists { + if _, exists := s.tables.Get(span); exists { s.mu.Unlock() log.Warn("add an exist table", zap.String("namespace", s.changefeedID.Namespace), zap.String("changefeed", s.changefeedID.ID), - zap.Int64("tableID", tableID)) + zap.Stringer("span", &span)) return } - s.tables[tableID] = &tableState{ch: s.channs[getDB(tableID, len(s.dbs))]} + s.tables.ReplaceOrInsert(span, &tableState{ + uniqueID: genUniqueID(), + ch: s.channs[getDB(span, len(s.dbs))], + }) s.mu.Unlock() } // RemoveTable implements engine.SortEngine. -func (s *EventSorter) RemoveTable(tableID model.TableID) { +func (s *EventSorter) RemoveTable(span tablepb.Span) { s.mu.Lock() - if _, exists := s.tables[tableID]; !exists { + if _, exists := s.tables.Get(span); !exists { s.mu.Unlock() log.Warn("remove an unexist table", zap.String("namespace", s.changefeedID.Namespace), zap.String("changefeed", s.changefeedID.ID), - zap.Int64("tableID", tableID)) + zap.Stringer("span", &span)) return } - delete(s.tables, tableID) + s.tables.Delete(span) s.mu.Unlock() } // Add implements engine.SortEngine. -func (s *EventSorter) Add(tableID model.TableID, events ...*model.PolymorphicEvent) error { +func (s *EventSorter) Add(span tablepb.Span, events ...*model.PolymorphicEvent) error { s.mu.RLock() - state, exists := s.tables[tableID] + state, exists := s.tables.Get(span) s.mu.RUnlock() if !exists { log.Panic("add events into an non-existent table", zap.String("namespace", s.changefeedID.Namespace), zap.String("changefeed", s.changefeedID.ID), - zap.Int64("tableID", tableID)) + zap.Stringer("span", &span)) } maxCommitTs := model.Ts(0) maxResolvedTs := model.Ts(0) for _, event := range events { - state.ch.In() <- eventWithTableID{tableID, event} + state.ch.In() <- eventWithTableID{uniqueID: state.uniqueID, span: span, event: event} if event.IsResolved() { if event.CRTs > maxResolvedTs { maxResolvedTs = event.CRTs @@ -180,23 +183,23 @@ func (s *EventSorter) Add(tableID model.TableID, events ...*model.PolymorphicEve } // GetResolvedTs implements engine.SortEngine. -func (s *EventSorter) GetResolvedTs(tableID model.TableID) model.Ts { +func (s *EventSorter) GetResolvedTs(span tablepb.Span) model.Ts { s.mu.RLock() - state, exists := s.tables[tableID] + state, exists := s.tables.Get(span) s.mu.RUnlock() if !exists { log.Panic("get resolved ts from an non-existent table", zap.String("namespace", s.changefeedID.Namespace), zap.String("changefeed", s.changefeedID.ID), - zap.Int64("tableID", tableID)) + zap.Stringer("span", &span)) } return state.maxReceivedResolvedTs.Load() } // OnResolve implements engine.SortEngine. -func (s *EventSorter) OnResolve(action func(model.TableID, model.Ts)) { +func (s *EventSorter) OnResolve(action func(tablepb.Span, model.Ts)) { s.mu.Lock() defer s.mu.Unlock() s.onResolves = append(s.onResolves, action) @@ -204,18 +207,18 @@ func (s *EventSorter) OnResolve(action func(model.TableID, model.Ts)) { // FetchByTable implements engine.SortEngine. func (s *EventSorter) FetchByTable( - tableID model.TableID, + span tablepb.Span, lowerBound, upperBound engine.Position, ) engine.EventIterator { s.mu.RLock() - state, exists := s.tables[tableID] + state, exists := s.tables.Get(span) s.mu.RUnlock() if !exists { log.Panic("fetch events from an non-existent table", zap.String("namespace", s.changefeedID.Namespace), zap.String("changefeed", s.changefeedID.ID), - zap.Int64("tableID", tableID)) + zap.Stringer("span", &span)) } sortedResolved := state.sortedResolved.Load() @@ -223,14 +226,14 @@ func (s *EventSorter) FetchByTable( log.Panic("fetch unresolved events", zap.String("namespace", s.changefeedID.Namespace), zap.String("changefeed", s.changefeedID.ID), - zap.Int64("tableID", tableID), + zap.Stringer("span", &span), zap.Uint64("upperBound", upperBound.CommitTs), zap.Uint64("resolved", sortedResolved)) } - db := s.dbs[getDB(tableID, len(s.dbs))] - iter := iterTable(db, s.uniqueID, tableID, lowerBound, upperBound) - return &EventIter{tableID: tableID, state: state, iter: iter, serde: s.serde} + db := s.dbs[getDB(span, len(s.dbs))] + iter := iterTable(db, state.uniqueID, span.TableID, lowerBound, upperBound) + return &EventIter{tableID: span.TableID, state: state, iter: iter, serde: s.serde} } // FetchAllTables implements engine.SortEngine. @@ -242,19 +245,19 @@ func (s *EventSorter) FetchAllTables(lowerBound engine.Position) engine.EventIte } // CleanByTable implements engine.SortEngine. -func (s *EventSorter) CleanByTable(tableID model.TableID, upperBound engine.Position) error { +func (s *EventSorter) CleanByTable(span tablepb.Span, upperBound engine.Position) error { s.mu.RLock() - state, exists := s.tables[tableID] + state, exists := s.tables.Get(span) s.mu.RUnlock() if !exists { log.Panic("clean an non-existent table", zap.String("namespace", s.changefeedID.Namespace), zap.String("changefeed", s.changefeedID.ID), - zap.Int64("tableID", tableID)) + zap.Stringer("span", &span)) } - return s.cleanTable(state, tableID, upperBound) + return s.cleanTable(state, span, upperBound) } // CleanAllTables implements engine.EventSortEngine. @@ -266,16 +269,16 @@ func (s *EventSorter) CleanAllTables(upperBound engine.Position) error { } // GetStatsByTable implements engine.SortEngine. -func (s *EventSorter) GetStatsByTable(tableID model.TableID) engine.TableStats { +func (s *EventSorter) GetStatsByTable(span tablepb.Span) engine.TableStats { s.mu.RLock() - state, exists := s.tables[tableID] + state, exists := s.tables.Get(span) s.mu.RUnlock() if !exists { log.Panic("Get stats from an non-existent table", zap.String("namespace", s.changefeedID.Namespace), zap.String("changefeed", s.changefeedID.ID), - zap.Int64("tableID", tableID)) + zap.Stringer("span", &span)) } maxCommitTs := state.maxReceivedCommitTs.Load() @@ -296,9 +299,10 @@ func (s *EventSorter) ReceivedEvents() int64 { s.mu.Lock() defer s.mu.Unlock() totalReceivedEvents := int64(0) - for _, state := range s.tables { + s.tables.Ascend(func(_ tablepb.Span, state *tableState) bool { totalReceivedEvents += state.receivedEvents.Load() - } + return true + }) return totalReceivedEvents } @@ -322,12 +326,18 @@ func (s *EventSorter) Close() error { s.mu.RLock() defer s.mu.RUnlock() - for tableID, state := range s.tables { - if err := s.cleanTable(state, tableID); err != nil { - return err + + var err error + s.tables.Ascend(func(span tablepb.Span, state *tableState) bool { + // TODO: maybe we can use a unified prefix for a changefeed, + // so that we can speed up it when closing a changefeed. + if err1 := s.cleanTable(state, span); err1 != nil { + err = err1 + return false } - } - return nil + return true + }) + return err } // Next implements sorter.EventIterator. @@ -364,11 +374,13 @@ func (s *EventIter) Close() error { } type eventWithTableID struct { - tableID model.TableID - event *model.PolymorphicEvent + uniqueID uint32 + span tablepb.Span + event *model.PolymorphicEvent } type tableState struct { + uniqueID uint32 ch *chann.Chann[eventWithTableID] sortedResolved atomic.Uint64 // indicates events are ready for fetching. // For statistics. @@ -391,14 +403,14 @@ func (s *EventSorter) handleEvents( batch := db.NewBatch() writeOpts := &pebble.WriteOptions{Sync: false} - newResolved := make(map[model.TableID]model.Ts) + newResolved := spanz.NewMap[model.Ts]() handleItem := func(item eventWithTableID) { if item.event.IsResolved() { - newResolved[item.tableID] = item.event.CRTs + newResolved.ReplaceOrInsert(item.span, item.event.CRTs) return } - key := encoding.EncodeKey(s.uniqueID, uint64(item.tableID), item.event) + key := encoding.EncodeKey(item.uniqueID, uint64(item.span.TableID), item.event) value, err := s.serde.Marshal(item.event, []byte{}) if err != nil { log.Panic("failed to marshal event", zap.Error(err), @@ -479,31 +491,34 @@ func (s *EventSorter) handleEvents( batch = db.NewBatch() } - for table, resolved := range newResolved { + newResolved.Ascend(func(span tablepb.Span, resolved uint64) bool { s.mu.RLock() - ts, ok := s.tables[table] + ts, ok := s.tables.Get(span) if !ok { log.Debug("Table is removed, skip updating resolved", zap.String("namespace", s.changefeedID.Namespace), zap.String("changefeed", s.changefeedID.ID), - zap.Int64("table", table), + zap.Stringer("span", &span), zap.Uint64("resolved", resolved)) s.mu.RUnlock() - continue + return false } ts.sortedResolved.Store(resolved) for _, onResolve := range s.onResolves { - onResolve(table, resolved) + onResolve(span, resolved) } s.mu.RUnlock() - } - newResolved = make(map[model.TableID]model.Ts) + return true + }) + newResolved = spanz.NewMap[model.Ts]() ioTokens <- struct{}{} } } // cleanTable uses DeleteRange to clean data of the given table. -func (s *EventSorter) cleanTable(state *tableState, tableID model.TableID, upperBound ...engine.Position) error { +func (s *EventSorter) cleanTable( + state *tableState, span tablepb.Span, upperBound ...engine.Position, +) error { var toClean engine.Position var start, end []byte @@ -520,11 +535,12 @@ func (s *EventSorter) cleanTable(state *tableState, tableID model.TableID, upper return nil } - start = encoding.EncodeTsKey(s.uniqueID, uint64(tableID), 0) + start = encoding.EncodeTsKey(state.uniqueID, uint64(span.TableID), 0) toCleanNext := toClean.Next() - end = encoding.EncodeTsKey(s.uniqueID, uint64(tableID), toCleanNext.CommitTs, toCleanNext.StartTs) + end = encoding.EncodeTsKey( + state.uniqueID, uint64(span.TableID), toCleanNext.CommitTs, toCleanNext.StartTs) - db := s.dbs[getDB(tableID, len(s.dbs))] + db := s.dbs[getDB(span, len(s.dbs))] err := db.DeleteRange(start, end, &pebble.WriteOptions{Sync: false}) if err != nil { return err @@ -547,10 +563,10 @@ func genUniqueID() uint32 { } // TODO: add test for this function. -func getDB(tableID model.TableID, dbCount int) int { +func getDB(span tablepb.Span, dbCount int) int { h := fnv.New64() b := [8]byte{} - binary.LittleEndian.PutUint64(b[:], uint64(tableID)) + binary.LittleEndian.PutUint64(b[:], uint64(span.TableID)) h.Write(b[:]) return int(h.Sum64() % uint64(dbCount)) } diff --git a/cdc/processor/sourcemanager/engine/pebble/event_sorter_test.go b/cdc/processor/sourcemanager/engine/pebble/event_sorter_test.go index 44bceb79086..cfda44b1c5a 100644 --- a/cdc/processor/sourcemanager/engine/pebble/event_sorter_test.go +++ b/cdc/processor/sourcemanager/engine/pebble/event_sorter_test.go @@ -22,7 +22,9 @@ import ( "github.com/cockroachdb/pebble" "github.com/pingcap/tiflow/cdc/model" "github.com/pingcap/tiflow/cdc/processor/sourcemanager/engine" + "github.com/pingcap/tiflow/cdc/processor/tablepb" "github.com/pingcap/tiflow/pkg/config" + "github.com/pingcap/tiflow/pkg/spanz" "github.com/stretchr/testify/require" ) @@ -38,13 +40,14 @@ func TestTableOperations(t *testing.T) { require.True(t, s.IsTableBased()) - s.AddTable(1) - s.AddTable(1) + span := spanz.TableIDToComparableSpan(1) + s.AddTable(span) + s.AddTable(span) - require.Equal(t, model.Ts(0), s.GetResolvedTs(1)) + require.Equal(t, model.Ts(0), s.GetResolvedTs(span)) - s.RemoveTable(1) - s.RemoveTable(1) + s.RemoveTable(span) + s.RemoveTable(span) } // TestNoResolvedTs tests resolved timestamps shouldn't be emitted. @@ -60,15 +63,16 @@ func TestNoResolvedTs(t *testing.T) { require.True(t, s.IsTableBased()) - s.AddTable(1) + span := spanz.TableIDToComparableSpan(1) + s.AddTable(span) resolvedTs := make(chan model.Ts) - s.OnResolve(func(_ model.TableID, ts model.Ts) { resolvedTs <- ts }) + s.OnResolve(func(_ tablepb.Span, ts model.Ts) { resolvedTs <- ts }) - s.Add(model.TableID(1), model.NewResolvedPolymorphicEvent(0, 1)) + s.Add(span, model.NewResolvedPolymorphicEvent(0, 1)) timer := time.NewTimer(100 * time.Millisecond) select { case ts := <-resolvedTs: - iter := s.FetchByTable(model.TableID(1), engine.Position{}, engine.Position{CommitTs: ts}) + iter := s.FetchByTable(span, engine.Position{}, engine.Position{CommitTs: ts}) event, _, err := iter.Next() require.Nil(t, event) require.Nil(t, err) @@ -90,9 +94,10 @@ func TestEventFetch(t *testing.T) { require.True(t, s.IsTableBased()) - s.AddTable(1) + span := spanz.TableIDToComparableSpan(1) + s.AddTable(span) resolvedTs := make(chan model.Ts) - s.OnResolve(func(_ model.TableID, ts model.Ts) { resolvedTs <- ts }) + s.OnResolve(func(_ tablepb.Span, ts model.Ts) { resolvedTs <- ts }) inputEvents := []*model.PolymorphicEvent{ model.NewPolymorphicEvent(&model.RawKVEntry{ @@ -121,8 +126,8 @@ func TestEventFetch(t *testing.T) { }), } - s.Add(1, inputEvents...) - s.Add(model.TableID(1), model.NewResolvedPolymorphicEvent(0, 4)) + s.Add(span, inputEvents...) + s.Add(span, model.NewResolvedPolymorphicEvent(0, 4)) sortedEvents := make([]*model.PolymorphicEvent, 0, len(inputEvents)) sortedPositions := make([]engine.Position, 0, len(inputEvents)) @@ -130,7 +135,8 @@ func TestEventFetch(t *testing.T) { timer := time.NewTimer(100 * time.Millisecond) select { case ts := <-resolvedTs: - iter := s.FetchByTable(model.TableID(1), engine.Position{}, engine.Position{CommitTs: ts, StartTs: ts - 1}) + iter := s.FetchByTable( + span, engine.Position{}, engine.Position{CommitTs: ts, StartTs: ts - 1}) for { event, pos, err := iter.Next() require.Nil(t, err) @@ -171,7 +177,10 @@ func TestCleanData(t *testing.T) { require.True(t, s.IsTableBased()) - s.AddTable(1) - require.Panics(t, func() { s.CleanByTable(2, engine.Position{}) }) - require.Nil(t, s.CleanByTable(1, engine.Position{})) + span := spanz.TableIDToComparableSpan(1) + s.AddTable(span) + require.Panics(t, func() { + s.CleanByTable(spanz.TableIDToComparableSpan(2), engine.Position{}) + }) + require.Nil(t, s.CleanByTable(span, engine.Position{})) } diff --git a/cdc/processor/sourcemanager/manager.go b/cdc/processor/sourcemanager/manager.go index 2b68e4ea40e..d9de9d22669 100644 --- a/cdc/processor/sourcemanager/manager.go +++ b/cdc/processor/sourcemanager/manager.go @@ -14,7 +14,6 @@ package sourcemanager import ( - "sync" "time" "github.com/pingcap/log" @@ -22,9 +21,11 @@ import ( "github.com/pingcap/tiflow/cdc/model" "github.com/pingcap/tiflow/cdc/processor/sourcemanager/engine" pullerwrapper "github.com/pingcap/tiflow/cdc/processor/sourcemanager/puller" + "github.com/pingcap/tiflow/cdc/processor/tablepb" "github.com/pingcap/tiflow/cdc/puller" cdccontext "github.com/pingcap/tiflow/pkg/context" cerrors "github.com/pingcap/tiflow/pkg/errors" + "github.com/pingcap/tiflow/pkg/spanz" "github.com/pingcap/tiflow/pkg/upstream" "go.uber.org/zap" ) @@ -43,7 +44,7 @@ type SourceManager struct { // engine is the source engine. engine engine.SortEngine // pullers is the puller wrapper map. - pullers sync.Map + pullers spanz.SyncMap // Used to report the error to the processor. errChan chan error // Used to indicate whether the changefeed is in BDR mode. @@ -70,59 +71,63 @@ func New( } // AddTable adds a table to the source manager. Start puller and register table to the engine. -func (m *SourceManager) AddTable(ctx cdccontext.Context, tableID model.TableID, tableName string, startTs model.Ts) { +func (m *SourceManager) AddTable( + ctx cdccontext.Context, span tablepb.Span, tableName string, startTs model.Ts, +) { // Add table to the engine first, so that the engine can receive the events from the puller. - m.engine.AddTable(tableID) - p := pullerwrapper.NewPullerWrapper(m.changefeedID, tableID, tableName, startTs, m.bdrMode) + m.engine.AddTable(span) + p := pullerwrapper.NewPullerWrapper(m.changefeedID, span, tableName, startTs, m.bdrMode) p.Start(ctx, m.up, m.engine, m.errChan) - m.pullers.Store(tableID, p) + m.pullers.Store(span, p) } // RemoveTable removes a table from the source manager. Stop puller and unregister table from the engine. -func (m *SourceManager) RemoveTable(tableID model.TableID) { - if wrapper, ok := m.pullers.Load(tableID); ok { +func (m *SourceManager) RemoveTable(span tablepb.Span) { + if wrapper, ok := m.pullers.Load(span); ok { wrapper.(*pullerwrapper.Wrapper).Close() - m.pullers.Delete(tableID) + m.pullers.Delete(span) } - m.engine.RemoveTable(tableID) + m.engine.RemoveTable(span) } // OnResolve just wrap the engine's OnResolve method. -func (m *SourceManager) OnResolve(action func(model.TableID, model.Ts)) { +func (m *SourceManager) OnResolve(action func(tablepb.Span, model.Ts)) { m.engine.OnResolve(action) } // FetchByTable just wrap the engine's FetchByTable method. -func (m *SourceManager) FetchByTable(tableID model.TableID, lowerBound, upperBound engine.Position) *engine.MountedEventIter { - iter := m.engine.FetchByTable(tableID, lowerBound, upperBound) +func (m *SourceManager) FetchByTable( + span tablepb.Span, lowerBound, upperBound engine.Position, +) *engine.MountedEventIter { + iter := m.engine.FetchByTable(span, lowerBound, upperBound) return engine.NewMountedEventIter(iter, m.mg, defaultMaxBatchSize) } // CleanByTable just wrap the engine's CleanByTable method. -func (m *SourceManager) CleanByTable(tableID model.TableID, upperBound engine.Position) error { - return m.engine.CleanByTable(tableID, upperBound) +func (m *SourceManager) CleanByTable(span tablepb.Span, upperBound engine.Position) error { + return m.engine.CleanByTable(span, upperBound) } // GetTableResolvedTs returns the resolved ts of the table. -func (m *SourceManager) GetTableResolvedTs(tableID model.TableID) model.Ts { - return m.engine.GetResolvedTs(tableID) +func (m *SourceManager) GetTableResolvedTs(span tablepb.Span) model.Ts { + return m.engine.GetResolvedTs(span) } // GetTablePullerStats returns the puller stats of the table. -func (m *SourceManager) GetTablePullerStats(tableID model.TableID) puller.Stats { - p, ok := m.pullers.Load(tableID) +func (m *SourceManager) GetTablePullerStats(span tablepb.Span) puller.Stats { + p, ok := m.pullers.Load(span) if !ok { log.Panic("Table puller not found when getting table puller stats", zap.String("namespace", m.changefeedID.Namespace), zap.String("changefeed", m.changefeedID.ID), - zap.Int64("tableID", tableID)) + zap.Stringer("span", &span)) } return p.(*pullerwrapper.Wrapper).GetStats() } // GetTableSorterStats returns the sorter stats of the table. -func (m *SourceManager) GetTableSorterStats(tableID model.TableID) engine.TableStats { - return m.engine.GetStatsByTable(tableID) +func (m *SourceManager) GetTableSorterStats(span tablepb.Span) engine.TableStats { + return m.engine.GetStatsByTable(span) } // ReceivedEvents returns the number of events in the engine that have not been sent to the sink. @@ -136,7 +141,7 @@ func (m *SourceManager) Close() error { zap.String("namespace", m.changefeedID.Namespace), zap.String("changefeed", m.changefeedID.ID)) start := time.Now() - m.pullers.Range(func(key, value interface{}) bool { + m.pullers.Range(func(span tablepb.Span, value interface{}) bool { value.(*pullerwrapper.Wrapper).Close() return true }) diff --git a/cdc/processor/sourcemanager/puller/puller_wrapper.go b/cdc/processor/sourcemanager/puller/puller_wrapper.go index 3d939c785bb..9683a67823a 100644 --- a/cdc/processor/sourcemanager/puller/puller_wrapper.go +++ b/cdc/processor/sourcemanager/puller/puller_wrapper.go @@ -26,7 +26,6 @@ import ( "github.com/pingcap/tiflow/pkg/config" cdccontext "github.com/pingcap/tiflow/pkg/context" cerrors "github.com/pingcap/tiflow/pkg/errors" - "github.com/pingcap/tiflow/pkg/spanz" "github.com/pingcap/tiflow/pkg/upstream" "github.com/pingcap/tiflow/pkg/util" ) @@ -34,7 +33,7 @@ import ( // Wrapper is a wrapper of puller used by source manager. type Wrapper struct { changefeed model.ChangeFeedID - tableID model.TableID + span tablepb.Span tableName string // quoted schema and table, used in metircs only p puller.Puller startTs model.Ts @@ -48,25 +47,20 @@ type Wrapper struct { // NewPullerWrapper creates a new puller wrapper. func NewPullerWrapper( changefeed model.ChangeFeedID, - tableID model.TableID, + span tablepb.Span, tableName string, startTs model.Ts, bdrMode bool, ) *Wrapper { return &Wrapper{ changefeed: changefeed, - tableID: tableID, + span: span, tableName: tableName, startTs: startTs, bdrMode: bdrMode, } } -// tableSpan returns the table span with the table ID. -func (n *Wrapper) tableSpan() []tablepb.Span { - return []tablepb.Span{spanz.TableIDToComparableSpan(n.tableID)} -} - // Start the puller wrapper. // We use cdc context to put capture info and role into context. func (n *Wrapper) Start( @@ -92,10 +86,10 @@ func (n *Wrapper) Start( up.KVStorage, up.PDClock, n.startTs, - n.tableSpan(), + []tablepb.Span{n.span}, kvCfg, n.changefeed, - n.tableID, + n.span.TableID, n.tableName, n.bdrMode, ) @@ -119,7 +113,7 @@ func (n *Wrapper) Start( continue } pEvent := model.NewPolymorphicEvent(rawKV) - if err := eventSortEngine.Add(n.tableID, pEvent); err != nil { + if err := eventSortEngine.Add(n.span, pEvent); err != nil { errChan <- err } } diff --git a/cdc/sinkv2/eventsink/mq/worker.go b/cdc/sinkv2/eventsink/mq/worker.go index 0ed873ed324..34f209b15d9 100644 --- a/cdc/sinkv2/eventsink/mq/worker.go +++ b/cdc/sinkv2/eventsink/mq/worker.go @@ -173,7 +173,7 @@ func (w *worker) batchEncodeRun(ctx context.Context) (retErr error) { eventsBuf := make([]mqEvent, flushBatchSize) for { start := time.Now() - endIndex, err := w.batch(ctx, eventsBuf) + endIndex, err := w.batch(ctx, eventsBuf, flushInterval) if err != nil { return errors.Trace(err) } @@ -195,7 +195,7 @@ func (w *worker) batchEncodeRun(ctx context.Context) (retErr error) { // batch collects a batch of messages to be sent to the DML producer. func (w *worker) batch( - ctx context.Context, events []mqEvent, + ctx context.Context, events []mqEvent, flushInterval time.Duration, ) (int, error) { index := 0 max := len(events) diff --git a/cdc/sinkv2/eventsink/mq/worker_test.go b/cdc/sinkv2/eventsink/mq/worker_test.go index c6f0db27719..c4ecf4d9ac6 100644 --- a/cdc/sinkv2/eventsink/mq/worker_test.go +++ b/cdc/sinkv2/eventsink/mq/worker_test.go @@ -148,7 +148,7 @@ func TestBatchEncode_Batch(t *testing.T) { // Test batching returns when the events count is equal to the batch size. batch := make([]mqEvent, 512) - endIndex, err := worker.batch(ctx, batch) + endIndex, err := worker.batch(ctx, batch, time.Minute) require.NoError(t, err) require.Equal(t, 512, endIndex) } diff --git a/pkg/config/debug.go b/pkg/config/debug.go index 54f01687e0a..3382a0d991e 100644 --- a/pkg/config/debug.go +++ b/pkg/config/debug.go @@ -63,13 +63,10 @@ func (c *DebugConfig) ValidateAndAdjust() error { return errors.Trace(err) } if c.Scheduler.RegionPerSpan != 0 { - if c.EnablePullBasedSink || !c.EnableNewSink { - // TODO: Removing this check once pull based sink is compatible with - // span replication. + if !c.EnableNewSink { return cerror.ErrInvalidServerOption.GenWithStackByArgs( "enabling span replication requires setting " + - "`debug.enable-new-sink` to be true and " + - "`debug.enable-pull-based-sink` to be false") + "`debug.enable-new-sink` to be true") } } if c.EnablePullBasedSink { diff --git a/pkg/spanz/sync_map.go b/pkg/spanz/sync_map.go new file mode 100644 index 00000000000..4027d092c2a --- /dev/null +++ b/pkg/spanz/sync_map.go @@ -0,0 +1,64 @@ +// Copyright 2022 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package spanz + +import ( + "sync" + + "github.com/pingcap/tiflow/cdc/processor/tablepb" +) + +// SyncMap is thread-safe map, its key is tablepb.Span. +type SyncMap struct { + m sync.Map +} + +// Load returns the value stored in the map for a key, or nil if no +// value is present. +// The ok result indicates whether value was found in the map. +func (m *SyncMap) Load(key tablepb.Span) (value any, ok bool) { + return m.m.Load(ToHashableSpan(key)) +} + +// Store sets the value for a key. +func (m *SyncMap) Store(key tablepb.Span, value any) { + m.m.Store(ToHashableSpan(key), value) +} + +// LoadOrStore returns the existing value for the key if present. +// Otherwise, it stores and returns the given value. +// The loaded result is true if the value was loaded, false if stored. +func (m *SyncMap) LoadOrStore(key tablepb.Span, value any) (actual any, loaded bool) { + return m.m.LoadOrStore(ToHashableSpan(key), value) +} + +// Delete deletes the value for a key. +func (m *SyncMap) Delete(key tablepb.Span) { + m.m.Delete(ToHashableSpan(key)) +} + +// LoadAndDelete deletes the value for a key, returning the previous value if any. +// The loaded result reports whether the key was present. +func (m *SyncMap) LoadAndDelete(key tablepb.Span) (value any, loaded bool) { + return m.m.LoadAndDelete(ToHashableSpan(key)) +} + +// Range calls f sequentially for each key and value present in the map. +// If f returns false, range stops the iteration. +func (m *SyncMap) Range(f func(span tablepb.Span, value any) bool) { + m.m.Range(func(key, value any) bool { + span := key.(HashableSpan).ToSpan() + return f(span, value) + }) +}