From 6482477a4a1d437b1ed223e81331002743d5a39b Mon Sep 17 00:00:00 2001 From: hongyunyan <649330952@qq.com> Date: Wed, 14 Aug 2024 16:42:46 +0800 Subject: [PATCH 01/12] support ddl in table event dispatcher --- downstreamadapter/dispatcher/dispatcher.go | 84 +- .../dispatcher/table_event_dispatcher.go | 168 +-- .../table_trigger_event_dispatcher.go | 80 +- .../event_dispatcher_manager.go | 55 +- .../dispatchermanager/heartbeat_collector.go | 91 +- downstreamadapter/sink/mysql_sink.go | 19 +- downstreamadapter/sink/sink.go | 4 +- .../sink/types/table_progress.go | 27 +- downstreamadapter/worker/mysql_worker.go | 145 +-- heartbeatpb/heartbeat.pb.go | 1098 +++++++++-------- heartbeatpb/heartbeat.proto | 30 +- maintainer/fake_dispatcher_manager.go | 12 +- pkg/common/txn_event.go | 23 +- 13 files changed, 891 insertions(+), 945 deletions(-) diff --git a/downstreamadapter/dispatcher/dispatcher.go b/downstreamadapter/dispatcher/dispatcher.go index 5f5bf5c81..75c6a9d3a 100644 --- a/downstreamadapter/dispatcher/dispatcher.go +++ b/downstreamadapter/dispatcher/dispatcher.go @@ -52,19 +52,20 @@ The workflow related to the dispatcher is as follows: type Dispatcher interface { GetSink() sink.Sink GetTableSpan() *common.TableSpan - GetState() *State GetEventChan() chan *common.TxnEvent GetResolvedTs() uint64 UpdateResolvedTs(uint64) GetCheckpointTs() uint64 GetId() string GetDispatcherType() DispatcherType - GetHeartBeatChan() chan *HeartBeatResponseMessage + GetDDLActions() chan *heartbeatpb.DispatcherAction //GetSyncPointInfo() *SyncPointInfo //GetMemoryUsage() *MemoryUsage // PushEvent(event *eventpb.TxnEvent) PushTxnEvent(event *common.TxnEvent) GetComponentStatus() heartbeatpb.ComponentState + + TryClose() (w heartbeatpb.Watermark, ok bool) } type DispatcherType uint64 @@ -74,63 +75,6 @@ const ( TableTriggerEventDispatcherType DispatcherType = 1 ) -type Action uint64 - -const ( - None Action = 0 - Write Action = 1 - Pass Action = 2 -) - -/* -State displays the status of advancing event and State is used to indicate whether the current pendingEvent can be advanced. -Only the following events can be blocked: -1. All the DDL Events -2. All the Sync Point Events -3. The Event after the DDL Event or the Sync Point Event -*/ -type State struct { - // False means the events are being pushed down continuously - // True means there is an event being blocked - isBlocked bool - // The blocked Event. - // pendingEvent could be nil when isBlocked is true. - // Such as one ddl event is sent to downstream, - // so the following events can be pushed down only when - // the ddl event is flushed to downstream successfully (that means sink is available). - pengdingEvent *common.TxnEvent - // The pendingEvent is waiting for the progress of these tableSpans to reach the blockTs. - blockTableSpan []*common.TableSpan - // the commitTs of the pendingEvent, also the ts the tableSpan in the blockTableSpan should reach. - blockTs uint64 - // True means the sink flushes all the previous event successfully, - // there is no event of these tableSpan in the Sink now. - sinkAvailable bool - // The action for the pendingEvent, - // it is used to decide whether the pendingEvent should write or just pass. - action Action // -} - -func NewState() *State { - return &State{ - isBlocked: false, - pengdingEvent: nil, - blockTableSpan: nil, - blockTs: 0, - sinkAvailable: false, - action: None, - } -} - -func (s *State) clear() { - s.isBlocked = false - s.pengdingEvent = nil - s.blockTableSpan = nil - s.blockTs = 0 - s.sinkAvailable = false - s.action = None -} - /* HeartBeatInfo is used to collect the message for HeartBeatRequest for each dispatcher. Mainly about the progress of each dispatcher: @@ -168,25 +112,3 @@ func CollectDispatcherHeartBeatInfo(d Dispatcher, h *HeartBeatInfo) { h.ComponentStatus = d.GetComponentStatus() h.TableSpan = d.GetTableSpan() } - -/* -TableSpanProgress shows the progress of the other tableSpan, including: -1. Whether the tableSpan is blocked, and the ts of blocked event -2. The checkpointTs of the tableSpan -*/ -type TableSpanProgress struct { - Span *common.TableSpan - IsBlocked bool - BlockTs uint64 - CheckpointTs uint64 -} - -/* -HeartBeatReponseMessage includes the message from the HeartBeatResponse, including: -1. The action for the blocked event -2. the progress of other tableSpan, which the dispatcher is waiting for. -*/ -type HeartBeatResponseMessage struct { // 最好需要一个对应,对应 blocked by 什么 event 的 信号,避免出现乱序的问题 - Action Action - OtherTableProgress []*TableSpanProgress -} diff --git a/downstreamadapter/dispatcher/table_event_dispatcher.go b/downstreamadapter/dispatcher/table_event_dispatcher.go index 8400fba99..0efa374a7 100644 --- a/downstreamadapter/dispatcher/table_event_dispatcher.go +++ b/downstreamadapter/dispatcher/table_event_dispatcher.go @@ -14,19 +14,15 @@ package dispatcher import ( - "bytes" "context" "sync" "time" "github.com/flowbehappy/tigate/downstreamadapter/sink" - "github.com/flowbehappy/tigate/eventpb" "github.com/flowbehappy/tigate/heartbeatpb" "github.com/flowbehappy/tigate/pkg/common" "github.com/google/uuid" - "github.com/pingcap/errors" "github.com/pingcap/log" - "github.com/pingcap/tiflow/cdc/model" "go.uber.org/zap" ) @@ -99,18 +95,13 @@ type TableEventDispatcher struct { tableSpan *common.TableSpan sink sink.Sink - state *State - - // 搞个 channel 来接收 heartbeat 产生的 信息,然后下推数据这个就可以做成 await 了 - // heartbeat 会更新依赖的 tableSpan 的 状态,然后满足了就删掉,下次发送就不用发了,但最终推动他变化的还是要收到 action - heartbeatChan chan *HeartBeatResponseMessage + ddlActions chan *heartbeatpb.DispatcherAction + tableSpanStatusesChan chan *heartbeatpb.TableSpanStatus //SyncPointInfo *SyncPointInfo //MemoryUsage *MemoryUsage - tableInfo *common.TableInfo // TODO:后续做成一整个 tableInfo Struct - componentStatus *ComponentStateWithMutex resolvedTs *TsWithMutex // 用来记 eventChan 中目前收到的 event 中收到的最大的 commitTs - 1,不代表 dispatcher 的 checkpointTs @@ -119,15 +110,15 @@ type TableEventDispatcher struct { wg sync.WaitGroup } -func NewTableEventDispatcher(tableSpan *common.TableSpan, sink sink.Sink, startTs uint64, syncPointInfo *SyncPointInfo) *TableEventDispatcher { +func NewTableEventDispatcher(tableSpan *common.TableSpan, sink sink.Sink, startTs uint64, syncPointInfo *SyncPointInfo, tableSpanStatusesChan chan *heartbeatpb.TableSpanStatus) *TableEventDispatcher { ctx, cancel := context.WithCancel(context.Background()) tableEventDispatcher := &TableEventDispatcher{ - id: uuid.NewString(), - eventCh: make(chan *common.TxnEvent, 16), - tableSpan: tableSpan, - sink: sink, - state: NewState(), - heartbeatChan: make(chan *HeartBeatResponseMessage, 16), + id: uuid.NewString(), + eventCh: make(chan *common.TxnEvent, 16), + tableSpan: tableSpan, + sink: sink, + ddlActions: make(chan *heartbeatpb.DispatcherAction, 16), + tableSpanStatusesChan: tableSpanStatusesChan, //SyncPointInfo: syncPointInfo, //MemoryUsage: NewMemoryUsage(), componentStatus: newComponentStateWithMutex(heartbeatpb.ComponentState_Working), @@ -143,6 +134,53 @@ func NewTableEventDispatcher(tableSpan *common.TableSpan, sink sink.Sink, startT return tableEventDispatcher } +// 1. 如果是单表内的 ddl,达到下推的条件为: sink 中没有还没执行完的当前表的 event +// 2. 如果是多表内的 ddl 或者是表间的 ddl,则需要满足的条件为: +// 2.1 sink 中没有还没执行完的当前表的 event +// 2.2 maintainer 通知自己可以 write 或者 pass event +func (d *TableEventDispatcher) AddDDLEventToSinkWhenAvailable(event *common.TxnEvent) { + if event.IsSingleTableDDL() { + if d.sink.IsEmpty(d.tableSpan) { + d.sink.AddDMLEvent(d.tableSpan, event) + return + } else { + // TODO:先写一个 定时 check 的逻辑,后面用 dynamic stream 改造 + timer := time.NewTimer(time.Millisecond * 50) + for { + select { + case <-timer.C: + if d.sink.IsEmpty(d.tableSpan) { + d.sink.AddDMLEvent(d.tableSpan, event) + return + } + } + } + } + } + + d.tableSpanStatusesChan <- &heartbeatpb.TableSpanStatus{ + Span: d.tableSpan.TableSpan, + ComponentStatus: heartbeatpb.ComponentState_Working, + State: &heartbeatpb.State{ + IsBlocked: true, + BlockTs: event.CommitTs, + BlockTableSpan: event.GetBlockedTableSpan(), // 这个包含自己的 span 是不是也无所谓,不然就要剔除掉 + NeedDroppedTableSpan: event.GetNeedDroppedTableSpan(), + NeedAddedTableSpan: event.GetNeedAddedTableSpan(), + }, + } + + for { + dispatcherAction := <-d.ddlActions + if dispatcherAction.CommitTs == event.CommitTs { + if dispatcherAction.Action == heartbeatpb.Action_Write { + d.sink.AddDDLAndSyncPointEvent(d.tableSpan, event) // 这个是同步写,所以写完的时候 sink 也 available 了 + } + return + } + } +} + func (d *TableEventDispatcher) DispatcherEvents(ctx context.Context) { defer d.wg.Done() tableSpan := d.GetTableSpan() @@ -152,9 +190,10 @@ func (d *TableEventDispatcher) DispatcherEvents(ctx context.Context) { case <-ctx.Done(): return case event := <-d.GetEventChan(): - //log.Info("TableEventDispatcher get events from eventChan", zap.Any("event is DML", event.IsDMLEvent()), zap.Any("dispatcher table id", d.tableSpan.TableID)) if event.IsDMLEvent() { sink.AddDMLEvent(tableSpan, event) + } else if event.IsDDLEvent() { + d.AddDDLEventToSinkWhenAvailable(event) } else { // resolvedTs d.resolvedTs.Set(event.ResolvedTs) @@ -171,10 +210,6 @@ func (d *TableEventDispatcher) GetTableSpan() *common.TableSpan { return d.tableSpan } -func (d *TableEventDispatcher) GetState() *State { - return d.state -} - func (d *TableEventDispatcher) GetEventChan() chan *common.TxnEvent { return d.eventCh } @@ -184,17 +219,10 @@ func (d *TableEventDispatcher) GetResolvedTs() uint64 { } func (d *TableEventDispatcher) GetCheckpointTs() uint64 { - var checkpointTs uint64 - smallestCommitTsInSink := d.GetSink().GetSmallestCommitTs(d.GetTableSpan()) - if smallestCommitTsInSink == 0 { - // state := d.GetState() - // if state.pengdingEvent != nil { - // checkpointTs = state.pengdingEvent.CommitTs - 1 - // } else { + checkpointTs := d.GetSink().GetCheckpointTs(d.GetTableSpan()) + if checkpointTs == 0 { + // 说明从没有数据写到过 sink,则选择用 resolveTs 作为 checkpointTs checkpointTs = d.GetResolvedTs() - //} - } else { - checkpointTs = smallestCommitTsInSink - 1 } return checkpointTs } @@ -211,8 +239,8 @@ func (d *TableEventDispatcher) GetDispatcherType() DispatcherType { return TableEventDispatcherType } -func (d *TableEventDispatcher) GetHeartBeatChan() chan *HeartBeatResponseMessage { - return d.heartbeatChan +func (d *TableEventDispatcher) GetDDLActions() chan *heartbeatpb.DispatcherAction { + return d.ddlActions } //func (d *TableEventDispatcher) GetSyncPointInfo() *SyncPointInfo { @@ -223,73 +251,10 @@ func (d *TableEventDispatcher) GetHeartBeatChan() chan *HeartBeatResponseMessage // return d.MemoryUsage // } -func (d *TableEventDispatcher) decodeEvent(rawTxnEvent *eventpb.TxnEvent) (*common.TxnEvent, error) { - txnEvent := &common.TxnEvent{} - for _, rawEvent := range rawTxnEvent.Events { - key, physicalTableID, err := decodeTableID(rawEvent.Key) - if err != nil { - return nil, err - } - - if len(rawEvent.OldValue) == 0 && len(rawEvent.Value) == 0 { - log.Warn("empty value and old value", - zap.Any("raw event", rawEvent)) - } - - baseInfo := baseKVEntry{ - StartTs: rawTxnEvent.StartTs, - CRTs: rawTxnEvent.CommitTs, - PhysicalTableID: physicalTableID, - Delete: rawEvent.OpType == eventpb.OpType_OpTypeDelete, - } - - row, _ := func() (*model.RowChangedEvent, error) { - if bytes.HasPrefix(key, recordPrefix) { - rowKV, err := unmarshalRowKVEntry(d.tableInfo, rawEvent.Key, rawEvent.Value, rawEvent.OldValue, baseInfo) - if err != nil { - return nil, errors.Trace(err) - } - if rowKV == nil { - return nil, nil - } - row, _, err := mountRowKVEntry(d.tableInfo, rowKV) - if err != nil { - return nil, err - } - return row, nil - } - return nil, nil - }() - - if err != nil { - return nil, err - } - txnEvent.Rows = append(txnEvent.Rows, &common.RowChangedEvent{ - PhysicalTableID: row.PhysicalTableID, - TableInfo: d.tableInfo, - ReplicatingTs: row.ReplicatingTs, - Columns: common.ColumnDatas2Columns(row.Columns, d.tableInfo), - PreColumns: common.ColumnDatas2Columns(row.PreColumns, d.tableInfo), - }) - } - return txnEvent, nil -} - -// func (d *TableEventDispatcher) PushEvent(rawTxnEvent *eventpb.TxnEvent) { -// // decode the raw event to common.TxnEvent -// event, _ := d.decodeEvent(rawTxnEvent) -// //d.GetMemoryUsage().Add(event.CommitTs, event.MemoryCost()) -// d.GetEventChan() <- event // 换成一个函数 -// } - func (d *TableEventDispatcher) PushTxnEvent(event *common.TxnEvent) { d.GetEventChan() <- event } -func (d *TableEventDispatcher) InitTableInfo(tableInfo *eventpb.TableInfo) { - //d.tableInfo = decodeTableInfo(tableInfo) // TODO -} - func (d *TableEventDispatcher) Remove() { // TODO: 修改这个 dispatcher 的 status 为 removing d.cancel() @@ -304,12 +269,7 @@ func (d *TableEventDispatcher) TryClose() (w heartbeatpb.Watermark, ok bool) { if d.sink.IsEmpty(d.tableSpan) { // calculate the checkpointTs, and clean the resource d.sink.RemoveTableSpan(d.tableSpan) - state := d.GetState() - if state.pengdingEvent != nil { - w.CheckpointTs = state.pengdingEvent.CommitTs - 1 - } else { - w.CheckpointTs = d.GetCheckpointTs() - } + w.CheckpointTs = d.GetCheckpointTs() w.ResolvedTs = d.GetResolvedTs() //d.MemoryUsage.Clear() diff --git a/downstreamadapter/dispatcher/table_trigger_event_dispatcher.go b/downstreamadapter/dispatcher/table_trigger_event_dispatcher.go index 21f63d559..74164c582 100644 --- a/downstreamadapter/dispatcher/table_trigger_event_dispatcher.go +++ b/downstreamadapter/dispatcher/table_trigger_event_dispatcher.go @@ -22,26 +22,51 @@ import ( "github.com/pingcap/log" ) +//filter 问题 -- 能收到这条就至少说明有相关的 table(比如 renames / create tables / exchange partitions -- 这个应该不支持一个在一个不在的),对于跟 table 有关的表来说,那就前面两种就可以在 ddl 生成的时候用 config 处理了 +// 这个要让 logService 传过来 新增 table span 删除 tableSpan 的问题 +/* double check 一下类型吧 +ActionCreateSchema -- 只需要执行下游语句 +ActionDropSchema -- 只需要执行下游语句 +ActionCreateTable -- maintainer 通知 table trigger event dispatcher 执行下游语句,成功后创建 dispatcher +ActionDropTable -- maintainer 通知 table trigger event dispatcher 执行下游语句,成功后删除 dispatcher +ActionTruncateTable -- maintainer 通知 table trigger event dispatcher 执行下游语句,成功后删除老 dispatcher,创建新 dispatcher +ActionRenameTable -- 先在下游执行这个语句,然后删掉老的 dispatcher,创建新的 dispatcher +ActionAddTablePartition -- 先执行这个语句,然后创建新的 dispatcher +ActionDropTablePartition -- 先执行这个语句,然后删除老的 dispatcher +ActionTruncateTablePartition -- 先执行这个语句,然后删除老的 dispatcher,创建新的 dispatcher +ActionRecoverTable -- 先执行这个语句,然后创建新的 dispatcher +ActionRepairTable -- 只需要执行 +ActionExchangeTablePartition -- 先执行这个语句,这个理论上 table id 变了,其他应该没怎么变,可以先考虑删了老 dispatcher 然后创建新的,后面也可以考虑要不要变成更轻量的修改。 +ActionRemovePartitioning -- 先执行这个语句,然后删除老的 dispatcher +ActionRenameTables -- rename table 的复数版 +ActionCreateTables -- create table 的复数版 +ActionReorganizePartition -- 先执行,然后该删删该加加 +ActionFlashbackCluster -- 执行语句,只对 tidb 有效果 +ActionCreateResourceGroup/ActionAlterResourceGroup/ActionDropResourceGroup 执行语句,且只对 tidb +*/ + /* -ActionCreateSchema -- 这个要保证自己是晚于 drop database d1 即可,另外要比自己后面的 ddl 早(以防后面有 create table d1.t1) -ActionDropSchema -- 也要保证顺序,要早于 create database d1,但不能早于其他这个 db 中的表执行到这个 ddl.commitTs, 不然会丢失数据 -- 每个 db 内的表都会收一份 -ActionCreateTable -- 要晚于 drop table t1 -ActionDropTable -- 要早于 create table t1,以及不能早于这个表本身执行到 ddl.CommitTs -- shared with 对应 table -ActionTruncateTable -- 这个也要卡在这个表本身执行到 ddl.CommitTs -- shared with 对应的 table -ActionRenameTable -- 卡对应 table 本身 -- shared with 对应 table -ActionAddTablePartition -- 卡在内部对应物理表的相同时间 -ActionDropTablePartition -- 卡在内部对应物理表的相同时间,主要是为了保证 add table partition 不会出问题 -ActionTruncateTablePartition -ActionRecoverTable -ActionRepairTable -ActionExchangeTablePartition -ActionRemovePartitioning -ActionRenameTables -ActionCreateTables -ActionReorganizePartition -ActionFlashbackCluster -- 只支持没有表结构和库结构变更时同步 -- 要做一下检查 -ActionMultiSchemaChange -- 这个只是一个 action,不改变 table 本身,但是要卡全局的 ts -ActionCreateResourceGroup/ActionAlterResourceGroup/ActionDropResourceGroup 卡全局 ts,但是只对下游是 tidb 的时候可用,别的时候都别往下扔 +TableTriggerEventDispatcher 需要接收所有跟创建表或者删表相关的 ddl,其中每一条 DDL 的推进,都要跟 maintainer 沟通,确认是否需要自己执行( pass / write),所以可以理解为同一个 changefeed 的多个节点的 tableTriggerEventDispatcher 进度至少是基本同步的。如果是 rename 这种 ddl,就还要等对应表的 checkpointTs 推到。 + + + + + +假设我有两个节点A,B, table C 一开始在 A 上同步,然后下一条 event 是 checkpointTs d 的 rename 操作,然后 A 和 B 的 tableTriggerEventTable 的 checkpointTs 也推到 t-1 了,现在大家都跟 maintainer 通信说了自己推到了 d-1,然后 maintainer 会通知某个tableTrigger 执行这条 ddl, 通知另一个节点 B 跳过这条 ddl,通知 这个表跳过这条 ddl(执行的先通知,跳过的是执行完才通知的)。 +如果这个时候这个表被迁移了,没收到那个跳过的通知,所以 maintainer 需要再次通知他跳过这条 ddl。 -- 这个是maintainer 需要做的事情。也就是要求maintainer 要至少知道 ddl 的推进进度,保证 skip 可以重发。 +那如果通知执行这条 ddl 后,却没有收到推进的消息,则应该选择同节点重发,不能换节点。 +对于 table trigger event table 的 ddl,我们需要严格按照顺序执行,满足前一条没有执行成功时,后一条不能开始执行的要求。skip 和下一条的执行是可以一起通知的。所有的 ddl 信息在收到以后快速等10ms 或者其他时间以后就按照心跳发送给 maintainer。如果没有什么 ddl,这个就跟着大家定期汇报进度,有 ddl 到了或者在排队等待的时候,就应该更高频?比如 20ms 这样可以发50个来回?每次最多发两个 ddl event 给上面。 + + + +在哪里生成那些没有 query 的 ddl query -- 单表的没有问题,多表的话也可以先都生成,我自己来做 filter + +add index 的问题 -- 这个异步去做,并且更改现在的状态为有 ddl 执行中,没执行完后续的 ddl 不能推进,dml 可以先正常推进知道卡到下一个 ddl 。 + + +所以本质来说 tableTriggerEvent 持续接收 ddl,然后跟maintainer 沟通决定是否能推进。 + + */ /* @@ -63,10 +88,10 @@ type TableTriggerEventDispatcher struct { Id string Ch chan *common.TxnEvent // 接受 event -- 先做个基础版本的,每次处理一条 ddl 的那种 Filter filter.Filter - Sink sink.Sink + sink sink.Sink HeartbeatChan chan *HeartBeatResponseMessage State *State - TableSpan *common.TableSpan // 给一个特殊的 tableSpan + tableSpan *common.TableSpan // 给一个特殊的 tableSpan ResolvedTs uint64 MemoryUsage *MemoryUsage @@ -127,3 +152,16 @@ func (d *TableTriggerEventDispatcher) GetCheckpointTs() uint64 { return 0 } func (d *TableTriggerEventDispatcher) GetComponentStatus() heartbeatpb.ComponentState { return heartbeatpb.ComponentState_Working } + +// TryClose try to close the tableTriggerEventDispatcher, +// It should first check whether the related events in sink is finished. +// If yes, then return checkpointTs, true, else return 0, false. +func (d *TableTriggerEventDispatcher) TryClose() (w heartbeatpb.Watermark, ok bool) { + if d.sink.IsEmpty(d.tableSpan) { + d.sink.RemoveTableSpan(d.tableSpan) + w.CheckpointTs = w.GetCheckpointTs() + w.ResolvedTs = d.ResolvedTs + return w, true + } + return w, false +} diff --git a/downstreamadapter/dispatchermanager/event_dispatcher_manager.go b/downstreamadapter/dispatchermanager/event_dispatcher_manager.go index d22f1bb2a..5e85f4630 100644 --- a/downstreamadapter/dispatchermanager/event_dispatcher_manager.go +++ b/downstreamadapter/dispatchermanager/event_dispatcher_manager.go @@ -56,8 +56,8 @@ type EventDispatcherManager struct { dispatcherMap *DispatcherMap tableTriggerEventDispatcher *dispatcher.TableTriggerEventDispatcher - //heartbeatResponseQueue *HeartbeatResponseQueue - heartbeatRequestQueue *HeartbeatRequestQueue + heartbeatResponseQueue *HeartbeatResponseQueue + heartbeatRequestQueue *HeartbeatRequestQueue //heartBeatSendTask *HeartBeatSendTask cancel context.CancelFunc wg sync.WaitGroup @@ -86,9 +86,9 @@ type EventDispatcherManager struct { func NewEventDispatcherManager(changefeedID model.ChangeFeedID, changefeedConfig *config.ChangefeedConfig, maintainerID messaging.ServerId) *EventDispatcherManager { ctx, cancel := context.WithCancel(context.Background()) eventDispatcherManager := &EventDispatcherManager{ - dispatcherMap: newDispatcherMap(), - changefeedID: changefeedID, - //heartbeatResponseQueue: NewHeartbeatResponseQueue(), + dispatcherMap: newDispatcherMap(), + changefeedID: changefeedID, + heartbeatResponseQueue: NewHeartbeatResponseQueue(), //enableSyncPoint: false, maintainerID: maintainerID, tableSpanStatusesChan: make(chan *heartbeatpb.TableSpanStatus, 10000), @@ -144,17 +144,31 @@ func (e *EventDispatcherManager) Init(startTs uint64) error { log.Error("create mysql sink failed", zap.Error(err)) return err } - // e.sink = sink.NewMysqlSink(*e.config.SinkConfig.MySQLConfig.WorkerCount, cfg, db) + e.sink = sink.NewMysqlSink(e.changefeedID, 16, cfg, db) - return nil - //} //Init Table Trigger Event Dispatcher e.tableTriggerEventDispatcher = e.newTableTriggerEventDispatcher(startTs) - // init heartbeat recv and send task - // No need to run recv task when there is no ddl event - //threadpool.GetTaskSchedulerInstance().HeartbeatTaskScheduler.Submit(newHeartbeatRecvTask(e)) + // get heartbeat response from HeartBeatResponseQueue, and send to each dispatcher + e.wg.Add(1) + go func() { + defer e.wg.Done() + for { + heartbeatResponse := e.GetHeartbeatResponseQueue().Dequeue() + dispatcherActions := heartbeatResponse.Actions + for _, dispatcherAction := range dispatcherActions { + tableSpan := dispatcherAction.Span + dispatcher, ok := e.dispatcherMap.Get(&common.TableSpan{TableSpan: tableSpan}) + if !ok { + log.Error("dispatcher not found", zap.Any("tableSpan", tableSpan)) + continue + } + dispatcher.GetDDLActions() <- dispatcherAction + } + } + }() + return nil } @@ -171,7 +185,7 @@ func (e *EventDispatcherManager) close() { e.cancel() e.wg.Wait() - toCloseDispatchers := make([]*dispatcher.TableEventDispatcher, 0) + toCloseDispatchers := make([]dispatcher.Dispatcher, 0) e.dispatcherMap.ForEach(func(tableSpan *common.TableSpan, dispatcher *dispatcher.TableEventDispatcher) { dispatcher.Remove() _, ok := dispatcher.TryClose() @@ -179,11 +193,18 @@ func (e *EventDispatcherManager) close() { toCloseDispatchers = append(toCloseDispatchers, dispatcher) } }) + + _, ok := e.tableTriggerEventDispatcher.TryClose() + if !ok { + toCloseDispatchers = append(toCloseDispatchers, e.tableTriggerEventDispatcher) + } + for _, dispatcher := range toCloseDispatchers { log.Info("waiting for dispatcher to close", zap.Any("tableSpan", dispatcher.GetTableSpan())) ok := false for !ok { _, ok = dispatcher.TryClose() + time.Sleep(10 * time.Millisecond) } } @@ -225,7 +246,7 @@ func (e *EventDispatcherManager) NewTableEventDispatcher(tableSpan *common.Table syncPointInfo.EnableSyncPoint = false } */ - tableEventDispatcher := dispatcher.NewTableEventDispatcher(tableSpan, e.sink, startTs, nil) + tableEventDispatcher := dispatcher.NewTableEventDispatcher(tableSpan, e.sink, startTs, nil, e.tableSpanStatusesChan) appcontext.GetService[*eventcollector.EventCollector](appcontext.EventCollector).RegisterDispatcher(tableEventDispatcher, startTs, nil) @@ -319,8 +340,8 @@ func (e *EventDispatcherManager) newTableTriggerEventDispatcher(startTs uint64) Ch: make(chan *common.TxnEvent, 1000), ResolvedTs: startTs, HeartbeatChan: make(chan *dispatcher.HeartBeatResponseMessage, 100), - Sink: e.sink, - TableSpan: &common.DDLSpan, + sink: e.sink, + tableSpan: &common.DDLSpan, State: dispatcher.NewState(), //MemoryUsage: dispatcher.NewMemoryUsage(), } @@ -432,6 +453,10 @@ func (e *EventDispatcherManager) GetChangeFeedID() model.ChangeFeedID { return e.changefeedID } +func (e *EventDispatcherManager) GetHeartbeatResponseQueue() *HeartbeatResponseQueue { + return e.heartbeatResponseQueue +} + func (e *EventDispatcherManager) GetHeartbeatRequestQueue() *HeartbeatRequestQueue { return e.heartbeatRequestQueue } diff --git a/downstreamadapter/dispatchermanager/heartbeat_collector.go b/downstreamadapter/dispatchermanager/heartbeat_collector.go index 17fc654b3..d9e8de3b9 100644 --- a/downstreamadapter/dispatchermanager/heartbeat_collector.go +++ b/downstreamadapter/dispatchermanager/heartbeat_collector.go @@ -15,9 +15,11 @@ package dispatchermanager import ( "context" + "fmt" "sync" "github.com/flowbehappy/tigate/heartbeatpb" + "github.com/flowbehappy/tigate/pkg/apperror" "github.com/flowbehappy/tigate/pkg/common" appcontext "github.com/flowbehappy/tigate/pkg/common/context" "github.com/flowbehappy/tigate/pkg/messaging" @@ -29,6 +31,56 @@ import ( const handleDispatcherRequestConcurrency = 16 +type ResponseChanMap struct { + responseChanMapMutex sync.RWMutex + m map[model.ChangeFeedID]*HeartbeatResponseQueue //changefeedID -> HeartbeatResponseQueue +} + +func NewResponseChanMap() *ResponseChanMap { + return &ResponseChanMap{ + m: make(map[model.ChangeFeedID]*HeartbeatResponseQueue), + } +} + +func (responseChanMap *ResponseChanMap) Get(changeFeedID model.ChangeFeedID) (heartbeatResponseQueue *HeartbeatResponseQueue, ok bool) { + responseChanMap.responseChanMapMutex.RLock() + defer responseChanMap.responseChanMapMutex.RUnlock() + heartbeatResponseQueue, ok = responseChanMap.m[changeFeedID] + return +} + +func (responseChanMap *ResponseChanMap) Set(changeFeedID model.ChangeFeedID, heartbeatResponseQueue *HeartbeatResponseQueue) { + responseChanMap.responseChanMapMutex.Lock() + defer responseChanMap.responseChanMapMutex.Unlock() + + responseChanMap.m[changeFeedID] = heartbeatResponseQueue +} + +type EventDispatcherManagerMap struct { + eventDispatcherManagerMutex sync.RWMutex + eventDispatcherManagerMap map[model.ChangeFeedID]*EventDispatcherManager // changefeedID -> EventDispatcherManager +} + +func NewEventDispatcherManagerMap() *EventDispatcherManagerMap { + return &EventDispatcherManagerMap{ + eventDispatcherManagerMap: make(map[model.ChangeFeedID]*EventDispatcherManager), + } +} + +func (eventDispatcherManagerMap *EventDispatcherManagerMap) Get(changeFeedID model.ChangeFeedID) (eventDispatcherManager *EventDispatcherManager, ok bool) { + eventDispatcherManagerMap.eventDispatcherManagerMutex.RLock() + defer eventDispatcherManagerMap.eventDispatcherManagerMutex.RUnlock() + eventDispatcherManager, ok = eventDispatcherManagerMap.eventDispatcherManagerMap[changeFeedID] + return +} + +func (eventDispatcherManagerMap *EventDispatcherManagerMap) Set(changeFeedID model.ChangeFeedID, eventDispatcherManager *EventDispatcherManager) { + eventDispatcherManagerMap.eventDispatcherManagerMutex.Lock() + defer eventDispatcherManagerMap.eventDispatcherManagerMutex.Unlock() + + eventDispatcherManagerMap.eventDispatcherManagerMap[changeFeedID] = eventDispatcherManager +} + /* HeartBeatCollect is responsible for sending heartbeat requests and receiving heartbeat responses by messageCenter HeartBeatCollector is an instance-level component. It will deal with all the heartbeat messages from all dispatchers in all dispatcher managers. @@ -37,12 +89,10 @@ type HeartBeatCollector struct { wg sync.WaitGroup from messaging.ServerId - eventDispatcherManagerMutex sync.RWMutex - eventDispatcherManagerMap map[model.ChangeFeedID]*EventDispatcherManager // changefeedID -> EventDispatcherManager + eventDispatcherManagerMap *EventDispatcherManagerMap + responseChanMap *ResponseChanMap - responseChanMapMutex sync.RWMutex - responseChanMap map[model.ChangeFeedID]*HeartbeatResponseQueue //changefeedID -> HeartbeatResponseQueue - requestQueue *HeartbeatRequestQueue + requestQueue *HeartbeatRequestQueue dispatcherRequestCh []chan *heartbeatpb.ScheduleDispatcherRequest } @@ -51,11 +101,11 @@ func NewHeartBeatCollector(serverId messaging.ServerId) *HeartBeatCollector { heartBeatCollector := HeartBeatCollector{ from: serverId, requestQueue: NewHeartbeatRequestQueue(), - responseChanMap: make(map[model.ChangeFeedID]*HeartbeatResponseQueue), - eventDispatcherManagerMap: make(map[model.ChangeFeedID]*EventDispatcherManager), + responseChanMap: NewResponseChanMap(), + eventDispatcherManagerMap: NewEventDispatcherManagerMap(), dispatcherRequestCh: make([]chan *heartbeatpb.ScheduleDispatcherRequest, handleDispatcherRequestConcurrency), } - //context.GetService[messaging.MessageCenter](context.MessageCenter).RegisterHandler(heartbeatResponseTopic, heartBeatCollector.RecvHeartBeatResponseMessages) + appcontext.GetService[messaging.MessageCenter](appcontext.MessageCenter).RegisterHandler(messaging.HeartbeatCollectorTopic, heartBeatCollector.RecvHeartBeatResponseMessages) appcontext.GetService[messaging.MessageCenter](appcontext.MessageCenter). RegisterHandler(messaging.HeartbeatCollectorTopic, heartBeatCollector.RecvSchedulerDispatcherRequestMessages) heartBeatCollector.wg.Add(1) @@ -83,14 +133,8 @@ func NewHeartBeatCollector(serverId messaging.ServerId) *HeartBeatCollector { func (c *HeartBeatCollector) RegisterEventDispatcherManager(m *EventDispatcherManager) error { m.SetHeartbeatRequestQueue(c.requestQueue) - c.eventDispatcherManagerMutex.Lock() - c.responseChanMapMutex.Lock() - - defer c.eventDispatcherManagerMutex.Unlock() - defer c.responseChanMapMutex.Unlock() - - //c.reponseChanMap[m.GetChangeFeedID()] = m.HeartbeatResponseQueue - c.eventDispatcherManagerMap[m.GetChangeFeedID()] = m + c.responseChanMap.Set(m.GetChangeFeedID(), m.GetHeartbeatResponseQueue()) + c.eventDispatcherManagerMap.Set(m.GetChangeFeedID(), m) return nil } @@ -110,8 +154,7 @@ func (c *HeartBeatCollector) SendHeartBeatMessages() { } } -/* -func (c *HeartBeatCollector) RecvHeartBeatResponseMessages(msg *messaging.TargetMessage) error { +func (c *HeartBeatCollector) RecvHeartBeatResponseMessages(ctx context.Context, msg *messaging.TargetMessage) error { heartbeatResponse, ok := msg.Message.(*heartbeatpb.HeartBeatResponse) if !ok { log.Error("invalid heartbeat response message", zap.Any("msg", msg)) @@ -119,13 +162,12 @@ func (c *HeartBeatCollector) RecvHeartBeatResponseMessages(msg *messaging.Target } changefeedID := model.DefaultChangeFeedID(heartbeatResponse.ChangefeedID) - c.responseChanMapMutex.RLock() - defer c.responseChanMapMutex.RUnlock() - if queue, ok := c.responseChanMap[changefeedID]; ok { + queue, ok := c.responseChanMap.Get(changefeedID) + if ok { queue.Enqueue(heartbeatResponse) } return nil -}*/ +} func (c *HeartBeatCollector) RecvSchedulerDispatcherRequestMessages(ctx context.Context, msg *messaging.TargetMessage) error { scheduleDispatcherRequest := msg.Message.(*heartbeatpb.ScheduleDispatcherRequest) @@ -144,10 +186,7 @@ func (c *HeartBeatCollector) handleDispatcherRequestMessages(req *heartbeatpb.Sc // start := time.Now() changefeedID := model.DefaultChangeFeedID(req.ChangefeedID) - c.eventDispatcherManagerMutex.RLock() - defer c.eventDispatcherManagerMutex.RUnlock() - - eventDispatcherManager, ok := c.eventDispatcherManagerMap[changefeedID] + eventDispatcherManager, ok := c.eventDispatcherManagerMap.Get(changefeedID) if !ok { // Maybe the message is received before the event dispatcher manager is registered, so just ingore it log.Warn("invalid changefeedID in scheduler dispatcher request message", zap.String("changefeedID", changefeedID.String())) diff --git a/downstreamadapter/sink/mysql_sink.go b/downstreamadapter/sink/mysql_sink.go index c8e17a5fb..9373a0fc7 100644 --- a/downstreamadapter/sink/mysql_sink.go +++ b/downstreamadapter/sink/mysql_sink.go @@ -94,8 +94,7 @@ type MysqlSink struct { conflictDetector *conflictdetector.ConflictDetector // 主要是要保持一样的生命周期?不然 channel 会对应不上 - //ddlWorker *worker.MysqlDDLWorker - //dmlWorkerTasks []*worker.MysqlWorkerDMLEventTask + ddlWorker *worker.MysqlDDLWorker tableStatuses *TableStatusMap wg sync.WaitGroup @@ -126,6 +125,8 @@ func (s *MysqlSink) initWorker(workerCount int, cfg *writer.MysqlConfig, db *sql // dml worker task will deal with all the dml events ctx, cancel := context.WithCancel(context.Background()) + s.ddlWorker = worker.NewMysqlDDLWorker(db, cfg, s.changefeedID) + s.cancel = cancel for i := 0; i < workerCount; i++ { s.wg.Add(1) @@ -204,8 +205,7 @@ func (s *MysqlSink) AddDMLEvent(tableSpan *common.TableSpan, event *common.TxnEv tableStatus.getCh() <- event } -/* -func (s *MysqlSink) AddDDLAndSyncPointEvent(tableSpan *common.TableSpan, event *common.TxnEvent) { // 或许 ddl 也可以考虑有专用的 worker? +func (s *MysqlSink) AddDDLAndSyncPointEvent(tableSpan *common.TableSpan, event *common.TxnEvent) { tableStatus, ok := s.tableStatuses.Get(tableSpan) if !ok { log.Error("unknown Span for Mysql Sink: ", zap.Any("tableSpan", tableSpan)) @@ -214,10 +214,9 @@ func (s *MysqlSink) AddDDLAndSyncPointEvent(tableSpan *common.TableSpan, event * tableStatus.getProgress().Add(event) event.PostTxnFlushed = func() { tableStatus.getProgress().Remove(event) } - task := worker.NewMysqlWorkerDDLEventTask(s.ddlWorker, event) // 先固定用 0 号 worker - threadpool.GetTaskSchedulerInstance().WorkerTaskScheduler.Submit(task, threadpool.IOTask, time.Time{}) - -}*/ + // TODO:这个 ddl 可以并发写么?如果不行的话,后面还要加锁或者排队 + s.ddlWorker.GetMysqlWriter().FlushDDLEvent(event) +} func (s *MysqlSink) AddTableSpan(tableSpan *common.TableSpan) { tableProgress := types.NewTableProgress() @@ -271,7 +270,7 @@ func (s *MysqlSink) IsEmpty(tableSpan *common.TableSpan) bool { return tableStatus.getProgress().Empty() } -func (s *MysqlSink) GetSmallestCommitTs(tableSpan *common.TableSpan) uint64 { +func (s *MysqlSink) GetCheckpointTs(tableSpan *common.TableSpan) uint64 { tableStatus, ok := s.tableStatuses.Get(tableSpan) if !ok { @@ -279,7 +278,7 @@ func (s *MysqlSink) GetSmallestCommitTs(tableSpan *common.TableSpan) uint64 { return math.MaxUint64 } - return tableStatus.getProgress().SmallestCommitTs() + return tableStatus.getProgress().MaxCheckpointTs() } func (s *MysqlSink) Close() { diff --git a/downstreamadapter/sink/sink.go b/downstreamadapter/sink/sink.go index 30efda55b..c39fd0a22 100644 --- a/downstreamadapter/sink/sink.go +++ b/downstreamadapter/sink/sink.go @@ -17,11 +17,11 @@ import "github.com/flowbehappy/tigate/pkg/common" type Sink interface { AddDMLEvent(tableSpan *common.TableSpan, event *common.TxnEvent) - //AddDDLAndSyncPointEvent(tableSpan *common.TableSpan, event *common.TxnEvent) + AddDDLAndSyncPointEvent(tableSpan *common.TableSpan, event *common.TxnEvent) IsEmpty(tableSpan *common.TableSpan) bool AddTableSpan(tableSpan *common.TableSpan) RemoveTableSpan(tableSpan *common.TableSpan) StopTableSpan(tableSpan *common.TableSpan) - GetSmallestCommitTs(tableSpan *common.TableSpan) uint64 + GetCheckpointTs(tableSpan *common.TableSpan) uint64 Close() } diff --git a/downstreamadapter/sink/types/table_progress.go b/downstreamadapter/sink/types/table_progress.go index 66bcf1604..1cdee1e86 100644 --- a/downstreamadapter/sink/types/table_progress.go +++ b/downstreamadapter/sink/types/table_progress.go @@ -27,9 +27,10 @@ import ( // 本质是要频繁的删除随机数据,插入递增数据,查询最小值,后面自己可以实现一个红黑树吧,或者其他结构,先用 list 苟一苟 // thread safe type TableProgress struct { - mutex sync.Mutex - list *list.List - elemMap map[Ts]*list.Element + mutex sync.Mutex + list *list.List + elemMap map[Ts]*list.Element + maxCommitTs uint64 } // 按 commitTs 为主,startTs 为辅排序 @@ -40,8 +41,9 @@ type Ts struct { func NewTableProgress() *TableProgress { tableProgress := &TableProgress{ - list: list.New(), - elemMap: make(map[Ts]*list.Element), + list: list.New(), + elemMap: make(map[Ts]*list.Element), + maxCommitTs: 0, } return tableProgress } @@ -52,6 +54,7 @@ func (p *TableProgress) Add(event *common.TxnEvent) { defer p.mutex.Unlock() elem := p.list.PushBack(ts) p.elemMap[ts] = elem + p.maxCommitTs = event.CommitTs } // 而且删除可以认为是批量的?但要不要做成批量可以后面再看 @@ -71,14 +74,18 @@ func (p *TableProgress) Empty() bool { return p.list.Len() == 0 } -// 返回目前 sink 还没 flush 下去的 event 中 commitTs 最小的。 -// 用于结合当前 dispatcher 收到的 resolvedTs,和 event list 中的 event 值计算 table checkpointTs -func (p *TableProgress) SmallestCommitTs() uint64 { +// 返回当前 tableSpan 中最大的 checkpointTs,也就是最大的 ts,并且 <= ts 之前的数据都已经成功写下去了 +// 1. 假设目前 sink 还有没 flush 下去的 event,就拿最小的这个 event的 commitTs。 +// 2. 反之,则选择收到过 event 中 commitTs 最大的那个。 +func (p *TableProgress) MaxCheckpointTs() uint64 { p.mutex.Lock() defer p.mutex.Unlock() if p.list.Len() == 0 { - return 0 + if p.maxCommitTs == 0 { + return 0 + } + return p.maxCommitTs - 1 } - return p.list.Front().Value.(Ts).commitTs + return p.list.Front().Value.(Ts).commitTs - 1 } diff --git a/downstreamadapter/worker/mysql_worker.go b/downstreamadapter/worker/mysql_worker.go index 4988f0f6f..cf10bed5a 100644 --- a/downstreamadapter/worker/mysql_worker.go +++ b/downstreamadapter/worker/mysql_worker.go @@ -66,149 +66,16 @@ func (t *MysqlWorker) GetID() int { return t.id } -/* type MysqlDDLWorker struct { - MysqlWriter *writer.MysqlWriter // 实际负责做 flush 操作 + mysqlWriter *writer.MysqlWriter } -// 这个 task 是单次出现的,执行完就结束,用于处理 ddl 和 sync point event -type MysqlWorkerDDLEventTask struct { - worker *MysqlDDLWorker - event *common.TxnEvent - taskStatus threadpool.TaskStatus -} - -func NewMysqlWorkerDDLEventTask(worker *MysqlDDLWorker, event *common.TxnEvent) *MysqlWorkerDDLEventTask { - return &MysqlWorkerDDLEventTask{ - worker: worker, - event: event, - taskStatus: threadpool.IOTask, - } -} - -func (t *MysqlWorkerDDLEventTask) GetStatus() threadpool.TaskStatus { - return t.taskStatus -} - -func (t *MysqlWorkerDDLEventTask) SetStatus(taskStatus threadpool.TaskStatus) { - t.taskStatus = taskStatus -} - -func (t *MysqlWorkerDDLEventTask) Execute() (threadpool.TaskStatus, time.Time) { - t.worker.MysqlWriter.FlushDDLEvent(t.event) - return threadpool.Done, time.Time{} -} - -func (t *MysqlWorkerDDLEventTask) Await() threadpool.TaskStatus { - log.Error("MysqlWorkerDDLEventTask should not call await()") - return threadpool.Done -} - -func (t *MysqlWorkerDDLEventTask) Release() { - // -} - -func (t *MysqlWorkerDDLEventTask) Cancel() { - // -} - -// 这个 task 应该是 event dispatcher manager 创建以后,会直接生成好的 task,扔到一个单独的 threadpool 中 -// task 的 生命周期应该是跟 event dispatcher manager 一样 -// 这个 task 只处理 dml event -type MysqlWorkerDMLEventTask struct { - worker *MysqlWorker - maxRows int - events []*common.TxnEvent - taskHandle *threadpool.TaskHandle -} - -func NewMysqlWorkerDMLEventTask(eventChan <-chan *common.TxnEvent, db *sql.DB, config *writer.MysqlConfig, maxRows int) *MysqlWorkerDMLEventTask { - task := &MysqlWorkerDMLEventTask{ - worker: &MysqlWorker{ - eventChan: eventChan, - mysqlWriter: writer.NewMysqlWriter(db, config), - }, - maxRows: maxRows, - } - task.taskHandle = threadpool.GetTaskSchedulerInstance().WorkerTaskScheduler.Submit(task, threadpool.CPUTask, time.Time{}) - return task -} - -func (t *MysqlWorkerDMLEventTask) Execute() (threadpool.TaskStatus, time.Time) { - switch t.taskStatus { - case threadpool.CPUTask: - return t.executeImpl() - case threadpool.IOTask: - return t.executeIOImpl() - default: - log.Error("Unexpected task status: ", zap.Any("status", t.taskStatus)) - return threadpool.Done, time.Time{} - } -} - -func (t *MysqlWorkerDMLEventTask) executeIOImpl() (threadpool.TaskStatus, time.Time) { - if len(t.events) == 0 { - log.Warn("here is no events to flush") - return threadpool.CPUTask, time.Time{} - } - // flush events - err := t.worker.mysqlWriter.Flush(t.events) - if err != nil { - log.Error("Failed to flush events", zap.Error(err)) - return threadpool.Done, time.Time{} - } - t.events = nil - return threadpool.CPUTask, time.Time{} -} - -func (t *MysqlWorkerDMLEventTask) executeImpl() (threadpool.TaskStatus, time.Time) { - // check events is empty - if len(t.events) > 0 { - log.Error("events is not empty in MysqlWorkerTask") - return threadpool.Done, time.Time{} - } - rows := 0 - - // check if channel has events, if not directly back to taskQueue - select { - case txnEvent := <-t.worker.eventChan: - t.events = append(t.events, txnEvent) - rows += len(txnEvent.Rows) - if rows >= t.maxRows { - return threadpool.IOTask, time.Time{} - } - default: - return threadpool.CPUTask, time.Time{} - } - - // get enough events or wait for 10 millseconds to make task go to IO State. -- 这边可以考虑到底是拿不到 event 就 换出去 flush 好还是要等好,具体等多久好 - // 这边甚至可以想一下是不是不用等 10ms,没数据就直接刷下去,flush 时间远超过这个攒批时间的 - for { - select { - case txnEvent := <-t.worker.eventChan: - t.events = append(t.events, txnEvent) - rows += len(txnEvent.Rows) - if rows >= t.maxRows { - return threadpool.IOTask, time.Time{} - } - default: - return threadpool.IOTask, time.Time{} - } +func NewMysqlDDLWorker(db *sql.DB, config *writer.MysqlConfig, changefeedID model.ChangeFeedID) *MysqlDDLWorker { + return &MysqlDDLWorker{ + mysqlWriter: writer.NewMysqlWriter(db, config, changefeedID), } } -func (t *MysqlWorkerDMLEventTask) Await() threadpool.TaskStatus { - log.Error("MysqlWorkerTask should not call await()") - return threadpool.Done -} - -// 只有重启或者出问题的时候才 release -func (t *MysqlWorkerDMLEventTask) Release() { - // 直接关闭应该就可以把? - // TODO:需要取出 events 么 - // 不知道要干嘛,不干会咋样么 -} - -func (t *MysqlWorkerDMLEventTask) Cancel() { +func (t *MysqlDDLWorker) GetMysqlWriter() *writer.MysqlWriter { + return t.mysqlWriter } -*/ diff --git a/heartbeatpb/heartbeat.pb.go b/heartbeatpb/heartbeat.pb.go index 56a9d28f6..fcb84c3e5 100644 --- a/heartbeatpb/heartbeat.pb.go +++ b/heartbeatpb/heartbeat.pb.go @@ -25,21 +25,18 @@ const _ = proto.GoGoProtoPackageIsVersion3 // please upgrade the proto package type Action int32 const ( - Action_None Action = 0 - Action_Write Action = 1 - Action_Pass Action = 2 + Action_Write Action = 0 + Action_Pass Action = 1 ) var Action_name = map[int32]string{ - 0: "None", - 1: "Write", - 2: "Pass", + 0: "Write", + 1: "Pass", } var Action_value = map[string]int32{ - "None": 0, - "Write": 1, - "Pass": 2, + "Write": 0, + "Pass": 1, } func (x Action) String() string { @@ -311,25 +308,24 @@ func (m *Watermark) GetResolvedTs() uint64 { return 0 } -type TableProgress struct { - Span *TableSpan `protobuf:"bytes,1,opt,name=span,proto3" json:"span,omitempty"` - IsBlocked bool `protobuf:"varint,2,opt,name=IsBlocked,proto3" json:"IsBlocked,omitempty"` - BlockTs uint64 `protobuf:"varint,3,opt,name=BlockTs,proto3" json:"BlockTs,omitempty"` - CheckpointTs uint64 `protobuf:"varint,4,opt,name=CheckpointTs,proto3" json:"CheckpointTs,omitempty"` +type DispatcherAction struct { + Span *TableSpan `protobuf:"bytes,1,opt,name=span,proto3" json:"span,omitempty"` + Action Action `protobuf:"varint,2,opt,name=action,proto3,enum=heartbeatpb.Action" json:"action,omitempty"` + CommitTs uint64 `protobuf:"varint,3,opt,name=CommitTs,proto3" json:"CommitTs,omitempty"` } -func (m *TableProgress) Reset() { *m = TableProgress{} } -func (m *TableProgress) String() string { return proto.CompactTextString(m) } -func (*TableProgress) ProtoMessage() {} -func (*TableProgress) Descriptor() ([]byte, []int) { +func (m *DispatcherAction) Reset() { *m = DispatcherAction{} } +func (m *DispatcherAction) String() string { return proto.CompactTextString(m) } +func (*DispatcherAction) ProtoMessage() {} +func (*DispatcherAction) Descriptor() ([]byte, []int) { return fileDescriptor_6d584080fdadb670, []int{3} } -func (m *TableProgress) XXX_Unmarshal(b []byte) error { +func (m *DispatcherAction) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) } -func (m *TableProgress) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { +func (m *DispatcherAction) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { if deterministic { - return xxx_messageInfo_TableProgress.Marshal(b, m, deterministic) + return xxx_messageInfo_DispatcherAction.Marshal(b, m, deterministic) } else { b = b[:cap(b)] n, err := m.MarshalToSizedBuffer(b) @@ -339,125 +335,49 @@ func (m *TableProgress) XXX_Marshal(b []byte, deterministic bool) ([]byte, error return b[:n], nil } } -func (m *TableProgress) XXX_Merge(src proto.Message) { - xxx_messageInfo_TableProgress.Merge(m, src) +func (m *DispatcherAction) XXX_Merge(src proto.Message) { + xxx_messageInfo_DispatcherAction.Merge(m, src) } -func (m *TableProgress) XXX_Size() int { +func (m *DispatcherAction) XXX_Size() int { return m.Size() } -func (m *TableProgress) XXX_DiscardUnknown() { - xxx_messageInfo_TableProgress.DiscardUnknown(m) +func (m *DispatcherAction) XXX_DiscardUnknown() { + xxx_messageInfo_DispatcherAction.DiscardUnknown(m) } -var xxx_messageInfo_TableProgress proto.InternalMessageInfo +var xxx_messageInfo_DispatcherAction proto.InternalMessageInfo -func (m *TableProgress) GetSpan() *TableSpan { +func (m *DispatcherAction) GetSpan() *TableSpan { if m != nil { return m.Span } return nil } -func (m *TableProgress) GetIsBlocked() bool { - if m != nil { - return m.IsBlocked - } - return false -} - -func (m *TableProgress) GetBlockTs() uint64 { - if m != nil { - return m.BlockTs - } - return 0 -} - -func (m *TableProgress) GetCheckpointTs() uint64 { - if m != nil { - return m.CheckpointTs - } - return 0 -} - -// 这名字重新取,我真的要吐了 -type TableProgressInfo struct { - Span *TableSpan `protobuf:"bytes,1,opt,name=span,proto3" json:"span,omitempty"` - TableProgresses []*TableProgress `protobuf:"bytes,2,rep,name=tableProgresses,proto3" json:"tableProgresses,omitempty"` - Action Action `protobuf:"varint,3,opt,name=action,proto3,enum=heartbeatpb.Action" json:"action,omitempty"` - SchedulerStatus ComponentState `protobuf:"varint,4,opt,name=scheduler_status,json=schedulerStatus,proto3,enum=heartbeatpb.ComponentState" json:"scheduler_status,omitempty"` -} - -func (m *TableProgressInfo) Reset() { *m = TableProgressInfo{} } -func (m *TableProgressInfo) String() string { return proto.CompactTextString(m) } -func (*TableProgressInfo) ProtoMessage() {} -func (*TableProgressInfo) Descriptor() ([]byte, []int) { - return fileDescriptor_6d584080fdadb670, []int{4} -} -func (m *TableProgressInfo) XXX_Unmarshal(b []byte) error { - return m.Unmarshal(b) -} -func (m *TableProgressInfo) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { - if deterministic { - return xxx_messageInfo_TableProgressInfo.Marshal(b, m, deterministic) - } else { - b = b[:cap(b)] - n, err := m.MarshalToSizedBuffer(b) - if err != nil { - return nil, err - } - return b[:n], nil - } -} -func (m *TableProgressInfo) XXX_Merge(src proto.Message) { - xxx_messageInfo_TableProgressInfo.Merge(m, src) -} -func (m *TableProgressInfo) XXX_Size() int { - return m.Size() -} -func (m *TableProgressInfo) XXX_DiscardUnknown() { - xxx_messageInfo_TableProgressInfo.DiscardUnknown(m) -} - -var xxx_messageInfo_TableProgressInfo proto.InternalMessageInfo - -func (m *TableProgressInfo) GetSpan() *TableSpan { - if m != nil { - return m.Span - } - return nil -} - -func (m *TableProgressInfo) GetTableProgresses() []*TableProgress { - if m != nil { - return m.TableProgresses - } - return nil -} - -func (m *TableProgressInfo) GetAction() Action { +func (m *DispatcherAction) GetAction() Action { if m != nil { return m.Action } - return Action_None + return Action_Write } -func (m *TableProgressInfo) GetSchedulerStatus() ComponentState { +func (m *DispatcherAction) GetCommitTs() uint64 { if m != nil { - return m.SchedulerStatus + return m.CommitTs } - return ComponentState_Unknown + return 0 } type HeartBeatResponse struct { - Info []*TableProgressInfo `protobuf:"bytes,1,rep,name=info,proto3" json:"info,omitempty"` - ChangefeedID string `protobuf:"bytes,2,opt,name=changefeedID,proto3" json:"changefeedID,omitempty"` + Actions []*DispatcherAction `protobuf:"bytes,1,rep,name=actions,proto3" json:"actions,omitempty"` + ChangefeedID string `protobuf:"bytes,2,opt,name=changefeedID,proto3" json:"changefeedID,omitempty"` } func (m *HeartBeatResponse) Reset() { *m = HeartBeatResponse{} } func (m *HeartBeatResponse) String() string { return proto.CompactTextString(m) } func (*HeartBeatResponse) ProtoMessage() {} func (*HeartBeatResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_6d584080fdadb670, []int{5} + return fileDescriptor_6d584080fdadb670, []int{4} } func (m *HeartBeatResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -486,9 +406,9 @@ func (m *HeartBeatResponse) XXX_DiscardUnknown() { var xxx_messageInfo_HeartBeatResponse proto.InternalMessageInfo -func (m *HeartBeatResponse) GetInfo() []*TableProgressInfo { +func (m *HeartBeatResponse) GetActions() []*DispatcherAction { if m != nil { - return m.Info + return m.Actions } return nil } @@ -509,7 +429,7 @@ func (m *DispatcherConfig) Reset() { *m = DispatcherConfig{} } func (m *DispatcherConfig) String() string { return proto.CompactTextString(m) } func (*DispatcherConfig) ProtoMessage() {} func (*DispatcherConfig) Descriptor() ([]byte, []int) { - return fileDescriptor_6d584080fdadb670, []int{6} + return fileDescriptor_6d584080fdadb670, []int{5} } func (m *DispatcherConfig) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -563,7 +483,7 @@ func (m *ScheduleDispatcherRequest) Reset() { *m = ScheduleDispatcherReq func (m *ScheduleDispatcherRequest) String() string { return proto.CompactTextString(m) } func (*ScheduleDispatcherRequest) ProtoMessage() {} func (*ScheduleDispatcherRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_6d584080fdadb670, []int{7} + return fileDescriptor_6d584080fdadb670, []int{6} } func (m *ScheduleDispatcherRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -628,7 +548,7 @@ func (m *MaintainerHeartbeat) Reset() { *m = MaintainerHeartbeat{} } func (m *MaintainerHeartbeat) String() string { return proto.CompactTextString(m) } func (*MaintainerHeartbeat) ProtoMessage() {} func (*MaintainerHeartbeat) Descriptor() ([]byte, []int) { - return fileDescriptor_6d584080fdadb670, []int{8} + return fileDescriptor_6d584080fdadb670, []int{7} } func (m *MaintainerHeartbeat) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -677,7 +597,7 @@ func (m *MaintainerStatus) Reset() { *m = MaintainerStatus{} } func (m *MaintainerStatus) String() string { return proto.CompactTextString(m) } func (*MaintainerStatus) ProtoMessage() {} func (*MaintainerStatus) Descriptor() ([]byte, []int) { - return fileDescriptor_6d584080fdadb670, []int{9} + return fileDescriptor_6d584080fdadb670, []int{8} } func (m *MaintainerStatus) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -756,7 +676,7 @@ func (m *CoordinatorBootstrapRequest) Reset() { *m = CoordinatorBootstra func (m *CoordinatorBootstrapRequest) String() string { return proto.CompactTextString(m) } func (*CoordinatorBootstrapRequest) ProtoMessage() {} func (*CoordinatorBootstrapRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_6d584080fdadb670, []int{10} + return fileDescriptor_6d584080fdadb670, []int{9} } func (m *CoordinatorBootstrapRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -800,7 +720,7 @@ func (m *CoordinatorBootstrapResponse) Reset() { *m = CoordinatorBootstr func (m *CoordinatorBootstrapResponse) String() string { return proto.CompactTextString(m) } func (*CoordinatorBootstrapResponse) ProtoMessage() {} func (*CoordinatorBootstrapResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_6d584080fdadb670, []int{11} + return fileDescriptor_6d584080fdadb670, []int{10} } func (m *CoordinatorBootstrapResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -845,7 +765,7 @@ func (m *DispatchMaintainerRequest) Reset() { *m = DispatchMaintainerReq func (m *DispatchMaintainerRequest) String() string { return proto.CompactTextString(m) } func (*DispatchMaintainerRequest) ProtoMessage() {} func (*DispatchMaintainerRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_6d584080fdadb670, []int{12} + return fileDescriptor_6d584080fdadb670, []int{11} } func (m *DispatchMaintainerRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -899,7 +819,7 @@ func (m *AddMaintainerRequest) Reset() { *m = AddMaintainerRequest{} } func (m *AddMaintainerRequest) String() string { return proto.CompactTextString(m) } func (*AddMaintainerRequest) ProtoMessage() {} func (*AddMaintainerRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_6d584080fdadb670, []int{13} + return fileDescriptor_6d584080fdadb670, []int{12} } func (m *AddMaintainerRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -965,7 +885,7 @@ func (m *RemoveMaintainerRequest) Reset() { *m = RemoveMaintainerRequest func (m *RemoveMaintainerRequest) String() string { return proto.CompactTextString(m) } func (*RemoveMaintainerRequest) ProtoMessage() {} func (*RemoveMaintainerRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_6d584080fdadb670, []int{14} + return fileDescriptor_6d584080fdadb670, []int{13} } func (m *RemoveMaintainerRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1017,7 +937,7 @@ func (m *MaintainerBootstrapRequest) Reset() { *m = MaintainerBootstrapR func (m *MaintainerBootstrapRequest) String() string { return proto.CompactTextString(m) } func (*MaintainerBootstrapRequest) ProtoMessage() {} func (*MaintainerBootstrapRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_6d584080fdadb670, []int{15} + return fileDescriptor_6d584080fdadb670, []int{14} } func (m *MaintainerBootstrapRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1069,7 +989,7 @@ func (m *MaintainerBootstrapResponse) Reset() { *m = MaintainerBootstrap func (m *MaintainerBootstrapResponse) String() string { return proto.CompactTextString(m) } func (*MaintainerBootstrapResponse) ProtoMessage() {} func (*MaintainerBootstrapResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_6d584080fdadb670, []int{16} + return fileDescriptor_6d584080fdadb670, []int{15} } func (m *MaintainerBootstrapResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1120,7 +1040,7 @@ func (m *MaintainerCloseRequest) Reset() { *m = MaintainerCloseRequest{} func (m *MaintainerCloseRequest) String() string { return proto.CompactTextString(m) } func (*MaintainerCloseRequest) ProtoMessage() {} func (*MaintainerCloseRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_6d584080fdadb670, []int{17} + return fileDescriptor_6d584080fdadb670, []int{16} } func (m *MaintainerCloseRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1165,7 +1085,7 @@ func (m *MaintainerCloseResponse) Reset() { *m = MaintainerCloseResponse func (m *MaintainerCloseResponse) String() string { return proto.CompactTextString(m) } func (*MaintainerCloseResponse) ProtoMessage() {} func (*MaintainerCloseResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_6d584080fdadb670, []int{18} + return fileDescriptor_6d584080fdadb670, []int{17} } func (m *MaintainerCloseResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1208,9 +1128,86 @@ func (m *MaintainerCloseResponse) GetSuccess() bool { return false } +type State struct { + IsBlocked bool `protobuf:"varint,1,opt,name=IsBlocked,proto3" json:"IsBlocked,omitempty"` + BlockTs uint64 `protobuf:"varint,2,opt,name=BlockTs,proto3" json:"BlockTs,omitempty"` + BlockTableSpan []*TableSpan `protobuf:"bytes,3,rep,name=BlockTableSpan,proto3" json:"BlockTableSpan,omitempty"` + NeedDroppedTableSpan []*TableSpan `protobuf:"bytes,4,rep,name=NeedDroppedTableSpan,proto3" json:"NeedDroppedTableSpan,omitempty"` + NeedAddedTableSpan []*TableSpan `protobuf:"bytes,5,rep,name=NeedAddedTableSpan,proto3" json:"NeedAddedTableSpan,omitempty"` +} + +func (m *State) Reset() { *m = State{} } +func (m *State) String() string { return proto.CompactTextString(m) } +func (*State) ProtoMessage() {} +func (*State) Descriptor() ([]byte, []int) { + return fileDescriptor_6d584080fdadb670, []int{18} +} +func (m *State) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *State) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_State.Marshal(b, m, deterministic) + } else { + b = b[:cap(b)] + n, err := m.MarshalToSizedBuffer(b) + if err != nil { + return nil, err + } + return b[:n], nil + } +} +func (m *State) XXX_Merge(src proto.Message) { + xxx_messageInfo_State.Merge(m, src) +} +func (m *State) XXX_Size() int { + return m.Size() +} +func (m *State) XXX_DiscardUnknown() { + xxx_messageInfo_State.DiscardUnknown(m) +} + +var xxx_messageInfo_State proto.InternalMessageInfo + +func (m *State) GetIsBlocked() bool { + if m != nil { + return m.IsBlocked + } + return false +} + +func (m *State) GetBlockTs() uint64 { + if m != nil { + return m.BlockTs + } + return 0 +} + +func (m *State) GetBlockTableSpan() []*TableSpan { + if m != nil { + return m.BlockTableSpan + } + return nil +} + +func (m *State) GetNeedDroppedTableSpan() []*TableSpan { + if m != nil { + return m.NeedDroppedTableSpan + } + return nil +} + +func (m *State) GetNeedAddedTableSpan() []*TableSpan { + if m != nil { + return m.NeedAddedTableSpan + } + return nil +} + type TableSpanStatus struct { Span *TableSpan `protobuf:"bytes,1,opt,name=span,proto3" json:"span,omitempty"` ComponentStatus ComponentState `protobuf:"varint,2,opt,name=component_status,json=componentStatus,proto3,enum=heartbeatpb.ComponentState" json:"component_status,omitempty"` + State *State `protobuf:"bytes,3,opt,name=state,proto3" json:"state,omitempty"` CheckpointTs uint64 `protobuf:"varint,4,opt,name=checkpoint_ts,json=checkpointTs,proto3" json:"checkpoint_ts,omitempty"` } @@ -1261,6 +1258,13 @@ func (m *TableSpanStatus) GetComponentStatus() ComponentState { return ComponentState_Unknown } +func (m *TableSpanStatus) GetState() *State { + if m != nil { + return m.State + } + return nil +} + func (m *TableSpanStatus) GetCheckpointTs() uint64 { if m != nil { return m.CheckpointTs @@ -1343,8 +1347,7 @@ func init() { proto.RegisterType((*TableSpan)(nil), "heartbeatpb.TableSpan") proto.RegisterType((*HeartBeatRequest)(nil), "heartbeatpb.HeartBeatRequest") proto.RegisterType((*Watermark)(nil), "heartbeatpb.Watermark") - proto.RegisterType((*TableProgress)(nil), "heartbeatpb.TableProgress") - proto.RegisterType((*TableProgressInfo)(nil), "heartbeatpb.TableProgressInfo") + proto.RegisterType((*DispatcherAction)(nil), "heartbeatpb.DispatcherAction") proto.RegisterType((*HeartBeatResponse)(nil), "heartbeatpb.HeartBeatResponse") proto.RegisterType((*DispatcherConfig)(nil), "heartbeatpb.DispatcherConfig") proto.RegisterType((*ScheduleDispatcherRequest)(nil), "heartbeatpb.ScheduleDispatcherRequest") @@ -1359,6 +1362,7 @@ func init() { proto.RegisterType((*MaintainerBootstrapResponse)(nil), "heartbeatpb.MaintainerBootstrapResponse") proto.RegisterType((*MaintainerCloseRequest)(nil), "heartbeatpb.MaintainerCloseRequest") proto.RegisterType((*MaintainerCloseResponse)(nil), "heartbeatpb.MaintainerCloseResponse") + proto.RegisterType((*State)(nil), "heartbeatpb.State") proto.RegisterType((*TableSpanStatus)(nil), "heartbeatpb.TableSpanStatus") proto.RegisterType((*RunningError)(nil), "heartbeatpb.RunningError") } @@ -1366,78 +1370,79 @@ func init() { func init() { proto.RegisterFile("heartbeatpb/heartbeat.proto", fileDescriptor_6d584080fdadb670) } var fileDescriptor_6d584080fdadb670 = []byte{ - // 1133 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xa4, 0x57, 0x4d, 0x6f, 0x1b, 0xc5, - 0x1b, 0xcf, 0xda, 0x8e, 0x5f, 0x9e, 0xb8, 0xce, 0x76, 0xfa, 0x57, 0xea, 0xbc, 0xd4, 0x4a, 0xf7, - 0x8f, 0x84, 0x49, 0xa5, 0x44, 0xa4, 0x20, 0x40, 0xe2, 0x92, 0x38, 0xad, 0x1a, 0x10, 0x10, 0xad, - 0x83, 0xd2, 0x72, 0x89, 0xc6, 0xbb, 0x4f, 0xec, 0x95, 0xed, 0x99, 0x65, 0x66, 0x9c, 0x28, 0xe2, - 0xce, 0x15, 0x6e, 0x7c, 0x07, 0x8e, 0x9c, 0xf9, 0x00, 0x1c, 0x7b, 0xe4, 0x84, 0x50, 0xf2, 0x2d, - 0x38, 0xa1, 0x99, 0xdd, 0xb5, 0x77, 0xed, 0x44, 0xb5, 0xc5, 0x6d, 0x9e, 0xd7, 0xf9, 0x3d, 0xaf, - 0xb3, 0x0b, 0x9b, 0x3d, 0xa4, 0x42, 0x75, 0x90, 0xaa, 0xb0, 0xb3, 0x37, 0x3e, 0xef, 0x86, 0x82, - 0x2b, 0x4e, 0x56, 0x52, 0x42, 0xe7, 0x0d, 0x54, 0x4e, 0x69, 0x67, 0x80, 0xed, 0x90, 0x32, 0x52, - 0x87, 0x92, 0x21, 0x8e, 0x8f, 0xea, 0xd6, 0xb6, 0xd5, 0x2c, 0xb8, 0x09, 0x49, 0x36, 0xa0, 0xdc, - 0x56, 0x54, 0xa8, 0x2f, 0xf1, 0xba, 0x9e, 0xdb, 0xb6, 0x9a, 0x55, 0x77, 0x4c, 0x93, 0x35, 0x28, - 0xbe, 0x60, 0xbe, 0x96, 0xe4, 0x8d, 0x24, 0xa6, 0x9c, 0xdf, 0x72, 0x60, 0xbf, 0xd2, 0x57, 0x1d, - 0x22, 0x55, 0x2e, 0x7e, 0x3f, 0x42, 0xa9, 0x88, 0x03, 0x55, 0xaf, 0x47, 0x59, 0x17, 0x2f, 0x10, - 0xfd, 0xf8, 0x9e, 0x8a, 0x9b, 0xe1, 0x91, 0x8f, 0xa0, 0x72, 0x45, 0x15, 0x8a, 0x21, 0x15, 0x7d, - 0x73, 0xdb, 0xca, 0xfe, 0xda, 0x6e, 0x0a, 0xf4, 0xee, 0x59, 0x22, 0x75, 0x27, 0x8a, 0xe4, 0x53, - 0x28, 0x4b, 0x45, 0xd5, 0x48, 0xa2, 0xac, 0xe7, 0xb7, 0xf3, 0xcd, 0x95, 0xfd, 0xad, 0x8c, 0xd1, - 0x38, 0xcc, 0xb6, 0xd1, 0x72, 0xc7, 0xda, 0xa4, 0x09, 0xab, 0x1e, 0x1f, 0x86, 0x38, 0x40, 0x85, - 0x91, 0xb0, 0x5e, 0xd8, 0xb6, 0x9a, 0x65, 0x77, 0x9a, 0x4d, 0x9e, 0x43, 0xe9, 0x8a, 0x0a, 0x16, - 0xb0, 0x6e, 0x7d, 0xd9, 0xe0, 0x5a, 0xcf, 0x5c, 0xe1, 0x8e, 0x98, 0x96, 0xbd, 0x10, 0x82, 0x0b, - 0x37, 0xd1, 0x24, 0xcf, 0x20, 0x8f, 0x42, 0xd4, 0x8b, 0xef, 0x32, 0xd0, 0x5a, 0xce, 0x37, 0x50, - 0x19, 0x47, 0x17, 0x25, 0x0b, 0xbd, 0x7e, 0xc8, 0x03, 0xa6, 0x4e, 0x65, 0x5c, 0x94, 0x0c, 0x8f, - 0x34, 0x00, 0x04, 0x4a, 0x3e, 0xb8, 0x44, 0xff, 0x54, 0x9a, 0x6c, 0x15, 0xdc, 0x14, 0xc7, 0xf9, - 0xc5, 0x82, 0x07, 0x26, 0xf4, 0x13, 0xc1, 0xbb, 0x02, 0xa5, 0x24, 0x3b, 0x50, 0x90, 0x21, 0x65, - 0xc6, 0xdb, 0x74, 0x66, 0xc7, 0x49, 0x72, 0x8d, 0x0e, 0xd9, 0x82, 0xca, 0xb1, 0x3c, 0x1c, 0x70, - 0xaf, 0x8f, 0xbe, 0x71, 0x5e, 0x76, 0x27, 0x0c, 0xdd, 0x2f, 0xe6, 0x78, 0x2a, 0x4d, 0xe9, 0x0b, - 0x6e, 0x42, 0x6a, 0xe4, 0xad, 0x34, 0xf2, 0x42, 0x84, 0x3c, 0xcd, 0x73, 0xfe, 0xb1, 0xe0, 0x61, - 0x06, 0xd9, 0x31, 0xbb, 0xe0, 0x0b, 0xa1, 0x3b, 0x82, 0x55, 0x95, 0x76, 0x80, 0x3a, 0x01, 0xba, - 0xf2, 0x1b, 0xb3, 0x66, 0x89, 0x8e, 0x3b, 0x6d, 0x42, 0x9e, 0x41, 0x91, 0x7a, 0x2a, 0xe0, 0xcc, - 0x04, 0x51, 0xdb, 0x7f, 0x94, 0x31, 0x3e, 0x30, 0x22, 0x37, 0x56, 0x21, 0x2f, 0xc1, 0x96, 0x5e, - 0x0f, 0xfd, 0xd1, 0x00, 0xc5, 0xb9, 0x9c, 0x34, 0x4b, 0x6d, 0x7f, 0x33, 0x63, 0xd6, 0xe2, 0xc3, - 0x90, 0x33, 0x64, 0x4a, 0x77, 0x0e, 0xba, 0xab, 0x63, 0xa3, 0xa8, 0x93, 0x9c, 0x3e, 0x3c, 0x4c, - 0xcd, 0x86, 0x0c, 0x39, 0x93, 0x48, 0xf6, 0xa1, 0x10, 0xb0, 0x0b, 0x5e, 0xb7, 0x4c, 0x10, 0x8d, - 0xfb, 0x83, 0xd0, 0x99, 0x72, 0x8d, 0xee, 0xcc, 0x40, 0xe5, 0x66, 0x07, 0xca, 0x79, 0x0d, 0xf6, - 0x51, 0x20, 0x43, 0xaa, 0xbc, 0x1e, 0x8a, 0x16, 0x67, 0x17, 0x41, 0x77, 0xa1, 0x3c, 0xd7, 0xa1, - 0x24, 0xf5, 0xb4, 0x8f, 0x1b, 0x2c, 0x21, 0x9d, 0xbf, 0x2c, 0x58, 0x6f, 0xc7, 0xa1, 0x4d, 0xae, - 0x58, 0x64, 0xd8, 0x3f, 0x86, 0xa2, 0x67, 0x10, 0xc5, 0x93, 0xfe, 0x24, 0x83, 0x64, 0x1a, 0xb6, - 0x1b, 0x2b, 0x93, 0x16, 0xd4, 0x92, 0x94, 0x1e, 0xa4, 0x8b, 0x97, 0xad, 0x42, 0x3b, 0xa3, 0xe2, - 0x4e, 0x99, 0x90, 0xa7, 0x50, 0x0d, 0xe4, 0xb9, 0x44, 0x8f, 0x33, 0x9f, 0x8a, 0xeb, 0x78, 0xea, - 0x57, 0x02, 0xd9, 0x4e, 0x58, 0xce, 0x09, 0x3c, 0xfa, 0x8a, 0x06, 0x4c, 0xd1, 0x80, 0xa1, 0x78, - 0x95, 0xb8, 0x26, 0x9f, 0xa5, 0x96, 0x4d, 0x54, 0xad, 0x2c, 0xee, 0x89, 0xcd, 0xf4, 0xb6, 0x71, - 0x7e, 0xca, 0x81, 0x3d, 0x2d, 0x9e, 0x2b, 0x53, 0x4f, 0x00, 0xf4, 0xc9, 0x74, 0x1d, 0xc6, 0x75, - 0xae, 0x68, 0x8e, 0x69, 0x31, 0xf2, 0x21, 0x2c, 0x47, 0x92, 0xfc, 0xbb, 0xdb, 0x31, 0xd2, 0x24, - 0xff, 0x87, 0x07, 0x93, 0x5d, 0x72, 0xae, 0xc6, 0x63, 0x9a, 0x59, 0x30, 0x99, 0x9d, 0x97, 0x5f, - 0x74, 0xe7, 0xe5, 0xe7, 0xd8, 0x79, 0x9f, 0xc0, 0x66, 0x8b, 0x73, 0xe1, 0x07, 0x8c, 0x2a, 0x2e, - 0x0e, 0x39, 0x57, 0x52, 0x09, 0x1a, 0x26, 0x5d, 0x54, 0x87, 0xd2, 0x25, 0x0a, 0xa9, 0x6b, 0xac, - 0xd3, 0x92, 0x77, 0x13, 0xd2, 0x79, 0x03, 0x5b, 0x77, 0x1b, 0xc6, 0xf3, 0xf4, 0x1f, 0xaa, 0xf4, - 0xbb, 0x05, 0xeb, 0x49, 0xf3, 0x4d, 0xd4, 0x12, 0x48, 0x5f, 0xc0, 0x2a, 0xf5, 0xfd, 0xf3, 0xe1, - 0x58, 0x90, 0xf8, 0x7f, 0x9a, 0xdd, 0x1d, 0xbe, 0x3f, 0x63, 0xeb, 0xd6, 0x68, 0x9a, 0x2b, 0x49, - 0x1b, 0x88, 0xc0, 0x21, 0xbf, 0xc4, 0x8c, 0xbb, 0x68, 0x8f, 0xbd, 0x97, 0xcd, 0x9c, 0x51, 0x9b, - 0xf5, 0xf8, 0x50, 0x4c, 0x09, 0xa4, 0xf3, 0xa3, 0x05, 0xff, 0xbb, 0xeb, 0x76, 0x52, 0x83, 0x5c, - 0xe0, 0xc7, 0xed, 0x95, 0x0b, 0x7c, 0xfd, 0x78, 0xa7, 0xc6, 0xaf, 0x3a, 0x9e, 0xaf, 0xe9, 0xd1, - 0xc8, 0xcf, 0x8c, 0xc6, 0x5c, 0xdd, 0xe3, 0xb4, 0xe0, 0xf1, 0x3d, 0xb0, 0x67, 0xa0, 0xd4, 0xa1, - 0xe4, 0x51, 0xe9, 0x51, 0x1f, 0xe3, 0x97, 0x26, 0x21, 0x9d, 0xd7, 0xb0, 0x31, 0x31, 0x9f, 0xe9, - 0x8f, 0x79, 0x66, 0xe7, 0x9e, 0x30, 0x9d, 0x1f, 0x60, 0xf3, 0x4e, 0xcf, 0x71, 0x03, 0xcd, 0xe3, - 0x3a, 0xfd, 0xdd, 0x91, 0x5b, 0xe4, 0xbb, 0xc3, 0xf9, 0x1c, 0xd6, 0x26, 0x97, 0xb7, 0x06, 0x5c, - 0xe2, 0x02, 0x21, 0x39, 0x67, 0xf0, 0x78, 0xc6, 0x7a, 0x01, 0xd8, 0x7a, 0xa7, 0x8f, 0x3c, 0x0f, - 0xa5, 0x4c, 0xb2, 0x1d, 0x93, 0xce, 0xaf, 0x16, 0xac, 0x4e, 0x81, 0x5e, 0xe8, 0xb5, 0x78, 0x09, - 0xb6, 0x97, 0xac, 0x9b, 0xe4, 0x89, 0xcc, 0xcd, 0xf1, 0x44, 0x7a, 0x69, 0x7a, 0x24, 0xe7, 0xeb, - 0x2f, 0x1f, 0xaa, 0xe9, 0x85, 0x42, 0x08, 0x14, 0x54, 0x30, 0xc4, 0x38, 0x64, 0x73, 0xd6, 0x3c, - 0xc6, 0xfd, 0x64, 0x65, 0x9a, 0xb3, 0xe6, 0x79, 0x9a, 0x97, 0x8f, 0x78, 0xfa, 0xac, 0x53, 0x32, - 0x44, 0x29, 0x69, 0x17, 0xcd, 0x55, 0x15, 0x37, 0x21, 0x77, 0xde, 0x87, 0x62, 0xfc, 0x64, 0x94, - 0xa1, 0xf0, 0x35, 0x67, 0x68, 0x2f, 0x91, 0x0a, 0x2c, 0x9f, 0x89, 0x40, 0xa1, 0x6d, 0x69, 0xe6, - 0x09, 0x95, 0xd2, 0xce, 0xed, 0x34, 0xa1, 0x96, 0x7d, 0x73, 0x08, 0x40, 0xb1, 0x25, 0x90, 0x2a, - 0x6d, 0x02, 0x50, 0x8c, 0x86, 0xc1, 0xb6, 0x76, 0x18, 0xd4, 0xb2, 0x09, 0x20, 0x2b, 0x50, 0xfa, - 0x96, 0xf5, 0x19, 0xbf, 0x62, 0x91, 0xea, 0x41, 0x47, 0x22, 0x53, 0xb6, 0x45, 0x1e, 0x40, 0xe5, - 0x44, 0x60, 0x48, 0x45, 0xc0, 0xba, 0x76, 0x8e, 0x54, 0xa1, 0x1c, 0x91, 0xe8, 0xdb, 0x79, 0x6d, - 0x75, 0xc6, 0x45, 0x5f, 0x8b, 0x0a, 0x5a, 0xd4, 0x56, 0x3c, 0x0c, 0x35, 0xb5, 0xac, 0x45, 0x86, - 0x42, 0xdf, 0x2e, 0x1e, 0xb6, 0xfe, 0xb8, 0x69, 0x58, 0x6f, 0x6f, 0x1a, 0xd6, 0xdf, 0x37, 0x0d, - 0xeb, 0xe7, 0xdb, 0xc6, 0xd2, 0xdb, 0xdb, 0xc6, 0xd2, 0x9f, 0xb7, 0x8d, 0xa5, 0xef, 0x3e, 0xe8, - 0x06, 0xaa, 0x37, 0xea, 0xec, 0x7a, 0x7c, 0xb8, 0x77, 0x31, 0xe0, 0x57, 0x1d, 0xec, 0xd1, 0x30, - 0xbc, 0xde, 0x53, 0x41, 0x97, 0x2a, 0xdc, 0x4b, 0x95, 0xac, 0x53, 0x34, 0x7f, 0x10, 0xcf, 0xff, - 0x0d, 0x00, 0x00, 0xff, 0xff, 0xff, 0xe4, 0xc0, 0x68, 0x60, 0x0c, 0x00, 0x00, + // 1140 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xa4, 0x57, 0xcf, 0x6f, 0x1b, 0xc5, + 0x17, 0xcf, 0xfa, 0xb7, 0x9f, 0x5d, 0xc7, 0x9d, 0x56, 0xa9, 0xd3, 0x24, 0x56, 0xba, 0xdf, 0xef, + 0xc1, 0xa4, 0x52, 0x22, 0x52, 0x50, 0x41, 0x42, 0x48, 0x89, 0xd3, 0xaa, 0x2d, 0x02, 0xa2, 0x75, + 0x50, 0x5a, 0x2e, 0xd1, 0x78, 0xf7, 0xc5, 0x5e, 0xc5, 0x9e, 0x59, 0x66, 0xc6, 0x89, 0x22, 0xee, + 0x70, 0x04, 0x89, 0xff, 0x84, 0x33, 0x7f, 0x00, 0xc7, 0x1e, 0x38, 0x70, 0x42, 0x28, 0xf9, 0x47, + 0xd0, 0xcc, 0xee, 0xda, 0xbb, 0xb6, 0x13, 0x62, 0x71, 0x9b, 0xf7, 0xeb, 0x33, 0x6f, 0xde, 0x7c, + 0xde, 0xbc, 0x5d, 0x58, 0xeb, 0x23, 0x15, 0xaa, 0x8b, 0x54, 0x05, 0xdd, 0x9d, 0xf1, 0x7a, 0x3b, + 0x10, 0x5c, 0x71, 0x52, 0x49, 0x18, 0xed, 0x77, 0x50, 0x3e, 0xa2, 0xdd, 0x01, 0x76, 0x02, 0xca, + 0x48, 0x03, 0x8a, 0x46, 0x78, 0x7d, 0xd0, 0xb0, 0x36, 0xad, 0x56, 0xce, 0x89, 0x45, 0xf2, 0x18, + 0x4a, 0x1d, 0x45, 0x85, 0xfa, 0x02, 0x2f, 0x1b, 0x99, 0x4d, 0xab, 0x55, 0x75, 0xc6, 0x32, 0x59, + 0x81, 0xc2, 0x0b, 0xe6, 0x69, 0x4b, 0xd6, 0x58, 0x22, 0xc9, 0xfe, 0x35, 0x03, 0xf5, 0x57, 0x7a, + 0xab, 0x7d, 0xa4, 0xca, 0xc1, 0xef, 0x46, 0x28, 0x15, 0xb1, 0xa1, 0xea, 0xf6, 0x29, 0xeb, 0xe1, + 0x29, 0xa2, 0x17, 0xed, 0x53, 0x76, 0x52, 0x3a, 0xf2, 0x11, 0x94, 0x2f, 0xa8, 0x42, 0x31, 0xa4, + 0xe2, 0xcc, 0xec, 0x56, 0xd9, 0x5d, 0xd9, 0x4e, 0x24, 0xbd, 0x7d, 0x1c, 0x5b, 0x9d, 0x89, 0x23, + 0xf9, 0x04, 0x4a, 0x52, 0x51, 0x35, 0x92, 0x28, 0x1b, 0xd9, 0xcd, 0x6c, 0xab, 0xb2, 0xbb, 0x9e, + 0x0a, 0x1a, 0x1f, 0xb3, 0x63, 0xbc, 0x9c, 0xb1, 0x37, 0x69, 0xc1, 0xb2, 0xcb, 0x87, 0x01, 0x0e, + 0x50, 0x61, 0x68, 0x6c, 0xe4, 0x36, 0xad, 0x56, 0xc9, 0x99, 0x56, 0x93, 0x67, 0x50, 0xbc, 0xa0, + 0x82, 0xf9, 0xac, 0xd7, 0xc8, 0x9b, 0xbc, 0x56, 0x53, 0x5b, 0x38, 0x23, 0xa6, 0x6d, 0x2f, 0x84, + 0xe0, 0xc2, 0x89, 0x3d, 0xc9, 0x53, 0xc8, 0xa2, 0x10, 0x8d, 0xc2, 0xbf, 0x05, 0x68, 0x2f, 0xfb, + 0x6b, 0x28, 0x8f, 0x4f, 0x17, 0x16, 0x0b, 0xdd, 0xb3, 0x80, 0xfb, 0x4c, 0x1d, 0xc9, 0xe8, 0x52, + 0x52, 0x3a, 0xd2, 0x04, 0x10, 0x28, 0xf9, 0xe0, 0x1c, 0xbd, 0x23, 0x69, 0xaa, 0x95, 0x73, 0x12, + 0x1a, 0xfb, 0x47, 0x0b, 0xea, 0x07, 0xbe, 0x0c, 0xa8, 0x72, 0xfb, 0x28, 0xf6, 0x5c, 0xe5, 0x73, + 0x46, 0xb6, 0x20, 0x27, 0x03, 0xca, 0x0c, 0xe0, 0x74, 0x71, 0xc7, 0x75, 0x72, 0x8c, 0x0f, 0x79, + 0x0a, 0x05, 0x6a, 0xa2, 0x0c, 0x78, 0x6d, 0xf7, 0x41, 0xca, 0x3b, 0x04, 0x74, 0x22, 0x17, 0xcd, + 0x93, 0x36, 0x1f, 0x0e, 0x7d, 0x9d, 0x6d, 0xd6, 0xe4, 0x32, 0x96, 0xed, 0x00, 0xee, 0x27, 0xe8, + 0x20, 0x03, 0xce, 0x24, 0x92, 0xe7, 0x50, 0x0c, 0x43, 0xf5, 0xe9, 0xf4, 0xa5, 0x6d, 0xa4, 0xe0, + 0xa7, 0x33, 0x77, 0x62, 0xef, 0x19, 0x22, 0x65, 0x66, 0x89, 0x64, 0xbf, 0x4d, 0x1e, 0xbd, 0xcd, + 0xd9, 0xa9, 0xdf, 0x5b, 0xe8, 0xe8, 0x0d, 0x28, 0x4a, 0xcd, 0xf2, 0x71, 0x61, 0x63, 0xd1, 0xfe, + 0xcb, 0x82, 0xd5, 0x8e, 0xdb, 0x47, 0x6f, 0x34, 0xc0, 0xc9, 0x16, 0x8b, 0x90, 0xfc, 0x63, 0x28, + 0xb8, 0x26, 0xa3, 0x88, 0xe1, 0x37, 0x9d, 0x3b, 0x4c, 0xdb, 0x89, 0x9c, 0x49, 0x1b, 0x6a, 0x32, + 0xda, 0x37, 0xac, 0x88, 0x29, 0x73, 0x6d, 0x77, 0x2d, 0x15, 0xde, 0x49, 0xb9, 0x38, 0x53, 0x21, + 0xe4, 0x09, 0x54, 0x7d, 0x79, 0x22, 0xd1, 0xe5, 0xcc, 0xa3, 0xe2, 0x32, 0x62, 0x7b, 0xc5, 0x97, + 0x9d, 0x58, 0x65, 0x1f, 0xc2, 0x83, 0x2f, 0xa9, 0xcf, 0x14, 0xf5, 0x19, 0x8a, 0x57, 0x31, 0x34, + 0xf9, 0x34, 0xd1, 0x64, 0xf3, 0xee, 0x6b, 0x12, 0x33, 0xdd, 0x65, 0xf6, 0x4f, 0x19, 0xa8, 0x4f, + 0x9b, 0xef, 0x54, 0xa9, 0x0d, 0x00, 0xbd, 0x3a, 0xd1, 0x48, 0x18, 0xdd, 0x73, 0x59, 0x6b, 0x34, + 0x06, 0x92, 0x0f, 0x21, 0x1f, 0x5a, 0xe6, 0x15, 0xa2, 0xcd, 0x87, 0x01, 0x67, 0xc8, 0x94, 0xf1, + 0x75, 0x42, 0x4f, 0xf2, 0x3f, 0xb8, 0x37, 0xe9, 0xa1, 0x13, 0x15, 0xb6, 0xfb, 0x74, 0x63, 0xa5, + 0x7a, 0x3d, 0xbb, 0x68, 0xaf, 0x67, 0xef, 0xd0, 0xeb, 0xcf, 0x61, 0xad, 0xcd, 0xb9, 0xf0, 0x7c, + 0x46, 0x15, 0x17, 0xfb, 0x9c, 0x2b, 0xa9, 0x04, 0x0d, 0x62, 0x16, 0x35, 0xa0, 0x78, 0x8e, 0x42, + 0xea, 0x3b, 0xd6, 0x65, 0xc9, 0x3a, 0xb1, 0x68, 0xbf, 0x83, 0xf5, 0xf9, 0x81, 0x51, 0x53, 0xfd, + 0x87, 0x5b, 0xfa, 0xcd, 0x82, 0xd5, 0x98, 0x7c, 0x13, 0xb7, 0x38, 0xa5, 0x37, 0xb0, 0x4c, 0x3d, + 0xef, 0x64, 0x38, 0x36, 0xc4, 0xf8, 0x4f, 0xd2, 0x8f, 0x82, 0xe7, 0xcd, 0xc4, 0x3a, 0x35, 0x9a, + 0xd4, 0x4a, 0xd2, 0x01, 0x22, 0x70, 0xc8, 0xcf, 0x31, 0x05, 0x97, 0x31, 0x70, 0xff, 0x4f, 0x57, + 0xce, 0xb8, 0xcd, 0x22, 0xde, 0x17, 0x53, 0x06, 0x69, 0xff, 0x60, 0xc1, 0xc3, 0x79, 0xbb, 0x93, + 0x1a, 0x64, 0x7c, 0x2f, 0xa2, 0x57, 0xc6, 0xf7, 0xf4, 0xd0, 0x4a, 0xb4, 0x5f, 0x75, 0xdc, 0x5f, + 0xd3, 0xad, 0x91, 0x9d, 0x69, 0x8d, 0x3b, 0xb1, 0xc7, 0x6e, 0xc3, 0xa3, 0x1b, 0xd2, 0x9e, 0x49, + 0xa5, 0x01, 0x45, 0x97, 0x4a, 0x97, 0x7a, 0x21, 0xb9, 0x4b, 0x4e, 0x2c, 0xda, 0x6f, 0xe1, 0xf1, + 0x24, 0x7c, 0x86, 0x1f, 0x77, 0xe9, 0x9d, 0x1b, 0x8e, 0x69, 0x7f, 0x0f, 0x6b, 0x73, 0x91, 0x23, + 0x02, 0xdd, 0x05, 0x3a, 0x39, 0x6f, 0x33, 0x8b, 0xcc, 0x5b, 0xfb, 0x33, 0x58, 0x99, 0x6c, 0xde, + 0x1e, 0x70, 0x89, 0x0b, 0x1c, 0xc9, 0x3e, 0x86, 0x47, 0x33, 0xd1, 0x0b, 0xa4, 0xad, 0xdf, 0xf4, + 0x91, 0xeb, 0xa2, 0x94, 0x71, 0xb5, 0x23, 0xd1, 0xfe, 0x25, 0x03, 0xf9, 0xf0, 0x49, 0x59, 0x87, + 0xf2, 0x6b, 0xb9, 0x3f, 0xe0, 0xee, 0x19, 0x86, 0x17, 0x55, 0x72, 0x26, 0x0a, 0x8d, 0x60, 0x96, + 0x93, 0xa9, 0x10, 0x89, 0xe4, 0x73, 0xa8, 0x85, 0xcb, 0xf8, 0xe8, 0xd1, 0x87, 0xc8, 0x4d, 0x53, + 0x66, 0xca, 0x9b, 0xbc, 0x81, 0x87, 0x5f, 0x21, 0x7a, 0x07, 0x82, 0x07, 0x01, 0x7a, 0x13, 0x94, + 0xdc, 0xad, 0x28, 0x73, 0x63, 0xc8, 0x4b, 0x20, 0x5a, 0xbf, 0xe7, 0x79, 0x49, 0xa4, 0xfc, 0xad, + 0x48, 0x73, 0x22, 0xec, 0x3f, 0x2c, 0x58, 0x9e, 0xba, 0xca, 0x85, 0x66, 0xe8, 0x4b, 0xa8, 0xbb, + 0xf1, 0x23, 0x7c, 0x12, 0x52, 0x20, 0xfa, 0x90, 0xb8, 0xf5, 0xa5, 0x5e, 0x76, 0x93, 0xf2, 0x48, + 0x7f, 0xa4, 0x25, 0x9e, 0xf9, 0xca, 0x2e, 0x49, 0xcf, 0xbb, 0x45, 0x5f, 0x77, 0xdb, 0x83, 0x6a, + 0xf2, 0x41, 0x26, 0x04, 0x72, 0xca, 0x1f, 0x62, 0x44, 0x19, 0xb3, 0xd6, 0x3a, 0xc6, 0xbd, 0x78, + 0xe4, 0x98, 0xb5, 0xd6, 0xb9, 0x5a, 0x97, 0x0d, 0x75, 0x7a, 0xad, 0x09, 0x31, 0x44, 0x29, 0x69, + 0x0f, 0xcd, 0x56, 0x65, 0x27, 0x16, 0xb7, 0x36, 0xa0, 0x10, 0x8d, 0xdc, 0x32, 0xe4, 0x8f, 0x85, + 0xaf, 0xb0, 0xbe, 0x44, 0x4a, 0x90, 0x3b, 0xa4, 0x52, 0xd6, 0xad, 0xad, 0x16, 0xd4, 0xd2, 0x93, + 0x9a, 0x00, 0x14, 0xda, 0x02, 0xa9, 0xf1, 0x03, 0x28, 0x84, 0x4f, 0x48, 0xdd, 0xda, 0x62, 0x50, + 0x4b, 0x17, 0x88, 0x54, 0xa0, 0xf8, 0x0d, 0x3b, 0x63, 0xfc, 0x82, 0x85, 0xae, 0x7b, 0x5d, 0x89, + 0x4c, 0xd5, 0x2d, 0x72, 0x0f, 0xca, 0x87, 0x02, 0x03, 0x2a, 0x7c, 0xd6, 0xab, 0x67, 0x48, 0x15, + 0x4a, 0xa1, 0x88, 0x5e, 0x3d, 0xab, 0xa3, 0x8e, 0xb9, 0x38, 0xd3, 0xa6, 0x9c, 0x36, 0x75, 0x14, + 0x0f, 0x02, 0x2d, 0xe5, 0xb5, 0xc9, 0x48, 0xe8, 0xd5, 0x0b, 0xfb, 0xed, 0xdf, 0xaf, 0x9a, 0xd6, + 0xfb, 0xab, 0xa6, 0xf5, 0xf7, 0x55, 0xd3, 0xfa, 0xf9, 0xba, 0xb9, 0xf4, 0xfe, 0xba, 0xb9, 0xf4, + 0xe7, 0x75, 0x73, 0xe9, 0xdb, 0x0f, 0x7a, 0xbe, 0xea, 0x8f, 0xba, 0xdb, 0x2e, 0x1f, 0xee, 0x9c, + 0x0e, 0xf8, 0x45, 0x17, 0xfb, 0x34, 0x08, 0x2e, 0x77, 0x94, 0xdf, 0xa3, 0x0a, 0x77, 0x12, 0xb7, + 0xd2, 0x2d, 0x98, 0xff, 0x8d, 0x67, 0xff, 0x04, 0x00, 0x00, 0xff, 0xff, 0x1c, 0x36, 0x63, 0x6a, + 0x8e, 0x0c, 0x00, 0x00, } func (m *TableSpan) Marshal() (dAtA []byte, err error) { @@ -1605,7 +1610,7 @@ func (m *Watermark) MarshalToSizedBuffer(dAtA []byte) (int, error) { return len(dAtA) - i, nil } -func (m *TableProgress) Marshal() (dAtA []byte, err error) { +func (m *DispatcherAction) Marshal() (dAtA []byte, err error) { size := m.Size() dAtA = make([]byte, size) n, err := m.MarshalToSizedBuffer(dAtA[:size]) @@ -1615,94 +1620,25 @@ func (m *TableProgress) Marshal() (dAtA []byte, err error) { return dAtA[:n], nil } -func (m *TableProgress) MarshalTo(dAtA []byte) (int, error) { +func (m *DispatcherAction) MarshalTo(dAtA []byte) (int, error) { size := m.Size() return m.MarshalToSizedBuffer(dAtA[:size]) } -func (m *TableProgress) MarshalToSizedBuffer(dAtA []byte) (int, error) { +func (m *DispatcherAction) MarshalToSizedBuffer(dAtA []byte) (int, error) { i := len(dAtA) _ = i var l int _ = l - if m.CheckpointTs != 0 { - i = encodeVarintHeartbeat(dAtA, i, uint64(m.CheckpointTs)) - i-- - dAtA[i] = 0x20 - } - if m.BlockTs != 0 { - i = encodeVarintHeartbeat(dAtA, i, uint64(m.BlockTs)) + if m.CommitTs != 0 { + i = encodeVarintHeartbeat(dAtA, i, uint64(m.CommitTs)) i-- dAtA[i] = 0x18 } - if m.IsBlocked { - i-- - if m.IsBlocked { - dAtA[i] = 1 - } else { - dAtA[i] = 0 - } - i-- - dAtA[i] = 0x10 - } - if m.Span != nil { - { - size, err := m.Span.MarshalToSizedBuffer(dAtA[:i]) - if err != nil { - return 0, err - } - i -= size - i = encodeVarintHeartbeat(dAtA, i, uint64(size)) - } - i-- - dAtA[i] = 0xa - } - return len(dAtA) - i, nil -} - -func (m *TableProgressInfo) Marshal() (dAtA []byte, err error) { - size := m.Size() - dAtA = make([]byte, size) - n, err := m.MarshalToSizedBuffer(dAtA[:size]) - if err != nil { - return nil, err - } - return dAtA[:n], nil -} - -func (m *TableProgressInfo) MarshalTo(dAtA []byte) (int, error) { - size := m.Size() - return m.MarshalToSizedBuffer(dAtA[:size]) -} - -func (m *TableProgressInfo) MarshalToSizedBuffer(dAtA []byte) (int, error) { - i := len(dAtA) - _ = i - var l int - _ = l - if m.SchedulerStatus != 0 { - i = encodeVarintHeartbeat(dAtA, i, uint64(m.SchedulerStatus)) - i-- - dAtA[i] = 0x20 - } if m.Action != 0 { i = encodeVarintHeartbeat(dAtA, i, uint64(m.Action)) i-- - dAtA[i] = 0x18 - } - if len(m.TableProgresses) > 0 { - for iNdEx := len(m.TableProgresses) - 1; iNdEx >= 0; iNdEx-- { - { - size, err := m.TableProgresses[iNdEx].MarshalToSizedBuffer(dAtA[:i]) - if err != nil { - return 0, err - } - i -= size - i = encodeVarintHeartbeat(dAtA, i, uint64(size)) - } - i-- - dAtA[i] = 0x12 - } + dAtA[i] = 0x10 } if m.Span != nil { { @@ -1746,10 +1682,10 @@ func (m *HeartBeatResponse) MarshalToSizedBuffer(dAtA []byte) (int, error) { i-- dAtA[i] = 0x12 } - if len(m.Info) > 0 { - for iNdEx := len(m.Info) - 1; iNdEx >= 0; iNdEx-- { + if len(m.Actions) > 0 { + for iNdEx := len(m.Actions) - 1; iNdEx >= 0; iNdEx-- { { - size, err := m.Info[iNdEx].MarshalToSizedBuffer(dAtA[:i]) + size, err := m.Actions[iNdEx].MarshalToSizedBuffer(dAtA[:i]) if err != nil { return 0, err } @@ -2331,6 +2267,86 @@ func (m *MaintainerCloseResponse) MarshalToSizedBuffer(dAtA []byte) (int, error) return len(dAtA) - i, nil } +func (m *State) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *State) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *State) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + _ = i + var l int + _ = l + if len(m.NeedAddedTableSpan) > 0 { + for iNdEx := len(m.NeedAddedTableSpan) - 1; iNdEx >= 0; iNdEx-- { + { + size, err := m.NeedAddedTableSpan[iNdEx].MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintHeartbeat(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x2a + } + } + if len(m.NeedDroppedTableSpan) > 0 { + for iNdEx := len(m.NeedDroppedTableSpan) - 1; iNdEx >= 0; iNdEx-- { + { + size, err := m.NeedDroppedTableSpan[iNdEx].MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintHeartbeat(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x22 + } + } + if len(m.BlockTableSpan) > 0 { + for iNdEx := len(m.BlockTableSpan) - 1; iNdEx >= 0; iNdEx-- { + { + size, err := m.BlockTableSpan[iNdEx].MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintHeartbeat(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x1a + } + } + if m.BlockTs != 0 { + i = encodeVarintHeartbeat(dAtA, i, uint64(m.BlockTs)) + i-- + dAtA[i] = 0x10 + } + if m.IsBlocked { + i-- + if m.IsBlocked { + dAtA[i] = 1 + } else { + dAtA[i] = 0 + } + i-- + dAtA[i] = 0x8 + } + return len(dAtA) - i, nil +} + func (m *TableSpanStatus) Marshal() (dAtA []byte, err error) { size := m.Size() dAtA = make([]byte, size) @@ -2356,6 +2372,18 @@ func (m *TableSpanStatus) MarshalToSizedBuffer(dAtA []byte) (int, error) { i-- dAtA[i] = 0x20 } + if m.State != nil { + { + size, err := m.State.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintHeartbeat(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x1a + } if m.ComponentStatus != 0 { i = encodeVarintHeartbeat(dAtA, i, uint64(m.ComponentStatus)) i-- @@ -2507,29 +2535,7 @@ func (m *Watermark) Size() (n int) { return n } -func (m *TableProgress) Size() (n int) { - if m == nil { - return 0 - } - var l int - _ = l - if m.Span != nil { - l = m.Span.Size() - n += 1 + l + sovHeartbeat(uint64(l)) - } - if m.IsBlocked { - n += 2 - } - if m.BlockTs != 0 { - n += 1 + sovHeartbeat(uint64(m.BlockTs)) - } - if m.CheckpointTs != 0 { - n += 1 + sovHeartbeat(uint64(m.CheckpointTs)) - } - return n -} - -func (m *TableProgressInfo) Size() (n int) { +func (m *DispatcherAction) Size() (n int) { if m == nil { return 0 } @@ -2539,17 +2545,11 @@ func (m *TableProgressInfo) Size() (n int) { l = m.Span.Size() n += 1 + l + sovHeartbeat(uint64(l)) } - if len(m.TableProgresses) > 0 { - for _, e := range m.TableProgresses { - l = e.Size() - n += 1 + l + sovHeartbeat(uint64(l)) - } - } if m.Action != 0 { n += 1 + sovHeartbeat(uint64(m.Action)) } - if m.SchedulerStatus != 0 { - n += 1 + sovHeartbeat(uint64(m.SchedulerStatus)) + if m.CommitTs != 0 { + n += 1 + sovHeartbeat(uint64(m.CommitTs)) } return n } @@ -2560,8 +2560,8 @@ func (m *HeartBeatResponse) Size() (n int) { } var l int _ = l - if len(m.Info) > 0 { - for _, e := range m.Info { + if len(m.Actions) > 0 { + for _, e := range m.Actions { l = e.Size() n += 1 + l + sovHeartbeat(uint64(l)) } @@ -2814,6 +2814,39 @@ func (m *MaintainerCloseResponse) Size() (n int) { return n } +func (m *State) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.IsBlocked { + n += 2 + } + if m.BlockTs != 0 { + n += 1 + sovHeartbeat(uint64(m.BlockTs)) + } + if len(m.BlockTableSpan) > 0 { + for _, e := range m.BlockTableSpan { + l = e.Size() + n += 1 + l + sovHeartbeat(uint64(l)) + } + } + if len(m.NeedDroppedTableSpan) > 0 { + for _, e := range m.NeedDroppedTableSpan { + l = e.Size() + n += 1 + l + sovHeartbeat(uint64(l)) + } + } + if len(m.NeedAddedTableSpan) > 0 { + for _, e := range m.NeedAddedTableSpan { + l = e.Size() + n += 1 + l + sovHeartbeat(uint64(l)) + } + } + return n +} + func (m *TableSpanStatus) Size() (n int) { if m == nil { return 0 @@ -2827,6 +2860,10 @@ func (m *TableSpanStatus) Size() (n int) { if m.ComponentStatus != 0 { n += 1 + sovHeartbeat(uint64(m.ComponentStatus)) } + if m.State != nil { + l = m.State.Size() + n += 1 + l + sovHeartbeat(uint64(l)) + } if m.CheckpointTs != 0 { n += 1 + sovHeartbeat(uint64(m.CheckpointTs)) } @@ -3278,151 +3315,7 @@ func (m *Watermark) Unmarshal(dAtA []byte) error { if wireType != 0 { return fmt.Errorf("proto: wrong wireType = %d for field CheckpointTs", wireType) } - m.CheckpointTs = 0 - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowHeartbeat - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - m.CheckpointTs |= uint64(b&0x7F) << shift - if b < 0x80 { - break - } - } - case 2: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field ResolvedTs", wireType) - } - m.ResolvedTs = 0 - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowHeartbeat - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - m.ResolvedTs |= uint64(b&0x7F) << shift - if b < 0x80 { - break - } - } - default: - iNdEx = preIndex - skippy, err := skipHeartbeat(dAtA[iNdEx:]) - if err != nil { - return err - } - if (skippy < 0) || (iNdEx+skippy) < 0 { - return ErrInvalidLengthHeartbeat - } - if (iNdEx + skippy) > l { - return io.ErrUnexpectedEOF - } - iNdEx += skippy - } - } - - if iNdEx > l { - return io.ErrUnexpectedEOF - } - return nil -} -func (m *TableProgress) Unmarshal(dAtA []byte) error { - l := len(dAtA) - iNdEx := 0 - for iNdEx < l { - preIndex := iNdEx - var wire uint64 - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowHeartbeat - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - wire |= uint64(b&0x7F) << shift - if b < 0x80 { - break - } - } - fieldNum := int32(wire >> 3) - wireType := int(wire & 0x7) - if wireType == 4 { - return fmt.Errorf("proto: TableProgress: wiretype end group for non-group") - } - if fieldNum <= 0 { - return fmt.Errorf("proto: TableProgress: illegal tag %d (wire type %d)", fieldNum, wire) - } - switch fieldNum { - case 1: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Span", wireType) - } - var msglen int - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowHeartbeat - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - msglen |= int(b&0x7F) << shift - if b < 0x80 { - break - } - } - if msglen < 0 { - return ErrInvalidLengthHeartbeat - } - postIndex := iNdEx + msglen - if postIndex < 0 { - return ErrInvalidLengthHeartbeat - } - if postIndex > l { - return io.ErrUnexpectedEOF - } - if m.Span == nil { - m.Span = &TableSpan{} - } - if err := m.Span.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { - return err - } - iNdEx = postIndex - case 2: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field IsBlocked", wireType) - } - var v int - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowHeartbeat - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - v |= int(b&0x7F) << shift - if b < 0x80 { - break - } - } - m.IsBlocked = bool(v != 0) - case 3: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field BlockTs", wireType) - } - m.BlockTs = 0 + m.CheckpointTs = 0 for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowHeartbeat @@ -3432,16 +3325,16 @@ func (m *TableProgress) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - m.BlockTs |= uint64(b&0x7F) << shift + m.CheckpointTs |= uint64(b&0x7F) << shift if b < 0x80 { break } } - case 4: + case 2: if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field CheckpointTs", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field ResolvedTs", wireType) } - m.CheckpointTs = 0 + m.ResolvedTs = 0 for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowHeartbeat @@ -3451,7 +3344,7 @@ func (m *TableProgress) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - m.CheckpointTs |= uint64(b&0x7F) << shift + m.ResolvedTs |= uint64(b&0x7F) << shift if b < 0x80 { break } @@ -3477,7 +3370,7 @@ func (m *TableProgress) Unmarshal(dAtA []byte) error { } return nil } -func (m *TableProgressInfo) Unmarshal(dAtA []byte) error { +func (m *DispatcherAction) Unmarshal(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -3500,10 +3393,10 @@ func (m *TableProgressInfo) Unmarshal(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: TableProgressInfo: wiretype end group for non-group") + return fmt.Errorf("proto: DispatcherAction: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: TableProgressInfo: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: DispatcherAction: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: @@ -3543,40 +3436,6 @@ func (m *TableProgressInfo) Unmarshal(dAtA []byte) error { } iNdEx = postIndex case 2: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field TableProgresses", wireType) - } - var msglen int - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowHeartbeat - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - msglen |= int(b&0x7F) << shift - if b < 0x80 { - break - } - } - if msglen < 0 { - return ErrInvalidLengthHeartbeat - } - postIndex := iNdEx + msglen - if postIndex < 0 { - return ErrInvalidLengthHeartbeat - } - if postIndex > l { - return io.ErrUnexpectedEOF - } - m.TableProgresses = append(m.TableProgresses, &TableProgress{}) - if err := m.TableProgresses[len(m.TableProgresses)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { - return err - } - iNdEx = postIndex - case 3: if wireType != 0 { return fmt.Errorf("proto: wrong wireType = %d for field Action", wireType) } @@ -3595,11 +3454,11 @@ func (m *TableProgressInfo) Unmarshal(dAtA []byte) error { break } } - case 4: + case 3: if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field SchedulerStatus", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field CommitTs", wireType) } - m.SchedulerStatus = 0 + m.CommitTs = 0 for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowHeartbeat @@ -3609,7 +3468,7 @@ func (m *TableProgressInfo) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - m.SchedulerStatus |= ComponentState(b&0x7F) << shift + m.CommitTs |= uint64(b&0x7F) << shift if b < 0x80 { break } @@ -3666,7 +3525,7 @@ func (m *HeartBeatResponse) Unmarshal(dAtA []byte) error { switch fieldNum { case 1: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Info", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Actions", wireType) } var msglen int for shift := uint(0); ; shift += 7 { @@ -3693,8 +3552,8 @@ func (m *HeartBeatResponse) Unmarshal(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.Info = append(m.Info, &TableProgressInfo{}) - if err := m.Info[len(m.Info)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + m.Actions = append(m.Actions, &DispatcherAction{}) + if err := m.Actions[len(m.Actions)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { return err } iNdEx = postIndex @@ -5261,6 +5120,197 @@ func (m *MaintainerCloseResponse) Unmarshal(dAtA []byte) error { } return nil } +func (m *State) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowHeartbeat + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: State: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: State: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field IsBlocked", wireType) + } + var v int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowHeartbeat + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + m.IsBlocked = bool(v != 0) + case 2: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field BlockTs", wireType) + } + m.BlockTs = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowHeartbeat + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.BlockTs |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field BlockTableSpan", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowHeartbeat + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthHeartbeat + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthHeartbeat + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.BlockTableSpan = append(m.BlockTableSpan, &TableSpan{}) + if err := m.BlockTableSpan[len(m.BlockTableSpan)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 4: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field NeedDroppedTableSpan", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowHeartbeat + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthHeartbeat + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthHeartbeat + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.NeedDroppedTableSpan = append(m.NeedDroppedTableSpan, &TableSpan{}) + if err := m.NeedDroppedTableSpan[len(m.NeedDroppedTableSpan)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 5: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field NeedAddedTableSpan", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowHeartbeat + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthHeartbeat + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthHeartbeat + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.NeedAddedTableSpan = append(m.NeedAddedTableSpan, &TableSpan{}) + if err := m.NeedAddedTableSpan[len(m.NeedAddedTableSpan)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipHeartbeat(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return ErrInvalidLengthHeartbeat + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} func (m *TableSpanStatus) Unmarshal(dAtA []byte) error { l := len(dAtA) iNdEx := 0 @@ -5345,6 +5395,42 @@ func (m *TableSpanStatus) Unmarshal(dAtA []byte) error { break } } + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field State", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowHeartbeat + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthHeartbeat + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthHeartbeat + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.State == nil { + m.State = &State{} + } + if err := m.State.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex case 4: if wireType != 0 { return fmt.Errorf("proto: wrong wireType = %d for field CheckpointTs", wireType) diff --git a/heartbeatpb/heartbeat.proto b/heartbeatpb/heartbeat.proto index 5857afa9c..81334898b 100644 --- a/heartbeatpb/heartbeat.proto +++ b/heartbeatpb/heartbeat.proto @@ -33,27 +33,18 @@ message Watermark { } enum Action { - None = 0; - Write = 1; - Pass = 2; + Write = 0; + Pass = 1; } -message TableProgress { +message DispatcherAction { TableSpan span = 1; - bool IsBlocked = 2; - uint64 BlockTs = 3; - uint64 CheckpointTs = 4; + Action action = 2; + uint64 CommitTs = 3; // DDLCommitTs } -// 这名字重新取,我真的要吐了 -message TableProgressInfo { - TableSpan span = 1; // for which dispatcher - repeated TableProgress tableProgresses = 2; // 救命了这名字取得 - Action action = 3; - ComponentState scheduler_status = 4; -} message HeartBeatResponse { - repeated TableProgressInfo info = 1; + repeated DispatcherAction actions = 1; string changefeedID = 2; } @@ -131,9 +122,18 @@ message MaintainerCloseResponse { bool success = 2; } +message State { + bool IsBlocked = 1; + uint64 BlockTs = 2; + repeated TableSpan BlockTableSpan = 3; + repeated TableSpan NeedDroppedTableSpan =4; + repeated TableSpan NeedAddedTableSpan = 5; +} + message TableSpanStatus { TableSpan span = 1; // for which dispatcher ComponentState component_status = 2; + State state = 3; uint64 checkpoint_ts = 4; } diff --git a/maintainer/fake_dispatcher_manager.go b/maintainer/fake_dispatcher_manager.go index 939df7dd9..10545be6c 100644 --- a/maintainer/fake_dispatcher_manager.go +++ b/maintainer/fake_dispatcher_manager.go @@ -128,12 +128,12 @@ func (m *FakeDispatcherManagerManager) Run(ctx context.Context) error { string("maintainer/"+manager.id.ID), &heartbeatpb.HeartBeatResponse{ ChangefeedID: manager.id.ID, - Info: []*heartbeatpb.TableProgressInfo{ - { - Span: absentSpan.TableSpan, - SchedulerStatus: heartbeatpb.ComponentState_Absent, - }, - }, + // Info: []*heartbeatpb.TableProgressInfo{ + // { + // Span: absentSpan.TableSpan, + // SchedulerStatus: heartbeatpb.ComponentState_Absent, + // }, + // }, }, )) if err != nil { diff --git a/pkg/common/txn_event.go b/pkg/common/txn_event.go index 718764274..1b2da160e 100644 --- a/pkg/common/txn_event.go +++ b/pkg/common/txn_event.go @@ -4,6 +4,7 @@ import ( "fmt" "unsafe" + "github.com/flowbehappy/tigate/heartbeatpb" "github.com/pingcap/log" "github.com/pingcap/tidb/pkg/parser/model" timodel "github.com/pingcap/tiflow/cdc/model" @@ -75,17 +76,19 @@ func (e *TxnEvent) IsDDLEvent() bool { return e.DDLEvent != nil } -func (e *TxnEvent) IsCrossTableDDL() bool { +func (e *TxnEvent) IsSingleTableDDL() bool { ddlType := e.GetDDLType() - return ddlType == model.ActionCreateSchema || ddlType == model.ActionDropSchema || - ddlType == model.ActionDropTable || - ddlType == model.ActionTruncateTable || ddlType == model.ActionRenameTable || - ddlType == model.ActionAddTablePartition || ddlType == model.ActionDropTablePartition || - ddlType == model.ActionTruncateTablePartition || ddlType == model.ActionRecoverTable || - ddlType == model.ActionRepairTable || ddlType == model.ActionExchangeTablePartition || - ddlType == model.ActionRemovePartitioning || ddlType == model.ActionRenameTables || - ddlType == model.ActionCreateTables || ddlType == model.ActionReorganizePartition || - ddlType == model.ActionFlashbackCluster || ddlType == model.ActionMultiSchemaChange + return ddlType == model.ActionAddColumn || ddlType == model.ActionDropColumn || ddlType == model.ActionModifyColumn || ddlType == model.ActionAddIndex || ddlType == model.ActionDropIndex || ddlType == model.ActionModifyTableComment || ddlType == model.ActionRebaseAutoID || ddlType == model.ActionSetDefaultValue || ddlType == model.ActionShardRowID || ddlType == model.ActionModifyTableCharsetAndCollate || ddlType == model.ActionCreateView || ddlType == model.ActionDropView || ddlType == model.ActionAddForeignKey || ddlType == model.ActionDropForeignKey || ddlType == model.ActionRenameIndex || ddlType == model.ActionLockTable || ddlType == model.ActionUnlockTable || ddlType == model.ActionSetTiFlashReplica || ddlType == model.ActionAddPrimaryKey || ddlType == model.ActionDropPrimaryKey || ddlType == model.ActionAddColumns || ddlType == model.ActionDropColumns || ddlType == model.ActionModifyTableAutoIdCache || ddlType == model.ActionRebaseAutoRandomBase || ddlType == model.ActionAlterIndexVisibility || ddlType == model.ActionAddCheckConstraint || ddlType == model.ActionDropCheckConstraint || ddlType == model.ActionAlterCheckConstraint || ddlType == model.ActionDropIndexes || ddlType == model.ActionAlterTableAttributes || ddlType == model.ActionAlterCacheTable || ddlType == model.ActionAlterNoCacheTable || ddlType == model.ActionMultiSchemaChange || ddlType == model.ActionAlterTTLInfo || ddlType == model.ActionAlterTTLRemove || ddlType == model.ActionRepairTable +} + +func (e *TxnEvent) GetBlockedTableSpan() []*heartbeatpb.TableSpan { + +} + +func (e *TxnEvent) GetNeedDroppedTableSpan() []*heartbeatpb.TableSpan { +} + +func (e *TxnEvent) GetNeedAddedTableSpan() []*heartbeatpb.TableSpan { } func (e *TxnEvent) IsSyncPointEvent() bool { From ee89a4d7abc3846bdcac8466b9cdd7286df1d815 Mon Sep 17 00:00:00 2001 From: hongyunyan <649330952@qq.com> Date: Wed, 14 Aug 2024 17:48:22 +0800 Subject: [PATCH 02/12] support table trigger event dispatcher --- downstreamadapter/dispatcher/dispatcher.go | 58 +++++++++ .../dispatcher/table_event_dispatcher.go | 54 +------- .../table_trigger_event_dispatcher.go | 122 +++++++++++++----- .../event_dispatcher_manager.go | 21 +-- .../eventcollector/event_collector.go | 7 +- downstreamadapter/sink/mysql_sink.go | 2 +- .../sink/types/table_progress.go | 2 +- 7 files changed, 159 insertions(+), 107 deletions(-) diff --git a/downstreamadapter/dispatcher/dispatcher.go b/downstreamadapter/dispatcher/dispatcher.go index 75c6a9d3a..24337128c 100644 --- a/downstreamadapter/dispatcher/dispatcher.go +++ b/downstreamadapter/dispatcher/dispatcher.go @@ -14,6 +14,8 @@ package dispatcher import ( + "time" + "github.com/flowbehappy/tigate/downstreamadapter/sink" "github.com/flowbehappy/tigate/heartbeatpb" "github.com/flowbehappy/tigate/pkg/common" @@ -64,6 +66,7 @@ type Dispatcher interface { // PushEvent(event *eventpb.TxnEvent) PushTxnEvent(event *common.TxnEvent) GetComponentStatus() heartbeatpb.ComponentState + GetTableSpanStatusesChan() chan *heartbeatpb.TableSpanStatus TryClose() (w heartbeatpb.Watermark, ok bool) } @@ -92,6 +95,61 @@ type HeartBeatInfo struct { ComponentStatus heartbeatpb.ComponentState } +// 1. 如果是单表内的 ddl,达到下推的条件为: sink 中没有还没执行完的当前表的 event +// 2. 如果是多表内的 ddl 或者是表间的 ddl,则需要满足的条件为: +// 2.1 sink 中没有还没执行完的当前表的 event +// 2.2 maintainer 通知自己可以 write 或者 pass event +func AddDDLEventToSinkWhenAvailable(d Dispatcher, event *common.TxnEvent) { + sink := d.GetSink() + tableSpan := d.GetTableSpan() + if event.IsSingleTableDDL() { + if sink.IsEmpty(tableSpan) { + sink.AddDMLEvent(tableSpan, event) + return + } else { + // TODO:先写一个 定时 check 的逻辑,后面用 dynamic stream 改造 + timer := time.NewTimer(time.Millisecond * 50) + for { + select { + case <-timer.C: + if sink.IsEmpty(tableSpan) { + sink.AddDMLEvent(tableSpan, event) + return + } + } + } + } + } + + d.GetTableSpanStatusesChan() <- &heartbeatpb.TableSpanStatus{ + Span: tableSpan.TableSpan, + ComponentStatus: heartbeatpb.ComponentState_Working, + State: &heartbeatpb.State{ + IsBlocked: true, + BlockTs: event.CommitTs, + BlockTableSpan: event.GetBlockedTableSpan(), // 这个包含自己的 span 是不是也无所谓,不然就要剔除掉 + NeedDroppedTableSpan: event.GetNeedDroppedTableSpan(), + NeedAddedTableSpan: event.GetNeedAddedTableSpan(), + }, + } + + for { + dispatcherAction := <-d.GetDDLActions() + if dispatcherAction.CommitTs == event.CommitTs { + if dispatcherAction.Action == heartbeatpb.Action_Write { + sink.AddDDLAndSyncPointEvent(tableSpan, event) // 这个是同步写,所以写完的时候 sink 也 available 了 + // 写完马上通知 maintainer 推进 checkpointTs + d.GetTableSpanStatusesChan() <- &heartbeatpb.TableSpanStatus{ + Span: tableSpan.TableSpan, + ComponentStatus: heartbeatpb.ComponentState_Working, + CheckpointTs: d.GetCheckpointTs(), + } + } + return + } + } +} + func CollectDispatcherHeartBeatInfo(d Dispatcher, h *HeartBeatInfo) { // use checkpointTs to release memory usage diff --git a/downstreamadapter/dispatcher/table_event_dispatcher.go b/downstreamadapter/dispatcher/table_event_dispatcher.go index 0efa374a7..74c712f26 100644 --- a/downstreamadapter/dispatcher/table_event_dispatcher.go +++ b/downstreamadapter/dispatcher/table_event_dispatcher.go @@ -134,53 +134,6 @@ func NewTableEventDispatcher(tableSpan *common.TableSpan, sink sink.Sink, startT return tableEventDispatcher } -// 1. 如果是单表内的 ddl,达到下推的条件为: sink 中没有还没执行完的当前表的 event -// 2. 如果是多表内的 ddl 或者是表间的 ddl,则需要满足的条件为: -// 2.1 sink 中没有还没执行完的当前表的 event -// 2.2 maintainer 通知自己可以 write 或者 pass event -func (d *TableEventDispatcher) AddDDLEventToSinkWhenAvailable(event *common.TxnEvent) { - if event.IsSingleTableDDL() { - if d.sink.IsEmpty(d.tableSpan) { - d.sink.AddDMLEvent(d.tableSpan, event) - return - } else { - // TODO:先写一个 定时 check 的逻辑,后面用 dynamic stream 改造 - timer := time.NewTimer(time.Millisecond * 50) - for { - select { - case <-timer.C: - if d.sink.IsEmpty(d.tableSpan) { - d.sink.AddDMLEvent(d.tableSpan, event) - return - } - } - } - } - } - - d.tableSpanStatusesChan <- &heartbeatpb.TableSpanStatus{ - Span: d.tableSpan.TableSpan, - ComponentStatus: heartbeatpb.ComponentState_Working, - State: &heartbeatpb.State{ - IsBlocked: true, - BlockTs: event.CommitTs, - BlockTableSpan: event.GetBlockedTableSpan(), // 这个包含自己的 span 是不是也无所谓,不然就要剔除掉 - NeedDroppedTableSpan: event.GetNeedDroppedTableSpan(), - NeedAddedTableSpan: event.GetNeedAddedTableSpan(), - }, - } - - for { - dispatcherAction := <-d.ddlActions - if dispatcherAction.CommitTs == event.CommitTs { - if dispatcherAction.Action == heartbeatpb.Action_Write { - d.sink.AddDDLAndSyncPointEvent(d.tableSpan, event) // 这个是同步写,所以写完的时候 sink 也 available 了 - } - return - } - } -} - func (d *TableEventDispatcher) DispatcherEvents(ctx context.Context) { defer d.wg.Done() tableSpan := d.GetTableSpan() @@ -193,9 +146,8 @@ func (d *TableEventDispatcher) DispatcherEvents(ctx context.Context) { if event.IsDMLEvent() { sink.AddDMLEvent(tableSpan, event) } else if event.IsDDLEvent() { - d.AddDDLEventToSinkWhenAvailable(event) + AddDDLEventToSinkWhenAvailable(d, event) } else { - // resolvedTs d.resolvedTs.Set(event.ResolvedTs) } } @@ -243,6 +195,10 @@ func (d *TableEventDispatcher) GetDDLActions() chan *heartbeatpb.DispatcherActio return d.ddlActions } +func (d *TableEventDispatcher) GetTableSpanStatusesChan() chan *heartbeatpb.TableSpanStatus { + return d.tableSpanStatusesChan +} + //func (d *TableEventDispatcher) GetSyncPointInfo() *SyncPointInfo { // return d.syncPointInfo // } diff --git a/downstreamadapter/dispatcher/table_trigger_event_dispatcher.go b/downstreamadapter/dispatcher/table_trigger_event_dispatcher.go index 74164c582..5e7a0be4d 100644 --- a/downstreamadapter/dispatcher/table_trigger_event_dispatcher.go +++ b/downstreamadapter/dispatcher/table_trigger_event_dispatcher.go @@ -14,12 +14,16 @@ package dispatcher import ( + "context" + "sync" + "github.com/flowbehappy/tigate/downstreamadapter/sink" "github.com/flowbehappy/tigate/heartbeatpb" "github.com/flowbehappy/tigate/pkg/common" - "github.com/pingcap/tiflow/pkg/filter" - + "github.com/google/uuid" "github.com/pingcap/log" + "github.com/pingcap/tiflow/pkg/filter" + "go.uber.org/zap" ) //filter 问题 -- 能收到这条就至少说明有相关的 table(比如 renames / create tables / exchange partitions -- 这个应该不支持一个在一个不在的),对于跟 table 有关的表来说,那就前面两种就可以在 ddl 生成的时候用 config 处理了 @@ -85,69 +89,121 @@ It also communicates with the Maintainer periodically to report self progress, and get the other dispatcher's progress and action of the blocked event. */ type TableTriggerEventDispatcher struct { - Id string - Ch chan *common.TxnEvent // 接受 event -- 先做个基础版本的,每次处理一条 ddl 的那种 - Filter filter.Filter - sink sink.Sink - HeartbeatChan chan *HeartBeatResponseMessage - State *State - tableSpan *common.TableSpan // 给一个特殊的 tableSpan - ResolvedTs uint64 + id string + eventCh chan *common.TxnEvent + filter filter.Filter + sink sink.Sink + ddlActions chan *heartbeatpb.DispatcherAction + tableSpan *common.TableSpan // 给一个特殊的 tableSpan + resolvedTs *TsWithMutex + componentStatus *ComponentStateWithMutex + + tableSpanStatusesChan chan *heartbeatpb.TableSpanStatus + + wg sync.WaitGroup + cancel context.CancelFunc + //MemoryUsage *MemoryUsage +} - MemoryUsage *MemoryUsage +func NewTableTriggerEventDispatcher(sink sink.Sink, startTs uint64, tableSpanStatusesChan chan *heartbeatpb.TableSpanStatus, filter filter.Filter) *TableTriggerEventDispatcher { + ctx, cancel := context.WithCancel(context.Background()) + tableTriggerEventDispatcher := &TableTriggerEventDispatcher{ + id: uuid.NewString(), + filter: filter, + eventCh: make(chan *common.TxnEvent, 1000), + resolvedTs: newTsWithMutex(startTs), + ddlActions: make(chan *heartbeatpb.DispatcherAction, 16), + tableSpanStatusesChan: tableSpanStatusesChan, + sink: sink, + tableSpan: &common.DDLSpan, + componentStatus: newComponentStateWithMutex(heartbeatpb.ComponentState_Working), + cancel: cancel, + //MemoryUsage: dispatcher.NewMemoryUsage(), + } + tableTriggerEventDispatcher.sink.AddTableSpan(tableTriggerEventDispatcher.tableSpan) + + tableTriggerEventDispatcher.wg.Add(1) + go tableTriggerEventDispatcher.DispatcherEvents(ctx) + + log.Info("table trigger event dispatcher created", zap.Any("DispatcherID", tableTriggerEventDispatcher.id)) + + return tableTriggerEventDispatcher } -func (d *TableTriggerEventDispatcher) GetSink() sink.Sink { - return d.Sink +func (d *TableTriggerEventDispatcher) DispatcherEvents(ctx context.Context) { + defer d.wg.Done() + for { + select { + case <-ctx.Done(): + return + case event := <-d.GetEventChan(): + if event.IsDDLEvent() { + AddDDLEventToSinkWhenAvailable(d, event) + } else { + d.resolvedTs.Set(event.ResolvedTs) + } + } + } } -func (d *TableTriggerEventDispatcher) GetTableSpan() *common.TableSpan { - return d.TableSpan +func (d *TableTriggerEventDispatcher) GetSink() sink.Sink { + return d.sink } -func (d *TableTriggerEventDispatcher) GetState() *State { - return d.State +func (d *TableTriggerEventDispatcher) GetTableSpan() *common.TableSpan { + return d.tableSpan } func (d *TableTriggerEventDispatcher) GetEventChan() chan *common.TxnEvent { - return d.Ch + return d.eventCh } func (d *TableTriggerEventDispatcher) GetResolvedTs() uint64 { - return d.ResolvedTs + return d.resolvedTs.Get() } func (d *TableTriggerEventDispatcher) GetId() string { - return d.Id + return d.id } func (d *TableTriggerEventDispatcher) GetDispatcherType() DispatcherType { return TableTriggerEventDispatcherType } -func (d *TableTriggerEventDispatcher) GetHeartBeatChan() chan *HeartBeatResponseMessage { - return d.HeartbeatChan +func (d *TableTriggerEventDispatcher) GetDDLActions() chan *heartbeatpb.DispatcherAction { + return d.ddlActions } -func (d *TableTriggerEventDispatcher) UpdateResolvedTs(ts uint64) { - d.ResolvedTs = ts +func (d *TableTriggerEventDispatcher) GetTableSpanStatusesChan() chan *heartbeatpb.TableSpanStatus { + return d.tableSpanStatusesChan } -func (d *TableTriggerEventDispatcher) GetSyncPointInfo() *SyncPointInfo { - log.Error("TableEventDispatcher.GetSyncPointInfo is not implemented") - return nil +func (d *TableTriggerEventDispatcher) UpdateResolvedTs(ts uint64) { + d.GetEventChan() <- &common.TxnEvent{ResolvedTs: ts} } -func (d *TableTriggerEventDispatcher) GetMemoryUsage() *MemoryUsage { - return d.MemoryUsage -} +// func (d *TableTriggerEventDispatcher) GetSyncPointInfo() *SyncPointInfo { +// log.Error("TableEventDispatcher.GetSyncPointInfo is not implemented") +// return nil +// } + +// func (d *TableTriggerEventDispatcher) GetMemoryUsage() *MemoryUsage { +// return d.MemoryUsage +// } func (d *TableTriggerEventDispatcher) PushTxnEvent(event *common.TxnEvent) { //d.GetMemoryUsage().Add(event.CommitTs, event.MemoryCost()) - d.Ch <- event // 换成一个函数 + d.GetEventChan() <- event // 换成一个函数 } -func (d *TableTriggerEventDispatcher) GetCheckpointTs() uint64 { return 0 } +func (d *TableTriggerEventDispatcher) GetCheckpointTs() uint64 { + checkpointTs := d.GetSink().GetCheckpointTs(d.GetTableSpan()) + if checkpointTs == 0 { + // 说明从没有数据写到过 sink,则选择用 resolveTs 作为 checkpointTs + checkpointTs = d.GetResolvedTs() + } + return checkpointTs +} func (d *TableTriggerEventDispatcher) GetComponentStatus() heartbeatpb.ComponentState { return heartbeatpb.ComponentState_Working @@ -160,7 +216,7 @@ func (d *TableTriggerEventDispatcher) TryClose() (w heartbeatpb.Watermark, ok bo if d.sink.IsEmpty(d.tableSpan) { d.sink.RemoveTableSpan(d.tableSpan) w.CheckpointTs = w.GetCheckpointTs() - w.ResolvedTs = d.ResolvedTs + w.ResolvedTs = d.GetResolvedTs() return w, true } return w, false diff --git a/downstreamadapter/dispatchermanager/event_dispatcher_manager.go b/downstreamadapter/dispatchermanager/event_dispatcher_manager.go index 5e85f4630..9533f0f2d 100644 --- a/downstreamadapter/dispatchermanager/event_dispatcher_manager.go +++ b/downstreamadapter/dispatchermanager/event_dispatcher_manager.go @@ -19,7 +19,6 @@ import ( "sync/atomic" "time" - "github.com/google/uuid" "github.com/pingcap/log" "github.com/pingcap/tiflow/pkg/filter" "github.com/prometheus/client_golang/prometheus" @@ -147,9 +146,6 @@ func (e *EventDispatcherManager) Init(startTs uint64) error { e.sink = sink.NewMysqlSink(e.changefeedID, 16, cfg, db) - //Init Table Trigger Event Dispatcher - e.tableTriggerEventDispatcher = e.newTableTriggerEventDispatcher(startTs) - // get heartbeat response from HeartBeatResponseQueue, and send to each dispatcher e.wg.Add(1) go func() { @@ -169,6 +165,9 @@ func (e *EventDispatcherManager) Init(startTs uint64) error { } }() + //Init Table Trigger Event Dispatcher + e.tableTriggerEventDispatcher = e.newTableTriggerEventDispatcher(startTs) + return nil } @@ -299,7 +298,6 @@ func (e *EventDispatcherManager) CollectHeartbeatInfoWhenStatesChanged(ctx conte var message heartbeatpb.HeartBeatRequest message.ChangefeedID = e.changefeedID.ID - //message := e.CollectHeartbeatInfo(false) message.Statuses = statusMessage e.GetHeartbeatRequestQueue().Enqueue(&HeartBeatRequestWithTargetID{TargetID: e.GetMaintainerID(), Request: &message}) } @@ -334,22 +332,11 @@ func (e *EventDispatcherManager) cleanTableEventDispatcher(tableSpan *common.Tab } func (e *EventDispatcherManager) newTableTriggerEventDispatcher(startTs uint64) *dispatcher.TableTriggerEventDispatcher { - tableTriggerEventDispatcher := &dispatcher.TableTriggerEventDispatcher{ - Id: uuid.NewString(), - Filter: e.filter, - Ch: make(chan *common.TxnEvent, 1000), - ResolvedTs: startTs, - HeartbeatChan: make(chan *dispatcher.HeartBeatResponseMessage, 100), - sink: e.sink, - tableSpan: &common.DDLSpan, - State: dispatcher.NewState(), - //MemoryUsage: dispatcher.NewMemoryUsage(), - } + tableTriggerEventDispatcher := dispatcher.NewTableTriggerEventDispatcher(e.sink, startTs, e.tableSpanStatusesChan, e.filter) appcontext.GetService[*eventcollector.EventCollector](appcontext.EventCollector).RegisterDispatcher(tableTriggerEventDispatcher, startTs, toFilterConfigPB(e.config.Filter)) return tableTriggerEventDispatcher - } func toFilterConfigPB(filter *cfg.FilterConfig) *eventpb.FilterConfig { diff --git a/downstreamadapter/eventcollector/event_collector.go b/downstreamadapter/eventcollector/event_collector.go index d7ff37e27..6d643f931 100644 --- a/downstreamadapter/eventcollector/event_collector.go +++ b/downstreamadapter/eventcollector/event_collector.go @@ -196,11 +196,6 @@ func (c *EventCollector) RecvEventsMessage(ctx context.Context, msg *messaging.T } dispatcherID := txnEvent.DispatcherID - // log.Info("Recv TxnEvent", zap.Any("dispatcherID", dispatcherID), zap.Any("event is dml event", txnEvent.IsDMLEvent())) - // if txnEvent.IsDMLEvent() { - // rowEvent := txnEvent.GetRows()[0] - // log.Info("Recv TxnEvent", zap.Any("dispatcherID", dispatcherID), zap.Any("event info", rowEvent.CommitTs), zap.Any("table name", rowEvent.TableInfo.TableName)) - // } if dispatcherItem, ok := c.dispatcherMap.Get(dispatcherID); ok { // check whether need to update speed ratio @@ -229,7 +224,7 @@ func (c *EventCollector) RecvEventsMessage(ctx context.Context, msg *messaging.T // } //for _, txnEvent := range eventFeeds.TxnEvents { // TODO: message 改过以后重写,先串起来。 - if txnEvent.IsDMLEvent() { + if txnEvent.IsDMLEvent() || txnEvent.IsDDLEvent() { dispatcherItem.PushTxnEvent(txnEvent) c.metricDispatcherReceivedKVEventCount.Inc() } else { diff --git a/downstreamadapter/sink/mysql_sink.go b/downstreamadapter/sink/mysql_sink.go index 9373a0fc7..6c214bd7e 100644 --- a/downstreamadapter/sink/mysql_sink.go +++ b/downstreamadapter/sink/mysql_sink.go @@ -278,7 +278,7 @@ func (s *MysqlSink) GetCheckpointTs(tableSpan *common.TableSpan) uint64 { return math.MaxUint64 } - return tableStatus.getProgress().MaxCheckpointTs() + return tableStatus.getProgress().GetCheckpointTs() } func (s *MysqlSink) Close() { diff --git a/downstreamadapter/sink/types/table_progress.go b/downstreamadapter/sink/types/table_progress.go index 1cdee1e86..e47c104c9 100644 --- a/downstreamadapter/sink/types/table_progress.go +++ b/downstreamadapter/sink/types/table_progress.go @@ -77,7 +77,7 @@ func (p *TableProgress) Empty() bool { // 返回当前 tableSpan 中最大的 checkpointTs,也就是最大的 ts,并且 <= ts 之前的数据都已经成功写下去了 // 1. 假设目前 sink 还有没 flush 下去的 event,就拿最小的这个 event的 commitTs。 // 2. 反之,则选择收到过 event 中 commitTs 最大的那个。 -func (p *TableProgress) MaxCheckpointTs() uint64 { +func (p *TableProgress) GetCheckpointTs() uint64 { p.mutex.Lock() defer p.mutex.Unlock() From 2190ef2a1188db7fd6ed5207120c4ca15f8d2ae0 Mon Sep 17 00:00:00 2001 From: hongyunyan <649330952@qq.com> Date: Wed, 14 Aug 2024 21:55:27 +0800 Subject: [PATCH 03/12] support missing message --- downstreamadapter/dispatcher/dispatcher.go | 103 +++- .../dispatcher/table_event_dispatcher.go | 42 +- .../table_trigger_event_dispatcher.go | 37 ++ .../event_dispatcher_manager.go | 2 +- downstreamadapter/sink/mysql_sink.go | 9 + downstreamadapter/sink/sink.go | 1 + .../sink/types/table_progress.go | 6 + heartbeatpb/heartbeat.pb.go | 458 ++++++++++++++---- heartbeatpb/heartbeat.proto | 15 +- pkg/common/txn_event.go | 8 +- 10 files changed, 553 insertions(+), 128 deletions(-) diff --git a/downstreamadapter/dispatcher/dispatcher.go b/downstreamadapter/dispatcher/dispatcher.go index 24337128c..fd7cb5b8c 100644 --- a/downstreamadapter/dispatcher/dispatcher.go +++ b/downstreamadapter/dispatcher/dispatcher.go @@ -14,11 +14,14 @@ package dispatcher import ( + "context" + "sync" "time" "github.com/flowbehappy/tigate/downstreamadapter/sink" "github.com/flowbehappy/tigate/heartbeatpb" "github.com/flowbehappy/tigate/pkg/common" + "github.com/pingcap/tiflow/pkg/filter" ) /* @@ -36,9 +39,9 @@ the dispatcher can't send event to Sink continuously all the time, The workflow related to the dispatcher is as follows: - +------------+ +----------------+ +------------+ +--------+ +--------+ +------------+ - | LogService | --> | EventCollector | --> | Dispatcher | --> | Sink | --> | Worker | --> | Downstream | - +------------+ +----------------+ +------------+ +--------+ +--------+ +------------+ + +--------------+ +----------------+ +------------+ +--------+ +--------+ +------------+ + | EventService | --> | EventCollector | --> | Dispatcher | --> | Sink | --> | Worker | --> | Downstream | + +--------------+ +----------------+ +------------+ +--------+ +--------+ +------------+ | HeartBeatResponse | HeartBeatRequest | @@ -61,6 +64,7 @@ type Dispatcher interface { GetId() string GetDispatcherType() DispatcherType GetDDLActions() chan *heartbeatpb.DispatcherAction + GetACKs() chan *heartbeatpb.ACK //GetSyncPointInfo() *SyncPointInfo //GetMemoryUsage() *MemoryUsage // PushEvent(event *eventpb.TxnEvent) @@ -69,6 +73,11 @@ type Dispatcher interface { GetTableSpanStatusesChan() chan *heartbeatpb.TableSpanStatus TryClose() (w heartbeatpb.Watermark, ok bool) + GetFilter() filter.Filter + GetWG() *sync.WaitGroup + GetDDLPendingEvent() *common.TxnEvent + SetDDLPendingEvent(event *common.TxnEvent) + GetDDLFinishCh() chan struct{} } type DispatcherType uint64 @@ -95,11 +104,59 @@ type HeartBeatInfo struct { ComponentStatus heartbeatpb.ComponentState } +func HandleDDLActions(d Dispatcher, ctx context.Context) { + defer d.GetWG().Done() + sink := d.GetSink() + tableSpan := d.GetTableSpan() + for { + select { + case <-ctx.Done(): + return + case dispatcherAction := <-d.GetDDLActions(): + event := d.GetDDLPendingEvent() + if event == nil { + // 只可能出现在 event 已经推进了,但是还重复收到了 action 消息的时候,则重发包含 checkpointTs 的心跳 + d.GetTableSpanStatusesChan() <- &heartbeatpb.TableSpanStatus{ + Span: tableSpan.TableSpan, + ComponentStatus: heartbeatpb.ComponentState_Working, + CheckpointTs: d.GetCheckpointTs(), + } + continue + } + if dispatcherAction.CommitTs == event.CommitTs { + if dispatcherAction.Action == heartbeatpb.Action_Write { + sink.AddDDLAndSyncPointEvent(tableSpan, event) // 这个是同步写,所以写完的时候 sink 也 available 了 + } else { + sink.PassDDLAndSyncPointEvent(tableSpan, event) // 为了更新 tableProgress,避免 checkpointTs 计算的 corner case + } + d.GetTableSpanStatusesChan() <- &heartbeatpb.TableSpanStatus{ + Span: tableSpan.TableSpan, + ComponentStatus: heartbeatpb.ComponentState_Working, + CheckpointTs: d.GetCheckpointTs(), + } + d.GetDDLFinishCh() <- struct{}{} + return + } + } + } +} + // 1. 如果是单表内的 ddl,达到下推的条件为: sink 中没有还没执行完的当前表的 event // 2. 如果是多表内的 ddl 或者是表间的 ddl,则需要满足的条件为: // 2.1 sink 中没有还没执行完的当前表的 event // 2.2 maintainer 通知自己可以 write 或者 pass event +// +// TODO:特殊处理有 add index 的逻辑 func AddDDLEventToSinkWhenAvailable(d Dispatcher, event *common.TxnEvent) { + //filter := d.GetFilter() + // TODO: filter 支持 + // 判断 ddl 是否需要处理,如果不需要处理,直接返回 + // if filter.ShouldIgnoreDDLEvent(event.GetDDLEvent()) { + // return + // } + + // 需要根据 filter 来判断 ddl.Query 中是否需要调整,只针对 query 中包含多个 sql 语句。所以 的ddl 传来的时候,需要对应传 sql 对应的 table id 信息,用于过滤 + sink := d.GetSink() tableSpan := d.GetTableSpan() if event.IsSingleTableDDL() { @@ -121,7 +178,10 @@ func AddDDLEventToSinkWhenAvailable(d Dispatcher, event *common.TxnEvent) { } } - d.GetTableSpanStatusesChan() <- &heartbeatpb.TableSpanStatus{ + d.SetDDLPendingEvent(event) + + // TODO:消息需要保证发送后收到 ack 才可以停止重发,具体重发时间需要调整 + message := &heartbeatpb.TableSpanStatus{ Span: tableSpan.TableSpan, ComponentStatus: heartbeatpb.ComponentState_Working, State: &heartbeatpb.State{ @@ -132,38 +192,29 @@ func AddDDLEventToSinkWhenAvailable(d Dispatcher, event *common.TxnEvent) { NeedAddedTableSpan: event.GetNeedAddedTableSpan(), }, } - + d.GetTableSpanStatusesChan() <- message + timer := time.NewTimer(time.Millisecond * 100) +loop: for { - dispatcherAction := <-d.GetDDLActions() - if dispatcherAction.CommitTs == event.CommitTs { - if dispatcherAction.Action == heartbeatpb.Action_Write { - sink.AddDDLAndSyncPointEvent(tableSpan, event) // 这个是同步写,所以写完的时候 sink 也 available 了 - // 写完马上通知 maintainer 推进 checkpointTs - d.GetTableSpanStatusesChan() <- &heartbeatpb.TableSpanStatus{ - Span: tableSpan.TableSpan, - ComponentStatus: heartbeatpb.ComponentState_Working, - CheckpointTs: d.GetCheckpointTs(), - } + select { + case <-timer.C: + // 重发消息 + d.GetTableSpanStatusesChan() <- message + case ack := <-d.GetACKs(): + if ack.CommitTs == event.CommitTs { + break loop } - return } } + + // 收到 ack 以后可以开始等 actions 来进行处理,等待 finish 信号 + <-d.GetDDLFinishCh() } func CollectDispatcherHeartBeatInfo(d Dispatcher, h *HeartBeatInfo) { - // use checkpointTs to release memory usage //d.GetMemoryUsage().Release(checkpointTs) - //state := d.GetState() - // return &HeartBeatInfo{ - // // IsBlocked: state.isBlocked, - // // BlockTs: state.blockTs, - // // BlockTableSpan: state.blockTableSpan, - // CheckpointTs: checkpointTs, - // //TableSpan: d.GetTableSpan(), - // Id: d.GetId(), - // } h.Watermark.CheckpointTs = d.GetCheckpointTs() h.Watermark.ResolvedTs = d.GetResolvedTs() h.Id = d.GetId() diff --git a/downstreamadapter/dispatcher/table_event_dispatcher.go b/downstreamadapter/dispatcher/table_event_dispatcher.go index 74c712f26..8c5e3cfe5 100644 --- a/downstreamadapter/dispatcher/table_event_dispatcher.go +++ b/downstreamadapter/dispatcher/table_event_dispatcher.go @@ -23,6 +23,7 @@ import ( "github.com/flowbehappy/tigate/pkg/common" "github.com/google/uuid" "github.com/pingcap/log" + "github.com/pingcap/tiflow/pkg/filter" "go.uber.org/zap" ) @@ -96,6 +97,7 @@ type TableEventDispatcher struct { sink sink.Sink ddlActions chan *heartbeatpb.DispatcherAction + acks chan *heartbeatpb.ACK tableSpanStatusesChan chan *heartbeatpb.TableSpanStatus //SyncPointInfo *SyncPointInfo @@ -104,13 +106,18 @@ type TableEventDispatcher struct { componentStatus *ComponentStateWithMutex + filter filter.Filter + resolvedTs *TsWithMutex // 用来记 eventChan 中目前收到的 event 中收到的最大的 commitTs - 1,不代表 dispatcher 的 checkpointTs cancel context.CancelFunc wg sync.WaitGroup + + ddlPendingEvent *common.TxnEvent + ddlFinishCh chan struct{} } -func NewTableEventDispatcher(tableSpan *common.TableSpan, sink sink.Sink, startTs uint64, syncPointInfo *SyncPointInfo, tableSpanStatusesChan chan *heartbeatpb.TableSpanStatus) *TableEventDispatcher { +func NewTableEventDispatcher(tableSpan *common.TableSpan, sink sink.Sink, startTs uint64, syncPointInfo *SyncPointInfo, tableSpanStatusesChan chan *heartbeatpb.TableSpanStatus, filter filter.Filter) *TableEventDispatcher { ctx, cancel := context.WithCancel(context.Background()) tableEventDispatcher := &TableEventDispatcher{ id: uuid.NewString(), @@ -118,17 +125,23 @@ func NewTableEventDispatcher(tableSpan *common.TableSpan, sink sink.Sink, startT tableSpan: tableSpan, sink: sink, ddlActions: make(chan *heartbeatpb.DispatcherAction, 16), + acks: make(chan *heartbeatpb.ACK, 16), tableSpanStatusesChan: tableSpanStatusesChan, //SyncPointInfo: syncPointInfo, //MemoryUsage: NewMemoryUsage(), componentStatus: newComponentStateWithMutex(heartbeatpb.ComponentState_Working), resolvedTs: newTsWithMutex(startTs), cancel: cancel, + filter: filter, + ddlFinishCh: make(chan struct{}), } tableEventDispatcher.sink.AddTableSpan(tableSpan) tableEventDispatcher.wg.Add(1) go tableEventDispatcher.DispatcherEvents(ctx) + tableEventDispatcher.wg.Add(1) + go HandleDDLActions(tableEventDispatcher, ctx) + log.Info("table event dispatcher created", zap.Any("DispatcherID", tableEventDispatcher.id)) return tableEventDispatcher @@ -195,6 +208,10 @@ func (d *TableEventDispatcher) GetDDLActions() chan *heartbeatpb.DispatcherActio return d.ddlActions } +func (d *TableEventDispatcher) GetACKs() chan *heartbeatpb.ACK { + return d.acks +} + func (d *TableEventDispatcher) GetTableSpanStatusesChan() chan *heartbeatpb.TableSpanStatus { return d.tableSpanStatusesChan } @@ -238,3 +255,26 @@ func (d *TableEventDispatcher) TryClose() (w heartbeatpb.Watermark, ok bool) { func (d *TableEventDispatcher) GetComponentStatus() heartbeatpb.ComponentState { return d.componentStatus.Get() } + +func (d *TableEventDispatcher) GetFilter() filter.Filter { + return d.filter +} + +func (d *TableEventDispatcher) GetWG() *sync.WaitGroup { + return &d.wg +} + +func (d *TableEventDispatcher) GetDDLPendingEvent() *common.TxnEvent { + return d.ddlPendingEvent +} + +func (d *TableEventDispatcher) SetDDLPendingEvent(event *common.TxnEvent) { + if d.ddlPendingEvent != nil { + log.Error("there is already a pending ddl event, can not set a new one") + return + } + d.ddlPendingEvent = event +} +func (d *TableEventDispatcher) GetDDLFinishCh() chan struct{} { + return d.ddlFinishCh +} diff --git a/downstreamadapter/dispatcher/table_trigger_event_dispatcher.go b/downstreamadapter/dispatcher/table_trigger_event_dispatcher.go index 5e7a0be4d..22a55facd 100644 --- a/downstreamadapter/dispatcher/table_trigger_event_dispatcher.go +++ b/downstreamadapter/dispatcher/table_trigger_event_dispatcher.go @@ -94,6 +94,7 @@ type TableTriggerEventDispatcher struct { filter filter.Filter sink sink.Sink ddlActions chan *heartbeatpb.DispatcherAction + acks chan *heartbeatpb.ACK tableSpan *common.TableSpan // 给一个特殊的 tableSpan resolvedTs *TsWithMutex componentStatus *ComponentStateWithMutex @@ -102,6 +103,9 @@ type TableTriggerEventDispatcher struct { wg sync.WaitGroup cancel context.CancelFunc + // ddl 相关的可以考虑塞进一个 struct 去 + ddlPendingEvent *common.TxnEvent + ddlFinishCh chan struct{} //MemoryUsage *MemoryUsage } @@ -113,11 +117,13 @@ func NewTableTriggerEventDispatcher(sink sink.Sink, startTs uint64, tableSpanSta eventCh: make(chan *common.TxnEvent, 1000), resolvedTs: newTsWithMutex(startTs), ddlActions: make(chan *heartbeatpb.DispatcherAction, 16), + acks: make(chan *heartbeatpb.ACK, 16), tableSpanStatusesChan: tableSpanStatusesChan, sink: sink, tableSpan: &common.DDLSpan, componentStatus: newComponentStateWithMutex(heartbeatpb.ComponentState_Working), cancel: cancel, + ddlFinishCh: make(chan struct{}), //MemoryUsage: dispatcher.NewMemoryUsage(), } tableTriggerEventDispatcher.sink.AddTableSpan(tableTriggerEventDispatcher.tableSpan) @@ -125,6 +131,9 @@ func NewTableTriggerEventDispatcher(sink sink.Sink, startTs uint64, tableSpanSta tableTriggerEventDispatcher.wg.Add(1) go tableTriggerEventDispatcher.DispatcherEvents(ctx) + tableTriggerEventDispatcher.wg.Add(1) + go HandleDDLActions(tableTriggerEventDispatcher, ctx) + log.Info("table trigger event dispatcher created", zap.Any("DispatcherID", tableTriggerEventDispatcher.id)) return tableTriggerEventDispatcher @@ -174,6 +183,10 @@ func (d *TableTriggerEventDispatcher) GetDDLActions() chan *heartbeatpb.Dispatch return d.ddlActions } +func (d *TableTriggerEventDispatcher) GetACKs() chan *heartbeatpb.ACK { + return d.acks +} + func (d *TableTriggerEventDispatcher) GetTableSpanStatusesChan() chan *heartbeatpb.TableSpanStatus { return d.tableSpanStatusesChan } @@ -221,3 +234,27 @@ func (d *TableTriggerEventDispatcher) TryClose() (w heartbeatpb.Watermark, ok bo } return w, false } + +func (d *TableTriggerEventDispatcher) GetFilter() filter.Filter { + return d.filter +} + +func (d *TableTriggerEventDispatcher) GetWG() *sync.WaitGroup { + return &d.wg +} + +func (d *TableTriggerEventDispatcher) GetDDLPendingEvent() *common.TxnEvent { + return d.ddlPendingEvent +} + +func (d *TableTriggerEventDispatcher) SetDDLPendingEvent(event *common.TxnEvent) { + if d.ddlPendingEvent != nil { + log.Error("there is already a pending ddl event, can not set a new one") + return + } + d.ddlPendingEvent = event +} + +func (d *TableTriggerEventDispatcher) GetDDLFinishCh() chan struct{} { + return d.ddlFinishCh +} diff --git a/downstreamadapter/dispatchermanager/event_dispatcher_manager.go b/downstreamadapter/dispatchermanager/event_dispatcher_manager.go index 9533f0f2d..8007687ff 100644 --- a/downstreamadapter/dispatchermanager/event_dispatcher_manager.go +++ b/downstreamadapter/dispatchermanager/event_dispatcher_manager.go @@ -245,7 +245,7 @@ func (e *EventDispatcherManager) NewTableEventDispatcher(tableSpan *common.Table syncPointInfo.EnableSyncPoint = false } */ - tableEventDispatcher := dispatcher.NewTableEventDispatcher(tableSpan, e.sink, startTs, nil, e.tableSpanStatusesChan) + tableEventDispatcher := dispatcher.NewTableEventDispatcher(tableSpan, e.sink, startTs, nil, e.tableSpanStatusesChan, e.filter) appcontext.GetService[*eventcollector.EventCollector](appcontext.EventCollector).RegisterDispatcher(tableEventDispatcher, startTs, nil) diff --git a/downstreamadapter/sink/mysql_sink.go b/downstreamadapter/sink/mysql_sink.go index 6c214bd7e..4dca0d34f 100644 --- a/downstreamadapter/sink/mysql_sink.go +++ b/downstreamadapter/sink/mysql_sink.go @@ -205,6 +205,15 @@ func (s *MysqlSink) AddDMLEvent(tableSpan *common.TableSpan, event *common.TxnEv tableStatus.getCh() <- event } +func (s *MysqlSink) PassDDLAndSyncPointEvent(tableSpan *common.TableSpan, event *common.TxnEvent) { + tableStatus, ok := s.tableStatuses.Get(tableSpan) + if !ok { + log.Error("unknown Span for Mysql Sink: ", zap.Any("tableSpan", tableSpan)) + return + } + tableStatus.getProgress().Pass(event) +} + func (s *MysqlSink) AddDDLAndSyncPointEvent(tableSpan *common.TableSpan, event *common.TxnEvent) { tableStatus, ok := s.tableStatuses.Get(tableSpan) if !ok { diff --git a/downstreamadapter/sink/sink.go b/downstreamadapter/sink/sink.go index c39fd0a22..c5d3796de 100644 --- a/downstreamadapter/sink/sink.go +++ b/downstreamadapter/sink/sink.go @@ -18,6 +18,7 @@ import "github.com/flowbehappy/tigate/pkg/common" type Sink interface { AddDMLEvent(tableSpan *common.TableSpan, event *common.TxnEvent) AddDDLAndSyncPointEvent(tableSpan *common.TableSpan, event *common.TxnEvent) + PassDDLAndSyncPointEvent(tableSpan *common.TableSpan, event *common.TxnEvent) IsEmpty(tableSpan *common.TableSpan) bool AddTableSpan(tableSpan *common.TableSpan) RemoveTableSpan(tableSpan *common.TableSpan) diff --git a/downstreamadapter/sink/types/table_progress.go b/downstreamadapter/sink/types/table_progress.go index e47c104c9..684f8ea29 100644 --- a/downstreamadapter/sink/types/table_progress.go +++ b/downstreamadapter/sink/types/table_progress.go @@ -74,6 +74,12 @@ func (p *TableProgress) Empty() bool { return p.list.Len() == 0 } +func (p *TableProgress) Pass(event *common.TxnEvent) { + p.mutex.Lock() + defer p.mutex.Unlock() + p.maxCommitTs = event.CommitTs +} + // 返回当前 tableSpan 中最大的 checkpointTs,也就是最大的 ts,并且 <= ts 之前的数据都已经成功写下去了 // 1. 假设目前 sink 还有没 flush 下去的 event,就拿最小的这个 event的 commitTs。 // 2. 反之,则选择收到过 event 中 commitTs 最大的那个。 diff --git a/heartbeatpb/heartbeat.pb.go b/heartbeatpb/heartbeat.pb.go index fcb84c3e5..6ed527516 100644 --- a/heartbeatpb/heartbeat.pb.go +++ b/heartbeatpb/heartbeat.pb.go @@ -368,16 +368,69 @@ func (m *DispatcherAction) GetCommitTs() uint64 { return 0 } +type ACK struct { + Span *TableSpan `protobuf:"bytes,1,opt,name=span,proto3" json:"span,omitempty"` + CommitTs uint64 `protobuf:"varint,2,opt,name=CommitTs,proto3" json:"CommitTs,omitempty"` +} + +func (m *ACK) Reset() { *m = ACK{} } +func (m *ACK) String() string { return proto.CompactTextString(m) } +func (*ACK) ProtoMessage() {} +func (*ACK) Descriptor() ([]byte, []int) { + return fileDescriptor_6d584080fdadb670, []int{4} +} +func (m *ACK) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *ACK) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_ACK.Marshal(b, m, deterministic) + } else { + b = b[:cap(b)] + n, err := m.MarshalToSizedBuffer(b) + if err != nil { + return nil, err + } + return b[:n], nil + } +} +func (m *ACK) XXX_Merge(src proto.Message) { + xxx_messageInfo_ACK.Merge(m, src) +} +func (m *ACK) XXX_Size() int { + return m.Size() +} +func (m *ACK) XXX_DiscardUnknown() { + xxx_messageInfo_ACK.DiscardUnknown(m) +} + +var xxx_messageInfo_ACK proto.InternalMessageInfo + +func (m *ACK) GetSpan() *TableSpan { + if m != nil { + return m.Span + } + return nil +} + +func (m *ACK) GetCommitTs() uint64 { + if m != nil { + return m.CommitTs + } + return 0 +} + type HeartBeatResponse struct { Actions []*DispatcherAction `protobuf:"bytes,1,rep,name=actions,proto3" json:"actions,omitempty"` ChangefeedID string `protobuf:"bytes,2,opt,name=changefeedID,proto3" json:"changefeedID,omitempty"` + Acks []*ACK `protobuf:"bytes,3,rep,name=acks,proto3" json:"acks,omitempty"` } func (m *HeartBeatResponse) Reset() { *m = HeartBeatResponse{} } func (m *HeartBeatResponse) String() string { return proto.CompactTextString(m) } func (*HeartBeatResponse) ProtoMessage() {} func (*HeartBeatResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_6d584080fdadb670, []int{4} + return fileDescriptor_6d584080fdadb670, []int{5} } func (m *HeartBeatResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -420,6 +473,13 @@ func (m *HeartBeatResponse) GetChangefeedID() string { return "" } +func (m *HeartBeatResponse) GetAcks() []*ACK { + if m != nil { + return m.Acks + } + return nil +} + type DispatcherConfig struct { Span *TableSpan `protobuf:"bytes,1,opt,name=span,proto3" json:"span,omitempty"` StartTs uint64 `protobuf:"varint,2,opt,name=startTs,proto3" json:"startTs,omitempty"` @@ -429,7 +489,7 @@ func (m *DispatcherConfig) Reset() { *m = DispatcherConfig{} } func (m *DispatcherConfig) String() string { return proto.CompactTextString(m) } func (*DispatcherConfig) ProtoMessage() {} func (*DispatcherConfig) Descriptor() ([]byte, []int) { - return fileDescriptor_6d584080fdadb670, []int{5} + return fileDescriptor_6d584080fdadb670, []int{6} } func (m *DispatcherConfig) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -483,7 +543,7 @@ func (m *ScheduleDispatcherRequest) Reset() { *m = ScheduleDispatcherReq func (m *ScheduleDispatcherRequest) String() string { return proto.CompactTextString(m) } func (*ScheduleDispatcherRequest) ProtoMessage() {} func (*ScheduleDispatcherRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_6d584080fdadb670, []int{6} + return fileDescriptor_6d584080fdadb670, []int{7} } func (m *ScheduleDispatcherRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -548,7 +608,7 @@ func (m *MaintainerHeartbeat) Reset() { *m = MaintainerHeartbeat{} } func (m *MaintainerHeartbeat) String() string { return proto.CompactTextString(m) } func (*MaintainerHeartbeat) ProtoMessage() {} func (*MaintainerHeartbeat) Descriptor() ([]byte, []int) { - return fileDescriptor_6d584080fdadb670, []int{7} + return fileDescriptor_6d584080fdadb670, []int{8} } func (m *MaintainerHeartbeat) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -597,7 +657,7 @@ func (m *MaintainerStatus) Reset() { *m = MaintainerStatus{} } func (m *MaintainerStatus) String() string { return proto.CompactTextString(m) } func (*MaintainerStatus) ProtoMessage() {} func (*MaintainerStatus) Descriptor() ([]byte, []int) { - return fileDescriptor_6d584080fdadb670, []int{8} + return fileDescriptor_6d584080fdadb670, []int{9} } func (m *MaintainerStatus) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -676,7 +736,7 @@ func (m *CoordinatorBootstrapRequest) Reset() { *m = CoordinatorBootstra func (m *CoordinatorBootstrapRequest) String() string { return proto.CompactTextString(m) } func (*CoordinatorBootstrapRequest) ProtoMessage() {} func (*CoordinatorBootstrapRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_6d584080fdadb670, []int{9} + return fileDescriptor_6d584080fdadb670, []int{10} } func (m *CoordinatorBootstrapRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -720,7 +780,7 @@ func (m *CoordinatorBootstrapResponse) Reset() { *m = CoordinatorBootstr func (m *CoordinatorBootstrapResponse) String() string { return proto.CompactTextString(m) } func (*CoordinatorBootstrapResponse) ProtoMessage() {} func (*CoordinatorBootstrapResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_6d584080fdadb670, []int{10} + return fileDescriptor_6d584080fdadb670, []int{11} } func (m *CoordinatorBootstrapResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -765,7 +825,7 @@ func (m *DispatchMaintainerRequest) Reset() { *m = DispatchMaintainerReq func (m *DispatchMaintainerRequest) String() string { return proto.CompactTextString(m) } func (*DispatchMaintainerRequest) ProtoMessage() {} func (*DispatchMaintainerRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_6d584080fdadb670, []int{11} + return fileDescriptor_6d584080fdadb670, []int{12} } func (m *DispatchMaintainerRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -819,7 +879,7 @@ func (m *AddMaintainerRequest) Reset() { *m = AddMaintainerRequest{} } func (m *AddMaintainerRequest) String() string { return proto.CompactTextString(m) } func (*AddMaintainerRequest) ProtoMessage() {} func (*AddMaintainerRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_6d584080fdadb670, []int{12} + return fileDescriptor_6d584080fdadb670, []int{13} } func (m *AddMaintainerRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -885,7 +945,7 @@ func (m *RemoveMaintainerRequest) Reset() { *m = RemoveMaintainerRequest func (m *RemoveMaintainerRequest) String() string { return proto.CompactTextString(m) } func (*RemoveMaintainerRequest) ProtoMessage() {} func (*RemoveMaintainerRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_6d584080fdadb670, []int{13} + return fileDescriptor_6d584080fdadb670, []int{14} } func (m *RemoveMaintainerRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -937,7 +997,7 @@ func (m *MaintainerBootstrapRequest) Reset() { *m = MaintainerBootstrapR func (m *MaintainerBootstrapRequest) String() string { return proto.CompactTextString(m) } func (*MaintainerBootstrapRequest) ProtoMessage() {} func (*MaintainerBootstrapRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_6d584080fdadb670, []int{14} + return fileDescriptor_6d584080fdadb670, []int{15} } func (m *MaintainerBootstrapRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -989,7 +1049,7 @@ func (m *MaintainerBootstrapResponse) Reset() { *m = MaintainerBootstrap func (m *MaintainerBootstrapResponse) String() string { return proto.CompactTextString(m) } func (*MaintainerBootstrapResponse) ProtoMessage() {} func (*MaintainerBootstrapResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_6d584080fdadb670, []int{15} + return fileDescriptor_6d584080fdadb670, []int{16} } func (m *MaintainerBootstrapResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1040,7 +1100,7 @@ func (m *MaintainerCloseRequest) Reset() { *m = MaintainerCloseRequest{} func (m *MaintainerCloseRequest) String() string { return proto.CompactTextString(m) } func (*MaintainerCloseRequest) ProtoMessage() {} func (*MaintainerCloseRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_6d584080fdadb670, []int{16} + return fileDescriptor_6d584080fdadb670, []int{17} } func (m *MaintainerCloseRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1085,7 +1145,7 @@ func (m *MaintainerCloseResponse) Reset() { *m = MaintainerCloseResponse func (m *MaintainerCloseResponse) String() string { return proto.CompactTextString(m) } func (*MaintainerCloseResponse) ProtoMessage() {} func (*MaintainerCloseResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_6d584080fdadb670, []int{17} + return fileDescriptor_6d584080fdadb670, []int{18} } func (m *MaintainerCloseResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1140,7 +1200,7 @@ func (m *State) Reset() { *m = State{} } func (m *State) String() string { return proto.CompactTextString(m) } func (*State) ProtoMessage() {} func (*State) Descriptor() ([]byte, []int) { - return fileDescriptor_6d584080fdadb670, []int{18} + return fileDescriptor_6d584080fdadb670, []int{19} } func (m *State) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1215,7 +1275,7 @@ func (m *TableSpanStatus) Reset() { *m = TableSpanStatus{} } func (m *TableSpanStatus) String() string { return proto.CompactTextString(m) } func (*TableSpanStatus) ProtoMessage() {} func (*TableSpanStatus) Descriptor() ([]byte, []int) { - return fileDescriptor_6d584080fdadb670, []int{19} + return fileDescriptor_6d584080fdadb670, []int{20} } func (m *TableSpanStatus) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1283,7 +1343,7 @@ func (m *RunningError) Reset() { *m = RunningError{} } func (m *RunningError) String() string { return proto.CompactTextString(m) } func (*RunningError) ProtoMessage() {} func (*RunningError) Descriptor() ([]byte, []int) { - return fileDescriptor_6d584080fdadb670, []int{20} + return fileDescriptor_6d584080fdadb670, []int{21} } func (m *RunningError) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1348,6 +1408,7 @@ func init() { proto.RegisterType((*HeartBeatRequest)(nil), "heartbeatpb.HeartBeatRequest") proto.RegisterType((*Watermark)(nil), "heartbeatpb.Watermark") proto.RegisterType((*DispatcherAction)(nil), "heartbeatpb.DispatcherAction") + proto.RegisterType((*ACK)(nil), "heartbeatpb.ACK") proto.RegisterType((*HeartBeatResponse)(nil), "heartbeatpb.HeartBeatResponse") proto.RegisterType((*DispatcherConfig)(nil), "heartbeatpb.DispatcherConfig") proto.RegisterType((*ScheduleDispatcherRequest)(nil), "heartbeatpb.ScheduleDispatcherRequest") @@ -1370,79 +1431,81 @@ func init() { func init() { proto.RegisterFile("heartbeatpb/heartbeat.proto", fileDescriptor_6d584080fdadb670) } var fileDescriptor_6d584080fdadb670 = []byte{ - // 1140 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xa4, 0x57, 0xcf, 0x6f, 0x1b, 0xc5, - 0x17, 0xcf, 0xfa, 0xb7, 0x9f, 0x5d, 0xc7, 0x9d, 0x56, 0xa9, 0xd3, 0x24, 0x56, 0xba, 0xdf, 0xef, - 0xc1, 0xa4, 0x52, 0x22, 0x52, 0x50, 0x41, 0x42, 0x48, 0x89, 0xd3, 0xaa, 0x2d, 0x02, 0xa2, 0x75, - 0x50, 0x5a, 0x2e, 0xd1, 0x78, 0xf7, 0xc5, 0x5e, 0xc5, 0x9e, 0x59, 0x66, 0xc6, 0x89, 0x22, 0xee, - 0x70, 0x04, 0x89, 0xff, 0x84, 0x33, 0x7f, 0x00, 0xc7, 0x1e, 0x38, 0x70, 0x42, 0x28, 0xf9, 0x47, - 0xd0, 0xcc, 0xee, 0xda, 0xbb, 0xb6, 0x13, 0x62, 0x71, 0x9b, 0xf7, 0xeb, 0x33, 0x6f, 0xde, 0x7c, - 0xde, 0xbc, 0x5d, 0x58, 0xeb, 0x23, 0x15, 0xaa, 0x8b, 0x54, 0x05, 0xdd, 0x9d, 0xf1, 0x7a, 0x3b, - 0x10, 0x5c, 0x71, 0x52, 0x49, 0x18, 0xed, 0x77, 0x50, 0x3e, 0xa2, 0xdd, 0x01, 0x76, 0x02, 0xca, - 0x48, 0x03, 0x8a, 0x46, 0x78, 0x7d, 0xd0, 0xb0, 0x36, 0xad, 0x56, 0xce, 0x89, 0x45, 0xf2, 0x18, - 0x4a, 0x1d, 0x45, 0x85, 0xfa, 0x02, 0x2f, 0x1b, 0x99, 0x4d, 0xab, 0x55, 0x75, 0xc6, 0x32, 0x59, - 0x81, 0xc2, 0x0b, 0xe6, 0x69, 0x4b, 0xd6, 0x58, 0x22, 0xc9, 0xfe, 0x35, 0x03, 0xf5, 0x57, 0x7a, - 0xab, 0x7d, 0xa4, 0xca, 0xc1, 0xef, 0x46, 0x28, 0x15, 0xb1, 0xa1, 0xea, 0xf6, 0x29, 0xeb, 0xe1, - 0x29, 0xa2, 0x17, 0xed, 0x53, 0x76, 0x52, 0x3a, 0xf2, 0x11, 0x94, 0x2f, 0xa8, 0x42, 0x31, 0xa4, - 0xe2, 0xcc, 0xec, 0x56, 0xd9, 0x5d, 0xd9, 0x4e, 0x24, 0xbd, 0x7d, 0x1c, 0x5b, 0x9d, 0x89, 0x23, - 0xf9, 0x04, 0x4a, 0x52, 0x51, 0x35, 0x92, 0x28, 0x1b, 0xd9, 0xcd, 0x6c, 0xab, 0xb2, 0xbb, 0x9e, - 0x0a, 0x1a, 0x1f, 0xb3, 0x63, 0xbc, 0x9c, 0xb1, 0x37, 0x69, 0xc1, 0xb2, 0xcb, 0x87, 0x01, 0x0e, - 0x50, 0x61, 0x68, 0x6c, 0xe4, 0x36, 0xad, 0x56, 0xc9, 0x99, 0x56, 0x93, 0x67, 0x50, 0xbc, 0xa0, - 0x82, 0xf9, 0xac, 0xd7, 0xc8, 0x9b, 0xbc, 0x56, 0x53, 0x5b, 0x38, 0x23, 0xa6, 0x6d, 0x2f, 0x84, - 0xe0, 0xc2, 0x89, 0x3d, 0xc9, 0x53, 0xc8, 0xa2, 0x10, 0x8d, 0xc2, 0xbf, 0x05, 0x68, 0x2f, 0xfb, - 0x6b, 0x28, 0x8f, 0x4f, 0x17, 0x16, 0x0b, 0xdd, 0xb3, 0x80, 0xfb, 0x4c, 0x1d, 0xc9, 0xe8, 0x52, - 0x52, 0x3a, 0xd2, 0x04, 0x10, 0x28, 0xf9, 0xe0, 0x1c, 0xbd, 0x23, 0x69, 0xaa, 0x95, 0x73, 0x12, - 0x1a, 0xfb, 0x47, 0x0b, 0xea, 0x07, 0xbe, 0x0c, 0xa8, 0x72, 0xfb, 0x28, 0xf6, 0x5c, 0xe5, 0x73, - 0x46, 0xb6, 0x20, 0x27, 0x03, 0xca, 0x0c, 0xe0, 0x74, 0x71, 0xc7, 0x75, 0x72, 0x8c, 0x0f, 0x79, - 0x0a, 0x05, 0x6a, 0xa2, 0x0c, 0x78, 0x6d, 0xf7, 0x41, 0xca, 0x3b, 0x04, 0x74, 0x22, 0x17, 0xcd, - 0x93, 0x36, 0x1f, 0x0e, 0x7d, 0x9d, 0x6d, 0xd6, 0xe4, 0x32, 0x96, 0xed, 0x00, 0xee, 0x27, 0xe8, - 0x20, 0x03, 0xce, 0x24, 0x92, 0xe7, 0x50, 0x0c, 0x43, 0xf5, 0xe9, 0xf4, 0xa5, 0x6d, 0xa4, 0xe0, - 0xa7, 0x33, 0x77, 0x62, 0xef, 0x19, 0x22, 0x65, 0x66, 0x89, 0x64, 0xbf, 0x4d, 0x1e, 0xbd, 0xcd, - 0xd9, 0xa9, 0xdf, 0x5b, 0xe8, 0xe8, 0x0d, 0x28, 0x4a, 0xcd, 0xf2, 0x71, 0x61, 0x63, 0xd1, 0xfe, - 0xcb, 0x82, 0xd5, 0x8e, 0xdb, 0x47, 0x6f, 0x34, 0xc0, 0xc9, 0x16, 0x8b, 0x90, 0xfc, 0x63, 0x28, - 0xb8, 0x26, 0xa3, 0x88, 0xe1, 0x37, 0x9d, 0x3b, 0x4c, 0xdb, 0x89, 0x9c, 0x49, 0x1b, 0x6a, 0x32, - 0xda, 0x37, 0xac, 0x88, 0x29, 0x73, 0x6d, 0x77, 0x2d, 0x15, 0xde, 0x49, 0xb9, 0x38, 0x53, 0x21, - 0xe4, 0x09, 0x54, 0x7d, 0x79, 0x22, 0xd1, 0xe5, 0xcc, 0xa3, 0xe2, 0x32, 0x62, 0x7b, 0xc5, 0x97, - 0x9d, 0x58, 0x65, 0x1f, 0xc2, 0x83, 0x2f, 0xa9, 0xcf, 0x14, 0xf5, 0x19, 0x8a, 0x57, 0x31, 0x34, - 0xf9, 0x34, 0xd1, 0x64, 0xf3, 0xee, 0x6b, 0x12, 0x33, 0xdd, 0x65, 0xf6, 0x4f, 0x19, 0xa8, 0x4f, - 0x9b, 0xef, 0x54, 0xa9, 0x0d, 0x00, 0xbd, 0x3a, 0xd1, 0x48, 0x18, 0xdd, 0x73, 0x59, 0x6b, 0x34, - 0x06, 0x92, 0x0f, 0x21, 0x1f, 0x5a, 0xe6, 0x15, 0xa2, 0xcd, 0x87, 0x01, 0x67, 0xc8, 0x94, 0xf1, - 0x75, 0x42, 0x4f, 0xf2, 0x3f, 0xb8, 0x37, 0xe9, 0xa1, 0x13, 0x15, 0xb6, 0xfb, 0x74, 0x63, 0xa5, - 0x7a, 0x3d, 0xbb, 0x68, 0xaf, 0x67, 0xef, 0xd0, 0xeb, 0xcf, 0x61, 0xad, 0xcd, 0xb9, 0xf0, 0x7c, - 0x46, 0x15, 0x17, 0xfb, 0x9c, 0x2b, 0xa9, 0x04, 0x0d, 0x62, 0x16, 0x35, 0xa0, 0x78, 0x8e, 0x42, - 0xea, 0x3b, 0xd6, 0x65, 0xc9, 0x3a, 0xb1, 0x68, 0xbf, 0x83, 0xf5, 0xf9, 0x81, 0x51, 0x53, 0xfd, - 0x87, 0x5b, 0xfa, 0xcd, 0x82, 0xd5, 0x98, 0x7c, 0x13, 0xb7, 0x38, 0xa5, 0x37, 0xb0, 0x4c, 0x3d, - 0xef, 0x64, 0x38, 0x36, 0xc4, 0xf8, 0x4f, 0xd2, 0x8f, 0x82, 0xe7, 0xcd, 0xc4, 0x3a, 0x35, 0x9a, - 0xd4, 0x4a, 0xd2, 0x01, 0x22, 0x70, 0xc8, 0xcf, 0x31, 0x05, 0x97, 0x31, 0x70, 0xff, 0x4f, 0x57, - 0xce, 0xb8, 0xcd, 0x22, 0xde, 0x17, 0x53, 0x06, 0x69, 0xff, 0x60, 0xc1, 0xc3, 0x79, 0xbb, 0x93, - 0x1a, 0x64, 0x7c, 0x2f, 0xa2, 0x57, 0xc6, 0xf7, 0xf4, 0xd0, 0x4a, 0xb4, 0x5f, 0x75, 0xdc, 0x5f, - 0xd3, 0xad, 0x91, 0x9d, 0x69, 0x8d, 0x3b, 0xb1, 0xc7, 0x6e, 0xc3, 0xa3, 0x1b, 0xd2, 0x9e, 0x49, - 0xa5, 0x01, 0x45, 0x97, 0x4a, 0x97, 0x7a, 0x21, 0xb9, 0x4b, 0x4e, 0x2c, 0xda, 0x6f, 0xe1, 0xf1, - 0x24, 0x7c, 0x86, 0x1f, 0x77, 0xe9, 0x9d, 0x1b, 0x8e, 0x69, 0x7f, 0x0f, 0x6b, 0x73, 0x91, 0x23, - 0x02, 0xdd, 0x05, 0x3a, 0x39, 0x6f, 0x33, 0x8b, 0xcc, 0x5b, 0xfb, 0x33, 0x58, 0x99, 0x6c, 0xde, - 0x1e, 0x70, 0x89, 0x0b, 0x1c, 0xc9, 0x3e, 0x86, 0x47, 0x33, 0xd1, 0x0b, 0xa4, 0xad, 0xdf, 0xf4, - 0x91, 0xeb, 0xa2, 0x94, 0x71, 0xb5, 0x23, 0xd1, 0xfe, 0x25, 0x03, 0xf9, 0xf0, 0x49, 0x59, 0x87, - 0xf2, 0x6b, 0xb9, 0x3f, 0xe0, 0xee, 0x19, 0x86, 0x17, 0x55, 0x72, 0x26, 0x0a, 0x8d, 0x60, 0x96, - 0x93, 0xa9, 0x10, 0x89, 0xe4, 0x73, 0xa8, 0x85, 0xcb, 0xf8, 0xe8, 0xd1, 0x87, 0xc8, 0x4d, 0x53, - 0x66, 0xca, 0x9b, 0xbc, 0x81, 0x87, 0x5f, 0x21, 0x7a, 0x07, 0x82, 0x07, 0x01, 0x7a, 0x13, 0x94, - 0xdc, 0xad, 0x28, 0x73, 0x63, 0xc8, 0x4b, 0x20, 0x5a, 0xbf, 0xe7, 0x79, 0x49, 0xa4, 0xfc, 0xad, - 0x48, 0x73, 0x22, 0xec, 0x3f, 0x2c, 0x58, 0x9e, 0xba, 0xca, 0x85, 0x66, 0xe8, 0x4b, 0xa8, 0xbb, - 0xf1, 0x23, 0x7c, 0x12, 0x52, 0x20, 0xfa, 0x90, 0xb8, 0xf5, 0xa5, 0x5e, 0x76, 0x93, 0xf2, 0x48, - 0x7f, 0xa4, 0x25, 0x9e, 0xf9, 0xca, 0x2e, 0x49, 0xcf, 0xbb, 0x45, 0x5f, 0x77, 0xdb, 0x83, 0x6a, - 0xf2, 0x41, 0x26, 0x04, 0x72, 0xca, 0x1f, 0x62, 0x44, 0x19, 0xb3, 0xd6, 0x3a, 0xc6, 0xbd, 0x78, - 0xe4, 0x98, 0xb5, 0xd6, 0xb9, 0x5a, 0x97, 0x0d, 0x75, 0x7a, 0xad, 0x09, 0x31, 0x44, 0x29, 0x69, - 0x0f, 0xcd, 0x56, 0x65, 0x27, 0x16, 0xb7, 0x36, 0xa0, 0x10, 0x8d, 0xdc, 0x32, 0xe4, 0x8f, 0x85, - 0xaf, 0xb0, 0xbe, 0x44, 0x4a, 0x90, 0x3b, 0xa4, 0x52, 0xd6, 0xad, 0xad, 0x16, 0xd4, 0xd2, 0x93, - 0x9a, 0x00, 0x14, 0xda, 0x02, 0xa9, 0xf1, 0x03, 0x28, 0x84, 0x4f, 0x48, 0xdd, 0xda, 0x62, 0x50, - 0x4b, 0x17, 0x88, 0x54, 0xa0, 0xf8, 0x0d, 0x3b, 0x63, 0xfc, 0x82, 0x85, 0xae, 0x7b, 0x5d, 0x89, - 0x4c, 0xd5, 0x2d, 0x72, 0x0f, 0xca, 0x87, 0x02, 0x03, 0x2a, 0x7c, 0xd6, 0xab, 0x67, 0x48, 0x15, - 0x4a, 0xa1, 0x88, 0x5e, 0x3d, 0xab, 0xa3, 0x8e, 0xb9, 0x38, 0xd3, 0xa6, 0x9c, 0x36, 0x75, 0x14, - 0x0f, 0x02, 0x2d, 0xe5, 0xb5, 0xc9, 0x48, 0xe8, 0xd5, 0x0b, 0xfb, 0xed, 0xdf, 0xaf, 0x9a, 0xd6, - 0xfb, 0xab, 0xa6, 0xf5, 0xf7, 0x55, 0xd3, 0xfa, 0xf9, 0xba, 0xb9, 0xf4, 0xfe, 0xba, 0xb9, 0xf4, - 0xe7, 0x75, 0x73, 0xe9, 0xdb, 0x0f, 0x7a, 0xbe, 0xea, 0x8f, 0xba, 0xdb, 0x2e, 0x1f, 0xee, 0x9c, - 0x0e, 0xf8, 0x45, 0x17, 0xfb, 0x34, 0x08, 0x2e, 0x77, 0x94, 0xdf, 0xa3, 0x0a, 0x77, 0x12, 0xb7, - 0xd2, 0x2d, 0x98, 0xff, 0x8d, 0x67, 0xff, 0x04, 0x00, 0x00, 0xff, 0xff, 0x1c, 0x36, 0x63, 0x6a, - 0x8e, 0x0c, 0x00, 0x00, + // 1171 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xa4, 0x57, 0x4b, 0x6f, 0xdb, 0xc6, + 0x13, 0x37, 0x45, 0x59, 0x8f, 0xb1, 0x22, 0x33, 0x9b, 0x20, 0x51, 0xe2, 0x44, 0x70, 0xf8, 0xcf, + 0x41, 0x7f, 0x07, 0xb0, 0x51, 0xa7, 0x45, 0x5a, 0xa0, 0x28, 0x60, 0x2b, 0x09, 0xf2, 0x40, 0xda, + 0x80, 0x4a, 0xe1, 0xa4, 0x17, 0x63, 0x45, 0x4e, 0x24, 0x42, 0xd2, 0x2e, 0xbb, 0xbb, 0x8a, 0x11, + 0xf4, 0xde, 0x1e, 0x5b, 0xa0, 0x40, 0x3f, 0x48, 0xcf, 0xfd, 0x00, 0x3d, 0xe6, 0xd0, 0x43, 0x4f, + 0x45, 0x91, 0x7c, 0x91, 0x62, 0x97, 0xa4, 0x44, 0x52, 0xb2, 0x6b, 0xa1, 0xb7, 0x9d, 0xd7, 0x6f, + 0x66, 0x67, 0xe7, 0x41, 0xc2, 0xd6, 0x10, 0xa9, 0x50, 0x7d, 0xa4, 0x2a, 0xea, 0xef, 0xcd, 0xce, + 0xbb, 0x91, 0xe0, 0x8a, 0x93, 0x8d, 0x8c, 0xd0, 0x7d, 0x05, 0xf5, 0x17, 0xb4, 0x3f, 0xc6, 0x5e, + 0x44, 0x19, 0x69, 0x41, 0xd5, 0x10, 0x8f, 0xef, 0xb7, 0xac, 0x6d, 0xab, 0x53, 0xf6, 0x52, 0x92, + 0x5c, 0x87, 0x5a, 0x4f, 0x51, 0xa1, 0x9e, 0xe2, 0xdb, 0x56, 0x69, 0xdb, 0xea, 0x34, 0xbc, 0x19, + 0x4d, 0xae, 0x40, 0xe5, 0x01, 0x0b, 0xb4, 0xc4, 0x36, 0x92, 0x84, 0x72, 0x7f, 0x2d, 0x81, 0xf3, + 0x48, 0xbb, 0x3a, 0x44, 0xaa, 0x3c, 0xfc, 0x76, 0x8a, 0x52, 0x11, 0x17, 0x1a, 0xfe, 0x90, 0xb2, + 0x01, 0xbe, 0x46, 0x0c, 0x12, 0x3f, 0x75, 0x2f, 0xc7, 0x23, 0x1f, 0x43, 0xfd, 0x84, 0x2a, 0x14, + 0x13, 0x2a, 0x46, 0xc6, 0xdb, 0xc6, 0xfe, 0x95, 0xdd, 0x4c, 0xd0, 0xbb, 0x47, 0xa9, 0xd4, 0x9b, + 0x2b, 0x92, 0x4f, 0xa1, 0x26, 0x15, 0x55, 0x53, 0x89, 0xb2, 0x65, 0x6f, 0xdb, 0x9d, 0x8d, 0xfd, + 0x1b, 0x39, 0xa3, 0xd9, 0x35, 0x7b, 0x46, 0xcb, 0x9b, 0x69, 0x93, 0x0e, 0x6c, 0xfa, 0x7c, 0x12, + 0xe1, 0x18, 0x15, 0xc6, 0xc2, 0x56, 0x79, 0xdb, 0xea, 0xd4, 0xbc, 0x22, 0x9b, 0xdc, 0x85, 0xea, + 0x09, 0x15, 0x2c, 0x64, 0x83, 0xd6, 0xba, 0x89, 0xeb, 0x5a, 0xce, 0x85, 0x37, 0x65, 0x5a, 0xf6, + 0x40, 0x08, 0x2e, 0xbc, 0x54, 0x93, 0xdc, 0x01, 0x1b, 0x85, 0x68, 0x55, 0xfe, 0xcd, 0x40, 0x6b, + 0xb9, 0x5f, 0x41, 0x7d, 0x76, 0xbb, 0x38, 0x59, 0xe8, 0x8f, 0x22, 0x1e, 0x32, 0xf5, 0x42, 0x26, + 0x8f, 0x92, 0xe3, 0x91, 0x36, 0x80, 0x40, 0xc9, 0xc7, 0x6f, 0x30, 0x78, 0x21, 0x4d, 0xb6, 0xca, + 0x5e, 0x86, 0xe3, 0xfe, 0x60, 0x81, 0x73, 0x3f, 0x94, 0x11, 0x55, 0xfe, 0x10, 0xc5, 0x81, 0xaf, + 0x42, 0xce, 0xc8, 0x0e, 0x94, 0x65, 0x44, 0x99, 0x01, 0x2c, 0x26, 0x77, 0x96, 0x27, 0xcf, 0xe8, + 0x90, 0x3b, 0x50, 0xa1, 0xc6, 0xca, 0x80, 0x37, 0xf7, 0x2f, 0xe5, 0xb4, 0x63, 0x40, 0x2f, 0x51, + 0xd1, 0x75, 0xd2, 0xe5, 0x93, 0x49, 0xa8, 0xa3, 0xb5, 0x4d, 0x2c, 0x33, 0xda, 0x7d, 0x06, 0xf6, + 0x41, 0xf7, 0xe9, 0x4a, 0xbe, 0xb3, 0x70, 0xa5, 0x02, 0xdc, 0x2f, 0x16, 0x5c, 0xcc, 0x94, 0x97, + 0x8c, 0x38, 0x93, 0x48, 0xee, 0x41, 0x35, 0x0e, 0x45, 0x67, 0x4b, 0x17, 0xc1, 0xcd, 0x9c, 0x83, + 0x62, 0x26, 0xbc, 0x54, 0x7b, 0xa1, 0x30, 0x4b, 0x4b, 0x0a, 0xf3, 0x36, 0x94, 0xa9, 0x3f, 0x4a, + 0xcb, 0xcb, 0xc9, 0x27, 0xa2, 0xfb, 0xd4, 0x33, 0x52, 0xf7, 0x65, 0x36, 0xe1, 0x5d, 0xce, 0x5e, + 0x87, 0x83, 0x95, 0x2e, 0xdd, 0x82, 0xaa, 0xd4, 0xbd, 0x35, 0xbb, 0x73, 0x4a, 0xba, 0x7f, 0x59, + 0x70, 0xad, 0xe7, 0x0f, 0x31, 0x98, 0x8e, 0x71, 0xee, 0x62, 0x95, 0xd6, 0xfa, 0x04, 0x2a, 0xbe, + 0x89, 0x28, 0xe9, 0xab, 0xd3, 0xb2, 0x13, 0x87, 0xed, 0x25, 0xca, 0xa4, 0x0b, 0x4d, 0x99, 0xf8, + 0x8d, 0xf3, 0x66, 0x1e, 0xb7, 0xb9, 0xbf, 0x95, 0x33, 0xef, 0xe5, 0x54, 0xbc, 0x82, 0x09, 0xb9, + 0x05, 0x8d, 0x50, 0x1e, 0x4b, 0xf4, 0x39, 0x0b, 0xa8, 0x78, 0x9b, 0xf4, 0xd8, 0x46, 0x28, 0x7b, + 0x29, 0xcb, 0x7d, 0x0e, 0x97, 0x9e, 0xd1, 0x90, 0x29, 0x1a, 0x32, 0x14, 0x8f, 0x52, 0x68, 0xf2, + 0x59, 0xa6, 0xb5, 0x97, 0xbd, 0xea, 0xdc, 0xa6, 0xd8, 0xdb, 0xee, 0x8f, 0x25, 0x70, 0x8a, 0xe2, + 0x73, 0x65, 0xea, 0x26, 0x80, 0x3e, 0x1d, 0x6b, 0x24, 0x4c, 0xaa, 0xa1, 0xae, 0x39, 0x1a, 0x03, + 0xc9, 0x47, 0xb0, 0x1e, 0x4b, 0x96, 0x25, 0xa2, 0xcb, 0x27, 0x11, 0x67, 0xc8, 0x94, 0xd1, 0xf5, + 0x62, 0x4d, 0xf2, 0x3f, 0xb8, 0x30, 0xef, 0xdc, 0x63, 0x15, 0x0f, 0x99, 0x62, 0x3b, 0xe7, 0x26, + 0x8c, 0xbd, 0xea, 0x84, 0xb1, 0xcf, 0x31, 0x61, 0xee, 0xc1, 0x56, 0x97, 0x73, 0x11, 0x84, 0x8c, + 0x2a, 0x2e, 0x0e, 0x39, 0x57, 0x52, 0x09, 0x1a, 0xa5, 0x55, 0xd4, 0x82, 0xea, 0x1b, 0x14, 0x52, + 0xbf, 0xb1, 0x4e, 0x8b, 0xed, 0xa5, 0xa4, 0xfb, 0x0a, 0x6e, 0x2c, 0x37, 0x4c, 0x5a, 0xef, 0x3f, + 0xbc, 0xd2, 0x6f, 0x16, 0x5c, 0x4b, 0x8b, 0x6f, 0xae, 0x96, 0x86, 0xf4, 0x04, 0x36, 0x69, 0x10, + 0x1c, 0x4f, 0x66, 0x82, 0x14, 0xff, 0x56, 0xbe, 0x03, 0x83, 0x60, 0xc1, 0xd6, 0x6b, 0xd2, 0x2c, + 0x57, 0x92, 0x1e, 0x10, 0x81, 0x13, 0xfe, 0x06, 0x73, 0x70, 0x25, 0x03, 0x77, 0x3b, 0x9f, 0x39, + 0xa3, 0xb6, 0x88, 0x78, 0x51, 0x14, 0x04, 0xd2, 0xfd, 0xde, 0x82, 0xcb, 0xcb, 0xbc, 0x93, 0x26, + 0x94, 0xc2, 0x20, 0x29, 0xaf, 0x52, 0x18, 0xe8, 0x55, 0x99, 0x69, 0xbf, 0xc6, 0xac, 0xbf, 0x8a, + 0xad, 0x61, 0x2f, 0xb4, 0xc6, 0xb9, 0xaa, 0xc7, 0xed, 0xc2, 0xd5, 0x53, 0xc2, 0x5e, 0x08, 0xa5, + 0x05, 0x55, 0x9f, 0x4a, 0x9f, 0x06, 0x71, 0x71, 0xd7, 0xbc, 0x94, 0x74, 0x5f, 0xc2, 0xf5, 0xb9, + 0xf9, 0x42, 0x7d, 0x9c, 0xa7, 0x77, 0x4e, 0xb9, 0xa6, 0xfb, 0x1d, 0x6c, 0x2d, 0x45, 0x4e, 0x0a, + 0xe8, 0x3c, 0xd0, 0xd9, 0x2d, 0x5f, 0x5a, 0x65, 0xcb, 0xbb, 0x9f, 0xc3, 0x95, 0xb9, 0xf3, 0xee, + 0x98, 0x4b, 0x5c, 0xe1, 0x4a, 0xee, 0x11, 0x5c, 0x5d, 0xb0, 0x5e, 0x21, 0x6c, 0x3d, 0xd3, 0xa7, + 0xbe, 0x8f, 0x52, 0xa6, 0xd9, 0x4e, 0x48, 0xf7, 0xe7, 0x12, 0xac, 0xc7, 0x23, 0xe5, 0x06, 0xd4, + 0x1f, 0xcb, 0xc3, 0x31, 0xf7, 0x47, 0x18, 0x3f, 0x54, 0xcd, 0x9b, 0x33, 0x34, 0x82, 0x39, 0xce, + 0xb7, 0x42, 0x42, 0x92, 0x2f, 0xa0, 0x19, 0x1f, 0xd3, 0xab, 0x27, 0xfb, 0xe9, 0xb4, 0x2d, 0x53, + 0xd0, 0x26, 0x4f, 0xe0, 0xf2, 0x97, 0x88, 0xc1, 0x7d, 0xc1, 0xa3, 0x08, 0x83, 0x39, 0x4a, 0xf9, + 0x4c, 0x94, 0xa5, 0x36, 0xe4, 0x21, 0x10, 0xcd, 0x3f, 0x08, 0x82, 0x2c, 0xd2, 0xfa, 0x99, 0x48, + 0x4b, 0x2c, 0xdc, 0x3f, 0x2c, 0xd8, 0x2c, 0x3c, 0xe5, 0x4a, 0x3b, 0xf4, 0x21, 0x38, 0x7e, 0x3a, + 0x84, 0x8f, 0xe3, 0x12, 0x48, 0x3e, 0x5f, 0xce, 0x9c, 0xd4, 0x9b, 0x7e, 0x96, 0x9e, 0xea, 0x4f, + 0xc3, 0xcc, 0x98, 0xdf, 0xd8, 0x27, 0xf9, 0x7d, 0xb7, 0xea, 0x74, 0x77, 0x03, 0x68, 0x64, 0x07, + 0x32, 0x21, 0x50, 0x56, 0xe1, 0x04, 0x93, 0x92, 0x31, 0x67, 0xcd, 0x63, 0x3c, 0x48, 0x57, 0x8e, + 0x39, 0x6b, 0x9e, 0xaf, 0x79, 0x76, 0xcc, 0xd3, 0x67, 0x5d, 0x10, 0x13, 0x94, 0x92, 0x0e, 0xd0, + 0xb8, 0xaa, 0x7b, 0x29, 0xb9, 0x73, 0x13, 0x2a, 0xc9, 0xca, 0xad, 0xc3, 0xfa, 0x91, 0x08, 0x15, + 0x3a, 0x6b, 0xa4, 0x06, 0xe5, 0xe7, 0x54, 0x4a, 0xc7, 0xda, 0xe9, 0x40, 0x33, 0xbf, 0xa9, 0x09, + 0x40, 0xa5, 0x2b, 0x90, 0x1a, 0x3d, 0x80, 0x4a, 0x3c, 0x42, 0x1c, 0x6b, 0x87, 0x41, 0x33, 0x9f, + 0x20, 0xb2, 0x01, 0xd5, 0xaf, 0xd9, 0x88, 0xf1, 0x13, 0x16, 0xab, 0x1e, 0xf4, 0x25, 0x32, 0xe5, + 0x58, 0xe4, 0x02, 0xd4, 0x9f, 0x0b, 0x8c, 0xa8, 0x08, 0xd9, 0xc0, 0x29, 0x91, 0x06, 0xd4, 0x62, + 0x12, 0x03, 0xc7, 0xd6, 0x56, 0x47, 0x5c, 0x8c, 0xb4, 0xa8, 0xac, 0x45, 0x3d, 0xc5, 0xa3, 0x48, + 0x53, 0xeb, 0x5a, 0x64, 0x28, 0x0c, 0x9c, 0xca, 0x61, 0xf7, 0xf7, 0xf7, 0x6d, 0xeb, 0xdd, 0xfb, + 0xb6, 0xf5, 0xf7, 0xfb, 0xb6, 0xf5, 0xd3, 0x87, 0xf6, 0xda, 0xbb, 0x0f, 0xed, 0xb5, 0x3f, 0x3f, + 0xb4, 0xd7, 0xbe, 0xf9, 0xff, 0x20, 0x54, 0xc3, 0x69, 0x7f, 0xd7, 0xe7, 0x93, 0xbd, 0xd7, 0x63, + 0x7e, 0xd2, 0xc7, 0x21, 0x8d, 0xa2, 0xb7, 0x7b, 0x2a, 0x1c, 0x50, 0x85, 0x7b, 0x99, 0x57, 0xe9, + 0x57, 0xcc, 0x5f, 0xce, 0xdd, 0x7f, 0x02, 0x00, 0x00, 0xff, 0xff, 0x41, 0x8a, 0xa0, 0x05, 0x04, + 0x0d, 0x00, 0x00, } func (m *TableSpan) Marshal() (dAtA []byte, err error) { @@ -1655,6 +1718,46 @@ func (m *DispatcherAction) MarshalToSizedBuffer(dAtA []byte) (int, error) { return len(dAtA) - i, nil } +func (m *ACK) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *ACK) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *ACK) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + _ = i + var l int + _ = l + if m.CommitTs != 0 { + i = encodeVarintHeartbeat(dAtA, i, uint64(m.CommitTs)) + i-- + dAtA[i] = 0x10 + } + if m.Span != nil { + { + size, err := m.Span.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintHeartbeat(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + func (m *HeartBeatResponse) Marshal() (dAtA []byte, err error) { size := m.Size() dAtA = make([]byte, size) @@ -1675,6 +1778,20 @@ func (m *HeartBeatResponse) MarshalToSizedBuffer(dAtA []byte) (int, error) { _ = i var l int _ = l + if len(m.Acks) > 0 { + for iNdEx := len(m.Acks) - 1; iNdEx >= 0; iNdEx-- { + { + size, err := m.Acks[iNdEx].MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintHeartbeat(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x1a + } + } if len(m.ChangefeedID) > 0 { i -= len(m.ChangefeedID) copy(dAtA[i:], m.ChangefeedID) @@ -2554,6 +2671,22 @@ func (m *DispatcherAction) Size() (n int) { return n } +func (m *ACK) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Span != nil { + l = m.Span.Size() + n += 1 + l + sovHeartbeat(uint64(l)) + } + if m.CommitTs != 0 { + n += 1 + sovHeartbeat(uint64(m.CommitTs)) + } + return n +} + func (m *HeartBeatResponse) Size() (n int) { if m == nil { return 0 @@ -2570,6 +2703,12 @@ func (m *HeartBeatResponse) Size() (n int) { if l > 0 { n += 1 + l + sovHeartbeat(uint64(l)) } + if len(m.Acks) > 0 { + for _, e := range m.Acks { + l = e.Size() + n += 1 + l + sovHeartbeat(uint64(l)) + } + } return n } @@ -3494,6 +3633,111 @@ func (m *DispatcherAction) Unmarshal(dAtA []byte) error { } return nil } +func (m *ACK) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowHeartbeat + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: ACK: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ACK: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Span", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowHeartbeat + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthHeartbeat + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthHeartbeat + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Span == nil { + m.Span = &TableSpan{} + } + if err := m.Span.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field CommitTs", wireType) + } + m.CommitTs = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowHeartbeat + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.CommitTs |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + default: + iNdEx = preIndex + skippy, err := skipHeartbeat(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return ErrInvalidLengthHeartbeat + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} func (m *HeartBeatResponse) Unmarshal(dAtA []byte) error { l := len(dAtA) iNdEx := 0 @@ -3589,6 +3833,40 @@ func (m *HeartBeatResponse) Unmarshal(dAtA []byte) error { } m.ChangefeedID = string(dAtA[iNdEx:postIndex]) iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Acks", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowHeartbeat + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthHeartbeat + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthHeartbeat + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Acks = append(m.Acks, &ACK{}) + if err := m.Acks[len(m.Acks)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex default: iNdEx = preIndex skippy, err := skipHeartbeat(dAtA[iNdEx:]) diff --git a/heartbeatpb/heartbeat.proto b/heartbeatpb/heartbeat.proto index 81334898b..7a5d162a9 100644 --- a/heartbeatpb/heartbeat.proto +++ b/heartbeatpb/heartbeat.proto @@ -9,15 +9,6 @@ message TableSpan { bytes EndKey = 3; } -// message TableSpanProgress { -// TableSpan span = 1; -// bool IsBlocked = 2; -// uint64 BlockTs = 3; -// repeated TableSpan BlockTableSpan = 4; -// uint64 CheckpointTs = 5; -// string dispatcherID = 6; -// } - message HeartBeatRequest { string changefeedID = 1; Watermark watermark = 2; @@ -43,9 +34,15 @@ message DispatcherAction { uint64 CommitTs = 3; // DDLCommitTs } +message ACK { + TableSpan span = 1; + uint64 CommitTs = 2; // DDLCommitTs +} + message HeartBeatResponse { repeated DispatcherAction actions = 1; string changefeedID = 2; + repeated ACK acks = 3; // only return when received State info in heartbeat request } enum ScheduleAction { diff --git a/pkg/common/txn_event.go b/pkg/common/txn_event.go index 1b2da160e..d24a94da0 100644 --- a/pkg/common/txn_event.go +++ b/pkg/common/txn_event.go @@ -45,6 +45,10 @@ type TxnEvent struct { PostTxnFlushed func() `msg:"-"` } +func (w *TxnEvent) GetDDLEvent() *DDLEvent { + return w.DDLEvent +} + func (w *TxnEvent) String() string { return fmt.Sprintf("TxnEvent{StartTs: %d, CommitTs: %d, Rows: %d, DDLEvent: %v}", w.StartTs, w.CommitTs, len(w.Rows), w.DDLEvent) } @@ -82,13 +86,15 @@ func (e *TxnEvent) IsSingleTableDDL() bool { } func (e *TxnEvent) GetBlockedTableSpan() []*heartbeatpb.TableSpan { - + return nil } func (e *TxnEvent) GetNeedDroppedTableSpan() []*heartbeatpb.TableSpan { + return nil } func (e *TxnEvent) GetNeedAddedTableSpan() []*heartbeatpb.TableSpan { + return nil } func (e *TxnEvent) IsSyncPointEvent() bool { From 45348a278eeaba17bc78cb2a19e2c165e8e3a2eb Mon Sep 17 00:00:00 2001 From: hongyunyan <649330952@qq.com> Date: Fri, 16 Aug 2024 11:36:34 +0800 Subject: [PATCH 04/12] temp --- downstreamadapter/dispatcher/dispatcher.go | 60 ++++- .../dispatcher/event_dispatcher_task.go | 254 ------------------ .../dispatcher/table_event_dispatcher.go | 55 ---- .../table_trigger_event_dispatcher.go | 8 + .../event_dispatcher_manager.go | 93 ++++--- .../dispatchermanager/heartbeat_collector.go | 4 +- .../dispatcher_manager_manager.go | 4 +- .../eventcollector/event_collector.go | 11 +- heartbeatpb/heartbeat.pb.go | 197 ++++++++------ heartbeatpb/heartbeat.proto | 1 + 10 files changed, 252 insertions(+), 435 deletions(-) delete mode 100644 downstreamadapter/dispatcher/event_dispatcher_task.go diff --git a/downstreamadapter/dispatcher/dispatcher.go b/downstreamadapter/dispatcher/dispatcher.go index fd7cb5b8c..c5e2bf66b 100644 --- a/downstreamadapter/dispatcher/dispatcher.go +++ b/downstreamadapter/dispatcher/dispatcher.go @@ -78,6 +78,7 @@ type Dispatcher interface { GetDDLPendingEvent() *common.TxnEvent SetDDLPendingEvent(event *common.TxnEvent) GetDDLFinishCh() chan struct{} + Remove() } type DispatcherType uint64 @@ -90,14 +91,9 @@ const ( /* HeartBeatInfo is used to collect the message for HeartBeatRequest for each dispatcher. Mainly about the progress of each dispatcher: -1. whether the dispatcher is blocked ? If blocked, the info about blocked should be collected. -2. The checkpointTs of the dispatcher, shows that all the events whose ts <= checkpointTs are flushed to downstream successfully. +1. The checkpointTs of the dispatcher, shows that all the events whose ts <= checkpointTs are flushed to downstream successfully. */ type HeartBeatInfo struct { - // IsBlocked bool - // BlockTs uint64 - // BlockTableSpan []*common.TableSpan - // TableSpan *common.TableSpan heartbeatpb.Watermark Id string TableSpan *common.TableSpan @@ -221,3 +217,55 @@ func CollectDispatcherHeartBeatInfo(d Dispatcher, h *HeartBeatInfo) { h.ComponentStatus = d.GetComponentStatus() h.TableSpan = d.GetTableSpan() } + +type SyncPointInfo struct { + EnableSyncPoint bool + SyncPointInterval time.Duration + NextSyncPointTs uint64 +} + +type ComponentStateWithMutex struct { + mutex sync.Mutex + componentStatus heartbeatpb.ComponentState +} + +func newComponentStateWithMutex(status heartbeatpb.ComponentState) *ComponentStateWithMutex { + return &ComponentStateWithMutex{ + componentStatus: status, + } +} + +func (s *ComponentStateWithMutex) Set(status heartbeatpb.ComponentState) { + s.mutex.Lock() + defer s.mutex.Unlock() + s.componentStatus = status +} + +func (s *ComponentStateWithMutex) Get() heartbeatpb.ComponentState { + s.mutex.Lock() + defer s.mutex.Unlock() + return s.componentStatus +} + +type TsWithMutex struct { + mutex sync.Mutex + ts uint64 +} + +func newTsWithMutex(ts uint64) *TsWithMutex { + return &TsWithMutex{ + ts: ts, + } +} + +func (r *TsWithMutex) Set(ts uint64) { + r.mutex.Lock() + defer r.mutex.Unlock() + r.ts = ts +} + +func (r *TsWithMutex) Get() uint64 { + r.mutex.Lock() + defer r.mutex.Unlock() + return r.ts +} diff --git a/downstreamadapter/dispatcher/event_dispatcher_task.go b/downstreamadapter/dispatcher/event_dispatcher_task.go deleted file mode 100644 index 7c85ca3f4..000000000 --- a/downstreamadapter/dispatcher/event_dispatcher_task.go +++ /dev/null @@ -1,254 +0,0 @@ -// Copyright 2024 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 dispatcher - -import ( - "github.com/flowbehappy/tigate/utils/threadpool" -) - -/* -One dispatcher corresponds to an event dispatcher task. -The task will create when the dispatcher creates and finish when the dispatcher is closed. - -EventDispatcherTask is responsible for determining whether the current event can be pushed down to the sink, -and pushes the event down to the sink if it is eligible. -*/ -type EventDispatcherTask struct { - dispatcher Dispatcher - //infos []*HeartBeatResponseMessage - taskHandle *threadpool.TaskHandle -} - -/* -func NewEventDispatcherTask(dispatcher Dispatcher) *EventDispatcherTask { - task := &EventDispatcherTask{ - dispatcher: dispatcher, - //infos: make([]*HeartBeatResponseMessage, 0), - } - task.taskHandle = threadpool.GetTaskSchedulerInstance().EventDispatcherTaskScheduler.Submit(task, threadpool.CPUTask, time.Time{}) - return task -} - -// func (t *EventDispatcherTask) GetStatus() threadpool.TaskStatus { -// t.mutex.Lock() -// defer t.mutex.Unlock() -// return t.taskStatus -// } - -// func (t *EventDispatcherTask) SetStatus(taskStatus threadpool.TaskStatus) { -// t.mutex.Lock() -// defer t.mutex.Unlock() -// // if t.taskStatus != threadpool.Canceled { -// // t.taskStatus = taskStatus -// // } -// } - -/* -func (t *EventDispatcherTask) updateState(state *State, heartBeatResponseMessages []*HeartBeatResponseMessage) { - for _, heartBeatResponseMessage := range heartBeatResponseMessages { - otherTableProgress := heartBeatResponseMessage.OtherTableProgress - spanToProgressMap := make(utils.NewBtreeMap[*common.TableSpan,*TableSpanProgress]) - for _, progress := range otherTableProgress { - span := progress.Span - spanToProgressMap[span] = progress - } - - newBlockTableSpan := make([]*common.TableSpan, 0) - for _, span := range state.blockTableSpan { - remove := false - if progress, ok := spanToProgressMap[span]; ok { - if progress.IsBlocked { - if progress.BlockTs >= state.blockTs { - remove = true - } - } else { - if progress.CheckpointTs >= state.blockTs { - remove = true - } - } - } - if !remove { - newBlockTableSpan = append(newBlockTableSpan, span) - } - } - state.blockTableSpan = newBlockTableSpan - - action := heartBeatResponseMessage.Action - // 这边理论上要保证 action 复写没有问题 - if state.action != action { - state.action = action - } - } - -} - -// Execution time is 10ms at most once -func (t *EventDispatcherTask) Execute() (threadpool.TaskStatus, time.Time) { - timer := time.NewTimer(10 * time.Millisecond) - - // 1. 先检查是否在 blocked 状态 - state := t.dispatcher.GetState() - tableSpan := t.dispatcher.GetTableSpan() - //dispatcherType := t.dispatcher.GetDispatcherType() - sink := t.dispatcher.GetSink() - - - if state.isBlocked { - // 拿着 infos 更新 - if len(t.infos) > 0 { - t.updateState(state, t.infos) - } - t.infos = t.infos[:0] //上锁 - - if !state.sinkAvailable { // 没达到 ture 的时候都要做检查 - if sink.IsEmpty(tableSpan) { - state.sinkAvailable = true - } - } - - if state.blockTableSpan != nil || (state.pengdingEvent != nil && state.action == None) { - return threadpool.Waiting - } - - if state.sinkAvailable == false { // 只有这个不满足的话就在这轮不用去 wait - return threadpool.Running - } - } - - - // 先检查 pendingEvent 有没有,有的话就先执行这个,通过 action 来确定 ddl / syncPoint 的执行模式 - - if state.pengdingEvent != nil { - - if state.pengdingEvent.IsDDLEvent() || state.pengdingEvent.IsSyncPointEvent() { - if state.action == Pass { - // 扔掉这条,开始写后续的 event - } else if state.action == Write { - // 下推 - sink.AddDDLAndSyncPointEvent(tableSpan, state.pengdingEvent) - state.clear() - - if dispatcherType == TableTriggerEventDispatcherType { - // 如果是 table trigger, 那就 进入后续处理,尝试拿取下一条 event 开始处理 - } else if dispatcherType == TableEventDispatcherType { - // state 改为 block ,等到下一轮发现 sink available 以后重新开始拿数据往下同步 - state = &State{ - isBlocked: true, - } - return threadpool.Running - } - } - } else { - // DML - // 直接下推 - sink.AddDMLEvent(tableSpan, state.pengdingEvent) - state.clear() - } - sink.AddDMLEvent(tableSpan, state.pengdingEvent) - state.clear() - } - - 还能继续写 - 1. pending 的是 ddl /syncpoint,但是 pass) - 2. pending 的是 dml,继续写 - 3. 没有 pending, 本身就在等前面 ddl / syncpoint 写完而已 - - for { - select { - case event := <-t.dispatcher.GetEventChan(): - if event.IsDMLEvent() { - sink.AddDMLEvent(tableSpan, event) - } - - else if event.IsDDLEvent() { - // DDL - if event.IsCrossTableDDL() { // cross 这个到时候还要加入 table trigger 会涉及到的,也算 cross. 也就是 table trigger 只能走到这个里面 - // TODO: 先不跑除了 create table 以外的 DDL,所以不会有 crossTable 出现 - // tableSpans := event.GetTableSpans() // 获得这个 ddl 中涉及到的其他 table 的 span,这边后面写的时候也要考虑一下 table trigger 这个特殊的啊 - // state = &State{ - // isBlocked: true, - // pengdingEvent: event, - // blockTableSpan: tableSpans, - // blockTs: event.CommitTs, - // } - return threadpool.Waiting - } else { - if sink.IsEmpty(tableSpan) { - sink.AddDDLAndSyncPointEvent(tableSpan, event) - state = &State{ - isBlocked: true, - } - } else { - state = &State{ - isBlocked: true, - pengdingEvent: event, - } - } - return threadpool.Running - } - } else { - // syncpoint event, TODO:要处理一下只有一张表的情况 - state = &State{ - isBlocked: true, - pengdingEvent: event, - // blockTableSpan: AllTables, // special span : TODO syncpointTs - blockTs: event.CommitTs, - } - return threadpool.Waiting - } - - case <-timer.C: - return threadpool.CPUTask, time.Time{} - default: - if !timer.Stop() { - <-timer.C - } - return threadpool.CPUTask, time.Time{} - } - } -} - - -func (t *EventDispatcherTask) Await() threadpool.TaskStatus { - - select { - case info := <-t.dispatcher.GetHeartBeatChan(): - // 把目前现有的全部拿出来 - t.infos = append(t.infos, info) - for { - select { - case info := <-t.dispatcher.GetHeartBeatChan(): - t.infos = append(t.infos, info) - default: - return threadpool.Running - } - } - default: - // 如果不是在等待其他 table / action 状态而是 sink 本身的话,就也切回去 - state := t.dispatcher.GetState() - if state.blockTableSpan == nil && (state.pengdingEvent == nil || state.action != None) && state.sinkAvailable == false { - return threadpool.Running - } - return threadpool.Waiting - } - - return threadpool.Done -} - - -// cancel 以后最多会再执行一次(也就是 cancel 的时候恰好在执行 Execute ) -func (t *EventDispatcherTask) Cancel() { - t.taskHandle.Cancel() -} -*/ diff --git a/downstreamadapter/dispatcher/table_event_dispatcher.go b/downstreamadapter/dispatcher/table_event_dispatcher.go index 8c5e3cfe5..e8a71d55c 100644 --- a/downstreamadapter/dispatcher/table_event_dispatcher.go +++ b/downstreamadapter/dispatcher/table_event_dispatcher.go @@ -16,7 +16,6 @@ package dispatcher import ( "context" "sync" - "time" "github.com/flowbehappy/tigate/downstreamadapter/sink" "github.com/flowbehappy/tigate/heartbeatpb" @@ -27,61 +26,7 @@ import ( "go.uber.org/zap" ) -type SyncPointInfo struct { - EnableSyncPoint bool - SyncPointInterval time.Duration - NextSyncPointTs uint64 -} - -type ComponentStateWithMutex struct { - mutex sync.Mutex - componentStatus heartbeatpb.ComponentState -} - -func newComponentStateWithMutex(status heartbeatpb.ComponentState) *ComponentStateWithMutex { - return &ComponentStateWithMutex{ - componentStatus: status, - } -} - -func (s *ComponentStateWithMutex) Set(status heartbeatpb.ComponentState) { - s.mutex.Lock() - defer s.mutex.Unlock() - s.componentStatus = status -} - -func (s *ComponentStateWithMutex) Get() heartbeatpb.ComponentState { - s.mutex.Lock() - defer s.mutex.Unlock() - return s.componentStatus -} - -type TsWithMutex struct { - mutex sync.Mutex - ts uint64 -} - -func newTsWithMutex(ts uint64) *TsWithMutex { - return &TsWithMutex{ - ts: ts, - } -} - -func (r *TsWithMutex) Set(ts uint64) { - r.mutex.Lock() - defer r.mutex.Unlock() - r.ts = ts -} - -func (r *TsWithMutex) Get() uint64 { - r.mutex.Lock() - defer r.mutex.Unlock() - return r.ts -} - /* -TableEventDispatcher implements the Dispatcher interface. - TableEventDispatcher is dispatcher the event of a normal tableSpan in a changefeed. It is responsible for getting the events about the tableSpan from the Logservice and sending them to the Sink in an appropriate order. diff --git a/downstreamadapter/dispatcher/table_trigger_event_dispatcher.go b/downstreamadapter/dispatcher/table_trigger_event_dispatcher.go index 22a55facd..7d8dc625f 100644 --- a/downstreamadapter/dispatcher/table_trigger_event_dispatcher.go +++ b/downstreamadapter/dispatcher/table_trigger_event_dispatcher.go @@ -258,3 +258,11 @@ func (d *TableTriggerEventDispatcher) SetDDLPendingEvent(event *common.TxnEvent) func (d *TableTriggerEventDispatcher) GetDDLFinishCh() chan struct{} { return d.ddlFinishCh } + +func (d *TableTriggerEventDispatcher) Remove() { + // TODO: 修改这个 dispatcher 的 status 为 removing + d.cancel() + d.sink.StopTableSpan(d.tableSpan) + log.Info("table event dispatcher component status changed to stopping", zap.String("table", d.tableSpan.String())) + d.componentStatus.Set(heartbeatpb.ComponentState_Stopping) +} diff --git a/downstreamadapter/dispatchermanager/event_dispatcher_manager.go b/downstreamadapter/dispatchermanager/event_dispatcher_manager.go index 8007687ff..506911164 100644 --- a/downstreamadapter/dispatchermanager/event_dispatcher_manager.go +++ b/downstreamadapter/dispatchermanager/event_dispatcher_manager.go @@ -82,7 +82,7 @@ type EventDispatcherManager struct { closed atomic.Bool } -func NewEventDispatcherManager(changefeedID model.ChangeFeedID, changefeedConfig *config.ChangefeedConfig, maintainerID messaging.ServerId) *EventDispatcherManager { +func NewEventDispatcherManager(changefeedID model.ChangeFeedID, changefeedConfig *config.ChangefeedConfig, maintainerID messaging.ServerId, createTableTriggerEventDispatcher bool) *EventDispatcherManager { ctx, cancel := context.WithCancel(context.Background()) eventDispatcherManager := &EventDispatcherManager{ dispatcherMap: newDispatcherMap(), @@ -110,7 +110,14 @@ func NewEventDispatcherManager(changefeedID model.ChangeFeedID, changefeedConfig appcontext.GetService[*HeartBeatCollector](appcontext.HeartbeatCollector).RegisterEventDispatcherManager(eventDispatcherManager) + if createTableTriggerEventDispatcher { + eventDispatcherManager.tableTriggerEventDispatcher = eventDispatcherManager.newTableTriggerEventDispatcher(eventDispatcherManager.config.StartTS) + } + + // TODO: 这些后续需要等有第一个 table 来的时候再初始化, 对于纯空的 event dispatcher manager 不要直接创建为好 + eventDispatcherManager.wg.Add(1) + // collect heartbeat info every 1s go func(ctx context.Context, e *EventDispatcherManager) { defer e.wg.Done() counter := 0 @@ -128,34 +135,18 @@ func NewEventDispatcherManager(changefeedID model.ChangeFeedID, changefeedConfig } }(ctx, eventDispatcherManager) - // TODO: 这个后续需要等有第一个 table 来的时候再初始化 - eventDispatcherManager.Init(eventDispatcherManager.config.StartTS) - - eventDispatcherManager.wg.Add(1) - go eventDispatcherManager.CollectHeartbeatInfoWhenStatesChanged(ctx) - - return eventDispatcherManager -} - -func (e *EventDispatcherManager) Init(startTs uint64) error { - cfg, db, err := writer.NewMysqlConfigAndDB(e.config.SinkURI) - if err != nil { - log.Error("create mysql sink failed", zap.Error(err)) - return err - } - - e.sink = sink.NewMysqlSink(e.changefeedID, 16, cfg, db) + eventDispatcherManager.InitSink() // get heartbeat response from HeartBeatResponseQueue, and send to each dispatcher - e.wg.Add(1) + eventDispatcherManager.wg.Add(1) go func() { - defer e.wg.Done() + defer eventDispatcherManager.wg.Done() for { - heartbeatResponse := e.GetHeartbeatResponseQueue().Dequeue() + heartbeatResponse := eventDispatcherManager.GetHeartbeatResponseQueue().Dequeue() dispatcherActions := heartbeatResponse.Actions for _, dispatcherAction := range dispatcherActions { tableSpan := dispatcherAction.Span - dispatcher, ok := e.dispatcherMap.Get(&common.TableSpan{TableSpan: tableSpan}) + dispatcher, ok := eventDispatcherManager.dispatcherMap.Get(&common.TableSpan{TableSpan: tableSpan}) if !ok { log.Error("dispatcher not found", zap.Any("tableSpan", tableSpan)) continue @@ -165,9 +156,20 @@ func (e *EventDispatcherManager) Init(startTs uint64) error { } }() - //Init Table Trigger Event Dispatcher - e.tableTriggerEventDispatcher = e.newTableTriggerEventDispatcher(startTs) + eventDispatcherManager.wg.Add(1) + go eventDispatcherManager.CollectHeartbeatInfoWhenStatesChanged(ctx) + + return eventDispatcherManager +} +func (e *EventDispatcherManager) InitSink() error { + cfg, db, err := writer.NewMysqlConfigAndDB(e.config.SinkURI) + if err != nil { + log.Error("create mysql sink failed", zap.Error(err)) + return err + } + + e.sink = sink.NewMysqlSink(e.changefeedID, 16, cfg, db) return nil } @@ -226,6 +228,27 @@ func calculateStartSyncPointTs(startTs uint64, syncPointInterval time.Duration) } */ +func (e *EventDispatcherManager) NewDispatcher(tableSpan *common.TableSpan, startTs uint64) { + if tableSpan == &common.DDLSpan { + e.newTableTriggerEventDispatcher(startTs) + } else { + e.NewTableEventDispatcher(tableSpan, startTs) + } +} + +// TODO:加一个 count 监控 +func (e *EventDispatcherManager) newTableTriggerEventDispatcher(startTs uint64) *dispatcher.TableTriggerEventDispatcher { + tableTriggerEventDispatcher := dispatcher.NewTableTriggerEventDispatcher(e.sink, startTs, e.tableSpanStatusesChan, e.filter) + + appcontext.GetService[*eventcollector.EventCollector](appcontext.EventCollector).RegisterDispatcher(tableTriggerEventDispatcher, startTs, toFilterConfigPB(e.config.Filter)) + + e.GetTableSpanStatusesChan() <- &heartbeatpb.TableSpanStatus{ + Span: tableTriggerEventDispatcher.GetTableSpan().TableSpan, + ComponentStatus: heartbeatpb.ComponentState_Working, + } + return tableTriggerEventDispatcher +} + // 收到 rpc 请求创建,需要通过 event dispatcher manager 来 func (e *EventDispatcherManager) NewTableEventDispatcher(tableSpan *common.TableSpan, startTs uint64) *dispatcher.TableEventDispatcher { start := time.Now() @@ -247,7 +270,7 @@ func (e *EventDispatcherManager) NewTableEventDispatcher(tableSpan *common.Table */ tableEventDispatcher := dispatcher.NewTableEventDispatcher(tableSpan, e.sink, startTs, nil, e.tableSpanStatusesChan, e.filter) - appcontext.GetService[*eventcollector.EventCollector](appcontext.EventCollector).RegisterDispatcher(tableEventDispatcher, startTs, nil) + appcontext.GetService[*eventcollector.EventCollector](appcontext.EventCollector).RegisterDispatcher(tableEventDispatcher, startTs, toFilterConfigPB(e.config.Filter)) e.dispatcherMap.Set(tableSpan, tableEventDispatcher) e.GetTableSpanStatusesChan() <- &heartbeatpb.TableSpanStatus{ @@ -304,8 +327,16 @@ func (e *EventDispatcherManager) CollectHeartbeatInfoWhenStatesChanged(ctx conte } } -func (e *EventDispatcherManager) RemoveTableEventDispatcher(tableSpan *common.TableSpan) { - dispatcher, ok := e.dispatcherMap.Get(tableSpan) +func (e *EventDispatcherManager) RemoveDispatcher(tableSpan *common.TableSpan) { + var dispatcher dispatcher.Dispatcher + var ok bool + if tableSpan == &common.DDLSpan { + dispatcher = e.tableTriggerEventDispatcher + ok = dispatcher.GetComponentStatus() == heartbeatpb.ComponentState_Working + } else { + dispatcher, ok = e.dispatcherMap.Get(tableSpan) + } + if ok { if dispatcher.GetComponentStatus() == heartbeatpb.ComponentState_Stopping { e.GetTableSpanStatusesChan() <- &heartbeatpb.TableSpanStatus{ @@ -331,14 +362,6 @@ func (e *EventDispatcherManager) cleanTableEventDispatcher(tableSpan *common.Tab log.Info("table event dispatcher completely stopped, and delete it from event dispatcher manager", zap.Any("tableSpan", tableSpan)) } -func (e *EventDispatcherManager) newTableTriggerEventDispatcher(startTs uint64) *dispatcher.TableTriggerEventDispatcher { - tableTriggerEventDispatcher := dispatcher.NewTableTriggerEventDispatcher(e.sink, startTs, e.tableSpanStatusesChan, e.filter) - - appcontext.GetService[*eventcollector.EventCollector](appcontext.EventCollector).RegisterDispatcher(tableTriggerEventDispatcher, startTs, toFilterConfigPB(e.config.Filter)) - - return tableTriggerEventDispatcher -} - func toFilterConfigPB(filter *cfg.FilterConfig) *eventpb.FilterConfig { filterConfig := &eventpb.FilterConfig{ Rules: filter.Rules, diff --git a/downstreamadapter/dispatchermanager/heartbeat_collector.go b/downstreamadapter/dispatchermanager/heartbeat_collector.go index d9e8de3b9..0f7a71d1a 100644 --- a/downstreamadapter/dispatchermanager/heartbeat_collector.go +++ b/downstreamadapter/dispatchermanager/heartbeat_collector.go @@ -197,7 +197,7 @@ func (c *HeartBeatCollector) handleDispatcherRequestMessages(req *heartbeatpb.Sc if scheduleAction == heartbeatpb.ScheduleAction_Create { // TODO: 后续需要优化这段逻辑,perpared 这种调度状态需要多发 message 回去 if !req.IsSecondary { - eventDispatcherManager.NewTableEventDispatcher(&common.TableSpan{TableSpan: config.Span}, config.StartTs) + eventDispatcherManager.NewDispatcher(&common.TableSpan{TableSpan: config.Span}, config.StartTs) } else { // eventDispatcherManager.GetTableSpanStatusesChan() <- &heartbeatpb.TableSpanStatus{ // Span: config.Span, @@ -205,7 +205,7 @@ func (c *HeartBeatCollector) handleDispatcherRequestMessages(req *heartbeatpb.Sc // } } } else if scheduleAction == heartbeatpb.ScheduleAction_Remove { - eventDispatcherManager.RemoveTableEventDispatcher(&common.TableSpan{TableSpan: config.Span}) + eventDispatcherManager.RemoveDispatcher(&common.TableSpan{TableSpan: config.Span}) } // log.Info("RecvSchedulerDispatcherRequestMessages handle dispatch msg", zap.Any("tableSpan", config.Span), diff --git a/downstreamadapter/dispathermanagermanager/dispatcher_manager_manager.go b/downstreamadapter/dispathermanagermanager/dispatcher_manager_manager.go index 11e5a8187..16f40ab72 100644 --- a/downstreamadapter/dispathermanagermanager/dispatcher_manager_manager.go +++ b/downstreamadapter/dispathermanagermanager/dispatcher_manager_manager.go @@ -58,7 +58,7 @@ func (m *DispatcherManagerManager) RecvMaintainerRequest(ctx context.Context, ms func (m *DispatcherManagerManager) handleAddDispatcherManager(from messaging.ServerId, maintainerBootstrapRequest *heartbeatpb.MaintainerBootstrapRequest) error { changefeedID := model.DefaultChangeFeedID(maintainerBootstrapRequest.ChangefeedID) - + createTableTriggerEventDispatcher := maintainerBootstrapRequest.GetCreateTableTriggerEventDispatcher() eventDispatcherManager, ok := m.dispatcherManagers[changefeedID] if !ok { // TODO: decode config @@ -71,7 +71,7 @@ func (m *DispatcherManagerManager) handleAddDispatcherManager(from messaging.Ser return err } // TODO: 这边额外判断一下创建是否失败,创建失败的话,想一下怎么做报错处理 - eventDispatcherManager := dispatchermanager.NewEventDispatcherManager(changefeedID, cfConfig, from) + eventDispatcherManager := dispatchermanager.NewEventDispatcherManager(changefeedID, cfConfig, from, createTableTriggerEventDispatcher) m.dispatcherManagers[changefeedID] = eventDispatcherManager metrics.EventDispatcherManagerGauge.WithLabelValues(changefeedID.Namespace, changefeedID.ID).Inc() diff --git a/downstreamadapter/eventcollector/event_collector.go b/downstreamadapter/eventcollector/event_collector.go index 6d643f931..d61e57a11 100644 --- a/downstreamadapter/eventcollector/event_collector.go +++ b/downstreamadapter/eventcollector/event_collector.go @@ -63,8 +63,9 @@ func (m *DispatcherMap) Delete(dispatcherId string) { } type RegisterInfo struct { - dispatcher dispatcher.Dispatcher - startTs uint64 + dispatcher dispatcher.Dispatcher + startTs uint64 + filterConfig *eventpb.FilterConfig } /* @@ -111,6 +112,7 @@ func NewEventCollector(globalMemoryQuota int64, serverId messaging.ServerId) *Ev Remove: false, StartTs: startTs, ServerId: c.serverId.String(), + FilterConfig: registerInfo.filterConfig, }}, }) if err != nil { @@ -148,8 +150,9 @@ func (c *EventCollector) RegisterDispatcher(d dispatcher.Dispatcher, startTs uin if err != nil { log.Error("failed to send register dispatcher request message", zap.Error(err)) c.registerMessageChan.In() <- &RegisterInfo{ - dispatcher: d, - startTs: startTs, + dispatcher: d, + startTs: startTs, + filterConfig: filterConfig, } return err } diff --git a/heartbeatpb/heartbeat.pb.go b/heartbeatpb/heartbeat.pb.go index 6ed527516..593fc52b3 100644 --- a/heartbeatpb/heartbeat.pb.go +++ b/heartbeatpb/heartbeat.pb.go @@ -989,8 +989,9 @@ func (m *RemoveMaintainerRequest) GetCascade() bool { } type MaintainerBootstrapRequest struct { - ChangefeedID string `protobuf:"bytes,1,opt,name=changefeedID,proto3" json:"changefeedID,omitempty"` - Config []byte `protobuf:"bytes,2,opt,name=config,proto3" json:"config,omitempty"` + ChangefeedID string `protobuf:"bytes,1,opt,name=changefeedID,proto3" json:"changefeedID,omitempty"` + Config []byte `protobuf:"bytes,2,opt,name=config,proto3" json:"config,omitempty"` + CreateTableTriggerEventDispatcher bool `protobuf:"varint,3,opt,name=createTableTriggerEventDispatcher,proto3" json:"createTableTriggerEventDispatcher,omitempty"` } func (m *MaintainerBootstrapRequest) Reset() { *m = MaintainerBootstrapRequest{} } @@ -1040,6 +1041,13 @@ func (m *MaintainerBootstrapRequest) GetConfig() []byte { return nil } +func (m *MaintainerBootstrapRequest) GetCreateTableTriggerEventDispatcher() bool { + if m != nil { + return m.CreateTableTriggerEventDispatcher + } + return false +} + type MaintainerBootstrapResponse struct { ChangefeedID string `protobuf:"bytes,1,opt,name=changefeedID,proto3" json:"changefeedID,omitempty"` Statuses []*TableSpanStatus `protobuf:"bytes,2,rep,name=statuses,proto3" json:"statuses,omitempty"` @@ -1431,81 +1439,83 @@ func init() { func init() { proto.RegisterFile("heartbeatpb/heartbeat.proto", fileDescriptor_6d584080fdadb670) } var fileDescriptor_6d584080fdadb670 = []byte{ - // 1171 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xa4, 0x57, 0x4b, 0x6f, 0xdb, 0xc6, - 0x13, 0x37, 0x45, 0x59, 0x8f, 0xb1, 0x22, 0x33, 0x9b, 0x20, 0x51, 0xe2, 0x44, 0x70, 0xf8, 0xcf, - 0x41, 0x7f, 0x07, 0xb0, 0x51, 0xa7, 0x45, 0x5a, 0xa0, 0x28, 0x60, 0x2b, 0x09, 0xf2, 0x40, 0xda, - 0x80, 0x4a, 0xe1, 0xa4, 0x17, 0x63, 0x45, 0x4e, 0x24, 0x42, 0xd2, 0x2e, 0xbb, 0xbb, 0x8a, 0x11, - 0xf4, 0xde, 0x1e, 0x5b, 0xa0, 0x40, 0x3f, 0x48, 0xcf, 0xfd, 0x00, 0x3d, 0xe6, 0xd0, 0x43, 0x4f, - 0x45, 0x91, 0x7c, 0x91, 0x62, 0x97, 0xa4, 0x44, 0x52, 0xb2, 0x6b, 0xa1, 0xb7, 0x9d, 0xd7, 0x6f, - 0x66, 0x67, 0xe7, 0x41, 0xc2, 0xd6, 0x10, 0xa9, 0x50, 0x7d, 0xa4, 0x2a, 0xea, 0xef, 0xcd, 0xce, - 0xbb, 0x91, 0xe0, 0x8a, 0x93, 0x8d, 0x8c, 0xd0, 0x7d, 0x05, 0xf5, 0x17, 0xb4, 0x3f, 0xc6, 0x5e, - 0x44, 0x19, 0x69, 0x41, 0xd5, 0x10, 0x8f, 0xef, 0xb7, 0xac, 0x6d, 0xab, 0x53, 0xf6, 0x52, 0x92, - 0x5c, 0x87, 0x5a, 0x4f, 0x51, 0xa1, 0x9e, 0xe2, 0xdb, 0x56, 0x69, 0xdb, 0xea, 0x34, 0xbc, 0x19, - 0x4d, 0xae, 0x40, 0xe5, 0x01, 0x0b, 0xb4, 0xc4, 0x36, 0x92, 0x84, 0x72, 0x7f, 0x2d, 0x81, 0xf3, - 0x48, 0xbb, 0x3a, 0x44, 0xaa, 0x3c, 0xfc, 0x76, 0x8a, 0x52, 0x11, 0x17, 0x1a, 0xfe, 0x90, 0xb2, - 0x01, 0xbe, 0x46, 0x0c, 0x12, 0x3f, 0x75, 0x2f, 0xc7, 0x23, 0x1f, 0x43, 0xfd, 0x84, 0x2a, 0x14, - 0x13, 0x2a, 0x46, 0xc6, 0xdb, 0xc6, 0xfe, 0x95, 0xdd, 0x4c, 0xd0, 0xbb, 0x47, 0xa9, 0xd4, 0x9b, - 0x2b, 0x92, 0x4f, 0xa1, 0x26, 0x15, 0x55, 0x53, 0x89, 0xb2, 0x65, 0x6f, 0xdb, 0x9d, 0x8d, 0xfd, - 0x1b, 0x39, 0xa3, 0xd9, 0x35, 0x7b, 0x46, 0xcb, 0x9b, 0x69, 0x93, 0x0e, 0x6c, 0xfa, 0x7c, 0x12, - 0xe1, 0x18, 0x15, 0xc6, 0xc2, 0x56, 0x79, 0xdb, 0xea, 0xd4, 0xbc, 0x22, 0x9b, 0xdc, 0x85, 0xea, - 0x09, 0x15, 0x2c, 0x64, 0x83, 0xd6, 0xba, 0x89, 0xeb, 0x5a, 0xce, 0x85, 0x37, 0x65, 0x5a, 0xf6, - 0x40, 0x08, 0x2e, 0xbc, 0x54, 0x93, 0xdc, 0x01, 0x1b, 0x85, 0x68, 0x55, 0xfe, 0xcd, 0x40, 0x6b, - 0xb9, 0x5f, 0x41, 0x7d, 0x76, 0xbb, 0x38, 0x59, 0xe8, 0x8f, 0x22, 0x1e, 0x32, 0xf5, 0x42, 0x26, - 0x8f, 0x92, 0xe3, 0x91, 0x36, 0x80, 0x40, 0xc9, 0xc7, 0x6f, 0x30, 0x78, 0x21, 0x4d, 0xb6, 0xca, - 0x5e, 0x86, 0xe3, 0xfe, 0x60, 0x81, 0x73, 0x3f, 0x94, 0x11, 0x55, 0xfe, 0x10, 0xc5, 0x81, 0xaf, - 0x42, 0xce, 0xc8, 0x0e, 0x94, 0x65, 0x44, 0x99, 0x01, 0x2c, 0x26, 0x77, 0x96, 0x27, 0xcf, 0xe8, - 0x90, 0x3b, 0x50, 0xa1, 0xc6, 0xca, 0x80, 0x37, 0xf7, 0x2f, 0xe5, 0xb4, 0x63, 0x40, 0x2f, 0x51, - 0xd1, 0x75, 0xd2, 0xe5, 0x93, 0x49, 0xa8, 0xa3, 0xb5, 0x4d, 0x2c, 0x33, 0xda, 0x7d, 0x06, 0xf6, - 0x41, 0xf7, 0xe9, 0x4a, 0xbe, 0xb3, 0x70, 0xa5, 0x02, 0xdc, 0x2f, 0x16, 0x5c, 0xcc, 0x94, 0x97, - 0x8c, 0x38, 0x93, 0x48, 0xee, 0x41, 0x35, 0x0e, 0x45, 0x67, 0x4b, 0x17, 0xc1, 0xcd, 0x9c, 0x83, - 0x62, 0x26, 0xbc, 0x54, 0x7b, 0xa1, 0x30, 0x4b, 0x4b, 0x0a, 0xf3, 0x36, 0x94, 0xa9, 0x3f, 0x4a, - 0xcb, 0xcb, 0xc9, 0x27, 0xa2, 0xfb, 0xd4, 0x33, 0x52, 0xf7, 0x65, 0x36, 0xe1, 0x5d, 0xce, 0x5e, - 0x87, 0x83, 0x95, 0x2e, 0xdd, 0x82, 0xaa, 0xd4, 0xbd, 0x35, 0xbb, 0x73, 0x4a, 0xba, 0x7f, 0x59, - 0x70, 0xad, 0xe7, 0x0f, 0x31, 0x98, 0x8e, 0x71, 0xee, 0x62, 0x95, 0xd6, 0xfa, 0x04, 0x2a, 0xbe, - 0x89, 0x28, 0xe9, 0xab, 0xd3, 0xb2, 0x13, 0x87, 0xed, 0x25, 0xca, 0xa4, 0x0b, 0x4d, 0x99, 0xf8, - 0x8d, 0xf3, 0x66, 0x1e, 0xb7, 0xb9, 0xbf, 0x95, 0x33, 0xef, 0xe5, 0x54, 0xbc, 0x82, 0x09, 0xb9, - 0x05, 0x8d, 0x50, 0x1e, 0x4b, 0xf4, 0x39, 0x0b, 0xa8, 0x78, 0x9b, 0xf4, 0xd8, 0x46, 0x28, 0x7b, - 0x29, 0xcb, 0x7d, 0x0e, 0x97, 0x9e, 0xd1, 0x90, 0x29, 0x1a, 0x32, 0x14, 0x8f, 0x52, 0x68, 0xf2, - 0x59, 0xa6, 0xb5, 0x97, 0xbd, 0xea, 0xdc, 0xa6, 0xd8, 0xdb, 0xee, 0x8f, 0x25, 0x70, 0x8a, 0xe2, - 0x73, 0x65, 0xea, 0x26, 0x80, 0x3e, 0x1d, 0x6b, 0x24, 0x4c, 0xaa, 0xa1, 0xae, 0x39, 0x1a, 0x03, - 0xc9, 0x47, 0xb0, 0x1e, 0x4b, 0x96, 0x25, 0xa2, 0xcb, 0x27, 0x11, 0x67, 0xc8, 0x94, 0xd1, 0xf5, - 0x62, 0x4d, 0xf2, 0x3f, 0xb8, 0x30, 0xef, 0xdc, 0x63, 0x15, 0x0f, 0x99, 0x62, 0x3b, 0xe7, 0x26, - 0x8c, 0xbd, 0xea, 0x84, 0xb1, 0xcf, 0x31, 0x61, 0xee, 0xc1, 0x56, 0x97, 0x73, 0x11, 0x84, 0x8c, - 0x2a, 0x2e, 0x0e, 0x39, 0x57, 0x52, 0x09, 0x1a, 0xa5, 0x55, 0xd4, 0x82, 0xea, 0x1b, 0x14, 0x52, - 0xbf, 0xb1, 0x4e, 0x8b, 0xed, 0xa5, 0xa4, 0xfb, 0x0a, 0x6e, 0x2c, 0x37, 0x4c, 0x5a, 0xef, 0x3f, - 0xbc, 0xd2, 0x6f, 0x16, 0x5c, 0x4b, 0x8b, 0x6f, 0xae, 0x96, 0x86, 0xf4, 0x04, 0x36, 0x69, 0x10, - 0x1c, 0x4f, 0x66, 0x82, 0x14, 0xff, 0x56, 0xbe, 0x03, 0x83, 0x60, 0xc1, 0xd6, 0x6b, 0xd2, 0x2c, - 0x57, 0x92, 0x1e, 0x10, 0x81, 0x13, 0xfe, 0x06, 0x73, 0x70, 0x25, 0x03, 0x77, 0x3b, 0x9f, 0x39, - 0xa3, 0xb6, 0x88, 0x78, 0x51, 0x14, 0x04, 0xd2, 0xfd, 0xde, 0x82, 0xcb, 0xcb, 0xbc, 0x93, 0x26, - 0x94, 0xc2, 0x20, 0x29, 0xaf, 0x52, 0x18, 0xe8, 0x55, 0x99, 0x69, 0xbf, 0xc6, 0xac, 0xbf, 0x8a, - 0xad, 0x61, 0x2f, 0xb4, 0xc6, 0xb9, 0xaa, 0xc7, 0xed, 0xc2, 0xd5, 0x53, 0xc2, 0x5e, 0x08, 0xa5, - 0x05, 0x55, 0x9f, 0x4a, 0x9f, 0x06, 0x71, 0x71, 0xd7, 0xbc, 0x94, 0x74, 0x5f, 0xc2, 0xf5, 0xb9, - 0xf9, 0x42, 0x7d, 0x9c, 0xa7, 0x77, 0x4e, 0xb9, 0xa6, 0xfb, 0x1d, 0x6c, 0x2d, 0x45, 0x4e, 0x0a, - 0xe8, 0x3c, 0xd0, 0xd9, 0x2d, 0x5f, 0x5a, 0x65, 0xcb, 0xbb, 0x9f, 0xc3, 0x95, 0xb9, 0xf3, 0xee, - 0x98, 0x4b, 0x5c, 0xe1, 0x4a, 0xee, 0x11, 0x5c, 0x5d, 0xb0, 0x5e, 0x21, 0x6c, 0x3d, 0xd3, 0xa7, - 0xbe, 0x8f, 0x52, 0xa6, 0xd9, 0x4e, 0x48, 0xf7, 0xe7, 0x12, 0xac, 0xc7, 0x23, 0xe5, 0x06, 0xd4, - 0x1f, 0xcb, 0xc3, 0x31, 0xf7, 0x47, 0x18, 0x3f, 0x54, 0xcd, 0x9b, 0x33, 0x34, 0x82, 0x39, 0xce, - 0xb7, 0x42, 0x42, 0x92, 0x2f, 0xa0, 0x19, 0x1f, 0xd3, 0xab, 0x27, 0xfb, 0xe9, 0xb4, 0x2d, 0x53, - 0xd0, 0x26, 0x4f, 0xe0, 0xf2, 0x97, 0x88, 0xc1, 0x7d, 0xc1, 0xa3, 0x08, 0x83, 0x39, 0x4a, 0xf9, - 0x4c, 0x94, 0xa5, 0x36, 0xe4, 0x21, 0x10, 0xcd, 0x3f, 0x08, 0x82, 0x2c, 0xd2, 0xfa, 0x99, 0x48, - 0x4b, 0x2c, 0xdc, 0x3f, 0x2c, 0xd8, 0x2c, 0x3c, 0xe5, 0x4a, 0x3b, 0xf4, 0x21, 0x38, 0x7e, 0x3a, - 0x84, 0x8f, 0xe3, 0x12, 0x48, 0x3e, 0x5f, 0xce, 0x9c, 0xd4, 0x9b, 0x7e, 0x96, 0x9e, 0xea, 0x4f, - 0xc3, 0xcc, 0x98, 0xdf, 0xd8, 0x27, 0xf9, 0x7d, 0xb7, 0xea, 0x74, 0x77, 0x03, 0x68, 0x64, 0x07, - 0x32, 0x21, 0x50, 0x56, 0xe1, 0x04, 0x93, 0x92, 0x31, 0x67, 0xcd, 0x63, 0x3c, 0x48, 0x57, 0x8e, - 0x39, 0x6b, 0x9e, 0xaf, 0x79, 0x76, 0xcc, 0xd3, 0x67, 0x5d, 0x10, 0x13, 0x94, 0x92, 0x0e, 0xd0, - 0xb8, 0xaa, 0x7b, 0x29, 0xb9, 0x73, 0x13, 0x2a, 0xc9, 0xca, 0xad, 0xc3, 0xfa, 0x91, 0x08, 0x15, - 0x3a, 0x6b, 0xa4, 0x06, 0xe5, 0xe7, 0x54, 0x4a, 0xc7, 0xda, 0xe9, 0x40, 0x33, 0xbf, 0xa9, 0x09, - 0x40, 0xa5, 0x2b, 0x90, 0x1a, 0x3d, 0x80, 0x4a, 0x3c, 0x42, 0x1c, 0x6b, 0x87, 0x41, 0x33, 0x9f, - 0x20, 0xb2, 0x01, 0xd5, 0xaf, 0xd9, 0x88, 0xf1, 0x13, 0x16, 0xab, 0x1e, 0xf4, 0x25, 0x32, 0xe5, - 0x58, 0xe4, 0x02, 0xd4, 0x9f, 0x0b, 0x8c, 0xa8, 0x08, 0xd9, 0xc0, 0x29, 0x91, 0x06, 0xd4, 0x62, - 0x12, 0x03, 0xc7, 0xd6, 0x56, 0x47, 0x5c, 0x8c, 0xb4, 0xa8, 0xac, 0x45, 0x3d, 0xc5, 0xa3, 0x48, - 0x53, 0xeb, 0x5a, 0x64, 0x28, 0x0c, 0x9c, 0xca, 0x61, 0xf7, 0xf7, 0xf7, 0x6d, 0xeb, 0xdd, 0xfb, - 0xb6, 0xf5, 0xf7, 0xfb, 0xb6, 0xf5, 0xd3, 0x87, 0xf6, 0xda, 0xbb, 0x0f, 0xed, 0xb5, 0x3f, 0x3f, - 0xb4, 0xd7, 0xbe, 0xf9, 0xff, 0x20, 0x54, 0xc3, 0x69, 0x7f, 0xd7, 0xe7, 0x93, 0xbd, 0xd7, 0x63, - 0x7e, 0xd2, 0xc7, 0x21, 0x8d, 0xa2, 0xb7, 0x7b, 0x2a, 0x1c, 0x50, 0x85, 0x7b, 0x99, 0x57, 0xe9, - 0x57, 0xcc, 0x5f, 0xce, 0xdd, 0x7f, 0x02, 0x00, 0x00, 0xff, 0xff, 0x41, 0x8a, 0xa0, 0x05, 0x04, - 0x0d, 0x00, 0x00, + // 1201 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xa4, 0x57, 0xcd, 0x6e, 0xdb, 0xc6, + 0x13, 0x37, 0x45, 0x59, 0x1f, 0x63, 0x47, 0x66, 0x36, 0x41, 0xa2, 0xc4, 0x89, 0xe0, 0xf0, 0x9f, + 0x83, 0xfe, 0x0e, 0x60, 0xa3, 0x4e, 0x8b, 0xb4, 0x40, 0x51, 0xc0, 0x56, 0x12, 0xe4, 0xa3, 0x69, + 0x03, 0xca, 0x85, 0x9b, 0x5e, 0x8c, 0x15, 0x39, 0x91, 0x08, 0x49, 0xbb, 0xec, 0xee, 0xca, 0x46, + 0xd0, 0x7b, 0x7b, 0x6c, 0x81, 0x02, 0x7d, 0x84, 0x3e, 0x40, 0xcf, 0x7d, 0x80, 0x1e, 0x73, 0xe8, + 0xa1, 0xa7, 0xa2, 0x48, 0x5e, 0xa4, 0xd8, 0x25, 0x29, 0x91, 0x94, 0x9c, 0x58, 0xe8, 0x6d, 0x67, + 0xe6, 0x37, 0x1f, 0x3b, 0x3b, 0x1f, 0x24, 0x6c, 0x0e, 0x90, 0x0a, 0xd5, 0x43, 0xaa, 0xa2, 0xde, + 0xee, 0xf4, 0xbc, 0x13, 0x09, 0xae, 0x38, 0x59, 0xcb, 0x08, 0xdd, 0x17, 0x50, 0x3f, 0xa4, 0xbd, + 0x11, 0x76, 0x23, 0xca, 0x48, 0x13, 0xaa, 0x86, 0x78, 0x7c, 0xbf, 0x69, 0x6d, 0x59, 0xed, 0xb2, + 0x97, 0x92, 0xe4, 0x3a, 0xd4, 0xba, 0x8a, 0x0a, 0xf5, 0x14, 0x5f, 0x35, 0x4b, 0x5b, 0x56, 0x7b, + 0xdd, 0x9b, 0xd2, 0xe4, 0x0a, 0x54, 0x1e, 0xb0, 0x40, 0x4b, 0x6c, 0x23, 0x49, 0x28, 0xf7, 0xb7, + 0x12, 0x38, 0x8f, 0xb4, 0xab, 0x03, 0xa4, 0xca, 0xc3, 0x6f, 0x27, 0x28, 0x15, 0x71, 0x61, 0xdd, + 0x1f, 0x50, 0xd6, 0xc7, 0x97, 0x88, 0x41, 0xe2, 0xa7, 0xee, 0xe5, 0x78, 0xe4, 0x43, 0xa8, 0x9f, + 0x52, 0x85, 0x62, 0x4c, 0xc5, 0xd0, 0x78, 0x5b, 0xdb, 0xbb, 0xb2, 0x93, 0x09, 0x7a, 0xe7, 0x28, + 0x95, 0x7a, 0x33, 0x20, 0xf9, 0x18, 0x6a, 0x52, 0x51, 0x35, 0x91, 0x28, 0x9b, 0xf6, 0x96, 0xdd, + 0x5e, 0xdb, 0xbb, 0x91, 0x53, 0x9a, 0x5e, 0xb3, 0x6b, 0x50, 0xde, 0x14, 0x4d, 0xda, 0xb0, 0xe1, + 0xf3, 0x71, 0x84, 0x23, 0x54, 0x18, 0x0b, 0x9b, 0xe5, 0x2d, 0xab, 0x5d, 0xf3, 0x8a, 0x6c, 0x72, + 0x17, 0xaa, 0xa7, 0x54, 0xb0, 0x90, 0xf5, 0x9b, 0xab, 0x26, 0xae, 0x6b, 0x39, 0x17, 0xde, 0x84, + 0x69, 0xd9, 0x03, 0x21, 0xb8, 0xf0, 0x52, 0x24, 0xb9, 0x03, 0x36, 0x0a, 0xd1, 0xac, 0xbc, 0x4f, + 0x41, 0xa3, 0xdc, 0x2f, 0xa1, 0x3e, 0xbd, 0x5d, 0x9c, 0x2c, 0xf4, 0x87, 0x11, 0x0f, 0x99, 0x3a, + 0x94, 0xc9, 0xa3, 0xe4, 0x78, 0xa4, 0x05, 0x20, 0x50, 0xf2, 0xd1, 0x09, 0x06, 0x87, 0xd2, 0x64, + 0xab, 0xec, 0x65, 0x38, 0xee, 0x0f, 0x16, 0x38, 0xf7, 0x43, 0x19, 0x51, 0xe5, 0x0f, 0x50, 0xec, + 0xfb, 0x2a, 0xe4, 0x8c, 0x6c, 0x43, 0x59, 0x46, 0x94, 0x19, 0x83, 0xc5, 0xe4, 0x4e, 0xf3, 0xe4, + 0x19, 0x0c, 0xb9, 0x03, 0x15, 0x6a, 0xb4, 0x8c, 0xf1, 0xc6, 0xde, 0xa5, 0x1c, 0x3a, 0x36, 0xe8, + 0x25, 0x10, 0x5d, 0x27, 0x1d, 0x3e, 0x1e, 0x87, 0x3a, 0x5a, 0xdb, 0xc4, 0x32, 0xa5, 0xdd, 0x67, + 0x60, 0xef, 0x77, 0x9e, 0x2e, 0xe5, 0x3b, 0x6b, 0xae, 0x54, 0x30, 0xf7, 0x8b, 0x05, 0x17, 0x33, + 0xe5, 0x25, 0x23, 0xce, 0x24, 0x92, 0x7b, 0x50, 0x8d, 0x43, 0xd1, 0xd9, 0xd2, 0x45, 0x70, 0x33, + 0xe7, 0xa0, 0x98, 0x09, 0x2f, 0x45, 0xcf, 0x15, 0x66, 0x69, 0x41, 0x61, 0xde, 0x86, 0x32, 0xf5, + 0x87, 0x69, 0x79, 0x39, 0xf9, 0x44, 0x74, 0x9e, 0x7a, 0x46, 0xea, 0x7e, 0x9d, 0x4d, 0x78, 0x87, + 0xb3, 0x97, 0x61, 0x7f, 0xa9, 0x4b, 0x37, 0xa1, 0x2a, 0x75, 0x6f, 0x4d, 0xef, 0x9c, 0x92, 0xee, + 0xdf, 0x16, 0x5c, 0xeb, 0xfa, 0x03, 0x0c, 0x26, 0x23, 0x9c, 0xb9, 0x58, 0xa6, 0xb5, 0x3e, 0x82, + 0x8a, 0x6f, 0x22, 0x4a, 0xfa, 0xea, 0xac, 0xec, 0xc4, 0x61, 0x7b, 0x09, 0x98, 0x74, 0xa0, 0x21, + 0x13, 0xbf, 0x71, 0xde, 0xcc, 0xe3, 0x36, 0xf6, 0x36, 0x73, 0xea, 0xdd, 0x1c, 0xc4, 0x2b, 0xa8, + 0x90, 0x5b, 0xb0, 0x1e, 0xca, 0x63, 0x89, 0x3e, 0x67, 0x01, 0x15, 0xaf, 0x92, 0x1e, 0x5b, 0x0b, + 0x65, 0x37, 0x65, 0xb9, 0xcf, 0xe1, 0xd2, 0x33, 0x1a, 0x32, 0x45, 0x43, 0x86, 0xe2, 0x51, 0x6a, + 0x9a, 0x7c, 0x92, 0x69, 0xed, 0x45, 0xaf, 0x3a, 0xd3, 0x29, 0xf6, 0xb6, 0xfb, 0x63, 0x09, 0x9c, + 0xa2, 0xf8, 0x5c, 0x99, 0xba, 0x09, 0xa0, 0x4f, 0xc7, 0xda, 0x12, 0x26, 0xd5, 0x50, 0xd7, 0x1c, + 0x6d, 0x03, 0xc9, 0x07, 0xb0, 0x1a, 0x4b, 0x16, 0x25, 0xa2, 0xc3, 0xc7, 0x11, 0x67, 0xc8, 0x94, + 0xc1, 0x7a, 0x31, 0x92, 0xfc, 0x0f, 0x2e, 0xcc, 0x3a, 0xf7, 0x58, 0xc5, 0x43, 0xa6, 0xd8, 0xce, + 0xb9, 0x09, 0x63, 0x2f, 0x3b, 0x61, 0xec, 0x73, 0x4c, 0x98, 0x7b, 0xb0, 0xd9, 0xe1, 0x5c, 0x04, + 0x21, 0xa3, 0x8a, 0x8b, 0x03, 0xce, 0x95, 0x54, 0x82, 0x46, 0x69, 0x15, 0x35, 0xa1, 0x7a, 0x82, + 0x42, 0xea, 0x37, 0xd6, 0x69, 0xb1, 0xbd, 0x94, 0x74, 0x5f, 0xc0, 0x8d, 0xc5, 0x8a, 0x49, 0xeb, + 0xfd, 0x87, 0x57, 0xfa, 0xdd, 0x82, 0x6b, 0x69, 0xf1, 0xcd, 0x60, 0x69, 0x48, 0x4f, 0x60, 0x83, + 0x06, 0xc1, 0xf1, 0x78, 0x2a, 0x48, 0xed, 0xdf, 0xca, 0x77, 0x60, 0x10, 0xcc, 0xe9, 0x7a, 0x0d, + 0x9a, 0xe5, 0x4a, 0xd2, 0x05, 0x22, 0x70, 0xcc, 0x4f, 0x30, 0x67, 0xae, 0x64, 0xcc, 0xdd, 0xce, + 0x67, 0xce, 0xc0, 0xe6, 0x2d, 0x5e, 0x14, 0x05, 0x81, 0x74, 0xbf, 0xb7, 0xe0, 0xf2, 0x22, 0xef, + 0xa4, 0x01, 0xa5, 0x30, 0x48, 0xca, 0xab, 0x14, 0x06, 0x7a, 0x55, 0x66, 0xda, 0x6f, 0x7d, 0xda, + 0x5f, 0xc5, 0xd6, 0xb0, 0xe7, 0x5a, 0xe3, 0x5c, 0xd5, 0xe3, 0x76, 0xe0, 0xea, 0x19, 0x61, 0xcf, + 0x85, 0xd2, 0x84, 0xaa, 0x4f, 0xa5, 0x4f, 0x83, 0xb8, 0xb8, 0x6b, 0x5e, 0x4a, 0xba, 0xbf, 0x5a, + 0x70, 0x7d, 0xa6, 0x3f, 0x57, 0x20, 0xe7, 0x69, 0x9e, 0xb3, 0xee, 0xf9, 0x39, 0xdc, 0xf2, 0x05, + 0x52, 0x85, 0x66, 0xe6, 0x1d, 0x8a, 0xb0, 0xdf, 0x47, 0xf1, 0xe0, 0x04, 0x99, 0x9a, 0x8d, 0x9e, + 0xe4, 0xf2, 0xef, 0x07, 0xba, 0xdf, 0xc1, 0xe6, 0xc2, 0x38, 0x93, 0x7a, 0x3c, 0x4f, 0xa0, 0xd9, + 0x8f, 0x86, 0xd2, 0x32, 0x1f, 0x0d, 0xee, 0xa7, 0x70, 0x65, 0xe6, 0xbc, 0x33, 0xe2, 0x12, 0x97, + 0x48, 0x90, 0x7b, 0x04, 0x57, 0xe7, 0xb4, 0x97, 0x08, 0x5b, 0xaf, 0x88, 0x89, 0xef, 0xa3, 0x94, + 0xe9, 0xe3, 0x25, 0xa4, 0xfb, 0x73, 0x09, 0x56, 0xe3, 0x09, 0x75, 0x03, 0xea, 0x8f, 0xe5, 0xc1, + 0x88, 0xfb, 0x43, 0x8c, 0xdf, 0xbd, 0xe6, 0xcd, 0x18, 0xda, 0x82, 0x39, 0xce, 0x96, 0x4c, 0x42, + 0x92, 0xcf, 0xa0, 0x11, 0x1f, 0xd3, 0xab, 0x27, 0xeb, 0xee, 0xac, 0xa5, 0x55, 0x40, 0x93, 0x27, + 0x70, 0xf9, 0x0b, 0xc4, 0xe0, 0xbe, 0xe0, 0x51, 0x84, 0xc1, 0xcc, 0x4a, 0xf9, 0x9d, 0x56, 0x16, + 0xea, 0x90, 0x87, 0x40, 0x34, 0x7f, 0x3f, 0x08, 0xb2, 0x96, 0x56, 0xdf, 0x69, 0x69, 0x81, 0x86, + 0xfb, 0xa7, 0x05, 0x1b, 0x85, 0xa7, 0x5c, 0x6a, 0x25, 0x3f, 0x04, 0xc7, 0x4f, 0x67, 0xfa, 0x71, + 0x5c, 0x02, 0xc9, 0xd7, 0xd0, 0x3b, 0x07, 0xff, 0x86, 0x9f, 0xa5, 0x27, 0xfa, 0x4b, 0x33, 0xb3, + 0x35, 0xd6, 0xf6, 0x48, 0x7e, 0x7d, 0x2e, 0xbb, 0x2c, 0xdc, 0x00, 0xd6, 0xb3, 0xf3, 0x9d, 0x10, + 0x28, 0xab, 0x70, 0x8c, 0x49, 0xc9, 0x98, 0xb3, 0xe6, 0x31, 0x1e, 0xa4, 0x1b, 0xcc, 0x9c, 0x35, + 0xcf, 0xd7, 0x3c, 0x3b, 0xe6, 0xe9, 0xb3, 0x2e, 0x88, 0x31, 0x4a, 0x49, 0xfb, 0x68, 0x5c, 0xd5, + 0xbd, 0x94, 0xdc, 0xbe, 0x09, 0x95, 0x64, 0x83, 0xd7, 0x61, 0xf5, 0x48, 0x84, 0x0a, 0x9d, 0x15, + 0x52, 0x83, 0xf2, 0x73, 0x2a, 0xa5, 0x63, 0x6d, 0xb7, 0xa1, 0x91, 0x5f, 0xfc, 0x04, 0xa0, 0xd2, + 0x31, 0xcd, 0xeb, 0xac, 0xe8, 0x73, 0x3c, 0x91, 0x1c, 0x6b, 0x9b, 0x41, 0x23, 0x9f, 0x20, 0xb2, + 0x06, 0xd5, 0xaf, 0xd8, 0x90, 0xf1, 0x53, 0x16, 0x43, 0xf7, 0x7b, 0x12, 0x99, 0x72, 0x2c, 0x72, + 0x01, 0xea, 0xcf, 0x05, 0x46, 0x54, 0x84, 0xac, 0xef, 0x94, 0xc8, 0x3a, 0xd4, 0x62, 0x12, 0x03, + 0xc7, 0xd6, 0x5a, 0x47, 0x5c, 0x0c, 0xb5, 0xa8, 0xac, 0x45, 0x5d, 0xc5, 0xa3, 0x48, 0x53, 0xab, + 0x5a, 0x64, 0x28, 0x0c, 0x9c, 0xca, 0x41, 0xe7, 0x8f, 0x37, 0x2d, 0xeb, 0xf5, 0x9b, 0x96, 0xf5, + 0xcf, 0x9b, 0x96, 0xf5, 0xd3, 0xdb, 0xd6, 0xca, 0xeb, 0xb7, 0xad, 0x95, 0xbf, 0xde, 0xb6, 0x56, + 0xbe, 0xf9, 0x7f, 0x3f, 0x54, 0x83, 0x49, 0x6f, 0xc7, 0xe7, 0xe3, 0xdd, 0x97, 0x23, 0x7e, 0xda, + 0xc3, 0x01, 0x8d, 0xa2, 0x57, 0xbb, 0x2a, 0xec, 0x53, 0x85, 0xbb, 0x99, 0x57, 0xe9, 0x55, 0xcc, + 0x4f, 0xd3, 0xdd, 0x7f, 0x03, 0x00, 0x00, 0xff, 0xff, 0x6f, 0x34, 0x44, 0xd9, 0x53, 0x0d, 0x00, + 0x00, } func (m *TableSpan) Marshal() (dAtA []byte, err error) { @@ -2253,6 +2263,16 @@ func (m *MaintainerBootstrapRequest) MarshalToSizedBuffer(dAtA []byte) (int, err _ = i var l int _ = l + if m.CreateTableTriggerEventDispatcher { + i-- + if m.CreateTableTriggerEventDispatcher { + dAtA[i] = 1 + } else { + dAtA[i] = 0 + } + i-- + dAtA[i] = 0x18 + } if len(m.Config) > 0 { i -= len(m.Config) copy(dAtA[i:], m.Config) @@ -2902,6 +2922,9 @@ func (m *MaintainerBootstrapRequest) Size() (n int) { if l > 0 { n += 1 + l + sovHeartbeat(uint64(l)) } + if m.CreateTableTriggerEventDispatcher { + n += 2 + } return n } @@ -5077,6 +5100,26 @@ func (m *MaintainerBootstrapRequest) Unmarshal(dAtA []byte) error { m.Config = []byte{} } iNdEx = postIndex + case 3: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field CreateTableTriggerEventDispatcher", wireType) + } + var v int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowHeartbeat + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + m.CreateTableTriggerEventDispatcher = bool(v != 0) default: iNdEx = preIndex skippy, err := skipHeartbeat(dAtA[iNdEx:]) diff --git a/heartbeatpb/heartbeat.proto b/heartbeatpb/heartbeat.proto index 7a5d162a9..556e02009 100644 --- a/heartbeatpb/heartbeat.proto +++ b/heartbeatpb/heartbeat.proto @@ -103,6 +103,7 @@ message RemoveMaintainerRequest { message MaintainerBootstrapRequest { string changefeedID = 1; bytes config = 2; + bool createTableTriggerEventDispatcher = 3; } message MaintainerBootstrapResponse { From 57792c5b821bd3b72faabf5de884b9b3d662a60e Mon Sep 17 00:00:00 2001 From: hongyunyan <649330952@qq.com> Date: Fri, 16 Aug 2024 11:57:09 +0800 Subject: [PATCH 05/12] unity dispatcher --- downstreamadapter/dispatcher/dispatcher.go | 300 ++++++--- downstreamadapter/dispatcher/helper.go | 574 ++---------------- .../dispatcher/table_event_dispatcher.go | 232 ------- .../table_trigger_event_dispatcher.go | 276 --------- .../event_dispatcher_manager.go | 104 +--- .../dispatcher_manager_manager.go | 2 +- .../eventcollector/event_collector.go | 14 +- 7 files changed, 290 insertions(+), 1212 deletions(-) delete mode 100644 downstreamadapter/dispatcher/table_event_dispatcher.go delete mode 100644 downstreamadapter/dispatcher/table_trigger_event_dispatcher.go diff --git a/downstreamadapter/dispatcher/dispatcher.go b/downstreamadapter/dispatcher/dispatcher.go index c8c860397..ddd1b2a9a 100644 --- a/downstreamadapter/dispatcher/dispatcher.go +++ b/downstreamadapter/dispatcher/dispatcher.go @@ -16,20 +16,26 @@ package dispatcher import ( "context" "sync" + "sync/atomic" "time" "github.com/flowbehappy/tigate/downstreamadapter/sink" "github.com/flowbehappy/tigate/heartbeatpb" "github.com/flowbehappy/tigate/pkg/common" + "github.com/google/uuid" + "github.com/pingcap/log" "github.com/pingcap/tiflow/pkg/filter" + "go.uber.org/zap" ) /* Dispatcher is responsible for getting events from LogService and sending them to Sink in appropriate order. Each dispatcher only deal with the events of one tableSpan in one changefeed. -Each dispatcher corresponds to an event dispatcher task, working for the core work of the dispatcher. +Here is a special dispatcher will deal with the events of the DDLSpan in one changefeed, we call it TableTriggerEventDispatcher +Each EventDispatcherManager will have multiple dispatchers. + All dispatchers in the changefeed of the same node will share the same Sink. -All dispatchers will communicate with the Maintainer about self progress and whether can push down the blocked event. +All dispatchers will communicate with the Maintainer about self progress and whether can push down the blocked ddl event. Because Sink does not flush events to the downstream in strict order. the dispatcher can't send event to Sink continuously all the time, @@ -54,55 +60,203 @@ The workflow related to the dispatcher is as follows: | Maintainer | +------------+ */ -type Dispatcher interface { - GetSink() sink.Sink - GetTableSpan() *common.TableSpan - GetEventChan() chan *common.TxnEvent - GetResolvedTs() uint64 - UpdateResolvedTs(uint64) - GetCheckpointTs() uint64 - GetId() string - GetDispatcherType() DispatcherType - GetDDLActions() chan *heartbeatpb.DispatcherAction - GetACKs() chan *heartbeatpb.ACK - //GetSyncPointInfo() *SyncPointInfo - //GetMemoryUsage() *MemoryUsage - // PushEvent(event *eventpb.TxnEvent) - PushTxnEvent(event *common.TxnEvent) - GetComponentStatus() heartbeatpb.ComponentState - GetTableSpanStatusesChan() chan *heartbeatpb.TableSpanStatus - - TryClose() (w heartbeatpb.Watermark, ok bool) - GetFilter() filter.Filter - GetWG() *sync.WaitGroup - GetDDLPendingEvent() *common.TxnEvent - SetDDLPendingEvent(event *common.TxnEvent) - GetDDLFinishCh() chan struct{} - Remove() - GetRemovingStatus() bool + +type Dispatcher struct { + id string + eventCh chan *common.TxnEvent // 转换成一个函数 + tableSpan *common.TableSpan + sink sink.Sink + + ddlActions chan *heartbeatpb.DispatcherAction + acks chan *heartbeatpb.ACK + tableSpanStatusesChan chan *heartbeatpb.TableSpanStatus + + //SyncPointInfo *SyncPointInfo + + //MemoryUsage *MemoryUsage + + componentStatus *ComponentStateWithMutex + + filter filter.Filter + + resolvedTs *TsWithMutex // 用来记 eventChan 中目前收到的 event 中收到的最大的 commitTs - 1,不代表 dispatcher 的 checkpointTs + + cancel context.CancelFunc + wg sync.WaitGroup + + ddlPendingEvent *common.TxnEvent + ddlFinishCh chan struct{} + isRemoving atomic.Bool } -type DispatcherType uint64 +func NewDispatcher(tableSpan *common.TableSpan, sink sink.Sink, startTs uint64, tableSpanStatusesChan chan *heartbeatpb.TableSpanStatus, filter filter.Filter) *Dispatcher { + ctx, cancel := context.WithCancel(context.Background()) + dispatcher := &Dispatcher{ + id: uuid.NewString(), + eventCh: make(chan *common.TxnEvent, 16), + tableSpan: tableSpan, + sink: sink, + ddlActions: make(chan *heartbeatpb.DispatcherAction, 16), + acks: make(chan *heartbeatpb.ACK, 16), + tableSpanStatusesChan: tableSpanStatusesChan, + //SyncPointInfo: syncPointInfo, + //MemoryUsage: NewMemoryUsage(), + componentStatus: newComponentStateWithMutex(heartbeatpb.ComponentState_Working), + resolvedTs: newTsWithMutex(startTs), + cancel: cancel, + filter: filter, + ddlFinishCh: make(chan struct{}), + isRemoving: atomic.Bool{}, + } -const ( - TableEventDispatcherType DispatcherType = 0 - TableTriggerEventDispatcherType DispatcherType = 1 -) + dispatcher.sink.AddTableSpan(tableSpan) + dispatcher.wg.Add(1) + go dispatcher.DispatcherEvents(ctx) -/* -HeartBeatInfo is used to collect the message for HeartBeatRequest for each dispatcher. -Mainly about the progress of each dispatcher: -1. The checkpointTs of the dispatcher, shows that all the events whose ts <= checkpointTs are flushed to downstream successfully. -*/ -type HeartBeatInfo struct { - heartbeatpb.Watermark - Id string - TableSpan *common.TableSpan - ComponentStatus heartbeatpb.ComponentState - IsRemoving bool + dispatcher.wg.Add(1) + go dispatcher.HandleDDLActions(ctx) + + log.Info("dispatcher created", zap.Any("DispatcherID", dispatcher.id)) + + return dispatcher +} + +func (d *Dispatcher) DispatcherEvents(ctx context.Context) { + defer d.wg.Done() + tableSpan := d.GetTableSpan() + sink := d.GetSink() + for { + select { + case <-ctx.Done(): + return + case event := <-d.GetEventChan(): + if event.IsDMLEvent() { + sink.AddDMLEvent(tableSpan, event) + } else if event.IsDDLEvent() { + d.AddDDLEventToSinkWhenAvailable(event) + } else { + d.resolvedTs.Set(event.ResolvedTs) + } + } + } +} + +func (d *Dispatcher) GetSink() sink.Sink { + return d.sink +} + +func (d *Dispatcher) GetTableSpan() *common.TableSpan { + return d.tableSpan +} + +func (d *Dispatcher) GetEventChan() chan *common.TxnEvent { + return d.eventCh +} + +func (d *Dispatcher) GetResolvedTs() uint64 { + return d.resolvedTs.Get() +} + +func (d *Dispatcher) GetCheckpointTs() uint64 { + checkpointTs := d.GetSink().GetCheckpointTs(d.GetTableSpan()) + if checkpointTs == 0 { + // 说明从没有数据写到过 sink,则选择用 resolveTs 作为 checkpointTs + checkpointTs = d.GetResolvedTs() + } + return checkpointTs +} + +func (d *Dispatcher) UpdateResolvedTs(ts uint64) { + d.GetEventChan() <- &common.TxnEvent{ResolvedTs: ts} +} + +func (d *Dispatcher) GetId() string { + return d.id +} + +// func (d *Dispatcher) GetDispatcherType() DispatcherType { +// return TableEventDispatcherType +// } + +func (d *Dispatcher) GetDDLActions() chan *heartbeatpb.DispatcherAction { + return d.ddlActions +} + +func (d *Dispatcher) GetACKs() chan *heartbeatpb.ACK { + return d.acks +} + +func (d *Dispatcher) GetTableSpanStatusesChan() chan *heartbeatpb.TableSpanStatus { + return d.tableSpanStatusesChan +} + +//func (d *Dispatcher) GetSyncPointInfo() *SyncPointInfo { +// return d.syncPointInfo +// } + +// func (d *Dispatcher) GetMemoryUsage() *MemoryUsage { +// return d.MemoryUsage +// } + +func (d *Dispatcher) PushTxnEvent(event *common.TxnEvent) { + d.GetEventChan() <- event +} + +func (d *Dispatcher) Remove() { + // TODO: 修改这个 dispatcher 的 status 为 removing + d.cancel() + d.sink.StopTableSpan(d.tableSpan) + log.Info("table event dispatcher component status changed to stopping", zap.String("table", d.tableSpan.String())) + d.isRemoving.Store(true) +} + +func (d *Dispatcher) TryClose() (w heartbeatpb.Watermark, ok bool) { + // removing 后每次收集心跳的时候,call TryClose, 来判断是否能关掉 dispatcher 了(sink.isEmpty) + // 如果不能关掉,返回 0, false; 可以关掉的话,就返回 checkpointTs, true -- 这个要对齐过(startTs 和 checkpointTs 的关系) + if d.sink.IsEmpty(d.tableSpan) { + // calculate the checkpointTs, and clean the resource + d.sink.RemoveTableSpan(d.tableSpan) + w.CheckpointTs = d.GetCheckpointTs() + w.ResolvedTs = d.GetResolvedTs() + + //d.MemoryUsage.Clear() + d.componentStatus.Set(heartbeatpb.ComponentState_Stopped) + return w, true + } + return w, false +} + +func (d *Dispatcher) GetComponentStatus() heartbeatpb.ComponentState { + return d.componentStatus.Get() +} + +func (d *Dispatcher) GetFilter() filter.Filter { + return d.filter +} + +func (d *Dispatcher) GetWG() *sync.WaitGroup { + return &d.wg +} + +func (d *Dispatcher) GetDDLPendingEvent() *common.TxnEvent { + return d.ddlPendingEvent +} + +func (d *Dispatcher) SetDDLPendingEvent(event *common.TxnEvent) { + if d.ddlPendingEvent != nil { + log.Error("there is already a pending ddl event, can not set a new one") + return + } + d.ddlPendingEvent = event +} +func (d *Dispatcher) GetDDLFinishCh() chan struct{} { + return d.ddlFinishCh +} +func (d *Dispatcher) GetRemovingStatus() bool { + return d.isRemoving.Load() } -func HandleDDLActions(d Dispatcher, ctx context.Context) { +func (d *Dispatcher) HandleDDLActions(ctx context.Context) { defer d.GetWG().Done() sink := d.GetSink() tableSpan := d.GetTableSpan() @@ -145,7 +299,7 @@ func HandleDDLActions(d Dispatcher, ctx context.Context) { // 2.2 maintainer 通知自己可以 write 或者 pass event // // TODO:特殊处理有 add index 的逻辑 -func AddDDLEventToSinkWhenAvailable(d Dispatcher, event *common.TxnEvent) { +func (d *Dispatcher) AddDDLEventToSinkWhenAvailable(event *common.TxnEvent) { //filter := d.GetFilter() // TODO: filter 支持 // 判断 ddl 是否需要处理,如果不需要处理,直接返回 @@ -209,7 +363,7 @@ loop: <-d.GetDDLFinishCh() } -func CollectDispatcherHeartBeatInfo(d Dispatcher, h *HeartBeatInfo) { +func (d *Dispatcher) CollectDispatcherHeartBeatInfo(h *HeartBeatInfo) { // use checkpointTs to release memory usage //d.GetMemoryUsage().Release(checkpointTs) @@ -220,55 +374,3 @@ func CollectDispatcherHeartBeatInfo(d Dispatcher, h *HeartBeatInfo) { h.TableSpan = d.GetTableSpan() h.IsRemoving = d.GetRemovingStatus() } - -type SyncPointInfo struct { - EnableSyncPoint bool - SyncPointInterval time.Duration - NextSyncPointTs uint64 -} - -type ComponentStateWithMutex struct { - mutex sync.Mutex - componentStatus heartbeatpb.ComponentState -} - -func newComponentStateWithMutex(status heartbeatpb.ComponentState) *ComponentStateWithMutex { - return &ComponentStateWithMutex{ - componentStatus: status, - } -} - -func (s *ComponentStateWithMutex) Set(status heartbeatpb.ComponentState) { - s.mutex.Lock() - defer s.mutex.Unlock() - s.componentStatus = status -} - -func (s *ComponentStateWithMutex) Get() heartbeatpb.ComponentState { - s.mutex.Lock() - defer s.mutex.Unlock() - return s.componentStatus -} - -type TsWithMutex struct { - mutex sync.Mutex - ts uint64 -} - -func newTsWithMutex(ts uint64) *TsWithMutex { - return &TsWithMutex{ - ts: ts, - } -} - -func (r *TsWithMutex) Set(ts uint64) { - r.mutex.Lock() - defer r.mutex.Unlock() - r.ts = ts -} - -func (r *TsWithMutex) Get() uint64 { - r.mutex.Lock() - defer r.mutex.Unlock() - return r.ts -} diff --git a/downstreamadapter/dispatcher/helper.go b/downstreamadapter/dispatcher/helper.go index 9194e0d36..aa03ead28 100644 --- a/downstreamadapter/dispatcher/helper.go +++ b/downstreamadapter/dispatcher/helper.go @@ -14,555 +14,81 @@ package dispatcher import ( - "bytes" - "fmt" - "math" + "sync" "time" - "unsafe" + "github.com/flowbehappy/tigate/heartbeatpb" "github.com/flowbehappy/tigate/pkg/common" - "github.com/pingcap/errors" - "github.com/pingcap/log" - "github.com/pingcap/tidb/pkg/kv" - timodel "github.com/pingcap/tidb/pkg/parser/model" - "github.com/pingcap/tidb/pkg/parser/mysql" - "github.com/pingcap/tidb/pkg/table" - "github.com/pingcap/tidb/pkg/tablecodec" - "github.com/pingcap/tidb/pkg/types" - "github.com/pingcap/tidb/pkg/util/codec" - "github.com/pingcap/tidb/pkg/util/rowcodec" - "github.com/pingcap/tiflow/cdc/model" - cerror "github.com/pingcap/tiflow/pkg/errors" - "github.com/pingcap/tiflow/pkg/integrity" - "go.uber.org/zap" ) -var ( - tablePrefix = []byte{'t'} - recordPrefix = []byte("_r") - metaPrefix = []byte("m") -) - -var ( - intLen = 8 - tablePrefixLen = len(tablePrefix) - recordPrefixLen = len(recordPrefix) - metaPrefixLen = len(metaPrefix) - prefixTableIDLen = tablePrefixLen + intLen /*tableID*/ - prefixRecordIDLen = recordPrefixLen + intLen /*recordID*/ -) - -var emptyBytes = make([]byte, 0) - -const ( - sizeOfEmptyColumn = int(unsafe.Sizeof(model.Column{})) - sizeOfEmptyBytes = int(unsafe.Sizeof(emptyBytes)) - sizeOfEmptyString = int(unsafe.Sizeof("")) -) - -func decodeTableID(key []byte) (rest []byte, tableID int64, err error) { - if len(key) < prefixTableIDLen || !bytes.HasPrefix(key, tablePrefix) { - return nil, 0, cerror.ErrInvalidRecordKey.GenWithStackByArgs(key) - } - key = key[tablePrefixLen:] - rest, tableID, err = codec.DecodeInt(key) - if err != nil { - return nil, 0, cerror.WrapError(cerror.ErrCodecDecode, err) - } - return +type SyncPointInfo struct { + EnableSyncPoint bool + SyncPointInterval time.Duration + NextSyncPointTs uint64 } -type baseKVEntry struct { - StartTs uint64 - // Commit or resolved TS - CRTs uint64 - - PhysicalTableID int64 - RecordID kv.Handle - Delete bool -} - -type rowKVEntry struct { - baseKVEntry - Row map[int64]types.Datum - PreRow map[int64]types.Datum - - // In some cases, row data may exist but not contain any Datum, - // use this RowExist/PreRowExist variable to distinguish between row data that does not exist - // or row data that does not contain any Datum. - RowExist bool - PreRowExist bool +type ComponentStateWithMutex struct { + mutex sync.Mutex + componentStatus heartbeatpb.ComponentState } -func decodeRow( - rawValue []byte, recordID kv.Handle, tableInfo *common.TableInfo, isPreColumns bool, -) (map[int64]types.Datum, bool, error) { - if len(rawValue) == 0 { - return map[int64]types.Datum{}, false, nil - } - handleColIDs, handleColFt, reqCols := tableInfo.GetRowColInfos() - var ( - datums map[int64]types.Datum - err error - ) - - if rowcodec.IsNewFormat(rawValue) { - decoder := rowcodec.NewDatumMapDecoder(reqCols, time.Local) - datums, err = decodeRowV2(decoder, rawValue) - } else { - datums, err = decodeRowV1(rawValue, tableInfo, time.Local) - } - - if err != nil { - return nil, false, errors.Trace(err) +func newComponentStateWithMutex(status heartbeatpb.ComponentState) *ComponentStateWithMutex { + return &ComponentStateWithMutex{ + componentStatus: status, } - - datums, err = tablecodec.DecodeHandleToDatumMap( - recordID, handleColIDs, handleColFt, time.Local, datums) - if err != nil { - return nil, false, errors.Trace(err) - } - - return datums, true, nil } -func unmarshalRowKVEntry( - tableInfo *common.TableInfo, - rawKey []byte, - rawValue []byte, - rawOldValue []byte, - base baseKVEntry, -) (*rowKVEntry, error) { - recordID, err := tablecodec.DecodeRowKey(rawKey) - if err != nil { - return nil, errors.Trace(err) - } - base.RecordID = recordID - - var ( - row, preRow map[int64]types.Datum - rowExist, preRowExist bool - ) - - row, rowExist, err = decodeRow(rawValue, recordID, tableInfo, false) - if err != nil { - return nil, errors.Trace(err) - } - - preRow, preRowExist, err = decodeRow(rawOldValue, recordID, tableInfo, true) - if err != nil { - return nil, errors.Trace(err) - } - - return &rowKVEntry{ - baseKVEntry: base, - Row: row, - PreRow: preRow, - RowExist: rowExist, - PreRowExist: preRowExist, - }, nil +func (s *ComponentStateWithMutex) Set(status heartbeatpb.ComponentState) { + s.mutex.Lock() + defer s.mutex.Unlock() + s.componentStatus = status } -func mountRowKVEntry(tableInfo *common.TableInfo, row *rowKVEntry) (*model.RowChangedEvent, model.RowChangedDatums, error) { - var ( - rawRow model.RowChangedDatums - //columnInfos []*timodel.ColumnInfo - err error - - checksum *integrity.Checksum - - checksumVersion int - corrupted bool - ) - - // Decode previous columns. - var ( - preCols []*model.ColumnData - preRawCols []types.Datum - preChecksum uint32 - ) - if row.PreRowExist { - // FIXME(leoppro): using pre table info to mounter pre column datum - // the pre column and current column in one event may using different table info - preCols, preRawCols, _, err = datum2Column(tableInfo, row.PreRow, time.Local) - if err != nil { - return nil, rawRow, errors.Trace(err) - } - } - - var ( - cols []*model.ColumnData - rawCols []types.Datum - currentChecksum uint32 - ) - if row.RowExist { - cols, rawCols, _, err = datum2Column(tableInfo, row.Row, time.Local) - if err != nil { - return nil, rawRow, errors.Trace(err) - } - } - - var intRowID int64 - if row.RecordID.IsInt() { - intRowID = row.RecordID.IntValue() - } - - rawRow.PreRowDatums = preRawCols - rawRow.RowDatums = rawCols - - // if both are 0, it means the checksum is not enabled - // so the checksum is nil to reduce memory allocation. - if preChecksum != 0 || currentChecksum != 0 { - checksum = &integrity.Checksum{ - Current: currentChecksum, - Previous: preChecksum, - Corrupted: corrupted, - Version: checksumVersion, - } - } - - return &model.RowChangedEvent{ - StartTs: row.StartTs, - CommitTs: row.CRTs, - RowID: intRowID, - HandleKey: row.RecordID, - PhysicalTableID: row.PhysicalTableID, - Columns: cols, - PreColumns: preCols, - - Checksum: checksum, - }, rawRow, nil -} - -func datum2Column( - tableInfo *common.TableInfo, datums map[int64]types.Datum, tz *time.Location, -) ([]*model.ColumnData, []types.Datum, []*timodel.ColumnInfo, error) { - cols := make([]*model.ColumnData, len(tableInfo.RowColumnsOffset)) - rawCols := make([]types.Datum, len(tableInfo.RowColumnsOffset)) - - // columnInfos should have the same length and order with cols - columnInfos := make([]*timodel.ColumnInfo, len(tableInfo.RowColumnsOffset)) - - for _, colInfo := range tableInfo.Columns { - if !model.IsColCDCVisible(colInfo) { - log.Debug("skip the column which is not visible", - zap.String("table", tableInfo.Name.O), zap.String("column", colInfo.Name.O)) - continue - } - - colID := colInfo.ID - colDatum, exist := datums[colID] - - var ( - colValue interface{} - size int - warn string - err error - ) - if exist { - colValue, size, warn, err = formatColVal(colDatum, colInfo) - } else { - colDatum, colValue, size, warn, err = getDefaultOrZeroValue(colInfo, tz) - } - if err != nil { - return nil, nil, nil, errors.Trace(err) - } - if warn != "" { - log.Warn(warn, zap.String("table", tableInfo.TableName.String()), - zap.String("column", colInfo.Name.String())) - } - - offset := tableInfo.RowColumnsOffset[colID] - rawCols[offset] = colDatum - cols[offset] = &model.ColumnData{ - ColumnID: colID, - Value: colValue, - // ApproximateBytes = column data size + column struct size - ApproximateBytes: size + sizeOfEmptyColumn, - } - columnInfos[offset] = colInfo - } - return cols, rawCols, columnInfos, nil -} - -func sizeOfString(s string) int { - // string data size + string struct size. - return len(s) + sizeOfEmptyString -} - -func sizeOfBytes(b []byte) int { - // bytes data size + bytes struct size. - return len(b) + sizeOfEmptyBytes +func (s *ComponentStateWithMutex) Get() heartbeatpb.ComponentState { + s.mutex.Lock() + defer s.mutex.Unlock() + return s.componentStatus } -func sizeOfDatum(d types.Datum) int { - array := [...]types.Datum{d} - return int(types.EstimatedMemUsage(array[:], 1)) +type TsWithMutex struct { + mutex sync.Mutex + ts uint64 } -// formatColVal return interface{} need to meet the same requirement as getDefaultOrZeroValue -func formatColVal(datum types.Datum, col *timodel.ColumnInfo) ( - value interface{}, size int, warn string, err error, -) { - if datum.IsNull() { - return nil, 0, "", nil - } - switch col.GetType() { - case mysql.TypeDate, mysql.TypeDatetime, mysql.TypeNewDate, mysql.TypeTimestamp: - v := datum.GetMysqlTime().String() - return v, sizeOfString(v), "", nil - case mysql.TypeDuration: - v := datum.GetMysqlDuration().String() - return v, sizeOfString(v), "", nil - case mysql.TypeJSON: - v := datum.GetMysqlJSON().String() - return v, sizeOfString(v), "", nil - case mysql.TypeNewDecimal: - d := datum.GetMysqlDecimal() - if d == nil { - // nil takes 0 byte. - return nil, 0, "", nil - } - v := d.String() - return v, sizeOfString(v), "", nil - case mysql.TypeEnum: - v := datum.GetMysqlEnum().Value - const sizeOfV = unsafe.Sizeof(v) - return v, int(sizeOfV), "", nil - case mysql.TypeSet: - v := datum.GetMysqlSet().Value - const sizeOfV = unsafe.Sizeof(v) - return v, int(sizeOfV), "", nil - case mysql.TypeBit: - // Encode bits as integers to avoid pingcap/tidb#10988 (which also affects MySQL itself) - v, err := datum.GetBinaryLiteral().ToInt(types.DefaultStmtNoWarningContext) - const sizeOfV = unsafe.Sizeof(v) - return v, int(sizeOfV), "", err - case mysql.TypeString, mysql.TypeVarString, mysql.TypeVarchar, - mysql.TypeTinyBlob, mysql.TypeMediumBlob, mysql.TypeLongBlob, mysql.TypeBlob: - b := datum.GetBytes() - if b == nil { - b = emptyBytes - } - return b, sizeOfBytes(b), "", nil - case mysql.TypeFloat: - v := datum.GetFloat32() - if math.IsNaN(float64(v)) || math.IsInf(float64(v), 1) || math.IsInf(float64(v), -1) { - warn = fmt.Sprintf("the value is invalid in column: %f", v) - v = 0 - } - const sizeOfV = unsafe.Sizeof(v) - return v, int(sizeOfV), warn, nil - case mysql.TypeDouble: - v := datum.GetFloat64() - if math.IsNaN(v) || math.IsInf(v, 1) || math.IsInf(v, -1) { - warn = fmt.Sprintf("the value is invalid in column: %f", v) - v = 0 - } - const sizeOfV = unsafe.Sizeof(v) - return v, int(sizeOfV), warn, nil - default: - // NOTICE: GetValue() may return some types that go sql not support, which will cause sink DML fail - // Make specified convert upper if you need - // Go sql support type ref to: https://github.com/golang/go/blob/go1.17.4/src/database/sql/driver/types.go#L236 - return datum.GetValue(), sizeOfDatum(datum), "", nil +func newTsWithMutex(ts uint64) *TsWithMutex { + return &TsWithMutex{ + ts: ts, } } -// Scenarios when call this function: -// (1) column define default null at creating + insert without explicit column -// (2) alter table add column default xxx + old existing data -// (3) amend + insert without explicit column + alter table add column default xxx -// (4) online DDL drop column + data insert at state delete-only -// -// getDefaultOrZeroValue return interface{} need to meet to require type in -// https://github.com/golang/go/blob/go1.17.4/src/database/sql/driver/types.go#L236 -// Supported type is: nil, basic type(Int, Int8,..., Float32, Float64, String), Slice(uint8), other types not support -// TODO: Check default expr support -func getDefaultOrZeroValue( - col *timodel.ColumnInfo, tz *time.Location, -) (types.Datum, any, int, string, error) { - var ( - d types.Datum - err error - ) - // NOTICE: SHOULD use OriginDefaultValue here, more info pls ref to - // https://github.com/pingcap/tiflow/issues/4048 - // FIXME: Too many corner cases may hit here, like type truncate, timezone - // (1) If this column is uk(no pk), will cause data inconsistency in Scenarios(2) - // (2) If not fix here, will cause data inconsistency in Scenarios(3) directly - // Ref: https://github.com/pingcap/tidb/blob/d2c352980a43bb593db81fd1db996f47af596d91/table/column.go#L489 - if col.GetOriginDefaultValue() != nil { - datum := types.NewDatum(col.GetOriginDefaultValue()) - d, err = datum.ConvertTo(types.DefaultStmtNoWarningContext, &col.FieldType) - if err != nil { - return d, d.GetValue(), sizeOfDatum(d), "", errors.Trace(err) - } - switch col.GetType() { - case mysql.TypeTimestamp: - t := d.GetMysqlTime() - err = t.ConvertTimeZone(time.UTC, tz) - if err != nil { - return d, d.GetValue(), sizeOfDatum(d), "", errors.Trace(err) - } - d.SetMysqlTime(t) - } - } else if !mysql.HasNotNullFlag(col.GetFlag()) { - // NOTICE: NotNullCheck need do after OriginDefaultValue check, as when TiDB meet "amend + add column default xxx", - // ref: https://github.com/pingcap/ticdc/issues/3929 - // must use null if TiDB not write the column value when default value is null - // and the value is null, see https://github.com/pingcap/tidb/issues/9304 - d = types.NewDatum(nil) - } else { - switch col.GetType() { - case mysql.TypeEnum: - // For enum type, if no default value and not null is set, - // the default value is the first element of the enum list - name := col.FieldType.GetElem(0) - enumValue, err := types.ParseEnumName(col.FieldType.GetElems(), name, col.GetCollate()) - if err != nil { - return d, nil, 0, "", errors.Trace(err) - } - d = types.NewMysqlEnumDatum(enumValue) - case mysql.TypeString, mysql.TypeVarString, mysql.TypeVarchar: - return d, emptyBytes, sizeOfEmptyBytes, "", nil - default: - d = table.GetZeroValue(col) - if d.IsNull() { - log.Error("meet unsupported column type", zap.String("columnInfo", col.FieldType.String())) - } - } - } - v, size, warn, err := formatColVal(d, col) - return d, v, size, warn, err +func (r *TsWithMutex) Set(ts uint64) { + r.mutex.Lock() + defer r.mutex.Unlock() + r.ts = ts } -// GetDDLDefaultDefinition returns the default definition of a column. -func GetDDLDefaultDefinition(col *timodel.ColumnInfo) interface{} { - defaultValue := col.GetDefaultValue() - if defaultValue == nil { - defaultValue = col.GetOriginDefaultValue() - } - defaultDatum := types.NewDatum(defaultValue) - return defaultDatum.GetValue() +func (r *TsWithMutex) Get() uint64 { + r.mutex.Lock() + defer r.mutex.Unlock() + return r.ts } -// decodeRowV1 decodes value data using old encoding format. -// Row layout: colID1, value1, colID2, value2, ..... -func decodeRowV1(b []byte, tableInfo *common.TableInfo, tz *time.Location) (map[int64]types.Datum, error) { - row := make(map[int64]types.Datum) - if len(b) == 1 && b[0] == codec.NilFlag { - b = b[1:] - } - var err error - var data []byte - for len(b) > 0 { - // Get col id. - data, b, err = codec.CutOne(b) - if err != nil { - return nil, cerror.WrapError(cerror.ErrCodecDecode, err) - } - _, cid, err := codec.DecodeOne(data) - if err != nil { - return nil, cerror.WrapError(cerror.ErrCodecDecode, err) - } - id := cid.GetInt64() - - // Get col value. - data, b, err = codec.CutOne(b) - if err != nil { - return nil, cerror.WrapError(cerror.ErrCodecDecode, err) - } - _, v, err := codec.DecodeOne(data) - if err != nil { - return nil, cerror.WrapError(cerror.ErrCodecDecode, err) - } +type DispatcherType uint64 - // unflatten value - colInfo, exist := tableInfo.GetColumnInfo(id) - if !exist { - // can not find column info, ignore this column because the column should be in WRITE ONLY state - continue - } - fieldType := &colInfo.FieldType - datum, err := unflatten(v, fieldType, tz) - if err != nil { - return nil, cerror.WrapError(cerror.ErrCodecDecode, err) - } - row[id] = datum - } - return row, nil -} - -// decodeRowV2 decodes value data using new encoding format. -// Ref: https://github.com/pingcap/tidb/pull/12634 -// -// https://github.com/pingcap/tidb/blob/master/docs/design/2018-07-19-row-format.md -func decodeRowV2( - decoder *rowcodec.DatumMapDecoder, data []byte, -) (map[int64]types.Datum, error) { - datums, err := decoder.DecodeToDatumMap(data, nil) - if err != nil { - return datums, cerror.WrapError(cerror.ErrDecodeRowToDatum, err) - } - return datums, nil -} +const ( + TableEventDispatcherType DispatcherType = 0 + TableTriggerEventDispatcherType DispatcherType = 1 +) -// unflatten converts a raw datum to a column datum. -func unflatten(datum types.Datum, ft *types.FieldType, loc *time.Location) (types.Datum, error) { - if datum.IsNull() { - return datum, nil - } - switch ft.GetType() { - case mysql.TypeFloat: - datum.SetFloat32(float32(datum.GetFloat64())) - return datum, nil - case mysql.TypeVarchar, mysql.TypeString, mysql.TypeVarString, mysql.TypeTinyBlob, - mysql.TypeMediumBlob, mysql.TypeBlob, mysql.TypeLongBlob: - datum.SetString(datum.GetString(), ft.GetCollate()) - case mysql.TypeTiny, mysql.TypeShort, mysql.TypeYear, mysql.TypeInt24, - mysql.TypeLong, mysql.TypeLonglong, mysql.TypeDouble: - return datum, nil - case mysql.TypeDate, mysql.TypeDatetime, mysql.TypeTimestamp: - t := types.NewTime(types.ZeroCoreTime, ft.GetType(), ft.GetDecimal()) - var err error - err = t.FromPackedUint(datum.GetUint64()) - if err != nil { - return datum, cerror.WrapError(cerror.ErrDatumUnflatten, err) - } - if ft.GetType() == mysql.TypeTimestamp && !t.IsZero() { - err = t.ConvertTimeZone(time.UTC, loc) - if err != nil { - return datum, cerror.WrapError(cerror.ErrDatumUnflatten, err) - } - } - datum.SetUint64(0) - datum.SetMysqlTime(t) - return datum, nil - case mysql.TypeDuration: // duration should read fsp from column meta data - dur := types.Duration{Duration: time.Duration(datum.GetInt64()), Fsp: ft.GetDecimal()} - datum.SetMysqlDuration(dur) - return datum, nil - case mysql.TypeEnum: - // ignore error deliberately, to read empty enum value. - enum, err := types.ParseEnumValue(ft.GetElems(), datum.GetUint64()) - if err != nil { - enum = types.Enum{} - } - datum.SetMysqlEnum(enum, ft.GetCollate()) - return datum, nil - case mysql.TypeSet: - set, err := types.ParseSetValue(ft.GetElems(), datum.GetUint64()) - if err != nil { - return datum, cerror.WrapError(cerror.ErrDatumUnflatten, err) - } - datum.SetMysqlSet(set, ft.GetCollate()) - return datum, nil - case mysql.TypeBit: - val := datum.GetUint64() - byteSize := (ft.GetFlen() + 7) >> 3 - datum.SetUint64(0) - datum.SetMysqlBit(types.NewBinaryLiteralFromUint(val, byteSize)) - } - return datum, nil +/* +HeartBeatInfo is used to collect the message for HeartBeatRequest for each dispatcher. +Mainly about the progress of each dispatcher: +1. The checkpointTs of the dispatcher, shows that all the events whose ts <= checkpointTs are flushed to downstream successfully. +*/ +type HeartBeatInfo struct { + heartbeatpb.Watermark + Id string + TableSpan *common.TableSpan + ComponentStatus heartbeatpb.ComponentState + IsRemoving bool } diff --git a/downstreamadapter/dispatcher/table_event_dispatcher.go b/downstreamadapter/dispatcher/table_event_dispatcher.go deleted file mode 100644 index 6f41fccf0..000000000 --- a/downstreamadapter/dispatcher/table_event_dispatcher.go +++ /dev/null @@ -1,232 +0,0 @@ -// Copyright 2024 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 dispatcher - -import ( - "context" - "sync" - "sync/atomic" - - "github.com/flowbehappy/tigate/downstreamadapter/sink" - "github.com/flowbehappy/tigate/heartbeatpb" - "github.com/flowbehappy/tigate/pkg/common" - "github.com/google/uuid" - "github.com/pingcap/log" - "github.com/pingcap/tiflow/pkg/filter" - "go.uber.org/zap" -) - -/* -TableEventDispatcher is dispatcher the event of a normal tableSpan in a changefeed. -It is responsible for getting the events about the tableSpan from the Logservice and sending them to the Sink in an appropriate order. - -It communicates with the Maintainer periodically to report self progress, -and get the other dispatcher's progress and action of the blocked event. - -Each EventDispatcherManager can have multiple TableEventDispatcher. -*/ -type TableEventDispatcher struct { - id string - eventCh chan *common.TxnEvent // 转换成一个函数 - tableSpan *common.TableSpan - sink sink.Sink - - ddlActions chan *heartbeatpb.DispatcherAction - acks chan *heartbeatpb.ACK - tableSpanStatusesChan chan *heartbeatpb.TableSpanStatus - - //SyncPointInfo *SyncPointInfo - - //MemoryUsage *MemoryUsage - - componentStatus *ComponentStateWithMutex - - filter filter.Filter - - resolvedTs *TsWithMutex // 用来记 eventChan 中目前收到的 event 中收到的最大的 commitTs - 1,不代表 dispatcher 的 checkpointTs - - cancel context.CancelFunc - wg sync.WaitGroup - - ddlPendingEvent *common.TxnEvent - ddlFinishCh chan struct{} - isRemoving atomic.Bool -} - -func NewTableEventDispatcher(tableSpan *common.TableSpan, sink sink.Sink, startTs uint64, syncPointInfo *SyncPointInfo, tableSpanStatusesChan chan *heartbeatpb.TableSpanStatus, filter filter.Filter) *TableEventDispatcher { - ctx, cancel := context.WithCancel(context.Background()) - tableEventDispatcher := &TableEventDispatcher{ - id: uuid.NewString(), - eventCh: make(chan *common.TxnEvent, 16), - tableSpan: tableSpan, - sink: sink, - ddlActions: make(chan *heartbeatpb.DispatcherAction, 16), - acks: make(chan *heartbeatpb.ACK, 16), - tableSpanStatusesChan: tableSpanStatusesChan, - //SyncPointInfo: syncPointInfo, - //MemoryUsage: NewMemoryUsage(), - componentStatus: newComponentStateWithMutex(heartbeatpb.ComponentState_Working), - resolvedTs: newTsWithMutex(startTs), - cancel: cancel, - filter: filter, - ddlFinishCh: make(chan struct{}), - isRemoving: atomic.Bool{}, - } - - tableEventDispatcher.sink.AddTableSpan(tableSpan) - tableEventDispatcher.wg.Add(1) - go tableEventDispatcher.DispatcherEvents(ctx) - - tableEventDispatcher.wg.Add(1) - go HandleDDLActions(tableEventDispatcher, ctx) - - log.Info("table event dispatcher created", zap.Any("DispatcherID", tableEventDispatcher.id)) - - return tableEventDispatcher -} - -func (d *TableEventDispatcher) DispatcherEvents(ctx context.Context) { - defer d.wg.Done() - tableSpan := d.GetTableSpan() - sink := d.GetSink() - for { - select { - case <-ctx.Done(): - return - case event := <-d.GetEventChan(): - if event.IsDMLEvent() { - sink.AddDMLEvent(tableSpan, event) - } else if event.IsDDLEvent() { - AddDDLEventToSinkWhenAvailable(d, event) - } else { - d.resolvedTs.Set(event.ResolvedTs) - } - } - } -} - -func (d *TableEventDispatcher) GetSink() sink.Sink { - return d.sink -} - -func (d *TableEventDispatcher) GetTableSpan() *common.TableSpan { - return d.tableSpan -} - -func (d *TableEventDispatcher) GetEventChan() chan *common.TxnEvent { - return d.eventCh -} - -func (d *TableEventDispatcher) GetResolvedTs() uint64 { - return d.resolvedTs.Get() -} - -func (d *TableEventDispatcher) GetCheckpointTs() uint64 { - checkpointTs := d.GetSink().GetCheckpointTs(d.GetTableSpan()) - if checkpointTs == 0 { - // 说明从没有数据写到过 sink,则选择用 resolveTs 作为 checkpointTs - checkpointTs = d.GetResolvedTs() - } - return checkpointTs -} - -func (d *TableEventDispatcher) UpdateResolvedTs(ts uint64) { - d.GetEventChan() <- &common.TxnEvent{ResolvedTs: ts} -} - -func (d *TableEventDispatcher) GetId() string { - return d.id -} - -func (d *TableEventDispatcher) GetDispatcherType() DispatcherType { - return TableEventDispatcherType -} - -func (d *TableEventDispatcher) GetDDLActions() chan *heartbeatpb.DispatcherAction { - return d.ddlActions -} - -func (d *TableEventDispatcher) GetACKs() chan *heartbeatpb.ACK { - return d.acks -} - -func (d *TableEventDispatcher) GetTableSpanStatusesChan() chan *heartbeatpb.TableSpanStatus { - return d.tableSpanStatusesChan -} - -//func (d *TableEventDispatcher) GetSyncPointInfo() *SyncPointInfo { -// return d.syncPointInfo -// } - -// func (d *TableEventDispatcher) GetMemoryUsage() *MemoryUsage { -// return d.MemoryUsage -// } - -func (d *TableEventDispatcher) PushTxnEvent(event *common.TxnEvent) { - d.GetEventChan() <- event -} - -func (d *TableEventDispatcher) Remove() { - // TODO: 修改这个 dispatcher 的 status 为 removing - d.cancel() - d.sink.StopTableSpan(d.tableSpan) - log.Info("table event dispatcher component status changed to stopping", zap.String("table", d.tableSpan.String())) - d.isRemoving.Store(true) -} - -func (d *TableEventDispatcher) TryClose() (w heartbeatpb.Watermark, ok bool) { - // removing 后每次收集心跳的时候,call TryClose, 来判断是否能关掉 dispatcher 了(sink.isEmpty) - // 如果不能关掉,返回 0, false; 可以关掉的话,就返回 checkpointTs, true -- 这个要对齐过(startTs 和 checkpointTs 的关系) - if d.sink.IsEmpty(d.tableSpan) { - // calculate the checkpointTs, and clean the resource - d.sink.RemoveTableSpan(d.tableSpan) - w.CheckpointTs = d.GetCheckpointTs() - w.ResolvedTs = d.GetResolvedTs() - - //d.MemoryUsage.Clear() - d.componentStatus.Set(heartbeatpb.ComponentState_Stopped) - return w, true - } - return w, false -} - -func (d *TableEventDispatcher) GetComponentStatus() heartbeatpb.ComponentState { - return d.componentStatus.Get() -} - -func (d *TableEventDispatcher) GetFilter() filter.Filter { - return d.filter -} - -func (d *TableEventDispatcher) GetWG() *sync.WaitGroup { - return &d.wg -} - -func (d *TableEventDispatcher) GetDDLPendingEvent() *common.TxnEvent { - return d.ddlPendingEvent -} - -func (d *TableEventDispatcher) SetDDLPendingEvent(event *common.TxnEvent) { - if d.ddlPendingEvent != nil { - log.Error("there is already a pending ddl event, can not set a new one") - return - } - d.ddlPendingEvent = event -} -func (d *TableEventDispatcher) GetDDLFinishCh() chan struct{} { - return d.ddlFinishCh -} -func (d *TableEventDispatcher) GetRemovingStatus() bool { - return d.isRemoving.Load() -} diff --git a/downstreamadapter/dispatcher/table_trigger_event_dispatcher.go b/downstreamadapter/dispatcher/table_trigger_event_dispatcher.go deleted file mode 100644 index 432496343..000000000 --- a/downstreamadapter/dispatcher/table_trigger_event_dispatcher.go +++ /dev/null @@ -1,276 +0,0 @@ -// Copyright 2024 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 dispatcher - -import ( - "context" - "sync" - "sync/atomic" - - "github.com/flowbehappy/tigate/downstreamadapter/sink" - "github.com/flowbehappy/tigate/heartbeatpb" - "github.com/flowbehappy/tigate/pkg/common" - "github.com/google/uuid" - "github.com/pingcap/log" - "github.com/pingcap/tiflow/pkg/filter" - "go.uber.org/zap" -) - -//filter 问题 -- 能收到这条就至少说明有相关的 table(比如 renames / create tables / exchange partitions -- 这个应该不支持一个在一个不在的),对于跟 table 有关的表来说,那就前面两种就可以在 ddl 生成的时候用 config 处理了 -// 这个要让 logService 传过来 新增 table span 删除 tableSpan 的问题 -/* double check 一下类型吧 -ActionCreateSchema -- 只需要执行下游语句 -ActionDropSchema -- 只需要执行下游语句 -ActionCreateTable -- maintainer 通知 table trigger event dispatcher 执行下游语句,成功后创建 dispatcher -ActionDropTable -- maintainer 通知 table trigger event dispatcher 执行下游语句,成功后删除 dispatcher -ActionTruncateTable -- maintainer 通知 table trigger event dispatcher 执行下游语句,成功后删除老 dispatcher,创建新 dispatcher -ActionRenameTable -- 先在下游执行这个语句,然后删掉老的 dispatcher,创建新的 dispatcher -ActionAddTablePartition -- 先执行这个语句,然后创建新的 dispatcher -ActionDropTablePartition -- 先执行这个语句,然后删除老的 dispatcher -ActionTruncateTablePartition -- 先执行这个语句,然后删除老的 dispatcher,创建新的 dispatcher -ActionRecoverTable -- 先执行这个语句,然后创建新的 dispatcher -ActionRepairTable -- 只需要执行 -ActionExchangeTablePartition -- 先执行这个语句,这个理论上 table id 变了,其他应该没怎么变,可以先考虑删了老 dispatcher 然后创建新的,后面也可以考虑要不要变成更轻量的修改。 -ActionRemovePartitioning -- 先执行这个语句,然后删除老的 dispatcher -ActionRenameTables -- rename table 的复数版 -ActionCreateTables -- create table 的复数版 -ActionReorganizePartition -- 先执行,然后该删删该加加 -ActionFlashbackCluster -- 执行语句,只对 tidb 有效果 -ActionCreateResourceGroup/ActionAlterResourceGroup/ActionDropResourceGroup 执行语句,且只对 tidb -*/ - -/* -TableTriggerEventDispatcher 需要接收所有跟创建表或者删表相关的 ddl,其中每一条 DDL 的推进,都要跟 maintainer 沟通,确认是否需要自己执行( pass / write),所以可以理解为同一个 changefeed 的多个节点的 tableTriggerEventDispatcher 进度至少是基本同步的。如果是 rename 这种 ddl,就还要等对应表的 checkpointTs 推到。 - - - - - -假设我有两个节点A,B, table C 一开始在 A 上同步,然后下一条 event 是 checkpointTs d 的 rename 操作,然后 A 和 B 的 tableTriggerEventTable 的 checkpointTs 也推到 t-1 了,现在大家都跟 maintainer 通信说了自己推到了 d-1,然后 maintainer 会通知某个tableTrigger 执行这条 ddl, 通知另一个节点 B 跳过这条 ddl,通知 这个表跳过这条 ddl(执行的先通知,跳过的是执行完才通知的)。 -如果这个时候这个表被迁移了,没收到那个跳过的通知,所以 maintainer 需要再次通知他跳过这条 ddl。 -- 这个是maintainer 需要做的事情。也就是要求maintainer 要至少知道 ddl 的推进进度,保证 skip 可以重发。 -那如果通知执行这条 ddl 后,却没有收到推进的消息,则应该选择同节点重发,不能换节点。 -对于 table trigger event table 的 ddl,我们需要严格按照顺序执行,满足前一条没有执行成功时,后一条不能开始执行的要求。skip 和下一条的执行是可以一起通知的。所有的 ddl 信息在收到以后快速等10ms 或者其他时间以后就按照心跳发送给 maintainer。如果没有什么 ddl,这个就跟着大家定期汇报进度,有 ddl 到了或者在排队等待的时候,就应该更高频?比如 20ms 这样可以发50个来回?每次最多发两个 ddl event 给上面。 - - - -在哪里生成那些没有 query 的 ddl query -- 单表的没有问题,多表的话也可以先都生成,我自己来做 filter - -add index 的问题 -- 这个异步去做,并且更改现在的状态为有 ddl 执行中,没执行完后续的 ddl 不能推进,dml 可以先正常推进知道卡到下一个 ddl 。 - - -所以本质来说 tableTriggerEvent 持续接收 ddl,然后跟maintainer 沟通决定是否能推进。 - - -*/ - -/* -TableTriggerEventDispatcher implements the Dispatcher interface. - -TableTriggerEventDispatcher is a speical dispatcher. - -It is responsible for getting the ddl events from the Logservice and sending them to the Sink in an appropriate order. -It only pay attention to the speical ddl events, which will leads to new table or remove table, -such as Create Table, Drop Table, Rename Table, Exchange Table Partition, etc. - -In each EventDispatcherManager, there is only one TableTriggerEventDispatcher, -and it also the first dispatcher in the EventDispatcherManager. - -It also communicates with the Maintainer periodically to report self progress, -and get the other dispatcher's progress and action of the blocked event. -*/ -type TableTriggerEventDispatcher struct { - id string - eventCh chan *common.TxnEvent - filter filter.Filter - sink sink.Sink - ddlActions chan *heartbeatpb.DispatcherAction - acks chan *heartbeatpb.ACK - tableSpan *common.TableSpan // 给一个特殊的 tableSpan - resolvedTs *TsWithMutex - componentStatus *ComponentStateWithMutex - - tableSpanStatusesChan chan *heartbeatpb.TableSpanStatus - - wg sync.WaitGroup - cancel context.CancelFunc - // ddl 相关的可以考虑塞进一个 struct 去 - ddlPendingEvent *common.TxnEvent - ddlFinishCh chan struct{} - //MemoryUsage *MemoryUsage - MemoryUsage *MemoryUsage - - IsRemoving atomic.Bool -} - -func NewTableTriggerEventDispatcher(sink sink.Sink, startTs uint64, tableSpanStatusesChan chan *heartbeatpb.TableSpanStatus, filter filter.Filter) *TableTriggerEventDispatcher { - ctx, cancel := context.WithCancel(context.Background()) - tableTriggerEventDispatcher := &TableTriggerEventDispatcher{ - id: uuid.NewString(), - filter: filter, - eventCh: make(chan *common.TxnEvent, 1000), - resolvedTs: newTsWithMutex(startTs), - ddlActions: make(chan *heartbeatpb.DispatcherAction, 16), - acks: make(chan *heartbeatpb.ACK, 16), - tableSpanStatusesChan: tableSpanStatusesChan, - sink: sink, - tableSpan: &common.DDLSpan, - componentStatus: newComponentStateWithMutex(heartbeatpb.ComponentState_Working), - cancel: cancel, - ddlFinishCh: make(chan struct{}), - //MemoryUsage: dispatcher.NewMemoryUsage(), - } - tableTriggerEventDispatcher.sink.AddTableSpan(tableTriggerEventDispatcher.tableSpan) - - tableTriggerEventDispatcher.wg.Add(1) - go tableTriggerEventDispatcher.DispatcherEvents(ctx) - - tableTriggerEventDispatcher.wg.Add(1) - go HandleDDLActions(tableTriggerEventDispatcher, ctx) - - log.Info("table trigger event dispatcher created", zap.Any("DispatcherID", tableTriggerEventDispatcher.id)) - - return tableTriggerEventDispatcher -} - -func (d *TableTriggerEventDispatcher) DispatcherEvents(ctx context.Context) { - defer d.wg.Done() - for { - select { - case <-ctx.Done(): - return - case event := <-d.GetEventChan(): - if event.IsDDLEvent() { - AddDDLEventToSinkWhenAvailable(d, event) - } else { - d.resolvedTs.Set(event.ResolvedTs) - } - } - } -} - -func (d *TableTriggerEventDispatcher) GetSink() sink.Sink { - return d.sink -} - -func (d *TableTriggerEventDispatcher) GetTableSpan() *common.TableSpan { - return d.tableSpan -} - -func (d *TableTriggerEventDispatcher) GetEventChan() chan *common.TxnEvent { - return d.eventCh -} - -func (d *TableTriggerEventDispatcher) GetResolvedTs() uint64 { - return d.resolvedTs.Get() -} - -func (d *TableTriggerEventDispatcher) GetId() string { - return d.id -} - -func (d *TableTriggerEventDispatcher) GetDispatcherType() DispatcherType { - return TableTriggerEventDispatcherType -} - -func (d *TableTriggerEventDispatcher) GetDDLActions() chan *heartbeatpb.DispatcherAction { - return d.ddlActions -} - -func (d *TableTriggerEventDispatcher) GetACKs() chan *heartbeatpb.ACK { - return d.acks -} - -func (d *TableTriggerEventDispatcher) GetTableSpanStatusesChan() chan *heartbeatpb.TableSpanStatus { - return d.tableSpanStatusesChan -} - -func (d *TableTriggerEventDispatcher) UpdateResolvedTs(ts uint64) { - d.GetEventChan() <- &common.TxnEvent{ResolvedTs: ts} -} - -// func (d *TableTriggerEventDispatcher) GetSyncPointInfo() *SyncPointInfo { -// log.Error("TableEventDispatcher.GetSyncPointInfo is not implemented") -// return nil -// } - -// func (d *TableTriggerEventDispatcher) GetMemoryUsage() *MemoryUsage { -// return d.MemoryUsage -// } - -func (d *TableTriggerEventDispatcher) PushTxnEvent(event *common.TxnEvent) { - //d.GetMemoryUsage().Add(event.CommitTs, event.MemoryCost()) - d.GetEventChan() <- event // 换成一个函数 -} - -func (d *TableTriggerEventDispatcher) GetCheckpointTs() uint64 { - checkpointTs := d.GetSink().GetCheckpointTs(d.GetTableSpan()) - if checkpointTs == 0 { - // 说明从没有数据写到过 sink,则选择用 resolveTs 作为 checkpointTs - checkpointTs = d.GetResolvedTs() - } - return checkpointTs -} - -func (d *TableTriggerEventDispatcher) GetComponentStatus() heartbeatpb.ComponentState { - return heartbeatpb.ComponentState_Working -} - -// TryClose try to close the tableTriggerEventDispatcher, -// It should first check whether the related events in sink is finished. -// If yes, then return checkpointTs, true, else return 0, false. -func (d *TableTriggerEventDispatcher) TryClose() (w heartbeatpb.Watermark, ok bool) { - if d.sink.IsEmpty(d.tableSpan) { - d.sink.RemoveTableSpan(d.tableSpan) - w.CheckpointTs = w.GetCheckpointTs() - w.ResolvedTs = d.GetResolvedTs() - return w, true - } - return w, false -} - -func (d *TableTriggerEventDispatcher) GetFilter() filter.Filter { - return d.filter -} - -func (d *TableTriggerEventDispatcher) GetWG() *sync.WaitGroup { - return &d.wg -} - -func (d *TableTriggerEventDispatcher) GetDDLPendingEvent() *common.TxnEvent { - return d.ddlPendingEvent -} - -func (d *TableTriggerEventDispatcher) SetDDLPendingEvent(event *common.TxnEvent) { - if d.ddlPendingEvent != nil { - log.Error("there is already a pending ddl event, can not set a new one") - return - } - d.ddlPendingEvent = event -} - -func (d *TableTriggerEventDispatcher) GetDDLFinishCh() chan struct{} { - return d.ddlFinishCh -} - -func (d *TableTriggerEventDispatcher) Remove() { - // TODO: 修改这个 dispatcher 的 status 为 removing - d.cancel() - d.sink.StopTableSpan(d.tableSpan) - log.Info("table event dispatcher component status changed to stopping", zap.String("table", d.tableSpan.String())) - d.IsRemoving.Store(true) -} - -func (d *TableTriggerEventDispatcher) GetRemovingStatus() bool { - return d.IsRemoving.Load() -} diff --git a/downstreamadapter/dispatchermanager/event_dispatcher_manager.go b/downstreamadapter/dispatchermanager/event_dispatcher_manager.go index 902dafa95..0036a4b5a 100644 --- a/downstreamadapter/dispatchermanager/event_dispatcher_manager.go +++ b/downstreamadapter/dispatchermanager/event_dispatcher_manager.go @@ -54,9 +54,8 @@ One EventDispatcherManager can only have one Sink. type EventDispatcherManager struct { dispatcherMap *DispatcherMap - tableTriggerEventDispatcher *dispatcher.TableTriggerEventDispatcher - heartbeatResponseQueue *HeartbeatResponseQueue - heartbeatRequestQueue *HeartbeatRequestQueue + heartbeatResponseQueue *HeartbeatResponseQueue + heartbeatRequestQueue *HeartbeatRequestQueue //heartBeatSendTask *HeartBeatSendTask cancel context.CancelFunc wg sync.WaitGroup @@ -111,7 +110,8 @@ func NewEventDispatcherManager(changefeedID model.ChangeFeedID, changefeedConfig appcontext.GetService[*HeartBeatCollector](appcontext.HeartbeatCollector).RegisterEventDispatcherManager(eventDispatcherManager) if createTableTriggerEventDispatcher { - eventDispatcherManager.tableTriggerEventDispatcher = eventDispatcherManager.newTableTriggerEventDispatcher(eventDispatcherManager.config.StartTS) + dispatcher := eventDispatcherManager.NewDispatcher(&common.DDLSpan, eventDispatcherManager.config.StartTS) + eventDispatcherManager.dispatcherMap.Set(&common.DDLSpan, dispatcher) } // TODO: 这些后续需要等有第一个 table 来的时候再初始化, 对于纯空的 event dispatcher manager 不要直接创建为好 @@ -186,8 +186,8 @@ func (e *EventDispatcherManager) close() { e.cancel() e.wg.Wait() - toCloseDispatchers := make([]dispatcher.Dispatcher, 0) - e.dispatcherMap.ForEach(func(tableSpan *common.TableSpan, dispatcher *dispatcher.TableEventDispatcher) { + toCloseDispatchers := make([]*dispatcher.Dispatcher, 0) + e.dispatcherMap.ForEach(func(tableSpan *common.TableSpan, dispatcher *dispatcher.Dispatcher) { dispatcher.Remove() _, ok := dispatcher.TryClose() if !ok { @@ -195,11 +195,6 @@ func (e *EventDispatcherManager) close() { } }) - _, ok := e.tableTriggerEventDispatcher.TryClose() - if !ok { - toCloseDispatchers = append(toCloseDispatchers, e.tableTriggerEventDispatcher) - } - for _, dispatcher := range toCloseDispatchers { log.Info("waiting for dispatcher to close", zap.Any("tableSpan", dispatcher.GetTableSpan())) ok := false @@ -228,61 +223,31 @@ func calculateStartSyncPointTs(startTs uint64, syncPointInterval time.Duration) } */ -func (e *EventDispatcherManager) NewDispatcher(tableSpan *common.TableSpan, startTs uint64) { - if tableSpan == &common.DDLSpan { - e.newTableTriggerEventDispatcher(startTs) - } else { - e.NewTableEventDispatcher(tableSpan, startTs) - } -} - -// TODO:加一个 count 监控 -func (e *EventDispatcherManager) newTableTriggerEventDispatcher(startTs uint64) *dispatcher.TableTriggerEventDispatcher { - tableTriggerEventDispatcher := dispatcher.NewTableTriggerEventDispatcher(e.sink, startTs, e.tableSpanStatusesChan, e.filter) - - appcontext.GetService[*eventcollector.EventCollector](appcontext.EventCollector).RegisterDispatcher(tableTriggerEventDispatcher, startTs, toFilterConfigPB(e.config.Filter)) - - e.GetTableSpanStatusesChan() <- &heartbeatpb.TableSpanStatus{ - Span: tableTriggerEventDispatcher.GetTableSpan().TableSpan, - ComponentStatus: heartbeatpb.ComponentState_Working, - } - return tableTriggerEventDispatcher -} - -// 收到 rpc 请求创建,需要通过 event dispatcher manager 来 -func (e *EventDispatcherManager) NewTableEventDispatcher(tableSpan *common.TableSpan, startTs uint64) *dispatcher.TableEventDispatcher { +func (e *EventDispatcherManager) NewDispatcher(tableSpan *common.TableSpan, startTs uint64) *dispatcher.Dispatcher { start := time.Now() - if _, ok := e.dispatcherMap.Get(tableSpan); ok { log.Debug("table span already exists", zap.Any("tableSpan", tableSpan)) return nil } - /* - var syncPointInfo *dispatcher.SyncPointInfo - if e.EnableSyncPoint { - syncPointInfo.EnableSyncPoint = true - syncPointInfo.SyncPointInterval = e.SyncPointInterval - syncPointInfo.NextSyncPointTs = calculateStartSyncPointTs(startTs, e.SyncPointInterval) - } else { - syncPointInfo.EnableSyncPoint = false - } - */ - tableEventDispatcher := dispatcher.NewTableEventDispatcher(tableSpan, e.sink, startTs, nil, e.tableSpanStatusesChan, e.filter) + dispatcher := dispatcher.NewDispatcher(tableSpan, e.sink, startTs, e.tableSpanStatusesChan, e.filter) - appcontext.GetService[*eventcollector.EventCollector](appcontext.EventCollector).RegisterDispatcher(tableEventDispatcher, startTs, toFilterConfigPB(e.config.Filter)) + appcontext.GetService[*eventcollector.EventCollector](appcontext.EventCollector).RegisterDispatcher(dispatcher, startTs, toFilterConfigPB(e.config.Filter)) - e.dispatcherMap.Set(tableSpan, tableEventDispatcher) + e.dispatcherMap.Set(tableSpan, dispatcher) e.GetTableSpanStatusesChan() <- &heartbeatpb.TableSpanStatus{ Span: tableSpan.TableSpan, ComponentStatus: heartbeatpb.ComponentState_Working, } + //TODO:区分 tableTriggerEventDIspatcher 的 metrics e.tableEventDispatcherCount.Inc() - log.Info("new table event dispatcher created", zap.Any("tableSpan", tableSpan), + + log.Info("new dispatcher created", zap.Any("tableSpan", tableSpan), zap.Int64("cost(ns)", time.Since(start).Nanoseconds()), zap.Time("start", start)) e.metricCreateDispatcherDuration.Observe(float64(time.Since(start).Seconds())) - return tableEventDispatcher + + return dispatcher } // CollectHeartbeatInfoWhenStatesChanged use to collect the heartbeat info when GetTableSpanStatusesChan() get infos @@ -328,14 +293,7 @@ func (e *EventDispatcherManager) CollectHeartbeatInfoWhenStatesChanged(ctx conte } func (e *EventDispatcherManager) RemoveDispatcher(tableSpan *common.TableSpan) { - var dispatcher dispatcher.Dispatcher - var ok bool - if tableSpan == &common.DDLSpan { - dispatcher = e.tableTriggerEventDispatcher - ok = dispatcher.GetComponentStatus() == heartbeatpb.ComponentState_Working - } else { - dispatcher, ok = e.dispatcherMap.Get(tableSpan) - } + dispatcher, ok := e.dispatcherMap.Get(tableSpan) if ok { if dispatcher.GetRemovingStatus() == true { @@ -407,14 +365,14 @@ func (e *EventDispatcherManager) CollectHeartbeatInfo(needCompleteStatus bool) * toReomveTableSpans := make([]*common.TableSpan, 0) allDispatchers := e.dispatcherMap.GetAllDispatchers() dispatcherHeartBeatInfo := &dispatcher.HeartBeatInfo{} - for _, tableEventDispatcher := range allDispatchers { + for _, dispatcherItem := range allDispatchers { // If the dispatcher is in removing state, we need to check if it's closed successfully. // If it's closed successfully, we could clean it up. // TODO: we need to consider how to deal with the checkpointTs of the removed dispatcher if the message will be discarded. - dispatcher.CollectDispatcherHeartBeatInfo(tableEventDispatcher, dispatcherHeartBeatInfo) + dispatcherItem.CollectDispatcherHeartBeatInfo(dispatcherHeartBeatInfo) if dispatcherHeartBeatInfo.IsRemoving == true { - watermark, ok := tableEventDispatcher.TryClose() + watermark, ok := dispatcherItem.TryClose() if ok { // remove successfully message.Watermark.UpdateMin(watermark) @@ -423,7 +381,7 @@ func (e *EventDispatcherManager) CollectHeartbeatInfo(needCompleteStatus bool) * Span: dispatcherHeartBeatInfo.TableSpan.TableSpan, ComponentStatus: heartbeatpb.ComponentState_Stopped, }) - toReomveTableSpans = append(toReomveTableSpans, tableEventDispatcher.GetTableSpan()) + toReomveTableSpans = append(toReomveTableSpans, dispatcherItem.GetTableSpan()) } } @@ -479,15 +437,15 @@ func (e *EventDispatcherManager) SetMaintainerID(maintainerID messaging.ServerId } type DispatcherMap struct { - dispatcherCacheForRead []*dispatcher.TableEventDispatcher + dispatcherCacheForRead []*dispatcher.Dispatcher mutex sync.Mutex - dispatchers *utils.BtreeMap[*common.TableSpan, *dispatcher.TableEventDispatcher] + dispatchers *utils.BtreeMap[*common.TableSpan, *dispatcher.Dispatcher] } func newDispatcherMap() *DispatcherMap { return &DispatcherMap{ - dispatcherCacheForRead: make([]*dispatcher.TableEventDispatcher, 0, 1024), - dispatchers: utils.NewBtreeMap[*common.TableSpan, *dispatcher.TableEventDispatcher](), + dispatcherCacheForRead: make([]*dispatcher.Dispatcher, 0, 1024), + dispatchers: utils.NewBtreeMap[*common.TableSpan, *dispatcher.Dispatcher](), } } @@ -497,7 +455,7 @@ func (d *DispatcherMap) Len() int { return d.dispatchers.Len() } -func (d *DispatcherMap) Get(tableSpan *common.TableSpan) (*dispatcher.TableEventDispatcher, bool) { +func (d *DispatcherMap) Get(tableSpan *common.TableSpan) (*dispatcher.Dispatcher, bool) { d.mutex.Lock() defer d.mutex.Unlock() return d.dispatchers.Get(tableSpan) @@ -509,16 +467,16 @@ func (d *DispatcherMap) Delete(tableSpan *common.TableSpan) { d.dispatchers.Delete(tableSpan) } -func (d *DispatcherMap) Set(tableSpan *common.TableSpan, dispatcher *dispatcher.TableEventDispatcher) { +func (d *DispatcherMap) Set(tableSpan *common.TableSpan, dispatcher *dispatcher.Dispatcher) { d.mutex.Lock() defer d.mutex.Unlock() d.dispatchers.ReplaceOrInsert(tableSpan, dispatcher) } -func (d *DispatcherMap) ForEach(fn func(tableSpan *common.TableSpan, dispatcher *dispatcher.TableEventDispatcher)) { +func (d *DispatcherMap) ForEach(fn func(tableSpan *common.TableSpan, dispatcher *dispatcher.Dispatcher)) { d.mutex.Lock() defer d.mutex.Unlock() - d.dispatchers.Ascend(func(tableSpan *common.TableSpan, dispatcherItem *dispatcher.TableEventDispatcher) bool { + d.dispatchers.Ascend(func(tableSpan *common.TableSpan, dispatcherItem *dispatcher.Dispatcher) bool { fn(tableSpan, dispatcherItem) return true }) @@ -526,17 +484,17 @@ func (d *DispatcherMap) ForEach(fn func(tableSpan *common.TableSpan, dispatcher func (d *DispatcherMap) resetDispatcherCache() { if cap(d.dispatcherCacheForRead) > 2048 && cap(d.dispatcherCacheForRead) > d.dispatchers.Len()*2 { - d.dispatcherCacheForRead = make([]*dispatcher.TableEventDispatcher, 0, d.dispatchers.Len()) + d.dispatcherCacheForRead = make([]*dispatcher.Dispatcher, 0, d.dispatchers.Len()) } d.dispatcherCacheForRead = d.dispatcherCacheForRead[:0] } -func (d *DispatcherMap) GetAllDispatchers() []*dispatcher.TableEventDispatcher { +func (d *DispatcherMap) GetAllDispatchers() []*dispatcher.Dispatcher { d.resetDispatcherCache() d.mutex.Lock() defer d.mutex.Unlock() - d.dispatchers.Ascend(func(tableSpan *common.TableSpan, dispatcherItem *dispatcher.TableEventDispatcher) bool { + d.dispatchers.Ascend(func(tableSpan *common.TableSpan, dispatcherItem *dispatcher.Dispatcher) bool { d.dispatcherCacheForRead = append(d.dispatcherCacheForRead, dispatcherItem) return true }) diff --git a/downstreamadapter/dispathermanagermanager/dispatcher_manager_manager.go b/downstreamadapter/dispathermanagermanager/dispatcher_manager_manager.go index 16f40ab72..c28b7f915 100644 --- a/downstreamadapter/dispathermanagermanager/dispatcher_manager_manager.go +++ b/downstreamadapter/dispathermanagermanager/dispatcher_manager_manager.go @@ -101,7 +101,7 @@ func (m *DispatcherManagerManager) handleAddDispatcherManager(from messaging.Ser response := &heartbeatpb.MaintainerBootstrapResponse{ Statuses: make([]*heartbeatpb.TableSpanStatus, 0, eventDispatcherManager.GetDispatcherMap().Len()), } - eventDispatcherManager.GetDispatcherMap().ForEach(func(tableSpan *common.TableSpan, tableEventDispatcher *dispatcher.TableEventDispatcher) { + eventDispatcherManager.GetDispatcherMap().ForEach(func(tableSpan *common.TableSpan, tableEventDispatcher *dispatcher.Dispatcher) { response.Statuses = append(response.Statuses, &heartbeatpb.TableSpanStatus{ Span: tableEventDispatcher.GetTableSpan().TableSpan, ComponentStatus: tableEventDispatcher.GetComponentStatus(), diff --git a/downstreamadapter/eventcollector/event_collector.go b/downstreamadapter/eventcollector/event_collector.go index d61e57a11..7f08cb2bb 100644 --- a/downstreamadapter/eventcollector/event_collector.go +++ b/downstreamadapter/eventcollector/event_collector.go @@ -34,23 +34,23 @@ import ( type DispatcherMap struct { mutex sync.Mutex - m map[string]dispatcher.Dispatcher // dispatcher_id --> dispatcher + m map[string]*dispatcher.Dispatcher // dispatcher_id --> dispatcher } func newDispatcherMap() *DispatcherMap { return &DispatcherMap{ - m: make(map[string]dispatcher.Dispatcher), + m: make(map[string]*dispatcher.Dispatcher), } } -func (m *DispatcherMap) Get(dispatcherId string) (dispatcher.Dispatcher, bool) { +func (m *DispatcherMap) Get(dispatcherId string) (*dispatcher.Dispatcher, bool) { m.mutex.Lock() defer m.mutex.Unlock() d, ok := m.m[dispatcherId] return d, ok } -func (m *DispatcherMap) Set(dispatcherId string, d dispatcher.Dispatcher) { +func (m *DispatcherMap) Set(dispatcherId string, d *dispatcher.Dispatcher) { m.mutex.Lock() defer m.mutex.Unlock() m.m[dispatcherId] = d @@ -63,7 +63,7 @@ func (m *DispatcherMap) Delete(dispatcherId string) { } type RegisterInfo struct { - dispatcher dispatcher.Dispatcher + dispatcher *dispatcher.Dispatcher startTs uint64 filterConfig *eventpb.FilterConfig } @@ -133,7 +133,7 @@ func NewEventCollector(globalMemoryQuota int64, serverId messaging.ServerId) *Ev // RegisterDispatcher register a dispatcher to event collector. // If the dispatcher is not table trigger event dispatcher, filterConfig will be nil. -func (c *EventCollector) RegisterDispatcher(d dispatcher.Dispatcher, startTs uint64, filterConfig *eventpb.FilterConfig) error { +func (c *EventCollector) RegisterDispatcher(d *dispatcher.Dispatcher, startTs uint64, filterConfig *eventpb.FilterConfig) error { err := appcontext.GetService[messaging.MessageCenter](appcontext.MessageCenter).SendEvent(&messaging.TargetMessage{ To: c.serverId, // demo 中 每个节点都有自己的 eventService Topic: messaging.EventServiceTopic, @@ -161,7 +161,7 @@ func (c *EventCollector) RegisterDispatcher(d dispatcher.Dispatcher, startTs uin return nil } -func (c *EventCollector) RemoveDispatcher(d dispatcher.Dispatcher) error { +func (c *EventCollector) RemoveDispatcher(d *dispatcher.Dispatcher) error { err := appcontext.GetService[messaging.MessageCenter](appcontext.MessageCenter).SendEvent(&messaging.TargetMessage{ To: c.serverId, Topic: messaging.EventServiceTopic, From 7202e4d9a76b8123049f5e2b956aa1d56c3eadd3 Mon Sep 17 00:00:00 2001 From: hongyunyan <649330952@qq.com> Date: Fri, 16 Aug 2024 14:54:25 +0800 Subject: [PATCH 06/12] support filter --- downstreamadapter/dispatcher/dispatcher.go | 25 +++++++++++-------- .../event_dispatcher_manager.go | 2 +- .../eventcollector/event_collector.go | 10 +++++--- pkg/apperror/errors.go | 2 ++ 4 files changed, 24 insertions(+), 15 deletions(-) diff --git a/downstreamadapter/dispatcher/dispatcher.go b/downstreamadapter/dispatcher/dispatcher.go index ddd1b2a9a..6da3028bc 100644 --- a/downstreamadapter/dispatcher/dispatcher.go +++ b/downstreamadapter/dispatcher/dispatcher.go @@ -22,9 +22,9 @@ import ( "github.com/flowbehappy/tigate/downstreamadapter/sink" "github.com/flowbehappy/tigate/heartbeatpb" "github.com/flowbehappy/tigate/pkg/common" + "github.com/flowbehappy/tigate/pkg/filter" "github.com/google/uuid" "github.com/pingcap/log" - "github.com/pingcap/tiflow/pkg/filter" "go.uber.org/zap" ) @@ -249,6 +249,7 @@ func (d *Dispatcher) SetDDLPendingEvent(event *common.TxnEvent) { } d.ddlPendingEvent = event } + func (d *Dispatcher) GetDDLFinishCh() chan struct{} { return d.ddlFinishCh } @@ -300,20 +301,22 @@ func (d *Dispatcher) HandleDDLActions(ctx context.Context) { // // TODO:特殊处理有 add index 的逻辑 func (d *Dispatcher) AddDDLEventToSinkWhenAvailable(event *common.TxnEvent) { - //filter := d.GetFilter() - // TODO: filter 支持 - // 判断 ddl 是否需要处理,如果不需要处理,直接返回 - // if filter.ShouldIgnoreDDLEvent(event.GetDDLEvent()) { - // return - // } - - // 需要根据 filter 来判断 ddl.Query 中是否需要调整,只针对 query 中包含多个 sql 语句。所以 的ddl 传来的时候,需要对应传 sql 对应的 table id 信息,用于过滤 + // 根据 filter 过滤 query 中不需要 send to downstream 的数据 + // 但应当不出现整个 query 都不需要 send to downstream 的 ddl,这种 ddl 不应该发给 dispatcher + // TODO: ddl 影响到的 tableSpan 也在 filter 中过滤一遍 + filter := d.GetFilter() + err := filter.FilterDDLQuery(event.GetDDLEvent()) + if err != nil { + log.Error("filter ddl query failed", zap.Error(err)) + // 这里怎么处理更合适呢?有错然后反上去让 changefeed 报错 + return + } sink := d.GetSink() tableSpan := d.GetTableSpan() if event.IsSingleTableDDL() { if sink.IsEmpty(tableSpan) { - sink.AddDMLEvent(tableSpan, event) + sink.AddDDLAndSyncPointEvent(tableSpan, event) return } else { // TODO:先写一个 定时 check 的逻辑,后面用 dynamic stream 改造 @@ -322,7 +325,7 @@ func (d *Dispatcher) AddDDLEventToSinkWhenAvailable(event *common.TxnEvent) { select { case <-timer.C: if sink.IsEmpty(tableSpan) { - sink.AddDMLEvent(tableSpan, event) + sink.AddDDLAndSyncPointEvent(tableSpan, event) return } } diff --git a/downstreamadapter/dispatchermanager/event_dispatcher_manager.go b/downstreamadapter/dispatchermanager/event_dispatcher_manager.go index 0036a4b5a..137d5942f 100644 --- a/downstreamadapter/dispatchermanager/event_dispatcher_manager.go +++ b/downstreamadapter/dispatchermanager/event_dispatcher_manager.go @@ -19,8 +19,8 @@ import ( "sync/atomic" "time" + "github.com/flowbehappy/tigate/pkg/filter" "github.com/pingcap/log" - "github.com/pingcap/tiflow/pkg/filter" "github.com/prometheus/client_golang/prometheus" "github.com/tikv/client-go/v2/oracle" diff --git a/downstreamadapter/eventcollector/event_collector.go b/downstreamadapter/eventcollector/event_collector.go index 7f08cb2bb..11c70772e 100644 --- a/downstreamadapter/eventcollector/event_collector.go +++ b/downstreamadapter/eventcollector/event_collector.go @@ -118,11 +118,15 @@ func NewEventCollector(globalMemoryQuota int64, serverId messaging.ServerId) *Ev if err != nil { log.Error("failed to send register dispatcher request message", zap.Error(err)) c.registerMessageChan.In() <- &RegisterInfo{ - dispatcher: d, - startTs: startTs, + dispatcher: d, + startTs: startTs, + filterConfig: registerInfo.filterConfig, } + // sleep for a while due to too many messages now + time.Sleep(10 * time.Millisecond) + continue } - time.Sleep(10 * time.Millisecond) // for test + c.dispatcherMap.Set(d.GetId(), d) metrics.EventCollectorRegisteredDispatcherCount.Inc() } diff --git a/pkg/apperror/errors.go b/pkg/apperror/errors.go index 711fbb4c2..c8c2ebd03 100644 --- a/pkg/apperror/errors.go +++ b/pkg/apperror/errors.go @@ -27,6 +27,8 @@ const ( ErrorTypeMessageSendFailed ErrorType = 206 ErrorTypeTargetNotFound ErrorType = 207 ErrorTypeInvalidMessage ErrorType = 208 + + ErrorInvalidDDLEvent ErrorType = 301 ) func (t ErrorType) String() string { From 5702999ec89e879686f50248011e0a7778d8d531 Mon Sep 17 00:00:00 2001 From: hongyunyan <649330952@qq.com> Date: Fri, 16 Aug 2024 14:55:00 +0800 Subject: [PATCH 07/12] add filter --- pkg/filter/expr_filter.go | 433 +++++++++++++++++++++++++++++++++ pkg/filter/filter.go | 294 ++++++++++++++++++++++ pkg/filter/sql_event_filter.go | 210 ++++++++++++++++ pkg/filter/utils.go | 149 ++++++++++++ 4 files changed, 1086 insertions(+) create mode 100644 pkg/filter/expr_filter.go create mode 100644 pkg/filter/filter.go create mode 100644 pkg/filter/sql_event_filter.go create mode 100644 pkg/filter/utils.go diff --git a/pkg/filter/expr_filter.go b/pkg/filter/expr_filter.go new file mode 100644 index 000000000..5ff6a58b3 --- /dev/null +++ b/pkg/filter/expr_filter.go @@ -0,0 +1,433 @@ +// 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 filter + +import ( + "strings" + "sync" + + "github.com/pingcap/errors" + "github.com/pingcap/log" + "github.com/pingcap/tidb/pkg/expression" + "github.com/pingcap/tidb/pkg/parser" + "github.com/pingcap/tidb/pkg/sessionctx" + "github.com/pingcap/tidb/pkg/types" + "github.com/pingcap/tidb/pkg/util/chunk" + "github.com/pingcap/tidb/pkg/util/dbterror/plannererrors" + tfilter "github.com/pingcap/tidb/pkg/util/table-filter" + "github.com/pingcap/tiflow/cdc/model" + "github.com/pingcap/tiflow/dm/pkg/utils" + "github.com/pingcap/tiflow/pkg/config" + cerror "github.com/pingcap/tiflow/pkg/errors" + "go.uber.org/zap" +) + +// dmlExprFilterRule only be used by dmlExprFilter. +// This struct is mostly a duplicate of `ExprFilterGroup` in dm/pkg/syncer, +// but have slightly changed to fit the usage of cdc. +type dmlExprFilterRule struct { + mu sync.Mutex + // Cache tableInfos to check if the table was changed. + tables map[string]*model.TableInfo + + insertExprs map[string]expression.Expression // tableName -> expr + updateOldExprs map[string]expression.Expression // tableName -> expr + updateNewExprs map[string]expression.Expression // tableName -> expr + deleteExprs map[string]expression.Expression // tableName -> expr + + tableMatcher tfilter.Filter + // All tables in this rule share the same config. + config *config.EventFilterRule + + sessCtx sessionctx.Context +} + +func newExprFilterRule( + sessCtx sessionctx.Context, + cfg *config.EventFilterRule, +) (*dmlExprFilterRule, error) { + tf, err := tfilter.Parse(cfg.Matcher) + if err != nil { + return nil, cerror.WrapError(cerror.ErrFilterRuleInvalid, err, cfg.Matcher) + } + + ret := &dmlExprFilterRule{ + tables: make(map[string]*model.TableInfo), + insertExprs: make(map[string]expression.Expression), + updateOldExprs: make(map[string]expression.Expression), + updateNewExprs: make(map[string]expression.Expression), + deleteExprs: make(map[string]expression.Expression), + config: cfg, + tableMatcher: tf, + sessCtx: sessCtx, + } + return ret, nil +} + +// verifyAndInitRule will verify and init the rule. +// It should only be called in dmlExprFilter's verify method. +// We ask users to set these expr only in default sql mode, +// so we just need to verify each expr in default sql mode +func (r *dmlExprFilterRule) verify(tableInfos []*model.TableInfo) error { + // verify expression filter rule syntax. + p := parser.New() + _, _, err := p.ParseSQL(completeExpression(r.config.IgnoreInsertValueExpr)) + if err != nil { + log.Error("failed to parse expression", zap.Error(err)) + return cerror.ErrExpressionParseFailed. + FastGenByArgs(r.config.IgnoreInsertValueExpr) + } + _, _, err = p.ParseSQL(completeExpression(r.config.IgnoreUpdateNewValueExpr)) + if err != nil { + log.Error("failed to parse expression", zap.Error(err)) + return cerror.ErrExpressionParseFailed. + FastGenByArgs(r.config.IgnoreUpdateNewValueExpr) + } + _, _, err = p.ParseSQL(completeExpression(r.config.IgnoreUpdateOldValueExpr)) + if err != nil { + log.Error("failed to parse expression", zap.Error(err)) + return cerror.ErrExpressionParseFailed. + FastGenByArgs(r.config.IgnoreUpdateOldValueExpr) + } + _, _, err = p.ParseSQL(completeExpression(r.config.IgnoreDeleteValueExpr)) + if err != nil { + log.Error("failed to parse expression", zap.Error(err)) + return cerror.ErrExpressionParseFailed. + FastGenByArgs(r.config.IgnoreDeleteValueExpr) + } + // verify expression filter rule. + for _, ti := range tableInfos { + tableName := ti.TableName.String() + if !r.tableMatcher.MatchTable(ti.TableName.Schema, ti.TableName.Table) { + continue + } + if r.config.IgnoreInsertValueExpr != "" { + e, err := r.getSimpleExprOfTable(r.config.IgnoreInsertValueExpr, ti) + if err != nil { + return err + } + r.insertExprs[tableName] = e + } + if r.config.IgnoreUpdateOldValueExpr != "" { + e, err := r.getSimpleExprOfTable(r.config.IgnoreUpdateOldValueExpr, ti) + if err != nil { + return err + } + r.updateOldExprs[tableName] = e + } + if r.config.IgnoreUpdateNewValueExpr != "" { + e, err := r.getSimpleExprOfTable(r.config.IgnoreUpdateNewValueExpr, ti) + if err != nil { + return err + } + r.updateNewExprs[tableName] = e + } + if r.config.IgnoreDeleteValueExpr != "" { + e, err := r.getSimpleExprOfTable(r.config.IgnoreDeleteValueExpr, ti) + if err != nil { + return err + } + r.deleteExprs[tableName] = e + } + } + return nil +} + +// The caller must hold r.mu.Lock() before calling this function. +func (r *dmlExprFilterRule) resetExpr(tableName string) { + delete(r.insertExprs, tableName) + delete(r.updateOldExprs, tableName) + delete(r.updateNewExprs, tableName) + delete(r.deleteExprs, tableName) +} + +// getInsertExprs returns the expression filter to filter INSERT events. +// This function will lazy calculate expressions if not initialized. +func (r *dmlExprFilterRule) getInsertExpr(ti *model.TableInfo) ( + expression.Expression, error, +) { + tableName := ti.TableName.String() + if r.insertExprs[tableName] != nil { + return r.insertExprs[tableName], nil + } + if r.config.IgnoreInsertValueExpr != "" { + expr, err := r.getSimpleExprOfTable(r.config.IgnoreInsertValueExpr, ti) + if err != nil { + return nil, err + } + r.insertExprs[tableName] = expr + } + return r.insertExprs[tableName], nil +} + +func (r *dmlExprFilterRule) getUpdateOldExpr(ti *model.TableInfo) ( + expression.Expression, error, +) { + tableName := ti.TableName.String() + if r.updateOldExprs[tableName] != nil { + return r.updateOldExprs[tableName], nil + } + + if r.config.IgnoreUpdateOldValueExpr != "" { + expr, err := r.getSimpleExprOfTable(r.config.IgnoreUpdateOldValueExpr, ti) + if err != nil { + return nil, err + } + r.updateOldExprs[tableName] = expr + } + return r.updateOldExprs[tableName], nil +} + +func (r *dmlExprFilterRule) getUpdateNewExpr(ti *model.TableInfo) ( + expression.Expression, error, +) { + tableName := ti.TableName.String() + if r.updateNewExprs[tableName] != nil { + return r.updateNewExprs[tableName], nil + } + + if r.config.IgnoreUpdateNewValueExpr != "" { + expr, err := r.getSimpleExprOfTable(r.config.IgnoreUpdateNewValueExpr, ti) + if err != nil { + return nil, err + } + r.updateNewExprs[tableName] = expr + } + return r.updateNewExprs[tableName], nil +} + +func (r *dmlExprFilterRule) getDeleteExpr(ti *model.TableInfo) ( + expression.Expression, error, +) { + tableName := ti.TableName.String() + if r.deleteExprs[tableName] != nil { + return r.deleteExprs[tableName], nil + } + + if r.config.IgnoreDeleteValueExpr != "" { + expr, err := r.getSimpleExprOfTable(r.config.IgnoreDeleteValueExpr, ti) + if err != nil { + return nil, err + } + r.deleteExprs[tableName] = expr + } + return r.deleteExprs[tableName], nil +} + +func (r *dmlExprFilterRule) getSimpleExprOfTable( + expr string, + ti *model.TableInfo, +) (expression.Expression, error) { + e, err := expression.ParseSimpleExprWithTableInfo(r.sessCtx.GetExprCtx(), expr, ti.TableInfo) + if err != nil { + // If an expression contains an unknown column, + // we return an error and stop the changefeed. + if plannererrors.ErrUnknownColumn.Equal(err) { + log.Error("meet unknown column when generating expression", + zap.String("expression", expr), + zap.Error(err)) + return nil, cerror.ErrExpressionColumnNotFound. + FastGenByArgs(getColumnFromError(err), ti.TableName.String(), expr) + } + log.Error("failed to parse expression", zap.Error(err)) + return nil, cerror.ErrExpressionParseFailed.FastGenByArgs(err, expr) + } + return e, nil +} + +func (r *dmlExprFilterRule) shouldSkipDML( + row *model.RowChangedEvent, + rawRow model.RowChangedDatums, + ti *model.TableInfo, +) (bool, error) { + tableName := ti.TableName.String() + + r.mu.Lock() + defer r.mu.Unlock() + + if oldTi, ok := r.tables[tableName]; ok { + // If one table's tableInfo was updated, we need to reset this rule + // and update the tableInfo in the cache. + if ti.Version != oldTi.Version { + r.tables[tableName] = ti.Clone() + r.resetExpr(ti.TableName.String()) + } + } else { + r.tables[tableName] = ti.Clone() + } + + switch { + case row.IsInsert(): + exprs, err := r.getInsertExpr(ti) + if err != nil { + return false, err + } + return r.skipDMLByExpression( + rawRow.RowDatums, + exprs, + ) + case row.IsUpdate(): + oldExprs, err := r.getUpdateOldExpr(ti) + if err != nil { + return false, err + } + newExprs, err := r.getUpdateNewExpr(ti) + if err != nil { + return false, err + } + ignoreOld, err := r.skipDMLByExpression( + rawRow.PreRowDatums, + oldExprs, + ) + if err != nil { + return false, err + } + ignoreNew, err := r.skipDMLByExpression( + rawRow.RowDatums, + newExprs, + ) + if err != nil { + return false, err + } + return ignoreOld || ignoreNew, nil + case row.IsDelete(): + exprs, err := r.getDeleteExpr(ti) + if err != nil { + return false, err + } + return r.skipDMLByExpression( + rawRow.PreRowDatums, + exprs, + ) + default: + log.Warn("unknown row changed event type") + return false, nil + } +} + +func (r *dmlExprFilterRule) skipDMLByExpression( + rowData []types.Datum, + expr expression.Expression, +) (bool, error) { + if len(rowData) == 0 || expr == nil { + return false, nil + } + + row := chunk.MutRowFromDatums(rowData).ToRow() + + d, err := expr.Eval(r.sessCtx.GetExprCtx().GetEvalCtx(), row) + if err != nil { + log.Error("failed to eval expression", zap.Error(err)) + return false, errors.Trace(err) + } + if d.GetInt64() == 1 { + return true, nil + } + return false, nil +} + +func getColumnFromError(err error) string { + if !plannererrors.ErrUnknownColumn.Equal(err) { + return err.Error() + } + column := strings.TrimSpace(strings.TrimPrefix(err.Error(), + "[planner:1054]Unknown column '")) + column = strings.TrimSuffix(column, "' in 'expression'") + return column +} + +// dmlExprFilter is a filter that filters DML events by SQL expression. +type dmlExprFilter struct { + rules []*dmlExprFilterRule +} + +func newExprFilter( + timezone string, + cfg *config.FilterConfig, +) (*dmlExprFilter, error) { + res := &dmlExprFilter{} + sessCtx := utils.NewSessionCtx(map[string]string{ + "time_zone": timezone, + }) + for _, rule := range cfg.EventFilters { + err := res.addRule(sessCtx, rule) + if err != nil { + return nil, err + } + } + return res, nil +} + +func (f *dmlExprFilter) addRule( + sessCtx sessionctx.Context, + cfg *config.EventFilterRule, +) error { + rule, err := newExprFilterRule(sessCtx, cfg) + if err != nil { + return errors.Trace(err) + } + f.rules = append(f.rules, rule) + return nil +} + +// verify checks if all rules in this filter is valid. +func (f *dmlExprFilter) verify(tableInfos []*model.TableInfo) error { + for _, rule := range f.rules { + err := rule.verify(tableInfos) + if err != nil { + log.Error("failed to verify expression filter rule", zap.Error(err)) + return errors.Trace(err) + } + } + return nil +} + +func (f *dmlExprFilter) getRules(schema, table string) []*dmlExprFilterRule { + res := make([]*dmlExprFilterRule, 0) + for _, rule := range f.rules { + if rule.tableMatcher.MatchTable(schema, table) { + res = append(res, rule) + } + } + return res +} + +// shouldSkipDML skips dml event by sql expression. +func (f *dmlExprFilter) shouldSkipDML( + row *model.RowChangedEvent, + rawRow model.RowChangedDatums, + ti *model.TableInfo, +) (bool, error) { + if len(f.rules) == 0 { + return false, nil + } + // for defense purpose, normally the row and ti should not be nil. + if ti == nil || row == nil || rawRow.IsEmpty() { + return false, nil + } + rules := f.getRules(row.TableInfo.GetSchemaName(), row.TableInfo.GetTableName()) + for _, rule := range rules { + ignore, err := rule.shouldSkipDML(row, rawRow, ti) + if err != nil { + if cerror.ShouldFailChangefeed(err) { + return false, err + } + return false, cerror.WrapError(cerror.ErrFailedToFilterDML, err, row) + } + if ignore { + return true, nil + } + } + return false, nil +} diff --git a/pkg/filter/filter.go b/pkg/filter/filter.go new file mode 100644 index 000000000..7f6cefa0e --- /dev/null +++ b/pkg/filter/filter.go @@ -0,0 +1,294 @@ +// Copyright 2020 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 filter + +import ( + "strings" + + "github.com/flowbehappy/tigate/pkg/apperror" + "github.com/flowbehappy/tigate/pkg/common" + "github.com/pingcap/log" + timodel "github.com/pingcap/tidb/pkg/parser/model" + tfilter "github.com/pingcap/tidb/pkg/util/table-filter" + "github.com/pingcap/tiflow/cdc/model" + bf "github.com/pingcap/tiflow/pkg/binlog-filter" + "github.com/pingcap/tiflow/pkg/config" + "go.uber.org/zap" +) + +const ( + // SyncPointTable is the tale name use to write ts-map when sync-point is enable. + SyncPointTable = "syncpoint_v1" + // TiCDCSystemSchema is the schema only use by TiCDC. + TiCDCSystemSchema = "tidb_cdc" + // LightningTaskInfoSchema is the schema only generated by Lightning + LightningTaskInfoSchema = "lightning_task_info" +) + +// ddlWhiteListMap is a map of all DDL types that can be applied to cdc's schema storage. +var ddlWhiteListMap = map[timodel.ActionType]bf.EventType{ + // schema related DDLs + timodel.ActionCreateSchema: bf.CreateDatabase, + timodel.ActionDropSchema: bf.DropDatabase, + timodel.ActionModifySchemaCharsetAndCollate: bf.ModifySchemaCharsetAndCollate, + + // table related DDLs + timodel.ActionCreateTable: bf.CreateTable, + timodel.ActionCreateTables: bf.CreateTable, + timodel.ActionDropTable: bf.DropTable, + timodel.ActionTruncateTable: bf.TruncateTable, + timodel.ActionRenameTable: bf.RenameTable, + timodel.ActionRenameTables: bf.RenameTable, + timodel.ActionRecoverTable: bf.RecoverTable, + timodel.ActionModifyTableComment: bf.ModifyTableComment, + timodel.ActionModifyTableCharsetAndCollate: bf.ModifyTableCharsetAndCollate, + + // view related DDLs + timodel.ActionCreateView: bf.CreateView, + timodel.ActionDropView: bf.DropView, + + // partition related DDLs + timodel.ActionAddTablePartition: bf.AddTablePartition, + timodel.ActionDropTablePartition: bf.DropTablePartition, + timodel.ActionTruncateTablePartition: bf.TruncateTablePartition, + timodel.ActionExchangeTablePartition: bf.ExchangePartition, + timodel.ActionReorganizePartition: bf.ReorganizePartition, + timodel.ActionAlterTablePartitioning: bf.AlterTablePartitioning, + timodel.ActionRemovePartitioning: bf.RemovePartitioning, + + // column related DDLs + timodel.ActionAddColumn: bf.AddColumn, + timodel.ActionDropColumn: bf.DropColumn, + timodel.ActionModifyColumn: bf.ModifyColumn, + timodel.ActionSetDefaultValue: bf.SetDefaultValue, + + // index related DDLs + timodel.ActionRebaseAutoID: bf.RebaseAutoID, + timodel.ActionAddPrimaryKey: bf.AddPrimaryKey, + timodel.ActionDropPrimaryKey: bf.DropPrimaryKey, + timodel.ActionAddIndex: bf.CreateIndex, + timodel.ActionDropIndex: bf.DropIndex, + timodel.ActionRenameIndex: bf.RenameIndex, + timodel.ActionAlterIndexVisibility: bf.AlterIndexVisibility, + + // TTL related DDLs + timodel.ActionAlterTTLInfo: bf.AlterTTLInfo, + timodel.ActionAlterTTLRemove: bf.AlterTTLRemove, + + // difficult to classify DDLs + timodel.ActionMultiSchemaChange: bf.MultiSchemaChange, + + // deprecated DDLs,see https://github.com/pingcap/tidb/pull/35862. + // DDL types below are deprecated in TiDB v6.2.0, but we still keep them here + // In case that some users will use TiCDC to replicate data from TiDB v6.1.x. + timodel.ActionAddColumns: bf.AddColumn, + timodel.ActionDropColumns: bf.DropColumn, +} + +// Filter are safe for concurrent use. +// TODO: find a better way to abstract this interface. +type Filter interface { + // ShouldIgnoreDMLEvent returns true if the DML event should not be sent to downstream. + ShouldIgnoreDMLEvent(dml *model.RowChangedEvent, rawRow model.RowChangedDatums, tableInfo *model.TableInfo) (bool, error) + // ShouldIgnoreDDLEvent returns true if the DDL event should not be sent to downstream. + ShouldIgnoreDDLEvent(ddl *model.DDLEvent) (bool, error) + // ShouldDiscardDDL returns true if this DDL should be discarded. + // If a ddl is discarded, it will neither be applied to cdc's schema storage + // nor sent to downstream. + ShouldDiscardDDL(ddlType timodel.ActionType, schema, table string) bool + // ShouldIgnoreTable returns true if the table should be ignored. + ShouldIgnoreTable(schema, table string) bool + // ShouldIgnoreSchema returns true if the schema should be ignored. + ShouldIgnoreSchema(schema string) bool + // Verify should only be called by create changefeed OpenAPI. + // Its purpose is to verify the expression filter config. + Verify(tableInfos []*model.TableInfo) error + + // 根据 ddl event 返回需要执行的 ddl query + FilterDDLQuery(ddl *common.DDLEvent) error +} + +// filter implements Filter. +type filter struct { + // tableFilter is used to filter in dml/ddl event by table name. + tableFilter tfilter.Filter + // dmlExprFilter is used to filter out dml event by its columns value. + dmlExprFilter *dmlExprFilter + // sqlEventFilter is used to filter out dml/ddl event by its type or query. + sqlEventFilter *sqlEventFilter + // ignoreTxnStartTs is used to filter out dml/ddl event by its starsTs. + ignoreTxnStartTs []uint64 +} + +// NewFilter creates a filter. +func NewFilter(cfg *config.ReplicaConfig, tz string) (Filter, error) { + f, err := VerifyTableRules(cfg.Filter) + if err != nil { + return nil, err + } + + if !cfg.CaseSensitive { + f = tfilter.CaseInsensitive(f) + } + + dmlExprFilter, err := newExprFilter(tz, cfg.Filter) + if err != nil { + return nil, err + } + sqlEventFilter, err := newSQLEventFilter(cfg.Filter) + if err != nil { + return nil, err + } + return &filter{ + tableFilter: f, + dmlExprFilter: dmlExprFilter, + sqlEventFilter: sqlEventFilter, + ignoreTxnStartTs: cfg.Filter.IgnoreTxnStartTs, + }, nil +} + +func (f *filter) FilterDDLQuery(ddl *common.DDLEvent) error { + query := ddl.Job.Query + queryList := strings.Split(query, ";") + if len(queryList) == 1 { + return nil + } + multiTableInfos := ddl.Job.BinlogInfo.MultipleTableInfos + schemaName := ddl.Job.SchemaName + if len(multiTableInfos) != len(queryList) { + log.Error("DDL Event is not valid, query count not equal to table count", zap.Any("ddl", ddl)) + return apperror.NewAppError(apperror.ErrorInvalidDDLEvent, "DDL Event is not valid, query count not equal to table count") + } + finalQuery := make([]string, 0, len(queryList)) + for i, query := range queryList { + tableInfo := multiTableInfos[i] + // 只需要 判断 table name 需不需要过滤就行,如果 schema name 要过滤的话,整个 query 就不会给 dispatcher 了 + tableName := tableInfo.Name.O + if !f.ShouldIgnoreTable(schemaName, tableName) { + finalQuery = append(finalQuery, query) + } + } + if len(finalQuery) != len(queryList) { + ddl.Job.Query = strings.Join(finalQuery, ";") + } + // TODO: 应该同时要更新一下 ddl 依赖的 table 信息 + return nil +} + +// ShouldIgnoreDMLEvent checks if a DML event should be ignore by conditions below: +// 0. By startTs. +// 1. By table name. +// 2. By type. +// 3. By columns value. +func (f *filter) ShouldIgnoreDMLEvent( + dml *model.RowChangedEvent, + rawRow model.RowChangedDatums, + ti *model.TableInfo, +) (bool, error) { + if f.shouldIgnoreStartTs(dml.StartTs) { + return true, nil + } + + if f.ShouldIgnoreTable(dml.TableInfo.GetSchemaName(), dml.TableInfo.GetTableName()) { + return true, nil + } + + ignoreByEventType, err := f.sqlEventFilter.shouldSkipDML(dml) + if err != nil { + return false, err + } + if ignoreByEventType { + return true, nil + } + return f.dmlExprFilter.shouldSkipDML(dml, rawRow, ti) +} + +// ShouldDiscardDDL checks if a DDL should be discarded by conditions below: +// 0. By allow list. +// 1. By schema name. +// 2. By table name. +func (f *filter) ShouldDiscardDDL(ddlType timodel.ActionType, schema, table string) bool { + if !isAllowedDDL(ddlType) { + return true + } + + if IsSchemaDDL(ddlType) { + return f.ShouldIgnoreSchema(schema) + } + return f.ShouldIgnoreTable(schema, table) +} + +// ShouldIgnoreDDLEvent checks if a DDL event should be ignore by conditions below: +// 1. By ddl type. +// 2. By ddl query. +// +// If a ddl is ignored, it will be applied to cdc's schema storage, +// but will not be sent to downstream. +// Note that a ignored ddl is different from a discarded ddl. For example, suppose +// we have a changefeed-test with the following config: +// - table filter: rules = ['test.*'] +// - event-filters: matcher = ["test.worker"] ignore-event = ["create table"] +// +// Then, for the following DDLs: +// 1. `CREATE TABLE test.worker` will be ignored, but the table will be replicated by changefeed-test. +// 2. `CREATE TABLE other.worker` will be discarded, and the table will not be replicated by changefeed-test. +func (f *filter) ShouldIgnoreDDLEvent(ddl *model.DDLEvent) (bool, error) { + if f.shouldIgnoreStartTs(ddl.StartTs) { + return true, nil + } + return f.sqlEventFilter.shouldSkipDDL(ddl) +} + +// ShouldIgnoreTable returns true if the specified table should be ignored by this changefeed. +// NOTICE: Set `tbl` to an empty string to test against the whole database. +func (f *filter) ShouldIgnoreTable(db, tbl string) bool { + if isSysSchema(db) { + return true + } + return !f.tableFilter.MatchTable(db, tbl) +} + +// ShouldIgnoreSchema returns true if the specified schema should be ignored by this changefeed. +func (f *filter) ShouldIgnoreSchema(schema string) bool { + return isSysSchema(schema) || !f.tableFilter.MatchSchema(schema) +} + +func (f *filter) Verify(tableInfos []*model.TableInfo) error { + return f.dmlExprFilter.verify(tableInfos) +} + +func (f *filter) shouldIgnoreStartTs(ts uint64) bool { + for _, ignoreTs := range f.ignoreTxnStartTs { + if ignoreTs == ts { + return true + } + } + return false +} + +func isAllowedDDL(actionType timodel.ActionType) bool { + _, ok := ddlWhiteListMap[actionType] + return ok +} + +// IsSchemaDDL returns true if the action type is a schema DDL. +func IsSchemaDDL(actionType timodel.ActionType) bool { + switch actionType { + case timodel.ActionCreateSchema, timodel.ActionDropSchema, + timodel.ActionModifySchemaCharsetAndCollate: + return true + default: + return false + } +} diff --git a/pkg/filter/sql_event_filter.go b/pkg/filter/sql_event_filter.go new file mode 100644 index 000000000..dd3b9c8a4 --- /dev/null +++ b/pkg/filter/sql_event_filter.go @@ -0,0 +1,210 @@ +// 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 filter + +import ( + "github.com/pingcap/errors" + "github.com/pingcap/log" + tfilter "github.com/pingcap/tidb/pkg/util/table-filter" + "github.com/pingcap/tiflow/cdc/model" + bf "github.com/pingcap/tiflow/pkg/binlog-filter" + "github.com/pingcap/tiflow/pkg/config" + cerror "github.com/pingcap/tiflow/pkg/errors" + "go.uber.org/zap" +) + +const ( + // binlogFilterSchemaPlaceholder is a place holder for schema name in binlog filter. + // Since we use table filter in rule as a matcher to match a dml/ddl event's schema and table, + // so we don't need to care about schema name when we calling binlog filter's method, + // we just use this place holder to call binlog filter's method whenever we need pass a schema. + binlogFilterSchemaPlaceholder = "binlogFilterSchema" + // binlogFilterTablePlaceholder is a place holder for table name in binlog filter. + // The reason we need it is the same as binlogFilterSchemaPlaceholder. + binlogFilterTablePlaceholder = "binlogFilterTable" + // dmlQuery is a place holder to call binlog filter to filter dml event. + dmlQuery = "" + // caseSensitive is use to create bf.BinlogEvent. + caseSensitive = false +) + +// sqlEventRule only be used by sqlEventFilter. +type sqlEventRule struct { + // we use table filter to match a dml/ddl event's schema and table. + // since binlog filter does not support syntax like `!test.t1`, + // which means not match `test.t1`. + tf tfilter.Filter + bf *bf.BinlogEvent +} + +func newSQLEventFilterRule(cfg *config.EventFilterRule) (*sqlEventRule, error) { + tf, err := tfilter.Parse(cfg.Matcher) + if err != nil { + return nil, cerror.WrapError(cerror.ErrFilterRuleInvalid, err, cfg.Matcher) + } + + res := &sqlEventRule{ + tf: tf, + } + + if err := verifyIgnoreEvents(cfg.IgnoreEvent); err != nil { + return nil, err + } + + bfRule := &bf.BinlogEventRule{ + SchemaPattern: binlogFilterSchemaPlaceholder, + TablePattern: binlogFilterTablePlaceholder, + Events: cfg.IgnoreEvent, + SQLPattern: cfg.IgnoreSQL, + Action: bf.Ignore, + } + + res.bf, err = bf.NewBinlogEvent(caseSensitive, []*bf.BinlogEventRule{bfRule}) + if err != nil { + return nil, cerror.WrapError(cerror.ErrFilterRuleInvalid, err, "failed to create binlog event filter") + } + + return res, nil +} + +func verifyIgnoreEvents(types []bf.EventType) error { + typesMap := make(map[bf.EventType]struct{}, len(SupportedEventTypes())) + for _, et := range SupportedEventTypes() { + typesMap[et] = struct{}{} + } + for _, et := range types { + if _, ok := typesMap[et]; !ok { + return cerror.ErrInvalidIgnoreEventType.GenWithStackByArgs(string(et)) + } + } + return nil +} + +// sqlEventFilter is a filter that filters DDL/DML event by its type or query. +type sqlEventFilter struct { + rules []*sqlEventRule +} + +func newSQLEventFilter(cfg *config.FilterConfig) (*sqlEventFilter, error) { + res := &sqlEventFilter{} + for _, rule := range cfg.EventFilters { + if err := res.addRule(rule); err != nil { + return nil, errors.Trace(err) + } + } + return res, nil +} + +func (f *sqlEventFilter) addRule(cfg *config.EventFilterRule) error { + rule, err := newSQLEventFilterRule(cfg) + if err != nil { + return errors.Trace(err) + } + f.rules = append(f.rules, rule) + return nil +} + +func (f *sqlEventFilter) getRules(schema, table string) []*sqlEventRule { + res := make([]*sqlEventRule, 0) + for _, rule := range f.rules { + if len(table) == 0 { + if rule.tf.MatchSchema(schema) { + res = append(res, rule) + } + } else { + if rule.tf.MatchTable(schema, table) { + res = append(res, rule) + } + } + } + return res +} + +// skipDDLEvent skips ddl event by its type and query. +func (f *sqlEventFilter) shouldSkipDDL(ddl *model.DDLEvent) (skip bool, err error) { + if len(f.rules) == 0 { + return false, nil + } + schema := ddl.TableInfo.TableName.Schema + table := ddl.TableInfo.TableName.Table + evenType := ddlToEventType(ddl.Type) + if evenType == bf.NullEvent { + log.Warn("sql event filter unsupported ddl type, do nothing", + zap.String("type", ddl.Type.String()), + zap.String("query", ddl.Query)) + return false, nil + } + + rules := f.getRules(schema, table) + for _, rule := range rules { + action, err := rule.bf.Filter( + binlogFilterSchemaPlaceholder, + binlogFilterTablePlaceholder, + evenType, ddl.Query) + if err != nil { + return false, errors.Trace(err) + } + if action == bf.Ignore { + return true, nil + } + + // If the ddl is alter table's subtype, + // we need try to filter it by bf.AlterTable. + if isAlterTable(ddl.Type) { + action, err = rule.bf.Filter( + binlogFilterSchemaPlaceholder, + binlogFilterTablePlaceholder, + bf.AlterTable, ddl.Query) + if err != nil { + return false, errors.Trace(err) + } + if action == bf.Ignore { + return true, nil + } + } + } + return false, nil +} + +// shouldSkipDML skips dml event by its type. +func (f *sqlEventFilter) shouldSkipDML(event *model.RowChangedEvent) (bool, error) { + if len(f.rules) == 0 { + return false, nil + } + + var et bf.EventType + switch { + case event.IsInsert(): + et = bf.InsertEvent + case event.IsUpdate(): + et = bf.UpdateEvent + case event.IsDelete(): + et = bf.DeleteEvent + default: + // It should never happen. + log.Warn("unknown row changed event type") + return false, nil + } + rules := f.getRules(event.TableInfo.GetSchemaName(), event.TableInfo.GetTableName()) + for _, rule := range rules { + action, err := rule.bf.Filter(binlogFilterSchemaPlaceholder, binlogFilterTablePlaceholder, et, dmlQuery) + if err != nil { + return false, cerror.WrapError(cerror.ErrFailedToFilterDML, err, event) + } + if action == bf.Ignore { + return true, nil + } + } + return false, nil +} diff --git a/pkg/filter/utils.go b/pkg/filter/utils.go new file mode 100644 index 000000000..808b6803a --- /dev/null +++ b/pkg/filter/utils.go @@ -0,0 +1,149 @@ +// 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 filter + +import ( + "fmt" + + timodel "github.com/pingcap/tidb/pkg/parser/model" + tifilter "github.com/pingcap/tidb/pkg/util/filter" + tfilter "github.com/pingcap/tidb/pkg/util/table-filter" + bf "github.com/pingcap/tiflow/pkg/binlog-filter" + "github.com/pingcap/tiflow/pkg/config" + cerror "github.com/pingcap/tiflow/pkg/errors" +) + +// isSysSchema returns true if the given schema is a system schema +func isSysSchema(db string) bool { + switch db { + // TiCDCSystemSchema is used by TiCDC only. + // Tables in TiCDCSystemSchema should not be replicated by cdc. + case TiCDCSystemSchema: + return true + case LightningTaskInfoSchema: + return true + default: + return tifilter.IsSystemSchema(db) + } +} + +// VerifyTableRules checks the table filter rules in the configuration +// and returns an invalid rule error if the verification fails, +// otherwise it will return a table filter. +func VerifyTableRules(cfg *config.FilterConfig) (tfilter.Filter, error) { + rules := cfg.Rules + if len(rules) == 0 { + rules = []string{"*.*"} + } + f, err := tfilter.Parse(rules) + if err != nil { + return nil, cerror.WrapError(cerror.ErrFilterRuleInvalid, err, cfg) + } + + return f, nil +} + +// ddlToEventType get event type from ddl query. +func ddlToEventType(jobType timodel.ActionType) bf.EventType { + evenType, ok := ddlWhiteListMap[jobType] + if ok { + return evenType + } + return bf.NullEvent +} + +var alterTableSubType = []timodel.ActionType{ + // table related DDLs + timodel.ActionRenameTable, + timodel.ActionRenameTables, + timodel.ActionModifyTableComment, + timodel.ActionModifyTableCharsetAndCollate, + + // partition related DDLs + timodel.ActionAddTablePartition, + timodel.ActionDropTablePartition, + timodel.ActionTruncateTablePartition, + timodel.ActionExchangeTablePartition, + timodel.ActionReorganizePartition, + timodel.ActionAlterTablePartitioning, + timodel.ActionRemovePartitioning, + + // column related DDLs + timodel.ActionAddColumn, + timodel.ActionDropColumn, + timodel.ActionModifyColumn, + timodel.ActionSetDefaultValue, + + // index related DDLs + timodel.ActionRebaseAutoID, + timodel.ActionAddPrimaryKey, + timodel.ActionDropPrimaryKey, + timodel.ActionAddIndex, + timodel.ActionDropIndex, + timodel.ActionRenameIndex, + timodel.ActionAlterIndexVisibility, + + // TTL related DDLs + timodel.ActionAlterTTLInfo, + timodel.ActionAlterTTLRemove, + + // difficult to classify DDLs + timodel.ActionMultiSchemaChange, + + // deprecated DDLs,see https://github.com/pingcap/tidb/pull/35862. + // DDL types below are deprecated in TiDB v6.2.0, but we still keep them here + // In case that some users will use TiCDC to replicate data from TiDB v6.1.x. + timodel.ActionAddColumns, + timodel.ActionDropColumns, +} + +// isAlterTable returns true if the given job type is alter table's subtype. +func isAlterTable(jobType timodel.ActionType) bool { + for _, t := range alterTableSubType { + if t == jobType { + return true + } + } + return false +} + +// SupportedEventTypes returns the supported event types. +func SupportedEventTypes() []bf.EventType { + supportedEventTypes := []bf.EventType{ + bf.AllDML, + bf.AllDDL, + + // dml events + bf.InsertEvent, + bf.UpdateEvent, + bf.DeleteEvent, + + // ddl events + bf.AlterTable, + bf.CreateSchema, + bf.DropSchema, + } + + for _, ddlType := range ddlWhiteListMap { + supportedEventTypes = append(supportedEventTypes, ddlType) + } + return supportedEventTypes +} + +func completeExpression(suffix string) string { + if suffix == "" { + return suffix + } + return fmt.Sprintf("select * from t where %s", suffix) +} From 63f7dabbcfe7e0c2bafa7854260ce48bba6b7989 Mon Sep 17 00:00:00 2001 From: hongyunyan <649330952@qq.com> Date: Fri, 16 Aug 2024 14:56:32 +0800 Subject: [PATCH 08/12] update --- downstreamadapter/dispatcher/dispatcher.go | 2 +- pkg/filter/filter.go | 6 +++--- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/downstreamadapter/dispatcher/dispatcher.go b/downstreamadapter/dispatcher/dispatcher.go index 6da3028bc..d0b5c09e3 100644 --- a/downstreamadapter/dispatcher/dispatcher.go +++ b/downstreamadapter/dispatcher/dispatcher.go @@ -305,7 +305,7 @@ func (d *Dispatcher) AddDDLEventToSinkWhenAvailable(event *common.TxnEvent) { // 但应当不出现整个 query 都不需要 send to downstream 的 ddl,这种 ddl 不应该发给 dispatcher // TODO: ddl 影响到的 tableSpan 也在 filter 中过滤一遍 filter := d.GetFilter() - err := filter.FilterDDLQuery(event.GetDDLEvent()) + err := filter.FilterDDLEvent(event.GetDDLEvent()) if err != nil { log.Error("filter ddl query failed", zap.Error(err)) // 这里怎么处理更合适呢?有错然后反上去让 changefeed 报错 diff --git a/pkg/filter/filter.go b/pkg/filter/filter.go index 7f6cefa0e..739de062f 100644 --- a/pkg/filter/filter.go +++ b/pkg/filter/filter.go @@ -115,8 +115,8 @@ type Filter interface { // Its purpose is to verify the expression filter config. Verify(tableInfos []*model.TableInfo) error - // 根据 ddl event 返回需要执行的 ddl query - FilterDDLQuery(ddl *common.DDLEvent) error + // filter ddl event to update query and influenced table spans + FilterDDLEvent(ddl *common.DDLEvent) error } // filter implements Filter. @@ -158,7 +158,7 @@ func NewFilter(cfg *config.ReplicaConfig, tz string) (Filter, error) { }, nil } -func (f *filter) FilterDDLQuery(ddl *common.DDLEvent) error { +func (f *filter) FilterDDLEvent(ddl *common.DDLEvent) error { query := ddl.Job.Query queryList := strings.Split(query, ";") if len(queryList) == 1 { From 0552e0fad46c1c525f958637463fd1a3cd458d84 Mon Sep 17 00:00:00 2001 From: hongyunyan <649330952@qq.com> Date: Fri, 16 Aug 2024 17:36:45 +0800 Subject: [PATCH 09/12] update --- downstreamadapter/dispatcher/dispatcher.go | 10 +- .../dispatcher/table_event_dispatcher_test.go | 113 ------------------ .../event_dispatcher_manager.go | 1 + downstreamadapter/sink/mysql_sink.go | 14 +-- downstreamadapter/sink/mysql_sink_test.go | 77 +++++++----- downstreamadapter/sink/sink.go | 2 +- .../sink/types/table_progress.go | 10 +- downstreamadapter/writer/mysql_writer.go | 4 +- downstreamadapter/writer/mysql_writer_test.go | 42 +++++-- 9 files changed, 100 insertions(+), 173 deletions(-) delete mode 100644 downstreamadapter/dispatcher/table_event_dispatcher_test.go diff --git a/downstreamadapter/dispatcher/dispatcher.go b/downstreamadapter/dispatcher/dispatcher.go index d0b5c09e3..15c82182f 100644 --- a/downstreamadapter/dispatcher/dispatcher.go +++ b/downstreamadapter/dispatcher/dispatcher.go @@ -107,6 +107,7 @@ func NewDispatcher(tableSpan *common.TableSpan, sink sink.Sink, startTs uint64, filter: filter, ddlFinishCh: make(chan struct{}), isRemoving: atomic.Bool{}, + ddlPendingEvent: nil, } dispatcher.sink.AddTableSpan(tableSpan) @@ -135,6 +136,7 @@ func (d *Dispatcher) DispatcherEvents(ctx context.Context) { } else if event.IsDDLEvent() { d.AddDDLEventToSinkWhenAvailable(event) } else { + log.Info("update resolved ts") d.resolvedTs.Set(event.ResolvedTs) } } @@ -158,10 +160,14 @@ func (d *Dispatcher) GetResolvedTs() uint64 { } func (d *Dispatcher) GetCheckpointTs() uint64 { - checkpointTs := d.GetSink().GetCheckpointTs(d.GetTableSpan()) + checkpointTs, isEmpty := d.GetSink().GetCheckpointTs(d.GetTableSpan()) if checkpointTs == 0 { // 说明从没有数据写到过 sink,则选择用 resolveTs 作为 checkpointTs - checkpointTs = d.GetResolvedTs() + return d.GetResolvedTs() + } + + if isEmpty { + return max(checkpointTs, d.GetResolvedTs()) } return checkpointTs } diff --git a/downstreamadapter/dispatcher/table_event_dispatcher_test.go b/downstreamadapter/dispatcher/table_event_dispatcher_test.go deleted file mode 100644 index a74741476..000000000 --- a/downstreamadapter/dispatcher/table_event_dispatcher_test.go +++ /dev/null @@ -1,113 +0,0 @@ -// Copyright 2024 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 dispatcher - -import ( - "database/sql" - "testing" - "time" - - "github.com/DATA-DOG/go-sqlmock" - "github.com/flowbehappy/tigate/downstreamadapter/sink" - "github.com/flowbehappy/tigate/downstreamadapter/writer" - "github.com/flowbehappy/tigate/heartbeatpb" - "github.com/flowbehappy/tigate/pkg/common" - "github.com/pingcap/tiflow/cdc/model" - "github.com/stretchr/testify/require" -) - -func newTestMockDB(t *testing.T) (db *sql.DB, mock sqlmock.Sqlmock) { - db, mock, err := sqlmock.New(sqlmock.QueryMatcherOption(sqlmock.QueryMatcherEqual)) - require.Nil(t, err) - return -} - -// 测一下我直接给dispatcher 数据塞 chan 里,检查 db 是否正确收到了,以及 消费前后的心跳和memory usage 是否正常 - -func TestTableEventDispatcher(t *testing.T) { - db, mock := newTestMockDB(t) - defer db.Close() - - mysqlSink := sink.NewMysqlSink(model.DefaultChangeFeedID("test1"), 8, writer.NewMysqlConfig(), db) - tableSpan := &common.TableSpan{TableSpan: &heartbeatpb.TableSpan{TableID: 1}} - startTs := uint64(100) - - tableEventDispatcher := NewTableEventDispatcher(tableSpan, mysqlSink, startTs, nil) - - tableEventDispatcher.PushTxnEvent(&common.TxnEvent{ - StartTs: 100, - CommitTs: 101, - Rows: []*common.RowChangedEvent{ - { - TableInfo: &common.TableInfo{ - TableName: common.TableName{ - Schema: "test_schema", - Table: "test_table", - }, - }, - Columns: []*common.Column{ - {Name: "id", Value: 1, Flag: common.HandleKeyFlag | common.PrimaryKeyFlag}, - {Name: "name", Value: "Alice"}, - }, - }, - }, - }) - - tableEventDispatcher.PushTxnEvent(&common.TxnEvent{ - StartTs: 102, - CommitTs: 105, - Rows: []*common.RowChangedEvent{ - { - TableInfo: &common.TableInfo{ - TableName: common.TableName{ - Schema: "test", - Table: "users", - }, - }, - PreColumns: []*common.Column{ - {Name: "id", Value: 1, Flag: common.HandleKeyFlag | common.PrimaryKeyFlag}, - {Name: "name", Value: "Alice"}, - }, - Columns: []*common.Column{ - {Name: "id", Value: 1, Flag: common.HandleKeyFlag | common.PrimaryKeyFlag}, - {Name: "name", Value: "Bob"}, - }, - }, - }, - }) - tableEventDispatcher.UpdateResolvedTs(110) - - heartBeatInfo := &HeartBeatInfo{} - CollectDispatcherHeartBeatInfo(tableEventDispatcher, heartBeatInfo) - require.Equal(t, uint64(100), heartBeatInfo.CheckpointTs) - //require.NotEqual(t, 0, tableEventDispatcher.GetMemoryUsage().GetUsedBytes()) - - mock.ExpectBegin() - mock.ExpectExec("INSERT INTO `test_schema`.`test_table` (`id`,`name`) VALUES (?,?)"). - WithArgs(1, "Alice"). - WillReturnResult(sqlmock.NewResult(1, 1)) - mock.ExpectExec("UPDATE `test`.`users` SET `id` = ?, `name` = ? WHERE `id` = ? LIMIT 1"). - WithArgs(1, "Bob", 1). - WillReturnResult(sqlmock.NewResult(1, 1)) - mock.ExpectCommit() - - time.Sleep(1 * time.Second) - - err := mock.ExpectationsWereMet() - require.NoError(t, err) - - CollectDispatcherHeartBeatInfo(tableEventDispatcher, heartBeatInfo) - require.Equal(t, uint64(110), heartBeatInfo.CheckpointTs) - //require.Equal(t, 0, tableEventDispatcher.GetMemoryUsage().GetUsedBytes()) -} diff --git a/downstreamadapter/dispatchermanager/event_dispatcher_manager.go b/downstreamadapter/dispatchermanager/event_dispatcher_manager.go index 137d5942f..1374d25ea 100644 --- a/downstreamadapter/dispatchermanager/event_dispatcher_manager.go +++ b/downstreamadapter/dispatchermanager/event_dispatcher_manager.go @@ -380,6 +380,7 @@ func (e *EventDispatcherManager) CollectHeartbeatInfo(needCompleteStatus bool) * message.Statuses = append(message.Statuses, &heartbeatpb.TableSpanStatus{ Span: dispatcherHeartBeatInfo.TableSpan.TableSpan, ComponentStatus: heartbeatpb.ComponentState_Stopped, + CheckpointTs: watermark.CheckpointTs, }) toReomveTableSpans = append(toReomveTableSpans, dispatcherItem.GetTableSpan()) } diff --git a/downstreamadapter/sink/mysql_sink.go b/downstreamadapter/sink/mysql_sink.go index 4dca0d34f..a2755fc0d 100644 --- a/downstreamadapter/sink/mysql_sink.go +++ b/downstreamadapter/sink/mysql_sink.go @@ -111,7 +111,6 @@ func NewMysqlSink(changefeedID model.ChangeFeedID, workerCount int, cfg *writer. BlockStrategy: causality.BlockStrategyWaitEmpty, }), tableStatuses: NewTableStatusMap(), - //dmlWorkerTasks: make([]*worker.MysqlWorkerDMLEventTask, workerCount), } mysqlSink.initWorker(workerCount, cfg, db) @@ -120,10 +119,6 @@ func NewMysqlSink(changefeedID model.ChangeFeedID, workerCount int, cfg *writer. } func (s *MysqlSink) initWorker(workerCount int, cfg *writer.MysqlConfig, db *sql.DB) { - // init ddl worker, which is for ddl event and sync point event - //s.ddlWorker = &worker.MysqlDDLWorker{MysqlWriter: writer.NewMysqlWriter(db, cfg)} - - // dml worker task will deal with all the dml events ctx, cancel := context.WithCancel(context.Background()) s.ddlWorker = worker.NewMysqlDDLWorker(db, cfg, s.changefeedID) @@ -131,7 +126,6 @@ func (s *MysqlSink) initWorker(workerCount int, cfg *writer.MysqlConfig, db *sql for i := 0; i < workerCount; i++ { s.wg.Add(1) workerId := i - // s.dmlWorkerTasks = append(s.dmlWorkerTasks, worker.NewMysqlWorkerDMLEventTask(s.conflictDetector.GetOutChByCacheID(int64(i)), db, cfg, 128)) go func(ctx context.Context, eventChan <-chan *common.TxnEvent, db *sql.DB, config *writer.MysqlConfig, maxRows int) { defer s.wg.Done() totalStart := time.Now() @@ -222,7 +216,9 @@ func (s *MysqlSink) AddDDLAndSyncPointEvent(tableSpan *common.TableSpan, event * } tableStatus.getProgress().Add(event) - event.PostTxnFlushed = func() { tableStatus.getProgress().Remove(event) } + event.PostTxnFlushed = func() { + tableStatus.getProgress().Remove(event) + } // TODO:这个 ddl 可以并发写么?如果不行的话,后面还要加锁或者排队 s.ddlWorker.GetMysqlWriter().FlushDDLEvent(event) } @@ -279,12 +275,12 @@ func (s *MysqlSink) IsEmpty(tableSpan *common.TableSpan) bool { return tableStatus.getProgress().Empty() } -func (s *MysqlSink) GetCheckpointTs(tableSpan *common.TableSpan) uint64 { +func (s *MysqlSink) GetCheckpointTs(tableSpan *common.TableSpan) (uint64, bool) { tableStatus, ok := s.tableStatuses.Get(tableSpan) if !ok { log.Error("unknown Span for Mysql Sink: ", zap.Any("tableSpan", tableSpan)) - return math.MaxUint64 + return math.MaxUint64, false } return tableStatus.getProgress().GetCheckpointTs() diff --git a/downstreamadapter/sink/mysql_sink_test.go b/downstreamadapter/sink/mysql_sink_test.go index 2bc5bb284..df3b015e0 100644 --- a/downstreamadapter/sink/mysql_sink_test.go +++ b/downstreamadapter/sink/mysql_sink_test.go @@ -21,6 +21,7 @@ import ( "github.com/flowbehappy/tigate/downstreamadapter/writer" "github.com/flowbehappy/tigate/heartbeatpb" "github.com/flowbehappy/tigate/pkg/common" + timodel "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tiflow/cdc/model" "github.com/stretchr/testify/require" "github.com/zeebo/assert" @@ -30,18 +31,15 @@ import ( func TestMysqlSinkBasicFunctionality(t *testing.T) { db, mock, err := sqlmock.New(sqlmock.QueryMatcherOption(sqlmock.QueryMatcherEqual)) require.Nil(t, err) - /* - mock.ExpectBegin() - mock.ExpectExec("USE `test`;").WillReturnResult(sqlmock.NewResult(1, 1)) - mock.ExpectExec("CREATE TABLE `test`.`t` (`id` INT PRIMARY KEY, `name` VARCHAR(255))").WillReturnResult(sqlmock.NewResult(1, 1)) - mock.ExpectCommit() - */ + mock.ExpectBegin() - mock.ExpectExec("INSERT INTO `test_schema`.`test_table` (`id`,`name`) VALUES (?,?)"). - WithArgs(1, "Alice"). - WillReturnResult(sqlmock.NewResult(1, 1)) - mock.ExpectExec("UPDATE `test`.`users` SET `id` = ?, `name` = ? WHERE `id` = ? LIMIT 1"). - WithArgs(1, "Bob", 1). + mock.ExpectExec("USE `test`;").WillReturnResult(sqlmock.NewResult(1, 1)) + mock.ExpectExec("CREATE TABLE `test`.`t` (`id` INT PRIMARY KEY, `name` VARCHAR(255))").WillReturnResult(sqlmock.NewResult(1, 1)) + mock.ExpectCommit() + + mock.ExpectBegin() + mock.ExpectExec("INSERT INTO `test_schema`.`test_table` (`id`,`name`) VALUES (?,?);UPDATE `test`.`users` SET `id` = ?, `name` = ? WHERE `id` = ? LIMIT 1"). + WithArgs(1, "Alice", 1, "Bob", 1). WillReturnResult(sqlmock.NewResult(1, 1)) mock.ExpectCommit() @@ -51,22 +49,25 @@ func TestMysqlSinkBasicFunctionality(t *testing.T) { tableSpan := common.TableSpan{TableSpan: &heartbeatpb.TableSpan{TableID: 1}} mysqlSink.AddTableSpan(&tableSpan) - /* - mysqlSink.AddDDLAndSyncPointEvent(&tableSpan, &common.TxnEvent{ - StartTs: 3, - CommitTs: 4, - DDLEvent: &common.DDLEvent{ - Job: &model.Job{ - Type: model.ActionCreateTable, - SchemaID: 10, - SchemaName: "test", - TableName: "t", - Query: "CREATE TABLE `test`.`t` (`id` INT PRIMARY KEY, `name` VARCHAR(255))", - }, - CommitTS: 4, + ts, isEmpty := mysqlSink.GetCheckpointTs(&tableSpan) + require.NotEqual(t, ts, 0) + require.Equal(t, isEmpty, true) + + mysqlSink.AddDDLAndSyncPointEvent(&tableSpan, &common.TxnEvent{ + StartTs: 1, + CommitTs: 1, + DDLEvent: &common.DDLEvent{ + Job: &timodel.Job{ + Type: timodel.ActionCreateTable, + SchemaID: 10, + SchemaName: "test", + TableName: "t", + Query: "CREATE TABLE `test`.`t` (`id` INT PRIMARY KEY, `name` VARCHAR(255))", }, - }) - */ + CommitTS: 1, + }, + }) + mysqlSink.AddDMLEvent(&tableSpan, &common.TxnEvent{ StartTs: 1, CommitTs: 2, @@ -109,14 +110,30 @@ func TestMysqlSinkBasicFunctionality(t *testing.T) { }, }) - require.Equal(t, mysqlSink.IsEmpty(&tableSpan), false) - require.NotEqual(t, mysqlSink.GetSmallestCommitTs(&tableSpan), 0) - time.Sleep(1 * time.Second) + + mysqlSink.PassDDLAndSyncPointEvent(&tableSpan, &common.TxnEvent{ + StartTs: 3, + CommitTs: 4, + DDLEvent: &common.DDLEvent{ + Job: &timodel.Job{ + Type: timodel.ActionCreateTable, + SchemaID: 10, + SchemaName: "test", + TableName: "t2", + Query: "CREATE TABLE `test`.`t2` (`id` INT PRIMARY KEY, `name` VARCHAR(255))", + }, + CommitTS: 4, + }, + }) + err = mock.ExpectationsWereMet() require.NoError(t, err) require.Equal(t, mysqlSink.IsEmpty(&tableSpan), true) - require.Equal(t, mysqlSink.GetSmallestCommitTs(&tableSpan), uint64(0)) + + ts, isEmpty = mysqlSink.GetCheckpointTs(&tableSpan) + require.Equal(t, ts, uint64(3)) + require.Equal(t, isEmpty, true) } diff --git a/downstreamadapter/sink/sink.go b/downstreamadapter/sink/sink.go index c5d3796de..6503c76c4 100644 --- a/downstreamadapter/sink/sink.go +++ b/downstreamadapter/sink/sink.go @@ -23,6 +23,6 @@ type Sink interface { AddTableSpan(tableSpan *common.TableSpan) RemoveTableSpan(tableSpan *common.TableSpan) StopTableSpan(tableSpan *common.TableSpan) - GetCheckpointTs(tableSpan *common.TableSpan) uint64 + GetCheckpointTs(tableSpan *common.TableSpan) (uint64, bool) Close() } diff --git a/downstreamadapter/sink/types/table_progress.go b/downstreamadapter/sink/types/table_progress.go index 684f8ea29..a8744a8fd 100644 --- a/downstreamadapter/sink/types/table_progress.go +++ b/downstreamadapter/sink/types/table_progress.go @@ -25,6 +25,7 @@ import ( // TableProgress 对外提供当前 table 是否有 event 在 sink 中等待被 flush 的能力--用于判断 ddl 是否达到下推条件 // // 本质是要频繁的删除随机数据,插入递增数据,查询最小值,后面自己可以实现一个红黑树吧,或者其他结构,先用 list 苟一苟 +// 需要加个测试保证插入数据不会出现 commitTs 倒退的问题 // thread safe type TableProgress struct { mutex sync.Mutex @@ -83,15 +84,16 @@ func (p *TableProgress) Pass(event *common.TxnEvent) { // 返回当前 tableSpan 中最大的 checkpointTs,也就是最大的 ts,并且 <= ts 之前的数据都已经成功写下去了 // 1. 假设目前 sink 还有没 flush 下去的 event,就拿最小的这个 event的 commitTs。 // 2. 反之,则选择收到过 event 中 commitTs 最大的那个。 -func (p *TableProgress) GetCheckpointTs() uint64 { +// 并且返回目前 是不是为空的状态,如果是空的话,resolvedTs 大于 checkpointTs,则用 resolvedTs 作为真的 checkpointTs +func (p *TableProgress) GetCheckpointTs() (uint64, bool) { p.mutex.Lock() defer p.mutex.Unlock() if p.list.Len() == 0 { if p.maxCommitTs == 0 { - return 0 + return 0, true } - return p.maxCommitTs - 1 + return p.maxCommitTs - 1, true } - return p.list.Front().Value.(Ts).commitTs - 1 + return p.list.Front().Value.(Ts).commitTs - 1, false } diff --git a/downstreamadapter/writer/mysql_writer.go b/downstreamadapter/writer/mysql_writer.go index 1ad29edf5..26352bf82 100644 --- a/downstreamadapter/writer/mysql_writer.go +++ b/downstreamadapter/writer/mysql_writer.go @@ -64,7 +64,9 @@ func (w *MysqlWriter) FlushDDLEvent(event *common.TxnEvent) error { return err } - event.PostTxnFlushed() + if event.PostTxnFlushed != nil { + event.PostTxnFlushed() + } return nil } diff --git a/downstreamadapter/writer/mysql_writer_test.go b/downstreamadapter/writer/mysql_writer_test.go index 1f1333132..504695bf0 100644 --- a/downstreamadapter/writer/mysql_writer_test.go +++ b/downstreamadapter/writer/mysql_writer_test.go @@ -10,6 +10,8 @@ import ( "github.com/zeebo/assert" "github.com/flowbehappy/tigate/pkg/common" + + timodel "github.com/pingcap/tidb/pkg/parser/model" ) func newTestMockDB(t *testing.T) (db *sql.DB, mock sqlmock.Sqlmock) { @@ -202,17 +204,31 @@ func TestMysqlWriter_Flush_EmptyEvents(t *testing.T) { require.NoError(t, err) } -/* -TODO: -func TestWithMysqlCluster(t *testing.T) { - _, db, err := NewMysqlConfigAndDB("tidb://root:@127.0.0.1:4000") +func TestMysqlWriter_FlushDDLEvent(t *testing.T) { + db, mock := newTestMockDB(t) + defer db.Close() + + cfg := &MysqlConfig{} + + writer := NewMysqlWriter(db, cfg, model.ChangeFeedID4Test("test", "test")) + + event := common.TxnEvent{DDLEvent: &common.DDLEvent{ + Job: &timodel.Job{ + SchemaName: "test", + TableName: "table1", + Query: "CREATE TABLE table1 (id INT PRIMARY KEY, name VARCHAR(255))", + }, + CommitTS: 10, + }} + + mock.ExpectBegin() + mock.ExpectExec("USE `test`;").WillReturnResult(sqlmock.NewResult(1, 1)) + mock.ExpectExec("CREATE TABLE table1 (id INT PRIMARY KEY, name VARCHAR(255))").WillReturnResult(sqlmock.NewResult(1, 1)) + mock.ExpectCommit() + + err := writer.FlushDDLEvent(&event) require.NoError(t, err) - rows, err := db.Query("SELECT * from d1.t1") - for rows.Next() { - var id int - var name string - err = rows.Scan(&id, &name) - require.NoError(t, err) - log.Info("id, name", zap.Any("id", id), zap.Any("name", name)) - } -}*/ + + err = mock.ExpectationsWereMet() + require.NoError(t, err) +} From b6d40fc606cfd38db2df0769782ec17ab33b485e Mon Sep 17 00:00:00 2001 From: hongyunyan <649330952@qq.com> Date: Fri, 16 Aug 2024 17:37:17 +0800 Subject: [PATCH 10/12] add test --- .../dispatcher/dispatcher_test.go | 310 ++++++++++++++++++ 1 file changed, 310 insertions(+) create mode 100644 downstreamadapter/dispatcher/dispatcher_test.go diff --git a/downstreamadapter/dispatcher/dispatcher_test.go b/downstreamadapter/dispatcher/dispatcher_test.go new file mode 100644 index 000000000..11d842b3b --- /dev/null +++ b/downstreamadapter/dispatcher/dispatcher_test.go @@ -0,0 +1,310 @@ +// Copyright 2024 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 dispatcher + +import ( + "database/sql" + "testing" + "time" + + "github.com/DATA-DOG/go-sqlmock" + "github.com/flowbehappy/tigate/downstreamadapter/sink" + "github.com/flowbehappy/tigate/downstreamadapter/writer" + "github.com/flowbehappy/tigate/heartbeatpb" + "github.com/flowbehappy/tigate/pkg/common" + "github.com/flowbehappy/tigate/pkg/filter" + timodel "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tiflow/cdc/model" + "github.com/pingcap/tiflow/pkg/config" + "github.com/stretchr/testify/require" +) + +func newTestMockDB(t *testing.T) (db *sql.DB, mock sqlmock.Sqlmock) { + db, mock, err := sqlmock.New(sqlmock.QueryMatcherOption(sqlmock.QueryMatcherEqual)) + require.Nil(t, err) + return +} + +// BasicDispatcher with normal dml cases +func TestBasicDispatcher(t *testing.T) { + db, mock := newTestMockDB(t) + defer db.Close() + + mysqlSink := sink.NewMysqlSink(model.DefaultChangeFeedID("test1"), 8, writer.NewMysqlConfig(), db) + tableSpan := &common.TableSpan{TableSpan: &heartbeatpb.TableSpan{TableID: 1}} + startTs := uint64(100) + + tableSpanStatusChan := make(chan *heartbeatpb.TableSpanStatus, 10) + filter, _ := filter.NewFilter(&config.ReplicaConfig{Filter: &config.FilterConfig{}}, "") + + dispatcher := NewDispatcher(tableSpan, mysqlSink, startTs, tableSpanStatusChan, filter) + + dispatcher.PushTxnEvent(&common.TxnEvent{ + StartTs: 100, + CommitTs: 101, + Rows: []*common.RowChangedEvent{ + { + TableInfo: &common.TableInfo{ + TableName: common.TableName{ + Schema: "test_schema", + Table: "test_table", + }, + }, + Columns: []*common.Column{ + {Name: "id", Value: 1, Flag: common.HandleKeyFlag | common.PrimaryKeyFlag}, + {Name: "name", Value: "Alice"}, + }, + }, + }, + }) + + dispatcher.PushTxnEvent(&common.TxnEvent{ + StartTs: 102, + CommitTs: 105, + Rows: []*common.RowChangedEvent{ + { + TableInfo: &common.TableInfo{ + TableName: common.TableName{ + Schema: "test", + Table: "users", + }, + }, + PreColumns: []*common.Column{ + {Name: "id", Value: 1, Flag: common.HandleKeyFlag | common.PrimaryKeyFlag}, + {Name: "name", Value: "Alice"}, + }, + Columns: []*common.Column{ + {Name: "id", Value: 1, Flag: common.HandleKeyFlag | common.PrimaryKeyFlag}, + {Name: "name", Value: "Bob"}, + }, + }, + }, + }) + dispatcher.UpdateResolvedTs(110) + + heartBeatInfo := &HeartBeatInfo{} + dispatcher.CollectDispatcherHeartBeatInfo(heartBeatInfo) + require.Equal(t, uint64(100), heartBeatInfo.CheckpointTs) + //require.NotEqual(t, 0, tableEventDispatcher.GetMemoryUsage().GetUsedBytes()) + + mock.ExpectBegin() + mock.ExpectExec("INSERT INTO `test_schema`.`test_table` (`id`,`name`) VALUES (?,?);UPDATE `test`.`users` SET `id` = ?, `name` = ? WHERE `id` = ? LIMIT 1"). + WithArgs(1, "Alice", 1, "Bob", 1). + WillReturnResult(sqlmock.NewResult(1, 1)) + mock.ExpectCommit() + + time.Sleep(1 * time.Second) + + err := mock.ExpectationsWereMet() + require.NoError(t, err) + + dispatcher.CollectDispatcherHeartBeatInfo(heartBeatInfo) + require.Equal(t, uint64(110), heartBeatInfo.CheckpointTs) + //require.Equal(t, 0, tableEventDispatcher.GetMemoryUsage().GetUsedBytes()) +} + +func TestDispatcherWithSingleTableDDL(t *testing.T) { + db, mock := newTestMockDB(t) + defer db.Close() + + mock.ExpectBegin() + mock.ExpectExec("USE `test_schema`;"). + WillReturnResult(sqlmock.NewResult(1, 1)) + mock.ExpectExec("ALTER TABLE `test_schema`.`test_table` ADD COLUMN `age` INT"). + WillReturnResult(sqlmock.NewResult(1, 1)) + mock.ExpectCommit() + + mysqlSink := sink.NewMysqlSink(model.DefaultChangeFeedID("test1"), 8, writer.NewMysqlConfig(), db) + tableSpan := &common.TableSpan{TableSpan: &heartbeatpb.TableSpan{TableID: 1}} + startTs := uint64(100) + + tableSpanStatusChan := make(chan *heartbeatpb.TableSpanStatus, 10) + filter, _ := filter.NewFilter(&config.ReplicaConfig{Filter: &config.FilterConfig{}}, "") + + dispatcher := NewDispatcher(tableSpan, mysqlSink, startTs, tableSpanStatusChan, filter) + + dispatcher.PushTxnEvent(&common.TxnEvent{ + StartTs: 102, + CommitTs: 102, + DDLEvent: &common.DDLEvent{ + Job: &timodel.Job{ + Type: timodel.ActionAddColumn, + SchemaName: "test_schema", + TableName: "test_table", + Query: "ALTER TABLE `test_schema`.`test_table` ADD COLUMN `age` INT", + }, + CommitTS: 102, + }, + }) + + time.Sleep(10 * time.Millisecond) + + heartBeatInfo := &HeartBeatInfo{} + dispatcher.CollectDispatcherHeartBeatInfo(heartBeatInfo) + require.Equal(t, uint64(101), heartBeatInfo.CheckpointTs) + + dispatcher.UpdateResolvedTs(110) + + time.Sleep(10 * time.Millisecond) + dispatcher.CollectDispatcherHeartBeatInfo(heartBeatInfo) + require.Equal(t, uint64(110), heartBeatInfo.CheckpointTs) + + err := mock.ExpectationsWereMet() + require.NoError(t, err) +} + +func TestDispatcherWithCrossTableDDL(t *testing.T) { + db, mock := newTestMockDB(t) + defer db.Close() + + mock.ExpectBegin() + mock.ExpectExec("USE `test_schema`;"). + WillReturnResult(sqlmock.NewResult(1, 1)) + mock.ExpectExec("Create table `test_schema`.`test_table` (id int primary key, name varchar(255))"). + WillReturnResult(sqlmock.NewResult(1, 1)) + mock.ExpectCommit() + + mysqlSink := sink.NewMysqlSink(model.DefaultChangeFeedID("test1"), 8, writer.NewMysqlConfig(), db) + tableSpan := &common.DDLSpan + startTs := uint64(100) + + tableSpanStatusChan := make(chan *heartbeatpb.TableSpanStatus, 10) + filter, _ := filter.NewFilter(&config.ReplicaConfig{Filter: &config.FilterConfig{}}, "") + + dispatcher := NewDispatcher(tableSpan, mysqlSink, startTs, tableSpanStatusChan, filter) + + dispatcher.PushTxnEvent(&common.TxnEvent{ + StartTs: 102, + CommitTs: 102, + DDLEvent: &common.DDLEvent{ + Job: &timodel.Job{ + Type: timodel.ActionCreateTable, + SchemaName: "test_schema", + TableName: "test_table", + Query: "Create table `test_schema`.`test_table` (id int primary key, name varchar(255))", + }, + CommitTS: 102, + }, + }) + + time.Sleep(10 * time.Millisecond) + + // 检查可以从 tableSpanStatusChan 中拿到消息 + <-dispatcher.GetTableSpanStatusesChan() + + require.NotEqual(t, dispatcher.ddlPendingEvent, nil) + + // mock maintainer 给 dispatcher 发 通知消息 + dispatcher.GetACKs() <- &heartbeatpb.ACK{CommitTs: 102} + + dispatcher.GetDDLActions() <- &heartbeatpb.DispatcherAction{Action: heartbeatpb.Action_Write, CommitTs: 102} + + time.Sleep(10 * time.Millisecond) + + err := mock.ExpectationsWereMet() + require.NoError(t, err) + + heartBeatInfo := &HeartBeatInfo{} + dispatcher.CollectDispatcherHeartBeatInfo(heartBeatInfo) + require.Equal(t, uint64(101), heartBeatInfo.CheckpointTs) + +} + +func TestDispatcherWithCrossTableDDLAndDML(t *testing.T) { + db, mock := newTestMockDB(t) + defer db.Close() + + mock.ExpectBegin() + mock.ExpectExec("USE `test_schema`;"). + WillReturnResult(sqlmock.NewResult(1, 1)) + mock.ExpectExec("Create table `test_schema`.`test_table` (id int primary key, name varchar(255))"). + WillReturnResult(sqlmock.NewResult(1, 1)) + mock.ExpectCommit() + + mock.ExpectBegin() + mock.ExpectExec("UPDATE `test`.`users` SET `id` = ?, `name` = ? WHERE `id` = ? LIMIT 1"). + WithArgs(1, "Bob", 1). + WillReturnResult(sqlmock.NewResult(1, 1)) + mock.ExpectCommit() + + mysqlSink := sink.NewMysqlSink(model.DefaultChangeFeedID("test1"), 8, writer.NewMysqlConfig(), db) + tableSpan := &common.DDLSpan + startTs := uint64(100) + + tableSpanStatusChan := make(chan *heartbeatpb.TableSpanStatus, 10) + filter, _ := filter.NewFilter(&config.ReplicaConfig{Filter: &config.FilterConfig{}}, "") + + dispatcher := NewDispatcher(tableSpan, mysqlSink, startTs, tableSpanStatusChan, filter) + + dispatcher.PushTxnEvent(&common.TxnEvent{ + StartTs: 102, + CommitTs: 102, + DDLEvent: &common.DDLEvent{ + Job: &timodel.Job{ + Type: timodel.ActionCreateTable, + SchemaName: "test_schema", + TableName: "test_table", + Query: "Create table `test_schema`.`test_table` (id int primary key, name varchar(255))", + }, + CommitTS: 102, + }, + }) + + dispatcher.PushTxnEvent(&common.TxnEvent{ + StartTs: 102, + CommitTs: 105, + Rows: []*common.RowChangedEvent{ + { + TableInfo: &common.TableInfo{ + TableName: common.TableName{ + Schema: "test", + Table: "users", + }, + }, + PreColumns: []*common.Column{ + {Name: "id", Value: 1, Flag: common.HandleKeyFlag | common.PrimaryKeyFlag}, + {Name: "name", Value: "Alice"}, + }, + Columns: []*common.Column{ + {Name: "id", Value: 1, Flag: common.HandleKeyFlag | common.PrimaryKeyFlag}, + {Name: "name", Value: "Bob"}, + }, + }, + }, + }) + + time.Sleep(10 * time.Millisecond) + + // 检查可以从 tableSpanStatusChan 中拿到消息 + <-dispatcher.GetTableSpanStatusesChan() + require.NotEqual(t, dispatcher.ddlPendingEvent, nil) + + heartBeatInfo := &HeartBeatInfo{} + dispatcher.CollectDispatcherHeartBeatInfo(heartBeatInfo) + require.Equal(t, uint64(100), heartBeatInfo.CheckpointTs) + + // mock maintainer 给 dispatcher 发 通知消息 + dispatcher.GetACKs() <- &heartbeatpb.ACK{CommitTs: 102} + + dispatcher.GetDDLActions() <- &heartbeatpb.DispatcherAction{Action: heartbeatpb.Action_Write, CommitTs: 102} + + time.Sleep(30 * time.Millisecond) + + err := mock.ExpectationsWereMet() + require.NoError(t, err) + + dispatcher.CollectDispatcherHeartBeatInfo(heartBeatInfo) + require.Equal(t, uint64(104), heartBeatInfo.CheckpointTs) + +} From 1be424c9589b88b529a2d5d3c79b2241906ed18b Mon Sep 17 00:00:00 2001 From: hongyunyan <649330952@qq.com> Date: Fri, 16 Aug 2024 17:42:51 +0800 Subject: [PATCH 11/12] update --- downstreamadapter/dispatcher/dispatcher.go | 1 - .../dispatchermanager/event_dispatcher_manager.go | 13 ++++++++++--- 2 files changed, 10 insertions(+), 4 deletions(-) diff --git a/downstreamadapter/dispatcher/dispatcher.go b/downstreamadapter/dispatcher/dispatcher.go index 15c82182f..9233bae02 100644 --- a/downstreamadapter/dispatcher/dispatcher.go +++ b/downstreamadapter/dispatcher/dispatcher.go @@ -136,7 +136,6 @@ func (d *Dispatcher) DispatcherEvents(ctx context.Context) { } else if event.IsDDLEvent() { d.AddDDLEventToSinkWhenAvailable(event) } else { - log.Info("update resolved ts") d.resolvedTs.Set(event.ResolvedTs) } } diff --git a/downstreamadapter/dispatchermanager/event_dispatcher_manager.go b/downstreamadapter/dispatchermanager/event_dispatcher_manager.go index 1374d25ea..fa3859b09 100644 --- a/downstreamadapter/dispatchermanager/event_dispatcher_manager.go +++ b/downstreamadapter/dispatchermanager/event_dispatcher_manager.go @@ -232,7 +232,10 @@ func (e *EventDispatcherManager) NewDispatcher(tableSpan *common.TableSpan, star dispatcher := dispatcher.NewDispatcher(tableSpan, e.sink, startTs, e.tableSpanStatusesChan, e.filter) - appcontext.GetService[*eventcollector.EventCollector](appcontext.EventCollector).RegisterDispatcher(dispatcher, startTs, toFilterConfigPB(e.config.Filter)) + // TODO:暂时不收 ddl 的 event + if tableSpan != &common.DDLSpan { + appcontext.GetService[*eventcollector.EventCollector](appcontext.EventCollector).RegisterDispatcher(dispatcher, startTs, toFilterConfigPB(e.config.Filter)) + } e.dispatcherMap.Set(tableSpan, dispatcher) e.GetTableSpanStatusesChan() <- &heartbeatpb.TableSpanStatus{ @@ -296,7 +299,7 @@ func (e *EventDispatcherManager) RemoveDispatcher(tableSpan *common.TableSpan) { dispatcher, ok := e.dispatcherMap.Get(tableSpan) if ok { - if dispatcher.GetRemovingStatus() == true { + if dispatcher.GetRemovingStatus() { return } appcontext.GetService[*eventcollector.EventCollector](appcontext.EventCollector).RemoveDispatcher(dispatcher) @@ -366,12 +369,16 @@ func (e *EventDispatcherManager) CollectHeartbeatInfo(needCompleteStatus bool) * allDispatchers := e.dispatcherMap.GetAllDispatchers() dispatcherHeartBeatInfo := &dispatcher.HeartBeatInfo{} for _, dispatcherItem := range allDispatchers { + // TODO:ddlSpan先不参与 + if dispatcherItem.GetTableSpan() == &common.DDLSpan { + continue + } // If the dispatcher is in removing state, we need to check if it's closed successfully. // If it's closed successfully, we could clean it up. // TODO: we need to consider how to deal with the checkpointTs of the removed dispatcher if the message will be discarded. dispatcherItem.CollectDispatcherHeartBeatInfo(dispatcherHeartBeatInfo) - if dispatcherHeartBeatInfo.IsRemoving == true { + if dispatcherHeartBeatInfo.IsRemoving { watermark, ok := dispatcherItem.TryClose() if ok { // remove successfully From 327672341f13cfa6bb82b17ac46a6b223362a437 Mon Sep 17 00:00:00 2001 From: hongyunyan <649330952@qq.com> Date: Fri, 16 Aug 2024 17:54:15 +0800 Subject: [PATCH 12/12] update --- .../eventcollector/event_collector.go | 24 +++++++++---------- 1 file changed, 12 insertions(+), 12 deletions(-) diff --git a/downstreamadapter/eventcollector/event_collector.go b/downstreamadapter/eventcollector/event_collector.go index ea1b92855..4cdcabf14 100644 --- a/downstreamadapter/eventcollector/event_collector.go +++ b/downstreamadapter/eventcollector/event_collector.go @@ -54,9 +54,9 @@ func (m *DispatcherMap) Delete(dispatcherId string) { } type RegisterInfo struct { - dispatcher *dispatcher.Dispatcher - startTs uint64 - filterConfig *eventpb.FilterConfig + Dispatcher *dispatcher.Dispatcher + StartTs uint64 + FilterConfig *eventpb.FilterConfig } /* @@ -92,10 +92,10 @@ func NewEventCollector(globalMemoryQuota int64, serverId messaging.ServerId) *Ev for { registerInfo := <-eventCollector.registerMessageChan.Out() var err error - if registerInfo.startTs > 0 { + if registerInfo.StartTs > 0 { err = eventCollector.RegisterDispatcher(registerInfo) } else { - err = eventCollector.RemoveDispatcher(registerInfo.dispatcher) + err = eventCollector.RemoveDispatcher(registerInfo.Dispatcher) } if err != nil { // Wait for a while to avoid sending too many requests, since the @@ -116,12 +116,12 @@ func (c *EventCollector) RegisterDispatcher(info RegisterInfo) error { Topic: messaging.EventServiceTopic, Type: messaging.TypeRegisterDispatcherRequest, Message: messaging.RegisterDispatcherRequest{RegisterDispatcherRequest: &eventpb.RegisterDispatcherRequest{ - DispatcherId: info.dispatcher.GetId(), - TableSpan: info.dispatcher.GetTableSpan().TableSpan, + DispatcherId: info.Dispatcher.GetId(), + TableSpan: info.Dispatcher.GetTableSpan().TableSpan, Remove: false, - StartTs: info.startTs, + StartTs: info.StartTs, ServerId: c.serverId.String(), - FilterConfig: info.filterConfig, + FilterConfig: info.FilterConfig, }}, }) if err != nil { @@ -129,7 +129,7 @@ func (c *EventCollector) RegisterDispatcher(info RegisterInfo) error { c.registerMessageChan.In() <- info return err } - c.dispatcherMap.Set(info.dispatcher.GetId(), info.dispatcher) + c.dispatcherMap.Set(info.Dispatcher.GetId(), info.Dispatcher) metrics.EventCollectorRegisteredDispatcherCount.Inc() return nil } @@ -150,8 +150,8 @@ func (c *EventCollector) RemoveDispatcher(d *dispatcher.Dispatcher) error { if err != nil { log.Error("failed to send register dispatcher request message", zap.Error(err)) c.registerMessageChan.In() <- RegisterInfo{ - dispatcher: d, - startTs: 0, + Dispatcher: d, + StartTs: 0, } return err }