From 87c4e59803480e0adf24c0c010a49d7050eee7e2 Mon Sep 17 00:00:00 2001 From: Neil Shen Date: Tue, 24 May 2022 17:50:39 +0800 Subject: [PATCH] tp: implement capture manager (#5562) * tp: refine table schedule proto definition * tp: implement capture status * tp: implement capture manager * tp: adjust Transport.Send * rename TableSchedule to table_schedule Signed-off-by: Neil Shen --- cdc/scheduler/internal/tp/agent.go | 6 +- .../internal/tp/balance_scheduler.go | 3 +- cdc/scheduler/internal/tp/capture_manager.go | 158 ++ .../internal/tp/capture_manager_test.go | 131 ++ cdc/scheduler/internal/tp/capture_status.go | 45 - cdc/scheduler/internal/tp/coordinator.go | 40 +- cdc/scheduler/internal/tp/drain_scheduler.go | 16 - .../internal/tp/replication_manager.go | 59 +- cdc/scheduler/internal/tp/replication_set.go | 102 +- .../internal/tp/replication_set_test.go | 134 +- ...bleSchedule.pb.go => table_schedule.pb.go} | 1444 ++++++++--------- cdc/scheduler/internal/tp/transport.go | 36 +- cdc/scheduler/internal/tp/transport_test.go | 12 +- errors.toml | 10 + ...bleSchedule.proto => table_schedule.proto} | 76 +- scripts/check-diff-line-width.sh | 5 +- scripts/generate-protobuf.sh | 2 +- tools/check/check-errdoc.sh | 2 +- 18 files changed, 1235 insertions(+), 1046 deletions(-) create mode 100644 cdc/scheduler/internal/tp/capture_manager.go create mode 100644 cdc/scheduler/internal/tp/capture_manager_test.go delete mode 100644 cdc/scheduler/internal/tp/capture_status.go delete mode 100644 cdc/scheduler/internal/tp/drain_scheduler.go rename cdc/scheduler/internal/tp/schedulepb/{TableSchedule.pb.go => table_schedule.pb.go} (83%) rename proto/{TableSchedule.proto => table_schedule.proto} (59%) diff --git a/cdc/scheduler/internal/tp/agent.go b/cdc/scheduler/internal/tp/agent.go index 16feab935cd..47f4e7b24d1 100644 --- a/cdc/scheduler/internal/tp/agent.go +++ b/cdc/scheduler/internal/tp/agent.go @@ -44,12 +44,12 @@ func (a *agent) Close() error { } func (a *agent) handleMessage(msg []*schedulepb.Message) { - // s.handleMessageAnnounce() + // s.handleMessageHeartbeat() // s.handleMessageDispatchTableRequest() } -func (a *agent) handleMessageAnnounce(msg *schedulepb.Sync) { - // TODO: build s.tables from Sync message. +func (a *agent) handleMessageHeartbeat(msg *schedulepb.Heartbeat) { + // TODO: build s.tables from Heartbeat message. } func (a *agent) handleMessageDispatchTableRequest(msg *schedulepb.DispatchTableResponse) { diff --git a/cdc/scheduler/internal/tp/balance_scheduler.go b/cdc/scheduler/internal/tp/balance_scheduler.go index 906874ff645..1166930dcba 100644 --- a/cdc/scheduler/internal/tp/balance_scheduler.go +++ b/cdc/scheduler/internal/tp/balance_scheduler.go @@ -19,6 +19,7 @@ var _ scheduler = &balancer{} type balancer struct{} +//nolint:deadcode func newBalancer() *balancer { return nil } @@ -31,7 +32,7 @@ func (b *balancer) Schedule( checkpointTs model.Ts, currentTables []model.TableID, captures map[model.CaptureID]*model.CaptureInfo, - captureTables map[model.CaptureID]captureStatus, + captureTables map[model.CaptureID]*CaptureStatus, ) []*scheduleTask { return nil } diff --git a/cdc/scheduler/internal/tp/capture_manager.go b/cdc/scheduler/internal/tp/capture_manager.go new file mode 100644 index 00000000000..241a6100331 --- /dev/null +++ b/cdc/scheduler/internal/tp/capture_manager.go @@ -0,0 +1,158 @@ +// 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 tp + +import ( + "github.com/pingcap/log" + "github.com/pingcap/tiflow/cdc/model" + "github.com/pingcap/tiflow/cdc/scheduler/internal/tp/schedulepb" + "go.uber.org/zap" +) + +// CaptureState is the state of a capture. +// +// ┌──────────────┐ Heartbeat Resp ┌─────────────┐ +// │ Uninitialize ├───────────────>│ Initialized │ +// └──────┬───────┘ └──────┬──────┘ +// │ │ +// IsStopping │ ┌──────────┐ │ IsStopping +// └────────> │ Stopping │ <───────┘ +// └──────────┘ +type CaptureState int + +const ( + // CaptureStateUninitialize means the capture status is unknown, + // no heartbeat response received yet. + CaptureStateUninitialize CaptureState = 1 + // CaptureStateInitialized means owner has received heartbeat response. + CaptureStateInitialized CaptureState = 2 + // CaptureStateStopping means the capture is removing, e.g., shutdown. + CaptureStateStopping CaptureState = 3 +) + +// CaptureStatus represent captrue's status. +type CaptureStatus struct { + OwnerRev schedulepb.OwnerRevision + Epoch schedulepb.ProcessorEpoch + State CaptureState +} + +func newCaptureStatus(rev schedulepb.OwnerRevision) *CaptureStatus { + return &CaptureStatus{OwnerRev: rev, State: CaptureStateUninitialize} +} + +func (c *CaptureStatus) handleHeartbeatResponse( + resp *schedulepb.HeartbeatResponse, epoch schedulepb.ProcessorEpoch, +) { + // Check epoch for initialized captures. + if c.State != CaptureStateUninitialize && c.Epoch.Epoch != epoch.Epoch { + log.Warn("tpscheduler: ignore heartbeat response", + zap.String("epoch", c.Epoch.Epoch), + zap.String("respEpoch", epoch.Epoch), + zap.Int64("ownerRev", c.OwnerRev.Revision)) + return + } + + if c.State == CaptureStateUninitialize { + c.Epoch = epoch + c.State = CaptureStateInitialized + } + if resp.IsStopping { + c.State = CaptureStateStopping + } +} + +type captureManager struct { + OwnerRev schedulepb.OwnerRevision + Captures map[model.CaptureID]*CaptureStatus + + // A logical clock counter, for heartbeat. + tickCounter int + heartbeatTick int +} + +func newCaptureManager(rev schedulepb.OwnerRevision, heartbeatTick int) *captureManager { + return &captureManager{ + OwnerRev: rev, + Captures: make(map[string]*CaptureStatus), + heartbeatTick: heartbeatTick, + } +} + +func (c *captureManager) captureTableSets() map[model.CaptureID]*CaptureStatus { + return c.Captures +} + +func (c *captureManager) checkAllCaptureInitialized() bool { + for _, captrueStatus := range c.Captures { + if captrueStatus.State == CaptureStateUninitialize { + return false + } + } + return true +} + +func (c *captureManager) tick() []*schedulepb.Message { + c.tickCounter++ + if c.tickCounter < c.heartbeatTick { + return nil + } + c.tickCounter = 0 + msgs := make([]*schedulepb.Message, 0, len(c.Captures)) + for to := range c.Captures { + msgs = append(msgs, &schedulepb.Message{ + To: to, + MsgType: schedulepb.MsgHeartbeat, + Heartbeat: &schedulepb.Heartbeat{}, + }) + } + return msgs +} + +func (c *captureManager) poll( + aliveCaptures map[model.CaptureID]*model.CaptureInfo, + msgs []*schedulepb.Message, +) ([]*schedulepb.Message, bool) { + outMsgs := c.onAliveCaptureUpdate(aliveCaptures) + for _, msg := range msgs { + if msg.MsgType == schedulepb.MsgHeartbeatResponse { + captureStatus, ok := c.Captures[msg.From] + if !ok { + continue + } + captureStatus.handleHeartbeatResponse( + msg.GetHeartbeatResponse(), msg.Header.ProcessorEpoch) + } + } + return outMsgs, c.checkAllCaptureInitialized() +} + +func (c *captureManager) onAliveCaptureUpdate( + aliveCaptures map[model.CaptureID]*model.CaptureInfo, +) []*schedulepb.Message { + msgs := make([]*schedulepb.Message, 0) + for id := range aliveCaptures { + if _, ok := c.Captures[id]; !ok { + // A new capture. + c.Captures[id] = newCaptureStatus(c.OwnerRev) + log.Info("tpscheduler: find a new capture", zap.String("newCapture", id)) + msgs = append(msgs, &schedulepb.Message{ + To: id, + MsgType: schedulepb.MsgHeartbeat, + Heartbeat: &schedulepb.Heartbeat{}, + }) + } + } + return msgs +} diff --git a/cdc/scheduler/internal/tp/capture_manager_test.go b/cdc/scheduler/internal/tp/capture_manager_test.go new file mode 100644 index 00000000000..b151c463140 --- /dev/null +++ b/cdc/scheduler/internal/tp/capture_manager_test.go @@ -0,0 +1,131 @@ +// 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 tp + +import ( + "testing" + + "github.com/pingcap/tiflow/cdc/model" + "github.com/pingcap/tiflow/cdc/scheduler/internal/tp/schedulepb" + "github.com/stretchr/testify/require" +) + +func TestCaptureStatusHandleHeartbeatResponse(t *testing.T) { + t.Parallel() + + rev := schedulepb.OwnerRevision{Revision: 1} + epoch := schedulepb.ProcessorEpoch{Epoch: "test"} + c := newCaptureStatus(rev) + require.Equal(t, CaptureStateUninitialize, c.State) + + // Uninitialize -> Initialized + c.handleHeartbeatResponse(&schedulepb.HeartbeatResponse{}, epoch) + require.Equal(t, CaptureStateInitialized, c.State) + require.Equal(t, epoch, c.Epoch) + + // Processor epoch mismatch + c.handleHeartbeatResponse(&schedulepb.HeartbeatResponse{ + IsStopping: true, + }, schedulepb.ProcessorEpoch{Epoch: "unknown"}) + require.Equal(t, CaptureStateInitialized, c.State) + + // Initialized -> Stopping + c.handleHeartbeatResponse(&schedulepb.HeartbeatResponse{IsStopping: true}, epoch) + require.Equal(t, CaptureStateStopping, c.State) + require.Equal(t, epoch, c.Epoch) +} + +func TestCaptureManagerPoll(t *testing.T) { + t.Parallel() + + rev := schedulepb.OwnerRevision{} + ms := map[model.CaptureID]*model.CaptureInfo{ + "1": {}, + "2": {}, + } + cm := newCaptureManager(rev, 2) + + // Initial poll for alive captures. + msgs, hasInit := cm.poll(ms, nil) + require.False(t, hasInit) + require.ElementsMatch(t, []*schedulepb.Message{ + {To: "1", MsgType: schedulepb.MsgHeartbeat, Heartbeat: &schedulepb.Heartbeat{}}, + {To: "2", MsgType: schedulepb.MsgHeartbeat, Heartbeat: &schedulepb.Heartbeat{}}, + }, msgs) + + // Poll one response + msgs, hasInit = cm.poll(ms, []*schedulepb.Message{ + { + Header: &schedulepb.Message_Header{}, From: "1", + MsgType: schedulepb.MsgHeartbeatResponse, + HeartbeatResponse: &schedulepb.HeartbeatResponse{}, + }, + }) + require.False(t, hasInit) + require.Empty(t, msgs) + + // Poll another response + msgs, hasInit = cm.poll(ms, []*schedulepb.Message{ + { + Header: &schedulepb.Message_Header{}, From: "2", + MsgType: schedulepb.MsgHeartbeatResponse, + HeartbeatResponse: &schedulepb.HeartbeatResponse{}, + }, + }) + require.True(t, hasInit, "%v %v", cm.Captures["1"], cm.Captures["2"]) + require.Empty(t, msgs) +} + +func TestCaptureManagerTick(t *testing.T) { + t.Parallel() + + rev := schedulepb.OwnerRevision{} + cm := newCaptureManager(rev, 2) + + // No heartbeat if there is no capture. + msgs := cm.tick() + require.Empty(t, msgs) + msgs = cm.tick() + require.Empty(t, msgs) + + ms := map[model.CaptureID]*model.CaptureInfo{ + "1": {}, + "2": {}, + } + _, hasInit := cm.poll(ms, nil) + require.False(t, hasInit) + + // Heartbeat even if capture is uninitialize. + msgs = cm.tick() + require.Empty(t, msgs) + msgs = cm.tick() + require.ElementsMatch(t, []*schedulepb.Message{ + {To: "1", MsgType: schedulepb.MsgHeartbeat, Heartbeat: &schedulepb.Heartbeat{}}, + {To: "2", MsgType: schedulepb.MsgHeartbeat, Heartbeat: &schedulepb.Heartbeat{}}, + }, msgs) + + // Heartbeat even if capture is initialized or stopping. + for _, s := range []CaptureState{CaptureStateInitialized, CaptureStateStopping} { + cm.Captures["1"].State = s + cm.Captures["2"].State = s + require.True(t, cm.checkAllCaptureInitialized()) + msgs = cm.tick() + require.Empty(t, msgs) + msgs = cm.tick() + require.ElementsMatch(t, []*schedulepb.Message{ + {To: "1", MsgType: schedulepb.MsgHeartbeat, Heartbeat: &schedulepb.Heartbeat{}}, + {To: "2", MsgType: schedulepb.MsgHeartbeat, Heartbeat: &schedulepb.Heartbeat{}}, + }, msgs) + } +} diff --git a/cdc/scheduler/internal/tp/capture_status.go b/cdc/scheduler/internal/tp/capture_status.go deleted file mode 100644 index 367bb8ff60f..00000000000 --- a/cdc/scheduler/internal/tp/capture_status.go +++ /dev/null @@ -1,45 +0,0 @@ -// 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 tp - -import ( - "github.com/pingcap/tiflow/cdc/model" - "github.com/pingcap/tiflow/cdc/scheduler/internal/tp/schedulepb" -) - -// captureState is the state of a capture. -// -// ┌────────┐ onSync() ┌────────┐ -// Absent ─> │ UnSync ├─────────>│ Synced │ -// └───┬────┘ └────┬───┘ -// │ │ -// onReject() │ ┌──────────┐ │ onReject() -// └──> │ Rejected │ <──┘ -// └──────────┘ -type captureState int - -const ( - captureStateUnSync captureState = 1 - captureStateSynced captureState = 2 - captureStateRejcted captureState = 3 -) - -type captureStatus struct { - state captureState - // tableSet is a set of table ID that is in a capture. - tableSet map[model.TableID]schedulepb.TableState -} - -func (c *captureStatus) onSync(sync *schedulepb.Sync) {} -func (c *captureStatus) onReject(sync *schedulepb.Sync) {} diff --git a/cdc/scheduler/internal/tp/coordinator.go b/cdc/scheduler/internal/tp/coordinator.go index cf7785c1e7e..4c62b049a49 100644 --- a/cdc/scheduler/internal/tp/coordinator.go +++ b/cdc/scheduler/internal/tp/coordinator.go @@ -19,7 +19,6 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/tiflow/cdc/model" "github.com/pingcap/tiflow/cdc/scheduler/internal" - "github.com/pingcap/tiflow/cdc/scheduler/internal/tp/schedulepb" ) type scheduler interface { @@ -28,17 +27,17 @@ type scheduler interface { checkpointTs model.Ts, currentTables []model.TableID, aliveCaptures map[model.CaptureID]*model.CaptureInfo, - captureTables map[model.CaptureID]captureStatus, + captureTables map[model.CaptureID]*CaptureStatus, ) []*scheduleTask } var _ internal.Scheduler = (*coordinator)(nil) type coordinator struct { - trans transport - manager *replicationManager - // balancer and drainer - scheduler []scheduler + trans transport + scheduler []scheduler + replicationM *replicationManager + captureM *captureManager } func (c *coordinator) Tick( @@ -69,28 +68,33 @@ func (c *coordinator) poll( ctx context.Context, checkpointTs model.Ts, currentTables []model.TableID, aliveCaptures map[model.CaptureID]*model.CaptureInfo, ) error { - captureTables := c.manager.captureTableSets() + recvMsgs, err := c.trans.Recv(ctx) + if err != nil { + return errors.Trace(err) + } + sentMsgs, hasInit := c.captureM.poll(aliveCaptures, recvMsgs) + if !hasInit { + err := c.trans.Send(ctx, sentMsgs) + return errors.Trace(err) + } + + captureTables := c.captureM.captureTableSets() allTasks := make([]*scheduleTask, 0) for _, sched := range c.scheduler { tasks := sched.Schedule(checkpointTs, currentTables, aliveCaptures, captureTables) allTasks = append(allTasks, tasks...) } - recvMsgs := c.recvMessages() - sentMsgs, err := c.manager.poll( + msgs, err := c.replicationM.poll( ctx, checkpointTs, currentTables, aliveCaptures, recvMsgs, allTasks) if err != nil { return errors.Trace(err) } - c.sendMessages(sentMsgs) + sentMsgs = append(sentMsgs, msgs...) + err = c.trans.Send(ctx, sentMsgs) + if err != nil { + return errors.Trace(err) + } // checkpoint calcuation return nil } - -func (c *coordinator) recvMessages() []*schedulepb.Message { - return nil -} - -func (c *coordinator) sendMessages(msgs []*schedulepb.Message) error { - return nil -} diff --git a/cdc/scheduler/internal/tp/drain_scheduler.go b/cdc/scheduler/internal/tp/drain_scheduler.go deleted file mode 100644 index f6f3bae6167..00000000000 --- a/cdc/scheduler/internal/tp/drain_scheduler.go +++ /dev/null @@ -1,16 +0,0 @@ -// 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 tp - -type drainer struct{} diff --git a/cdc/scheduler/internal/tp/replication_manager.go b/cdc/scheduler/internal/tp/replication_manager.go index 16c51c29ca5..6f2f480c05a 100644 --- a/cdc/scheduler/internal/tp/replication_manager.go +++ b/cdc/scheduler/internal/tp/replication_manager.go @@ -16,6 +16,7 @@ package tp import ( "context" + "github.com/pingcap/errors" "github.com/pingcap/tiflow/cdc/model" "github.com/pingcap/tiflow/cdc/scheduler/internal/tp/schedulepb" ) @@ -24,6 +25,7 @@ type callback func(model.TableID) // burstBalance for changefeed set up or unplaned TiCDC node failure. // TiCDC needs to balance intrrupted tables as soon as possible. +//nolint:deadcode type burstBalance struct { tables map[model.TableID]model.CaptureID @@ -59,72 +61,81 @@ type scheduleTask struct { } type replicationManager struct { - tables map[model.TableID]ReplicationSet + version string + tables map[model.TableID]*ReplicationSet runningTasks map[model.TableID]*scheduleTask - captures map[model.CaptureID]captureStatus } -func (s *replicationManager) captureTableSets() map[model.CaptureID]captureStatus { - return s.captures -} - -func (s *replicationManager) poll( +func (r *replicationManager) poll( ctx context.Context, // Latest global checkpoint of the changefeed checkpointTs model.Ts, // All tables that SHOULD be replicated (or started) at the current checkpoint. currentTables []model.TableID, // All captures that are alive according to the latest Etcd states. - captures map[model.CaptureID]*model.CaptureInfo, + aliveCaptures map[model.CaptureID]*model.CaptureInfo, msgs []*schedulepb.Message, tasks []*scheduleTask, ) ([]*schedulepb.Message, error) { - // s.handleMessage(msgs) - // - // s.handleTasks(tasks) - // - // s.sendMessages(msgs) + msgBuf := make([]*schedulepb.Message, 0) - return nil, nil + sendMsgs, err := r.handleMessage(msgs) + if err != nil { + return nil, errors.Trace(err) + } + msgBuf = append(msgBuf, sendMsgs...) + + sendMsgs, err = r.handleMessage(msgs) + if err != nil { + return nil, errors.Trace(err) + } + msgBuf = append(msgBuf, sendMsgs...) + + return msgBuf, nil } -func (s *replicationManager) handleMessage(msg []*schedulepb.Message) { - // s.handleMessageSync() +func (r *replicationManager) handleMessage( + msg []*schedulepb.Message, +) ([]*schedulepb.Message, error) { + // s.handleMessageHeartbeat() // s.handleMessageCheckpoint() // s.handleMessageDispatchTableResponse() + return nil, nil } -func (s *replicationManager) handleMessageSync(msg *schedulepb.Sync) { - // TODO: build s.tables from Sync message. +func (r *replicationManager) handleMessageHeartbeat(msg *schedulepb.Heartbeat) { + // TODO: build s.tables from Heartbeat message. } -func (s *replicationManager) handleMessageDispatchTableResponse(msg *schedulepb.DispatchTableResponse) { +func (r *replicationManager) handleMessageDispatchTableResponse( + msg *schedulepb.DispatchTableResponse, +) { // TODO: update s.tables from DispatchTableResponse message. } -func (s *replicationManager) handleMessageCheckpoint(msg *schedulepb.Checkpoint) { +func (r *replicationManager) handleMessageCheckpoint(msg *schedulepb.Checkpoint) { // TODO: update s.tables from Checkpoint message. } // ======== -func (s *replicationManager) handleTasks(tasks []*scheduleTask) { +func (r *replicationManager) handleTasks(tasks []*scheduleTask) { // s.handleTaskAddTable(nil) // s.handleTaskMoveTable(nil) // s.handleTaskDeleteTable(nil) } -func (s *replicationManager) handleTaskMoveTable(task *moveTable) error { +func (r *replicationManager) handleTaskMoveTable(task *moveTable) error { // TODO: update s.runingTasks and s.tables. return nil } -func (s *replicationManager) handleTaskAddTable(task *addTable) error { +func (r *replicationManager) handleTaskAddTable(task *addTable) error { // TODO: update s.runingTasks and s.tables. return nil } -func (s *replicationManager) handleTaskDeleteTable(task *deleteTable) error { +func (r *replicationManager) handleTaskDeleteTable(task *deleteTable) error { // TODO: update s.runingTasks and s.tables. return nil } diff --git a/cdc/scheduler/internal/tp/replication_set.go b/cdc/scheduler/internal/tp/replication_set.go index 554f2d508e6..aae2fb52bcb 100644 --- a/cdc/scheduler/internal/tp/replication_set.go +++ b/cdc/scheduler/internal/tp/replication_set.go @@ -97,7 +97,7 @@ func newReplicationSet( } switch table.State { - case schedulepb.TableState_Replicating: + case schedulepb.TableStateReplicating: // Recognize primary if it's table is in replicating state. if len(r.Primary) == 0 { r.Primary = captureID @@ -107,18 +107,18 @@ func newReplicationSet( table, captureID, "tpscheduler: multiple primary", zap.Any("status", tableStatus)) } - case schedulepb.TableState_Preparing: + case schedulepb.TableStatePreparing: // Recognize secondary if it's table is in preparing state. r.Secondary = captureID r.Captures[captureID] = struct{}{} - case schedulepb.TableState_Prepared: + case schedulepb.TableStatePrepared: // Recognize secondary and Commit state if it's table is in prepared state. committed = true r.Secondary = captureID r.Captures[captureID] = struct{}{} - case schedulepb.TableState_Absent, - schedulepb.TableState_Stopping, - schedulepb.TableState_Stopped: + case schedulepb.TableStateAbsent, + schedulepb.TableStateStopping, + schedulepb.TableStateStopped: // Ignore stop state. default: log.Warn("tpscheduler: unknown table state", @@ -154,7 +154,8 @@ func (r *ReplicationSet) inconsistentError( fields = append(fields, []zap.Field{ zap.String("captureID", captureID), zap.Stringer("tableState", input), - zap.Any("replicationSet", r)}...) + zap.Any("replicationSet", r), + }...) log.L().WithOptions(zap.AddCallerSkip(1)).Error(msg, fields...) return cerror.ErrReplicationSetInconsistent.GenWithStackByArgs( fmt.Sprintf("tableID %d, %s", r.TableID, msg)) @@ -166,7 +167,8 @@ func (r *ReplicationSet) multiplePrimaryError( fields = append(fields, []zap.Field{ zap.String("captureID", captureID), zap.Stringer("tableState", input), - zap.Any("replicationSet", r)}...) + zap.Any("replicationSet", r), + }...) log.L().WithOptions(zap.AddCallerSkip(1)).Error(msg, fields...) return cerror.ErrReplicationSetMultiplePrimaryError.GenWithStackByArgs( fmt.Sprintf("tableID %d, %s", r.TableID, msg)) @@ -220,7 +222,7 @@ func (r *ReplicationSet) poll( zap.String("captureID", captureID), zap.Any("replicationSet", r)) return []*schedulepb.Message{{ To: captureID, - MsgType: schedulepb.MessageType_MsgDispatchTableRequest, + MsgType: schedulepb.MsgDispatchTableRequest, DispatchTableRequest: &schedulepb.DispatchTableRequest{ RemoveTable: &schedulepb.RemoveTableRequest{TableID: r.TableID}, }, @@ -268,7 +270,7 @@ func (r *ReplicationSet) pollOnAbsent( input *schedulepb.TableStatus, captureID model.CaptureID, ) (*schedulepb.Message, bool, error) { switch input.State { - case schedulepb.TableState_Absent: + case schedulepb.TableStateAbsent: if r.Primary != "" || r.Secondary != "" { return nil, false, r.inconsistentError( input, captureID, "tpscheduler: there must be no primary or secondary") @@ -277,11 +279,11 @@ func (r *ReplicationSet) pollOnAbsent( r.Secondary = captureID return nil, true, nil - case schedulepb.TableState_Preparing, - schedulepb.TableState_Prepared, - schedulepb.TableState_Replicating, - schedulepb.TableState_Stopping, - schedulepb.TableState_Stopped: + case schedulepb.TableStatePreparing, + schedulepb.TableStatePrepared, + schedulepb.TableStateReplicating, + schedulepb.TableStateStopping, + schedulepb.TableStateStopped: } log.Warn("tpscheduler: ingore input, unexpected replication set state", zap.Stringer("tableState", input), @@ -294,13 +296,13 @@ func (r *ReplicationSet) pollOnPrepare( input *schedulepb.TableStatus, captureID model.CaptureID, ) (*schedulepb.Message, bool, error) { switch input.State { - case schedulepb.TableState_Absent, - schedulepb.TableState_Stopping, - schedulepb.TableState_Stopped: + case schedulepb.TableStateAbsent, + schedulepb.TableStateStopping, + schedulepb.TableStateStopped: if r.Secondary == captureID { return &schedulepb.Message{ To: captureID, - MsgType: schedulepb.MessageType_MsgDispatchTableRequest, + MsgType: schedulepb.MsgDispatchTableRequest, DispatchTableRequest: &schedulepb.DispatchTableRequest{ AddTable: &schedulepb.AddTableRequest{ TableID: r.TableID, @@ -310,18 +312,18 @@ func (r *ReplicationSet) pollOnPrepare( }, }, false, nil } - case schedulepb.TableState_Preparing: + case schedulepb.TableStatePreparing: if r.Secondary == captureID { // Ignore secondary Preparing, it may take a long time. return nil, false, nil } - case schedulepb.TableState_Prepared: + case schedulepb.TableStatePrepared: if r.Secondary == captureID { // Secondary is prepared, transit to Commit state. r.State = ReplicationSetStateCommit return nil, true, nil } - case schedulepb.TableState_Replicating: + case schedulepb.TableStateReplicating: if r.Primary == captureID { return nil, false, nil } @@ -337,14 +339,14 @@ func (r *ReplicationSet) pollOnCommit( input *schedulepb.TableStatus, captureID model.CaptureID, ) (*schedulepb.Message, bool, error) { switch input.State { - case schedulepb.TableState_Prepared: + case schedulepb.TableStatePrepared: if r.Secondary == captureID { if r.Primary != "" { // Secondary capture is prepared and waiting for stopping primary. // Send message to primary, ask for stopping. return &schedulepb.Message{ To: r.Primary, - MsgType: schedulepb.MessageType_MsgDispatchTableRequest, + MsgType: schedulepb.MsgDispatchTableRequest, DispatchTableRequest: &schedulepb.DispatchTableRequest{ RemoveTable: &schedulepb.RemoveTableRequest{TableID: r.TableID}, }, @@ -360,7 +362,7 @@ func (r *ReplicationSet) pollOnCommit( zap.String("captureID", captureID)) return &schedulepb.Message{ To: captureID, - MsgType: schedulepb.MessageType_MsgDispatchTableRequest, + MsgType: schedulepb.MsgDispatchTableRequest, DispatchTableRequest: &schedulepb.DispatchTableRequest{ AddTable: &schedulepb.AddTableRequest{ TableID: r.TableID, @@ -370,7 +372,7 @@ func (r *ReplicationSet) pollOnCommit( }, }, false, nil } - case schedulepb.TableState_Stopped, schedulepb.TableState_Absent: + case schedulepb.TableStateStopped, schedulepb.TableStateAbsent: if r.Primary == captureID && r.Secondary != "" { // Primary is stopped, promote secondary to primary. original := r.Primary @@ -382,7 +384,7 @@ func (r *ReplicationSet) pollOnCommit( zap.String("captureID", captureID)) return &schedulepb.Message{ To: r.Primary, - MsgType: schedulepb.MessageType_MsgDispatchTableRequest, + MsgType: schedulepb.MsgDispatchTableRequest, DispatchTableRequest: &schedulepb.DispatchTableRequest{ AddTable: &schedulepb.AddTableRequest{ TableID: r.TableID, @@ -393,12 +395,12 @@ func (r *ReplicationSet) pollOnCommit( }, false, nil } - case schedulepb.TableState_Replicating: + case schedulepb.TableStateReplicating: if r.Secondary != "" && r.Primary == captureID { // Original primary is not stopped, ask for stopping. return &schedulepb.Message{ To: captureID, - MsgType: schedulepb.MessageType_MsgDispatchTableRequest, + MsgType: schedulepb.MsgDispatchTableRequest, DispatchTableRequest: &schedulepb.DispatchTableRequest{ RemoveTable: &schedulepb.RemoveTableRequest{ TableID: r.TableID, @@ -416,11 +418,11 @@ func (r *ReplicationSet) pollOnCommit( return nil, false, r.multiplePrimaryError( input, captureID, "tpscheduler: multiple primary") - case schedulepb.TableState_Stopping: + case schedulepb.TableStateStopping: if r.Primary == captureID && r.Secondary != "" { return nil, false, nil } - case schedulepb.TableState_Preparing: + case schedulepb.TableStatePreparing: } log.Warn("tpscheduler: ingore input, unexpected replication set state", zap.Stringer("tableState", input), @@ -433,18 +435,18 @@ func (r *ReplicationSet) pollOnReplicating( input *schedulepb.TableStatus, captureID model.CaptureID, ) (*schedulepb.Message, bool, error) { switch input.State { - case schedulepb.TableState_Replicating: + case schedulepb.TableStateReplicating: if r.Primary == captureID { return nil, false, nil } return nil, false, r.multiplePrimaryError( input, captureID, "tpscheduler: multiple primary") - case schedulepb.TableState_Absent: - case schedulepb.TableState_Preparing: - case schedulepb.TableState_Prepared: - case schedulepb.TableState_Stopping: - case schedulepb.TableState_Stopped: + case schedulepb.TableStateAbsent: + case schedulepb.TableStatePreparing: + case schedulepb.TableStatePrepared: + case schedulepb.TableStateStopping: + case schedulepb.TableStateStopped: } log.Warn("tpscheduler: ingore input, unexpected replication set state", zap.Stringer("tableState", input), @@ -457,18 +459,18 @@ func (r *ReplicationSet) pollOnRemoving( input *schedulepb.TableStatus, captureID model.CaptureID, ) (*schedulepb.Message, bool, error) { switch input.State { - case schedulepb.TableState_Absent, - schedulepb.TableState_Preparing, - schedulepb.TableState_Prepared, - schedulepb.TableState_Replicating: + case schedulepb.TableStateAbsent, + schedulepb.TableStatePreparing, + schedulepb.TableStatePrepared, + schedulepb.TableStateReplicating: return &schedulepb.Message{ To: captureID, - MsgType: schedulepb.MessageType_MsgDispatchTableRequest, + MsgType: schedulepb.MsgDispatchTableRequest, DispatchTableRequest: &schedulepb.DispatchTableRequest{ RemoveTable: &schedulepb.RemoveTableRequest{TableID: r.TableID}, }, }, false, nil - case schedulepb.TableState_Stopped: + case schedulepb.TableStateStopped: if r.Primary == captureID { r.Primary = "" } else if r.Secondary == captureID { @@ -479,7 +481,7 @@ func (r *ReplicationSet) pollOnRemoving( zap.Stringer("tableState", input), zap.String("captureID", captureID)) return nil, false, nil - case schedulepb.TableState_Stopping: + case schedulepb.TableStateStopping: return nil, false, nil } log.Warn("tpscheduler: ingore input, unexpected replication set state", @@ -511,8 +513,8 @@ func (r *ReplicationSet) handleAddTable( r.Captures[captureID] = struct{}{} status := &schedulepb.TableStatus{ TableID: r.TableID, - State: schedulepb.TableState_Absent, - Checkpoint: &schedulepb.Checkpoint{}, + State: schedulepb.TableStateAbsent, + Checkpoint: schedulepb.Checkpoint{}, } return r.poll(status, captureID) } @@ -540,8 +542,8 @@ func (r *ReplicationSet) handleMoveTable( r.Captures[dest] = struct{}{} status := &schedulepb.TableStatus{ TableID: r.TableID, - State: schedulepb.TableState_Absent, - Checkpoint: &schedulepb.Checkpoint{}, + State: schedulepb.TableStateAbsent, + Checkpoint: schedulepb.Checkpoint{}, } return r.poll(status, r.Secondary) } @@ -565,8 +567,8 @@ func (r *ReplicationSet) handleRemoveTable() ([]*schedulepb.Message, error) { zap.Stringer("old", oldState), zap.Stringer("new", r.State)) status := &schedulepb.TableStatus{ TableID: r.TableID, - State: schedulepb.TableState_Replicating, - Checkpoint: &schedulepb.Checkpoint{CheckpointTs: r.CheckpointTs}, + State: schedulepb.TableStateReplicating, + Checkpoint: schedulepb.Checkpoint{CheckpointTs: r.CheckpointTs}, } return r.poll(status, r.Primary) } diff --git a/cdc/scheduler/internal/tp/replication_set_test.go b/cdc/scheduler/internal/tp/replication_set_test.go index b95386d61a3..043a14751b8 100644 --- a/cdc/scheduler/internal/tp/replication_set_test.go +++ b/cdc/scheduler/internal/tp/replication_set_test.go @@ -67,8 +67,10 @@ func TestNewReplicationSet(t *testing.T) { Captures: map[string]struct{}{"1": {}}, }, tableStatus: map[model.CaptureID]*schedulepb.TableStatus{ - "1": {State: schedulepb.TableState_Replicating, - Checkpoint: &schedulepb.Checkpoint{}}, + "1": { + State: schedulepb.TableStateReplicating, + Checkpoint: schedulepb.Checkpoint{}, + }, }, }, { @@ -79,8 +81,10 @@ func TestNewReplicationSet(t *testing.T) { Captures: map[string]struct{}{"1": {}}, }, tableStatus: map[model.CaptureID]*schedulepb.TableStatus{ - "1": {State: schedulepb.TableState_Preparing, - Checkpoint: &schedulepb.Checkpoint{}}, + "1": { + State: schedulepb.TableStatePreparing, + Checkpoint: schedulepb.Checkpoint{}, + }, }, }, { @@ -92,10 +96,14 @@ func TestNewReplicationSet(t *testing.T) { Captures: map[string]struct{}{"1": {}, "2": {}}, }, tableStatus: map[model.CaptureID]*schedulepb.TableStatus{ - "1": {State: schedulepb.TableState_Preparing, - Checkpoint: &schedulepb.Checkpoint{}}, - "2": {State: schedulepb.TableState_Replicating, - Checkpoint: &schedulepb.Checkpoint{}}, + "1": { + State: schedulepb.TableStatePreparing, + Checkpoint: schedulepb.Checkpoint{}, + }, + "2": { + State: schedulepb.TableStateReplicating, + Checkpoint: schedulepb.Checkpoint{}, + }, }, }, { @@ -107,10 +115,14 @@ func TestNewReplicationSet(t *testing.T) { Captures: map[string]struct{}{"1": {}, "2": {}}, }, tableStatus: map[model.CaptureID]*schedulepb.TableStatus{ - "1": {State: schedulepb.TableState_Prepared, - Checkpoint: &schedulepb.Checkpoint{}}, - "2": {State: schedulepb.TableState_Replicating, - Checkpoint: &schedulepb.Checkpoint{}}, + "1": { + State: schedulepb.TableStatePrepared, + Checkpoint: schedulepb.Checkpoint{}, + }, + "2": { + State: schedulepb.TableStateReplicating, + Checkpoint: schedulepb.Checkpoint{}, + }, }, }, { @@ -121,10 +133,14 @@ func TestNewReplicationSet(t *testing.T) { Captures: map[string]struct{}{"1": {}}, }, tableStatus: map[model.CaptureID]*schedulepb.TableStatus{ - "1": {State: schedulepb.TableState_Prepared, - Checkpoint: &schedulepb.Checkpoint{}}, - "2": {State: schedulepb.TableState_Stopping, - Checkpoint: &schedulepb.Checkpoint{}}, + "1": { + State: schedulepb.TableStatePrepared, + Checkpoint: schedulepb.Checkpoint{}, + }, + "2": { + State: schedulepb.TableStateStopping, + Checkpoint: schedulepb.Checkpoint{}, + }, }, }, { @@ -135,10 +151,14 @@ func TestNewReplicationSet(t *testing.T) { Captures: map[string]struct{}{"1": {}}, }, tableStatus: map[model.CaptureID]*schedulepb.TableStatus{ - "1": {State: schedulepb.TableState_Prepared, - Checkpoint: &schedulepb.Checkpoint{}}, - "2": {State: schedulepb.TableState_Stopped, - Checkpoint: &schedulepb.Checkpoint{}}, + "1": { + State: schedulepb.TableStatePrepared, + Checkpoint: schedulepb.Checkpoint{}, + }, + "2": { + State: schedulepb.TableStateStopped, + Checkpoint: schedulepb.Checkpoint{}, + }, }, }, { @@ -148,20 +168,28 @@ func TestNewReplicationSet(t *testing.T) { Captures: map[string]struct{}{}, }, tableStatus: map[model.CaptureID]*schedulepb.TableStatus{ - "1": {State: schedulepb.TableState_Stopping, - Checkpoint: &schedulepb.Checkpoint{}}, - "2": {State: schedulepb.TableState_Stopping, - Checkpoint: &schedulepb.Checkpoint{}}, + "1": { + State: schedulepb.TableStateStopping, + Checkpoint: schedulepb.Checkpoint{}, + }, + "2": { + State: schedulepb.TableStateStopping, + Checkpoint: schedulepb.Checkpoint{}, + }, }, }, { // Multiple primary error. set: nil, tableStatus: map[model.CaptureID]*schedulepb.TableStatus{ - "1": {State: schedulepb.TableState_Replicating, - Checkpoint: &schedulepb.Checkpoint{}}, - "2": {State: schedulepb.TableState_Replicating, - Checkpoint: &schedulepb.Checkpoint{}}, + "1": { + State: schedulepb.TableStateReplicating, + Checkpoint: schedulepb.Checkpoint{}, + }, + "2": { + State: schedulepb.TableStateReplicating, + Checkpoint: schedulepb.Checkpoint{}, + }, }, }, } @@ -184,7 +212,7 @@ func TestReplicationSetPoll(t *testing.T) { var testcases []map[string]schedulepb.TableState for state1 := range schedulepb.TableState_name { for state2 := range schedulepb.TableState_name { - if state1 == state2 && state1 == int32(schedulepb.TableState_Replicating) { + if state1 == state2 && state1 == int32(schedulepb.TableStateReplicating) { continue } tc := map[string]schedulepb.TableState{ @@ -210,7 +238,7 @@ func TestReplicationSetPoll(t *testing.T) { status[id] = &schedulepb.TableStatus{ TableID: 1, State: state, - Checkpoint: &schedulepb.Checkpoint{}, + Checkpoint: schedulepb.Checkpoint{}, } } r, err := newReplicationSet(1, status) @@ -257,7 +285,7 @@ func TestReplicationSetAddTable(t *testing.T) { require.Len(t, msgs, 1) require.EqualValues(t, &schedulepb.Message{ To: from, - MsgType: schedulepb.MessageType_MsgDispatchTableRequest, + MsgType: schedulepb.MsgDispatchTableRequest, DispatchTableRequest: &schedulepb.DispatchTableRequest{ AddTable: &schedulepb.AddTableRequest{ TableID: r.TableID, @@ -277,13 +305,13 @@ func TestReplicationSetAddTable(t *testing.T) { // AddTableRequest is lost somehow, send AddTableRequest again. msgs, err = r.handleTableStatus(from, &schedulepb.TableStatus{ TableID: tableID, - State: schedulepb.TableState_Absent, + State: schedulepb.TableStateAbsent, }) require.Nil(t, err) require.Len(t, msgs, 1) require.EqualValues(t, &schedulepb.Message{ To: from, - MsgType: schedulepb.MessageType_MsgDispatchTableRequest, + MsgType: schedulepb.MsgDispatchTableRequest, DispatchTableRequest: &schedulepb.DispatchTableRequest{ AddTable: &schedulepb.AddTableRequest{ TableID: r.TableID, @@ -298,7 +326,7 @@ func TestReplicationSetAddTable(t *testing.T) { // Prepare is in-progress. msgs, err = r.handleTableStatus(from, &schedulepb.TableStatus{ TableID: tableID, - State: schedulepb.TableState_Preparing, + State: schedulepb.TableStatePreparing, }) require.Nil(t, err) require.Len(t, msgs, 0) @@ -308,13 +336,13 @@ func TestReplicationSetAddTable(t *testing.T) { // Prepare -> Commit. msgs, err = r.handleTableStatus(from, &schedulepb.TableStatus{ TableID: tableID, - State: schedulepb.TableState_Prepared, + State: schedulepb.TableStatePrepared, }) require.Nil(t, err) require.Len(t, msgs, 1) require.EqualValues(t, &schedulepb.Message{ To: from, - MsgType: schedulepb.MessageType_MsgDispatchTableRequest, + MsgType: schedulepb.MsgDispatchTableRequest, DispatchTableRequest: &schedulepb.DispatchTableRequest{ AddTable: &schedulepb.AddTableRequest{ TableID: r.TableID, @@ -330,7 +358,7 @@ func TestReplicationSetAddTable(t *testing.T) { // Commit -> Replicating msgs, err = r.handleTableStatus(from, &schedulepb.TableStatus{ TableID: tableID, - State: schedulepb.TableState_Replicating, + State: schedulepb.TableStateReplicating, }) require.Nil(t, err) require.Len(t, msgs, 0) @@ -362,7 +390,7 @@ func TestReplicationSetRemoveTable(t *testing.T) { require.Len(t, msgs, 1) require.EqualValues(t, &schedulepb.Message{ To: from, - MsgType: schedulepb.MessageType_MsgDispatchTableRequest, + MsgType: schedulepb.MsgDispatchTableRequest, DispatchTableRequest: &schedulepb.DispatchTableRequest{ RemoveTable: &schedulepb.RemoveTableRequest{ TableID: r.TableID, @@ -380,7 +408,7 @@ func TestReplicationSetRemoveTable(t *testing.T) { // Removing is in-progress. msgs, err = r.handleTableStatus(from, &schedulepb.TableStatus{ TableID: tableID, - State: schedulepb.TableState_Stopping, + State: schedulepb.TableStateStopping, }) require.Nil(t, err) require.Len(t, msgs, 0) @@ -390,7 +418,7 @@ func TestReplicationSetRemoveTable(t *testing.T) { // Removed msgs, err = r.handleTableStatus(from, &schedulepb.TableStatus{ TableID: tableID, - State: schedulepb.TableState_Stopped, + State: schedulepb.TableStateStopped, }) require.Nil(t, err) require.Len(t, msgs, 0) @@ -426,7 +454,7 @@ func TestReplicationSetMoveTable(t *testing.T) { require.Len(t, msgs, 1) require.EqualValues(t, &schedulepb.Message{ To: dest, - MsgType: schedulepb.MessageType_MsgDispatchTableRequest, + MsgType: schedulepb.MsgDispatchTableRequest, DispatchTableRequest: &schedulepb.DispatchTableRequest{ AddTable: &schedulepb.AddTableRequest{ TableID: r.TableID, @@ -447,13 +475,13 @@ func TestReplicationSetMoveTable(t *testing.T) { // AddTableRequest is lost somehow, send AddTableRequest again. msgs, err = r.handleTableStatus(dest, &schedulepb.TableStatus{ TableID: tableID, - State: schedulepb.TableState_Absent, + State: schedulepb.TableStateAbsent, }) require.Nil(t, err) require.Len(t, msgs, 1) require.EqualValues(t, &schedulepb.Message{ To: dest, - MsgType: schedulepb.MessageType_MsgDispatchTableRequest, + MsgType: schedulepb.MsgDispatchTableRequest, DispatchTableRequest: &schedulepb.DispatchTableRequest{ AddTable: &schedulepb.AddTableRequest{ TableID: r.TableID, @@ -468,13 +496,13 @@ func TestReplicationSetMoveTable(t *testing.T) { // Prepare -> Commit. msgs, err = r.handleTableStatus(dest, &schedulepb.TableStatus{ TableID: tableID, - State: schedulepb.TableState_Prepared, + State: schedulepb.TableStatePrepared, }) require.Nil(t, err) require.Len(t, msgs, 1) require.EqualValues(t, &schedulepb.Message{ To: original, - MsgType: schedulepb.MessageType_MsgDispatchTableRequest, + MsgType: schedulepb.MsgDispatchTableRequest, DispatchTableRequest: &schedulepb.DispatchTableRequest{ RemoveTable: &schedulepb.RemoveTableRequest{ TableID: r.TableID, @@ -488,13 +516,13 @@ func TestReplicationSetMoveTable(t *testing.T) { // Source updates it's table status msgs, err = r.handleTableStatus(original, &schedulepb.TableStatus{ TableID: tableID, - State: schedulepb.TableState_Replicating, + State: schedulepb.TableStateReplicating, }) require.Nil(t, err) require.Len(t, msgs, 1, "%v", r) require.EqualValues(t, &schedulepb.Message{ To: original, - MsgType: schedulepb.MessageType_MsgDispatchTableRequest, + MsgType: schedulepb.MsgDispatchTableRequest, DispatchTableRequest: &schedulepb.DispatchTableRequest{ RemoveTable: &schedulepb.RemoveTableRequest{ TableID: r.TableID, @@ -508,7 +536,7 @@ func TestReplicationSetMoveTable(t *testing.T) { // Removing source is in-progress. msgs, err = r.handleTableStatus(original, &schedulepb.TableStatus{ TableID: tableID, - State: schedulepb.TableState_Stopping, + State: schedulepb.TableStateStopping, }) require.Nil(t, err) require.Len(t, msgs, 0) @@ -520,13 +548,13 @@ func TestReplicationSetMoveTable(t *testing.T) { rSnapshot := *r msgs, err = r.handleTableStatus(original, &schedulepb.TableStatus{ TableID: tableID, - State: schedulepb.TableState_Stopped, + State: schedulepb.TableStateStopped, }) require.Nil(t, err) require.Len(t, msgs, 1) require.EqualValues(t, &schedulepb.Message{ To: dest, - MsgType: schedulepb.MessageType_MsgDispatchTableRequest, + MsgType: schedulepb.MsgDispatchTableRequest, DispatchTableRequest: &schedulepb.DispatchTableRequest{ AddTable: &schedulepb.AddTableRequest{ TableID: r.TableID, @@ -542,13 +570,13 @@ func TestReplicationSetMoveTable(t *testing.T) { // Source stopped message is lost somehow. msgs, err = rSnapshot.handleTableStatus(original, &schedulepb.TableStatus{ TableID: tableID, - State: schedulepb.TableState_Absent, + State: schedulepb.TableStateAbsent, }) require.Nil(t, err) require.Len(t, msgs, 1) require.EqualValues(t, &schedulepb.Message{ To: dest, - MsgType: schedulepb.MessageType_MsgDispatchTableRequest, + MsgType: schedulepb.MsgDispatchTableRequest, DispatchTableRequest: &schedulepb.DispatchTableRequest{ AddTable: &schedulepb.AddTableRequest{ TableID: r.TableID, @@ -564,7 +592,7 @@ func TestReplicationSetMoveTable(t *testing.T) { // Commit -> Replicating msgs, err = r.handleTableStatus(dest, &schedulepb.TableStatus{ TableID: tableID, - State: schedulepb.TableState_Replicating, + State: schedulepb.TableStateReplicating, }) require.Nil(t, err) require.Len(t, msgs, 0) diff --git a/cdc/scheduler/internal/tp/schedulepb/TableSchedule.pb.go b/cdc/scheduler/internal/tp/schedulepb/table_schedule.pb.go similarity index 83% rename from cdc/scheduler/internal/tp/schedulepb/TableSchedule.pb.go rename to cdc/scheduler/internal/tp/schedulepb/table_schedule.pb.go index 43b0b948e93..363e494c466 100644 --- a/cdc/scheduler/internal/tp/schedulepb/TableSchedule.pb.go +++ b/cdc/scheduler/internal/tp/schedulepb/table_schedule.pb.go @@ -1,5 +1,5 @@ // Code generated by protoc-gen-gogo. DO NOT EDIT. -// source: TableSchedule.proto +// source: table_schedule.proto package schedulepb @@ -36,13 +36,13 @@ const _ = proto.GoGoProtoPackageIsVersion3 // please upgrade the proto package type TableState int32 const ( - TableState_Unknown TableState = 0 - TableState_Absent TableState = 1 - TableState_Preparing TableState = 2 - TableState_Prepared TableState = 3 - TableState_Replicating TableState = 4 - TableState_Stopping TableState = 5 - TableState_Stopped TableState = 6 + TableStateUnknown TableState = 0 + TableStateAbsent TableState = 1 + TableStatePreparing TableState = 2 + TableStatePrepared TableState = 3 + TableStateReplicating TableState = 4 + TableStateStopping TableState = 5 + TableStateStopped TableState = 6 ) var TableState_name = map[int32]string{ @@ -70,18 +70,18 @@ func (x TableState) String() string { } func (TableState) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_5b343da219629f69, []int{0} + return fileDescriptor_ab4bb9c6b16cfa4d, []int{0} } type MessageType int32 const ( - MessageType_MsgUnknown MessageType = 0 - MessageType_MsgCheckpoint MessageType = 1 - MessageType_MsgDispatchTableRequest MessageType = 2 - MessageType_MsgDispatchTableResponse MessageType = 3 - MessageType_MsgAnnounce MessageType = 4 - MessageType_MsgSync MessageType = 5 + MsgUnknown MessageType = 0 + MsgCheckpoint MessageType = 1 + MsgDispatchTableRequest MessageType = 2 + MsgDispatchTableResponse MessageType = 3 + MsgHeartbeat MessageType = 4 + MsgHeartbeatResponse MessageType = 5 ) var MessageType_name = map[int32]string{ @@ -89,8 +89,8 @@ var MessageType_name = map[int32]string{ 1: "MsgCheckpoint", 2: "MsgDispatchTableRequest", 3: "MsgDispatchTableResponse", - 4: "MsgAnnounce", - 5: "MsgSync", + 4: "MsgHeartbeat", + 5: "MsgHeartbeatResponse", } var MessageType_value = map[string]int32{ @@ -98,8 +98,8 @@ var MessageType_value = map[string]int32{ "MsgCheckpoint": 1, "MsgDispatchTableRequest": 2, "MsgDispatchTableResponse": 3, - "MsgAnnounce": 4, - "MsgSync": 5, + "MsgHeartbeat": 4, + "MsgHeartbeatResponse": 5, } func (x MessageType) String() string { @@ -107,95 +107,7 @@ func (x MessageType) String() string { } func (MessageType) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_5b343da219629f69, []int{1} -} - -type OwnerRevision struct { - Revision int64 `protobuf:"varint,1,opt,name=revision,proto3" json:"revision,omitempty"` -} - -func (m *OwnerRevision) Reset() { *m = OwnerRevision{} } -func (m *OwnerRevision) String() string { return proto.CompactTextString(m) } -func (*OwnerRevision) ProtoMessage() {} -func (*OwnerRevision) Descriptor() ([]byte, []int) { - return fileDescriptor_5b343da219629f69, []int{0} -} -func (m *OwnerRevision) XXX_Unmarshal(b []byte) error { - return m.Unmarshal(b) -} -func (m *OwnerRevision) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { - if deterministic { - return xxx_messageInfo_OwnerRevision.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 *OwnerRevision) XXX_Merge(src proto.Message) { - xxx_messageInfo_OwnerRevision.Merge(m, src) -} -func (m *OwnerRevision) XXX_Size() int { - return m.Size() -} -func (m *OwnerRevision) XXX_DiscardUnknown() { - xxx_messageInfo_OwnerRevision.DiscardUnknown(m) -} - -var xxx_messageInfo_OwnerRevision proto.InternalMessageInfo - -func (m *OwnerRevision) GetRevision() int64 { - if m != nil { - return m.Revision - } - return 0 -} - -type ProcessorEpoch struct { - Epoch string `protobuf:"bytes,1,opt,name=epoch,proto3" json:"epoch,omitempty"` -} - -func (m *ProcessorEpoch) Reset() { *m = ProcessorEpoch{} } -func (m *ProcessorEpoch) String() string { return proto.CompactTextString(m) } -func (*ProcessorEpoch) ProtoMessage() {} -func (*ProcessorEpoch) Descriptor() ([]byte, []int) { - return fileDescriptor_5b343da219629f69, []int{1} -} -func (m *ProcessorEpoch) XXX_Unmarshal(b []byte) error { - return m.Unmarshal(b) -} -func (m *ProcessorEpoch) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { - if deterministic { - return xxx_messageInfo_ProcessorEpoch.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 *ProcessorEpoch) XXX_Merge(src proto.Message) { - xxx_messageInfo_ProcessorEpoch.Merge(m, src) -} -func (m *ProcessorEpoch) XXX_Size() int { - return m.Size() -} -func (m *ProcessorEpoch) XXX_DiscardUnknown() { - xxx_messageInfo_ProcessorEpoch.DiscardUnknown(m) -} - -var xxx_messageInfo_ProcessorEpoch proto.InternalMessageInfo - -func (m *ProcessorEpoch) GetEpoch() string { - if m != nil { - return m.Epoch - } - return "" + return fileDescriptor_ab4bb9c6b16cfa4d, []int{1} } type Checkpoint struct { @@ -207,7 +119,7 @@ func (m *Checkpoint) Reset() { *m = Checkpoint{} } func (m *Checkpoint) String() string { return proto.CompactTextString(m) } func (*Checkpoint) ProtoMessage() {} func (*Checkpoint) Descriptor() ([]byte, []int) { - return fileDescriptor_5b343da219629f69, []int{2} + return fileDescriptor_ab4bb9c6b16cfa4d, []int{0} } func (m *Checkpoint) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -260,7 +172,7 @@ func (m *AddTableRequest) Reset() { *m = AddTableRequest{} } func (m *AddTableRequest) String() string { return proto.CompactTextString(m) } func (*AddTableRequest) ProtoMessage() {} func (*AddTableRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_5b343da219629f69, []int{3} + return fileDescriptor_ab4bb9c6b16cfa4d, []int{1} } func (m *AddTableRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -318,7 +230,7 @@ func (m *RemoveTableRequest) Reset() { *m = RemoveTableRequest{} } func (m *RemoveTableRequest) String() string { return proto.CompactTextString(m) } func (*RemoveTableRequest) ProtoMessage() {} func (*RemoveTableRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_5b343da219629f69, []int{4} + return fileDescriptor_ab4bb9c6b16cfa4d, []int{2} } func (m *RemoveTableRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -355,16 +267,15 @@ func (m *RemoveTableRequest) GetTableID() github_com_pingcap_tiflow_cdc_model.Ta } type DispatchTableRequest struct { - OwnerRevision *OwnerRevision `protobuf:"bytes,1,opt,name=owner_revision,json=ownerRevision,proto3" json:"owner_revision,omitempty"` - AddTable *AddTableRequest `protobuf:"bytes,2,opt,name=add_table,json=addTable,proto3" json:"add_table,omitempty"` - RemoveTable *RemoveTableRequest `protobuf:"bytes,3,opt,name=remove_table,json=removeTable,proto3" json:"remove_table,omitempty"` + AddTable *AddTableRequest `protobuf:"bytes,1,opt,name=add_table,json=addTable,proto3" json:"add_table,omitempty"` + RemoveTable *RemoveTableRequest `protobuf:"bytes,2,opt,name=remove_table,json=removeTable,proto3" json:"remove_table,omitempty"` } func (m *DispatchTableRequest) Reset() { *m = DispatchTableRequest{} } func (m *DispatchTableRequest) String() string { return proto.CompactTextString(m) } func (*DispatchTableRequest) ProtoMessage() {} func (*DispatchTableRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_5b343da219629f69, []int{5} + return fileDescriptor_ab4bb9c6b16cfa4d, []int{3} } func (m *DispatchTableRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -393,13 +304,6 @@ func (m *DispatchTableRequest) XXX_DiscardUnknown() { var xxx_messageInfo_DispatchTableRequest proto.InternalMessageInfo -func (m *DispatchTableRequest) GetOwnerRevision() *OwnerRevision { - if m != nil { - return m.OwnerRevision - } - return nil -} - func (m *DispatchTableRequest) GetAddTable() *AddTableRequest { if m != nil { return m.AddTable @@ -425,7 +329,7 @@ func (m *AddTableResponse) Reset() { *m = AddTableResponse{} } func (m *AddTableResponse) String() string { return proto.CompactTextString(m) } func (*AddTableResponse) ProtoMessage() {} func (*AddTableResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_5b343da219629f69, []int{6} + return fileDescriptor_ab4bb9c6b16cfa4d, []int{4} } func (m *AddTableResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -492,7 +396,7 @@ func (m *RemoveTableResponse) Reset() { *m = RemoveTableResponse{} } func (m *RemoveTableResponse) String() string { return proto.CompactTextString(m) } func (*RemoveTableResponse) ProtoMessage() {} func (*RemoveTableResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_5b343da219629f69, []int{7} + return fileDescriptor_ab4bb9c6b16cfa4d, []int{5} } func (m *RemoveTableResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -543,16 +447,15 @@ func (m *RemoveTableResponse) GetCheckpoint() *Checkpoint { } type DispatchTableResponse struct { - ProcessorEpoch *ProcessorEpoch `protobuf:"bytes,1,opt,name=processor_epoch,json=processorEpoch,proto3" json:"processor_epoch,omitempty"` - AddTable *AddTableResponse `protobuf:"bytes,2,opt,name=add_table,json=addTable,proto3" json:"add_table,omitempty"` - RemoveTable *RemoveTableResponse `protobuf:"bytes,3,opt,name=remove_table,json=removeTable,proto3" json:"remove_table,omitempty"` + AddTable *AddTableResponse `protobuf:"bytes,1,opt,name=add_table,json=addTable,proto3" json:"add_table,omitempty"` + RemoveTable *RemoveTableResponse `protobuf:"bytes,2,opt,name=remove_table,json=removeTable,proto3" json:"remove_table,omitempty"` } func (m *DispatchTableResponse) Reset() { *m = DispatchTableResponse{} } func (m *DispatchTableResponse) String() string { return proto.CompactTextString(m) } func (*DispatchTableResponse) ProtoMessage() {} func (*DispatchTableResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_5b343da219629f69, []int{8} + return fileDescriptor_ab4bb9c6b16cfa4d, []int{6} } func (m *DispatchTableResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -581,13 +484,6 @@ func (m *DispatchTableResponse) XXX_DiscardUnknown() { var xxx_messageInfo_DispatchTableResponse proto.InternalMessageInfo -func (m *DispatchTableResponse) GetProcessorEpoch() *ProcessorEpoch { - if m != nil { - return m.ProcessorEpoch - } - return nil -} - func (m *DispatchTableResponse) GetAddTable() *AddTableResponse { if m != nil { return m.AddTable @@ -602,22 +498,21 @@ func (m *DispatchTableResponse) GetRemoveTable() *RemoveTableResponse { return nil } -type Announce struct { - OwnerRevision *OwnerRevision `protobuf:"bytes,1,opt,name=owner_revision,json=ownerRevision,proto3" json:"owner_revision,omitempty"` +type Heartbeat struct { } -func (m *Announce) Reset() { *m = Announce{} } -func (m *Announce) String() string { return proto.CompactTextString(m) } -func (*Announce) ProtoMessage() {} -func (*Announce) Descriptor() ([]byte, []int) { - return fileDescriptor_5b343da219629f69, []int{9} +func (m *Heartbeat) Reset() { *m = Heartbeat{} } +func (m *Heartbeat) String() string { return proto.CompactTextString(m) } +func (*Heartbeat) ProtoMessage() {} +func (*Heartbeat) Descriptor() ([]byte, []int) { + return fileDescriptor_ab4bb9c6b16cfa4d, []int{7} } -func (m *Announce) XXX_Unmarshal(b []byte) error { +func (m *Heartbeat) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) } -func (m *Announce) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { +func (m *Heartbeat) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { if deterministic { - return xxx_messageInfo_Announce.Marshal(b, m, deterministic) + return xxx_messageInfo_Heartbeat.Marshal(b, m, deterministic) } else { b = b[:cap(b)] n, err := m.MarshalToSizedBuffer(b) @@ -627,36 +522,29 @@ func (m *Announce) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { return b[:n], nil } } -func (m *Announce) XXX_Merge(src proto.Message) { - xxx_messageInfo_Announce.Merge(m, src) +func (m *Heartbeat) XXX_Merge(src proto.Message) { + xxx_messageInfo_Heartbeat.Merge(m, src) } -func (m *Announce) XXX_Size() int { +func (m *Heartbeat) XXX_Size() int { return m.Size() } -func (m *Announce) XXX_DiscardUnknown() { - xxx_messageInfo_Announce.DiscardUnknown(m) +func (m *Heartbeat) XXX_DiscardUnknown() { + xxx_messageInfo_Heartbeat.DiscardUnknown(m) } -var xxx_messageInfo_Announce proto.InternalMessageInfo - -func (m *Announce) GetOwnerRevision() *OwnerRevision { - if m != nil { - return m.OwnerRevision - } - return nil -} +var xxx_messageInfo_Heartbeat proto.InternalMessageInfo type TableStatus struct { TableID github_com_pingcap_tiflow_cdc_model.TableID `protobuf:"varint,1,opt,name=table_id,json=tableId,proto3,casttype=github.com/pingcap/tiflow/cdc/model.TableID" json:"table_id,omitempty"` State TableState `protobuf:"varint,2,opt,name=state,proto3,enum=pingcap.tiflow.cdc.schedulepb.TableState" json:"state,omitempty"` - Checkpoint *Checkpoint `protobuf:"bytes,3,opt,name=checkpoint,proto3" json:"checkpoint,omitempty"` + Checkpoint Checkpoint `protobuf:"bytes,3,opt,name=checkpoint,proto3" json:"checkpoint"` } func (m *TableStatus) Reset() { *m = TableStatus{} } func (m *TableStatus) String() string { return proto.CompactTextString(m) } func (*TableStatus) ProtoMessage() {} func (*TableStatus) Descriptor() ([]byte, []int) { - return fileDescriptor_5b343da219629f69, []int{10} + return fileDescriptor_ab4bb9c6b16cfa4d, []int{8} } func (m *TableStatus) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -696,33 +584,33 @@ func (m *TableStatus) GetState() TableState { if m != nil { return m.State } - return TableState_Unknown + return TableStateUnknown } -func (m *TableStatus) GetCheckpoint() *Checkpoint { +func (m *TableStatus) GetCheckpoint() Checkpoint { if m != nil { return m.Checkpoint } - return nil + return Checkpoint{} } -type Sync struct { - ProcessorEpoch *ProcessorEpoch `protobuf:"bytes,1,opt,name=processor_epoch,json=processorEpoch,proto3" json:"processor_epoch,omitempty"` - Tables []*TableStatus `protobuf:"bytes,2,rep,name=tables,proto3" json:"tables,omitempty"` +type HeartbeatResponse struct { + Tables []TableStatus `protobuf:"bytes,1,rep,name=tables,proto3" json:"tables"` + IsStopping bool `protobuf:"varint,2,opt,name=is_stopping,json=isStopping,proto3" json:"is_stopping,omitempty"` } -func (m *Sync) Reset() { *m = Sync{} } -func (m *Sync) String() string { return proto.CompactTextString(m) } -func (*Sync) ProtoMessage() {} -func (*Sync) Descriptor() ([]byte, []int) { - return fileDescriptor_5b343da219629f69, []int{11} +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_ab4bb9c6b16cfa4d, []int{9} } -func (m *Sync) XXX_Unmarshal(b []byte) error { +func (m *HeartbeatResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) } -func (m *Sync) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { +func (m *HeartbeatResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { if deterministic { - return xxx_messageInfo_Sync.Marshal(b, m, deterministic) + return xxx_messageInfo_HeartbeatResponse.Marshal(b, m, deterministic) } else { b = b[:cap(b)] n, err := m.MarshalToSizedBuffer(b) @@ -732,30 +620,118 @@ func (m *Sync) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { return b[:n], nil } } -func (m *Sync) XXX_Merge(src proto.Message) { - xxx_messageInfo_Sync.Merge(m, src) +func (m *HeartbeatResponse) XXX_Merge(src proto.Message) { + xxx_messageInfo_HeartbeatResponse.Merge(m, src) } -func (m *Sync) XXX_Size() int { +func (m *HeartbeatResponse) XXX_Size() int { return m.Size() } -func (m *Sync) XXX_DiscardUnknown() { - xxx_messageInfo_Sync.DiscardUnknown(m) +func (m *HeartbeatResponse) XXX_DiscardUnknown() { + xxx_messageInfo_HeartbeatResponse.DiscardUnknown(m) } -var xxx_messageInfo_Sync proto.InternalMessageInfo +var xxx_messageInfo_HeartbeatResponse proto.InternalMessageInfo -func (m *Sync) GetProcessorEpoch() *ProcessorEpoch { +func (m *HeartbeatResponse) GetTables() []TableStatus { if m != nil { - return m.ProcessorEpoch + return m.Tables } return nil } -func (m *Sync) GetTables() []*TableStatus { +func (m *HeartbeatResponse) GetIsStopping() bool { if m != nil { - return m.Tables + return m.IsStopping } - return nil + return false +} + +type OwnerRevision struct { + Revision int64 `protobuf:"varint,1,opt,name=revision,proto3" json:"revision,omitempty"` +} + +func (m *OwnerRevision) Reset() { *m = OwnerRevision{} } +func (m *OwnerRevision) String() string { return proto.CompactTextString(m) } +func (*OwnerRevision) ProtoMessage() {} +func (*OwnerRevision) Descriptor() ([]byte, []int) { + return fileDescriptor_ab4bb9c6b16cfa4d, []int{10} +} +func (m *OwnerRevision) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *OwnerRevision) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_OwnerRevision.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 *OwnerRevision) XXX_Merge(src proto.Message) { + xxx_messageInfo_OwnerRevision.Merge(m, src) +} +func (m *OwnerRevision) XXX_Size() int { + return m.Size() +} +func (m *OwnerRevision) XXX_DiscardUnknown() { + xxx_messageInfo_OwnerRevision.DiscardUnknown(m) +} + +var xxx_messageInfo_OwnerRevision proto.InternalMessageInfo + +func (m *OwnerRevision) GetRevision() int64 { + if m != nil { + return m.Revision + } + return 0 +} + +type ProcessorEpoch struct { + Epoch string `protobuf:"bytes,1,opt,name=epoch,proto3" json:"epoch,omitempty"` +} + +func (m *ProcessorEpoch) Reset() { *m = ProcessorEpoch{} } +func (m *ProcessorEpoch) String() string { return proto.CompactTextString(m) } +func (*ProcessorEpoch) ProtoMessage() {} +func (*ProcessorEpoch) Descriptor() ([]byte, []int) { + return fileDescriptor_ab4bb9c6b16cfa4d, []int{11} +} +func (m *ProcessorEpoch) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *ProcessorEpoch) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_ProcessorEpoch.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 *ProcessorEpoch) XXX_Merge(src proto.Message) { + xxx_messageInfo_ProcessorEpoch.Merge(m, src) +} +func (m *ProcessorEpoch) XXX_Size() int { + return m.Size() +} +func (m *ProcessorEpoch) XXX_DiscardUnknown() { + xxx_messageInfo_ProcessorEpoch.DiscardUnknown(m) +} + +var xxx_messageInfo_ProcessorEpoch proto.InternalMessageInfo + +func (m *ProcessorEpoch) GetEpoch() string { + if m != nil { + return m.Epoch + } + return "" } type Message struct { @@ -765,16 +741,16 @@ type Message struct { To github_com_pingcap_tiflow_cdc_model.CaptureID `protobuf:"bytes,4,opt,name=to,proto3,casttype=github.com/pingcap/tiflow/cdc/model.CaptureID" json:"to,omitempty"` DispatchTableRequest *DispatchTableRequest `protobuf:"bytes,5,opt,name=dispatch_table_request,json=dispatchTableRequest,proto3" json:"dispatch_table_request,omitempty"` DispatchTableResponse *DispatchTableResponse `protobuf:"bytes,6,opt,name=dispatch_table_response,json=dispatchTableResponse,proto3" json:"dispatch_table_response,omitempty"` - Checkpoint *Checkpoint `protobuf:"bytes,7,opt,name=checkpoint,proto3" json:"checkpoint,omitempty"` - Announce *Announce `protobuf:"bytes,8,opt,name=announce,proto3" json:"announce,omitempty"` - Sync *Sync `protobuf:"bytes,9,opt,name=sync,proto3" json:"sync,omitempty"` + Heartbeat *Heartbeat `protobuf:"bytes,7,opt,name=heartbeat,proto3" json:"heartbeat,omitempty"` + HeartbeatResponse *HeartbeatResponse `protobuf:"bytes,8,opt,name=heartbeat_response,json=heartbeatResponse,proto3" json:"heartbeat_response,omitempty"` + Checkpoint *Checkpoint `protobuf:"bytes,9,opt,name=checkpoint,proto3" json:"checkpoint,omitempty"` } func (m *Message) Reset() { *m = Message{} } func (m *Message) String() string { return proto.CompactTextString(m) } func (*Message) ProtoMessage() {} func (*Message) Descriptor() ([]byte, []int) { - return fileDescriptor_5b343da219629f69, []int{12} + return fileDescriptor_ab4bb9c6b16cfa4d, []int{12} } func (m *Message) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -814,7 +790,7 @@ func (m *Message) GetMsgType() MessageType { if m != nil { return m.MsgType } - return MessageType_MsgUnknown + return MsgUnknown } func (m *Message) GetFrom() github_com_pingcap_tiflow_cdc_model.CaptureID { @@ -845,37 +821,39 @@ func (m *Message) GetDispatchTableResponse() *DispatchTableResponse { return nil } -func (m *Message) GetCheckpoint() *Checkpoint { +func (m *Message) GetHeartbeat() *Heartbeat { if m != nil { - return m.Checkpoint + return m.Heartbeat } return nil } -func (m *Message) GetAnnounce() *Announce { +func (m *Message) GetHeartbeatResponse() *HeartbeatResponse { if m != nil { - return m.Announce + return m.HeartbeatResponse } return nil } -func (m *Message) GetSync() *Sync { +func (m *Message) GetCheckpoint() *Checkpoint { if m != nil { - return m.Sync + return m.Checkpoint } return nil } type Message_Header struct { // The semantic version of the node that sent this message. - Version string `protobuf:"bytes,1,opt,name=version,proto3" json:"version,omitempty"` + Version string `protobuf:"bytes,1,opt,name=version,proto3" json:"version,omitempty"` + OwnerRevision OwnerRevision `protobuf:"bytes,2,opt,name=owner_revision,json=ownerRevision,proto3" json:"owner_revision"` + ProcessorEpoch ProcessorEpoch `protobuf:"bytes,3,opt,name=processor_epoch,json=processorEpoch,proto3" json:"processor_epoch"` } func (m *Message_Header) Reset() { *m = Message_Header{} } func (m *Message_Header) String() string { return proto.CompactTextString(m) } func (*Message_Header) ProtoMessage() {} func (*Message_Header) Descriptor() ([]byte, []int) { - return fileDescriptor_5b343da219629f69, []int{12, 0} + return fileDescriptor_ab4bb9c6b16cfa4d, []int{12, 0} } func (m *Message_Header) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -911,11 +889,23 @@ func (m *Message_Header) GetVersion() string { return "" } +func (m *Message_Header) GetOwnerRevision() OwnerRevision { + if m != nil { + return m.OwnerRevision + } + return OwnerRevision{} +} + +func (m *Message_Header) GetProcessorEpoch() ProcessorEpoch { + if m != nil { + return m.ProcessorEpoch + } + return ProcessorEpoch{} +} + func init() { proto.RegisterEnum("pingcap.tiflow.cdc.schedulepb.TableState", TableState_name, TableState_value) proto.RegisterEnum("pingcap.tiflow.cdc.schedulepb.MessageType", MessageType_name, MessageType_value) - proto.RegisterType((*OwnerRevision)(nil), "pingcap.tiflow.cdc.schedulepb.OwnerRevision") - proto.RegisterType((*ProcessorEpoch)(nil), "pingcap.tiflow.cdc.schedulepb.ProcessorEpoch") proto.RegisterType((*Checkpoint)(nil), "pingcap.tiflow.cdc.schedulepb.Checkpoint") proto.RegisterType((*AddTableRequest)(nil), "pingcap.tiflow.cdc.schedulepb.AddTableRequest") proto.RegisterType((*RemoveTableRequest)(nil), "pingcap.tiflow.cdc.schedulepb.RemoveTableRequest") @@ -923,82 +913,92 @@ func init() { proto.RegisterType((*AddTableResponse)(nil), "pingcap.tiflow.cdc.schedulepb.AddTableResponse") proto.RegisterType((*RemoveTableResponse)(nil), "pingcap.tiflow.cdc.schedulepb.RemoveTableResponse") proto.RegisterType((*DispatchTableResponse)(nil), "pingcap.tiflow.cdc.schedulepb.DispatchTableResponse") - proto.RegisterType((*Announce)(nil), "pingcap.tiflow.cdc.schedulepb.Announce") + proto.RegisterType((*Heartbeat)(nil), "pingcap.tiflow.cdc.schedulepb.Heartbeat") proto.RegisterType((*TableStatus)(nil), "pingcap.tiflow.cdc.schedulepb.TableStatus") - proto.RegisterType((*Sync)(nil), "pingcap.tiflow.cdc.schedulepb.Sync") + proto.RegisterType((*HeartbeatResponse)(nil), "pingcap.tiflow.cdc.schedulepb.HeartbeatResponse") + proto.RegisterType((*OwnerRevision)(nil), "pingcap.tiflow.cdc.schedulepb.OwnerRevision") + proto.RegisterType((*ProcessorEpoch)(nil), "pingcap.tiflow.cdc.schedulepb.ProcessorEpoch") proto.RegisterType((*Message)(nil), "pingcap.tiflow.cdc.schedulepb.Message") proto.RegisterType((*Message_Header)(nil), "pingcap.tiflow.cdc.schedulepb.Message.Header") } -func init() { proto.RegisterFile("TableSchedule.proto", fileDescriptor_5b343da219629f69) } - -var fileDescriptor_5b343da219629f69 = []byte{ - // 978 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xe4, 0x57, 0xcf, 0x6f, 0x1b, 0x45, - 0x14, 0xce, 0x3a, 0xfe, 0xf9, 0x36, 0x76, 0x97, 0x69, 0xda, 0x5a, 0x01, 0x9c, 0xb2, 0xa0, 0x52, - 0x5c, 0xba, 0x56, 0x5d, 0x24, 0x24, 0x2e, 0x28, 0x4e, 0x23, 0x11, 0x15, 0xab, 0xd5, 0xd8, 0xe1, - 0xc0, 0xc5, 0x5a, 0xcf, 0x4c, 0xd7, 0x4b, 0xec, 0x9d, 0x65, 0x67, 0x9c, 0xc8, 0x7f, 0x02, 0x17, - 0xd4, 0x03, 0x7f, 0x01, 0x47, 0xfe, 0x12, 0x8e, 0x3d, 0x21, 0x4e, 0x11, 0x4a, 0x0e, 0x5c, 0x39, - 0x87, 0x0b, 0xda, 0x99, 0xb5, 0x37, 0x6e, 0x2c, 0xec, 0x40, 0xc3, 0x85, 0xdb, 0x3c, 0xef, 0xbc, - 0xef, 0xbd, 0xf9, 0xe6, 0xfb, 0xde, 0xc8, 0x70, 0xb3, 0xeb, 0xf6, 0x87, 0xac, 0x43, 0x06, 0x8c, - 0x8e, 0x87, 0xcc, 0x09, 0x23, 0x2e, 0x39, 0x7a, 0x37, 0xf4, 0x03, 0x8f, 0xb8, 0xa1, 0x23, 0xfd, - 0x17, 0x43, 0x7e, 0xec, 0x10, 0x4a, 0x1c, 0x91, 0x6c, 0x09, 0xfb, 0x5b, 0x9b, 0x1e, 0xf7, 0xb8, - 0xda, 0xd9, 0x88, 0x57, 0x3a, 0xc9, 0x7e, 0x00, 0xe5, 0x67, 0xc7, 0x01, 0x8b, 0x30, 0x3b, 0xf2, - 0x85, 0xcf, 0x03, 0xb4, 0x05, 0xc5, 0x28, 0x59, 0x57, 0x8d, 0xbb, 0xc6, 0xfd, 0x75, 0x3c, 0x8b, - 0xed, 0x7b, 0x50, 0x79, 0x1e, 0x71, 0xc2, 0x84, 0xe0, 0xd1, 0x5e, 0xc8, 0xc9, 0x00, 0x6d, 0x42, - 0x8e, 0xc5, 0x0b, 0xb5, 0xb5, 0x84, 0x75, 0x60, 0xff, 0x64, 0x00, 0xec, 0x0e, 0x18, 0x39, 0x0c, - 0xb9, 0x1f, 0x48, 0xf4, 0x0c, 0xca, 0x64, 0x16, 0xf5, 0xa4, 0x50, 0x9b, 0xb3, 0xad, 0xfa, 0xf9, - 0xc9, 0xf6, 0x3d, 0xcf, 0x97, 0x83, 0x71, 0xdf, 0x21, 0x7c, 0xd4, 0x48, 0xda, 0x6f, 0xe8, 0xf6, - 0x1b, 0x84, 0x92, 0xc6, 0x88, 0x53, 0x36, 0x74, 0xba, 0x02, 0x6f, 0xa4, 0x00, 0x5d, 0x81, 0x9e, - 0x82, 0x19, 0x31, 0xc1, 0x87, 0x47, 0x8c, 0xc6, 0x70, 0x99, 0x2b, 0xc3, 0xc1, 0x34, 0xbd, 0x2b, - 0xec, 0x5f, 0x0c, 0xb8, 0xb1, 0x43, 0xa9, 0x62, 0x14, 0xb3, 0x6f, 0xc7, 0x4c, 0x48, 0x74, 0x00, - 0x45, 0x19, 0xc7, 0x3d, 0x9f, 0x6a, 0x12, 0x5a, 0x9f, 0x9d, 0x9e, 0x6c, 0x17, 0xd4, 0x9e, 0xfd, - 0x27, 0xe7, 0x27, 0xdb, 0x0f, 0x56, 0x2a, 0xa4, 0xb7, 0xe3, 0x82, 0xc2, 0xda, 0xa7, 0xe8, 0x3d, - 0xd8, 0xf0, 0x45, 0x4f, 0x30, 0xc2, 0x03, 0xea, 0x46, 0x13, 0xd5, 0x78, 0x11, 0x9b, 0xbe, 0xe8, - 0x4c, 0x7f, 0x42, 0xfb, 0x00, 0xe9, 0x51, 0xab, 0xeb, 0x77, 0x8d, 0xfb, 0x66, 0xf3, 0x23, 0xe7, - 0x6f, 0x6f, 0xd6, 0x49, 0xa9, 0xc6, 0x17, 0x92, 0xed, 0x43, 0x40, 0x98, 0x8d, 0xf8, 0x11, 0xfb, - 0x0f, 0x8e, 0x66, 0x7f, 0x9f, 0x81, 0xcd, 0x27, 0xbe, 0x08, 0x5d, 0x49, 0x06, 0x73, 0xf5, 0x3a, - 0x50, 0xe1, 0xb1, 0xc0, 0x7a, 0x73, 0xaa, 0x32, 0x9b, 0x1f, 0x2f, 0x39, 0xd4, 0x9c, 0x2a, 0x71, - 0x99, 0xcf, 0x89, 0xf4, 0x29, 0x94, 0x5c, 0x4a, 0x7b, 0xaa, 0xb8, 0x62, 0xd1, 0x6c, 0x3a, 0x4b, - 0xf0, 0x5e, 0xbb, 0x62, 0x5c, 0x74, 0x93, 0x1f, 0x50, 0x17, 0x36, 0x22, 0xc5, 0x53, 0x82, 0xa7, - 0x49, 0x7f, 0xb4, 0x04, 0xef, 0x32, 0xb5, 0xd8, 0x8c, 0xd2, 0xdf, 0xec, 0x97, 0x19, 0xb0, 0xd2, - 0x9a, 0x22, 0xe4, 0x81, 0x60, 0xd7, 0xa5, 0xab, 0x16, 0xe4, 0x85, 0x74, 0xe5, 0x58, 0x24, 0x5c, - 0xd4, 0x97, 0xf4, 0xae, 0xa7, 0x87, 0xca, 0xc0, 0x49, 0xe6, 0x1b, 0x14, 0x1e, 0xba, 0x0d, 0xf9, - 0x88, 0x7d, 0xc3, 0x88, 0xac, 0x66, 0x95, 0xc0, 0x93, 0xc8, 0xfe, 0xd3, 0x80, 0x9b, 0x73, 0xb4, - 0xfd, 0x9f, 0x58, 0xb1, 0x7f, 0xc8, 0xc0, 0xad, 0xd7, 0x1c, 0x92, 0x9c, 0xff, 0x2b, 0xb8, 0x11, - 0x4e, 0xc7, 0x6a, 0x2f, 0x1d, 0xa7, 0x66, 0xf3, 0xe1, 0x92, 0x4a, 0xf3, 0xc3, 0x18, 0x57, 0xc2, - 0xf9, 0xe1, 0xfc, 0xe5, 0x65, 0x97, 0x34, 0x56, 0x76, 0x89, 0xee, 0xed, 0x82, 0x4d, 0x0e, 0x16, - 0xda, 0xa4, 0x79, 0x15, 0x9b, 0x24, 0x98, 0x73, 0x3e, 0xe9, 0x41, 0x71, 0x27, 0x08, 0xf8, 0x38, - 0x20, 0xec, 0x5a, 0x66, 0x85, 0xfd, 0x87, 0x01, 0xe6, 0x85, 0xab, 0xbd, 0x2e, 0xb5, 0x7d, 0x0e, - 0xb9, 0x58, 0x33, 0x9a, 0xe8, 0xca, 0x52, 0x91, 0xcc, 0x3a, 0x62, 0x58, 0xe7, 0xbd, 0x49, 0xa9, - 0xfd, 0x68, 0x40, 0xb6, 0x33, 0x09, 0xc8, 0xb5, 0x29, 0xab, 0x05, 0x79, 0x75, 0xee, 0xd8, 0x5a, - 0xeb, 0x57, 0xb5, 0x96, 0xce, 0xb4, 0x7f, 0xcf, 0x41, 0xa1, 0xcd, 0x84, 0x70, 0x3d, 0x86, 0xf6, - 0x20, 0x3f, 0x60, 0x2e, 0x65, 0xd1, 0x8a, 0xed, 0x25, 0x79, 0xce, 0x17, 0x2a, 0x09, 0x27, 0xc9, - 0x68, 0x0f, 0x8a, 0x23, 0xe1, 0xf5, 0xe4, 0x24, 0x9c, 0x5e, 0x43, 0x7d, 0x35, 0xa0, 0xee, 0x24, - 0x64, 0xb8, 0x30, 0x12, 0x5e, 0xbc, 0x40, 0x7b, 0x90, 0x7d, 0x11, 0xf1, 0x91, 0xba, 0x83, 0x52, - 0xeb, 0xd1, 0xf9, 0xc9, 0xf6, 0xc3, 0x55, 0x24, 0xb1, 0xeb, 0x86, 0x72, 0x1c, 0xc5, 0xa2, 0x50, - 0xe9, 0x68, 0x07, 0x32, 0x92, 0xab, 0x11, 0xf8, 0x8f, 0x40, 0x32, 0x92, 0x23, 0x1f, 0x6e, 0xd3, - 0x64, 0x64, 0x68, 0xd7, 0xf5, 0x22, 0xfd, 0xd6, 0x54, 0x73, 0x8a, 0xa7, 0xc7, 0x4b, 0x8e, 0xb7, - 0xe8, 0x45, 0xc6, 0x9b, 0x74, 0xd1, 0x3b, 0x3d, 0x84, 0x3b, 0x97, 0x4a, 0x69, 0xbf, 0x56, 0xf3, - 0xaa, 0xd6, 0x27, 0x57, 0xab, 0x95, 0x78, 0xfd, 0x16, 0x5d, 0x38, 0xf2, 0xe6, 0xc5, 0x5e, 0xf8, - 0x37, 0xaf, 0xcd, 0x2e, 0x14, 0xdd, 0x64, 0x80, 0x54, 0x8b, 0x0a, 0xe8, 0xc3, 0x65, 0x43, 0x2e, - 0xd9, 0x8e, 0x67, 0x89, 0xe8, 0x53, 0xc8, 0x8a, 0x49, 0x40, 0xaa, 0x25, 0x05, 0xf0, 0xfe, 0x12, - 0x80, 0xd8, 0x5b, 0x58, 0x25, 0x6c, 0xd9, 0x90, 0xd7, 0x22, 0x44, 0x55, 0x28, 0x1c, 0xb1, 0x68, - 0x36, 0xb5, 0x4a, 0x78, 0x1a, 0xd6, 0x03, 0x80, 0xd4, 0xee, 0xc8, 0x84, 0xc2, 0x41, 0x70, 0x18, - 0xf0, 0xe3, 0xc0, 0x5a, 0x43, 0x00, 0xf9, 0x9d, 0xbe, 0x60, 0x81, 0xb4, 0x0c, 0x54, 0x86, 0xd2, - 0xf3, 0x88, 0x85, 0x6e, 0xe4, 0x07, 0x9e, 0x95, 0x41, 0x1b, 0x50, 0xd4, 0x21, 0xa3, 0xd6, 0x3a, - 0xba, 0x01, 0x26, 0x66, 0xe1, 0xd0, 0x27, 0xae, 0x8c, 0x3f, 0x67, 0xe3, 0xcf, 0x1d, 0xc9, 0xc3, - 0xb8, 0x51, 0x2b, 0x17, 0x83, 0xaa, 0x88, 0x51, 0x2b, 0x5f, 0xff, 0xce, 0x00, 0xf3, 0x82, 0xb0, - 0x51, 0x05, 0xa0, 0x2d, 0xbc, 0xb4, 0xe8, 0x5b, 0x50, 0x6e, 0x0b, 0x2f, 0xa5, 0xd3, 0x32, 0xd0, - 0xdb, 0x70, 0xa7, 0x2d, 0xbc, 0x45, 0x72, 0xb1, 0x32, 0xe8, 0x1d, 0xa8, 0x5e, 0xfe, 0xa8, 0x2f, - 0x52, 0x77, 0xd6, 0x16, 0xde, 0x94, 0x53, 0x2b, 0x1b, 0xf7, 0xd2, 0x16, 0x5e, 0xcc, 0x91, 0x95, - 0x6b, 0x7d, 0xf0, 0xf3, 0x69, 0xcd, 0x78, 0x75, 0x5a, 0x33, 0x7e, 0x3b, 0xad, 0x19, 0x2f, 0xcf, - 0x6a, 0x6b, 0xaf, 0xce, 0x6a, 0x6b, 0xbf, 0x9e, 0xd5, 0xd6, 0xbe, 0x86, 0x94, 0xd0, 0x7e, 0x5e, - 0xfd, 0x19, 0x79, 0xfc, 0x57, 0x00, 0x00, 0x00, 0xff, 0xff, 0xc4, 0x70, 0x7d, 0x36, 0xd8, 0x0c, - 0x00, 0x00, +func init() { proto.RegisterFile("table_schedule.proto", fileDescriptor_ab4bb9c6b16cfa4d) } + +var fileDescriptor_ab4bb9c6b16cfa4d = []byte{ + // 1114 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xe4, 0x57, 0x4f, 0x6f, 0xe3, 0x44, + 0x14, 0x8f, 0x93, 0x34, 0x7f, 0x5e, 0xda, 0xae, 0x3b, 0x9b, 0x6e, 0x83, 0x81, 0xc4, 0x58, 0xab, + 0xaa, 0x64, 0xd9, 0x84, 0xed, 0x72, 0x40, 0x7b, 0x41, 0xcd, 0x6e, 0x51, 0xab, 0xa5, 0xea, 0xca, + 0x4d, 0x0f, 0x20, 0xa4, 0xc8, 0xb1, 0x67, 0x1d, 0xd3, 0x24, 0x63, 0x3c, 0x6e, 0xab, 0x5e, 0xf8, + 0x00, 0x3e, 0xed, 0x89, 0x9b, 0x0f, 0x5c, 0xf9, 0x06, 0x48, 0x7c, 0x80, 0x3d, 0x70, 0xe8, 0x09, + 0x21, 0x21, 0x55, 0xd0, 0x7e, 0x07, 0x0e, 0xe5, 0x82, 0x3c, 0x9e, 0xd8, 0x71, 0x9b, 0x25, 0x29, + 0xcb, 0x72, 0xe1, 0x36, 0xf3, 0xfe, 0xfc, 0xde, 0xbc, 0xf7, 0x7e, 0xef, 0x59, 0x86, 0xb2, 0xab, + 0x75, 0xfb, 0xb8, 0x43, 0xf5, 0x1e, 0x36, 0x0e, 0xfb, 0xb8, 0x61, 0x3b, 0xc4, 0x25, 0xe8, 0x5d, + 0xdb, 0x1a, 0x9a, 0xba, 0x66, 0x37, 0x5c, 0xeb, 0x79, 0x9f, 0x1c, 0x37, 0x74, 0x43, 0x6f, 0x8c, + 0x4c, 0xec, 0xae, 0x54, 0x36, 0x89, 0x49, 0x98, 0x65, 0x33, 0x38, 0x85, 0x4e, 0xca, 0xf7, 0x02, + 0xc0, 0xe3, 0x1e, 0xd6, 0x0f, 0x6c, 0x62, 0x0d, 0x5d, 0xb4, 0x0b, 0x0b, 0x7a, 0x74, 0xeb, 0xb8, + 0xb4, 0x22, 0xc8, 0xc2, 0x5a, 0xb6, 0x55, 0xbf, 0x3c, 0xab, 0xad, 0x9a, 0x96, 0xdb, 0x3b, 0xec, + 0x36, 0x74, 0x32, 0x68, 0xf2, 0x48, 0xcd, 0x30, 0x52, 0x53, 0x37, 0xf4, 0xe6, 0x80, 0x18, 0xb8, + 0xdf, 0x68, 0x53, 0x75, 0x3e, 0x06, 0x68, 0x53, 0xf4, 0x14, 0x4a, 0x0e, 0xa6, 0xa4, 0x7f, 0x84, + 0x8d, 0x00, 0x2e, 0x7d, 0x63, 0x38, 0x18, 0xb9, 0xb7, 0xa9, 0xf2, 0xb3, 0x00, 0xb7, 0x36, 0x0c, + 0xa3, 0x1d, 0x64, 0xaf, 0xe2, 0xaf, 0x0f, 0x31, 0x75, 0xd1, 0x3e, 0x14, 0xc2, 0x6a, 0x58, 0x06, + 0x7b, 0x6c, 0xa6, 0xf5, 0xe8, 0xfc, 0xac, 0x96, 0x67, 0x36, 0xdb, 0x4f, 0x2e, 0xcf, 0x6a, 0xf7, + 0x66, 0x0a, 0x14, 0x9a, 0xab, 0x79, 0x86, 0xb5, 0x6d, 0xa0, 0xf7, 0x60, 0xde, 0xa2, 0x1d, 0x8a, + 0x75, 0x32, 0x34, 0x34, 0xe7, 0x84, 0x3d, 0xbc, 0xa0, 0x96, 0x2c, 0xba, 0x37, 0x12, 0xa1, 0x6d, + 0x80, 0x38, 0xd5, 0x4a, 0x46, 0x16, 0xd6, 0x4a, 0xeb, 0xef, 0x37, 0xfe, 0xb6, 0x09, 0x8d, 0xb8, + 0xd4, 0xea, 0x98, 0xb3, 0x72, 0x00, 0x48, 0xc5, 0x03, 0x72, 0x84, 0xff, 0x83, 0xd4, 0x94, 0x1f, + 0x04, 0x28, 0x3f, 0xb1, 0xa8, 0xad, 0xb9, 0x7a, 0x2f, 0x11, 0xef, 0x29, 0x14, 0x35, 0xc3, 0xe8, + 0x30, 0x3b, 0x16, 0xb0, 0xb4, 0xde, 0x98, 0x92, 0xcf, 0x95, 0x6e, 0xa8, 0x05, 0x8d, 0x0b, 0x50, + 0x1b, 0xe6, 0x1d, 0x96, 0x12, 0xc7, 0x4b, 0x33, 0xbc, 0x07, 0x53, 0xf0, 0xae, 0x57, 0x41, 0x2d, + 0x39, 0xb1, 0x4c, 0x79, 0x91, 0x06, 0x31, 0x8e, 0x49, 0x6d, 0x32, 0xa4, 0xf8, 0x4d, 0x51, 0xa0, + 0x05, 0x39, 0xea, 0x6a, 0xee, 0x21, 0xe5, 0x6f, 0xaf, 0x4f, 0x79, 0x3b, 0x03, 0xd8, 0x63, 0x1e, + 0x2a, 0xf7, 0xfc, 0x17, 0x39, 0x82, 0xee, 0x40, 0xce, 0xc1, 0x5f, 0x61, 0xdd, 0xad, 0x64, 0x19, + 0x17, 0xf9, 0x4d, 0xf9, 0x53, 0x80, 0xdb, 0x89, 0xb2, 0xfd, 0x9f, 0xaa, 0xa2, 0xfc, 0x28, 0xc0, + 0xf2, 0x15, 0x32, 0xf3, 0xfc, 0x3f, 0xbb, 0xce, 0xe6, 0xe6, 0xcc, 0x6c, 0x0e, 0x31, 0xc6, 0xe8, + 0xbc, 0x3f, 0x91, 0xce, 0xeb, 0x37, 0xa1, 0x33, 0xc7, 0x4c, 0xf0, 0xb9, 0x04, 0xc5, 0x2d, 0xac, + 0x39, 0x6e, 0x17, 0x6b, 0xae, 0xf2, 0x87, 0x00, 0xa5, 0xb1, 0x72, 0xbd, 0xa9, 0x0e, 0x7e, 0x02, + 0x73, 0x41, 0x1f, 0xc2, 0x1c, 0x16, 0xa7, 0x16, 0x3e, 0x7a, 0x11, 0x56, 0x43, 0x3f, 0xb4, 0xfb, + 0x5a, 0xed, 0x6b, 0x65, 0x5f, 0x9e, 0xd5, 0x52, 0x89, 0x26, 0x7e, 0x03, 0x4b, 0x51, 0x15, 0xa2, + 0xfe, 0x6d, 0x41, 0x8e, 0xbd, 0x38, 0xf8, 0x06, 0x65, 0x6e, 0x46, 0x34, 0x1e, 0x82, 0xfb, 0xa3, + 0x1a, 0x94, 0x82, 0x5d, 0xee, 0x12, 0x3b, 0x40, 0xe0, 0xab, 0x1c, 0x2c, 0xba, 0xc7, 0x25, 0xca, + 0x3d, 0x58, 0xd8, 0x3d, 0x1e, 0x62, 0x47, 0xc5, 0x47, 0x16, 0xb5, 0xc8, 0x10, 0x49, 0x50, 0x70, + 0xf8, 0x39, 0xac, 0xbc, 0x1a, 0xdd, 0x95, 0x55, 0x58, 0x7c, 0xe6, 0x10, 0x1d, 0x53, 0x4a, 0x9c, + 0x4d, 0x9b, 0xe8, 0x3d, 0x54, 0x86, 0x39, 0x1c, 0x1c, 0x98, 0x69, 0x51, 0x0d, 0x2f, 0xca, 0x4f, + 0x79, 0xc8, 0xef, 0x60, 0x4a, 0x35, 0x13, 0xa3, 0x4d, 0xc8, 0xf5, 0xb0, 0x66, 0x60, 0x87, 0x13, + 0xf1, 0xfe, 0x94, 0x5c, 0xb8, 0x5f, 0x63, 0x8b, 0x39, 0xa9, 0xdc, 0x19, 0x6d, 0x42, 0x61, 0x40, + 0xcd, 0x8e, 0x7b, 0x62, 0x8f, 0x9a, 0x57, 0x9f, 0x0d, 0xa8, 0x7d, 0x62, 0x63, 0x35, 0x3f, 0xa0, + 0x66, 0x70, 0x40, 0x9b, 0x90, 0x7d, 0xee, 0x90, 0x01, 0xeb, 0x5c, 0xb1, 0xf5, 0xe0, 0xf2, 0xac, + 0x76, 0x7f, 0x16, 0x22, 0x3d, 0xd6, 0x6c, 0xf7, 0xd0, 0x09, 0xa8, 0xc4, 0xdc, 0xd1, 0x06, 0xa4, + 0x5d, 0xc2, 0x96, 0xd1, 0x3f, 0x02, 0x49, 0xbb, 0x04, 0x59, 0x70, 0xc7, 0xe0, 0xc3, 0x1b, 0xce, + 0x55, 0xc7, 0x09, 0xb7, 0x7e, 0x65, 0x8e, 0xd5, 0xe9, 0xe1, 0x94, 0xf4, 0x26, 0x7d, 0xc6, 0xd4, + 0xb2, 0x31, 0xe9, 0xe3, 0xd6, 0x87, 0x95, 0x6b, 0xa1, 0x42, 0xa6, 0x55, 0x72, 0x2c, 0xd6, 0x47, + 0x37, 0x8b, 0xc5, 0xa7, 0x79, 0xd9, 0x98, 0xb8, 0x7c, 0x3e, 0x85, 0x62, 0x6f, 0xc4, 0xe8, 0x4a, + 0x9e, 0xe1, 0xaf, 0x4d, 0xc1, 0x8f, 0x27, 0x20, 0x76, 0x45, 0x1d, 0x40, 0xd1, 0x25, 0x7e, 0x70, + 0x81, 0x01, 0x7e, 0x38, 0x33, 0xe0, 0xe8, 0xb1, 0x4b, 0xbd, 0x6b, 0x53, 0x96, 0x5c, 0xc5, 0xc5, + 0xd7, 0x58, 0xc5, 0xd2, 0xaf, 0x02, 0xe4, 0x42, 0xc2, 0xa2, 0x0a, 0xe4, 0x8f, 0xb0, 0x13, 0x8d, + 0x4f, 0x51, 0x1d, 0x5d, 0xd1, 0xe7, 0xb0, 0x48, 0x82, 0x51, 0xeb, 0x44, 0xf3, 0x15, 0x6e, 0xd2, + 0x0f, 0xa6, 0xc4, 0x4c, 0xcc, 0x27, 0x9f, 0xef, 0x05, 0x92, 0x18, 0xda, 0x2f, 0xe1, 0x96, 0x3d, + 0x1a, 0xcc, 0x4e, 0x38, 0x90, 0x99, 0x99, 0xa6, 0x2d, 0x39, 0xce, 0x1c, 0x7c, 0xd1, 0x4e, 0x48, + 0xeb, 0xdf, 0xa6, 0x01, 0xe2, 0x55, 0x88, 0x14, 0xc8, 0xef, 0x0f, 0x0f, 0x86, 0xe4, 0x78, 0x28, + 0xa6, 0xa4, 0x65, 0xcf, 0x97, 0x97, 0x62, 0x25, 0x57, 0x20, 0x19, 0x72, 0x1b, 0x5d, 0x8a, 0x87, + 0xae, 0x28, 0x48, 0x65, 0xcf, 0x97, 0xc5, 0xd8, 0x24, 0x94, 0xa3, 0x55, 0x28, 0x3e, 0x73, 0xb0, + 0xad, 0x39, 0xd6, 0xd0, 0x14, 0xd3, 0xd2, 0x8a, 0xe7, 0xcb, 0xb7, 0x63, 0xa3, 0x48, 0x85, 0xee, + 0x42, 0x21, 0xbc, 0x60, 0x43, 0xcc, 0x48, 0x77, 0x3c, 0x5f, 0x46, 0x57, 0xcd, 0xb0, 0x81, 0xea, + 0x50, 0x52, 0xb1, 0xdd, 0xb7, 0x74, 0xcd, 0x0d, 0xf0, 0xb2, 0xd2, 0x5b, 0x9e, 0x2f, 0x2f, 0x8f, + 0xed, 0xef, 0x58, 0x19, 0x20, 0x8e, 0xd6, 0x9f, 0x38, 0x77, 0x15, 0x71, 0xa4, 0x09, 0xb2, 0x64, + 0x67, 0x6c, 0x88, 0xb9, 0xab, 0x59, 0x72, 0x45, 0xfd, 0xbb, 0x34, 0x94, 0xc6, 0xd6, 0x0c, 0xaa, + 0x02, 0xec, 0x50, 0x33, 0x2e, 0xce, 0xa2, 0xe7, 0xcb, 0x63, 0x12, 0x74, 0x17, 0x16, 0x76, 0xa8, + 0x19, 0x73, 0x48, 0x14, 0xa4, 0x25, 0xcf, 0x97, 0x93, 0x42, 0xf4, 0x31, 0xac, 0xec, 0x50, 0x73, + 0xd2, 0x7c, 0x8b, 0x69, 0xe9, 0x6d, 0xcf, 0x97, 0x5f, 0xa5, 0x46, 0x8f, 0xa0, 0x72, 0x5d, 0x15, + 0xb2, 0x5d, 0xcc, 0x48, 0xef, 0x78, 0xbe, 0xfc, 0x4a, 0x3d, 0x52, 0x60, 0x7e, 0x87, 0x9a, 0xd1, + 0xe0, 0x88, 0x59, 0x49, 0xf4, 0x7c, 0x39, 0x21, 0x43, 0xeb, 0x50, 0x1e, 0xbf, 0x47, 0xd8, 0x73, + 0x52, 0xc5, 0xf3, 0xe5, 0x89, 0xba, 0xd6, 0xda, 0xe9, 0xef, 0xd5, 0xd4, 0xcb, 0xf3, 0xaa, 0x70, + 0x7a, 0x5e, 0x15, 0x7e, 0x3b, 0xaf, 0x0a, 0x2f, 0x2e, 0xaa, 0xa9, 0xd3, 0x8b, 0x6a, 0xea, 0x97, + 0x8b, 0x6a, 0xea, 0x0b, 0x88, 0xb9, 0xd8, 0xcd, 0xb1, 0xdf, 0xb2, 0x87, 0x7f, 0x05, 0x00, 0x00, + 0xff, 0xff, 0x65, 0xd5, 0x46, 0xda, 0xe3, 0x0d, 0x00, 0x00, } -func (m *OwnerRevision) Marshal() (dAtA []byte, err error) { +func (m *Checkpoint) Marshal() (dAtA []byte, err error) { size := m.Size() dAtA = make([]byte, size) n, err := m.MarshalToSizedBuffer(dAtA[:size]) @@ -1008,25 +1008,30 @@ func (m *OwnerRevision) Marshal() (dAtA []byte, err error) { return dAtA[:n], nil } -func (m *OwnerRevision) MarshalTo(dAtA []byte) (int, error) { +func (m *Checkpoint) MarshalTo(dAtA []byte) (int, error) { size := m.Size() return m.MarshalToSizedBuffer(dAtA[:size]) } -func (m *OwnerRevision) MarshalToSizedBuffer(dAtA []byte) (int, error) { +func (m *Checkpoint) MarshalToSizedBuffer(dAtA []byte) (int, error) { i := len(dAtA) _ = i var l int _ = l - if m.Revision != 0 { - i = encodeVarintTableSchedule(dAtA, i, uint64(m.Revision)) + if m.ResolvedTs != 0 { + i = encodeVarintTableSchedule(dAtA, i, uint64(m.ResolvedTs)) + i-- + dAtA[i] = 0x10 + } + if m.CheckpointTs != 0 { + i = encodeVarintTableSchedule(dAtA, i, uint64(m.CheckpointTs)) i-- dAtA[i] = 0x8 } return len(dAtA) - i, nil } -func (m *ProcessorEpoch) Marshal() (dAtA []byte, err error) { +func (m *AddTableRequest) Marshal() (dAtA []byte, err error) { size := m.Size() dAtA = make([]byte, size) n, err := m.MarshalToSizedBuffer(dAtA[:size]) @@ -1036,90 +1041,27 @@ func (m *ProcessorEpoch) Marshal() (dAtA []byte, err error) { return dAtA[:n], nil } -func (m *ProcessorEpoch) MarshalTo(dAtA []byte) (int, error) { +func (m *AddTableRequest) MarshalTo(dAtA []byte) (int, error) { size := m.Size() return m.MarshalToSizedBuffer(dAtA[:size]) } -func (m *ProcessorEpoch) MarshalToSizedBuffer(dAtA []byte) (int, error) { +func (m *AddTableRequest) MarshalToSizedBuffer(dAtA []byte) (int, error) { i := len(dAtA) _ = i var l int _ = l - if len(m.Epoch) > 0 { - i -= len(m.Epoch) - copy(dAtA[i:], m.Epoch) - i = encodeVarintTableSchedule(dAtA, i, uint64(len(m.Epoch))) + if m.Checkpoint != nil { + { + size, err := m.Checkpoint.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintTableSchedule(dAtA, i, uint64(size)) + } i-- - dAtA[i] = 0xa - } - return len(dAtA) - i, nil -} - -func (m *Checkpoint) 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 *Checkpoint) MarshalTo(dAtA []byte) (int, error) { - size := m.Size() - return m.MarshalToSizedBuffer(dAtA[:size]) -} - -func (m *Checkpoint) MarshalToSizedBuffer(dAtA []byte) (int, error) { - i := len(dAtA) - _ = i - var l int - _ = l - if m.ResolvedTs != 0 { - i = encodeVarintTableSchedule(dAtA, i, uint64(m.ResolvedTs)) - i-- - dAtA[i] = 0x10 - } - if m.CheckpointTs != 0 { - i = encodeVarintTableSchedule(dAtA, i, uint64(m.CheckpointTs)) - i-- - dAtA[i] = 0x8 - } - return len(dAtA) - i, nil -} - -func (m *AddTableRequest) 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 *AddTableRequest) MarshalTo(dAtA []byte) (int, error) { - size := m.Size() - return m.MarshalToSizedBuffer(dAtA[:size]) -} - -func (m *AddTableRequest) MarshalToSizedBuffer(dAtA []byte) (int, error) { - i := len(dAtA) - _ = i - var l int - _ = l - if m.Checkpoint != nil { - { - size, err := m.Checkpoint.MarshalToSizedBuffer(dAtA[:i]) - if err != nil { - return 0, err - } - i -= size - i = encodeVarintTableSchedule(dAtA, i, uint64(size)) - } - i-- - dAtA[i] = 0x1a + dAtA[i] = 0x1a } if m.IsSecondary { i-- @@ -1197,7 +1139,7 @@ func (m *DispatchTableRequest) MarshalToSizedBuffer(dAtA []byte) (int, error) { i = encodeVarintTableSchedule(dAtA, i, uint64(size)) } i-- - dAtA[i] = 0x1a + dAtA[i] = 0x12 } if m.AddTable != nil { { @@ -1209,18 +1151,6 @@ func (m *DispatchTableRequest) MarshalToSizedBuffer(dAtA []byte) (int, error) { i = encodeVarintTableSchedule(dAtA, i, uint64(size)) } i-- - dAtA[i] = 0x12 - } - if m.OwnerRevision != nil { - { - size, err := m.OwnerRevision.MarshalToSizedBuffer(dAtA[:i]) - if err != nil { - return 0, err - } - i -= size - i = encodeVarintTableSchedule(dAtA, i, uint64(size)) - } - i-- dAtA[i] = 0xa } return len(dAtA) - i, nil @@ -1370,7 +1300,7 @@ func (m *DispatchTableResponse) MarshalToSizedBuffer(dAtA []byte) (int, error) { i = encodeVarintTableSchedule(dAtA, i, uint64(size)) } i-- - dAtA[i] = 0x1a + dAtA[i] = 0x12 } if m.AddTable != nil { { @@ -1382,24 +1312,12 @@ func (m *DispatchTableResponse) MarshalToSizedBuffer(dAtA []byte) (int, error) { i = encodeVarintTableSchedule(dAtA, i, uint64(size)) } i-- - dAtA[i] = 0x12 - } - if m.ProcessorEpoch != nil { - { - size, err := m.ProcessorEpoch.MarshalToSizedBuffer(dAtA[:i]) - if err != nil { - return 0, err - } - i -= size - i = encodeVarintTableSchedule(dAtA, i, uint64(size)) - } - i-- dAtA[i] = 0xa } return len(dAtA) - i, nil } -func (m *Announce) Marshal() (dAtA []byte, err error) { +func (m *Heartbeat) Marshal() (dAtA []byte, err error) { size := m.Size() dAtA = make([]byte, size) n, err := m.MarshalToSizedBuffer(dAtA[:size]) @@ -1409,28 +1327,16 @@ func (m *Announce) Marshal() (dAtA []byte, err error) { return dAtA[:n], nil } -func (m *Announce) MarshalTo(dAtA []byte) (int, error) { +func (m *Heartbeat) MarshalTo(dAtA []byte) (int, error) { size := m.Size() return m.MarshalToSizedBuffer(dAtA[:size]) } -func (m *Announce) MarshalToSizedBuffer(dAtA []byte) (int, error) { +func (m *Heartbeat) MarshalToSizedBuffer(dAtA []byte) (int, error) { i := len(dAtA) _ = i var l int _ = l - if m.OwnerRevision != nil { - { - size, err := m.OwnerRevision.MarshalToSizedBuffer(dAtA[:i]) - if err != nil { - return 0, err - } - i -= size - i = encodeVarintTableSchedule(dAtA, i, uint64(size)) - } - i-- - dAtA[i] = 0xa - } return len(dAtA) - i, nil } @@ -1454,18 +1360,16 @@ func (m *TableStatus) MarshalToSizedBuffer(dAtA []byte) (int, error) { _ = i var l int _ = l - if m.Checkpoint != nil { - { - size, err := m.Checkpoint.MarshalToSizedBuffer(dAtA[:i]) - if err != nil { - return 0, err - } - i -= size - i = encodeVarintTableSchedule(dAtA, i, uint64(size)) + { + size, err := m.Checkpoint.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err } - i-- - dAtA[i] = 0x1a + i -= size + i = encodeVarintTableSchedule(dAtA, i, uint64(size)) } + i-- + dAtA[i] = 0x1a if m.State != 0 { i = encodeVarintTableSchedule(dAtA, i, uint64(m.State)) i-- @@ -1479,7 +1383,7 @@ func (m *TableStatus) MarshalToSizedBuffer(dAtA []byte) (int, error) { return len(dAtA) - i, nil } -func (m *Sync) Marshal() (dAtA []byte, err error) { +func (m *HeartbeatResponse) Marshal() (dAtA []byte, err error) { size := m.Size() dAtA = make([]byte, size) n, err := m.MarshalToSizedBuffer(dAtA[:size]) @@ -1489,16 +1393,26 @@ func (m *Sync) Marshal() (dAtA []byte, err error) { return dAtA[:n], nil } -func (m *Sync) MarshalTo(dAtA []byte) (int, error) { +func (m *HeartbeatResponse) MarshalTo(dAtA []byte) (int, error) { size := m.Size() return m.MarshalToSizedBuffer(dAtA[:size]) } -func (m *Sync) MarshalToSizedBuffer(dAtA []byte) (int, error) { +func (m *HeartbeatResponse) MarshalToSizedBuffer(dAtA []byte) (int, error) { i := len(dAtA) _ = i var l int _ = l + if m.IsStopping { + i-- + if m.IsStopping { + dAtA[i] = 1 + } else { + dAtA[i] = 0 + } + i-- + dAtA[i] = 0x10 + } if len(m.Tables) > 0 { for iNdEx := len(m.Tables) - 1; iNdEx >= 0; iNdEx-- { { @@ -1510,18 +1424,64 @@ func (m *Sync) MarshalToSizedBuffer(dAtA []byte) (int, error) { i = encodeVarintTableSchedule(dAtA, i, uint64(size)) } i-- - dAtA[i] = 0x12 + dAtA[i] = 0xa } } - if m.ProcessorEpoch != nil { - { - size, err := m.ProcessorEpoch.MarshalToSizedBuffer(dAtA[:i]) - if err != nil { - return 0, err - } - i -= size - i = encodeVarintTableSchedule(dAtA, i, uint64(size)) - } + return len(dAtA) - i, nil +} + +func (m *OwnerRevision) 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 *OwnerRevision) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *OwnerRevision) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + _ = i + var l int + _ = l + if m.Revision != 0 { + i = encodeVarintTableSchedule(dAtA, i, uint64(m.Revision)) + i-- + dAtA[i] = 0x8 + } + return len(dAtA) - i, nil +} + +func (m *ProcessorEpoch) 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 *ProcessorEpoch) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *ProcessorEpoch) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + _ = i + var l int + _ = l + if len(m.Epoch) > 0 { + i -= len(m.Epoch) + copy(dAtA[i:], m.Epoch) + i = encodeVarintTableSchedule(dAtA, i, uint64(len(m.Epoch))) i-- dAtA[i] = 0xa } @@ -1548,9 +1508,9 @@ func (m *Message) MarshalToSizedBuffer(dAtA []byte) (int, error) { _ = i var l int _ = l - if m.Sync != nil { + if m.Checkpoint != nil { { - size, err := m.Sync.MarshalToSizedBuffer(dAtA[:i]) + size, err := m.Checkpoint.MarshalToSizedBuffer(dAtA[:i]) if err != nil { return 0, err } @@ -1560,9 +1520,9 @@ func (m *Message) MarshalToSizedBuffer(dAtA []byte) (int, error) { i-- dAtA[i] = 0x4a } - if m.Announce != nil { + if m.HeartbeatResponse != nil { { - size, err := m.Announce.MarshalToSizedBuffer(dAtA[:i]) + size, err := m.HeartbeatResponse.MarshalToSizedBuffer(dAtA[:i]) if err != nil { return 0, err } @@ -1572,9 +1532,9 @@ func (m *Message) MarshalToSizedBuffer(dAtA []byte) (int, error) { i-- dAtA[i] = 0x42 } - if m.Checkpoint != nil { + if m.Heartbeat != nil { { - size, err := m.Checkpoint.MarshalToSizedBuffer(dAtA[:i]) + size, err := m.Heartbeat.MarshalToSizedBuffer(dAtA[:i]) if err != nil { return 0, err } @@ -1662,6 +1622,26 @@ func (m *Message_Header) MarshalToSizedBuffer(dAtA []byte) (int, error) { _ = i var l int _ = l + { + size, err := m.ProcessorEpoch.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintTableSchedule(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x1a + { + size, err := m.OwnerRevision.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintTableSchedule(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x12 if len(m.Version) > 0 { i -= len(m.Version) copy(dAtA[i:], m.Version) @@ -1683,31 +1663,6 @@ func encodeVarintTableSchedule(dAtA []byte, offset int, v uint64) int { dAtA[offset] = uint8(v) return base } -func (m *OwnerRevision) Size() (n int) { - if m == nil { - return 0 - } - var l int - _ = l - if m.Revision != 0 { - n += 1 + sovTableSchedule(uint64(m.Revision)) - } - return n -} - -func (m *ProcessorEpoch) Size() (n int) { - if m == nil { - return 0 - } - var l int - _ = l - l = len(m.Epoch) - if l > 0 { - n += 1 + l + sovTableSchedule(uint64(l)) - } - return n -} - func (m *Checkpoint) Size() (n int) { if m == nil { return 0 @@ -1760,10 +1715,6 @@ func (m *DispatchTableRequest) Size() (n int) { } var l int _ = l - if m.OwnerRevision != nil { - l = m.OwnerRevision.Size() - n += 1 + l + sovTableSchedule(uint64(l)) - } if m.AddTable != nil { l = m.AddTable.Size() n += 1 + l + sovTableSchedule(uint64(l)) @@ -1824,10 +1775,6 @@ func (m *DispatchTableResponse) Size() (n int) { } var l int _ = l - if m.ProcessorEpoch != nil { - l = m.ProcessorEpoch.Size() - n += 1 + l + sovTableSchedule(uint64(l)) - } if m.AddTable != nil { l = m.AddTable.Size() n += 1 + l + sovTableSchedule(uint64(l)) @@ -1839,16 +1786,12 @@ func (m *DispatchTableResponse) Size() (n int) { return n } -func (m *Announce) Size() (n int) { +func (m *Heartbeat) Size() (n int) { if m == nil { return 0 } var l int _ = l - if m.OwnerRevision != nil { - l = m.OwnerRevision.Size() - n += 1 + l + sovTableSchedule(uint64(l)) - } return n } @@ -1864,29 +1807,51 @@ func (m *TableStatus) Size() (n int) { if m.State != 0 { n += 1 + sovTableSchedule(uint64(m.State)) } - if m.Checkpoint != nil { - l = m.Checkpoint.Size() - n += 1 + l + sovTableSchedule(uint64(l)) - } + l = m.Checkpoint.Size() + n += 1 + l + sovTableSchedule(uint64(l)) return n } -func (m *Sync) Size() (n int) { +func (m *HeartbeatResponse) Size() (n int) { if m == nil { return 0 } var l int _ = l - if m.ProcessorEpoch != nil { - l = m.ProcessorEpoch.Size() - n += 1 + l + sovTableSchedule(uint64(l)) - } if len(m.Tables) > 0 { for _, e := range m.Tables { l = e.Size() n += 1 + l + sovTableSchedule(uint64(l)) } } + if m.IsStopping { + n += 2 + } + return n +} + +func (m *OwnerRevision) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Revision != 0 { + n += 1 + sovTableSchedule(uint64(m.Revision)) + } + return n +} + +func (m *ProcessorEpoch) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + l = len(m.Epoch) + if l > 0 { + n += 1 + l + sovTableSchedule(uint64(l)) + } return n } @@ -1919,16 +1884,16 @@ func (m *Message) Size() (n int) { l = m.DispatchTableResponse.Size() n += 1 + l + sovTableSchedule(uint64(l)) } - if m.Checkpoint != nil { - l = m.Checkpoint.Size() + if m.Heartbeat != nil { + l = m.Heartbeat.Size() n += 1 + l + sovTableSchedule(uint64(l)) } - if m.Announce != nil { - l = m.Announce.Size() + if m.HeartbeatResponse != nil { + l = m.HeartbeatResponse.Size() n += 1 + l + sovTableSchedule(uint64(l)) } - if m.Sync != nil { - l = m.Sync.Size() + if m.Checkpoint != nil { + l = m.Checkpoint.Size() n += 1 + l + sovTableSchedule(uint64(l)) } return n @@ -1944,6 +1909,10 @@ func (m *Message_Header) Size() (n int) { if l > 0 { n += 1 + l + sovTableSchedule(uint64(l)) } + l = m.OwnerRevision.Size() + n += 1 + l + sovTableSchedule(uint64(l)) + l = m.ProcessorEpoch.Size() + n += 1 + l + sovTableSchedule(uint64(l)) return n } @@ -1953,157 +1922,6 @@ func sovTableSchedule(x uint64) (n int) { func sozTableSchedule(x uint64) (n int) { return sovTableSchedule(uint64((x << 1) ^ uint64((int64(x) >> 63)))) } -func (m *OwnerRevision) 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 ErrIntOverflowTableSchedule - } - 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: OwnerRevision: wiretype end group for non-group") - } - if fieldNum <= 0 { - return fmt.Errorf("proto: OwnerRevision: illegal tag %d (wire type %d)", fieldNum, wire) - } - switch fieldNum { - case 1: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field Revision", wireType) - } - m.Revision = 0 - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowTableSchedule - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - m.Revision |= int64(b&0x7F) << shift - if b < 0x80 { - break - } - } - default: - iNdEx = preIndex - skippy, err := skipTableSchedule(dAtA[iNdEx:]) - if err != nil { - return err - } - if (skippy < 0) || (iNdEx+skippy) < 0 { - return ErrInvalidLengthTableSchedule - } - if (iNdEx + skippy) > l { - return io.ErrUnexpectedEOF - } - iNdEx += skippy - } - } - - if iNdEx > l { - return io.ErrUnexpectedEOF - } - return nil -} -func (m *ProcessorEpoch) 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 ErrIntOverflowTableSchedule - } - 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: ProcessorEpoch: wiretype end group for non-group") - } - if fieldNum <= 0 { - return fmt.Errorf("proto: ProcessorEpoch: illegal tag %d (wire type %d)", fieldNum, wire) - } - switch fieldNum { - case 1: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Epoch", wireType) - } - var stringLen uint64 - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowTableSchedule - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - stringLen |= uint64(b&0x7F) << shift - if b < 0x80 { - break - } - } - intStringLen := int(stringLen) - if intStringLen < 0 { - return ErrInvalidLengthTableSchedule - } - postIndex := iNdEx + intStringLen - if postIndex < 0 { - return ErrInvalidLengthTableSchedule - } - if postIndex > l { - return io.ErrUnexpectedEOF - } - m.Epoch = string(dAtA[iNdEx:postIndex]) - iNdEx = postIndex - default: - iNdEx = preIndex - skippy, err := skipTableSchedule(dAtA[iNdEx:]) - if err != nil { - return err - } - if (skippy < 0) || (iNdEx+skippy) < 0 { - return ErrInvalidLengthTableSchedule - } - if (iNdEx + skippy) > l { - return io.ErrUnexpectedEOF - } - iNdEx += skippy - } - } - - if iNdEx > l { - return io.ErrUnexpectedEOF - } - return nil -} func (m *Checkpoint) Unmarshal(dAtA []byte) error { l := len(dAtA) iNdEx := 0 @@ -2416,42 +2234,6 @@ func (m *DispatchTableRequest) Unmarshal(dAtA []byte) error { } switch fieldNum { case 1: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field OwnerRevision", wireType) - } - var msglen int - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowTableSchedule - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - msglen |= int(b&0x7F) << shift - if b < 0x80 { - break - } - } - if msglen < 0 { - return ErrInvalidLengthTableSchedule - } - postIndex := iNdEx + msglen - if postIndex < 0 { - return ErrInvalidLengthTableSchedule - } - if postIndex > l { - return io.ErrUnexpectedEOF - } - if m.OwnerRevision == nil { - m.OwnerRevision = &OwnerRevision{} - } - if err := m.OwnerRevision.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { - return err - } - iNdEx = postIndex - case 2: if wireType != 2 { return fmt.Errorf("proto: wrong wireType = %d for field AddTable", wireType) } @@ -2487,7 +2269,7 @@ func (m *DispatchTableRequest) Unmarshal(dAtA []byte) error { return err } iNdEx = postIndex - case 3: + case 2: if wireType != 2 { return fmt.Errorf("proto: wrong wireType = %d for field RemoveTable", wireType) } @@ -2876,42 +2658,6 @@ func (m *DispatchTableResponse) Unmarshal(dAtA []byte) error { } switch fieldNum { case 1: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field ProcessorEpoch", wireType) - } - var msglen int - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowTableSchedule - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - msglen |= int(b&0x7F) << shift - if b < 0x80 { - break - } - } - if msglen < 0 { - return ErrInvalidLengthTableSchedule - } - postIndex := iNdEx + msglen - if postIndex < 0 { - return ErrInvalidLengthTableSchedule - } - if postIndex > l { - return io.ErrUnexpectedEOF - } - if m.ProcessorEpoch == nil { - m.ProcessorEpoch = &ProcessorEpoch{} - } - if err := m.ProcessorEpoch.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { - return err - } - iNdEx = postIndex - case 2: if wireType != 2 { return fmt.Errorf("proto: wrong wireType = %d for field AddTable", wireType) } @@ -2947,7 +2693,7 @@ func (m *DispatchTableResponse) Unmarshal(dAtA []byte) error { return err } iNdEx = postIndex - case 3: + case 2: if wireType != 2 { return fmt.Errorf("proto: wrong wireType = %d for field RemoveTable", wireType) } @@ -3004,7 +2750,7 @@ func (m *DispatchTableResponse) Unmarshal(dAtA []byte) error { } return nil } -func (m *Announce) Unmarshal(dAtA []byte) error { +func (m *Heartbeat) Unmarshal(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -3027,48 +2773,12 @@ func (m *Announce) Unmarshal(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: Announce: wiretype end group for non-group") + return fmt.Errorf("proto: Heartbeat: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: Announce: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: Heartbeat: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { - case 1: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field OwnerRevision", wireType) - } - var msglen int - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowTableSchedule - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - msglen |= int(b&0x7F) << shift - if b < 0x80 { - break - } - } - if msglen < 0 { - return ErrInvalidLengthTableSchedule - } - postIndex := iNdEx + msglen - if postIndex < 0 { - return ErrInvalidLengthTableSchedule - } - if postIndex > l { - return io.ErrUnexpectedEOF - } - if m.OwnerRevision == nil { - m.OwnerRevision = &OwnerRevision{} - } - if err := m.OwnerRevision.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { - return err - } - iNdEx = postIndex default: iNdEx = preIndex skippy, err := skipTableSchedule(dAtA[iNdEx:]) @@ -3186,9 +2896,6 @@ func (m *TableStatus) Unmarshal(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - if m.Checkpoint == nil { - m.Checkpoint = &Checkpoint{} - } if err := m.Checkpoint.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -3214,7 +2921,7 @@ func (m *TableStatus) Unmarshal(dAtA []byte) error { } return nil } -func (m *Sync) Unmarshal(dAtA []byte) error { +func (m *HeartbeatResponse) Unmarshal(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -3237,15 +2944,15 @@ func (m *Sync) Unmarshal(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: Sync: wiretype end group for non-group") + return fmt.Errorf("proto: HeartbeatResponse: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: Sync: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: HeartbeatResponse: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field ProcessorEpoch", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Tables", wireType) } var msglen int for shift := uint(0); ; shift += 7 { @@ -3272,18 +2979,155 @@ func (m *Sync) Unmarshal(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - if m.ProcessorEpoch == nil { - m.ProcessorEpoch = &ProcessorEpoch{} - } - if err := m.ProcessorEpoch.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + m.Tables = append(m.Tables, TableStatus{}) + if err := m.Tables[len(m.Tables)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { return err } iNdEx = postIndex case 2: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field IsStopping", wireType) + } + var v int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowTableSchedule + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + m.IsStopping = bool(v != 0) + default: + iNdEx = preIndex + skippy, err := skipTableSchedule(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return ErrInvalidLengthTableSchedule + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *OwnerRevision) 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 ErrIntOverflowTableSchedule + } + 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: OwnerRevision: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: OwnerRevision: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Revision", wireType) + } + m.Revision = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowTableSchedule + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.Revision |= int64(b&0x7F) << shift + if b < 0x80 { + break + } + } + default: + iNdEx = preIndex + skippy, err := skipTableSchedule(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return ErrInvalidLengthTableSchedule + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *ProcessorEpoch) 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 ErrIntOverflowTableSchedule + } + 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: ProcessorEpoch: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ProcessorEpoch: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Tables", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Epoch", wireType) } - var msglen int + var stringLen uint64 for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowTableSchedule @@ -3293,25 +3137,23 @@ func (m *Sync) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - msglen |= int(b&0x7F) << shift + stringLen |= uint64(b&0x7F) << shift if b < 0x80 { break } } - if msglen < 0 { + intStringLen := int(stringLen) + if intStringLen < 0 { return ErrInvalidLengthTableSchedule } - postIndex := iNdEx + msglen + postIndex := iNdEx + intStringLen if postIndex < 0 { return ErrInvalidLengthTableSchedule } if postIndex > l { return io.ErrUnexpectedEOF } - m.Tables = append(m.Tables, &TableStatus{}) - if err := m.Tables[len(m.Tables)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { - return err - } + m.Epoch = string(dAtA[iNdEx:postIndex]) iNdEx = postIndex default: iNdEx = preIndex @@ -3556,7 +3398,7 @@ func (m *Message) Unmarshal(dAtA []byte) error { iNdEx = postIndex case 7: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Checkpoint", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Heartbeat", wireType) } var msglen int for shift := uint(0); ; shift += 7 { @@ -3583,16 +3425,16 @@ func (m *Message) Unmarshal(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - if m.Checkpoint == nil { - m.Checkpoint = &Checkpoint{} + if m.Heartbeat == nil { + m.Heartbeat = &Heartbeat{} } - if err := m.Checkpoint.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + if err := m.Heartbeat.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { return err } iNdEx = postIndex case 8: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Announce", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field HeartbeatResponse", wireType) } var msglen int for shift := uint(0); ; shift += 7 { @@ -3619,16 +3461,16 @@ func (m *Message) Unmarshal(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - if m.Announce == nil { - m.Announce = &Announce{} + if m.HeartbeatResponse == nil { + m.HeartbeatResponse = &HeartbeatResponse{} } - if err := m.Announce.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + if err := m.HeartbeatResponse.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { return err } iNdEx = postIndex case 9: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Sync", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Checkpoint", wireType) } var msglen int for shift := uint(0); ; shift += 7 { @@ -3655,10 +3497,10 @@ func (m *Message) Unmarshal(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - if m.Sync == nil { - m.Sync = &Sync{} + if m.Checkpoint == nil { + m.Checkpoint = &Checkpoint{} } - if err := m.Sync.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + if err := m.Checkpoint.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { return err } iNdEx = postIndex @@ -3744,6 +3586,72 @@ func (m *Message_Header) Unmarshal(dAtA []byte) error { } m.Version = string(dAtA[iNdEx:postIndex]) iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field OwnerRevision", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowTableSchedule + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthTableSchedule + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthTableSchedule + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if err := m.OwnerRevision.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ProcessorEpoch", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowTableSchedule + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthTableSchedule + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthTableSchedule + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if err := m.ProcessorEpoch.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex default: iNdEx = preIndex skippy, err := skipTableSchedule(dAtA[iNdEx:]) diff --git a/cdc/scheduler/internal/tp/transport.go b/cdc/scheduler/internal/tp/transport.go index ccdb6d1c2b3..9eff5fa8650 100644 --- a/cdc/scheduler/internal/tp/transport.go +++ b/cdc/scheduler/internal/tp/transport.go @@ -28,7 +28,7 @@ import ( ) type transport interface { - Send(ctx context.Context, to model.CaptureID, msgs []*schedulepb.Message) (bool, error) + Send(ctx context.Context, msgs []*schedulepb.Message) error Recv(ctx context.Context) ([]*schedulepb.Message, error) } @@ -82,23 +82,24 @@ func newTranport( } func (t *p2pTransport) Send( - ctx context.Context, to model.CaptureID, msgs []*schedulepb.Message, -) (bool, error) { - client := t.messageRouter.GetClient(to) - if client == nil { - log.Warn("tpscheduler: no message client found, retry later", - zap.String("namespace", t.changefeed.Namespace), - zap.String("changefeed", t.changefeed.ID), - zap.String("to", to)) - return false, nil - } - + ctx context.Context, msgs []*schedulepb.Message, +) error { for i := range msgs { value := msgs[i] + to := value.To + client := t.messageRouter.GetClient(to) + if client == nil { + log.Warn("tpscheduler: no message client found, retry later", + zap.String("namespace", t.changefeed.Namespace), + zap.String("changefeed", t.changefeed.ID), + zap.String("to", to)) + continue + } + _, err := client.TrySendMessage(ctx, t.topic, value) if err != nil { if cerror.ErrPeerMessageSendTryAgain.Equal(err) { - return false, nil + return nil } if cerror.ErrPeerMessageClientClosed.Equal(err) { log.Warn("tpscheduler: peer messaging client is closed"+ @@ -107,18 +108,17 @@ func (t *p2pTransport) Send( zap.String("namespace", t.changefeed.Namespace), zap.String("changefeed", t.changefeed.ID), zap.String("to", to)) - return false, nil + return nil } - return false, errors.Trace(err) + return errors.Trace(err) } } log.Debug("tpscheduler: all messages sent", zap.String("namespace", t.changefeed.Namespace), zap.String("changefeed", t.changefeed.ID), - zap.Int("len", len(msgs)), - zap.String("to", to)) - return true, nil + zap.Int("len", len(msgs))) + return nil } func (t *p2pTransport) Recv(ctx context.Context) ([]*schedulepb.Message, error) { diff --git a/cdc/scheduler/internal/tp/transport_test.go b/cdc/scheduler/internal/tp/transport_test.go index d6c9053853b..4593b2bf2aa 100644 --- a/cdc/scheduler/internal/tp/transport_test.go +++ b/cdc/scheduler/internal/tp/transport_test.go @@ -24,7 +24,7 @@ import ( "github.com/stretchr/testify/require" ) -func TestSendRecv(t *testing.T) { +func TestTransSendRecv(t *testing.T) { t.Parallel() cluster := p2p.NewMockCluster(t, 3) @@ -44,9 +44,8 @@ func TestSendRecv(t *testing.T) { // Send messages for _, trans := range transMap { for addr := range transMap { - done, err := trans.Send(ctx, addr, []*schedulepb.Message{{To: addr}}) + err := trans.Send(ctx, []*schedulepb.Message{{To: addr}}) require.Nil(t, err) - require.True(t, done) } } @@ -64,7 +63,7 @@ func TestSendRecv(t *testing.T) { } } -func TestUnknownAddr(t *testing.T) { +func TestTransUnknownAddr(t *testing.T) { t.Parallel() cluster := p2p.NewMockCluster(t, 3) @@ -84,13 +83,12 @@ func TestUnknownAddr(t *testing.T) { unknownAddr := "unknown" require.NotContains(t, transMap, unknownAddr) for _, trans := range transMap { - done, err := trans.Send(ctx, unknownAddr, []*schedulepb.Message{}) + err := trans.Send(ctx, []*schedulepb.Message{{To: unknownAddr}}) require.Nil(t, err) - require.False(t, done) } } -func TestEmptyRecv(t *testing.T) { +func TestTransEmptyRecv(t *testing.T) { t.Parallel() cluster := p2p.NewMockCluster(t, 3) diff --git a/errors.toml b/errors.toml index 19b67382322..a171dde8b21 100755 --- a/errors.toml +++ b/errors.toml @@ -851,6 +851,16 @@ error = ''' regions not completely left cover span, span %v regions: %v ''' +["CDC:ErrReplicationSetInconsistent"] +error = ''' +replication set inconsistent: %s +''' + +["CDC:ErrReplicationSetMultiplePrimaryError"] +error = ''' +replication set multiple primary: %s +''' + ["CDC:ErrResolveLocks"] error = ''' resolve locks failed diff --git a/proto/TableSchedule.proto b/proto/table_schedule.proto similarity index 59% rename from proto/TableSchedule.proto rename to proto/table_schedule.proto index 000f2901c7f..08f768107c9 100644 --- a/proto/TableSchedule.proto +++ b/proto/table_schedule.proto @@ -17,15 +17,11 @@ option go_package = "schedulepb"; import "gogoproto/gogo.proto"; -message OwnerRevision { int64 revision = 1; } - -message ProcessorEpoch { string epoch = 1; } +option(gogoproto.goproto_enum_prefix_all) = false; message Checkpoint { - uint64 checkpoint_ts = - 1 [(gogoproto.casttype) = "github.com/pingcap/tiflow/cdc/model.Ts"]; - uint64 resolved_ts = - 2 [(gogoproto.casttype) = "github.com/pingcap/tiflow/cdc/model.Ts"]; + uint64 checkpoint_ts = 1 [(gogoproto.casttype) = "github.com/pingcap/tiflow/cdc/model.Ts"]; + uint64 resolved_ts = 2 [(gogoproto.casttype) = "github.com/pingcap/tiflow/cdc/model.Ts"]; } message AddTableRequest { @@ -45,9 +41,8 @@ message RemoveTableRequest { } message DispatchTableRequest { - OwnerRevision owner_revision = 1; - AddTableRequest add_table = 2; - RemoveTableRequest remove_table = 3; + AddTableRequest add_table = 1; + RemoveTableRequest remove_table = 2; } message AddTableResponse { @@ -70,13 +65,11 @@ message RemoveTableResponse { } message DispatchTableResponse { - ProcessorEpoch processor_epoch = 1; - - AddTableResponse add_table = 2; - RemoveTableResponse remove_table = 3; + AddTableResponse add_table = 1; + RemoveTableResponse remove_table = 2; } -message Announce { OwnerRevision owner_revision = 1; } +message Heartbeat {} // TableState is the state of table repliction in processor. // @@ -88,13 +81,13 @@ message Announce { OwnerRevision owner_revision = 1; } // │ Stopped │ <─┤ Stopping │ <─┤ Replicating │ // └─────────┘ └──────────┘ └─────────────┘ enum TableState { - Unknown = 0; - Absent = 1; - Preparing = 2; - Prepared = 3; - Replicating = 4; - Stopping = 5; - Stopped = 6; + Unknown = 0 [(gogoproto.enumvalue_customname) = "TableStateUnknown"]; + Absent = 1 [(gogoproto.enumvalue_customname) = "TableStateAbsent"]; + Preparing = 2 [(gogoproto.enumvalue_customname) = "TableStatePreparing"]; + Prepared = 3 [(gogoproto.enumvalue_customname) = "TableStatePrepared"]; + Replicating = 4 [(gogoproto.enumvalue_customname) = "TableStateReplicating"]; + Stopping = 5 [(gogoproto.enumvalue_customname) = "TableStateStopping"]; + Stopped = 6 [(gogoproto.enumvalue_customname) = "TableStateStopped"]; } message TableStatus { @@ -103,38 +96,41 @@ message TableStatus { (gogoproto.customname) = "TableID" ]; TableState state = 2; - Checkpoint checkpoint = 3; + Checkpoint checkpoint = 3 [(gogoproto.nullable) = false]; } -message Sync { - ProcessorEpoch processor_epoch = 1; - - repeated TableStatus tables = 2; +message HeartbeatResponse { + repeated TableStatus tables = 1 [(gogoproto.nullable) = false]; + bool is_stopping = 2; } enum MessageType { - MsgUnknown = 0; - MsgCheckpoint = 1; - MsgDispatchTableRequest = 2; - MsgDispatchTableResponse = 3; - MsgAnnounce = 4; - MsgSync = 5; + MsgUnknown = 0 [(gogoproto.enumvalue_customname) = "MsgUnknown"]; + MsgCheckpoint = 1 [(gogoproto.enumvalue_customname) = "MsgCheckpoint"]; + MsgDispatchTableRequest = 2 [(gogoproto.enumvalue_customname) = "MsgDispatchTableRequest"]; + MsgDispatchTableResponse = 3 [(gogoproto.enumvalue_customname) = "MsgDispatchTableResponse"]; + MsgHeartbeat = 4 [(gogoproto.enumvalue_customname) = "MsgHeartbeat"]; + MsgHeartbeatResponse = 5 [(gogoproto.enumvalue_customname) = "MsgHeartbeatResponse"]; } +message OwnerRevision { int64 revision = 1; } + +message ProcessorEpoch { string epoch = 1; } + message Message { message Header { // The semantic version of the node that sent this message. string version = 1; + OwnerRevision owner_revision = 2 [(gogoproto.nullable) = false]; + ProcessorEpoch processor_epoch = 3 [(gogoproto.nullable) = false]; } Header header = 1; MessageType msg_type = 2; - string from = 3 [(gogoproto.casttype) = - "github.com/pingcap/tiflow/cdc/model.CaptureID"]; - string to = 4 [(gogoproto.casttype) = - "github.com/pingcap/tiflow/cdc/model.CaptureID"]; + string from = 3 [(gogoproto.casttype) = "github.com/pingcap/tiflow/cdc/model.CaptureID"]; + string to = 4 [(gogoproto.casttype) = "github.com/pingcap/tiflow/cdc/model.CaptureID"]; DispatchTableRequest dispatch_table_request = 5; DispatchTableResponse dispatch_table_response = 6; - Checkpoint checkpoint = 7; - Announce announce = 8; - Sync sync = 9; + Heartbeat heartbeat = 7; + HeartbeatResponse heartbeat_response = 8; + Checkpoint checkpoint = 9; } diff --git a/scripts/check-diff-line-width.sh b/scripts/check-diff-line-width.sh index 08c712dcdfb..254d9227ce4 100755 --- a/scripts/check-diff-line-width.sh +++ b/scripts/check-diff-line-width.sh @@ -23,7 +23,10 @@ BASE_HASH=$(git --no-pager log -E --grep='\(#[0-9]+\)$' -n 1 --format=format:%H) WARN_THRESHOLD=80 ERROR_THRESHOLD=100 -git --no-pager diff $BASE_HASH -U0 -- cdc pkg cmd -- ':(exclude)*_gen.go' -- ':(exclude)*_gen_test.go' | +git --no-pager diff $BASE_HASH -U0 -- cdc pkg cmd \ + -- ':(exclude)*_gen.go' \ + -- ':(exclude)*_gen_test.go' \ + -- ':(exclude)*.pb.go' | grep -E '^\+' | grep -vE '^\+\+\+' | sed 's/\t/ /g' | awk " diff --git a/scripts/generate-protobuf.sh b/scripts/generate-protobuf.sh index 69838409560..5c46be65734 100755 --- a/scripts/generate-protobuf.sh +++ b/scripts/generate-protobuf.sh @@ -55,4 +55,4 @@ echo "generate schedulepb..." mkdir -p ./cdc/scheduler/internal/tp/schedulepb $TOOLS_BIN_DIR/protoc -I"./proto" -I"$TOOLS_INCLUDE_DIR" \ --plugin=protoc-gen-gogofaster="$GOGO_FASTER" \ - --gogofaster_out=plugins=grpc:./cdc/scheduler/internal/tp/schedulepb ./proto/TableSchedule.proto + --gogofaster_out=plugins=grpc:./cdc/scheduler/internal/tp/schedulepb ./proto/table_schedule.proto diff --git a/tools/check/check-errdoc.sh b/tools/check/check-errdoc.sh index 236ccf31f00..04c7c2828fb 100755 --- a/tools/check/check-errdoc.sh +++ b/tools/check/check-errdoc.sh @@ -17,5 +17,5 @@ set -euo pipefail cd -P . cp errors.toml /tmp/errors.toml.before -./tools/bin/errdoc-gen --source . --module github.com/pingcap/tiflow --output errors.toml --ignore proto,dm,deployments,engine +./tools/bin/errdoc-gen --source . --module github.com/pingcap/tiflow --output errors.toml --ignore proto,dm,deployments,engine,cdc/scheduler/internal diff -q errors.toml /tmp/errors.toml.before