From ddd140af93046dc6762a9ca65f81845285647737 Mon Sep 17 00:00:00 2001 From: Neil Shen Date: Mon, 7 Mar 2022 11:59:48 +0800 Subject: [PATCH 1/7] owner(ticdc): fix prometheus panic (#4759) close pingcap/tiflow#4742 --- cdc/owner/metrics.go | 2 +- cdc/owner/owner.go | 20 +++++++++++++------- cdc/processor/pipeline/metrics.go | 26 ++++++++------------------ cdc/sorter/leveldb/metrics.go | 8 -------- metrics/grafana/ticdc.json | 8 ++++---- 5 files changed, 26 insertions(+), 38 deletions(-) diff --git a/cdc/owner/metrics.go b/cdc/owner/metrics.go index 734d0fedccc..fa29fec9e84 100644 --- a/cdc/owner/metrics.go +++ b/cdc/owner/metrics.go @@ -61,7 +61,7 @@ var ( Subsystem: "owner", Name: "maintain_table_num", Help: "number of replicated tables maintained in owner", - }, []string{"changefeed", "type"}) + }, []string{"changefeed", "capture", "type"}) changefeedStatusGauge = prometheus.NewGaugeVec( prometheus.GaugeOpts{ Namespace: "ticdc", diff --git a/cdc/owner/owner.go b/cdc/owner/owner.go index d428a053a06..a4224d67505 100644 --- a/cdc/owner/owner.go +++ b/cdc/owner/owner.go @@ -352,23 +352,29 @@ func (o *ownerImpl) updateMetrics(state *orchestrator.GlobalReactorState) { pendingCounts := infoProvider.GetPendingTableCounts() for captureID, info := range o.captures { - ownerMaintainTableNumGauge.WithLabelValues( - cfID, info.AdvertiseAddr, maintainTableTypeTotal).Set(float64(totalCounts[captureID])) - ownerMaintainTableNumGauge.WithLabelValues( - cfID, info.AdvertiseAddr, maintainTableTypeWip).Set(float64(pendingCounts[captureID])) + ownerMaintainTableNumGauge. + WithLabelValues(cfID, info.AdvertiseAddr, maintainTableTypeTotal). + Set(float64(totalCounts[captureID])) + ownerMaintainTableNumGauge. + WithLabelValues(cfID, info.AdvertiseAddr, maintainTableTypeWip). + Set(float64(pendingCounts[captureID])) } } return } for changefeedID, changefeedState := range state.Changefeeds { - for captureID := range state.Captures { + for captureID, captureInfo := range state.Captures { taskStatus, exist := changefeedState.TaskStatuses[captureID] if !exist { continue } - ownerMaintainTableNumGauge.WithLabelValues(changefeedID, maintainTableTypeTotal).Set(float64(len(taskStatus.Tables))) - ownerMaintainTableNumGauge.WithLabelValues(changefeedID, maintainTableTypeWip).Set(float64(len(taskStatus.Operation))) + ownerMaintainTableNumGauge. + WithLabelValues(changefeedID, captureInfo.AdvertiseAddr, maintainTableTypeTotal). + Set(float64(len(taskStatus.Tables))) + ownerMaintainTableNumGauge. + WithLabelValues(changefeedID, captureInfo.AdvertiseAddr, maintainTableTypeWip). + Set(float64(len(taskStatus.Operation))) if changefeedState.Info != nil { changefeedStatusGauge.WithLabelValues(changefeedID).Set(float64(changefeedState.Info.State.ToInt())) } diff --git a/cdc/processor/pipeline/metrics.go b/cdc/processor/pipeline/metrics.go index 12775288cff..6a593278a5b 100644 --- a/cdc/processor/pipeline/metrics.go +++ b/cdc/processor/pipeline/metrics.go @@ -17,26 +17,16 @@ import ( "github.com/prometheus/client_golang/prometheus" ) -var ( - txnCounter = prometheus.NewCounterVec( - prometheus.CounterOpts{ - Namespace: "ticdc", - Subsystem: "processor", - Name: "txn_count", - Help: "txn count received/executed by this processor", - }, []string{"type", "changefeed"}) - tableMemoryHistogram = prometheus.NewHistogramVec( - prometheus.HistogramOpts{ - Namespace: "ticdc", - Subsystem: "processor", - Name: "table_memory_consumption", - Help: "estimated memory consumption for a table after the sorter", - Buckets: prometheus.ExponentialBuckets(1*1024*1024 /* mb */, 2, 10), - }, []string{"changefeed"}) -) +var tableMemoryHistogram = prometheus.NewHistogramVec( + prometheus.HistogramOpts{ + Namespace: "ticdc", + Subsystem: "processor", + Name: "table_memory_consumption", + Help: "estimated memory consumption for a table after the sorter", + Buckets: prometheus.ExponentialBuckets(1*1024*1024 /* mb */, 2, 10), + }, []string{"changefeed"}) // InitMetrics registers all metrics used in processor func InitMetrics(registry *prometheus.Registry) { - registry.MustRegister(txnCounter) registry.MustRegister(tableMemoryHistogram) } diff --git a/cdc/sorter/leveldb/metrics.go b/cdc/sorter/leveldb/metrics.go index dfdcf815679..3fb0c3a3558 100644 --- a/cdc/sorter/leveldb/metrics.go +++ b/cdc/sorter/leveldb/metrics.go @@ -49,13 +49,6 @@ var ( Help: "Bucketed histogram of db sorter iterator read duration", Buckets: prometheus.ExponentialBuckets(0.004, 2.0, 20), }, []string{"id", "call"}) - - sorterCleanupKVCounter = prometheus.NewCounterVec(prometheus.CounterOpts{ - Namespace: "ticdc", - Subsystem: "sorter", - Name: "db_cleanup_kv_total", - Help: "The total number of cleaned up kv entries", - }, []string{"id"}) ) // InitMetrics registers all metrics in this file @@ -64,5 +57,4 @@ func InitMetrics(registry *prometheus.Registry) { registry.MustRegister(sorterCompactDurationHistogram) registry.MustRegister(sorterWriteBytesHistogram) registry.MustRegister(sorterIterReadDurationHistogram) - registry.MustRegister(sorterCleanupKVCounter) } diff --git a/metrics/grafana/ticdc.json b/metrics/grafana/ticdc.json index 076b2d04862..c21ae2e056a 100644 --- a/metrics/grafana/ticdc.json +++ b/metrics/grafana/ticdc.json @@ -1322,19 +1322,19 @@ ], "targets": [ { - "expr": "sum(ticdc_owner_maintain_table_num{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\",instance=~\"$ticdc_instance\",type=\"total\"}) by (instance)", + "expr": "sum(ticdc_owner_maintain_table_num{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\",instance=~\"$ticdc_instance\",type=\"total\"}) by (capture)", "format": "time_series", "interval": "", "intervalFactor": 1, - "legendFormat": "{{instance}}-total", + "legendFormat": "{{capture}}-total", "refId": "A" }, { - "expr": "sum(ticdc_owner_maintain_table_num{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\",instance=~\"$ticdc_instance\",type=\"wip\"}) by (instance)", + "expr": "sum(ticdc_owner_maintain_table_num{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\",instance=~\"$ticdc_instance\",type=\"wip\"}) by (capture)", "format": "time_series", "interval": "", "intervalFactor": 1, - "legendFormat": "{{instance}}-wip", + "legendFormat": "{{capture}}-wip", "refId": "B" } ], From 0578db337d783643cfab9f25ccb28a5dd0de5806 Mon Sep 17 00:00:00 2001 From: Zixiong Liu Date: Mon, 7 Mar 2022 14:45:48 +0800 Subject: [PATCH 2/7] scheduler(cdc): add ProcessorEpoch (#4768) close pingcap/tiflow#4769 --- cdc/model/protocol.go | 25 ++- cdc/model/protocol_test.go | 8 +- cdc/owner/scheduler.go | 29 +++- cdc/processor/agent.go | 97 ++++++++--- cdc/processor/agent_test.go | 7 +- cdc/scheduler/agent.go | 65 ++++++-- cdc/scheduler/agent_mock.go | 8 +- cdc/scheduler/agent_test.go | 129 ++++++++++++--- cdc/scheduler/schedule_dispatcher.go | 51 +++++- cdc/scheduler/schedule_dispatcher_test.go | 192 ++++++++++++++++------ errors.toml | 2 +- pkg/errors/errors.go | 2 +- 12 files changed, 491 insertions(+), 124 deletions(-) diff --git a/cdc/model/protocol.go b/cdc/model/protocol.go index b29a67da275..ea6475135e2 100644 --- a/cdc/model/protocol.go +++ b/cdc/model/protocol.go @@ -31,9 +31,10 @@ func DispatchTableTopic(changefeedID ChangeFeedID) p2p.Topic { // DispatchTableMessage is the message body for dispatching a table. type DispatchTableMessage struct { - OwnerRev int64 `json:"owner-rev"` - ID TableID `json:"id"` - IsDelete bool `json:"is-delete"` + OwnerRev int64 `json:"owner-rev"` + Epoch ProcessorEpoch `json:"epoch"` + ID TableID `json:"id"` + IsDelete bool `json:"is-delete"` } // DispatchTableResponseTopic returns a message topic for the result of @@ -44,7 +45,8 @@ func DispatchTableResponseTopic(changefeedID ChangeFeedID) p2p.Topic { // DispatchTableResponseMessage is the message body for the result of dispatching a table. type DispatchTableResponseMessage struct { - ID TableID `json:"id"` + ID TableID `json:"id"` + Epoch ProcessorEpoch `json:"epoch"` } // AnnounceTopic returns a message topic for announcing an ownership change. @@ -64,14 +66,23 @@ func SyncTopic(changefeedID ChangeFeedID) p2p.Topic { return fmt.Sprintf("send-status-resp/%s", changefeedID) } +// ProcessorEpoch designates a continuous period of the processor working normally. +type ProcessorEpoch = string + // SyncMessage is the message body for syncing the current states of a processor. // MsgPack serialization has been implemented to minimize the size of the message. type SyncMessage struct { // Sends the processor's version for compatibility check ProcessorVersion string - Running []TableID - Adding []TableID - Removing []TableID + + // Epoch is reset to a unique value when the processor has + // encountered an internal error or other events so that + // it has to re-sync its states with the Owner. + Epoch ProcessorEpoch + + Running []TableID + Adding []TableID + Removing []TableID } // Marshal serializes the message into MsgPack format. diff --git a/cdc/model/protocol_test.go b/cdc/model/protocol_test.go index 6c4654b9a72..a15e58c3cca 100644 --- a/cdc/model/protocol_test.go +++ b/cdc/model/protocol_test.go @@ -65,21 +65,23 @@ func makeVeryLargeSyncMessage() *SyncMessage { func TestMarshalDispatchTableMessage(t *testing.T) { msg := &DispatchTableMessage{ OwnerRev: 1, + Epoch: "test-epoch", ID: TableID(1), IsDelete: true, } bytes, err := json.Marshal(msg) require.NoError(t, err) - require.Equal(t, `{"owner-rev":1,"id":1,"is-delete":true}`, string(bytes)) + require.Equal(t, `{"owner-rev":1,"epoch":"test-epoch","id":1,"is-delete":true}`, string(bytes)) } func TestMarshalDispatchTableResponseMessage(t *testing.T) { msg := &DispatchTableResponseMessage{ - ID: TableID(1), + ID: TableID(1), + Epoch: "test-epoch", } bytes, err := json.Marshal(msg) require.NoError(t, err) - require.Equal(t, `{"id":1}`, string(bytes)) + require.Equal(t, `{"id":1,"epoch":"test-epoch"}`, string(bytes)) } func TestMarshalAnnounceMessage(t *testing.T) { diff --git a/cdc/owner/scheduler.go b/cdc/owner/scheduler.go index 7f9650948a1..351da5fb861 100644 --- a/cdc/owner/scheduler.go +++ b/cdc/owner/scheduler.go @@ -127,14 +127,27 @@ func (s *schedulerV2) DispatchTable( tableID model.TableID, captureID model.CaptureID, isDelete bool, + epoch model.ProcessorEpoch, ) (done bool, err error) { topic := model.DispatchTableTopic(changeFeedID) message := &model.DispatchTableMessage{ OwnerRev: ctx.GlobalVars().OwnerRevision, ID: tableID, IsDelete: isDelete, + Epoch: epoch, } + defer func() { + if err != nil { + return + } + log.Info("schedulerV2: DispatchTable", + zap.Any("message", message), + zap.Any("successful", done), + zap.String("changefeedID", changeFeedID), + zap.String("captureID", captureID)) + }() + ok, err := s.trySendMessage(ctx, captureID, topic, message) if err != nil { return false, errors.Trace(err) @@ -155,13 +168,24 @@ func (s *schedulerV2) Announce( ctx context.Context, changeFeedID model.ChangeFeedID, captureID model.CaptureID, -) (bool, error) { +) (done bool, err error) { topic := model.AnnounceTopic(changeFeedID) message := &model.AnnounceMessage{ OwnerRev: ctx.GlobalVars().OwnerRevision, OwnerVersion: version.ReleaseSemver(), } + defer func() { + if err != nil { + return + } + log.Info("schedulerV2: Announce", + zap.Any("message", message), + zap.Any("successful", done), + zap.String("changefeedID", changeFeedID), + zap.String("captureID", captureID)) + }() + ok, err := s.trySendMessage(ctx, captureID, topic, message) if err != nil { return false, errors.Trace(err) @@ -239,7 +263,7 @@ func (s *schedulerV2) registerPeerMessageHandlers(ctx context.Context) (ret erro func(sender string, messageI interface{}) error { message := messageI.(*model.DispatchTableResponseMessage) s.stats.RecordDispatchResponse() - s.OnAgentFinishedTableOperation(sender, message.ID) + s.OnAgentFinishedTableOperation(sender, message.ID, message.Epoch) return nil }) if err != nil { @@ -256,6 +280,7 @@ func (s *schedulerV2) registerPeerMessageHandlers(ctx context.Context) (ret erro s.stats.RecordSync() s.OnAgentSyncTaskStatuses( sender, + message.Epoch, message.Running, message.Adding, message.Removing) diff --git a/cdc/processor/agent.go b/cdc/processor/agent.go index 18d8762c9cb..406f3779fd2 100644 --- a/cdc/processor/agent.go +++ b/cdc/processor/agent.go @@ -17,6 +17,8 @@ import ( stdContext "context" "time" + "go.uber.org/zap/zapcore" + "github.com/benbjohnson/clock" "github.com/pingcap/errors" "github.com/pingcap/log" @@ -161,11 +163,23 @@ func (a *agentImpl) Tick(ctx context.Context) error { func (a *agentImpl) FinishTableOperation( ctx context.Context, tableID model.TableID, -) (bool, error) { - done, err := a.trySendMessage( + epoch model.ProcessorEpoch, +) (done bool, err error) { + message := &model.DispatchTableResponseMessage{ID: tableID, Epoch: epoch} + defer func() { + if err != nil { + return + } + log.Info("SchedulerAgent: FinishTableOperation", zap.Any("message", message), + zap.Bool("successful", done), + zap.String("changefeedID", a.changeFeed), + zap.String("ownerID", a.ownerCaptureID)) + }() + + done, err = a.trySendMessage( ctx, a.ownerCaptureID, model.DispatchTableResponseTopic(a.changeFeed), - &model.DispatchTableResponseMessage{ID: tableID}) + message) if err != nil { return false, errors.Trace(err) } @@ -173,19 +187,46 @@ func (a *agentImpl) FinishTableOperation( } func (a *agentImpl) SyncTaskStatuses( - ctx context.Context, - running, adding, removing []model.TableID, -) (bool, error) { - done, err := a.trySendMessage( + ctx context.Context, epoch model.ProcessorEpoch, adding, removing, running []model.TableID, +) (done bool, err error) { + if !a.Barrier(ctx) { + // The Sync message needs to be strongly ordered w.r.t. other messages. + return false, nil + } + + message := &model.SyncMessage{ + ProcessorVersion: version.ReleaseSemver(), + Epoch: epoch, + Running: running, + Adding: adding, + Removing: removing, + } + + defer func() { + if err != nil { + return + } + if log.GetLevel() == zapcore.DebugLevel { + // The message can be REALLY large, so we do not print it + // unless the log level is debug. + log.Debug("SchedulerAgent: SyncTaskStatuses", + zap.Any("message", message), + zap.Bool("successful", done), + zap.String("changefeedID", a.changeFeed), + zap.String("ownerID", a.ownerCaptureID)) + return + } + log.Info("SchedulerAgent: SyncTaskStatuses", + zap.Bool("successful", done), + zap.String("changefeedID", a.changeFeed), + zap.String("ownerID", a.ownerCaptureID)) + }() + + done, err = a.trySendMessage( ctx, a.ownerCaptureID, model.SyncTopic(a.changeFeed), - &model.SyncMessage{ - ProcessorVersion: version.ReleaseSemver(), - Running: running, - Adding: adding, - Removing: removing, - }) + message) if err != nil { return false, errors.Trace(err) } @@ -196,15 +237,30 @@ func (a *agentImpl) SendCheckpoint( ctx context.Context, checkpointTs model.Ts, resolvedTs model.Ts, -) (bool, error) { - done, err := a.trySendMessage( +) (done bool, err error) { + message := &model.CheckpointMessage{ + CheckpointTs: checkpointTs, + ResolvedTs: resolvedTs, + } + + defer func() { + if err != nil { + return + } + // This log is very often, so we only print it if the + // log level is debug. + log.Debug("SchedulerAgent: SendCheckpoint", + zap.Any("message", message), + zap.Bool("successful", done), + zap.String("changefeedID", a.changeFeed), + zap.String("ownerID", a.ownerCaptureID)) + }() + + done, err = a.trySendMessage( ctx, a.ownerCaptureID, model.CheckpointTopic(a.changeFeed), - &model.CheckpointMessage{ - CheckpointTs: checkpointTs, - ResolvedTs: resolvedTs, - }) + message) if err != nil { return false, errors.Trace(err) } @@ -339,7 +395,8 @@ func (a *agentImpl) registerPeerMessageHandlers() (ret error) { ownerCapture, message.OwnerRev, message.ID, - message.IsDelete) + message.IsDelete, + message.Epoch) return nil }) if err != nil { diff --git a/cdc/processor/agent_test.go b/cdc/processor/agent_test.go index f6ed64d2c17..9037e55039c 100644 --- a/cdc/processor/agent_test.go +++ b/cdc/processor/agent_test.go @@ -203,6 +203,7 @@ func TestAgentBasics(t *testing.T) { case syncMsg := <-suite.syncCh: require.Equal(t, &model.SyncMessage{ ProcessorVersion: version.ReleaseSemver(), + Epoch: agent.CurrentEpoch(), Running: nil, Adding: nil, Removing: nil, @@ -211,6 +212,7 @@ func TestAgentBasics(t *testing.T) { _, err = suite.ownerMessageClient.SendMessage(suite.ctx, model.DispatchTableTopic("cf-1"), &model.DispatchTableMessage{ OwnerRev: 1, + Epoch: agent.CurrentEpoch(), ID: 1, IsDelete: false, }) @@ -263,7 +265,8 @@ func TestAgentBasics(t *testing.T) { return false case msg := <-suite.dispatchResponseCh: require.Equal(t, &model.DispatchTableResponseMessage{ - ID: 1, + ID: 1, + Epoch: agent.CurrentEpoch(), }, msg) return true default: @@ -317,6 +320,7 @@ func TestAgentNoOwnerAtStartUp(t *testing.T) { case syncMsg := <-suite.syncCh: require.Equal(t, &model.SyncMessage{ ProcessorVersion: version.ReleaseSemver(), + Epoch: agent.CurrentEpoch(), Running: nil, Adding: nil, Removing: nil, @@ -371,6 +375,7 @@ func TestAgentTolerateClientClosed(t *testing.T) { case syncMsg := <-suite.syncCh: require.Equal(t, &model.SyncMessage{ ProcessorVersion: version.ReleaseSemver(), + Epoch: agent.CurrentEpoch(), Running: nil, Adding: nil, Removing: nil, diff --git a/cdc/scheduler/agent.go b/cdc/scheduler/agent.go index 4a874283c5a..1fae1c10014 100644 --- a/cdc/scheduler/agent.go +++ b/cdc/scheduler/agent.go @@ -18,6 +18,7 @@ import ( "time" "github.com/edwingeng/deque" + "github.com/google/uuid" "github.com/pingcap/errors" "github.com/pingcap/log" "github.com/pingcap/tiflow/cdc/model" @@ -70,12 +71,11 @@ type TableExecutor interface { // by the owner. type ProcessorMessenger interface { // FinishTableOperation notifies the owner that a table operation has finished. - FinishTableOperation(ctx context.Context, tableID model.TableID) (done bool, err error) + FinishTableOperation(ctx context.Context, tableID model.TableID, epoch model.ProcessorEpoch) (done bool, err error) // SyncTaskStatuses informs the owner of the processor's current internal state. - SyncTaskStatuses(ctx context.Context, running, adding, removing []model.TableID) (done bool, err error) + SyncTaskStatuses(ctx context.Context, epoch model.ProcessorEpoch, adding, removing, running []model.TableID) (done bool, err error) // SendCheckpoint sends the owner the processor's local watermarks, i.e., checkpoint-ts and resolved-ts. SendCheckpoint(ctx context.Context, checkpointTs model.Ts, resolvedTs model.Ts) (done bool, err error) - // Barrier returns whether there is a pending message not yet acknowledged by the owner. Barrier(ctx context.Context) (done bool) // OnOwnerChanged is called when the owner is changed. @@ -97,6 +97,10 @@ type BaseAgent struct { executor TableExecutor communicator ProcessorMessenger + epochMu sync.RWMutex + // epoch is reset on each Sync message. + epoch model.ProcessorEpoch + // pendingOpsMu protects pendingOps. // Note that we need a mutex because some methods are expected // to be called from a message handler goroutine. @@ -136,7 +140,7 @@ func NewBaseAgent( config *BaseAgentConfig, ) *BaseAgent { logger := log.L().With(zap.String("changefeed", changeFeedID)) - return &BaseAgent{ + ret := &BaseAgent{ pendingOps: deque.NewDeque(), tableOperations: map[model.TableID]*agentOperation{}, logger: logger, @@ -148,6 +152,8 @@ func NewBaseAgent( ownerHasChanged: atomic.NewBool(false), config: config, } + ret.resetEpoch() + return ret } type agentOperationStatus int32 @@ -161,6 +167,10 @@ const ( type agentOperation struct { TableID model.TableID IsDelete bool + Epoch model.ProcessorEpoch + + // FromOwnerID is for debugging purposesFromOwnerID + FromOwnerID model.CaptureID status agentOperationStatus } @@ -185,6 +195,7 @@ func (a *BaseAgent) Tick(ctx context.Context) error { } if a.needSyncNow.Load() { + a.resetEpoch() done, err := a.sendSync(ctx) if err != nil { return errors.Trace(err) @@ -204,6 +215,12 @@ func (a *BaseAgent) Tick(ctx context.Context) error { opsToApply := a.popPendingOps() for _, op := range opsToApply { + if op.Epoch != a.getEpoch() { + a.logger.Info("dispatch request epoch does not match", + zap.String("epoch", op.Epoch), + zap.String("expectedEpoch", a.getEpoch())) + continue + } if _, ok := a.tableOperations[op.TableID]; ok { a.logger.DPanic("duplicate operation", zap.Any("op", op)) return cerrors.ErrProcessorDuplicateOperations.GenWithStackByArgs(op.TableID) @@ -259,7 +276,7 @@ func (a *BaseAgent) sendSync(ctx context.Context) (bool, error) { util.SortTableIDs(running) util.SortTableIDs(adding) util.SortTableIDs(removing) - done, err := a.communicator.SyncTaskStatuses(ctx, running, adding, removing) + done, err := a.communicator.SyncTaskStatuses(ctx, a.getEpoch(), adding, removing, running) if err != nil { return false, errors.Trace(err) } @@ -272,6 +289,7 @@ func (a *BaseAgent) processOperations(ctx context.Context) error { for tableID, op := range a.tableOperations { switch op.status { case operationReceived: + a.logger.Info("Agent start processing operation", zap.Any("op", op)) if !op.IsDelete { // add table done, err := a.executor.AddTable(ctx, op.TableID) @@ -306,7 +324,8 @@ func (a *BaseAgent) processOperations(ctx context.Context) error { op.status = operationFinished fallthrough case operationFinished: - done, err := a.communicator.FinishTableOperation(ctx, op.TableID) + a.logger.Info("Agent finish processing operation", zap.Any("op", op)) + done, err := a.communicator.FinishTableOperation(ctx, op.TableID, a.getEpoch()) if err != nil { return errors.Trace(err) } @@ -343,6 +362,7 @@ func (a *BaseAgent) OnOwnerDispatchedTask( ownerRev int64, tableID model.TableID, isDelete bool, + epoch model.ProcessorEpoch, ) { if !a.updateOwnerInfo(ownerCaptureID, ownerRev) { a.logger.Info("task from stale owner ignored", @@ -355,13 +375,15 @@ func (a *BaseAgent) OnOwnerDispatchedTask( defer a.pendingOpsMu.Unlock() op := &agentOperation{ - TableID: tableID, - IsDelete: isDelete, - status: operationReceived, + TableID: tableID, + IsDelete: isDelete, + Epoch: epoch, + FromOwnerID: ownerCaptureID, + status: operationReceived, } a.pendingOps.PushBack(op) - a.logger.Debug("OnOwnerDispatchedTask", + a.logger.Info("OnOwnerDispatchedTask", zap.String("ownerCaptureID", ownerCaptureID), zap.Int64("ownerRev", ownerRev), zap.Any("op", op)) @@ -456,3 +478,26 @@ func (a *BaseAgent) currentOwner() model.CaptureID { return a.ownerInfo.OwnerCaptureID } + +func (a *BaseAgent) resetEpoch() { + a.epochMu.Lock() + defer a.epochMu.Unlock() + + // We are using UUIDs because we only need uniqueness guarantee for the epoch, + // BUT NOT ordering guarantees. The reason is that the Sync messages are themselves + // barriers, so there is no need to accommodate messages from future epochs. + a.epoch = uuid.New().String() +} + +func (a *BaseAgent) getEpoch() model.ProcessorEpoch { + a.epochMu.RLock() + defer a.epochMu.RUnlock() + + return a.epoch +} + +// CurrentEpoch is a public function used in unit tests for +// checking epoch-related invariants. +func (a *BaseAgent) CurrentEpoch() model.ProcessorEpoch { + return a.getEpoch() +} diff --git a/cdc/scheduler/agent_mock.go b/cdc/scheduler/agent_mock.go index 113cd1f5141..a0d11476652 100644 --- a/cdc/scheduler/agent_mock.go +++ b/cdc/scheduler/agent_mock.go @@ -30,14 +30,14 @@ type MockProcessorMessenger struct { } // FinishTableOperation marks this function as being called. -func (m *MockProcessorMessenger) FinishTableOperation(ctx cdcContext.Context, tableID model.TableID) (bool, error) { - args := m.Called(ctx, tableID) +func (m *MockProcessorMessenger) FinishTableOperation(ctx cdcContext.Context, tableID model.TableID, epoch model.ProcessorEpoch) (bool, error) { + args := m.Called(ctx, tableID, epoch) return args.Bool(0), args.Error(1) } // SyncTaskStatuses marks this function as being called. -func (m *MockProcessorMessenger) SyncTaskStatuses(ctx cdcContext.Context, running, adding, removing []model.TableID) (bool, error) { - args := m.Called(ctx, running, adding, removing) +func (m *MockProcessorMessenger) SyncTaskStatuses(ctx cdcContext.Context, epoch model.ProcessorEpoch, adding, removing, running []model.TableID) (bool, error) { + args := m.Called(ctx, epoch, running, adding, removing) return args.Bool(0), args.Error(1) } diff --git a/cdc/scheduler/agent_test.go b/cdc/scheduler/agent_test.go index 92330ade8ae..fe3b0385688 100644 --- a/cdc/scheduler/agent_test.go +++ b/cdc/scheduler/agent_test.go @@ -31,15 +31,19 @@ func TestAgentAddTable(t *testing.T) { executor := NewMockTableExecutor(t) messenger := &MockProcessorMessenger{} agent := NewBaseAgent("test-cf", executor, messenger, agentConfigForTesting) - messenger.On("SyncTaskStatuses", mock.Anything, []model.TableID(nil), []model.TableID(nil), []model.TableID(nil)). - Return(true, nil) + var epoch model.ProcessorEpoch + messenger.On("SyncTaskStatuses", mock.Anything, mock.AnythingOfType("string"), []model.TableID(nil), []model.TableID(nil), []model.TableID(nil)). + Return(true, nil). + Run(func(args mock.Arguments) { + epoch = args.String(1) + }) err := agent.Tick(ctx) require.NoError(t, err) messenger.AssertExpectations(t) executor.ExpectedCalls = nil messenger.ExpectedCalls = nil - agent.OnOwnerDispatchedTask("capture-1", 1, model.TableID(1), false) + agent.OnOwnerDispatchedTask("capture-1", 1, model.TableID(1), false, epoch) executor.On("AddTable", mock.Anything, model.TableID(1)).Return(true, nil) messenger.On("OnOwnerChanged", mock.Anything, "capture-1") @@ -53,7 +57,7 @@ func TestAgentAddTable(t *testing.T) { executor.Running[model.TableID(1)] = struct{}{} executor.On("GetCheckpoint").Return(model.Ts(1002), model.Ts(1000)) messenger.On("SendCheckpoint", mock.Anything, model.Ts(1002), model.Ts(1000)).Return(true, nil) - messenger.On("FinishTableOperation", mock.Anything, model.TableID(1)).Return(true, nil) + messenger.On("FinishTableOperation", mock.Anything, model.TableID(1), epoch).Return(true, nil) err = agent.Tick(ctx) require.NoError(t, err) @@ -81,8 +85,13 @@ func TestAgentRemoveTable(t *testing.T) { messenger := &MockProcessorMessenger{} agent := NewBaseAgent("test-cf", executor, messenger, agentConfigForTesting) agent.OnOwnerAnnounce("capture-2", 1) - messenger.On("SyncTaskStatuses", mock.Anything, []model.TableID{1, 2}, []model.TableID(nil), []model.TableID(nil)). - Return(true, nil) + + var epoch model.ProcessorEpoch + messenger.On("SyncTaskStatuses", mock.Anything, mock.AnythingOfType("string"), []model.TableID{1, 2}, []model.TableID(nil), []model.TableID(nil)). + Return(true, nil). + Run(func(args mock.Arguments) { + epoch = args.String(1) + }) messenger.On("OnOwnerChanged", mock.Anything, "capture-2") executor.On("GetCheckpoint").Return(model.Ts(1000), model.Ts(1000)) messenger.On("SendCheckpoint", mock.Anything, model.Ts(1000), model.Ts(1000)).Return(true, nil) @@ -92,7 +101,7 @@ func TestAgentRemoveTable(t *testing.T) { executor.ExpectedCalls = nil messenger.ExpectedCalls = nil - agent.OnOwnerDispatchedTask("capture-2", 1, model.TableID(1), true) + agent.OnOwnerDispatchedTask("capture-2", 1, model.TableID(1), true, epoch) executor.On("GetCheckpoint").Return(model.Ts(1000), model.Ts(1000)) messenger.On("SendCheckpoint", mock.Anything, model.Ts(1000), model.Ts(1000)).Return(true, nil) executor.On("RemoveTable", mock.Anything, model.TableID(1)).Return(true, nil) @@ -105,8 +114,17 @@ func TestAgentRemoveTable(t *testing.T) { executor.ExpectedCalls = nil messenger.ExpectedCalls = nil executor.On("GetCheckpoint").Return(model.Ts(1000), model.Ts(1000)) - messenger.On("SyncTaskStatuses", mock.Anything, []model.TableID{2}, []model.TableID(nil), []model.TableID{1}). - Return(true, nil) + messenger.On("SyncTaskStatuses", + mock.Anything, + mock.AnythingOfType("string"), + []model.TableID{2}, + []model.TableID(nil), + []model.TableID{1}, + ). + Return(true, nil). + Run(func(args mock.Arguments) { + epoch = args.String(1) + }) messenger.On("OnOwnerChanged", mock.Anything, "capture-3") messenger.On("SendCheckpoint", mock.Anything, model.Ts(1000), model.Ts(1000)).Return(true, nil) messenger.On("Barrier", mock.Anything).Return(true) @@ -120,7 +138,7 @@ func TestAgentRemoveTable(t *testing.T) { delete(executor.Removing, model.TableID(1)) executor.On("GetCheckpoint").Return(model.Ts(1002), model.Ts(1000)) messenger.On("Barrier", mock.Anything).Return(true) - messenger.On("FinishTableOperation", mock.Anything, model.TableID(1)).Return(true, nil) + messenger.On("FinishTableOperation", mock.Anything, model.TableID(1), epoch).Return(true, nil) messenger.On("SendCheckpoint", mock.Anything, model.Ts(1002), model.Ts(1000)).Return(true, nil) err = agent.Tick(ctx) @@ -134,13 +152,22 @@ func TestAgentOwnerChangedWhileAddingTable(t *testing.T) { executor := NewMockTableExecutor(t) messenger := &MockProcessorMessenger{} agent := NewBaseAgent("test-cf", executor, messenger, agentConfigForTesting) - messenger.On("SyncTaskStatuses", mock.Anything, []model.TableID(nil), []model.TableID(nil), []model.TableID(nil)). - Return(true, nil) + + var epoch model.ProcessorEpoch + messenger.On("SyncTaskStatuses", + mock.Anything, + mock.AnythingOfType("string"), + []model.TableID(nil), []model.TableID(nil), []model.TableID(nil), + ). + Return(true, nil). + Run(func(args mock.Arguments) { + epoch = args.String(1) + }) err := agent.Tick(ctx) require.NoError(t, err) messenger.AssertExpectations(t) - agent.OnOwnerDispatchedTask("capture-1", 1, model.TableID(1), false) + agent.OnOwnerDispatchedTask("capture-1", 1, model.TableID(1), false, epoch) executor.On("AddTable", mock.Anything, model.TableID(1)).Return(true, nil) messenger.On("OnOwnerChanged", mock.Anything, "capture-1") @@ -161,8 +188,16 @@ func TestAgentOwnerChangedWhileAddingTable(t *testing.T) { messenger.ExpectedCalls = nil agent.OnOwnerAnnounce("capture-2", 2) messenger.On("OnOwnerChanged", mock.Anything, "capture-2") - messenger.On("SyncTaskStatuses", mock.Anything, []model.TableID(nil), []model.TableID{1}, []model.TableID(nil)). - Return(true, nil) + messenger.On( + "SyncTaskStatuses", + mock.Anything, + mock.AnythingOfType("string"), + []model.TableID(nil), []model.TableID{1}, []model.TableID(nil), + ). + Return(true, nil). + Run(func(args mock.Arguments) { + epoch = args.String(1) + }) messenger.On("Barrier", mock.Anything).Return(true) executor.On("GetCheckpoint").Return(model.Ts(1002), model.Ts(1000)) messenger.On("SendCheckpoint", mock.Anything, model.Ts(1002), model.Ts(1000)).Return(true, nil) @@ -179,13 +214,18 @@ func TestAgentReceiveFromStaleOwner(t *testing.T) { messenger := &MockProcessorMessenger{} agent := NewBaseAgent("test-cf", executor, messenger, agentConfigForTesting) agent.checkpointSender = &mockCheckpointSender{} - messenger.On("SyncTaskStatuses", mock.Anything, []model.TableID(nil), []model.TableID(nil), []model.TableID(nil)). - Return(true, nil) + + var epoch model.ProcessorEpoch + messenger.On("SyncTaskStatuses", mock.Anything, mock.AnythingOfType("string"), + []model.TableID(nil), []model.TableID(nil), []model.TableID(nil)). + Return(true, nil).Run(func(args mock.Arguments) { + epoch = args.String(1) + }) err := agent.Tick(ctx) require.NoError(t, err) messenger.AssertExpectations(t) - agent.OnOwnerDispatchedTask("capture-1", 1, model.TableID(1), false) + agent.OnOwnerDispatchedTask("capture-1", 1, model.TableID(1), false, epoch) executor.On("AddTable", mock.Anything, model.TableID(1)).Return(true, nil) messenger.On("OnOwnerChanged", mock.Anything, "capture-1") @@ -197,7 +237,7 @@ func TestAgentReceiveFromStaleOwner(t *testing.T) { messenger.ExpectedCalls = nil executor.On("GetCheckpoint").Return(model.Ts(1002), model.Ts(1000)) // Stale owner - agent.OnOwnerDispatchedTask("capture-2", 0, model.TableID(2), false) + agent.OnOwnerDispatchedTask("capture-2", 0, model.TableID(2), false, defaultEpoch) err = agent.Tick(ctx) require.NoError(t, err) @@ -220,7 +260,8 @@ func TestOwnerMismatchShouldPanic(t *testing.T) { messenger := &MockProcessorMessenger{} agent := NewBaseAgent("test-cf", executor, messenger, agentConfigForTesting) agent.checkpointSender = &mockCheckpointSender{} - messenger.On("SyncTaskStatuses", mock.Anything, []model.TableID(nil), []model.TableID(nil), []model.TableID(nil)). + messenger.On("SyncTaskStatuses", mock.Anything, mock.AnythingOfType("string"), + []model.TableID(nil), []model.TableID(nil), []model.TableID(nil)). Return(true, nil) err := agent.Tick(ctx) require.NoError(t, err) @@ -239,3 +280,51 @@ func TestOwnerMismatchShouldPanic(t *testing.T) { agent.OnOwnerAnnounce("capture-2", 1) }, "should have panicked") } + +func TestIgnoreStaleEpoch(t *testing.T) { + ctx := cdcContext.NewBackendContext4Test(false) + + executor := NewMockTableExecutor(t) + messenger := &MockProcessorMessenger{} + agent := NewBaseAgent("test-cf", executor, messenger, agentConfigForTesting) + agent.checkpointSender = &mockCheckpointSender{} + + var epoch, newEpoch model.ProcessorEpoch + messenger.On("SyncTaskStatuses", mock.Anything, mock.AnythingOfType("string"), + []model.TableID(nil), []model.TableID(nil), []model.TableID(nil)). + Return(true, nil).Run(func(args mock.Arguments) { + epoch = args.String(1) + }) + + err := agent.Tick(ctx) + require.NoError(t, err) + messenger.AssertExpectations(t) + + agent.OnOwnerAnnounce("capture-1", 1) + messenger.On("OnOwnerChanged", mock.Anything, "capture-1") + + err = agent.Tick(ctx) + require.NoError(t, err) + messenger.AssertExpectations(t) + + messenger.ExpectedCalls = nil + messenger.On("OnOwnerChanged", mock.Anything, "capture-1") + messenger.On("SyncTaskStatuses", mock.Anything, mock.AnythingOfType("string"), + []model.TableID(nil), []model.TableID(nil), []model.TableID(nil)). + Return(true, nil).Run(func(args mock.Arguments) { + newEpoch = args.String(1) + }) + agent.OnOwnerAnnounce("capture-1", 1) + + err = agent.Tick(ctx) + require.NoError(t, err) + messenger.AssertExpectations(t) + + require.NotEqual(t, epoch, newEpoch) + agent.OnOwnerDispatchedTask("capture-1", 1, model.TableID(2), false, epoch) + + err = agent.Tick(ctx) + require.NoError(t, err) + messenger.AssertExpectations(t) + executor.AssertNotCalled(t, "AddTable", mock.Anything, model.TableID(1)) +} diff --git a/cdc/scheduler/schedule_dispatcher.go b/cdc/scheduler/schedule_dispatcher.go index 63cde335b05..ac2de59c94c 100644 --- a/cdc/scheduler/schedule_dispatcher.go +++ b/cdc/scheduler/schedule_dispatcher.go @@ -61,7 +61,8 @@ type ScheduleDispatcherCommunicator interface { changeFeedID model.ChangeFeedID, tableID model.TableID, captureID model.CaptureID, - isDelete bool, // True when we want to remove a table from the capture. + isDelete bool, + epoch model.ProcessorEpoch, ) (done bool, err error) // Announce announces to the specified capture that the current node has become the Owner. @@ -125,6 +126,10 @@ type captureStatus struct { // dispatch a table. SyncStatus captureSyncStatus + // Epoch is reset when the processor's internal states + // have been reset. + Epoch model.ProcessorEpoch + // Watermark fields CheckpointTs model.Ts ResolvedTs model.Ts @@ -396,7 +401,9 @@ func (s *BaseScheduleDispatcher) addTable( } } - ok, err = s.communicator.DispatchTable(ctx, s.changeFeedID, tableID, target, false) + epoch := s.captureStatus[target].Epoch + ok, err = s.communicator.DispatchTable( + ctx, s.changeFeedID, tableID, target, false, epoch) if err != nil { return false, errors.Trace(err) } @@ -428,7 +435,8 @@ func (s *BaseScheduleDispatcher) removeTable( } // need to delete table captureID := record.CaptureID - ok, err = s.communicator.DispatchTable(ctx, s.changeFeedID, tableID, captureID, true) + epoch := s.captureStatus[captureID].Epoch + ok, err = s.communicator.DispatchTable(ctx, s.changeFeedID, tableID, captureID, true, epoch) if err != nil { return false, errors.Trace(err) } @@ -496,8 +504,10 @@ func (s *BaseScheduleDispatcher) rebalance(ctx context.Context) (done bool, err zap.Any("tableRecord", record)) } + epoch := s.captureStatus[record.CaptureID].Epoch // Removes the table from the current capture - ok, err := s.communicator.DispatchTable(ctx, s.changeFeedID, record.TableID, record.CaptureID, true) + ok, err := s.communicator.DispatchTable( + ctx, s.changeFeedID, record.TableID, record.CaptureID, true, epoch) if err != nil { return false, errors.Trace(err) } @@ -513,13 +523,18 @@ func (s *BaseScheduleDispatcher) rebalance(ctx context.Context) (done bool, err // OnAgentFinishedTableOperation is called when a table operation has been finished by // the processor. -func (s *BaseScheduleDispatcher) OnAgentFinishedTableOperation(captureID model.CaptureID, tableID model.TableID) { +func (s *BaseScheduleDispatcher) OnAgentFinishedTableOperation( + captureID model.CaptureID, + tableID model.TableID, + epoch model.ProcessorEpoch, +) { s.mu.Lock() defer s.mu.Unlock() logger := s.logger.With( zap.String("captureID", captureID), zap.Int64("tableID", tableID), + zap.String("epoch", epoch), ) if _, ok := s.captures[captureID]; !ok { @@ -527,6 +542,18 @@ func (s *BaseScheduleDispatcher) OnAgentFinishedTableOperation(captureID model.C return } + captureSt, ok := s.captureStatus[captureID] + if !ok { + logger.Warn("Message from an unknown processor, ignore") + return + } + + if captureSt.Epoch != epoch { + logger.Warn("Processor epoch does not match", + zap.String("expected", captureSt.Epoch)) + return + } + record, ok := s.tables.GetTableRecord(tableID) if !ok { logger.Warn("response about a stale table, ignore") @@ -553,12 +580,18 @@ func (s *BaseScheduleDispatcher) OnAgentFinishedTableOperation(captureID model.C } // OnAgentSyncTaskStatuses is called when the processor sends its complete current state. -func (s *BaseScheduleDispatcher) OnAgentSyncTaskStatuses(captureID model.CaptureID, running, adding, removing []model.TableID) { +func (s *BaseScheduleDispatcher) OnAgentSyncTaskStatuses( + captureID model.CaptureID, + epoch model.ProcessorEpoch, + running, adding, removing []model.TableID, +) { s.mu.Lock() defer s.mu.Unlock() logger := s.logger.With(zap.String("captureID", captureID)) - logger.Info("scheduler received sync", zap.String("captureID", captureID)) + logger.Info("scheduler received sync", + zap.String("captureID", captureID), + zap.String("epoch", epoch)) if ce := logger.Check(zap.DebugLevel, "OnAgentSyncTaskStatuses"); ce != nil { // Print this information only in debug mode. @@ -604,7 +637,9 @@ func (s *BaseScheduleDispatcher) OnAgentSyncTaskStatuses(captureID model.Capture s.tables.AddTableRecord(&util.TableRecord{TableID: tableID, CaptureID: captureID, Status: util.RemovingTable}) } - s.captureStatus[captureID].SyncStatus = captureSyncFinished + status := s.captureStatus[captureID] + status.SyncStatus = captureSyncFinished + status.Epoch = epoch } // OnAgentCheckpoint is called when the processor sends a checkpoint. diff --git a/cdc/scheduler/schedule_dispatcher_test.go b/cdc/scheduler/schedule_dispatcher_test.go index e9443cc78a7..3e0db461213 100644 --- a/cdc/scheduler/schedule_dispatcher_test.go +++ b/cdc/scheduler/schedule_dispatcher_test.go @@ -28,6 +28,11 @@ import ( var _ ScheduleDispatcherCommunicator = (*mockScheduleDispatcherCommunicator)(nil) +const ( + defaultEpoch = "default-epoch" + nextEpoch = "next-epoch" +) + type mockScheduleDispatcherCommunicator struct { mock.Mock addTableRecords map[model.CaptureID][]model.TableID @@ -56,20 +61,22 @@ func (m *mockScheduleDispatcherCommunicator) DispatchTable( tableID model.TableID, captureID model.CaptureID, isDelete bool, + epoch model.ProcessorEpoch, ) (done bool, err error) { if !m.isBenchmark { log.Info("dispatch table called", zap.String("changefeed", changeFeedID), zap.Int64("tableID", tableID), zap.String("captureID", captureID), - zap.Bool("isDelete", isDelete)) + zap.Bool("isDelete", isDelete), + zap.String("epoch", epoch)) if !isDelete { m.addTableRecords[captureID] = append(m.addTableRecords[captureID], tableID) } else { m.removeTableRecords[captureID] = append(m.removeTableRecords[captureID], tableID) } } - args := m.Called(ctx, changeFeedID, tableID, captureID, isDelete) + args := m.Called(ctx, changeFeedID, tableID, captureID, isDelete, epoch) return args.Bool(0), args.Error(1) } @@ -109,12 +116,12 @@ func TestDispatchTable(t *testing.T) { require.Equal(t, CheckpointCannotProceed, resolvedTs) communicator.AssertExpectations(t) - dispatcher.OnAgentSyncTaskStatuses("capture-1", []model.TableID{}, []model.TableID{}, []model.TableID{}) - dispatcher.OnAgentSyncTaskStatuses("capture-2", []model.TableID{}, []model.TableID{}, []model.TableID{}) + dispatcher.OnAgentSyncTaskStatuses("capture-1", defaultEpoch, []model.TableID{}, []model.TableID{}, []model.TableID{}) + dispatcher.OnAgentSyncTaskStatuses("capture-2", defaultEpoch, []model.TableID{}, []model.TableID{}, []model.TableID{}) communicator.Reset() // Injects a dispatch table failure - communicator.On("DispatchTable", mock.Anything, "cf-1", mock.Anything, mock.Anything, false). + communicator.On("DispatchTable", mock.Anything, "cf-1", mock.Anything, mock.Anything, false, defaultEpoch). Return(false, nil) checkpointTs, resolvedTs, err = dispatcher.Tick(ctx, 1000, []model.TableID{1, 2, 3}, defaultMockCaptureInfos) require.NoError(t, err) @@ -123,11 +130,11 @@ func TestDispatchTable(t *testing.T) { communicator.AssertExpectations(t) communicator.Reset() - communicator.On("DispatchTable", mock.Anything, "cf-1", model.TableID(1), mock.Anything, false). + communicator.On("DispatchTable", mock.Anything, "cf-1", model.TableID(1), mock.Anything, false, defaultEpoch). Return(true, nil) - communicator.On("DispatchTable", mock.Anything, "cf-1", model.TableID(2), mock.Anything, false). + communicator.On("DispatchTable", mock.Anything, "cf-1", model.TableID(2), mock.Anything, false, defaultEpoch). Return(true, nil) - communicator.On("DispatchTable", mock.Anything, "cf-1", model.TableID(3), mock.Anything, false). + communicator.On("DispatchTable", mock.Anything, "cf-1", model.TableID(3), mock.Anything, false, defaultEpoch). Return(true, nil) checkpointTs, resolvedTs, err = dispatcher.Tick(ctx, 1000, []model.TableID{1, 2, 3}, defaultMockCaptureInfos) require.NoError(t, err) @@ -152,7 +159,7 @@ func TestDispatchTable(t *testing.T) { for captureID, tables := range communicator.addTableRecords { for _, tableID := range tables { - dispatcher.OnAgentFinishedTableOperation(captureID, tableID) + dispatcher.OnAgentFinishedTableOperation(captureID, tableID, defaultEpoch) } } @@ -194,24 +201,24 @@ func TestSyncCaptures(t *testing.T) { require.Equal(t, CheckpointCannotProceed, checkpointTs) require.Equal(t, CheckpointCannotProceed, resolvedTs) - dispatcher.OnAgentSyncTaskStatuses("capture-1", []model.TableID{1, 2, 3}, []model.TableID{4, 5}, []model.TableID{6, 7}) + dispatcher.OnAgentSyncTaskStatuses("capture-1", defaultEpoch, []model.TableID{1, 2, 3}, []model.TableID{4, 5}, []model.TableID{6, 7}) checkpointTs, resolvedTs, err = dispatcher.Tick(ctx, 1500, []model.TableID{1, 2, 3, 4, 5}, defaultMockCaptureInfos) require.NoError(t, err) require.Equal(t, CheckpointCannotProceed, checkpointTs) require.Equal(t, CheckpointCannotProceed, resolvedTs) communicator.Reset() - dispatcher.OnAgentFinishedTableOperation("capture-1", 4) - dispatcher.OnAgentFinishedTableOperation("capture-1", 5) - dispatcher.OnAgentSyncTaskStatuses("capture-2", []model.TableID(nil), []model.TableID(nil), []model.TableID(nil)) + dispatcher.OnAgentFinishedTableOperation("capture-1", 4, defaultEpoch) + dispatcher.OnAgentFinishedTableOperation("capture-1", 5, defaultEpoch) + dispatcher.OnAgentSyncTaskStatuses("capture-2", defaultEpoch, []model.TableID(nil), []model.TableID(nil), []model.TableID(nil)) checkpointTs, resolvedTs, err = dispatcher.Tick(ctx, 1500, []model.TableID{1, 2, 3, 4, 5}, defaultMockCaptureInfos) require.NoError(t, err) require.Equal(t, CheckpointCannotProceed, checkpointTs) require.Equal(t, CheckpointCannotProceed, resolvedTs) communicator.Reset() - dispatcher.OnAgentFinishedTableOperation("capture-1", 6) - dispatcher.OnAgentFinishedTableOperation("capture-1", 7) + dispatcher.OnAgentFinishedTableOperation("capture-1", 6, defaultEpoch) + dispatcher.OnAgentFinishedTableOperation("capture-1", 7, defaultEpoch) checkpointTs, resolvedTs, err = dispatcher.Tick(ctx, 1500, []model.TableID{1, 2, 3, 4, 5}, defaultMockCaptureInfos) require.NoError(t, err) require.Equal(t, model.Ts(1500), checkpointTs) @@ -229,7 +236,7 @@ func TestSyncUnknownCapture(t *testing.T) { dispatcher.captureStatus = map[model.CaptureID]*captureStatus{} // empty capture status // Sends a sync from an unknown capture - dispatcher.OnAgentSyncTaskStatuses("capture-1", []model.TableID{1, 2, 3}, []model.TableID{4, 5}, []model.TableID{6, 7}) + dispatcher.OnAgentSyncTaskStatuses("capture-1", defaultEpoch, []model.TableID{1, 2, 3}, []model.TableID{4, 5}, []model.TableID{6, 7}) // We expect the `Sync` to be ignored. checkpointTs, resolvedTs, err := dispatcher.Tick(ctx, 1500, []model.TableID{1, 2, 3, 4, 5}, mockCaptureInfos) @@ -249,11 +256,13 @@ func TestRemoveTable(t *testing.T) { SyncStatus: captureSyncFinished, CheckpointTs: 1500, ResolvedTs: 1500, + Epoch: defaultEpoch, }, "capture-2": { SyncStatus: captureSyncFinished, CheckpointTs: 1500, ResolvedTs: 1500, + Epoch: defaultEpoch, }, } dispatcher.tables.AddTableRecord(&util.TableRecord{ @@ -278,7 +287,7 @@ func TestRemoveTable(t *testing.T) { require.Equal(t, model.Ts(1500), resolvedTs) // Inject a dispatch table failure - communicator.On("DispatchTable", mock.Anything, "cf-1", model.TableID(3), "capture-1", true). + communicator.On("DispatchTable", mock.Anything, "cf-1", model.TableID(3), "capture-1", true, defaultEpoch). Return(false, nil) checkpointTs, resolvedTs, err = dispatcher.Tick(ctx, 1500, []model.TableID{1, 2}, defaultMockCaptureInfos) require.NoError(t, err) @@ -287,7 +296,7 @@ func TestRemoveTable(t *testing.T) { communicator.AssertExpectations(t) communicator.Reset() - communicator.On("DispatchTable", mock.Anything, "cf-1", model.TableID(3), "capture-1", true). + communicator.On("DispatchTable", mock.Anything, "cf-1", model.TableID(3), "capture-1", true, defaultEpoch). Return(true, nil) checkpointTs, resolvedTs, err = dispatcher.Tick(ctx, 1500, []model.TableID{1, 2}, defaultMockCaptureInfos) require.NoError(t, err) @@ -295,7 +304,7 @@ func TestRemoveTable(t *testing.T) { require.Equal(t, CheckpointCannotProceed, resolvedTs) communicator.AssertExpectations(t) - dispatcher.OnAgentFinishedTableOperation("capture-1", 3) + dispatcher.OnAgentFinishedTableOperation("capture-1", 3, defaultEpoch) communicator.Reset() checkpointTs, resolvedTs, err = dispatcher.Tick(ctx, 1500, []model.TableID{1, 2}, defaultMockCaptureInfos) require.NoError(t, err) @@ -322,11 +331,13 @@ func TestCaptureGone(t *testing.T) { SyncStatus: captureSyncFinished, CheckpointTs: 1500, ResolvedTs: 1500, + Epoch: defaultEpoch, }, "capture-2": { SyncStatus: captureSyncFinished, CheckpointTs: 1500, ResolvedTs: 1500, + Epoch: defaultEpoch, }, } dispatcher.tables.AddTableRecord(&util.TableRecord{ @@ -345,7 +356,7 @@ func TestCaptureGone(t *testing.T) { Status: util.RunningTable, }) - communicator.On("DispatchTable", mock.Anything, "cf-1", model.TableID(2), "capture-1", false). + communicator.On("DispatchTable", mock.Anything, "cf-1", model.TableID(2), "capture-1", false, defaultEpoch). Return(true, nil) checkpointTs, resolvedTs, err := dispatcher.Tick(ctx, 1500, []model.TableID{1, 2, 3}, mockCaptureInfos) require.NoError(t, err) @@ -365,11 +376,13 @@ func TestCaptureRestarts(t *testing.T) { SyncStatus: captureSyncFinished, CheckpointTs: 1500, ResolvedTs: 1500, + Epoch: defaultEpoch, }, "capture-2": { SyncStatus: captureSyncFinished, CheckpointTs: 1500, ResolvedTs: 1500, + Epoch: defaultEpoch, }, } dispatcher.tables.AddTableRecord(&util.TableRecord{ @@ -388,8 +401,8 @@ func TestCaptureRestarts(t *testing.T) { Status: util.RunningTable, }) - dispatcher.OnAgentSyncTaskStatuses("capture-2", []model.TableID{}, []model.TableID{}, []model.TableID{}) - communicator.On("DispatchTable", mock.Anything, "cf-1", model.TableID(2), "capture-2", false). + dispatcher.OnAgentSyncTaskStatuses("capture-2", nextEpoch, []model.TableID{}, []model.TableID{}, []model.TableID{}) + communicator.On("DispatchTable", mock.Anything, "cf-1", model.TableID(2), "capture-2", false, nextEpoch). Return(true, nil) checkpointTs, resolvedTs, err := dispatcher.Tick(ctx, 1500, []model.TableID{1, 2, 3}, defaultMockCaptureInfos) require.NoError(t, err) @@ -420,11 +433,13 @@ func TestCaptureGoneWhileMovingTable(t *testing.T) { SyncStatus: captureSyncFinished, CheckpointTs: 1300, ResolvedTs: 1600, + Epoch: defaultEpoch, }, "capture-2": { SyncStatus: captureSyncFinished, CheckpointTs: 1500, ResolvedTs: 1550, + Epoch: defaultEpoch, }, } dispatcher.tables.AddTableRecord(&util.TableRecord{ @@ -444,7 +459,7 @@ func TestCaptureGoneWhileMovingTable(t *testing.T) { }) dispatcher.MoveTable(1, "capture-2") - communicator.On("DispatchTable", mock.Anything, "cf-1", model.TableID(1), "capture-1", true). + communicator.On("DispatchTable", mock.Anything, "cf-1", model.TableID(1), "capture-1", true, defaultEpoch). Return(true, nil) checkpointTs, resolvedTs, err := dispatcher.Tick(ctx, 1300, []model.TableID{1, 2, 3}, mockCaptureInfos) require.NoError(t, err) @@ -453,11 +468,11 @@ func TestCaptureGoneWhileMovingTable(t *testing.T) { communicator.AssertExpectations(t) delete(mockCaptureInfos, "capture-2") - dispatcher.OnAgentFinishedTableOperation("capture-1", 1) + dispatcher.OnAgentFinishedTableOperation("capture-1", 1, defaultEpoch) communicator.Reset() - communicator.On("DispatchTable", mock.Anything, "cf-1", model.TableID(1), mock.Anything, false). + communicator.On("DispatchTable", mock.Anything, "cf-1", model.TableID(1), mock.Anything, false, defaultEpoch). Return(true, nil) - communicator.On("DispatchTable", mock.Anything, "cf-1", model.TableID(2), mock.Anything, false). + communicator.On("DispatchTable", mock.Anything, "cf-1", model.TableID(2), mock.Anything, false, defaultEpoch). Return(true, nil) checkpointTs, resolvedTs, err = dispatcher.Tick(ctx, 1300, []model.TableID{1, 2, 3}, mockCaptureInfos) require.NoError(t, err) @@ -492,16 +507,19 @@ func TestRebalance(t *testing.T) { SyncStatus: captureSyncFinished, CheckpointTs: 1300, ResolvedTs: 1600, + Epoch: defaultEpoch, }, "capture-2": { SyncStatus: captureSyncFinished, CheckpointTs: 1500, ResolvedTs: 1550, + Epoch: defaultEpoch, }, "capture-3": { SyncStatus: captureSyncFinished, CheckpointTs: 1400, ResolvedTs: 1650, + Epoch: defaultEpoch, }, } for i := 1; i <= 6; i++ { @@ -513,7 +531,7 @@ func TestRebalance(t *testing.T) { } dispatcher.Rebalance() - communicator.On("DispatchTable", mock.Anything, "cf-1", mock.Anything, mock.Anything, true). + communicator.On("DispatchTable", mock.Anything, "cf-1", mock.Anything, mock.Anything, true, defaultEpoch). Return(false, nil) checkpointTs, resolvedTs, err := dispatcher.Tick(ctx, 1300, []model.TableID{1, 2, 3, 4, 5, 6}, mockCaptureInfos) require.NoError(t, err) @@ -523,7 +541,7 @@ func TestRebalance(t *testing.T) { communicator.AssertNumberOfCalls(t, "DispatchTable", 1) communicator.Reset() - communicator.On("DispatchTable", mock.Anything, "cf-1", mock.Anything, mock.Anything, true). + communicator.On("DispatchTable", mock.Anything, "cf-1", mock.Anything, mock.Anything, true, defaultEpoch). Return(true, nil) checkpointTs, resolvedTs, err = dispatcher.Tick(ctx, 1300, []model.TableID{1, 2, 3, 4, 5, 6}, mockCaptureInfos) require.NoError(t, err) @@ -633,11 +651,13 @@ func TestIgnoreUnsyncedCaptures(t *testing.T) { SyncStatus: captureSyncFinished, CheckpointTs: 1300, ResolvedTs: 1600, + Epoch: defaultEpoch, }, "capture-2": { SyncStatus: captureSyncSent, // not synced CheckpointTs: 1400, ResolvedTs: 1500, + Epoch: "garbage", }, } @@ -656,7 +676,7 @@ func TestIgnoreUnsyncedCaptures(t *testing.T) { require.Equal(t, CheckpointCannotProceed, resolvedTs) communicator.Reset() - dispatcher.OnAgentSyncTaskStatuses("capture-2", []model.TableID{2, 4, 6}, []model.TableID{}, []model.TableID{}) + dispatcher.OnAgentSyncTaskStatuses("capture-2", defaultEpoch, []model.TableID{2, 4, 6}, []model.TableID{}, []model.TableID{}) checkpointTs, resolvedTs, err = dispatcher.Tick(ctx, 1300, []model.TableID{1, 2, 3, 4, 5, 6}, defaultMockCaptureInfos) require.NoError(t, err) require.Equal(t, model.Ts(1300), checkpointTs) @@ -675,11 +695,13 @@ func TestRebalanceWhileAddingTable(t *testing.T) { SyncStatus: captureSyncFinished, CheckpointTs: 1300, ResolvedTs: 1600, + Epoch: defaultEpoch, }, "capture-2": { SyncStatus: captureSyncFinished, CheckpointTs: 1500, ResolvedTs: 1550, + Epoch: defaultEpoch, }, } for i := 1; i <= 6; i++ { @@ -690,7 +712,7 @@ func TestRebalanceWhileAddingTable(t *testing.T) { }) } - communicator.On("DispatchTable", mock.Anything, "cf-1", model.TableID(7), "capture-2", false). + communicator.On("DispatchTable", mock.Anything, "cf-1", model.TableID(7), "capture-2", false, defaultEpoch). Return(true, nil) checkpointTs, resolvedTs, err := dispatcher.Tick(ctx, 1300, []model.TableID{1, 2, 3, 4, 5, 6, 7}, defaultMockCaptureInfos) require.NoError(t, err) @@ -706,9 +728,9 @@ func TestRebalanceWhileAddingTable(t *testing.T) { require.Equal(t, CheckpointCannotProceed, resolvedTs) communicator.AssertExpectations(t) - dispatcher.OnAgentFinishedTableOperation("capture-2", model.TableID(7)) + dispatcher.OnAgentFinishedTableOperation("capture-2", model.TableID(7), defaultEpoch) communicator.Reset() - communicator.On("DispatchTable", mock.Anything, "cf-1", mock.Anything, mock.Anything, true). + communicator.On("DispatchTable", mock.Anything, "cf-1", mock.Anything, mock.Anything, true, defaultEpoch). Return(true, nil) checkpointTs, resolvedTs, err = dispatcher.Tick(ctx, 1300, []model.TableID{1, 2, 3, 4, 5, 6, 7}, defaultMockCaptureInfos) require.NoError(t, err) @@ -729,11 +751,13 @@ func TestManualMoveTableWhileAddingTable(t *testing.T) { SyncStatus: captureSyncFinished, CheckpointTs: 1300, ResolvedTs: 1600, + Epoch: defaultEpoch, }, "capture-2": { SyncStatus: captureSyncFinished, CheckpointTs: 1500, ResolvedTs: 1550, + Epoch: defaultEpoch, }, } dispatcher.tables.AddTableRecord(&util.TableRecord{ @@ -747,7 +771,7 @@ func TestManualMoveTableWhileAddingTable(t *testing.T) { Status: util.RunningTable, }) - communicator.On("DispatchTable", mock.Anything, "cf-1", model.TableID(1), "capture-2", false). + communicator.On("DispatchTable", mock.Anything, "cf-1", model.TableID(1), "capture-2", false, defaultEpoch). Return(true, nil) checkpointTs, resolvedTs, err := dispatcher.Tick(ctx, 1300, []model.TableID{1, 2, 3}, defaultMockCaptureInfos) require.NoError(t, err) @@ -761,9 +785,9 @@ func TestManualMoveTableWhileAddingTable(t *testing.T) { require.Equal(t, CheckpointCannotProceed, resolvedTs) communicator.AssertExpectations(t) - dispatcher.OnAgentFinishedTableOperation("capture-2", 1) + dispatcher.OnAgentFinishedTableOperation("capture-2", 1, defaultEpoch) communicator.Reset() - communicator.On("DispatchTable", mock.Anything, "cf-1", model.TableID(1), "capture-2", true). + communicator.On("DispatchTable", mock.Anything, "cf-1", model.TableID(1), "capture-2", true, defaultEpoch). Return(true, nil) checkpointTs, resolvedTs, err = dispatcher.Tick(ctx, 1300, []model.TableID{1, 2, 3}, defaultMockCaptureInfos) require.NoError(t, err) @@ -771,9 +795,9 @@ func TestManualMoveTableWhileAddingTable(t *testing.T) { require.Equal(t, CheckpointCannotProceed, resolvedTs) communicator.AssertExpectations(t) - dispatcher.OnAgentFinishedTableOperation("capture-2", 1) + dispatcher.OnAgentFinishedTableOperation("capture-2", 1, defaultEpoch) communicator.Reset() - communicator.On("DispatchTable", mock.Anything, "cf-1", model.TableID(1), "capture-1", false). + communicator.On("DispatchTable", mock.Anything, "cf-1", model.TableID(1), "capture-1", false, defaultEpoch). Return(true, nil) checkpointTs, resolvedTs, err = dispatcher.Tick(ctx, 1300, []model.TableID{1, 2, 3}, defaultMockCaptureInfos) require.NoError(t, err) @@ -817,15 +841,15 @@ func TestAutoRebalanceOnCaptureOnline(t *testing.T) { require.Equal(t, CheckpointCannotProceed, resolvedTs) communicator.AssertExpectations(t) - dispatcher.OnAgentSyncTaskStatuses("capture-1", []model.TableID{}, []model.TableID{}, []model.TableID{}) - dispatcher.OnAgentSyncTaskStatuses("capture-2", []model.TableID{}, []model.TableID{}, []model.TableID{}) + dispatcher.OnAgentSyncTaskStatuses("capture-1", defaultEpoch, []model.TableID{}, []model.TableID{}, []model.TableID{}) + dispatcher.OnAgentSyncTaskStatuses("capture-2", defaultEpoch, []model.TableID{}, []model.TableID{}, []model.TableID{}) communicator.Reset() - communicator.On("DispatchTable", mock.Anything, "cf-1", model.TableID(1), mock.Anything, false). + communicator.On("DispatchTable", mock.Anything, "cf-1", model.TableID(1), mock.Anything, false, defaultEpoch). Return(true, nil) - communicator.On("DispatchTable", mock.Anything, "cf-1", model.TableID(2), mock.Anything, false). + communicator.On("DispatchTable", mock.Anything, "cf-1", model.TableID(2), mock.Anything, false, defaultEpoch). Return(true, nil) - communicator.On("DispatchTable", mock.Anything, "cf-1", model.TableID(3), mock.Anything, false). + communicator.On("DispatchTable", mock.Anything, "cf-1", model.TableID(3), mock.Anything, false, defaultEpoch). Return(true, nil) checkpointTs, resolvedTs, err = dispatcher.Tick(ctx, 1000, []model.TableID{1, 2, 3}, captureList) require.NoError(t, err) @@ -861,7 +885,7 @@ func TestAutoRebalanceOnCaptureOnline(t *testing.T) { communicator.AssertExpectations(t) communicator.ExpectedCalls = nil - dispatcher.OnAgentSyncTaskStatuses("capture-3", []model.TableID{}, []model.TableID{}, []model.TableID{}) + dispatcher.OnAgentSyncTaskStatuses("capture-3", defaultEpoch, []model.TableID{}, []model.TableID{}, []model.TableID{}) checkpointTs, resolvedTs, err = dispatcher.Tick(ctx, 1000, []model.TableID{1, 2, 3}, captureList) require.NoError(t, err) require.Equal(t, CheckpointCannotProceed, checkpointTs) @@ -870,13 +894,13 @@ func TestAutoRebalanceOnCaptureOnline(t *testing.T) { for captureID, tables := range communicator.addTableRecords { for _, tableID := range tables { - dispatcher.OnAgentFinishedTableOperation(captureID, tableID) + dispatcher.OnAgentFinishedTableOperation(captureID, tableID, defaultEpoch) } } communicator.Reset() var removeTableFromCapture model.CaptureID - communicator.On("DispatchTable", mock.Anything, "cf-1", mock.Anything, mock.Anything, true). + communicator.On("DispatchTable", mock.Anything, "cf-1", mock.Anything, mock.Anything, true, defaultEpoch). Return(true, nil).Run(func(args mock.Arguments) { removeTableFromCapture = args.Get(3).(model.CaptureID) }) @@ -888,11 +912,11 @@ func TestAutoRebalanceOnCaptureOnline(t *testing.T) { removedTableID := communicator.removeTableRecords[removeTableFromCapture][0] - dispatcher.OnAgentFinishedTableOperation(removeTableFromCapture, removedTableID) + dispatcher.OnAgentFinishedTableOperation(removeTableFromCapture, removedTableID, defaultEpoch) dispatcher.OnAgentCheckpoint("capture-1", 1100, 1400) dispatcher.OnAgentCheckpoint("capture-2", 1200, 1300) communicator.ExpectedCalls = nil - communicator.On("DispatchTable", mock.Anything, "cf-1", removedTableID, "capture-3", false). + communicator.On("DispatchTable", mock.Anything, "cf-1", removedTableID, "capture-3", false, defaultEpoch). Return(true, nil) checkpointTs, resolvedTs, err = dispatcher.Tick(ctx, 1000, []model.TableID{1, 2, 3}, captureList) require.NoError(t, err) @@ -901,6 +925,80 @@ func TestAutoRebalanceOnCaptureOnline(t *testing.T) { communicator.AssertExpectations(t) } +func TestInvalidFinishedTableOperation(t *testing.T) { + t.Parallel() + + ctx := cdcContext.NewBackendContext4Test(false) + communicator := NewMockScheduleDispatcherCommunicator() + dispatcher := NewBaseScheduleDispatcher("cf-1", communicator, 1000) + dispatcher.captureStatus = map[model.CaptureID]*captureStatus{ + "capture-1": { + SyncStatus: captureSyncFinished, + CheckpointTs: 1300, + ResolvedTs: 1600, + Epoch: defaultEpoch, + }, + "capture-2": { + SyncStatus: captureSyncFinished, + CheckpointTs: 1500, + ResolvedTs: 1550, + Epoch: defaultEpoch, + }, + } + dispatcher.tables.AddTableRecord(&util.TableRecord{ + TableID: 2, + CaptureID: "capture-1", + Status: util.RunningTable, + }) + dispatcher.tables.AddTableRecord(&util.TableRecord{ + TableID: 3, + CaptureID: "capture-1", + Status: util.RunningTable, + }) + + communicator.On("DispatchTable", mock.Anything, "cf-1", model.TableID(1), "capture-2", false, defaultEpoch). + Return(true, nil) + checkpointTs, resolvedTs, err := dispatcher.Tick(ctx, 1300, []model.TableID{1, 2, 3}, defaultMockCaptureInfos) + require.NoError(t, err) + require.Equal(t, CheckpointCannotProceed, checkpointTs) + require.Equal(t, CheckpointCannotProceed, resolvedTs) + + // Invalid epoch + dispatcher.OnAgentFinishedTableOperation("capture-2", model.TableID(1), "invalid-epoch") + checkpointTs, resolvedTs, err = dispatcher.Tick(ctx, 1300, []model.TableID{1, 2, 3}, defaultMockCaptureInfos) + require.NoError(t, err) + require.Equal(t, CheckpointCannotProceed, checkpointTs) + require.Equal(t, CheckpointCannotProceed, resolvedTs) + record, ok := dispatcher.tables.GetTableRecord(model.TableID(1)) + require.True(t, ok) + require.Equal(t, record.Status, util.AddingTable) + + // Invalid capture + dispatcher.OnAgentFinishedTableOperation("capture-invalid", model.TableID(1), defaultEpoch) + checkpointTs, resolvedTs, err = dispatcher.Tick(ctx, 1300, []model.TableID{1, 2, 3}, defaultMockCaptureInfos) + require.NoError(t, err) + require.Equal(t, CheckpointCannotProceed, checkpointTs) + require.Equal(t, CheckpointCannotProceed, resolvedTs) + record, ok = dispatcher.tables.GetTableRecord(model.TableID(1)) + require.True(t, ok) + require.Equal(t, record.Status, util.AddingTable) + + // Invalid table + dispatcher.OnAgentFinishedTableOperation("capture-1", model.TableID(999), defaultEpoch) + checkpointTs, resolvedTs, err = dispatcher.Tick(ctx, 1300, []model.TableID{1, 2, 3}, defaultMockCaptureInfos) + require.NoError(t, err) + require.Equal(t, CheckpointCannotProceed, checkpointTs) + require.Equal(t, CheckpointCannotProceed, resolvedTs) + record, ok = dispatcher.tables.GetTableRecord(model.TableID(1)) + require.True(t, ok) + require.Equal(t, record.Status, util.AddingTable) + + // Capture not matching + require.Panics(t, func() { + dispatcher.OnAgentFinishedTableOperation("capture-1", model.TableID(1), defaultEpoch) + }) +} + func BenchmarkAddTable(b *testing.B) { ctx := cdcContext.NewBackendContext4Test(false) diff --git a/errors.toml b/errors.toml index 53da43512cf..9d7ba10d022 100755 --- a/errors.toml +++ b/errors.toml @@ -663,7 +663,7 @@ peer-to-peer message client has failed permanently, no need to reconnect: %s ["CDC:ErrPeerMessageDataLost"] error = ''' -peer-to-peer message data lost, topic: %s, seq: %s +peer-to-peer message data lost, topic: %s, seq: %d ''' ["CDC:ErrPeerMessageDecodeError"] diff --git a/pkg/errors/errors.go b/pkg/errors/errors.go index 4a37306b2c3..41595b7e3fd 100644 --- a/pkg/errors/errors.go +++ b/pkg/errors/errors.go @@ -281,7 +281,7 @@ var ( ErrPeerMessageStaleConnection = errors.Normalize("peer-to-peer message stale connection: old-epoch %d, new-epoch %d", errors.RFCCodeText("CDC:ErrPeerMessageStaleConnection")) ErrPeerMessageDuplicateConnection = errors.Normalize("peer-to-peer message duplicate connection: epoch %d", errors.RFCCodeText("CDC:ErrPeerMessageDuplicateConnection")) ErrPeerMessageServerClosed = errors.Normalize("peer-to-peer message server has closed connection: %s.", errors.RFCCodeText("CDC:ErrPeerMessageServerClosed")) - ErrPeerMessageDataLost = errors.Normalize("peer-to-peer message data lost, topic: %s, seq: %s", errors.RFCCodeText("CDC:ErrPeerMessageDataLost")) + ErrPeerMessageDataLost = errors.Normalize("peer-to-peer message data lost, topic: %s, seq: %d", errors.RFCCodeText("CDC:ErrPeerMessageDataLost")) ErrPeerMessageToManyPeers = errors.Normalize("peer-to-peer message server got too many peers: %d peers", errors.RFCCodeText("CDC:ErrPeerMessageToManyPeers")) ErrPeerMessageDecodeError = errors.Normalize("failed to decode peer-to-peer message", errors.RFCCodeText("CDC:ErrPeerMessageDecodeError")) ErrPeerMessageTaskQueueCongested = errors.Normalize("peer-to-peer message server has too many pending tasks", errors.RFCCodeText("CDC:ErrPeerMessageTaskQueueCongested")) From b3014061250cb3598c491624135217d4cd76d386 Mon Sep 17 00:00:00 2001 From: GMHDBJD <35025882+GMHDBJD@users.noreply.github.com> Date: Mon, 7 Mar 2022 15:17:49 +0800 Subject: [PATCH 3/7] Optimistic: dm-master support conflict DDL in optimistic mode. (#4297) ref pingcap/tiflow#3787 --- dm/_utils/terror_gen/errors_release.txt | 1 + dm/dm/master/shardddl/optimist.go | 20 +- dm/errors.toml | 6 + dm/pkg/shardddl/optimism/column.go | 20 +- dm/pkg/shardddl/optimism/column_test.go | 28 +- dm/pkg/shardddl/optimism/lock.go | 846 ++++++++++++++++-------- dm/pkg/shardddl/optimism/lock_test.go | 752 +++++++++++++++++++-- dm/pkg/shardddl/optimism/operation.go | 7 +- dm/pkg/terror/error_list.go | 6 + dm/syncer/optimist.go | 6 + dm/tests/shardddl4_1/run.sh | 5 +- 11 files changed, 1355 insertions(+), 342 deletions(-) diff --git a/dm/_utils/terror_gen/errors_release.txt b/dm/_utils/terror_gen/errors_release.txt index 9a844927578..8f17fbf6015 100644 --- a/dm/_utils/terror_gen/errors_release.txt +++ b/dm/_utils/terror_gen/errors_release.txt @@ -130,6 +130,7 @@ ErrMetadataNoBinlogLoc,[code=11123:class=functional:scope=upstream:level=low], " ErrPreviousGTIDNotExist,[code=11124:class=functional:scope=internal:level=high], "Message: no previous gtid event from binlog %s" ErrNoMasterStatus,[code=11125:class=functional:scope=upstream:level=medium], "Message: upstream returns an empty result for SHOW MASTER STATUS, Workaround: Please check the upstream settings like privileges, RDS settings to read data from SHOW MASTER STATUS." ErrBinlogNotLogColumn,[code=11126:class=binlog-op:scope=upstream:level=high], "Message: upstream didn't log enough columns in binlog, Workaround: Please check if session `binlog_row_image` variable is not FULL, restart task to the location from where FULL binlog_row_image is used." +ErrShardDDLOptimismNeedSkipAndRedirect,[code=11127:class=functional:scope=internal:level=low], "Message: receive conflict DDL for the optimistic shard ddl lock %s: %s" ErrConfigCheckItemNotSupport,[code=20001:class=config:scope=internal:level=medium], "Message: checking item %s is not supported\n%s, Workaround: Please check `ignore-checking-items` config in task configuration file, which can be set including `all`/`dump_privilege`/`replication_privilege`/`version`/`binlog_enable`/`binlog_format`/`binlog_row_image`/`table_schema`/`schema_of_shard_tables`/`auto_increment_ID`." ErrConfigTomlTransform,[code=20002:class=config:scope=internal:level=medium], "Message: %s, Workaround: Please check the configuration file has correct TOML format." ErrConfigYamlTransform,[code=20003:class=config:scope=internal:level=medium], "Message: %s, Workaround: Please check the configuration file has correct YAML format." diff --git a/dm/dm/master/shardddl/optimist.go b/dm/dm/master/shardddl/optimist.go index 58a899b4c05..dab6f8fa3e8 100644 --- a/dm/dm/master/shardddl/optimist.go +++ b/dm/dm/master/shardddl/optimist.go @@ -586,18 +586,26 @@ func (o *Optimist) handleOperation(op optimism.Operation) { // handleLock handles a single shard DDL lock. func (o *Optimist) handleLock(info optimism.Info, tts []optimism.TargetTable, skipDone bool) error { - cfStage := optimism.ConflictNone - cfMsg := "" + var ( + cfStage = optimism.ConflictNone + cfMsg = "" + ) + lockID, newDDLs, cols, err := o.lk.TrySync(o.cli, info, tts) switch { case info.IgnoreConflict: o.logger.Warn("error occur when trying to sync for shard DDL info, this often means shard DDL conflict detected", zap.String("lock", lockID), zap.String("info", info.ShortString()), zap.Bool("is deleted", info.IsDeleted), log.ShortError(err)) case err != nil: - cfStage = optimism.ConflictDetected // we treat any errors returned from `TrySync` as conflict detected now. - cfMsg = err.Error() - o.logger.Warn("error occur when trying to sync for shard DDL info, this often means shard DDL conflict detected", - zap.String("lock", lockID), zap.String("info", info.ShortString()), zap.Bool("is deleted", info.IsDeleted), log.ShortError(err)) + if terror.ErrShardDDLOptimismNeedSkipAndRedirect.Equal(err) { + cfStage = optimism.ConflictSkipWaitRedirect + o.logger.Warn("Please make sure all sharding tables execute this DDL in order", log.ShortError(err)) + } else { + cfStage = optimism.ConflictDetected // we treat any errors returned from `TrySync` as conflict detected now. + cfMsg = err.Error() + o.logger.Warn("error occur when trying to sync for shard DDL info, this often means shard DDL conflict detected", + zap.String("lock", lockID), zap.String("info", info.ShortString()), zap.Bool("is deleted", info.IsDeleted), log.ShortError(err)) + } default: o.logger.Info("the shard DDL lock returned some DDLs", zap.String("lock", lockID), zap.Strings("ddls", newDDLs), zap.Strings("cols", cols), zap.String("info", info.ShortString()), zap.Bool("is deleted", info.IsDeleted)) diff --git a/dm/errors.toml b/dm/errors.toml index 1df34489a2e..704cae40ea4 100644 --- a/dm/errors.toml +++ b/dm/errors.toml @@ -790,6 +790,12 @@ description = "" workaround = "Please check if session `binlog_row_image` variable is not FULL, restart task to the location from where FULL binlog_row_image is used." tags = ["upstream", "high"] +[error.DM-functional-11127] +message = "receive conflict DDL for the optimistic shard ddl lock %s: %s" +description = "" +workaround = "" +tags = ["internal", "low"] + [error.DM-config-20001] message = "checking item %s is not supported\n%s" description = "" diff --git a/dm/pkg/shardddl/optimism/column.go b/dm/pkg/shardddl/optimism/column.go index bddf728b841..396d4c05130 100644 --- a/dm/pkg/shardddl/optimism/column.go +++ b/dm/pkg/shardddl/optimism/column.go @@ -67,17 +67,21 @@ func GetAllDroppedColumns(cli *clientv3.Client) (map[string]map[string]map[strin return colm, rev, nil } -// PutDroppedColumn puts the partially dropped column name into ectd. +// PutDroppedColumn puts the partially dropped column names into ectd. // When we drop a column, we save this column's name in etcd. -func PutDroppedColumn(cli *clientv3.Client, lockID, column, source, upSchema, upTable string, done DropColumnStage) (rev int64, putted bool, err error) { - key := common.ShardDDLOptimismDroppedColumnsKeyAdapter.Encode(lockID, column, source, upSchema, upTable) - val, err := json.Marshal(done) - if err != nil { - return 0, false, err +func PutDroppedColumns(cli *clientv3.Client, lockID, source, upSchema, upTable string, cols []string, done DropColumnStage) (int64, bool, error) { + ops := make([]clientv3.Op, 0, len(cols)) + for _, column := range cols { + key := common.ShardDDLOptimismDroppedColumnsKeyAdapter.Encode(lockID, column, source, upSchema, upTable) + val, err := json.Marshal(done) + if err != nil { + return 0, false, err + } + op := clientv3.OpPut(key, string(val)) + ops = append(ops, op) } - op := clientv3.OpPut(key, string(val)) - resp, rev, err := etcdutil.DoOpsInOneTxnWithRetry(cli, op) + resp, rev, err := etcdutil.DoOpsInOneTxnWithRetry(cli, ops...) if err != nil { return 0, false, err } diff --git a/dm/pkg/shardddl/optimism/column_test.go b/dm/pkg/shardddl/optimism/column_test.go index 6b2aafe842d..82d46e8425f 100644 --- a/dm/pkg/shardddl/optimism/column_test.go +++ b/dm/pkg/shardddl/optimism/column_test.go @@ -37,21 +37,25 @@ func (t *testColumn) TestColumnETCD(c *C) { info1 = NewInfo(task, source1, upSchema1, upTable1, downSchema, downTable, nil, nil, nil) lockID = genDDLLockID(info1) ) - rev1, putted, err := PutDroppedColumn(etcdTestCli, lockID, "a", source1, upSchema1, upTable1, DropNotDone) + rev1, putted, err := PutDroppedColumns(etcdTestCli, lockID, source1, upSchema1, upTable1, []string{"a"}, DropNotDone) c.Assert(err, IsNil) c.Assert(putted, IsTrue) - rev2, putted, err := PutDroppedColumn(etcdTestCli, lockID, "b", source1, upSchema1, upTable1, DropNotDone) + rev2, putted, err := PutDroppedColumns(etcdTestCli, lockID, source1, upSchema1, upTable1, []string{"b"}, DropNotDone) c.Assert(err, IsNil) c.Assert(putted, IsTrue) c.Assert(rev2, Greater, rev1) - rev3, putted, err := PutDroppedColumn(etcdTestCli, lockID, "b", source1, upSchema2, upTable2, DropNotDone) + rev3, putted, err := PutDroppedColumns(etcdTestCli, lockID, source1, upSchema2, upTable2, []string{"b"}, DropNotDone) c.Assert(err, IsNil) c.Assert(putted, IsTrue) c.Assert(rev3, Greater, rev2) - rev4, putted, err := PutDroppedColumn(etcdTestCli, lockID, "b", source2, upSchema1, upTable1, DropNotDone) + rev4, putted, err := PutDroppedColumns(etcdTestCli, lockID, source2, upSchema1, upTable1, []string{"b"}, DropNotDone) c.Assert(err, IsNil) c.Assert(putted, IsTrue) c.Assert(rev4, Greater, rev3) + rev5, putted, err := PutDroppedColumns(etcdTestCli, lockID, source2, upSchema1, upTable1, []string{"b", "c"}, DropNotDone) + c.Assert(err, IsNil) + c.Assert(putted, IsTrue) + c.Assert(rev5, Greater, rev4) expectedColm := map[string]map[string]map[string]map[string]map[string]DropColumnStage{ lockID: { @@ -63,21 +67,25 @@ func (t *testColumn) TestColumnETCD(c *C) { }, source2: {upSchema1: {upTable1: DropNotDone}}, }, + "c": { + source2: {upSchema1: {upTable1: DropNotDone}}, + }, }, } - colm, rev5, err := GetAllDroppedColumns(etcdTestCli) + colm, rev6, err := GetAllDroppedColumns(etcdTestCli) c.Assert(err, IsNil) c.Assert(colm, DeepEquals, expectedColm) - c.Assert(rev5, Equals, rev4) + c.Assert(rev6, Equals, rev5) - rev6, deleted, err := DeleteDroppedColumns(etcdTestCli, lockID, "b") + rev7, deleted, err := DeleteDroppedColumns(etcdTestCli, lockID, "b", "c") c.Assert(err, IsNil) c.Assert(deleted, IsTrue) - c.Assert(rev6, Greater, rev5) + c.Assert(rev7, Greater, rev6) delete(expectedColm[lockID], "b") - colm, rev7, err := GetAllDroppedColumns(etcdTestCli) + delete(expectedColm[lockID], "c") + colm, rev8, err := GetAllDroppedColumns(etcdTestCli) c.Assert(err, IsNil) c.Assert(colm, DeepEquals, expectedColm) - c.Assert(rev7, Equals, rev6) + c.Assert(rev8, Equals, rev7) } diff --git a/dm/pkg/shardddl/optimism/lock.go b/dm/pkg/shardddl/optimism/lock.go index d1b207a9f48..c08760cda87 100644 --- a/dm/pkg/shardddl/optimism/lock.go +++ b/dm/pkg/shardddl/optimism/lock.go @@ -19,6 +19,7 @@ import ( "fmt" "sync" + "github.com/pingcap/errors" "github.com/pingcap/tidb-tools/pkg/dbutil" "github.com/pingcap/tidb-tools/pkg/schemacmp" "github.com/pingcap/tidb/parser" @@ -47,6 +48,18 @@ const ( DropDone ) +type tableType int + +const ( + // normalTables represents upstream table info record in checkpoint. + normalTables tableType = iota + // conflictTables represents upstream table info after executing conflict DDL. + conflictTables + // finalTables combines normalTables and conflcitTables, + // which represents all upstream table infos after executing all conflict DDLs. + finalTables +) + // Lock represents the shard DDL lock in memory. // This information does not need to be persistent, and can be re-constructed from the shard DDL info. type Lock struct { @@ -60,12 +73,20 @@ type Lock struct { DownSchema string // downstream schema name DownTable string // downstream table name - // current joined info. - joined schemacmp.Table + // first prevTable when a lock created + // only use when fetchTableInfo return an error. + initTable schemacmp.Table // per-table's table info, // upstream source ID -> upstream schema name -> upstream table name -> table info. // if all of them are the same, then we call the lock `synced`. tables map[string]map[string]map[string]schemacmp.Table + // conflictTables is used for conflict DDL coordination + // upstream source ID -> upstream schema name -> upstream table name -> table info. + conflictTables map[string]map[string]map[string]schemacmp.Table + // finalTables combine tables and conflcitTables + // it represents final state of all tables + // upstream source ID -> upstream schema name -> upstream table name -> table info. + finalTables map[string]map[string]map[string]schemacmp.Table synced bool @@ -87,15 +108,17 @@ type Lock struct { } // NewLock creates a new Lock instance. -func NewLock(cli *clientv3.Client, id, task, downSchema, downTable string, joined schemacmp.Table, tts []TargetTable, downstreamMeta *DownstreamMeta) *Lock { +func NewLock(cli *clientv3.Client, id, task, downSchema, downTable string, initTable schemacmp.Table, tts []TargetTable, downstreamMeta *DownstreamMeta) *Lock { l := &Lock{ cli: cli, ID: id, Task: task, DownSchema: downSchema, DownTable: downTable, - joined: joined, + initTable: initTable, tables: make(map[string]map[string]map[string]schemacmp.Table), + conflictTables: make(map[string]map[string]map[string]schemacmp.Table), + finalTables: make(map[string]map[string]map[string]schemacmp.Table), done: make(map[string]map[string]map[string]bool), synced: true, versions: make(map[string]map[string]map[string]int64), @@ -104,8 +127,6 @@ func NewLock(cli *clientv3.Client, id, task, downSchema, downTable string, joine } l.addTables(tts) metrics.ReportDDLPending(task, metrics.DDLPendingNone, metrics.DDLPendingSynced) - // pre join because tables may have different schema at the beginning - l.joinTable() return l } @@ -144,32 +165,6 @@ func (l *Lock) FetchTableInfos(task, source, schema, table string) (*model.Table return ti, nil } -// joinTable join tables for a lock and update l.joined. -func (l *Lock) joinTable() { - var ( - joined = l.joined - firstTable = true - ) - for _, schemaTables := range l.tables { - for _, tables := range schemaTables { - for _, ti := range tables { - if firstTable { - joined = ti - firstTable = false - } else { - newJoined, err := joined.Join(ti) - if err != nil { - log.L().Error(fmt.Sprintf("fail to join table info %s with %s", joined, ti), zap.String("lockID", l.ID), log.ShortError(err)) - return - } - joined = newJoined - } - } - } - } - l.joined = joined -} - // TrySync tries to sync the lock, re-entrant. // new upstream sources may join when the DDL lock is in syncing, // so we need to merge these new sources. @@ -189,23 +184,14 @@ func (l *Lock) TrySync(info Info, tts []TargetTable) (newDDLs []string, cols []s callerSchema = info.UpSchema callerTable = info.UpTable ddls = info.DDLs + emptyDDLs = []string{} emptyCols = []string{} newTIs = info.TableInfosAfter infoVersion = info.Version ignoreConflict = info.IgnoreConflict oldSynced = l.synced - emptyDDLs = []string{} ) l.mu.Lock() - defer func() { - if len(newDDLs) > 0 { - // revert the `done` status if need to wait for the new operation to be done. - // Now, we wait for the new operation to be done if any DDLs returned. - l.tryRevertDone(callerSource, callerSchema, callerTable) - } - l.mu.Unlock() - }() - defer func() { _, remain := l.syncStatus() l.synced = remain == 0 @@ -216,37 +202,45 @@ func (l *Lock) TrySync(info Info, tts []TargetTable) (newDDLs []string, cols []s metrics.ReportDDLPending(l.Task, metrics.DDLPendingUnSynced, metrics.DDLPendingSynced) } } - }() - - // joinTable join other tables - joinTable := func(joined schemacmp.Table) (schemacmp.Table, error) { - for source, schemaTables := range l.tables { - for schema, tables := range schemaTables { - for table, ti := range tables { - if source != callerSource || schema != callerSchema || table != callerTable { - newJoined, err2 := joined.Join(ti) - if err2 != nil { - // NOTE: conflict detected. - return newJoined, terror.ErrShardDDLOptimismTrySyncFail.Delegate( - err2, l.ID, fmt.Sprintf("fail to join table info %s with %s", joined, ti)) - } - joined = newJoined - } - } - } + if len(newDDLs) > 0 || (err != nil && terror.ErrShardDDLOptimismNeedSkipAndRedirect.Equal(err)) { + // revert the `done` status if need to wait for the new operation to be done. + // Now, we wait for the new operation to be done if any DDLs returned. + l.tryRevertDone(callerSource, callerSchema, callerTable) } - return joined, nil - } + l.mu.Unlock() + }() // should not happen if len(ddls) != len(newTIs) || len(newTIs) == 0 { - return ddls, emptyCols, terror.ErrMasterInconsistentOptimisticDDLsAndInfo.Generate(len(ddls), len(newTIs)) + return emptyDDLs, emptyCols, terror.ErrMasterInconsistentOptimisticDDLsAndInfo.Generate(len(ddls), len(newTIs)) } - // should not happen if info.TableInfoBefore == nil { - return ddls, emptyCols, terror.ErrMasterOptimisticTableInfoBeforeNotExist.Generate(ddls) + return emptyDDLs, emptyCols, terror.ErrMasterOptimisticTableInfoBeforeNotExist.Generate(ddls) } + + defer func() { + if err == nil && len(cols) > 0 { + err = l.AddDroppedColumns(callerSource, callerSchema, callerTable, cols) + } + // only update table info if no error or ignore conflict or conflict DDL + if err != nil { + var revertInfo schemacmp.Table + switch { + case ignoreConflict: + // forcely set schema for --ignore-conflict + revertInfo = schemacmp.Encode(newTIs[len(newTIs)-1]) + case terror.ErrShardDDLOptimismNeedSkipAndRedirect.Equal(err): + return + default: + revertInfo = schemacmp.Encode(info.TableInfoBefore) + } + l.tables[callerSource][callerSchema][callerTable] = revertInfo + l.finalTables[callerSource][callerSchema][callerTable] = revertInfo + l.removeConflictTable(callerSource, callerSchema, callerTable) + } + }() + // handle the case where // is not in old source tables and current new source tables. // duplicate append is not a problem. @@ -258,181 +252,32 @@ func (l *Lock) TrySync(info Info, tts []TargetTable) (newDDLs []string, cols []s l.versions[callerSource][callerSchema][callerTable] = infoVersion } - lastTableInfo := schemacmp.Encode(newTIs[len(newTIs)-1]) - defer func() { - // only update table info if no error or ignore conflict - if ignoreConflict || err == nil { - log.L().Info("update table info", zap.String("lock", l.ID), zap.String("source", callerSource), zap.String("schema", callerSchema), zap.String("table", callerTable), - zap.Stringer("from", l.tables[callerSource][callerSchema][callerTable]), zap.Stringer("to", lastTableInfo), zap.Strings("ddls", ddls)) - l.tables[callerSource][callerSchema][callerTable] = lastTableInfo - } - }() - - prevTable := schemacmp.Encode(info.TableInfoBefore) - // if preTable not equal table in master, we always use preTable - // this often happens when an info TrySync twice, e.g. worker restart/resume task - if cmp, err2 := prevTable.Compare(l.tables[callerSource][callerSchema][callerTable]); err2 != nil || cmp != 0 { - log.L().Warn("table-info-before not equal table saved in master", zap.Stringer("master-table", l.tables[callerSource][callerSchema][callerTable]), zap.Stringer("table-info-before", prevTable)) - l.tables[callerSource][callerSchema][callerTable] = prevTable - prevJoined, err2 := joinTable(prevTable) - if err2 != nil { - return emptyDDLs, emptyCols, err2 - } - l.joined = prevJoined - } - oldJoined := l.joined - - lastJoined, err := joinTable(lastTableInfo) - if err != nil { - return emptyDDLs, emptyCols, err - } - - defer func() { - if err == nil { - // update the current joined table info, it should be logged in `if cmp != 0` block below. - l.joined = lastJoined - } - }() - newDDLs = []string{} cols = []string{} - nextTable := prevTable - newJoined := oldJoined - + prevTable := schemacmp.Encode(info.TableInfoBefore) // join and compare every new table info - for idx, newTI := range newTIs { - prevTable = nextTable - oldJoined = newJoined - nextTable = schemacmp.Encode(newTI) - // special case: check whether DDLs making the schema become part of larger and another part of smaller. - if _, err = prevTable.Compare(nextTable); err != nil { - return emptyDDLs, emptyCols, terror.ErrShardDDLOptimismTrySyncFail.Delegate( - err, l.ID, fmt.Sprintf("there will be conflicts if DDLs %s are applied to the downstream. old table info: %s, new table info: %s", ddls, prevTable, nextTable)) - } - - // special case: if the DDL does not affect the schema at all, assume it is - // idempotent and just execute the DDL directly. - // if any real conflicts after joined exist, they will be detected by the following steps. - // this often happens when executing `CREATE TABLE` statement - var cmp int - if cmp, err = nextTable.Compare(oldJoined); err == nil && cmp == 0 { - if col, err2 := GetColumnName(l.ID, ddls[idx], ast.AlterTableAddColumns); err2 != nil { - return newDDLs, cols, err2 - } else if len(col) > 0 && l.IsDroppedColumn(info.Source, info.UpSchema, info.UpTable, col) { - return newDDLs, cols, terror.ErrShardDDLOptimismTrySyncFail.Generate( - l.ID, fmt.Sprintf("add column %s that wasn't fully dropped in downstream. ddl: %s", col, ddls[idx])) - } - newDDLs = append(newDDLs, ddls[idx]) - continue - } - - // try to join tables. - newJoined, err = joinTable(nextTable) - if err != nil { - return emptyDDLs, emptyCols, err - } - - cmp, err = oldJoined.Compare(newJoined) - // FIXME: Compute DDLs through schema diff instead of propagating DDLs directly. - // and now we MUST ensure different sources execute same DDLs to the downstream multiple times is safe. - if err != nil { - // resolving conflict in non-intrusive mode. - log.L().Warn("resolving conflict", zap.String("lock", l.ID), zap.String("source", callerSource), zap.String("schema", callerSchema), zap.String("table", callerTable), - zap.Stringer("joined-from", oldJoined), zap.Stringer("joined-to", newJoined), zap.Strings("ddls", ddls)) - return ddls, cols, nil - } - if cmp != 0 { - // < 0: the joined schema become larger after applied these DDLs. - // this often happens when executing `ADD COLUMN` for the FIRST table. - // > 0: the joined schema become smaller after applied these DDLs. - // this often happens when executing `DROP COLUMN` for the LAST table. - // for these two cases, we should execute the DDLs to the downstream to update the schema. - log.L().Info("joined table info changed", zap.String("lock", l.ID), zap.Int("cmp", cmp), zap.Stringer("from", oldJoined), zap.Stringer("to", newJoined), - zap.String("source", callerSource), zap.String("schema", callerSchema), zap.String("table", callerTable), zap.Strings("ddls", ddls)) - if cmp < 0 { - // check for add column with a larger field len - if col, err2 := AddDifferentFieldLenColumns(l.ID, ddls[idx], oldJoined, newJoined); err2 != nil { - return ddls, cols, err2 - } else if len(col) > 0 && l.IsDroppedColumn(info.Source, info.UpSchema, info.UpTable, col) { - return ddls, cols, terror.ErrShardDDLOptimismTrySyncFail.Generate( - l.ID, fmt.Sprintf("add column %s that wasn't fully dropped in downstream. ddl: %s", col, ddls[idx])) - } - } else { - if col, err2 := GetColumnName(l.ID, ddls[idx], ast.AlterTableDropColumn); err2 != nil { - return ddls, cols, err2 - } else if len(col) > 0 { - err = l.AddDroppedColumn(info, col) - if err != nil { - log.L().Error("fail to add dropped column info in etcd", zap.Error(err)) - return ddls, cols, terror.ErrShardDDLOptimismTrySyncFail.Generate(l.ID, "fail to add dropped column info in etcd") - } - cols = append(cols, col) - } - } - newDDLs = append(newDDLs, ddls[idx]) - continue - } - - // NOTE: now, different DM-workers do not wait for each other when executing DDL/DML, - // when coordinating the shard DDL between multiple DM-worker instances, - // a possible sequences: - // 1. DM-worker-A do this `trySync` and DM-master let it to `ADD COLUMN`. - // 2. DM-worker-B do this `trySync` again. - // 3. DM-worker-B replicate DML to the downstream. - // 4. DM-worker-A replicate `ADD COLUMN` to the downstream. - // in order to support DML from DM-worker-B matches the downstream schema, - // two strategies exist: - // A. DM-worker-B waits for DM-worker-A to finish the replication of the DDL before replicating DML. - // B. DM-worker-B also replicates the DDL before replicating DML, - // but this MUST ensure we can tolerate replicating the DDL multiple times. - // for `DROP COLUMN` or other DDL which makes the schema become smaller, - // this is not a problem because all DML with larger schema should already replicated to the downstream, - // and any DML with smaller schema can fit both the larger or smaller schema. - // To make it easy to implement, we will temporarily choose strategy-B. - - cmp, _ = prevTable.Compare(nextTable) // we have checked `err` returned above. - if cmp < 0 { - // check for add column with a smaller field len - if col, err2 := AddDifferentFieldLenColumns(l.ID, ddls[idx], nextTable, newJoined); err2 != nil { - return ddls, cols, err2 - } else if len(col) > 0 && l.IsDroppedColumn(info.Source, info.UpSchema, info.UpTable, col) { - return ddls, cols, terror.ErrShardDDLOptimismTrySyncFail.Generate( - l.ID, fmt.Sprintf("add column %s that wasn't fully dropped in downstream. ddl: %s", col, ddls[idx])) - } - // let every table to replicate the DDL. - newDDLs = append(newDDLs, ddls[idx]) - continue - } else if cmp > 0 { - if col, err2 := GetColumnName(l.ID, ddls[idx], ast.AlterTableDropColumn); err2 != nil { - return ddls, cols, err2 - } else if len(col) > 0 { - err = l.AddDroppedColumn(info, col) - if err != nil { - log.L().Error("fail to add dropped column info in etcd", zap.Error(err)) - return ddls, cols, terror.ErrShardDDLOptimismTrySyncFail.Generate(l.ID, "fail to add dropped column info in etcd") - } + for idx, ti := range newTIs { + postTable := schemacmp.Encode(ti) + schemaChanged, conflictStage := l.trySyncForOneDDL(callerSource, callerSchema, callerTable, prevTable, postTable) + + switch conflictStage { + case ConflictDetected: + return emptyDDLs, emptyCols, terror.ErrShardDDLOptimismTrySyncFail.Generate(l.ID, fmt.Sprintf("there will be conflicts if DDLs %s are applied to the downstream. old table info: %s, new table info: %s", ddls[idx], prevTable, postTable)) + case ConflictNone: + if col, err := l.checkAddDropColumn(callerSource, callerSchema, callerTable, ddls[idx], prevTable, postTable, cols); err != nil { + return emptyDDLs, emptyCols, err + } else if len(col) != 0 { cols = append(cols, col) } - // last shard table won't go here - continue + case ConflictSkipWaitRedirect: + return newDDLs, cols, terror.ErrShardDDLOptimismNeedSkipAndRedirect.Generate(l.ID, ddls[idx]) } - // compare the current table's info with joined info. - cmp, err = nextTable.Compare(newJoined) - if err != nil { - return emptyDDLs, emptyCols, terror.ErrShardDDLOptimismTrySyncFail.Delegate( - err, l.ID, "can't compare table info (new table info) %s with (new joined table info) %s", nextTable, newJoined) // NOTE: this should not happen. - } - if cmp < 0 { - // no need to replicate DDLs, because has a larger joined schema (in the downstream). - // FIXME: if the previous tables reached the joined schema has not replicated to the downstream, - // now, they should re-try until replicated successfully, try to implement better strategy later. - continue + if schemaChanged { + newDDLs = append(newDDLs, ddls[idx]) } - log.L().Warn("new table info >= new joined table info", zap.Stringer("table info", nextTable), zap.Stringer("joined table info", newJoined)) - return ddls, cols, nil // NOTE: this should not happen. + prevTable = postTable } - return newDDLs, cols, nil } @@ -459,6 +304,8 @@ func (l *Lock) TryRemoveTable(source, schema, table string) bool { } delete(l.tables[source][schema], table) + delete(l.finalTables[source][schema], table) + l.removeConflictTable(source, schema, table) _, remain := l.syncStatus() l.synced = remain == 0 delete(l.done[source][schema], table) @@ -492,6 +339,8 @@ func (l *Lock) TryRemoveTableBySources(sources []string) []string { } delete(l.tables, source) + delete(l.finalTables, source) + delete(l.conflictTables, source) _, remain := l.syncStatus() l.synced = remain == 0 delete(l.done, source) @@ -543,10 +392,10 @@ func (l *Lock) Ready() map[string]map[string]map[string]bool { } // Joined returns the joined table info. -func (l *Lock) Joined() schemacmp.Table { +func (l *Lock) Joined() (schemacmp.Table, error) { l.mu.RLock() defer l.mu.RUnlock() - return l.joined + return l.joinNormalTables() } // TryMarkDone tries to mark the operation of the source table as done. @@ -618,7 +467,8 @@ func (l *Lock) IsResolved() bool { func (l *Lock) syncStatus() (map[string]map[string]map[string]bool, int) { ready := make(map[string]map[string]map[string]bool) remain := 0 - for source, schemaTables := range l.tables { + joined, joinedErr := l.joinFinalTables() + for source, schemaTables := range l.finalTables { if _, ok := ready[source]; !ok { ready[source] = make(map[string]map[string]bool) } @@ -627,12 +477,14 @@ func (l *Lock) syncStatus() (map[string]map[string]map[string]bool, int) { ready[source][schema] = make(map[string]bool) } for table, ti := range tables { - if cmp, err := l.joined.Compare(ti); err == nil && cmp == 0 { - ready[source][schema][table] = true - } else { - ready[source][schema][table] = false - remain++ + if joinedErr == nil { + if cmp, err := joined.Compare(ti); err == nil && cmp == 0 { + ready[source][schema][table] = true + continue + } } + ready[source][schema][table] = false + remain++ } } } @@ -659,12 +511,14 @@ func (l *Lock) addTables(tts []TargetTable) { for _, tt := range tts { if _, ok := l.tables[tt.Source]; !ok { l.tables[tt.Source] = make(map[string]map[string]schemacmp.Table) + l.finalTables[tt.Source] = make(map[string]map[string]schemacmp.Table) l.done[tt.Source] = make(map[string]map[string]bool) l.versions[tt.Source] = make(map[string]map[string]int64) } for schema, tables := range tt.UpTables { if _, ok := l.tables[tt.Source][schema]; !ok { l.tables[tt.Source][schema] = make(map[string]schemacmp.Table) + l.finalTables[tt.Source][schema] = make(map[string]schemacmp.Table) l.done[tt.Source][schema] = make(map[string]bool) l.versions[tt.Source][schema] = make(map[string]int64) } @@ -672,18 +526,20 @@ func (l *Lock) addTables(tts []TargetTable) { if _, ok := l.tables[tt.Source][schema][table]; !ok { ti, err := l.FetchTableInfos(tt.Task, tt.Source, schema, table) if err != nil { - log.L().Error("source table info not found, use joined table info instead", zap.String("task", tt.Task), zap.String("source", tt.Source), zap.String("schema", schema), zap.String("table", table), log.ShortError(err)) - l.tables[tt.Source][schema][table] = l.joined + log.L().Error("source table info not found, use init table info instead", zap.String("task", tt.Task), zap.String("source", tt.Source), zap.String("schema", schema), zap.String("table", table), log.ShortError(err)) + l.tables[tt.Source][schema][table] = l.initTable + l.finalTables[tt.Source][schema][table] = l.initTable } else { t := schemacmp.Encode(ti) log.L().Debug("get source table info", zap.String("task", tt.Task), zap.String("source", tt.Source), zap.String("schema", schema), zap.String("table", table), zap.Stringer("info", t)) l.tables[tt.Source][schema][table] = t + l.finalTables[tt.Source][schema][table] = t } l.done[tt.Source][schema][table] = false l.versions[tt.Source][schema][table] = 0 log.L().Info("table added to the lock", zap.String("lock", l.ID), zap.String("source", tt.Source), zap.String("schema", schema), zap.String("table", table), - zap.Stringer("table info", l.joined)) + zap.Stringer("table info", l.initTable)) } } } @@ -716,28 +572,32 @@ func (l *Lock) IsDroppedColumn(source, upSchema, upTable, col string) bool { } // AddDroppedColumn adds a dropped column name in both etcd and lock's column map. -func (l *Lock) AddDroppedColumn(info Info, col string) error { - source, upSchema, upTable := info.Source, info.UpSchema, info.UpTable - if l.IsDroppedColumn(source, upSchema, upTable, col) { - return nil +func (l *Lock) AddDroppedColumns(source, schema, table string, cols []string) error { + newCols := make([]string, 0, len(cols)) + for _, col := range cols { + if !l.IsDroppedColumn(source, schema, table, col) { + newCols = append(newCols, col) + } } - log.L().Info("add partially dropped columns", zap.String("column", col), zap.String("info", info.ShortString())) + log.L().Info("add partially dropped columns", zap.Strings("columns", newCols), zap.String("source", source), zap.String("schema", schema), zap.String("table", table)) - _, _, err := PutDroppedColumn(l.cli, genDDLLockID(info), col, info.Source, info.UpSchema, info.UpTable, DropNotDone) + _, _, err := PutDroppedColumns(l.cli, l.ID, source, schema, table, newCols, DropNotDone) if err != nil { return err } - if _, ok := l.columns[col]; !ok { - l.columns[col] = make(map[string]map[string]map[string]DropColumnStage) - } - if _, ok := l.columns[col][source]; !ok { - l.columns[col][source] = make(map[string]map[string]DropColumnStage) - } - if _, ok := l.columns[col][source][upSchema]; !ok { - l.columns[col][source][upSchema] = make(map[string]DropColumnStage) + for _, col := range newCols { + if _, ok := l.columns[col]; !ok { + l.columns[col] = make(map[string]map[string]map[string]DropColumnStage) + } + if _, ok := l.columns[col][source]; !ok { + l.columns[col][source] = make(map[string]map[string]DropColumnStage) + } + if _, ok := l.columns[col][source][schema]; !ok { + l.columns[col][source][schema] = make(map[string]DropColumnStage) + } + l.columns[col][source][schema][table] = DropNotDone } - l.columns[col][source][upSchema][upTable] = DropNotDone return nil } @@ -769,7 +629,7 @@ func (l *Lock) DeleteColumnsByOp(op Operation) error { done = DropDone } // mark col PartiallyDone/Done - _, _, err := PutDroppedColumn(l.cli, op.ID, col, op.Source, op.UpSchema, op.UpTable, done) + _, _, err := PutDroppedColumns(l.cli, op.ID, op.Source, op.UpSchema, op.UpTable, []string{col}, done) if err != nil { log.L().Error("cannot put drop column to etcd", log.ShortError(err)) return err @@ -815,20 +675,6 @@ func (l *Lock) DeleteColumnsByOp(op Operation) error { return nil } -// TableExist check whether table exists. -func (l *Lock) TableExist(source, schema, table string) bool { - if _, ok := l.tables[source]; !ok { - return false - } - if _, ok := l.tables[source][schema]; !ok { - return false - } - if _, ok := l.tables[source][schema][table]; !ok { - return false - } - return true -} - // AddDifferentFieldLenColumns checks whether dm adds columns with different field lengths. func AddDifferentFieldLenColumns(lockID, ddl string, oldJoined, newJoined schemacmp.Table) (string, error) { col, err := GetColumnName(lockID, ddl, ast.AlterTableAddColumns) @@ -842,7 +688,7 @@ func AddDifferentFieldLenColumns(lockID, ddl string, oldJoined, newJoined schema newCol, ok2 := newJoinedCols[col] if ok1 && ok2 && newCol.Flen != oldCol.Flen { return col, terror.ErrShardDDLOptimismTrySyncFail.Generate( - lockID, fmt.Sprintf("add columns with different field lengths."+ + lockID, fmt.Sprintf("add columns with different field lengths. "+ "ddl: %s, origLen: %d, newLen: %d", ddl, oldCol.Flen, newCol.Flen)) } } @@ -871,3 +717,457 @@ func GetColumnName(lockID, ddl string, tp ast.AlterTableType) (string, error) { } return "", nil } + +func contains(s []string, e string) bool { + for _, a := range s { + if a == e { + return true + } + } + return false +} + +// checkAddDropColumn check for ALTER TABLE ADD/DROP COLUMN statement +// FOR ADD COLUMN, check whether add column with a different field or add a dropped column +// FOR DROP COLUMN, return the droped column. +func (l *Lock) checkAddDropColumn(source, schema, table string, ddl string, prevTable, postTable schemacmp.Table, newDropColumns []string) (string, error) { + currTable := l.tables[source][schema][table] + defer func() { + l.tables[source][schema][table] = currTable + }() + + l.tables[source][schema][table] = prevTable + oldJoined, err := l.joinNormalTables() + if err != nil { + // nolint:nilerr + return "", nil + } + + l.tables[source][schema][table] = postTable + newJoined, err := l.joinNormalTables() + if err != nil { + // nolint:nilerr + return "", nil + } + + cmp, err := oldJoined.Compare(newJoined) + if err != nil { + // nolint:nilerr + return "", nil + } + + if cmp <= 0 { + if col, err2 := AddDifferentFieldLenColumns(l.ID, ddl, oldJoined, newJoined); err2 != nil { + // check for add column with a larger field len + return "", err2 + } else if _, err2 = AddDifferentFieldLenColumns(l.ID, ddl, postTable, newJoined); err2 != nil { + // check for add column with a smaller field len + return "", err2 + } else if len(col) > 0 && (l.IsDroppedColumn(source, schema, table, col) || contains(newDropColumns, col)) { + return "", terror.ErrShardDDLOptimismTrySyncFail.Generate(l.ID, fmt.Sprintf("add column %s that wasn't fully dropped in downstream. ddl: %s", col, ddl)) + } + } + + if cmp >= 0 { + if col, err2 := GetColumnName(l.ID, ddl, ast.AlterTableDropColumn); err2 != nil { + return "", err2 + } else if len(col) > 0 { + return col, nil + } + } + return "", nil +} + +// trySyncForOneDDL try sync for a DDL operation. +// e.g. `ALTER TABLE ADD COLUMN a, RENAME b TO c, DROP COLUMN d' will call this func three times. +// return whether joined table is changed and whether there is a conflict. +func (l *Lock) trySyncForOneDDL(source, schema, table string, prevTable, postTable schemacmp.Table) (schemaChanged bool, conflictStage ConflictStage) { + // we only support resolve one conflict DDL per table, + // so reset conflict table after receive new table info. + l.removeConflictTable(source, schema, table) + l.finalTables[source][schema][table] = l.tables[source][schema][table] + + // For idempotent DDL + // this often happens when an info TrySync twice, e.g. worker restart/resume task + idempotent := false + if cmp, err := prevTable.Compare(l.tables[source][schema][table]); err != nil || cmp != 0 { + if cmp, err := postTable.Compare(l.tables[source][schema][table]); err == nil && cmp == 0 { + idempotent = true + } + log.L().Warn("prev-table not equal table saved in master", zap.Stringer("master-table", l.tables[source][schema][table]), zap.Stringer("prev-table", prevTable)) + l.tables[source][schema][table] = prevTable + l.finalTables[source][schema][table] = prevTable + } + + tableCmp, tableErr := prevTable.Compare(postTable) + // Normal DDL + if tableErr == nil { + log.L().Debug("receive a normal DDL", zap.String("source", source), zap.String("schema", schema), zap.String("table", table), zap.Stringer("prevTable", prevTable), zap.Stringer("postTable", postTable)) + oldJoined, oldErr := l.joinNormalTables() + + l.tables[source][schema][table] = postTable + l.finalTables[source][schema][table] = postTable + + newJoined, newErr := l.joinNormalTables() + // normal DDL can be sync if no error + if newErr == nil { + // if a normal DDL let all final tables become no conflict + // return ConflictNone + if len(l.conflictTables) > 0 && l.noConflictForFinalTables() { + log.L().Info("all conflict resolved for the DDL", zap.String("source", source), zap.String("schema", schema), zap.String("table", table), zap.Stringer("prevTable", prevTable), zap.Stringer("postTable", postTable)) + err := l.redirectForConflictTables(source, schema, table) + if err != nil { + log.L().Error("failed to put redirect operation for conflict tables", log.ShortError(err)) + return false, ConflictDetected + } + l.resolveTables() + return true, ConflictNone + } + + if oldErr != nil { + return true, ConflictNone + } + joinedCmp, joinedErr := oldJoined.Compare(newJoined) + // special case: if the DDL does not affect the schema at all, assume it is + // idempotent and just execute the DDL directly. + // this often happens when executing `CREATE TABLE` statement + cmp, err2 := postTable.Compare(oldJoined) + + // return schema changed in 3 cases + // oldJoined != newJoined + // postTable == oldJoined (CREATE TABLE) + // prevTable < postTable + return (joinedErr != nil || joinedCmp != 0) || (err2 == nil && cmp == 0) || tableCmp < 0, ConflictNone + } + } + + log.L().Info("found conflict for DDL", zap.String("source", source), zap.String("schema", schema), zap.String("table", table), zap.Stringer("prevTable", prevTable), zap.Stringer("postTable", postTable), log.ShortError(tableErr)) + + if idempotent { + log.L().Info("return conflict DDL for idempotent DDL", zap.String("source", source), zap.String("schema", schema), zap.String("table", table), zap.Stringer("prevTable", prevTable), zap.Stringer("postTable", postTable)) + l.tables[source][schema][table] = postTable + l.finalTables[source][schema][table] = postTable + return true, ConflictNone + } + + // meet conflict DDL + // revert tables and update conflictTables and finalTables + l.tables[source][schema][table] = prevTable + l.addConflictTable(source, schema, table, postTable) + l.finalTables[source][schema][table] = postTable + + // if more than one conflict tables and this conflict DDL has no conflict with normal tables + // e.g. tb1,tb2 put ddl1(rename a to b); tb1 put ddl2(rename c to d); tb2 crash and reput ddl1(rename a to b) + // now tb2's ddl1 is a conflict DDL but has no conflict with normal tables + if l.multipleConflictTables() && l.noConflictWithNormalTables(source, schema, table, postTable) { + l.removeConflictTable(source, schema, table) + l.tables[source][schema][table] = postTable + return true, ConflictNone + } + + // if any conflict happened between conflict DDLs, return error + // e.g. tb1: "ALTER TABLE RENAME a TO b", tb2: "ALTER TABLE RENAME c TO d" + if !l.noConflictForConflictTables() { + log.L().Error("conflict happened with other conflict tables", zap.String("source", source), zap.String("schema", schema), zap.String("table", table), zap.Stringer("prevTable", prevTable), zap.Stringer("postTable", postTable)) + return false, ConflictDetected + } + + if l.noConflictForFinalTables() { + log.L().Info("all conflict resolved for the DDL", zap.String("source", source), zap.String("schema", schema), zap.String("table", table), zap.Stringer("prevTable", prevTable), zap.Stringer("postTable", postTable)) + err := l.redirectForConflictTables(source, schema, table) + if err != nil { + log.L().Error("failed to put redirect operation for conflict tables", log.ShortError(err)) + return false, ConflictDetected + } + l.resolveTables() + return true, ConflictNone + } + log.L().Debug("conflict hasn't been resolved", zap.String("source", source), zap.String("schema", schema), zap.String("table", table), zap.Stringer("prevTable", prevTable), zap.Stringer("postTable", postTable)) + return false, ConflictSkipWaitRedirect +} + +// joinTables join tables by tableType. +func (l *Lock) joinTables(tp tableType) (schemacmp.Table, error) { + var ( + joined schemacmp.Table + allTables map[string]map[string]map[string]schemacmp.Table + firstTable = true + ) + + switch tp { + case conflictTables: + allTables = l.conflictTables + case finalTables: + allTables = l.finalTables + default: + allTables = l.tables + } + + for source, schemaTables := range allTables { + for schema, tables := range schemaTables { + for table, ti := range tables { + if firstTable { + joined = ti + firstTable = false + continue + } + + newJoined, err := joined.Join(ti) + if err != nil { + return newJoined, errors.Errorf("failed to join tables with %s.%s.%s, joined: %s, table: %s, root cause: %s", source, schema, table, joined.String(), ti.String(), err.Error()) + } + joined = newJoined + } + } + } + + return joined, nil +} + +// Compare(joined,prev_tbx) == error +// For a conflict DDL make table become part of larger and another part of smaller, +// this function make sure all tables that need to be judged become part of smaller. +// e.g. `ALTER TABLE RENAME a TO b`, this function check whether all tables do not contain `a`. +// Prove: +// Compare(joined,prev_tbk) == error +// => Joined ⊇ prev_tbk-{a}+{b} && Joined ⊅ prev_tbk +// => a ∉ Joined. +func (l *Lock) allTableSmaller(tp tableType) bool { + var ( + joined schemacmp.Table + err error + ) + switch tp { + case conflictTables: + joined, err = l.joinConflictTables() + default: + joined, err = l.joinFinalTables() + } + + if err != nil { + return false + } + + for source, schemaTables := range l.conflictTables { + for schema, tables := range schemaTables { + for table := range tables { + ti := l.tables[source][schema][table] + + if _, err = joined.Compare(ti); err == nil { + return false + } + } + } + } + return true +} + +// Compare(Join(prev_tbx,tabley),post_tbx)>=0 +// For a conflict DDL make table become part of larger and another part of smaller, +// this function make sure all the tables that need to be judged become part of larger. +// e.g `ALTER TABLE RENAME a TO b`, this function check whether all tables contain `b`. +// Prove: +// Compare(Join(prev_tbx,tabley),post_tbx)>=0 +// => Compare(Join(prev_tbk,tabley),prev_tbk-{a}+{b})>=0 +// => Join(prev_tbk,tabley) ⊇ prev_tbk-{a}+{b} +// => b ∈ tabley. +func (l *Lock) allTableLarger(tp tableType) bool { + var judgeTables map[string]map[string]map[string]schemacmp.Table + + switch tp { + case normalTables: + judgeTables = l.tables + case conflictTables: + judgeTables = l.conflictTables + default: + judgeTables = l.finalTables + } + + for source, schemaTables := range l.conflictTables { + for schema, tables := range schemaTables { + for table, conflictTi := range tables { + // for every conflict table's prev_table + ti := l.tables[source][schema][table] + + // for every judge table + for _, sTables := range judgeTables { + for _, ts := range sTables { + for _, finalTi := range ts { + joined, err := ti.Join(finalTi) + if err != nil { + // modify column + joined = finalTi + } + if cmp, err := joined.Compare(conflictTi); err != nil || cmp < 0 { + return false + } + } + } + } + } + } + } + return true +} + +func (l *Lock) joinNormalTables() (schemacmp.Table, error) { + return l.joinTables(normalTables) +} + +func (l *Lock) joinFinalTables() (schemacmp.Table, error) { + return l.joinTables(finalTables) +} + +func (l *Lock) joinConflictTables() (schemacmp.Table, error) { + return l.joinTables(conflictTables) +} + +func (l *Lock) allConflictTableSmaller() bool { + return l.allTableSmaller(conflictTables) +} + +func (l *Lock) allFinalTableSmaller() bool { + return l.allTableSmaller(finalTables) +} + +func (l *Lock) allConflictTableLarger() bool { + return l.allTableLarger(conflictTables) +} + +func (l *Lock) allFinalTableLarger() bool { + return l.allTableLarger(finalTables) +} + +// judge whether a conflict DDL has no conflict with all normal tables. +func (l *Lock) noConflictWithNormalTables(source, schema, table string, postTable schemacmp.Table) bool { + // revert conflict tables and final tables + currentConflictTables := l.conflictTables + currentFinalTables := l.finalTables + defer func() { + l.conflictTables = currentConflictTables + l.finalTables = currentFinalTables + }() + + // reset conflict tables and final tables + l.conflictTables = make(map[string]map[string]map[string]schemacmp.Table) + l.finalTables = make(map[string]map[string]map[string]schemacmp.Table) + for source, schemaTables := range l.tables { + l.finalTables[source] = make(map[string]map[string]schemacmp.Table) + for schema, tables := range schemaTables { + l.finalTables[source][schema] = make(map[string]schemacmp.Table) + for table, ti := range tables { + l.finalTables[source][schema][table] = ti + } + } + } + // update for current conflict DDL + l.addConflictTable(source, schema, table, postTable) + l.finalTables[source][schema][table] = postTable + + return l.noConflictForFinalTables() +} + +// judge whether all conflict tables has no conflict. +func (l *Lock) noConflictForConflictTables() bool { + if _, err := l.joinConflictTables(); err != nil { + return false + } + if !l.allConflictTableSmaller() { + return false + } + if !l.allConflictTableLarger() { + return false + } + return true +} + +// judge whether all final tables has no conflict. +func (l *Lock) noConflictForFinalTables() bool { + if _, err := l.joinFinalTables(); err != nil { + return false + } + if !l.allFinalTableSmaller() { + return false + } + if !l.allFinalTableLarger() { + return false + } + return true +} + +func (l *Lock) addConflictTable(source, schema, table string, ti schemacmp.Table) { + if _, ok := l.conflictTables[source]; !ok { + l.conflictTables[source] = make(map[string]map[string]schemacmp.Table) + } + if _, ok := l.conflictTables[source][schema]; !ok { + l.conflictTables[source][schema] = make(map[string]schemacmp.Table) + } + l.conflictTables[source][schema][table] = ti +} + +func (l *Lock) removeConflictTable(source, schema, table string) { + if _, ok := l.conflictTables[source]; !ok { + return + } + if _, ok := l.conflictTables[source][schema]; !ok { + return + } + delete(l.conflictTables[source][schema], table) + if len(l.conflictTables[source][schema]) == 0 { + delete(l.conflictTables[source], schema) + } + if len(l.conflictTables[source]) == 0 { + delete(l.conflictTables, source) + } +} + +// resolveTables reset conflictTables and copy tables from final tables. +func (l *Lock) resolveTables() { + l.conflictTables = make(map[string]map[string]map[string]schemacmp.Table) + for source, schemaTables := range l.finalTables { + for schema, tables := range schemaTables { + for table, ti := range tables { + l.tables[source][schema][table] = ti + } + } + } +} + +// redirectForConflictTables put redirect Ops for all conflict tables. +func (l *Lock) redirectForConflictTables(callerSource, callerSchema, callerTable string) error { + for source, schemaTables := range l.conflictTables { + for schema, tables := range schemaTables { + for table := range tables { + if source == callerSource && schema == callerSchema && table == callerTable { + // no redirect for caller table + continue + } + op := NewOperation(l.ID, l.Task, source, schema, table, nil, ConflictResolved, "", false, nil) + // TODO(GMHDBJD): put these operation in one transaction + rev, succ, err := PutOperation(l.cli, false, op, 0) + if err != nil { + return err + } + log.L().Info("put redirect operation for conflict table", zap.String("lock", l.ID), + zap.Stringer("operation", op), zap.Bool("succeed", !succ), zap.Int64("revision", rev)) + } + } + } + return nil +} + +// multipleConflictTables check whether a lock has multiple conflict tables. +func (l *Lock) multipleConflictTables() bool { + cnt := 0 + for _, schemaTables := range l.conflictTables { + for _, tables := range schemaTables { + for range tables { + cnt++ + if cnt > 1 { + return true + } + } + } + } + return false +} diff --git a/dm/pkg/shardddl/optimism/lock_test.go b/dm/pkg/shardddl/optimism/lock_test.go index 5960b139491..40a590a7e11 100644 --- a/dm/pkg/shardddl/optimism/lock_test.go +++ b/dm/pkg/shardddl/optimism/lock_test.go @@ -588,7 +588,7 @@ func (t *testLock) TestLockTrySyncNoDiff(c *C) { p = parser.New() se = mock.NewContext() tblID int64 = 111 - DDLs1 = []string{"ALTER TABLE bar DROP COLUMN c1, ADD COLUMN c2 INT"} + DDLs1 = []string{"ALTER TABLE bar RENAME c1 TO c2"} ti0 = createTableInfo(c, p, se, tblID, `CREATE TABLE bar (id INT PRIMARY KEY, c1 INT)`) ti1 = createTableInfo(c, p, se, tblID, `CREATE TABLE bar (id INT PRIMARY KEY, c2 INT)`) // `c1` dropped, `c2` added @@ -615,7 +615,7 @@ func (t *testLock) TestLockTrySyncNoDiff(c *C) { // try sync for one table. info := newInfoWithVersion(task, source, db, tbls[0], downSchema, downTable, DDLs1, ti0, []*model.TableInfo{ti1}, vers) DDLs, cols, err := l.TrySync(info, tts) - c.Assert(terror.ErrShardDDLOptimismTrySyncFail.Equal(err), IsTrue) + c.Assert(terror.ErrShardDDLOptimismNeedSkipAndRedirect.Equal(err), IsTrue) c.Assert(DDLs, DeepEquals, []string{}) c.Assert(cols, DeepEquals, []string{}) c.Assert(l.versions, DeepEquals, vers) @@ -695,10 +695,10 @@ func (t *testLock) TestLockTrySyncNewTable(c *C) { c.Assert(ready[source1], HasLen, 1) c.Assert(ready[source1][db1], HasLen, 2) c.Assert(ready[source1][db1][tbl1], IsTrue) - c.Assert(ready[source1][db1][tbl2], IsTrue) + c.Assert(ready[source1][db1][tbl2], IsFalse) // new table use ti0 as init table c.Assert(ready[source2], HasLen, 1) c.Assert(ready[source2][db2], HasLen, 2) - c.Assert(ready[source2][db2][tbl1], IsTrue) + c.Assert(ready[source2][db2][tbl1], IsFalse) c.Assert(ready[source2][db2][tbl2], IsTrue) info = newInfoWithVersion(task, source1, db1, tbl2, downSchema, downTable, DDLs1, ti0, []*model.TableInfo{ti1}, vers) @@ -716,7 +716,7 @@ func (t *testLock) TestLockTrySyncNewTable(c *C) { c.Assert(ready[source1][db1][tbl2], IsTrue) c.Assert(ready[source2], HasLen, 1) c.Assert(ready[source2][db2], HasLen, 2) - c.Assert(ready[source2][db2][tbl1], IsTrue) + c.Assert(ready[source2][db2][tbl1], IsFalse) c.Assert(ready[source2][db2][tbl2], IsTrue) } @@ -779,10 +779,13 @@ func (t *testLock) TestLockTrySyncRevert(c *C) { ready := l.Ready() c.Assert(ready[source][db][tbls[0]], IsTrue) c.Assert(ready[source][db][tbls[1]], IsFalse) - cmp, err := l.tables[source][db][tbls[0]].Compare(l.Joined()) + + joined, err := l.Joined() + c.Assert(err, IsNil) + cmp, err := l.tables[source][db][tbls[0]].Compare(joined) c.Assert(err, IsNil) c.Assert(cmp, Equals, 0) - cmp, err = l.tables[source][db][tbls[1]].Compare(l.Joined()) + cmp, err = l.tables[source][db][tbls[1]].Compare(joined) c.Assert(err, IsNil) c.Assert(cmp, Equals, -1) @@ -811,10 +814,12 @@ func (t *testLock) TestLockTrySyncRevert(c *C) { ready = l.Ready() c.Assert(ready[source][db][tbls[0]], IsTrue) c.Assert(ready[source][db][tbls[1]], IsFalse) - cmp, err = l.tables[source][db][tbls[0]].Compare(l.Joined()) + joined, err = l.Joined() + c.Assert(err, IsNil) + cmp, err = l.tables[source][db][tbls[0]].Compare(joined) c.Assert(err, IsNil) c.Assert(cmp, Equals, 0) - cmp, err = l.tables[source][db][tbls[1]].Compare(l.Joined()) + cmp, err = l.tables[source][db][tbls[1]].Compare(joined) c.Assert(err, IsNil) c.Assert(cmp, Equals, -1) @@ -827,10 +832,12 @@ func (t *testLock) TestLockTrySyncRevert(c *C) { c.Assert(l.versions, DeepEquals, vers) ready = l.Ready() c.Assert(ready[source][db][tbls[1]], IsFalse) - cmp, err = l.tables[source][db][tbls[0]].Compare(l.Joined()) + joined, err = l.Joined() + c.Assert(err, IsNil) + cmp, err = l.tables[source][db][tbls[0]].Compare(joined) c.Assert(err, IsNil) c.Assert(cmp, Equals, 0) - cmp, err = l.tables[source][db][tbls[1]].Compare(l.Joined()) + cmp, err = l.tables[source][db][tbls[1]].Compare(joined) c.Assert(err, IsNil) c.Assert(cmp, Equals, -1) @@ -860,10 +867,13 @@ func (t *testLock) TestLockTrySyncRevert(c *C) { c.Assert(l.versions, DeepEquals, vers) ready = l.Ready() c.Assert(ready[source][db][tbls[1]], IsFalse) - cmp, err = l.tables[source][db][tbls[0]].Compare(l.Joined()) + + joined, err = l.Joined() + c.Assert(err, IsNil) + cmp, err = l.tables[source][db][tbls[0]].Compare(joined) c.Assert(err, IsNil) c.Assert(cmp, Equals, 0) - cmp, err = l.tables[source][db][tbls[1]].Compare(l.Joined()) + cmp, err = l.tables[source][db][tbls[1]].Compare(joined) c.Assert(err, IsNil) c.Assert(cmp, Equals, -1) @@ -876,10 +886,13 @@ func (t *testLock) TestLockTrySyncRevert(c *C) { c.Assert(l.versions, DeepEquals, vers) ready = l.Ready() c.Assert(ready[source][db][tbls[1]], IsFalse) - cmp, err = l.tables[source][db][tbls[0]].Compare(l.Joined()) + + joined, err = l.Joined() + c.Assert(err, IsNil) + cmp, err = l.tables[source][db][tbls[0]].Compare(joined) c.Assert(err, IsNil) c.Assert(cmp, Equals, 0) - cmp, err = l.tables[source][db][tbls[1]].Compare(l.Joined()) + cmp, err = l.tables[source][db][tbls[1]].Compare(joined) c.Assert(err, IsNil) c.Assert(cmp, Equals, -1) @@ -943,10 +956,12 @@ func (t *testLock) TestLockTrySyncConflictNonIntrusive(c *C) { ready := l.Ready() c.Assert(ready[source][db][tbls[0]], IsTrue) c.Assert(ready[source][db][tbls[1]], IsFalse) - cmp, err := l.tables[source][db][tbls[0]].Compare(l.Joined()) + joined, err := l.Joined() + c.Assert(err, IsNil) + cmp, err := l.tables[source][db][tbls[0]].Compare(joined) c.Assert(err, IsNil) c.Assert(cmp, Equals, 0) - cmp, err = l.tables[source][db][tbls[1]].Compare(l.Joined()) + cmp, err = l.tables[source][db][tbls[1]].Compare(joined) c.Assert(err, IsNil) c.Assert(cmp, Equals, -1) @@ -957,7 +972,9 @@ func (t *testLock) TestLockTrySyncConflictNonIntrusive(c *C) { c.Assert(DDLs, DeepEquals, []string{}) c.Assert(cols, DeepEquals, []string{}) c.Assert(l.versions, DeepEquals, vers) - cmp, err = l.tables[source][db][tbls[1]].Compare(l.Joined()) + joined, err = l.Joined() + c.Assert(err, IsNil) + cmp, err = l.tables[source][db][tbls[1]].Compare(joined) // join table isn't updated c.Assert(err, IsNil) c.Assert(cmp, Equals, -1) @@ -974,10 +991,12 @@ func (t *testLock) TestLockTrySyncConflictNonIntrusive(c *C) { ready = l.Ready() // all table ready c.Assert(ready[source][db][tbls[0]], IsTrue) c.Assert(ready[source][db][tbls[1]], IsTrue) - cmp, err = l.tables[source][db][tbls[0]].Compare(l.Joined()) + joined, err = l.Joined() + c.Assert(err, IsNil) + cmp, err = l.tables[source][db][tbls[0]].Compare(joined) c.Assert(err, IsNil) c.Assert(cmp, Equals, 0) - cmp, err = l.tables[source][db][tbls[1]].Compare(l.Joined()) + cmp, err = l.tables[source][db][tbls[1]].Compare(joined) c.Assert(err, IsNil) c.Assert(cmp, Equals, 0) @@ -988,7 +1007,9 @@ func (t *testLock) TestLockTrySyncConflictNonIntrusive(c *C) { c.Assert(DDLs, DeepEquals, DDLs2) c.Assert(cols, DeepEquals, []string{}) c.Assert(l.versions, DeepEquals, vers) - cmp, err = l.tables[source][db][tbls[1]].Compare(l.Joined()) + joined, err = l.Joined() + c.Assert(err, IsNil) + cmp, err = l.tables[source][db][tbls[1]].Compare(joined) c.Assert(err, IsNil) c.Assert(cmp, Equals, 0) ready = l.Ready() @@ -1066,10 +1087,12 @@ func (t *testLock) TestLockTrySyncConflictIntrusive(c *C) { ready := l.Ready() c.Assert(ready[source][db][tbls[0]], IsTrue) c.Assert(ready[source][db][tbls[1]], IsFalse) - cmp, err := l.tables[source][db][tbls[0]].Compare(l.Joined()) + joined, err := l.Joined() + c.Assert(err, IsNil) + cmp, err := l.tables[source][db][tbls[0]].Compare(joined) c.Assert(err, IsNil) c.Assert(cmp, Equals, 0) - cmp, err = l.tables[source][db][tbls[1]].Compare(l.Joined()) + cmp, err = l.tables[source][db][tbls[1]].Compare(joined) c.Assert(err, IsNil) c.Assert(cmp, Equals, -1) @@ -1080,7 +1103,9 @@ func (t *testLock) TestLockTrySyncConflictIntrusive(c *C) { c.Assert(DDLs, DeepEquals, []string{}) c.Assert(cols, DeepEquals, []string{}) c.Assert(l.versions, DeepEquals, vers) - cmp, err = l.tables[source][db][tbls[1]].Compare(l.Joined()) + joined, err = l.Joined() + c.Assert(err, IsNil) + cmp, err = l.tables[source][db][tbls[1]].Compare(joined) // join table isn't updated c.Assert(err, IsNil) c.Assert(cmp, Equals, -1) @@ -1094,7 +1119,9 @@ func (t *testLock) TestLockTrySyncConflictIntrusive(c *C) { c.Assert(DDLs, DeepEquals, []string{}) c.Assert(cols, DeepEquals, []string{}) c.Assert(l.versions, DeepEquals, vers) - cmp, err = l.tables[source][db][tbls[1]].Compare(l.Joined()) + joined, err = l.Joined() + c.Assert(err, IsNil) + cmp, err = l.tables[source][db][tbls[1]].Compare(joined) c.Assert(err, IsNil) c.Assert(cmp, Equals, -1) @@ -1105,7 +1132,9 @@ func (t *testLock) TestLockTrySyncConflictIntrusive(c *C) { c.Assert(DDLs, DeepEquals, []string{}) c.Assert(cols, DeepEquals, []string{}) c.Assert(l.versions, DeepEquals, vers) - cmp, err = l.tables[source][db][tbls[1]].Compare(l.Joined()) + joined, err = l.Joined() + c.Assert(err, IsNil) + cmp, err = l.tables[source][db][tbls[1]].Compare(joined) c.Assert(err, IsNil) c.Assert(cmp, Equals, -1) ready = l.Ready() @@ -1118,7 +1147,9 @@ func (t *testLock) TestLockTrySyncConflictIntrusive(c *C) { c.Assert(DDLs, DeepEquals, DDLs) c.Assert(cols, DeepEquals, []string{}) c.Assert(l.versions, DeepEquals, vers) - cmp, err = l.tables[source][db][tbls[1]].Compare(l.Joined()) + joined, err = l.Joined() + c.Assert(err, IsNil) + cmp, err = l.tables[source][db][tbls[1]].Compare(joined) c.Assert(err, IsNil) c.Assert(cmp, Equals, 0) t.checkLockSynced(c, l) @@ -1135,10 +1166,12 @@ func (t *testLock) TestLockTrySyncConflictIntrusive(c *C) { ready = l.Ready() c.Assert(ready[source][db][tbls[0]], IsTrue) c.Assert(ready[source][db][tbls[1]], IsFalse) - cmp, err = l.tables[source][db][tbls[0]].Compare(l.Joined()) + joined, err = l.Joined() + c.Assert(err, IsNil) + cmp, err = l.tables[source][db][tbls[0]].Compare(joined) c.Assert(err, IsNil) c.Assert(cmp, Equals, 0) - cmp, err = l.tables[source][db][tbls[1]].Compare(l.Joined()) + cmp, err = l.tables[source][db][tbls[1]].Compare(joined) c.Assert(err, IsNil) c.Assert(cmp, Equals, -1) @@ -1148,7 +1181,9 @@ func (t *testLock) TestLockTrySyncConflictIntrusive(c *C) { c.Assert(terror.ErrShardDDLOptimismTrySyncFail.Equal(err), IsTrue) c.Assert(DDLs, DeepEquals, []string{}) c.Assert(cols, DeepEquals, []string{}) - cmp, err = l.tables[source][db][tbls[1]].Compare(l.Joined()) + joined, err = l.Joined() + c.Assert(err, IsNil) + cmp, err = l.tables[source][db][tbls[1]].Compare(joined) c.Assert(err, IsNil) c.Assert(cmp, Equals, -1) c.Assert(l.versions, DeepEquals, vers) @@ -1166,10 +1201,12 @@ func (t *testLock) TestLockTrySyncConflictIntrusive(c *C) { ready = l.Ready() c.Assert(ready[source][db][tbls[0]], IsFalse) c.Assert(ready[source][db][tbls[1]], IsFalse) - cmp, err = l.tables[source][db][tbls[0]].Compare(l.Joined()) + joined, err = l.Joined() + c.Assert(err, IsNil) + cmp, err = l.tables[source][db][tbls[0]].Compare(joined) c.Assert(err, IsNil) c.Assert(cmp, Equals, -1) - cmp, err = l.tables[source][db][tbls[1]].Compare(l.Joined()) + cmp, err = l.tables[source][db][tbls[1]].Compare(joined) c.Assert(err, IsNil) c.Assert(cmp, Equals, -1) @@ -1251,13 +1288,13 @@ func (t *testLock) TestLockTrySyncMultipleChangeDDL(c *C) { // inconsistent ddls and table infos info := newInfoWithVersion(task, sources[0], dbs[0], tbls[0], downSchema, downTable, DDLs1[:1], ti0, []*model.TableInfo{ti1_1, ti1}, vers) DDLs, cols, err := l.TrySync(info, tts) - c.Assert(DDLs, DeepEquals, DDLs1[:1]) + c.Assert(DDLs, DeepEquals, []string{}) c.Assert(cols, DeepEquals, []string{}) c.Assert(terror.ErrMasterInconsistentOptimisticDDLsAndInfo.Equal(err), IsTrue) info = newInfoWithVersion(task, sources[0], dbs[0], tbls[0], downSchema, downTable, DDLs1, ti0, []*model.TableInfo{ti1}, vers) DDLs, cols, err = l.TrySync(info, tts) - c.Assert(DDLs, DeepEquals, DDLs1) + c.Assert(DDLs, DeepEquals, []string{}) c.Assert(cols, DeepEquals, []string{}) c.Assert(terror.ErrMasterInconsistentOptimisticDDLsAndInfo.Equal(err), IsTrue) @@ -1411,7 +1448,7 @@ func (t *testLock) TestTryRemoveTable(c *C) { c.Assert(ready[source][db][tbl1], IsTrue) c.Assert(l.versions, DeepEquals, vers) - // CASE: remove a table will not rebuild joined schema now. + // CASE: remove a table will rebuild joined schema now. // TrySync to add the second back. vers[source][db][tbl2] = 0 info = newInfoWithVersion(task, source, db, tbl2, downSchema, downTable, DDLs2, ti1, []*model.TableInfo{ti2}, vers) @@ -1434,7 +1471,7 @@ func (t *testLock) TestTryRemoveTable(c *C) { c.Assert(ready, HasLen, 1) c.Assert(ready[source], HasLen, 1) c.Assert(ready[source][db], HasLen, 1) - c.Assert(ready[source][db][tbl1], IsFalse) // the joined schema is not rebuild. + c.Assert(ready[source][db][tbl1], IsTrue) // the joined schema is rebuild. c.Assert(l.versions, DeepEquals, vers) // CASE: try to remove for not-exists table. @@ -1707,7 +1744,7 @@ func (t *testLock) TestAddDifferentFieldLenColumns(c *C) { info = newInfoWithVersion(task, source, db, tbls[1], downSchema, downTable, DDLs2, ti0, []*model.TableInfo{ti2}, vers) DDLs, cols, err = l.TrySync(info, tts) c.Assert(err, ErrorMatches, ".*add columns with different field lengths.*") - c.Assert(DDLs, DeepEquals, DDLs2) + c.Assert(DDLs, DeepEquals, []string{}) c.Assert(cols, DeepEquals, []string{}) c.Assert(l.versions, DeepEquals, vers) @@ -1730,7 +1767,7 @@ func (t *testLock) TestAddDifferentFieldLenColumns(c *C) { info = newInfoWithVersion(task, source, db, tbls[0], downSchema, downTable, DDLs1, ti0, []*model.TableInfo{ti1}, vers) DDLs, cols, err = l.TrySync(info, tts) c.Assert(err, ErrorMatches, ".*add columns with different field lengths.*") - c.Assert(DDLs, DeepEquals, DDLs1) + c.Assert(DDLs, DeepEquals, []string{}) c.Assert(cols, DeepEquals, []string{}) c.Assert(l.versions, DeepEquals, vers) } @@ -1757,6 +1794,7 @@ func (t *testLock) TestAddNotFullyDroppedColumns(c *C) { DDLs2 = []string{"ALTER TABLE bar DROP COLUMN b"} DDLs3 = []string{"ALTER TABLE bar ADD COLUMN b INT"} DDLs4 = []string{"ALTER TABLE bar ADD COLUMN c INT"} + DDLs5 = []string{"ALTER TABLE bar DROP COLUMN c", "ALTER TABLE bar ADD COLUMN c INT"} tables = map[string]map[string]struct{}{db: {tbls[0]: struct{}{}, tbls[1]: struct{}{}}} tts = []TargetTable{newTargetTable(task, source, downSchema, downTable, tables)} @@ -1880,6 +1918,9 @@ func (t *testLock) TestAddNotFullyDroppedColumns(c *C) { c.Assert(cols, DeepEquals, []string{}) c.Assert(l.versions, DeepEquals, vers) c.Assert(l.IsResolved(), IsFalse) + + _, _, err = l.TrySync(newInfoWithVersion(task, source, db, tbls[0], downSchema, downTable, DDLs5, ti0, []*model.TableInfo{ti1, ti0}, vers), tts) + c.Assert(err, ErrorMatches, ".*add column c that wasn't fully dropped in downstream.*") } func (t *testLock) trySyncForAllTablesLarger(c *C, l *Lock, @@ -1999,7 +2040,9 @@ func (t *testLock) TestLockTrySyncDifferentIndex(c *C) { c.Assert(synced, IsFalse) c.Assert(remain, Equals, 1) - cmp, err = l.tables[source][db][tbls[0]].Compare(l.joined) + joined, err := l.joinFinalTables() + c.Assert(err, IsNil) + cmp, err = l.tables[source][db][tbls[0]].Compare(joined) c.Assert(err, IsNil) c.Assert(cmp, Equals, 0) @@ -2071,3 +2114,634 @@ func (t *testLock) TestFetchTableInfo(c *C) { c.Assert(mock.ExpectationsWereMet(), IsNil) c.Assert(ti, DeepEquals, ti0) } + +func (t *testLock) TestCheckAddDropColumns(c *C) { + var ( + ID = "test-`foo`.`bar`" + task = "test" + source = "mysql-replica-1" + downSchema = "db" + downTable = "bar" + db = "db" + tbls = []string{"bar1", "bar2"} + p = parser.New() + se = mock.NewContext() + tblID int64 = 111 + DDLs1 = "ALTER TABLE bar ADD COLUMN a VARCHAR(1)" + DDLs2 = "ALTER TABLE bar ADD COLUMN a VARCHAR(2)" + DDLs3 = "ALTER TABLE bar DROP COLUMN col" + DDLs4 = "ALTER TABLE bar ADD COLUMN col int" + ti0 = createTableInfo(c, p, se, tblID, `CREATE TABLE bar (id INT PRIMARY KEY, col int)`) + ti1 = createTableInfo(c, p, se, tblID, `CREATE TABLE bar (id INT PRIMARY KEY, col int, a VARCHAR(1))`) + ti2 = createTableInfo(c, p, se, tblID, `CREATE TABLE bar (id INT PRIMARY KEY, col int, a VARCHAR(2))`) + ti3 = createTableInfo(c, p, se, tblID, `CREATE TABLE bar (id INT PRIMARY KEY, a VARCHAR(2))`) + tables = map[string]map[string]struct{}{ + db: {tbls[0]: struct{}{}, tbls[1]: struct{}{}}, + } + tts = []TargetTable{ + newTargetTable(task, source, downSchema, downTable, tables), + } + + l = NewLock(etcdTestCli, ID, task, downSchema, downTable, schemacmp.Encode(ti0), tts, nil) + ) + + l.tables[source][db][tbls[0]] = schemacmp.Encode(ti0) + l.tables[source][db][tbls[1]] = schemacmp.Encode(ti1) + + col, err := l.checkAddDropColumn(source, db, tbls[0], DDLs1, schemacmp.Encode(ti0), schemacmp.Encode(ti1), nil) + c.Assert(err, IsNil) + c.Assert(len(col), Equals, 0) + + l.tables[source][db][tbls[0]] = schemacmp.Encode(ti1) + col, err = l.checkAddDropColumn(source, db, tbls[1], DDLs2, schemacmp.Encode(ti0), schemacmp.Encode(ti2), nil) + c.Assert(err, NotNil) + c.Assert(err, ErrorMatches, ".*add columns with different field lengths.*") + c.Assert(len(col), Equals, 0) + + col, err = l.checkAddDropColumn(source, db, tbls[0], DDLs3, schemacmp.Encode(ti2), schemacmp.Encode(ti3), nil) + c.Assert(err, IsNil) + c.Assert(col, Equals, "col") + + l.columns = map[string]map[string]map[string]map[string]DropColumnStage{ + "col": { + source: { + db: {tbls[0]: DropNotDone}, + }, + }, + } + + col, err = l.checkAddDropColumn(source, db, tbls[0], DDLs4, schemacmp.Encode(ti3), schemacmp.Encode(ti2), nil) + c.Assert(err, NotNil) + c.Assert(err, ErrorMatches, ".*add column .* that wasn't fully dropped in downstream.*") + c.Assert(len(col), Equals, 0) +} + +func (t *testLock) TestJoinTables(c *C) { + var ( + source = "mysql-replica-1" + db = "db" + tbls = []string{"bar1", "bar2"} + p = parser.New() + se = mock.NewContext() + tblID int64 = 111 + ti0 = createTableInfo(c, p, se, tblID, `CREATE TABLE bar (id INT PRIMARY KEY, col int)`) + ti1 = createTableInfo(c, p, se, tblID, `CREATE TABLE bar (id INT PRIMARY KEY, col int, a VARCHAR(1))`) + ti2 = createTableInfo(c, p, se, tblID, `CREATE TABLE bar (id INT PRIMARY KEY, col varchar(4))`) + t0 = schemacmp.Encode(ti0) + t1 = schemacmp.Encode(ti1) + t2 = schemacmp.Encode(ti2) + ) + + l := &Lock{ + tables: map[string]map[string]map[string]schemacmp.Table{ + source: { + db: {tbls[0]: t0, tbls[1]: t0}, + }, + }, + finalTables: map[string]map[string]map[string]schemacmp.Table{ + source: { + db: {tbls[0]: t0, tbls[1]: t0}, + }, + }, + } + + joined, err := l.joinNormalTables() + c.Assert(err, IsNil) + cmp, err := joined.Compare(t0) + c.Assert(err, IsNil) + c.Assert(cmp, Equals, 0) + joined, err = l.joinFinalTables() + c.Assert(err, IsNil) + cmp, err = joined.Compare(t0) + c.Assert(err, IsNil) + c.Assert(cmp, Equals, 0) + _, err = l.joinConflictTables() + c.Assert(err, IsNil) + + l.tables[source][db][tbls[0]] = t1 + l.finalTables[source][db][tbls[0]] = t1 + + joined, err = l.joinNormalTables() + c.Assert(err, IsNil) + cmp, err = joined.Compare(t1) + c.Assert(err, IsNil) + c.Assert(cmp, Equals, 0) + joined, err = l.joinFinalTables() + c.Assert(err, IsNil) + cmp, err = joined.Compare(t1) + c.Assert(err, IsNil) + c.Assert(cmp, Equals, 0) + _, err = l.joinConflictTables() + c.Assert(err, IsNil) + + l.tables[source][db][tbls[1]] = t1 + l.finalTables[source][db][tbls[1]] = t1 + l.conflictTables = map[string]map[string]map[string]schemacmp.Table{ + source: { + db: {tbls[0]: t2}, + }, + } + + joined, err = l.joinNormalTables() + c.Assert(err, IsNil) + cmp, err = joined.Compare(t1) + c.Assert(err, IsNil) + c.Assert(cmp, Equals, 0) + joined, err = l.joinFinalTables() + c.Assert(err, IsNil) + cmp, err = joined.Compare(t1) + c.Assert(err, IsNil) + c.Assert(cmp, Equals, 0) + joined, err = l.joinConflictTables() + c.Assert(err, IsNil) + cmp, err = joined.Compare(t2) + c.Assert(err, IsNil) + c.Assert(cmp, Equals, 0) + + l.tables[source][db][tbls[1]] = t2 + _, err = l.joinNormalTables() + c.Assert(err, NotNil) + c.Assert(err, ErrorMatches, ".*incompatible mysql type.*") + + l.resolveTables() + c.Assert(l.conflictTables, HasLen, 0) + c.Assert(l.tables, DeepEquals, l.finalTables) +} + +func (t *testLock) TestAddRemoveConflictTable(c *C) { + var ( + source = "source" + schema = "schema" + table1 = "table1" + table2 = "table2" + table3 = "table3" + p = parser.New() + se = mock.NewContext() + tblID int64 = 111 + ti0 = createTableInfo(c, p, se, tblID, `CREATE TABLE bar (id INT PRIMARY KEY, col int)`) + t0 = schemacmp.Encode(ti0) + ) + l := &Lock{ + conflictTables: make(map[string]map[string]map[string]schemacmp.Table), + } + c.Assert(l.conflictTables, HasLen, 0) + + l.addConflictTable(source, schema, table1, t0) + c.Assert(l.conflictTables, HasLen, 1) + c.Assert(l.conflictTables[source], HasLen, 1) + c.Assert(l.conflictTables[source][schema], HasLen, 1) + tb := l.conflictTables[source][schema][table1] + cmp, err := tb.Compare(t0) + c.Assert(err, IsNil) + c.Assert(cmp, Equals, 0) + + l.addConflictTable(source, schema, table1, t0) + c.Assert(l.conflictTables, HasLen, 1) + c.Assert(l.conflictTables[source], HasLen, 1) + c.Assert(l.conflictTables[source][schema], HasLen, 1) + + l.addConflictTable(source, schema, table2, t0) + c.Assert(l.conflictTables, HasLen, 1) + c.Assert(l.conflictTables[source], HasLen, 1) + c.Assert(l.conflictTables[source][schema], HasLen, 2) + tb = l.conflictTables[source][schema][table2] + cmp, err = tb.Compare(t0) + c.Assert(err, IsNil) + c.Assert(cmp, Equals, 0) + + l.removeConflictTable(source, schema, table3) + c.Assert(l.conflictTables[source][schema], HasLen, 2) + + l.removeConflictTable(source, schema, table1) + c.Assert(l.conflictTables[source][schema], HasLen, 1) + tb = l.conflictTables[source][schema][table2] + cmp, err = tb.Compare(t0) + c.Assert(err, IsNil) + c.Assert(cmp, Equals, 0) + + l.removeConflictTable(source, schema, table2) + c.Assert(l.conflictTables, HasLen, 0) +} + +func (t *testLock) TestAllTableSmallerLarger(c *C) { + var ( + source = "source" + schema = "schema" + table1 = "table1" + table2 = "table2" + p = parser.New() + se = mock.NewContext() + tblID int64 = 111 + ti0 = createTableInfo(c, p, se, tblID, `CREATE TABLE bar (id INT PRIMARY KEY, col int)`) + ti1 = createTableInfo(c, p, se, tblID, `CREATE TABLE bar (id INT PRIMARY KEY, new_col int)`) + ti2 = createTableInfo(c, p, se, tblID, `CREATE TABLE bar (id INT PRIMARY KEY, new_col varchar(4))`) + ti3 = createTableInfo(c, p, se, tblID, `CREATE TABLE bar (a INT PRIMARY KEY, new_col varchar(4))`) + ti4 = createTableInfo(c, p, se, tblID, `CREATE TABLE bar (id INT PRIMARY KEY, new_col2 varchar(4))`) + ti5 = createTableInfo(c, p, se, tblID, `CREATE TABLE bar (id varchar(4) PRIMARY KEY, new_col int)`) + ti6 = createTableInfo(c, p, se, tblID, `CREATE TABLE bar (id INT PRIMARY KEY, col int, new_col int)`) + ti7 = createTableInfo(c, p, se, tblID, `CREATE TABLE bar (id INT PRIMARY KEY, col int, new_col varchar(4))`) + ti8 = createTableInfo(c, p, se, tblID, `CREATE TABLE bar (id INT PRIMARY KEY, col int, col2 int not null)`) + ti9 = createTableInfo(c, p, se, tblID, `CREATE TABLE bar (a INT PRIMARY KEY, new_col int)`) + t0 = schemacmp.Encode(ti0) + t1 = schemacmp.Encode(ti1) + t2 = schemacmp.Encode(ti2) + t3 = schemacmp.Encode(ti3) + t4 = schemacmp.Encode(ti4) + t5 = schemacmp.Encode(ti5) + t6 = schemacmp.Encode(ti6) + t7 = schemacmp.Encode(ti7) + t8 = schemacmp.Encode(ti8) + t9 = schemacmp.Encode(ti9) + ) + l := &Lock{ + tables: map[string]map[string]map[string]schemacmp.Table{ + source: { + schema: {table1: t0, table2: t0}, + }, + }, + finalTables: map[string]map[string]map[string]schemacmp.Table{ + source: { + schema: {table1: t0, table2: t0}, + }, + }, + conflictTables: make(map[string]map[string]map[string]schemacmp.Table), + } + c.Assert(l.allFinalTableSmaller(), IsTrue) + + // rename table + l.addConflictTable(source, schema, table1, t1) + l.finalTables[source][schema][table1] = t1 + c.Assert(l.allConflictTableSmaller(), IsTrue) + c.Assert(l.allConflictTableLarger(), IsTrue) + c.Assert(l.allFinalTableSmaller(), IsFalse) + c.Assert(l.allFinalTableLarger(), IsFalse) + l.addConflictTable(source, schema, table2, t1) + l.finalTables[source][schema][table2] = t1 + c.Assert(l.allConflictTableSmaller(), IsTrue) + c.Assert(l.allConflictTableLarger(), IsTrue) + c.Assert(l.allFinalTableSmaller(), IsTrue) + c.Assert(l.allFinalTableLarger(), IsTrue) + // reset + l.resolveTables() + c.Assert(l.conflictTables, HasLen, 0) + c.Assert(l.tables, DeepEquals, l.finalTables) + c.Assert(l.tables[source][schema], HasLen, 2) + + // modify column + l.addConflictTable(source, schema, table1, t2) + l.finalTables[source][schema][table1] = t2 + c.Assert(l.allConflictTableSmaller(), IsTrue) + c.Assert(l.allConflictTableLarger(), IsTrue) + c.Assert(l.allFinalTableSmaller(), IsFalse) + c.Assert(l.allFinalTableLarger(), IsFalse) + l.addConflictTable(source, schema, table2, t2) + l.finalTables[source][schema][table2] = t2 + c.Assert(l.allConflictTableSmaller(), IsTrue) + c.Assert(l.allConflictTableLarger(), IsTrue) + c.Assert(l.allFinalTableSmaller(), IsTrue) + c.Assert(l.allFinalTableLarger(), IsTrue) + // reset + l.resolveTables() + c.Assert(l.conflictTables, HasLen, 0) + c.Assert(l.tables, DeepEquals, l.finalTables) + c.Assert(l.tables[source][schema], HasLen, 2) + c.Assert(l.tables[source][schema][table1], DeepEquals, t2) + c.Assert(l.tables[source][schema][table2], DeepEquals, t2) + + // different rename + l.addConflictTable(source, schema, table1, t3) + l.finalTables[source][schema][table1] = t3 + c.Assert(l.allConflictTableSmaller(), IsTrue) + c.Assert(l.allConflictTableLarger(), IsTrue) + c.Assert(l.allFinalTableSmaller(), IsFalse) + c.Assert(l.allFinalTableLarger(), IsFalse) + l.addConflictTable(source, schema, table2, t4) + l.finalTables[source][schema][table2] = t4 + c.Assert(l.allConflictTableSmaller(), IsFalse) + c.Assert(l.allConflictTableLarger(), IsFalse) + c.Assert(l.allFinalTableSmaller(), IsFalse) + c.Assert(l.allFinalTableLarger(), IsFalse) + // reset + l.finalTables[source][schema][table1] = t1 + l.finalTables[source][schema][table2] = t1 + l.resolveTables() + c.Assert(l.conflictTables, HasLen, 0) + c.Assert(l.tables, DeepEquals, l.finalTables) + c.Assert(l.tables[source][schema], HasLen, 2) + c.Assert(l.tables[source][schema][table1], DeepEquals, t1) + c.Assert(l.tables[source][schema][table2], DeepEquals, t1) + + // different modify + l.addConflictTable(source, schema, table1, t2) + l.finalTables[source][schema][table1] = t2 + c.Assert(l.allConflictTableSmaller(), IsTrue) + c.Assert(l.allConflictTableLarger(), IsTrue) + c.Assert(l.allFinalTableSmaller(), IsFalse) + c.Assert(l.allFinalTableLarger(), IsFalse) + l.addConflictTable(source, schema, table2, t5) + l.finalTables[source][schema][table2] = t5 + c.Assert(l.allConflictTableSmaller(), IsFalse) + c.Assert(l.allConflictTableLarger(), IsFalse) + c.Assert(l.allFinalTableSmaller(), IsFalse) + c.Assert(l.allFinalTableLarger(), IsFalse) + // reset + l.finalTables[source][schema][table1] = t1 + l.finalTables[source][schema][table2] = t1 + l.resolveTables() + c.Assert(l.conflictTables, HasLen, 0) + c.Assert(l.tables, DeepEquals, l.finalTables) + c.Assert(l.tables[source][schema], HasLen, 2) + c.Assert(l.tables[source][schema][table1], DeepEquals, t1) + c.Assert(l.tables[source][schema][table2], DeepEquals, t1) + + // one table rename, one table modify + l.addConflictTable(source, schema, table1, t4) + l.finalTables[source][schema][table1] = t4 + c.Assert(l.allConflictTableSmaller(), IsTrue) + c.Assert(l.allConflictTableLarger(), IsTrue) + c.Assert(l.allFinalTableSmaller(), IsFalse) + c.Assert(l.allFinalTableLarger(), IsFalse) + l.addConflictTable(source, schema, table2, t5) + l.finalTables[source][schema][table2] = t5 + c.Assert(l.allConflictTableSmaller(), IsFalse) + c.Assert(l.allConflictTableLarger(), IsFalse) + c.Assert(l.allFinalTableSmaller(), IsFalse) + c.Assert(l.allFinalTableLarger(), IsFalse) + // reset + l.finalTables[source][schema][table1] = t0 + l.finalTables[source][schema][table2] = t0 + l.resolveTables() + c.Assert(l.conflictTables, HasLen, 0) + c.Assert(l.tables, DeepEquals, l.finalTables) + c.Assert(l.tables[source][schema], HasLen, 2) + c.Assert(l.tables[source][schema][table1], DeepEquals, t0) + c.Assert(l.tables[source][schema][table2], DeepEquals, t0) + + // one table rename, one table add and drop + l.addConflictTable(source, schema, table1, t1) + l.finalTables[source][schema][table1] = t1 + c.Assert(l.allConflictTableSmaller(), IsTrue) + c.Assert(l.allConflictTableLarger(), IsTrue) + c.Assert(l.allFinalTableSmaller(), IsFalse) + c.Assert(l.allFinalTableLarger(), IsFalse) + l.finalTables[source][schema][table2] = t6 + c.Assert(l.allConflictTableSmaller(), IsTrue) + c.Assert(l.allConflictTableLarger(), IsTrue) + c.Assert(l.allFinalTableSmaller(), IsFalse) + c.Assert(l.allFinalTableLarger(), IsTrue) + l.finalTables[source][schema][table2] = t1 + c.Assert(l.allConflictTableSmaller(), IsTrue) + c.Assert(l.allConflictTableLarger(), IsTrue) + c.Assert(l.allFinalTableSmaller(), IsTrue) + c.Assert(l.allFinalTableLarger(), IsTrue) + // reset + l.finalTables[source][schema][table1] = t0 + l.finalTables[source][schema][table2] = t0 + l.resolveTables() + c.Assert(l.conflictTables, HasLen, 0) + c.Assert(l.tables, DeepEquals, l.finalTables) + c.Assert(l.tables[source][schema], HasLen, 2) + c.Assert(l.tables[source][schema][table1], DeepEquals, t0) + c.Assert(l.tables[source][schema][table2], DeepEquals, t0) + + // one table modify, one table add and drop + l.addConflictTable(source, schema, table1, t2) + l.finalTables[source][schema][table1] = t2 + c.Assert(l.allConflictTableSmaller(), IsTrue) + c.Assert(l.allConflictTableLarger(), IsTrue) + c.Assert(l.allFinalTableSmaller(), IsFalse) + c.Assert(l.allFinalTableLarger(), IsFalse) + l.finalTables[source][schema][table2] = t7 + c.Assert(l.allConflictTableSmaller(), IsTrue) + c.Assert(l.allConflictTableLarger(), IsTrue) + c.Assert(l.allFinalTableSmaller(), IsFalse) + c.Assert(l.allFinalTableLarger(), IsTrue) + l.finalTables[source][schema][table2] = t2 + c.Assert(l.allConflictTableSmaller(), IsTrue) + c.Assert(l.allConflictTableLarger(), IsTrue) + c.Assert(l.allFinalTableSmaller(), IsTrue) + c.Assert(l.allFinalTableLarger(), IsTrue) + // reset + l.finalTables[source][schema][table1] = t0 + l.finalTables[source][schema][table2] = t0 + l.resolveTables() + c.Assert(l.conflictTables, HasLen, 0) + c.Assert(l.tables, DeepEquals, l.finalTables) + c.Assert(l.tables[source][schema], HasLen, 2) + c.Assert(l.tables[source][schema][table1], DeepEquals, t0) + c.Assert(l.tables[source][schema][table2], DeepEquals, t0) + + // not null no default + l.addConflictTable(source, schema, table1, t8) + l.finalTables[source][schema][table1] = t8 + c.Assert(l.allConflictTableSmaller(), IsTrue) + c.Assert(l.allConflictTableLarger(), IsTrue) + c.Assert(l.allFinalTableSmaller(), IsFalse) + c.Assert(l.allFinalTableLarger(), IsFalse) + l.addConflictTable(source, schema, table2, t8) + l.finalTables[source][schema][table2] = t8 + c.Assert(l.allConflictTableSmaller(), IsTrue) + c.Assert(l.allConflictTableLarger(), IsTrue) + c.Assert(l.allFinalTableSmaller(), IsTrue) + c.Assert(l.allFinalTableLarger(), IsTrue) + // reset + l.finalTables[source][schema][table1] = t0 + l.finalTables[source][schema][table2] = t0 + l.resolveTables() + c.Assert(l.conflictTables, HasLen, 0) + c.Assert(l.tables, DeepEquals, l.finalTables) + c.Assert(l.tables[source][schema], HasLen, 2) + c.Assert(l.tables[source][schema][table1], DeepEquals, t0) + c.Assert(l.tables[source][schema][table2], DeepEquals, t0) + + // multiple rename + // tb1: rename col to new_col + l.addConflictTable(source, schema, table1, t1) + l.finalTables[source][schema][table1] = t1 + c.Assert(l.allConflictTableSmaller(), IsTrue) + c.Assert(l.allConflictTableLarger(), IsTrue) + c.Assert(l.allFinalTableSmaller(), IsFalse) + c.Assert(l.allFinalTableLarger(), IsFalse) + // tb2: rename col to new_col + l.addConflictTable(source, schema, table2, t1) + l.finalTables[source][schema][table2] = t1 + c.Assert(l.allConflictTableSmaller(), IsTrue) + c.Assert(l.allConflictTableLarger(), IsTrue) + c.Assert(l.allFinalTableSmaller(), IsTrue) + c.Assert(l.allFinalTableLarger(), IsTrue) + l.resolveTables() + // tb1: rename id to a + l.addConflictTable(source, schema, table1, t9) + l.finalTables[source][schema][table1] = t9 + c.Assert(l.noConflictWithNormalTables(source, schema, table1, t1), IsFalse) + c.Assert(l.allConflictTableSmaller(), IsTrue) + c.Assert(l.allConflictTableLarger(), IsTrue) + c.Assert(l.allFinalTableSmaller(), IsFalse) + c.Assert(l.allFinalTableLarger(), IsFalse) + // tb2: rename col to new_col (idempotent) + l.tables[source][schema][table2] = t0 + l.addConflictTable(source, schema, table2, t1) + l.finalTables[source][schema][table2] = t1 + c.Assert(l.noConflictWithNormalTables(source, schema, table2, t1), IsTrue) + l.removeConflictTable(source, schema, table2) + l.tables[source][schema][table2] = t1 + // tb2: rename id to a + l.addConflictTable(source, schema, table2, t9) + l.finalTables[source][schema][table2] = t9 + c.Assert(l.noConflictWithNormalTables(source, schema, table2, t9), IsFalse) + c.Assert(l.allConflictTableSmaller(), IsTrue) + c.Assert(l.allConflictTableLarger(), IsTrue) + c.Assert(l.allFinalTableSmaller(), IsTrue) + c.Assert(l.allFinalTableLarger(), IsTrue) + // reset + l.finalTables[source][schema][table1] = t0 + l.finalTables[source][schema][table2] = t0 + l.resolveTables() + c.Assert(l.conflictTables, HasLen, 0) + c.Assert(l.tables, DeepEquals, l.finalTables) + c.Assert(l.tables[source][schema], HasLen, 2) + c.Assert(l.tables[source][schema][table1], DeepEquals, t0) + c.Assert(l.tables[source][schema][table2], DeepEquals, t0) +} + +func checkRedirectOp(c *C, task, source, schema, table string) bool { + ops, _, err := GetAllOperations(etcdTestCli) + c.Assert(err, IsNil) + if _, ok := ops[task]; !ok { + return false + } + if _, ok := ops[task][source]; !ok { + return false + } + if _, ok := ops[task][source][schema]; !ok { + return false + } + op, ok := ops[task][source][schema][table] + if !ok { + return false + } + return op.ConflictStage == ConflictResolved +} + +func (t *testLock) TestTrySyncForOneDDL(c *C) { + var ( + ID = "test-`foo`.`bar`" + task = "test" + source = "source" + schema = "schema" + downSchema = "downSchema" + downTable = "downTable" + table1 = "table1" + table2 = "table2" + p = parser.New() + se = mock.NewContext() + tblID int64 = 111 + ti0 = createTableInfo(c, p, se, tblID, `CREATE TABLE bar (id INT PRIMARY KEY, col1 int)`) + ti1 = createTableInfo(c, p, se, tblID, `CREATE TABLE bar (id INT PRIMARY KEY, col1 int, col2 int)`) + ti2 = createTableInfo(c, p, se, tblID, `CREATE TABLE bar (id INT PRIMARY KEY)`) + ti3 = createTableInfo(c, p, se, tblID, `CREATE TABLE bar (id INT PRIMARY KEY, col1 int, col3 int)`) + ti4 = createTableInfo(c, p, se, tblID, `CREATE TABLE bar (id INT PRIMARY KEY, col2 int)`) + ti5 = createTableInfo(c, p, se, tblID, `CREATE TABLE bar (id INT PRIMARY KEY, col3 int)`) + ti6 = createTableInfo(c, p, se, tblID, `CREATE TABLE bar (id INT PRIMARY KEY, col3 varchar(4))`) + ti7 = createTableInfo(c, p, se, tblID, `CREATE TABLE bar (id INT PRIMARY KEY, col3 int)`) + ti8 = createTableInfo(c, p, se, tblID, `CREATE TABLE bar (id INT PRIMARY KEY, col3 varchar(4), col4 int not null)`) + ti9 = createTableInfo(c, p, se, tblID, `CREATE TABLE bar (id INT PRIMARY KEY, col2 varchar(4), col4 int not null)`) + ti10 = createTableInfo(c, p, se, tblID, `CREATE TABLE bar (id INT PRIMARY KEY, col3 int, col4 int not null)`) + t0 = schemacmp.Encode(ti0) + t1 = schemacmp.Encode(ti1) + t2 = schemacmp.Encode(ti2) + t3 = schemacmp.Encode(ti3) + t4 = schemacmp.Encode(ti4) + t5 = schemacmp.Encode(ti5) + t6 = schemacmp.Encode(ti6) + t7 = schemacmp.Encode(ti7) + t8 = schemacmp.Encode(ti8) + t9 = schemacmp.Encode(ti9) + t10 = schemacmp.Encode(ti10) + tables = map[string]map[string]struct{}{ + schema: {table1: struct{}{}, table2: struct{}{}}, + } + tts = []TargetTable{ + newTargetTable(task, source, downSchema, downTable, tables), + } + l = NewLock(etcdTestCli, ID, task, downSchema, downTable, t0, tts, nil) + ) + + // check create table statement + schemaChanged, conflictStage := l.trySyncForOneDDL(source, schema, table1, t0, t0) + c.Assert(schemaChanged, IsTrue) + c.Assert(conflictStage, Equals, ConflictNone) + + // check alter table add column + schemaChanged, conflictStage = l.trySyncForOneDDL(source, schema, table1, t0, t1) + c.Assert(schemaChanged, IsTrue) + c.Assert(conflictStage, Equals, ConflictNone) + + // check alter table drop column + schemaChanged, conflictStage = l.trySyncForOneDDL(source, schema, table2, t0, t2) + c.Assert(schemaChanged, IsFalse) + c.Assert(conflictStage, Equals, ConflictNone) + + // check table rename column + schemaChanged, conflictStage = l.trySyncForOneDDL(source, schema, table1, t1, t3) + c.Assert(schemaChanged, IsFalse) + c.Assert(conflictStage, Equals, ConflictSkipWaitRedirect) + + // check other table add column + schemaChanged, conflictStage = l.trySyncForOneDDL(source, schema, table2, t2, t4) + c.Assert(schemaChanged, IsTrue) + c.Assert(conflictStage, Equals, ConflictNone) + + // check all table rename column + schemaChanged, conflictStage = l.trySyncForOneDDL(source, schema, table2, t4, t5) + c.Assert(schemaChanged, IsTrue) + c.Assert(conflictStage, Equals, ConflictNone) + // table1 redirect + c.Assert(checkRedirectOp(c, task, source, schema, table1), IsTrue) + + // check one table modify column + schemaChanged, conflictStage = l.trySyncForOneDDL(source, schema, table2, t5, t6) + c.Assert(schemaChanged, IsFalse) + c.Assert(conflictStage, Equals, ConflictSkipWaitRedirect) + + // check other table drop column + schemaChanged, conflictStage = l.trySyncForOneDDL(source, schema, table1, t3, t7) + c.Assert(schemaChanged, IsTrue) + c.Assert(conflictStage, Equals, ConflictNone) + + // check all table modify column + schemaChanged, conflictStage = l.trySyncForOneDDL(source, schema, table1, t7, t6) + c.Assert(schemaChanged, IsTrue) + c.Assert(conflictStage, Equals, ConflictNone) + // table2 redirect + c.Assert(checkRedirectOp(c, task, source, schema, table2), IsTrue) + + // check add column not null no default + schemaChanged, conflictStage = l.trySyncForOneDDL(source, schema, table1, t6, t8) + c.Assert(schemaChanged, IsFalse) + c.Assert(conflictStage, Equals, ConflictSkipWaitRedirect) + // check idempotent. + schemaChanged, conflictStage = l.trySyncForOneDDL(source, schema, table1, t6, t8) + c.Assert(schemaChanged, IsFalse) + c.Assert(conflictStage, Equals, ConflictSkipWaitRedirect) + + schemaChanged, conflictStage = l.trySyncForOneDDL(source, schema, table2, t6, t8) + c.Assert(schemaChanged, IsTrue) + c.Assert(conflictStage, Equals, ConflictNone) + // table1 redirect + c.Assert(checkRedirectOp(c, task, source, schema, table2), IsTrue) + // check idempotent. + schemaChanged, conflictStage = l.trySyncForOneDDL(source, schema, table2, t6, t8) + c.Assert(schemaChanged, IsTrue) + c.Assert(conflictStage, Equals, ConflictNone) + + // check multiple conflict DDL + // tb1 rename column + schemaChanged, conflictStage = l.trySyncForOneDDL(source, schema, table1, t8, t9) + c.Assert(schemaChanged, IsFalse) + c.Assert(conflictStage, Equals, ConflictSkipWaitRedirect) + // tb2 modify column + schemaChanged, conflictStage = l.trySyncForOneDDL(source, schema, table2, t8, t10) + c.Assert(schemaChanged, IsFalse) + c.Assert(conflictStage, Equals, ConflictDetected) +} diff --git a/dm/pkg/shardddl/optimism/operation.go b/dm/pkg/shardddl/optimism/operation.go index 99dcd7dda52..3c6b214c6f9 100644 --- a/dm/pkg/shardddl/optimism/operation.go +++ b/dm/pkg/shardddl/optimism/operation.go @@ -36,9 +36,12 @@ const ( // in this stage, DM-worker should not execute/skip DDL/DML, // but it should still try to find the DDL which can resolve the conflict in the binlog stream. ConflictDetected ConflictStage = "detected" - // ConflictResolved indicates a conflict will be resolved after applied the shard DDL. - // in this stage, DM-worker should replay DML skipped in ConflictDetected to downstream. + // ConflictResolved indicates a conflict DDL be resolved. + // in this stage, DM-worker should redirect to the conflict DDL. ConflictResolved ConflictStage = "resolved" + // ConflictSkipWaitRedirect indicates a conflict hapend and will be skipped and redirected until all tables has no conflict + // in this stage, DM-worker should skip all DML and DDL for the conflict table until redirect. + ConflictSkipWaitRedirect ConflictStage = "skip and wait for redirect" // #nosec ) // Operation represents a shard DDL coordinate operation. diff --git a/dm/pkg/terror/error_list.go b/dm/pkg/terror/error_list.go index 0c28b123d28..42a16631e9b 100644 --- a/dm/pkg/terror/error_list.go +++ b/dm/pkg/terror/error_list.go @@ -191,6 +191,9 @@ const ( // pkg/binlog. codeBinlogNotLogColumn + + // pkg/shardddl/optimism. + codeShardDDLOptimismNeedSkipAndRedirect ) // Config related error code list. @@ -859,6 +862,9 @@ var ( // pkg/binlog. ErrBinlogNotLogColumn = New(codeBinlogNotLogColumn, ClassBinlogOp, ScopeUpstream, LevelHigh, "upstream didn't log enough columns in binlog", "Please check if session `binlog_row_image` variable is not FULL, restart task to the location from where FULL binlog_row_image is used.") + // pkg/shardddl/optimism. + ErrShardDDLOptimismNeedSkipAndRedirect = New(codeShardDDLOptimismNeedSkipAndRedirect, ClassFunctional, ScopeInternal, LevelLow, "receive conflict DDL for the optimistic shard ddl lock %s: %s", "") + // Config related error. ErrConfigCheckItemNotSupport = New(codeConfigCheckItemNotSupport, ClassConfig, ScopeInternal, LevelMedium, "checking item %s is not supported\n%s", "Please check `ignore-checking-items` config in task configuration file, which can be set including `all`/`dump_privilege`/`replication_privilege`/`version`/`binlog_enable`/`binlog_format`/`binlog_row_image`/`table_schema`/`schema_of_shard_tables`/`auto_increment_ID`.") ErrConfigTomlTransform = New(codeConfigTomlTransform, ClassConfig, ScopeInternal, LevelMedium, "%s", "Please check the configuration file has correct TOML format.") diff --git a/dm/syncer/optimist.go b/dm/syncer/optimist.go index 575dcdb5473..a1b018e17b0 100644 --- a/dm/syncer/optimist.go +++ b/dm/syncer/optimist.go @@ -176,6 +176,12 @@ func (s *Syncer) handleQueryEventOptimistic(qec *queryEventContext) error { return terror.ErrSyncerShardDDLConflict.Generate(qec.needHandleDDLs, op.ConflictMsg) } + // TODO: support redirect for DM worker + // return error to pass IT now + if op.ConflictStage == optimism.ConflictSkipWaitRedirect { + return terror.ErrSyncerShardDDLConflict.Generate(qec.needHandleDDLs, "there will be conflicts if DDLs .* are applied to the downstream. old table info: .*, new table info: .*") + } + // updated needHandleDDLs to DDLs received from DM-master. qec.needHandleDDLs = op.DDLs diff --git a/dm/tests/shardddl4_1/run.sh b/dm/tests/shardddl4_1/run.sh index 1f350740ba7..d93a025c10d 100644 --- a/dm/tests/shardddl4_1/run.sh +++ b/dm/tests/shardddl4_1/run.sh @@ -735,12 +735,9 @@ function DM_151_CASE { run_sql_source2 "alter table ${shardddl1}.${tb2} modify column a double;" check_sync_diff $WORK_DIR $cur/conf/diff_config.toml else - # ddl: "modify column a double" is passed in optimistic mode and will be executed downstream. - # but changing the int column to a double column is not allowed, so task is paused run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" \ "query-status test" \ - '"stage": "Paused"' 1 \ - "incompatible mysql type" 1 + "because schema conflict detected" 1 fi } From eeacb25f11b3cbfcf953d903f604918326a875d4 Mon Sep 17 00:00:00 2001 From: lance6716 Date: Mon, 7 Mar 2022 16:05:49 +0800 Subject: [PATCH 4/7] *(both): update dep of etcd, go-mysql and others (#4755) ref pingcap/tiflow#4418 --- cdc/api/owner.go | 7 +- cdc/api/owner_test.go | 2 +- cdc/api/status.go | 2 +- cdc/capture/capture.go | 13 +- cdc/processor/agent.go | 7 +- cdc/processor/agent_test.go | 9 +- cdc/redo/writer/file.go | 7 +- cdc/server.go | 19 +- cdc/server_test.go | 11 +- dm/dm/config/source_config.go | 26 +-- dm/dm/config/source_config_test.go | 17 +- dm/dm/config/source_converter_test.go | 5 +- dm/dm/config/subtask.go | 5 + dm/dm/config/subtask.toml | 114 ++++++++++++ dm/dm/config/task_converters_test.go | 12 +- dm/dm/ctl/common/util.go | 2 +- dm/dm/ctl/master/config.go | 2 +- dm/dm/ctl/master/operate_source.go | 2 +- dm/dm/master/bootstrap_test.go | 11 +- dm/dm/master/config.go | 25 ++- dm/dm/master/config_test.go | 55 ++---- dm/dm/master/election_test.go | 4 +- dm/dm/master/etcd.go | 2 +- dm/dm/master/etcd_test.go | 8 +- dm/dm/master/openapi_view_test.go | 23 +-- dm/dm/master/scheduler/scheduler.go | 2 +- dm/dm/master/scheduler/scheduler_test.go | 34 ++-- dm/dm/master/server.go | 6 +- dm/dm/master/server_test.go | 98 ++++++---- dm/dm/master/shardddl/optimist.go | 2 +- dm/dm/master/shardddl/optimist_test.go | 140 +++++++------- dm/dm/master/shardddl/pessimist.go | 2 +- dm/dm/master/shardddl/pessimist_test.go | 172 ++++++++--------- dm/dm/worker/server.go | 2 +- dm/dm/worker/server_test.go | 15 +- dm/dm/worker/source_worker.go | 2 +- dm/dm/worker/source_worker_test.go | 10 +- dm/dm/worker/subtask.go | 2 +- dm/dm/worker/subtask_test.go | 2 +- dm/dm/worker/utils.go | 2 +- dm/dm/worker/utils_test.go | 5 +- dm/loader/lightning.go | 2 +- dm/loader/loader.go | 2 +- dm/loader/util.go | 2 +- dm/pkg/election/election.go | 6 +- dm/pkg/election/election_test.go | 4 +- dm/pkg/etcdutil/etcdutil.go | 4 +- dm/pkg/etcdutil/etcdutil_test.go | 13 +- dm/pkg/ha/bound.go | 4 +- dm/pkg/ha/keepalive.go | 4 +- dm/pkg/ha/load_task.go | 4 +- dm/pkg/ha/openapi_task_config.go | 4 +- dm/pkg/ha/ops.go | 4 +- dm/pkg/ha/ops_test.go | 2 +- dm/pkg/ha/relay.go | 4 +- dm/pkg/ha/source.go | 2 +- dm/pkg/ha/source_test.go | 5 +- dm/pkg/ha/stage.go | 4 +- dm/pkg/ha/stage_test.go | 2 +- dm/pkg/ha/subtask.go | 2 +- dm/pkg/ha/subtask_test.go | 7 +- dm/pkg/ha/task_cli_args.go | 2 +- dm/pkg/ha/worker.go | 2 +- dm/pkg/shardddl/optimism/column.go | 2 +- dm/pkg/shardddl/optimism/info.go | 4 +- dm/pkg/shardddl/optimism/info_test.go | 5 +- dm/pkg/shardddl/optimism/keeper.go | 2 +- dm/pkg/shardddl/optimism/keeper_test.go | 3 +- dm/pkg/shardddl/optimism/lock.go | 2 +- dm/pkg/shardddl/optimism/lock_test.go | 3 +- dm/pkg/shardddl/optimism/operation.go | 6 +- dm/pkg/shardddl/optimism/ops.go | 2 +- dm/pkg/shardddl/optimism/table.go | 4 +- dm/pkg/shardddl/pessimism/info.go | 6 +- dm/pkg/shardddl/pessimism/info_test.go | 5 +- dm/pkg/shardddl/pessimism/operation.go | 6 +- dm/pkg/shardddl/pessimism/operation_test.go | 4 +- dm/pkg/shardddl/pessimism/ops.go | 2 +- dm/pkg/upgrade/upgrade.go | 2 +- dm/pkg/upgrade/upgrade_test.go | 54 ++++-- dm/pkg/upgrade/version.go | 2 +- dm/syncer/safe_mode_test.go | 8 +- dm/syncer/shardddl/optimist.go | 2 +- dm/syncer/shardddl/pessimist.go | 2 +- dm/syncer/shardddl/pessimist_test.go | 5 +- dm/syncer/syncer.go | 2 +- dm/tests/_utils/run_dm_master_info_log | 28 +++ dm/tests/adjust_gtid/data/db2.increment.sql | 5 +- dm/tests/all_mode/run.sh | 3 +- dm/tests/lightning_mode/run.sh | 2 +- dm/tests/tls/run.sh | 3 +- go.mod | 44 ++--- go.sum | 183 ++++++++++++++----- pkg/cmd/cli/cli_capture_list.go | 5 +- pkg/cmd/factory/factory_impl.go | 13 +- pkg/errorutil/ignore.go | 2 +- pkg/errorutil/ignore_test.go | 2 +- pkg/etcd/client.go | 11 +- pkg/etcd/client_test.go | 4 +- pkg/etcd/etcd.go | 18 +- pkg/etcd/etcd_test.go | 20 +- pkg/etcd/util.go | 2 +- pkg/etcd/util_test.go | 2 +- pkg/leakutil/leak_helper.go | 2 +- pkg/orchestrator/etcd_worker.go | 17 +- pkg/orchestrator/etcd_worker_test.go | 11 +- pkg/util/testleak/leaktest.go | 2 +- tests/integration_tests/move_table/main.go | 13 +- tests/utils/cdc_state_checker/cdc_monitor.go | 11 +- 109 files changed, 906 insertions(+), 601 deletions(-) create mode 100644 dm/dm/config/subtask.toml create mode 100755 dm/tests/_utils/run_dm_master_info_log diff --git a/cdc/api/owner.go b/cdc/api/owner.go index 9777b0a6098..c11a9cd10df 100644 --- a/cdc/api/owner.go +++ b/cdc/api/owner.go @@ -24,13 +24,14 @@ import ( "github.com/gin-gonic/gin" "github.com/pingcap/errors" "github.com/pingcap/log" + "github.com/tikv/client-go/v2/oracle" + "go.etcd.io/etcd/client/v3/concurrency" + "go.uber.org/zap" + "github.com/pingcap/tiflow/cdc/capture" "github.com/pingcap/tiflow/cdc/model" cerror "github.com/pingcap/tiflow/pkg/errors" "github.com/pingcap/tiflow/pkg/logutil" - "github.com/tikv/client-go/v2/oracle" - "go.etcd.io/etcd/clientv3/concurrency" - "go.uber.org/zap" ) const ( diff --git a/cdc/api/owner_test.go b/cdc/api/owner_test.go index bd09d0ec7cc..6be9226407b 100644 --- a/cdc/api/owner_test.go +++ b/cdc/api/owner_test.go @@ -23,7 +23,7 @@ import ( "github.com/gin-gonic/gin" "github.com/stretchr/testify/require" - "go.etcd.io/etcd/clientv3/concurrency" + "go.etcd.io/etcd/client/v3/concurrency" ) func TestHTTPStatus(t *testing.T) { diff --git a/cdc/api/status.go b/cdc/api/status.go index e71c1528c03..79d9da7537d 100644 --- a/cdc/api/status.go +++ b/cdc/api/status.go @@ -24,7 +24,7 @@ import ( "github.com/pingcap/tiflow/cdc/capture" "github.com/pingcap/tiflow/pkg/etcd" "github.com/pingcap/tiflow/pkg/version" - "go.etcd.io/etcd/clientv3" + clientv3 "go.etcd.io/etcd/client/v3" ) // status of cdc server diff --git a/cdc/capture/capture.go b/cdc/capture/capture.go index 8154531ed67..535d59e7185 100644 --- a/cdc/capture/capture.go +++ b/cdc/capture/capture.go @@ -25,6 +25,13 @@ import ( "github.com/pingcap/failpoint" "github.com/pingcap/log" tidbkv "github.com/pingcap/tidb/kv" + "github.com/tikv/client-go/v2/tikv" + pd "github.com/tikv/pd/client" + "go.etcd.io/etcd/client/v3/concurrency" + "go.etcd.io/etcd/server/v3/mvcc" + "go.uber.org/zap" + "golang.org/x/time/rate" + "github.com/pingcap/tiflow/cdc/kv" "github.com/pingcap/tiflow/cdc/model" "github.com/pingcap/tiflow/cdc/owner" @@ -39,12 +46,6 @@ import ( "github.com/pingcap/tiflow/pkg/p2p" "github.com/pingcap/tiflow/pkg/pdtime" "github.com/pingcap/tiflow/pkg/version" - "github.com/tikv/client-go/v2/tikv" - pd "github.com/tikv/pd/client" - "go.etcd.io/etcd/clientv3/concurrency" - "go.etcd.io/etcd/mvcc" - "go.uber.org/zap" - "golang.org/x/time/rate" ) // Capture represents a Capture server, it monitors the changefeed information in etcd and schedules Task on it. diff --git a/cdc/processor/agent.go b/cdc/processor/agent.go index 406f3779fd2..a73312a207d 100644 --- a/cdc/processor/agent.go +++ b/cdc/processor/agent.go @@ -22,6 +22,10 @@ import ( "github.com/benbjohnson/clock" "github.com/pingcap/errors" "github.com/pingcap/log" + "go.etcd.io/etcd/client/v3/concurrency" + "go.uber.org/zap" + "golang.org/x/time/rate" + "github.com/pingcap/tiflow/cdc/model" "github.com/pingcap/tiflow/cdc/scheduler" "github.com/pingcap/tiflow/pkg/config" @@ -30,9 +34,6 @@ import ( "github.com/pingcap/tiflow/pkg/etcd" "github.com/pingcap/tiflow/pkg/p2p" "github.com/pingcap/tiflow/pkg/version" - "go.etcd.io/etcd/clientv3/concurrency" - "go.uber.org/zap" - "golang.org/x/time/rate" ) const ( diff --git a/cdc/processor/agent_test.go b/cdc/processor/agent_test.go index 9037e55039c..ea12a4b91a0 100644 --- a/cdc/processor/agent_test.go +++ b/cdc/processor/agent_test.go @@ -19,16 +19,17 @@ import ( "time" "github.com/pingcap/failpoint" + "github.com/stretchr/testify/mock" + "github.com/stretchr/testify/require" + "go.etcd.io/etcd/api/v3/mvccpb" + clientv3 "go.etcd.io/etcd/client/v3" + "github.com/pingcap/tiflow/cdc/model" pscheduler "github.com/pingcap/tiflow/cdc/scheduler" cdcContext "github.com/pingcap/tiflow/pkg/context" "github.com/pingcap/tiflow/pkg/etcd" "github.com/pingcap/tiflow/pkg/p2p" "github.com/pingcap/tiflow/pkg/version" - "github.com/stretchr/testify/mock" - "github.com/stretchr/testify/require" - "go.etcd.io/etcd/clientv3" - "go.etcd.io/etcd/mvcc/mvccpb" ) const ( diff --git a/cdc/redo/writer/file.go b/cdc/redo/writer/file.go index fb4e78953e9..2e263e357a7 100644 --- a/cdc/redo/writer/file.go +++ b/cdc/redo/writer/file.go @@ -29,13 +29,14 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/log" "github.com/pingcap/tidb/br/pkg/storage" - "github.com/pingcap/tiflow/cdc/redo/common" - cerror "github.com/pingcap/tiflow/pkg/errors" "github.com/prometheus/client_golang/prometheus" "github.com/uber-go/atomic" - pioutil "go.etcd.io/etcd/pkg/ioutil" + pioutil "go.etcd.io/etcd/pkg/v3/ioutil" "go.uber.org/multierr" "go.uber.org/zap" + + "github.com/pingcap/tiflow/cdc/redo/common" + cerror "github.com/pingcap/tiflow/pkg/errors" ) const ( diff --git a/cdc/server.go b/cdc/server.go index 185ec75d273..a4a6aaaceb0 100644 --- a/cdc/server.go +++ b/cdc/server.go @@ -28,6 +28,16 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/log" tidbkv "github.com/pingcap/tidb/kv" + "github.com/prometheus/client_golang/prometheus" + pd "github.com/tikv/pd/client" + "go.etcd.io/etcd/client/pkg/v3/logutil" + clientv3 "go.etcd.io/etcd/client/v3" + "go.uber.org/zap" + "go.uber.org/zap/zapcore" + "golang.org/x/sync/errgroup" + "google.golang.org/grpc" + "google.golang.org/grpc/backoff" + "github.com/pingcap/tiflow/cdc/capture" "github.com/pingcap/tiflow/cdc/kv" "github.com/pingcap/tiflow/cdc/sorter/unified" @@ -41,15 +51,6 @@ import ( "github.com/pingcap/tiflow/pkg/util" "github.com/pingcap/tiflow/pkg/version" p2pProto "github.com/pingcap/tiflow/proto/p2p" - "github.com/prometheus/client_golang/prometheus" - pd "github.com/tikv/pd/client" - "go.etcd.io/etcd/clientv3" - "go.etcd.io/etcd/pkg/logutil" - "go.uber.org/zap" - "go.uber.org/zap/zapcore" - "golang.org/x/sync/errgroup" - "google.golang.org/grpc" - "google.golang.org/grpc/backoff" ) const ( diff --git a/cdc/server_test.go b/cdc/server_test.go index 784b7f15543..df512965b6d 100644 --- a/cdc/server_test.go +++ b/cdc/server_test.go @@ -29,6 +29,12 @@ import ( "time" "github.com/pingcap/tidb/br/pkg/httputil" + "github.com/stretchr/testify/require" + "github.com/tikv/pd/pkg/tempurl" + clientv3 "go.etcd.io/etcd/client/v3" + "go.etcd.io/etcd/server/v3/embed" + "golang.org/x/sync/errgroup" + "github.com/pingcap/tiflow/cdc/capture" "github.com/pingcap/tiflow/cdc/model" "github.com/pingcap/tiflow/pkg/config" @@ -37,11 +43,6 @@ import ( "github.com/pingcap/tiflow/pkg/retry" security2 "github.com/pingcap/tiflow/pkg/security" "github.com/pingcap/tiflow/pkg/util" - "github.com/stretchr/testify/require" - "github.com/tikv/pd/pkg/tempurl" - "go.etcd.io/etcd/clientv3" - "go.etcd.io/etcd/embed" - "golang.org/x/sync/errgroup" ) type testServer struct { diff --git a/dm/dm/config/source_config.go b/dm/dm/config/source_config.go index f8cd9870375..27d60ae2bdf 100644 --- a/dm/dm/config/source_config.go +++ b/dm/dm/config/source_config.go @@ -47,11 +47,11 @@ const ( var getAllServerIDFunc = utils.GetAllServerID -// SampleConfigFile is sample config file of source. +// SampleSourceConfig is sample config file of source. // The embed source.yaml is a copy of dm/master/source.yaml, because embed // can only match regular files in the current directory and subdirectories. //go:embed source.yaml -var SampleConfigFile string +var SampleSourceConfig string // PurgeConfig is the configuration for Purger. type PurgeConfig struct { @@ -170,6 +170,18 @@ func ParseYaml(content string) (*SourceConfig, error) { return c, nil } +// ParseYamlAndVerify does ParseYaml and Verify. +func ParseYamlAndVerify(content string) (*SourceConfig, error) { + c, err := ParseYaml(content) + if err != nil { + return nil, err + } + if err = c.Verify(); err != nil { + return nil, err + } + return c, nil +} + // EncodeToml encodes config. func (c *SourceConfig) EncodeToml() (string, error) { buf := new(bytes.Buffer) @@ -348,19 +360,11 @@ func (c *SourceConfig) AdjustServerID(ctx context.Context, db *sql.DB) error { // LoadFromFile loads config from file. func LoadFromFile(path string) (*SourceConfig, error) { - c := newSourceConfig() content, err := os.ReadFile(path) if err != nil { return nil, terror.ErrConfigReadCfgFromFile.Delegate(err, path) } - if err = yaml.UnmarshalStrict(content, c); err != nil { - return nil, terror.ErrConfigYamlTransform.Delegate(err, "decode source config") - } - c.adjust() - if err = c.Verify(); err != nil { - return nil, err - } - return c, nil + return ParseYaml(string(content)) } func (c *SourceConfig) check(metaData *toml.MetaData, err error) error { diff --git a/dm/dm/config/source_config_test.go b/dm/dm/config/source_config_test.go index 248c51d04de..7ea1a641bb3 100644 --- a/dm/dm/config/source_config_test.go +++ b/dm/dm/config/source_config_test.go @@ -31,11 +31,8 @@ import ( "github.com/pingcap/tiflow/dm/pkg/utils" ) -// do not forget to update this path if the file removed/renamed. -const sourceSampleFile = "../worker/source.yaml" - func (t *testConfig) TestConfig(c *C) { - cfg, err := LoadFromFile(sourceSampleFile) + cfg, err := ParseYaml(SampleSourceConfig) c.Assert(err, IsNil) cfg.RelayDir = "./xx" c.Assert(cfg.RelayDir, Equals, "./xx") @@ -134,7 +131,7 @@ aaa: xxx func (t *testConfig) TestConfigVerify(c *C) { newConfig := func() *SourceConfig { - cfg, err := LoadFromFile(sourceSampleFile) + cfg, err := ParseYaml(SampleSourceConfig) c.Assert(err, IsNil) cfg.RelayDir = "./xx" return cfg @@ -240,7 +237,7 @@ func (t *testConfig) TestConfigVerify(c *C) { } func (t *testConfig) TestSourceConfigForDowngrade(c *C) { - cfg, err := LoadFromFile(sourceSampleFile) + cfg, err := ParseYaml(SampleSourceConfig) c.Assert(err, IsNil) // make sure all new field were added @@ -274,7 +271,7 @@ func subtestFlavor(c *C, cfg *SourceConfig, sqlInfo, expectedFlavor, expectedErr } func (t *testConfig) TestAdjustFlavor(c *C) { - cfg, err := LoadFromFile(sourceSampleFile) + cfg, err := ParseYaml(SampleSourceConfig) c.Assert(err, IsNil) cfg.RelayDir = "./xx" @@ -297,7 +294,7 @@ func (t *testConfig) TestAdjustServerID(c *C) { }() getAllServerIDFunc = getMockServerIDs - cfg, err := LoadFromFile(sourceSampleFile) + cfg, err := ParseYaml(SampleSourceConfig) c.Assert(err, IsNil) cfg.RelayDir = "./xx" @@ -317,7 +314,7 @@ func getMockServerIDs(ctx context.Context, db *sql.DB) (map[uint32]struct{}, err } func (t *testConfig) TestAdjustCaseSensitive(c *C) { - cfg, err := LoadFromFile(sourceSampleFile) + cfg, err := ParseYaml(SampleSourceConfig) c.Assert(err, IsNil) db, mock, err := sqlmock.New() @@ -339,5 +336,5 @@ func (t *testConfig) TestAdjustCaseSensitive(c *C) { func (t *testConfig) TestEmbedSampleFile(c *C) { data, err := os.ReadFile("./source.yaml") c.Assert(err, IsNil) - c.Assert(SampleConfigFile, Equals, string(data)) + c.Assert(SampleSourceConfig, Equals, string(data)) } diff --git a/dm/dm/config/source_converter_test.go b/dm/dm/config/source_converter_test.go index 013880269ae..500ddfc3948 100644 --- a/dm/dm/config/source_converter_test.go +++ b/dm/dm/config/source_converter_test.go @@ -15,11 +15,12 @@ package config import ( "github.com/pingcap/check" + "github.com/pingcap/tiflow/dm/openapi/fixtures" ) func (t *testConfig) TestConverterWithSourceAndOpenAPISource(c *check.C) { - sourceCfg1, err := LoadFromFile(sourceSampleFile) + sourceCfg1, err := ParseYaml(SampleSourceConfig) c.Assert(err, check.IsNil) // 1. test user create source from dmctl, after convert to openapi.Source then convert back to source config @@ -28,6 +29,8 @@ func (t *testConfig) TestConverterWithSourceAndOpenAPISource(c *check.C) { // we need set ServerID and MaxAllowedPacket manually, because user don't need to config those field in openapi sourceCfg2.ServerID = sourceCfg1.ServerID sourceCfg2.From.MaxAllowedPacket = sourceCfg1.From.MaxAllowedPacket + // TODO: OpenAPI source config will add flavor + sourceCfg1.Flavor = "" // we only need to make sure the source config that user can see is the same as the source config that user create c.Assert(sourceCfg1.String(), check.Equals, sourceCfg2.String()) diff --git a/dm/dm/config/subtask.go b/dm/dm/config/subtask.go index 23347cbc580..e57664c7b0d 100644 --- a/dm/dm/config/subtask.go +++ b/dm/dm/config/subtask.go @@ -15,6 +15,7 @@ package config import ( "bytes" + _ "embed" "encoding/json" "flag" "fmt" @@ -270,6 +271,10 @@ type SubTaskConfig struct { } `yaml:"experimental" toml:"experimental" json:"experimental"` } +// SampleSubtaskConfig is the content of subtask.toml in current folder. +//go:embed subtask.toml +var SampleSubtaskConfig string + // NewSubTaskConfig creates a new SubTaskConfig. func NewSubTaskConfig() *SubTaskConfig { cfg := &SubTaskConfig{} diff --git a/dm/dm/config/subtask.toml b/dm/dm/config/subtask.toml new file mode 100644 index 00000000000..b46f8f0aa4a --- /dev/null +++ b/dm/dm/config/subtask.toml @@ -0,0 +1,114 @@ +# Base Configuration. + +# sub task name as identifier, should be unique +name = "sub-task-name" + +# sub task mode, can be `all`, `full`, `increment` +mode = "all" + +# used for binlog replication +server-id = 101 +source-id = "mysql-replica-01" + +# which flavor mysql/mariadb +flavor = "mysql" + +# meta schema in downstreaming database to store meta informaton of dm +meta-schema = "dm_meta" +# whether to disable heartbeat for calculating lag between master and syncer +enable-heartbeat = false + +# replicate from relay log or remote binlog +binlog-type = "local" + +# Mydumper configuration + +# -t, --threads +#threads = 16 + +# -F, --chunk-filesize +#chunk-filesize = 64 + +# --skip-tz-utc +#skip-tz-utc = true + +# other extra args +#extra-args = "-B test -T t1,t2 --no-locks" + + +# Loader configuration + +# Number of threads restoring concurrently for loader worker pool. Each worker restore one file at a time, increase this as TiKV nodes increase +pool-size = 16 + +# Directory of the dump to import +dir = "./dumped_data" + + +# Syncer configuration + +# different sub task should have different meta +meta-file = "./syncer.subTaskA.meta" + +worker-count = 16 +batch = 1000 + +# filter + +# block allow list provides a library to filter replicate on schema/table by given rules +# support regular expression , start with '~' declare use regular expression. +# rules like replication rules in MySQL, ref document: +# https://dev.mysql.com/doc/refman/8.0/en/replication-rules-db-options.html +# https://dev.mysql.com/doc/refman/8.0/en/replication-rules-table-options.html +#[block-allow-list] +#do-dbs = ["~^b.*","s1"] +# +#ignore-dbs = ["~^b.*","s1"] +# +#[[block-allow-list.do-tables]] +#db-name = "your db" +#tbl-name = "your table" +# +#[[block-allow-list.ignore-tables]] +#db-name = "~^b.*" +#tbl-name = "~^a.*" + + +# filter-rules filter binlog events +#[[filter-rules]] +#schema-pattern = "shard_db_*" +#table-pattern = "shard_table_*" +# do all dml, ignore all ddl +#events = ["all dml"] +#action = "Do" + +#[[filter-rules]] +#schema-pattern = "shard_db_*" +#table-pattern = "shard_table_*" +# ignore delete sqls and drop table ddls +#events = ["delete", "drop table"] +# regular expression, ignore drop procedure queries +# sql-pattern = ["^DROP\\s+PROCEDURE"] +#action = "Ignore" + + +# route table + +# applied after filter +# our Selector is a two level selector (schemal level, and table level) +# [[route-rules]] +# schema-pattern = "shard_db_*" +# target-schema = "shard_db" +# [[route-rules]] +# schema-pattern = "shard_db_*" +# table-pattern = "shard_table_*" +# target-schema = "shard_db" +# target-table = "shard_table" + +# to: the target db +# from: the source db, not in this file, auto get it from worker's config +[to] +host = "127.0.0.1" +user = "root" +password = "" +port = 4000 diff --git a/dm/dm/config/task_converters_test.go b/dm/dm/config/task_converters_test.go index d8d402e0ff6..56158f219c0 100644 --- a/dm/dm/config/task_converters_test.go +++ b/dm/dm/config/task_converters_test.go @@ -51,7 +51,7 @@ func (t *testConfig) TestOpenAPITaskToSubTaskConfigs(c *check.C) { func testNoShardTaskToSubTaskConfigs(c *check.C) { task, err := fixtures.GenNoShardOpenAPITaskForTest() c.Assert(err, check.IsNil) - sourceCfg1, err := LoadFromFile(sourceSampleFile) + sourceCfg1, err := ParseYamlAndVerify(SampleSourceConfig) c.Assert(err, check.IsNil) source1Name := task.SourceConfig.SourceConf[0].SourceName sourceCfg1.SourceID = task.SourceConfig.SourceConf[0].SourceName @@ -115,11 +115,11 @@ func testNoShardTaskToSubTaskConfigs(c *check.C) { func testShardAndFilterTaskToSubTaskConfigs(c *check.C) { task, err := fixtures.GenShardAndFilterOpenAPITaskForTest() c.Assert(err, check.IsNil) - sourceCfg1, err := LoadFromFile(sourceSampleFile) + sourceCfg1, err := ParseYamlAndVerify(SampleSourceConfig) c.Assert(err, check.IsNil) source1Name := task.SourceConfig.SourceConf[0].SourceName sourceCfg1.SourceID = source1Name - sourceCfg2, err := LoadFromFile(sourceSampleFile) + sourceCfg2, err := ParseYamlAndVerify(SampleSourceConfig) c.Assert(err, check.IsNil) source2Name := task.SourceConfig.SourceConf[1].SourceName sourceCfg2.SourceID = source2Name @@ -256,7 +256,7 @@ func (t *testConfig) TestSubTaskConfigsToOpenAPITask(c *check.C) { func testNoShardSubTaskConfigsToOpenAPITask(c *check.C) { task, err := fixtures.GenNoShardOpenAPITaskForTest() c.Assert(err, check.IsNil) - sourceCfg1, err := LoadFromFile(sourceSampleFile) + sourceCfg1, err := ParseYamlAndVerify(SampleSourceConfig) c.Assert(err, check.IsNil) source1Name := task.SourceConfig.SourceConf[0].SourceName sourceCfg1.SourceID = task.SourceConfig.SourceConf[0].SourceName @@ -286,11 +286,11 @@ func testNoShardSubTaskConfigsToOpenAPITask(c *check.C) { func testShardAndFilterSubTaskConfigsToOpenAPITask(c *check.C) { task, err := fixtures.GenShardAndFilterOpenAPITaskForTest() c.Assert(err, check.IsNil) - sourceCfg1, err := LoadFromFile(sourceSampleFile) + sourceCfg1, err := ParseYamlAndVerify(SampleSourceConfig) c.Assert(err, check.IsNil) source1Name := task.SourceConfig.SourceConf[0].SourceName sourceCfg1.SourceID = source1Name - sourceCfg2, err := LoadFromFile(sourceSampleFile) + sourceCfg2, err := ParseYamlAndVerify(SampleSourceConfig) c.Assert(err, check.IsNil) source2Name := task.SourceConfig.SourceConf[1].SourceName sourceCfg2.SourceID = source2Name diff --git a/dm/dm/ctl/common/util.go b/dm/dm/ctl/common/util.go index b0e37396185..1855bf6e0bc 100644 --- a/dm/dm/ctl/common/util.go +++ b/dm/dm/ctl/common/util.go @@ -32,7 +32,7 @@ import ( toolutils "github.com/pingcap/tidb-tools/pkg/utils" "github.com/pingcap/tidb/parser" "github.com/spf13/cobra" - "go.etcd.io/etcd/clientv3" + clientv3 "go.etcd.io/etcd/client/v3" "go.uber.org/zap" "google.golang.org/grpc" "google.golang.org/grpc/codes" diff --git a/dm/dm/ctl/master/config.go b/dm/dm/ctl/master/config.go index 122b9ea2c1c..e689b3172f1 100644 --- a/dm/dm/ctl/master/config.go +++ b/dm/dm/ctl/master/config.go @@ -23,7 +23,7 @@ import ( "github.com/pingcap/errors" "github.com/spf13/cobra" - "go.etcd.io/etcd/clientv3" + clientv3 "go.etcd.io/etcd/client/v3" "github.com/pingcap/tiflow/dm/dm/config" "github.com/pingcap/tiflow/dm/dm/ctl/common" diff --git a/dm/dm/ctl/master/operate_source.go b/dm/dm/ctl/master/operate_source.go index bba20d1c7b8..2841fec3e7a 100644 --- a/dm/dm/ctl/master/operate_source.go +++ b/dm/dm/ctl/master/operate_source.go @@ -65,7 +65,7 @@ func operateSourceFunc(cmd *cobra.Command, _ []string) error { } if printSampleConfig { - fmt.Println(config.SampleConfigFile) + fmt.Println(config.SampleSourceConfig) return nil } diff --git a/dm/dm/master/bootstrap_test.go b/dm/dm/master/bootstrap_test.go index d4daaec6bd5..fc1a93ee856 100644 --- a/dm/dm/master/bootstrap_test.go +++ b/dm/dm/master/bootstrap_test.go @@ -33,11 +33,6 @@ import ( "github.com/pingcap/tiflow/dm/pkg/terror" ) -const ( - // do not forget to update this path if the file removed/renamed. - subTaskSampleFile = "../worker/subtask.toml" -) - func (t *testMaster) TestCollectSourceConfigFilesV1Import(c *C) { s := testDefaultMasterServer(c) defer s.Close() @@ -67,7 +62,7 @@ func (t *testMaster) TestCollectSourceConfigFilesV1Import(c *C) { } password := os.Getenv("MYSQL_PSWD") - cfg1, err := config.LoadFromFile("./source.yaml") + cfg1, err := config.ParseYaml(config.SampleSourceConfig) c.Assert(err, IsNil) // fix empty map after marshal/unmarshal becomes nil cfg1.From.Adjust() @@ -124,7 +119,7 @@ func (t *testMaster) TestWaitWorkersReadyV1Import(c *C) { s.cfg.V1SourcesPath = c.MkDir() c.Assert(s.scheduler.Start(ctx, t.etcdTestCli), IsNil) - cfg1, err := config.LoadFromFile("./source.yaml") + cfg1, err := config.ParseYaml(config.SampleSourceConfig) c.Assert(err, IsNil) cfg2 := cfg1.Clone() cfg2.SourceID = "mysql-replica-02" @@ -179,7 +174,7 @@ func (t *testMaster) TestSubtaskCfgsStagesV1Import(c *C) { ) cfg11 := config.NewSubTaskConfig() - c.Assert(cfg11.DecodeFile(subTaskSampleFile, true), IsNil) + c.Assert(cfg11.Decode(config.SampleSubtaskConfig, true), IsNil) cfg11.Dir = "./dump_data" cfg11.ChunkFilesize = "64" cfg11.Name = taskName1 diff --git a/dm/dm/master/config.go b/dm/dm/master/config.go index 9dbac15a4b2..479249e13e4 100644 --- a/dm/dm/master/config.go +++ b/dm/dm/master/config.go @@ -26,7 +26,7 @@ import ( "time" "github.com/BurntSushi/toml" - "go.etcd.io/etcd/embed" + "go.etcd.io/etcd/server/v3/embed" "go.uber.org/zap" "github.com/pingcap/tiflow/dm/dm/config" @@ -48,9 +48,9 @@ const ( quotaBackendBytesLowerBound = 500 * 1024 * 1024 // 500MB ) -// SampleConfigFile is sample config file of dm-master. +// SampleConfig is sample config of dm-master. //go:embed dm-master.toml -var SampleConfigFile string +var SampleConfig string // NewConfig creates a config for dm-master. func NewConfig() *Config { @@ -178,7 +178,7 @@ func (c *Config) Parse(arguments []string) error { } if c.printSampleConfig { - fmt.Println(SampleConfigFile) + fmt.Println(SampleConfig) return flag.ErrHelp } @@ -220,6 +220,23 @@ func (c *Config) configFromFile(path string) error { return nil } +// FromContent loads config from TOML format content. +func (c *Config) FromContent(content string) error { + metaData, err := toml.Decode(content, c) + if err != nil { + return terror.ErrMasterConfigTomlTransform.Delegate(err) + } + undecoded := metaData.Undecoded() + if len(undecoded) > 0 { + var undecodedItems []string + for _, item := range undecoded { + undecodedItems = append(undecodedItems, item.String()) + } + return terror.ErrMasterConfigUnknownItem.Generate(strings.Join(undecodedItems, ",")) + } + return c.adjust() +} + // adjust adjusts configs. func (c *Config) adjust() error { c.MasterAddr = utils.UnwrapScheme(c.MasterAddr) diff --git a/dm/dm/master/config_test.go b/dm/dm/master/config_test.go index 7b1f1531e3e..7425bd0edc5 100644 --- a/dm/dm/master/config_test.go +++ b/dm/dm/master/config_test.go @@ -23,16 +23,13 @@ import ( capturer "github.com/kami-zh/go-capturer" "github.com/pingcap/check" - "go.etcd.io/etcd/embed" + "go.etcd.io/etcd/server/v3/embed" "github.com/pingcap/tiflow/dm/pkg/log" "github.com/pingcap/tiflow/dm/pkg/terror" ) -var ( - defaultConfigFile = "./dm-master.toml" - _ = check.Suite(&testConfigSuite{}) -) +var _ = check.Suite(&testConfigSuite{}) type testConfigSuite struct{} @@ -42,30 +39,21 @@ func (t *testConfigSuite) SetUpSuite(c *check.C) { } func (t *testConfigSuite) TestPrintSampleConfig(c *check.C) { - buf, err := os.ReadFile(defaultConfigFile) - c.Assert(err, check.IsNil) - // test print sample config out := capturer.CaptureStdout(func() { cfg := NewConfig() - err = cfg.Parse([]string{"-print-sample-config"}) + err := cfg.Parse([]string{"-print-sample-config"}) c.Assert(err, check.ErrorMatches, flag.ErrHelp.Error()) }) - c.Assert(strings.TrimSpace(out), check.Equals, strings.TrimSpace(string(buf))) + c.Assert(strings.TrimSpace(out), check.Equals, strings.TrimSpace(SampleConfig)) } func (t *testConfigSuite) TestConfig(c *check.C) { var ( - err error - cfg = &Config{} - masterAddr = ":8261" - advertiseAddr = "127.0.0.1:8261" - name = "dm-master" - dataDir = "default.dm-master" - peerURLs = "http://127.0.0.1:8291" - advertisePeerURLs = "http://127.0.0.1:8291" - initialCluster = "dm-master=http://127.0.0.1:8291" - cases = []struct { + err error + cfg = &Config{} + masterAddr = ":8261" + cases = []struct { args []string hasError bool errorReg string @@ -85,15 +73,11 @@ func (t *testConfigSuite) TestConfig(c *check.C) { true, ".*'invalid' is an invalid flag.*", }, - { - []string{"--config=./dm-master.toml"}, - false, - "", - }, } ) - cfg.ConfigFile = defaultConfigFile + err = cfg.FromContent(SampleConfig) + c.Assert(err, check.IsNil) err = cfg.Reload() c.Assert(err, check.IsNil) c.Assert(cfg.MasterAddr, check.Equals, masterAddr) @@ -103,19 +87,6 @@ func (t *testConfigSuite) TestConfig(c *check.C) { err = cfg.Parse(tc.args) if tc.hasError { c.Assert(err, check.ErrorMatches, tc.errorReg) - } else { - c.Assert(cfg.MasterAddr, check.Equals, masterAddr) - c.Assert(cfg.AdvertiseAddr, check.Equals, advertiseAddr) - c.Assert(cfg.Name, check.Equals, name) - c.Assert(cfg.DataDir, check.Equals, dataDir) - c.Assert(cfg.PeerUrls, check.Equals, peerURLs) - c.Assert(cfg.AdvertisePeerUrls, check.Equals, advertisePeerURLs) - c.Assert(cfg.InitialCluster, check.Equals, initialCluster) - c.Assert(cfg.InitialClusterState, check.Equals, embed.ClusterStateFlagNew) - c.Assert(cfg.Join, check.Equals, "") - c.Assert(cfg.String(), check.Matches, fmt.Sprintf("{.*master-addr\":\"%s\".*}", masterAddr)) - c.Assert(cfg.ExperimentalFeatures.OpenAPI, check.Equals, false) - c.Assert(cfg.OpenAPI, check.Equals, false) } } } @@ -283,7 +254,7 @@ func (t *testConfigSuite) TestParseURLs(c *check.C) { func (t *testConfigSuite) TestAdjustAddr(c *check.C) { cfg := NewConfig() - c.Assert(cfg.configFromFile(defaultConfigFile), check.IsNil) + c.Assert(cfg.FromContent(SampleConfig), check.IsNil) c.Assert(cfg.adjust(), check.IsNil) // invalid `advertise-addr` @@ -303,14 +274,14 @@ func (t *testConfigSuite) TestAdjustAddr(c *check.C) { func (t *testConfigSuite) TestAdjustOpenAPI(c *check.C) { cfg := NewConfig() - c.Assert(cfg.configFromFile(defaultConfigFile), check.IsNil) + c.Assert(cfg.FromContent(SampleConfig), check.IsNil) c.Assert(cfg.adjust(), check.IsNil) // test default value c.Assert(cfg.OpenAPI, check.Equals, false) c.Assert(cfg.ExperimentalFeatures.OpenAPI, check.Equals, false) - // adjust openapi from experimental-features + // adjust openapi from experimental-features cfg.ExperimentalFeatures.OpenAPI = true c.Assert(cfg.adjust(), check.IsNil) c.Assert(cfg.OpenAPI, check.Equals, true) diff --git a/dm/dm/master/election_test.go b/dm/dm/master/election_test.go index a521611f0f5..8c52ddb4a0c 100644 --- a/dm/dm/master/election_test.go +++ b/dm/dm/master/election_test.go @@ -47,7 +47,7 @@ func (t *testElectionSuite) TestFailToStartLeader(c *check.C) { // create a new cluster cfg1 := NewConfig() - c.Assert(cfg1.Parse([]string{"-config=./dm-master.toml"}), check.IsNil) + c.Assert(cfg1.FromContent(SampleConfig), check.IsNil) cfg1.Name = "dm-master-1" cfg1.DataDir = c.MkDir() cfg1.MasterAddr = tempurl.Alloc()[len("http://"):] @@ -65,7 +65,7 @@ func (t *testElectionSuite) TestFailToStartLeader(c *check.C) { // join to an existing cluster cfg2 := NewConfig() - c.Assert(cfg2.Parse([]string{"-config=./dm-master.toml"}), check.IsNil) + c.Assert(cfg2.FromContent(SampleConfig), check.IsNil) cfg2.Name = "dm-master-2" cfg2.DataDir = c.MkDir() cfg2.MasterAddr = tempurl.Alloc()[len("http://"):] diff --git a/dm/dm/master/etcd.go b/dm/dm/master/etcd.go index a10b53af54b..d0b4adfd3ab 100644 --- a/dm/dm/master/etcd.go +++ b/dm/dm/master/etcd.go @@ -22,7 +22,7 @@ import ( "time" toolutils "github.com/pingcap/tidb-tools/pkg/utils" - "go.etcd.io/etcd/embed" + "go.etcd.io/etcd/server/v3/embed" "go.uber.org/zap" "google.golang.org/grpc" diff --git a/dm/dm/master/etcd_test.go b/dm/dm/master/etcd_test.go index 402396e0e40..3d63c91fe6e 100644 --- a/dm/dm/master/etcd_test.go +++ b/dm/dm/master/etcd_test.go @@ -24,8 +24,8 @@ import ( "github.com/pingcap/check" "github.com/tikv/pd/pkg/tempurl" - "go.etcd.io/etcd/clientv3" - "go.etcd.io/etcd/embed" + clientv3 "go.etcd.io/etcd/client/v3" + "go.etcd.io/etcd/server/v3/embed" "github.com/pingcap/tiflow/dm/pkg/log" "github.com/pingcap/tiflow/dm/pkg/terror" @@ -178,7 +178,7 @@ func (t *testEtcdSuite) TestPrepareJoinEtcd(c *check.C) { cfgAfter2.AdvertisePeerUrls = cfgAfter2.PeerUrls err = prepareJoinEtcd(cfgAfter2) c.Assert(terror.ErrMasterJoinEmbedEtcdFail.Equal(err), check.IsTrue) - c.Assert(err, check.ErrorMatches, ".*fail to join embed etcd: there is a member that has not joined successfully, continue the join or remove it.*") + c.Assert(err, check.ErrorMatches, ".*context deadline exceeded.*") // start the joining etcd cfgAfterEtcd := genEmbedEtcdConfigWithLogger("info") @@ -227,7 +227,7 @@ func (t *testEtcdSuite) TestIsDirExist(c *check.C) { func (t *testEtcdSuite) TestEtcdAutoCompaction(c *check.C) { cfg := NewConfig() - c.Assert(cfg.Parse([]string{"-config=./dm-master.toml"}), check.IsNil) + c.Assert(cfg.FromContent(SampleConfig), check.IsNil) cfg.DataDir = c.MkDir() cfg.MasterAddr = tempurl.Alloc()[len("http://"):] diff --git a/dm/dm/master/openapi_view_test.go b/dm/dm/master/openapi_view_test.go index 4ca476ba39e..73126859543 100644 --- a/dm/dm/master/openapi_view_test.go +++ b/dm/dm/master/openapi_view_test.go @@ -29,12 +29,11 @@ import ( "github.com/pingcap/failpoint" "github.com/stretchr/testify/require" "github.com/tikv/pd/pkg/tempurl" - "go.etcd.io/etcd/clientv3" - "go.etcd.io/etcd/integration" + clientv3 "go.etcd.io/etcd/client/v3" + "go.etcd.io/etcd/tests/v3/integration" "github.com/pingcap/tiflow/dm/checker" "github.com/pingcap/tiflow/dm/dm/config" - "github.com/pingcap/tiflow/dm/dm/master/workerrpc" "github.com/pingcap/tiflow/dm/dm/pb" "github.com/pingcap/tiflow/dm/dm/pbmock" "github.com/pingcap/tiflow/dm/openapi" @@ -57,22 +56,20 @@ type openAPISuite struct { etcdTestCli *clientv3.Client testEtcdCluster *integration.ClusterV3 - workerClients map[string]workerrpc.Client } func (t *openAPISuite) SetUpSuite(c *check.C) { checkAndAdjustSourceConfigFunc = checkAndNoAdjustSourceConfigMock + t.testEtcdCluster = integration.NewClusterV3(t.testT, &integration.ClusterConfig{Size: 1}) + t.etcdTestCli = t.testEtcdCluster.RandClient() } func (t *openAPISuite) TearDownSuite(c *check.C) { checkAndAdjustSourceConfigFunc = checkAndAdjustSourceConfig + t.testEtcdCluster.Terminate(t.testT) } func (t *openAPISuite) SetUpTest(c *check.C) { - t.testEtcdCluster = integration.NewClusterV3(t.testT, &integration.ClusterConfig{Size: 1}) - t.etcdTestCli = t.testEtcdCluster.RandClient() - t.workerClients = make(map[string]workerrpc.Client) - c.Assert(ha.ClearTestInfoOperation(t.etcdTestCli), check.IsNil) } @@ -82,7 +79,7 @@ func (t *openAPISuite) TestRedirectRequestToLeader(c *check.C) { // create a new cluster cfg1 := NewConfig() - c.Assert(cfg1.Parse([]string{"-config=./dm-master.toml"}), check.IsNil) + c.Assert(cfg1.FromContent(SampleConfig), check.IsNil) cfg1.Name = "dm-master-1" cfg1.DataDir = c.MkDir() cfg1.MasterAddr = tempurl.Alloc()[len("http://"):] @@ -104,7 +101,7 @@ func (t *openAPISuite) TestRedirectRequestToLeader(c *check.C) { // join to an existing cluster cfg2 := NewConfig() - c.Assert(cfg2.Parse([]string{"-config=./dm-master.toml"}), check.IsNil) + c.Assert(cfg2.FromContent(SampleConfig), check.IsNil) cfg2.Name = "dm-master-2" cfg2.DataDir = c.MkDir() cfg2.MasterAddr = tempurl.Alloc()[len("http://"):] @@ -143,7 +140,7 @@ func (t *openAPISuite) TestRedirectRequestToLeader(c *check.C) { func (t *openAPISuite) TestOpenAPIWillNotStartInDefaultConfig(c *check.C) { // create a new cluster cfg1 := NewConfig() - c.Assert(cfg1.Parse([]string{"-config=./dm-master.toml"}), check.IsNil) + c.Assert(cfg1.FromContent(SampleConfig), check.IsNil) cfg1.Name = "dm-master-1" cfg1.DataDir = c.MkDir() cfg1.MasterAddr = tempurl.Alloc()[len("http://"):] @@ -627,7 +624,7 @@ func (t *openAPISuite) TestClusterAPI(c *check.C) { // join a new master node to an existing cluster cfg2 := NewConfig() - c.Assert(cfg2.Parse([]string{"-config=./dm-master.toml"}), check.IsNil) + c.Assert(cfg2.FromContent(SampleConfig), check.IsNil) cfg2.Name = "dm-master-2" cfg2.DataDir = c.MkDir() cfg2.MasterAddr = tempurl.Alloc()[len("http://"):] @@ -823,7 +820,7 @@ func setupTestServer(ctx context.Context, t *testing.T) *Server { t.Helper() // create a new cluster cfg1 := NewConfig() - require.Nil(t, cfg1.Parse([]string{"-config=./dm-master.toml"})) + require.NoError(t, cfg1.FromContent(SampleConfig)) cfg1.Name = "dm-master-1" cfg1.DataDir = t.TempDir() cfg1.MasterAddr = tempurl.Alloc()[len("http://"):] diff --git a/dm/dm/master/scheduler/scheduler.go b/dm/dm/master/scheduler/scheduler.go index 8f5de662757..85852ced6fe 100644 --- a/dm/dm/master/scheduler/scheduler.go +++ b/dm/dm/master/scheduler/scheduler.go @@ -21,7 +21,7 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/failpoint" - "go.etcd.io/etcd/clientv3" + clientv3 "go.etcd.io/etcd/client/v3" "go.uber.org/atomic" "go.uber.org/zap" diff --git a/dm/dm/master/scheduler/scheduler_test.go b/dm/dm/master/scheduler/scheduler_test.go index b5f03c6d543..3c2195fd2ec 100644 --- a/dm/dm/master/scheduler/scheduler_test.go +++ b/dm/dm/master/scheduler/scheduler_test.go @@ -22,9 +22,9 @@ import ( . "github.com/pingcap/check" "github.com/pingcap/failpoint" - "go.etcd.io/etcd/clientv3" - v3rpc "go.etcd.io/etcd/etcdserver/api/v3rpc/rpctypes" - "go.etcd.io/etcd/integration" + v3rpc "go.etcd.io/etcd/api/v3/v3rpc/rpctypes" + clientv3 "go.etcd.io/etcd/client/v3" + "go.etcd.io/etcd/tests/v3/integration" "github.com/pingcap/tiflow/dm/dm/config" "github.com/pingcap/tiflow/dm/dm/master/workerrpc" @@ -35,13 +35,6 @@ import ( "github.com/pingcap/tiflow/dm/pkg/utils" ) -const ( - // do not forget to update this path if the file removed/renamed. - sourceSampleFile = "../../worker/source.yaml" - // do not forget to update this path if the file removed/renamed. - subTaskSampleFile = "../../worker/subtask.toml" -) - const ( noRestart = iota // do nothing in rebuildPessimist, just keep testing restartOnly // restart without building new instance. mock leader role transfer @@ -60,6 +53,7 @@ func TestScheduler(t *testing.T) { t.Fatal(err) } + integration.BeforeTestExternal(t) mockCluster := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1}) defer mockCluster.Terminate(t) @@ -116,13 +110,13 @@ func (t *testScheduler) testSchedulerProgress(c *C, restart int) { } } ) - sourceCfg1, err := config.LoadFromFile(sourceSampleFile) + sourceCfg1, err := config.ParseYamlAndVerify(config.SampleSourceConfig) c.Assert(err, IsNil) sourceCfg1.SourceID = sourceID1 sourceCfg2 := *sourceCfg1 sourceCfg2.SourceID = sourceID2 - c.Assert(subtaskCfg1.DecodeFile(subTaskSampleFile, true), IsNil) + c.Assert(subtaskCfg1.Decode(config.SampleSubtaskConfig, true), IsNil) subtaskCfg1.SourceID = sourceID1 subtaskCfg1.Name = taskName1 c.Assert(subtaskCfg1.Adjust(true), IsNil) @@ -813,7 +807,7 @@ func (t *testScheduler) TestRestartScheduler(c *C) { wg sync.WaitGroup keepAliveTTL = int64(2) // NOTE: this should be >= minLeaseTTL, in second. ) - sourceCfg1, err := config.LoadFromFile(sourceSampleFile) + sourceCfg1, err := config.ParseYamlAndVerify(config.SampleSourceConfig) c.Assert(err, IsNil) sourceCfg1.SourceID = sourceID1 @@ -974,7 +968,7 @@ func (t *testScheduler) TestWatchWorkerEventEtcdCompact(c *C) { workerAddr4 = "127.0.0.1:18462" keepAliveTTL = int64(2) // NOTE: this should be >= minLeaseTTL, in second. ) - sourceCfg1, err := config.LoadFromFile(sourceSampleFile) + sourceCfg1, err := config.ParseYamlAndVerify(config.SampleSourceConfig) c.Assert(err, IsNil) sourceCfg1.SourceID = sourceID1 sourceCfg2 := *sourceCfg1 @@ -1114,7 +1108,7 @@ func (t *testScheduler) TestLastBound(c *C) { workerName4 = "dm-worker-4" ) - sourceCfg1, err := config.LoadFromFile(sourceSampleFile) + sourceCfg1, err := config.ParseYamlAndVerify(config.SampleSourceConfig) c.Assert(err, IsNil) sourceCfg1.SourceID = sourceID1 sourceCfg2 := sourceCfg1 @@ -1187,7 +1181,7 @@ func (t *testScheduler) TestInvalidLastBound(c *C) { workerName1 = "dm-worker-1" ) - sourceCfg1, err := config.LoadFromFile(sourceSampleFile) + sourceCfg1, err := config.ParseYamlAndVerify(config.SampleSourceConfig) c.Assert(err, IsNil) sourceCfg1.SourceID = sourceID1 sourceCfg2 := sourceCfg1 @@ -1870,7 +1864,7 @@ func (t *testScheduler) TestWorkerHasDiffRelayAndBound(c *C) { Worker: workerName1, } - sourceCfg, err := config.LoadFromFile("../source.yaml") + sourceCfg, err := config.ParseYamlAndVerify(config.SampleSourceConfig) c.Assert(err, IsNil) sourceCfg.Checker.BackoffMax = config.Duration{Duration: 5 * time.Second} @@ -1929,7 +1923,7 @@ func (t *testScheduler) TestUpgradeCauseConflictRelayType(c *C) { Worker: workerName1, } - sourceCfg, err := config.LoadFromFile("../source.yaml") + sourceCfg, err := config.ParseYamlAndVerify(config.SampleSourceConfig) c.Assert(err, IsNil) sourceCfg.Checker.BackoffMax = config.Duration{Duration: 5 * time.Second} @@ -1986,7 +1980,7 @@ func (t *testScheduler) TestOperateValidatorTask(c *C) { keepAlive = int64(2) subtaskCfg config.SubTaskConfig ) - c.Assert(subtaskCfg.DecodeFile(subTaskSampleFile, true), IsNil) + c.Assert(subtaskCfg.Decode(config.SampleSubtaskConfig, true), IsNil) subtaskCfg.SourceID = sourceID1 subtaskCfg.Name = taskName subtaskCfg.ValidatorCfg = config.ValidatorConfig{Mode: config.ValidationNone} @@ -1997,7 +1991,7 @@ func (t *testScheduler) TestOperateValidatorTask(c *C) { Source: sourceID1, Worker: workerName1, } - sourceCfg, err := config.LoadFromFile("../source.yaml") + sourceCfg, err := config.ParseYamlAndVerify(config.SampleSourceConfig) c.Assert(err, IsNil) s.etcdCli = etcdTestCli sourceCfg.SourceID = sourceID1 diff --git a/dm/dm/master/server.go b/dm/dm/master/server.go index 6c5014166ee..fb324ca56c9 100644 --- a/dm/dm/master/server.go +++ b/dm/dm/master/server.go @@ -32,8 +32,8 @@ import ( "github.com/pingcap/failpoint" "github.com/pingcap/tidb-tools/pkg/dbutil" toolutils "github.com/pingcap/tidb-tools/pkg/utils" - "go.etcd.io/etcd/clientv3" - "go.etcd.io/etcd/embed" + clientv3 "go.etcd.io/etcd/client/v3" + "go.etcd.io/etcd/server/v3/embed" "go.uber.org/atomic" "go.uber.org/zap" "google.golang.org/grpc" @@ -210,7 +210,7 @@ func (s *Server) Start(ctx context.Context) (err error) { gRPCSvr := func(gs *grpc.Server) { pb.RegisterMasterServer(gs, s) } // start embed etcd server, gRPC API server and HTTP (API, status and debug) server. - s.etcd, err = startEtcd(etcdCfg, gRPCSvr, userHandles, etcdStartTimeout) + s.etcd, err = startEtcd(etcdCfg, gRPCSvr, userHandles, 10*time.Second) if err != nil { return } diff --git a/dm/dm/master/server_test.go b/dm/dm/master/server_test.go index b3c23e14403..5f6b1b8793f 100644 --- a/dm/dm/master/server_test.go +++ b/dm/dm/master/server_test.go @@ -40,8 +40,9 @@ import ( "github.com/pingcap/tidb/sessionctx" tidbmock "github.com/pingcap/tidb/util/mock" "github.com/tikv/pd/pkg/tempurl" - "go.etcd.io/etcd/clientv3" - "go.etcd.io/etcd/integration" + clientv3 "go.etcd.io/etcd/client/v3" + "go.etcd.io/etcd/server/v3/verify" + "go.etcd.io/etcd/tests/v3/integration" "google.golang.org/grpc" "github.com/pingcap/tiflow/dm/checker" @@ -168,13 +169,21 @@ type testMaster struct { etcdTestCli *clientv3.Client } -var testSuite = check.Suite(&testMaster{}) +var ( + testSuite = check.SerialSuites(&testMaster{}) + pwd string +) func TestMaster(t *testing.T) { err := log.InitLogger(&log.Config{}) if err != nil { t.Fatal(err) } + pwd, err = os.Getwd() + if err != nil { + t.Fatal(err) + } + integration.BeforeTestExternal(t) // inject *testing.T to testMaster s := testSuite.(*testMaster) s.testT = t @@ -344,7 +353,7 @@ func makeWorkerClientsForHandle(ctrl *gomock.Controller, taskName string, source func testDefaultMasterServer(c *check.C) *Server { cfg := NewConfig() - err := cfg.Parse([]string{"-config=./dm-master.toml"}) + err := cfg.FromContent(SampleConfig) c.Assert(err, check.IsNil) cfg.DataDir = c.MkDir() server := NewServer(cfg) @@ -431,7 +440,7 @@ func (t *testMaster) testMockSchedulerForRelay(ctx context.Context, wg *sync.Wai func generateServerConfig(c *check.C, name string) *Config { // create a new cluster cfg1 := NewConfig() - c.Assert(cfg1.Parse([]string{"-config=./dm-master.toml"}), check.IsNil) + c.Assert(cfg1.FromContent(SampleConfig), check.IsNil) cfg1.Name = name cfg1.DataDir = c.MkDir() cfg1.MasterAddr = tempurl.Alloc()[len("http://"):] @@ -1367,7 +1376,7 @@ func (t *testMaster) TestOperateWorkerRelayTask(c *check.C) { func (t *testMaster) TestServer(c *check.C) { var err error cfg := NewConfig() - c.Assert(cfg.Parse([]string{"-config=./dm-master.toml"}), check.IsNil) + c.Assert(cfg.FromContent(SampleConfig), check.IsNil) cfg.PeerUrls = "http://127.0.0.1:8294" cfg.DataDir = c.MkDir() cfg.MasterAddr = tempurl.Alloc()[len("http://"):] @@ -1383,12 +1392,19 @@ func (t *testMaster) TestServer(c *check.C) { basicServiceCheck(c, cfg) // try to start another server with the same address. Expect it to fail + // unset an etcd variable because it will cause checking on exit, and block forever + err = os.Unsetenv(verify.ENV_VERIFY) + c.Assert(err, check.IsNil) + dupServer := NewServer(cfg) ctx, cancel := context.WithCancel(context.Background()) defer cancel() err1 := dupServer.Start(ctx) c.Assert(terror.ErrMasterStartEmbedEtcdFail.Equal(err1), check.IsTrue) c.Assert(err1.Error(), check.Matches, ".*bind: address already in use.*") + + err = os.Setenv(verify.ENV_VERIFY, verify.ENV_VERIFY_ALL_VALUE) + c.Assert(err, check.IsNil) }) // test the listen address is 0.0.0.0 @@ -1411,6 +1427,10 @@ func (t *testMaster) TestMasterTLS(c *check.C) { _, peerPort, err := net.SplitHostPort(peerAddr) c.Assert(err, check.IsNil) + caPath := pwd + "/tls_for_test/ca.pem" + certPath := pwd + "/tls_for_test/dm.pem" + keyPath := pwd + "/tls_for_test/dm.key" + // all with `https://` prefix cfg := NewConfig() c.Assert(cfg.Parse([]string{ @@ -1421,9 +1441,9 @@ func (t *testMaster) TestMasterTLS(c *check.C) { fmt.Sprintf("--peer-urls=https://%s", peerAddr), fmt.Sprintf("--advertise-peer-urls=https://%s", peerAddr), fmt.Sprintf("--initial-cluster=master-tls=https://%s", peerAddr), - "--ssl-ca=./tls_for_test/ca.pem", - "--ssl-cert=./tls_for_test/dm.pem", - "--ssl-key=./tls_for_test/dm.key", + "--ssl-ca=" + caPath, + "--ssl-cert=" + certPath, + "--ssl-key=" + keyPath, }), check.IsNil) t.testTLSPrefix(c, cfg) c.Assert(cfg.MasterAddr, check.Equals, masterAddr) @@ -1442,9 +1462,9 @@ func (t *testMaster) TestMasterTLS(c *check.C) { fmt.Sprintf("--peer-urls=https://%s", peerAddr), fmt.Sprintf("--advertise-peer-urls=https://%s", peerAddr), fmt.Sprintf("--initial-cluster=master-tls=https://%s", peerAddr), - "--ssl-ca=./tls_for_test/ca.pem", - "--ssl-cert=./tls_for_test/dm.pem", - "--ssl-key=./tls_for_test/dm.key", + "--ssl-ca=" + caPath, + "--ssl-cert=" + certPath, + "--ssl-key=" + keyPath, }), check.IsNil) t.testTLSPrefix(c, cfg) @@ -1458,9 +1478,9 @@ func (t *testMaster) TestMasterTLS(c *check.C) { fmt.Sprintf("--peer-urls=https://%s", peerAddr), fmt.Sprintf("--advertise-peer-urls=https://%s", peerAddr), fmt.Sprintf("--initial-cluster=master-tls=https://%s", peerAddr), - "--ssl-ca=./tls_for_test/ca.pem", - "--ssl-cert=./tls_for_test/dm.pem", - "--ssl-key=./tls_for_test/dm.key", + "--ssl-ca=" + caPath, + "--ssl-cert=" + certPath, + "--ssl-key=" + keyPath, }), check.IsNil) t.testTLSPrefix(c, cfg) @@ -1474,9 +1494,9 @@ func (t *testMaster) TestMasterTLS(c *check.C) { fmt.Sprintf("--peer-urls=%s", peerAddr), fmt.Sprintf("--advertise-peer-urls=https://%s", peerAddr), fmt.Sprintf("--initial-cluster=master-tls=https://%s", peerAddr), - "--ssl-ca=./tls_for_test/ca.pem", - "--ssl-cert=./tls_for_test/dm.pem", - "--ssl-key=./tls_for_test/dm.key", + "--ssl-ca=" + caPath, + "--ssl-cert=" + certPath, + "--ssl-key=" + keyPath, }), check.IsNil) t.testTLSPrefix(c, cfg) @@ -1490,9 +1510,9 @@ func (t *testMaster) TestMasterTLS(c *check.C) { fmt.Sprintf("--peer-urls=%s", peerAddr), fmt.Sprintf("--advertise-peer-urls=%s", peerAddr), fmt.Sprintf("--initial-cluster=master-tls=https://%s", peerAddr), - "--ssl-ca=./tls_for_test/ca.pem", - "--ssl-cert=./tls_for_test/dm.pem", - "--ssl-key=./tls_for_test/dm.key", + "--ssl-ca=" + caPath, + "--ssl-cert=" + certPath, + "--ssl-key=" + keyPath, }), check.IsNil) t.testTLSPrefix(c, cfg) @@ -1506,9 +1526,9 @@ func (t *testMaster) TestMasterTLS(c *check.C) { fmt.Sprintf("--peer-urls=%s", peerAddr), fmt.Sprintf("--advertise-peer-urls=%s", peerAddr), fmt.Sprintf("--initial-cluster=master-tls=%s", peerAddr), - "--ssl-ca=./tls_for_test/ca.pem", - "--ssl-cert=./tls_for_test/dm.pem", - "--ssl-key=./tls_for_test/dm.key", + "--ssl-ca=" + caPath, + "--ssl-cert=" + certPath, + "--ssl-key=" + keyPath, }), check.IsNil) t.testTLSPrefix(c, cfg) c.Assert(cfg.MasterAddr, check.Equals, masterAddr) @@ -1527,9 +1547,9 @@ func (t *testMaster) TestMasterTLS(c *check.C) { fmt.Sprintf("--peer-urls=http://%s", peerAddr), fmt.Sprintf("--advertise-peer-urls=http://%s", peerAddr), fmt.Sprintf("--initial-cluster=master-tls=http://%s", peerAddr), - "--ssl-ca=./tls_for_test/ca.pem", - "--ssl-cert=./tls_for_test/dm.pem", - "--ssl-key=./tls_for_test/dm.key", + "--ssl-ca=" + caPath, + "--ssl-cert=" + certPath, + "--ssl-key=" + keyPath, }), check.IsNil) c.Assert(cfg.MasterAddr, check.Equals, masterAddr) c.Assert(cfg.AdvertiseAddr, check.Equals, masterAddr) @@ -1547,9 +1567,9 @@ func (t *testMaster) TestMasterTLS(c *check.C) { fmt.Sprintf("--peer-urls=https://%s", peerAddr), fmt.Sprintf("--advertise-peer-urls=https://%s", peerAddr), fmt.Sprintf("--initial-cluster=master-tls=http://%s", peerAddr), - "--ssl-ca=./tls_for_test/ca.pem", - "--ssl-cert=./tls_for_test/dm.pem", - "--ssl-key=./tls_for_test/dm.key", + "--ssl-ca=" + caPath, + "--ssl-cert=" + certPath, + "--ssl-key=" + keyPath, }), check.IsNil) c.Assert(cfg.MasterAddr, check.Equals, masterAddr) c.Assert(cfg.AdvertiseAddr, check.Equals, masterAddr) @@ -1568,9 +1588,9 @@ func (t *testMaster) TestMasterTLS(c *check.C) { fmt.Sprintf("--peer-urls=0.0.0.0:%s", peerPort), fmt.Sprintf("--advertise-peer-urls=https://%s", peerAddr), fmt.Sprintf("--initial-cluster=master-tls=https://%s", peerAddr), - "--ssl-ca=./tls_for_test/ca.pem", - "--ssl-cert=./tls_for_test/dm.pem", - "--ssl-key=./tls_for_test/dm.key", + "--ssl-ca=" + caPath, + "--ssl-cert=" + certPath, + "--ssl-key=" + keyPath, }), check.IsNil) t.testTLSPrefix(c, cfg) } @@ -1603,7 +1623,7 @@ func (t *testMaster) testNormalServerLifecycle(c *check.C, cfg *Config, checkLog func (t *testMaster) testHTTPInterface(c *check.C, url string, contain []byte) { // we use HTTPS in some test cases. - tls, err := toolutils.NewTLS("./tls_for_test/ca.pem", "./tls_for_test/dm.pem", "./tls_for_test/dm.key", url, []string{}) + tls, err := toolutils.NewTLS(pwd+"/tls_for_test/ca.pem", pwd+"/tls_for_test/dm.pem", pwd+"/tls_for_test/dm.key", url, []string{}) c.Assert(err, check.IsNil) cli := toolutils.ClientWithTLS(tls.TLSConfig()) @@ -1623,7 +1643,7 @@ func (t *testMaster) TestJoinMember(c *check.C) { // create a new cluster cfg1 := NewConfig() - c.Assert(cfg1.Parse([]string{"-config=./dm-master.toml"}), check.IsNil) + c.Assert(cfg1.FromContent(SampleConfig), check.IsNil) cfg1.Name = "dm-master-1" cfg1.DataDir = c.MkDir() cfg1.MasterAddr = tempurl.Alloc()[len("http://"):] @@ -1643,7 +1663,7 @@ func (t *testMaster) TestJoinMember(c *check.C) { // join to an existing cluster cfg2 := NewConfig() - c.Assert(cfg2.Parse([]string{"-config=./dm-master.toml"}), check.IsNil) + c.Assert(cfg2.FromContent(SampleConfig), check.IsNil) cfg2.Name = "dm-master-2" cfg2.DataDir = c.MkDir() cfg2.MasterAddr = tempurl.Alloc()[len("http://"):] @@ -1678,7 +1698,7 @@ func (t *testMaster) TestJoinMember(c *check.C) { c.Assert(leaderID, check.Equals, cfg1.Name) cfg3 := NewConfig() - c.Assert(cfg3.Parse([]string{"-config=./dm-master.toml"}), check.IsNil) + c.Assert(cfg3.FromContent(SampleConfig), check.IsNil) cfg3.Name = "dm-master-3" cfg3.DataDir = c.MkDir() cfg3.MasterAddr = tempurl.Alloc()[len("http://"):] @@ -1721,7 +1741,7 @@ func (t *testMaster) TestOperateSource(c *check.C) { // create a new cluster cfg1 := NewConfig() - c.Assert(cfg1.Parse([]string{"-config=./dm-master.toml"}), check.IsNil) + c.Assert(cfg1.FromContent(SampleConfig), check.IsNil) cfg1.Name = "dm-master-1" cfg1.DataDir = c.MkDir() cfg1.MasterAddr = tempurl.Alloc()[len("http://"):] @@ -1734,7 +1754,7 @@ func (t *testMaster) TestOperateSource(c *check.C) { s1.leader.Store(oneselfLeader) c.Assert(s1.Start(ctx), check.IsNil) defer s1.Close() - mysqlCfg, err := config.LoadFromFile("./source.yaml") + mysqlCfg, err := config.ParseYamlAndVerify(config.SampleSourceConfig) c.Assert(err, check.IsNil) mysqlCfg.From.Password = os.Getenv("MYSQL_PSWD") task, err := mysqlCfg.Yaml() diff --git a/dm/dm/master/shardddl/optimist.go b/dm/dm/master/shardddl/optimist.go index dab6f8fa3e8..b2778e9f76b 100644 --- a/dm/dm/master/shardddl/optimist.go +++ b/dm/dm/master/shardddl/optimist.go @@ -22,7 +22,7 @@ import ( "github.com/pingcap/failpoint" "github.com/pingcap/tidb-tools/pkg/dbutil" - "go.etcd.io/etcd/clientv3" + clientv3 "go.etcd.io/etcd/client/v3" "go.uber.org/zap" "github.com/pingcap/tiflow/dm/dm/common" diff --git a/dm/dm/master/shardddl/optimist_test.go b/dm/dm/master/shardddl/optimist_test.go index b3e6c84e13e..f10f5eaa06c 100644 --- a/dm/dm/master/shardddl/optimist_test.go +++ b/dm/dm/master/shardddl/optimist_test.go @@ -26,8 +26,8 @@ import ( "github.com/pingcap/tidb/parser/model" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/util/mock" - "go.etcd.io/etcd/clientv3" - "go.etcd.io/etcd/integration" + clientv3 "go.etcd.io/etcd/client/v3" + "go.etcd.io/etcd/tests/v3/integration" "github.com/pingcap/tiflow/dm/dm/config" "github.com/pingcap/tiflow/dm/dm/pb" @@ -36,13 +36,25 @@ import ( "github.com/pingcap/tiflow/dm/pkg/utils" ) -type testOptimist struct{} +type testOptimist struct { + mockCluster *integration.ClusterV3 + etcdTestCli *clientv3.Client +} var _ = SerialSuites(&testOptimist{}) +func (t *testOptimist) SetUpSuite(c *C) { + t.mockCluster = integration.NewClusterV3(tt, &integration.ClusterConfig{Size: 1}) + t.etcdTestCli = t.mockCluster.RandClient() +} + +func (t *testOptimist) TearDownSuite(c *C) { + t.mockCluster.Terminate(tt) +} + // clear keys in etcd test cluster. -func clearOptimistTestSourceInfoOperation(c *C) { - c.Assert(optimism.ClearTestInfoOperationColumn(etcdTestCli), IsNil) +func (t *testOptimist) clearOptimistTestSourceInfoOperation(c *C) { + c.Assert(optimism.ClearTestInfoOperationColumn(t.etcdTestCli), IsNil) } func createTableInfo(c *C, p *parser.Parser, se sessionctx.Context, tableID int64, sql string) *model.TableInfo { @@ -99,7 +111,7 @@ func watchExactOneOperation( } func (t *testOptimist) TestOptimistSourceTables(c *C) { - defer clearOptimistTestSourceInfoOperation(c) + defer t.clearOptimistTestSourceInfoOperation(c) var ( logger = log.L() @@ -122,17 +134,17 @@ func (t *testOptimist) TestOptimistSourceTables(c *C) { defer cancel() // CASE 1: start without any previous kv and no etcd operation. - c.Assert(o.Start(ctx, etcdTestCli), IsNil) + c.Assert(o.Start(ctx, t.etcdTestCli), IsNil) c.Assert(o.tk.FindTables(task, downSchema, downTable), IsNil) o.Close() o.Close() // close multiple times. // CASE 2: start again without any previous kv. - c.Assert(o.Start(ctx, etcdTestCli), IsNil) + c.Assert(o.Start(ctx, t.etcdTestCli), IsNil) c.Assert(o.tk.FindTables(task, downSchema, downTable), IsNil) // PUT st1, should find tables. - _, err := optimism.PutSourceTables(etcdTestCli, st1) + _, err := optimism.PutSourceTables(t.etcdTestCli, st1) c.Assert(err, IsNil) c.Assert(utils.WaitSomething(30, 100*time.Millisecond, func() bool { tts := o.tk.FindTables(task, downSchema, downTable) @@ -144,13 +156,13 @@ func (t *testOptimist) TestOptimistSourceTables(c *C) { o.Close() // CASE 3: start again with previous source tables. - c.Assert(o.Start(ctx, etcdTestCli), IsNil) + c.Assert(o.Start(ctx, t.etcdTestCli), IsNil) tts = o.tk.FindTables(task, downSchema, downTable) c.Assert(tts, HasLen, 1) c.Assert(tts[0], DeepEquals, st1.TargetTable(downSchema, downTable)) // PUT st2, should find more tables. - _, err = optimism.PutSourceTables(etcdTestCli, st2) + _, err = optimism.PutSourceTables(t.etcdTestCli, st2) c.Assert(err, IsNil) c.Assert(utils.WaitSomething(30, 100*time.Millisecond, func() bool { tts = o.tk.FindTables(task, downSchema, downTable) @@ -164,14 +176,14 @@ func (t *testOptimist) TestOptimistSourceTables(c *C) { // CASE 4: create (not re-start) a new optimist with previous source tables. o = NewOptimist(&logger, getDownstreamMeta) - c.Assert(o.Start(ctx, etcdTestCli), IsNil) + c.Assert(o.Start(ctx, t.etcdTestCli), IsNil) tts = o.tk.FindTables(task, downSchema, downTable) c.Assert(tts, HasLen, 2) c.Assert(tts[0], DeepEquals, st1.TargetTable(downSchema, downTable)) c.Assert(tts[1], DeepEquals, st2.TargetTable(downSchema, downTable)) // DELETE st1, should find less tables. - _, err = optimism.DeleteSourceTables(etcdTestCli, st1) + _, err = optimism.DeleteSourceTables(t.etcdTestCli, st1) c.Assert(err, IsNil) c.Assert(utils.WaitSomething(30, 100*time.Millisecond, func() bool { tts = o.tk.FindTables(task, downSchema, downTable) @@ -184,14 +196,10 @@ func (t *testOptimist) TestOptimistSourceTables(c *C) { } func (t *testOptimist) TestOptimist(c *C) { - cluster := integration.NewClusterV3(tt, &integration.ClusterConfig{Size: 1}) - defer cluster.Terminate(tt) - - cli := cluster.RandClient() - t.testOptimist(c, cli, noRestart) - t.testOptimist(c, cli, restartOnly) - t.testOptimist(c, cli, restartNewInstance) - t.testSortInfos(c, cli) + t.testOptimist(c, t.etcdTestCli, noRestart) + t.testOptimist(c, t.etcdTestCli, restartOnly) + t.testOptimist(c, t.etcdTestCli, restartNewInstance) + t.testSortInfos(c, t.etcdTestCli) } func (t *testOptimist) testOptimist(c *C, cli *clientv3.Client, restart int) { @@ -609,7 +617,7 @@ func (t *testOptimist) testOptimist(c *C, cli *clientv3.Client, restart int) { } func (t *testOptimist) TestOptimistLockConflict(c *C) { - defer clearOptimistTestSourceInfoOperation(c) + defer t.clearOptimistTestSourceInfoOperation(c) var ( watchTimeout = 5 * time.Second @@ -638,23 +646,23 @@ func (t *testOptimist) TestOptimistLockConflict(c *C) { st1.AddTable("foo", "bar-2", downSchema, downTable) // put source tables first. - _, err := optimism.PutSourceTables(etcdTestCli, st1) + _, err := optimism.PutSourceTables(t.etcdTestCli, st1) c.Assert(err, IsNil) ctx, cancel := context.WithCancel(context.Background()) defer cancel() - c.Assert(o.Start(ctx, etcdTestCli), IsNil) + c.Assert(o.Start(ctx, t.etcdTestCli), IsNil) c.Assert(o.Locks(), HasLen, 0) // PUT i1, will create a lock but not synced. - rev1, err := optimism.PutInfo(etcdTestCli, i1) + rev1, err := optimism.PutInfo(t.etcdTestCli, i1) c.Assert(err, IsNil) // wait operation for i1 become available. opCh := make(chan optimism.Operation, 10) errCh := make(chan error, 10) ctx2, cancel2 := context.WithCancel(ctx) - go optimism.WatchOperationPut(ctx2, etcdTestCli, i1.Task, i1.Source, i1.UpSchema, i1.UpTable, rev1, opCh, errCh) + go optimism.WatchOperationPut(ctx2, t.etcdTestCli, i1.Task, i1.Source, i1.UpSchema, i1.UpTable, rev1, opCh, errCh) select { case <-time.After(watchTimeout): c.Fatal("timeout") @@ -670,14 +678,14 @@ func (t *testOptimist) TestOptimistLockConflict(c *C) { c.Assert(len(errCh), Equals, 0) // PUT i2, conflict will be detected. - rev2, err := optimism.PutInfo(etcdTestCli, i2) + rev2, err := optimism.PutInfo(t.etcdTestCli, i2) c.Assert(err, IsNil) // wait operation for i2 become available. opCh = make(chan optimism.Operation, 10) errCh = make(chan error, 10) ctx2, cancel2 = context.WithCancel(ctx) - go optimism.WatchOperationPut(ctx2, etcdTestCli, i2.Task, i2.Source, i2.UpSchema, i2.UpTable, rev2, opCh, errCh) + go optimism.WatchOperationPut(ctx2, t.etcdTestCli, i2.Task, i2.Source, i2.UpSchema, i2.UpTable, rev2, opCh, errCh) select { case <-time.After(watchTimeout): c.Fatal("timeout") @@ -694,13 +702,13 @@ func (t *testOptimist) TestOptimistLockConflict(c *C) { // PUT i3, no conflict now. // case for handle-error replace - rev3, err := optimism.PutInfo(etcdTestCli, i3) + rev3, err := optimism.PutInfo(t.etcdTestCli, i3) c.Assert(err, IsNil) // wait operation for i3 become available. opCh = make(chan optimism.Operation, 10) errCh = make(chan error, 10) ctx2, cancel2 = context.WithCancel(ctx) - go optimism.WatchOperationPut(ctx2, etcdTestCli, i3.Task, i3.Source, i3.UpSchema, i3.UpTable, rev3, opCh, errCh) + go optimism.WatchOperationPut(ctx2, t.etcdTestCli, i3.Task, i3.Source, i3.UpSchema, i3.UpTable, rev3, opCh, errCh) select { case <-time.After(watchTimeout): c.Fatal("timeout") @@ -716,7 +724,7 @@ func (t *testOptimist) TestOptimistLockConflict(c *C) { } func (t *testOptimist) TestOptimistLockMultipleTarget(c *C) { - defer clearOptimistTestSourceInfoOperation(c) + defer t.clearOptimistTestSourceInfoOperation(c) var ( backOff = 30 @@ -752,19 +760,19 @@ func (t *testOptimist) TestOptimistLockMultipleTarget(c *C) { sts.AddTable(upSchema, upTables[3], downSchema, downTable2) // put source tables first. - _, err := optimism.PutSourceTables(etcdTestCli, sts) + _, err := optimism.PutSourceTables(t.etcdTestCli, sts) c.Assert(err, IsNil) ctx, cancel := context.WithCancel(context.Background()) defer cancel() - c.Assert(o.Start(ctx, etcdTestCli), IsNil) + c.Assert(o.Start(ctx, t.etcdTestCli), IsNil) c.Assert(o.Locks(), HasLen, 0) // PUT i11 and i21, will create two locks but no synced. - _, err = optimism.PutInfo(etcdTestCli, i11) + _, err = optimism.PutInfo(t.etcdTestCli, i11) c.Assert(err, IsNil) - _, err = optimism.PutInfo(etcdTestCli, i21) + _, err = optimism.PutInfo(t.etcdTestCli, i21) c.Assert(err, IsNil) c.Assert(utils.WaitSomething(backOff, waitTime, func() bool { return len(o.Locks()) == 2 @@ -807,9 +815,9 @@ func (t *testOptimist) TestOptimistLockMultipleTarget(c *C) { c.Assert(locks[1], DeepEquals, expectedLock[locks[1].ID]) // put i12 and i22, both of locks will be synced. - rev1, err := optimism.PutInfo(etcdTestCli, i12) + rev1, err := optimism.PutInfo(t.etcdTestCli, i12) c.Assert(err, IsNil) - rev2, err := optimism.PutInfo(etcdTestCli, i22) + rev2, err := optimism.PutInfo(t.etcdTestCli, i22) c.Assert(err, IsNil) c.Assert(utils.WaitSomething(backOff, waitTime, func() bool { synced1, _ := o.Locks()[lockID1].IsSynced() @@ -837,7 +845,7 @@ func (t *testOptimist) TestOptimistLockMultipleTarget(c *C) { errCh := make(chan error, 10) var op12 optimism.Operation ctx2, cancel2 := context.WithCancel(ctx) - go optimism.WatchOperationPut(ctx2, etcdTestCli, i12.Task, i12.Source, i12.UpSchema, i12.UpTable, rev1, opCh, errCh) + go optimism.WatchOperationPut(ctx2, t.etcdTestCli, i12.Task, i12.Source, i12.UpSchema, i12.UpTable, rev1, opCh, errCh) select { case <-time.After(watchTimeout): c.Fatal("timeout") @@ -855,12 +863,12 @@ func (t *testOptimist) TestOptimistLockMultipleTarget(c *C) { op11c := op12 op11c.Done = true op11c.UpTable = i11.UpTable // overwrite `UpTable`. - _, putted, err := optimism.PutOperation(etcdTestCli, false, op11c, 0) + _, putted, err := optimism.PutOperation(t.etcdTestCli, false, op11c, 0) c.Assert(err, IsNil) c.Assert(putted, IsTrue) op12c := op12 op12c.Done = true - _, putted, err = optimism.PutOperation(etcdTestCli, false, op12c, 0) + _, putted, err = optimism.PutOperation(t.etcdTestCli, false, op12c, 0) c.Assert(err, IsNil) c.Assert(putted, IsTrue) c.Assert(utils.WaitSomething(backOff, waitTime, func() bool { @@ -876,7 +884,7 @@ func (t *testOptimist) TestOptimistLockMultipleTarget(c *C) { errCh = make(chan error, 10) var op22 optimism.Operation ctx2, cancel2 = context.WithCancel(ctx) - go optimism.WatchOperationPut(ctx2, etcdTestCli, i22.Task, i22.Source, i22.UpSchema, i22.UpTable, rev2, opCh, errCh) + go optimism.WatchOperationPut(ctx2, t.etcdTestCli, i22.Task, i22.Source, i22.UpSchema, i22.UpTable, rev2, opCh, errCh) select { case <-time.After(watchTimeout): c.Fatal("timeout") @@ -894,12 +902,12 @@ func (t *testOptimist) TestOptimistLockMultipleTarget(c *C) { op21c := op22 op21c.Done = true op21c.UpTable = i21.UpTable // overwrite `UpTable`. - _, putted, err = optimism.PutOperation(etcdTestCli, false, op21c, 0) + _, putted, err = optimism.PutOperation(t.etcdTestCli, false, op21c, 0) c.Assert(err, IsNil) c.Assert(putted, IsTrue) op22c := op22 op22c.Done = true - _, putted, err = optimism.PutOperation(etcdTestCli, false, op22c, 0) + _, putted, err = optimism.PutOperation(t.etcdTestCli, false, op22c, 0) c.Assert(err, IsNil) c.Assert(putted, IsTrue) c.Assert(utils.WaitSomething(backOff, waitTime, func() bool { @@ -911,7 +919,7 @@ func (t *testOptimist) TestOptimistLockMultipleTarget(c *C) { } func (t *testOptimist) TestOptimistInitSchema(c *C) { - defer clearOptimistTestSourceInfoOperation(c) + defer t.clearOptimistTestSourceInfoOperation(c) var ( backOff = 30 @@ -944,17 +952,17 @@ func (t *testOptimist) TestOptimistInitSchema(c *C) { st.AddTable(upSchema, upTables[1], downSchema, downTable) // put source tables first. - _, err := optimism.PutSourceTables(etcdTestCli, st) + _, err := optimism.PutSourceTables(t.etcdTestCli, st) c.Assert(err, IsNil) ctx, cancel := context.WithCancel(context.Background()) defer cancel() - c.Assert(o.Start(ctx, etcdTestCli), IsNil) + c.Assert(o.Start(ctx, t.etcdTestCli), IsNil) c.Assert(o.Locks(), HasLen, 0) // PUT i11, will creat a lock. - _, err = optimism.PutInfo(etcdTestCli, i11) + _, err = optimism.PutInfo(t.etcdTestCli, i11) c.Assert(err, IsNil) c.Assert(utils.WaitSomething(backOff, waitTime, func() bool { return len(o.Locks()) == 1 @@ -962,7 +970,7 @@ func (t *testOptimist) TestOptimistInitSchema(c *C) { time.Sleep(waitTime) // sleep one more time to wait for update of init schema. // PUT i12, the lock will be synced. - rev1, err := optimism.PutInfo(etcdTestCli, i12) + rev1, err := optimism.PutInfo(t.etcdTestCli, i12) c.Assert(err, IsNil) // wait operation for i12 become available. @@ -970,7 +978,7 @@ func (t *testOptimist) TestOptimistInitSchema(c *C) { errCh := make(chan error, 10) var op12 optimism.Operation ctx2, cancel2 := context.WithCancel(ctx) - go optimism.WatchOperationPut(ctx2, etcdTestCli, i12.Task, i12.Source, i12.UpSchema, i12.UpTable, rev1, opCh, errCh) + go optimism.WatchOperationPut(ctx2, t.etcdTestCli, i12.Task, i12.Source, i12.UpSchema, i12.UpTable, rev1, opCh, errCh) select { case <-time.After(watchTimeout): c.Fatal("timeout") @@ -988,12 +996,12 @@ func (t *testOptimist) TestOptimistInitSchema(c *C) { op11c := op12 op11c.Done = true op11c.UpTable = i11.UpTable // overwrite `UpTable`. - _, putted, err := optimism.PutOperation(etcdTestCli, false, op11c, 0) + _, putted, err := optimism.PutOperation(t.etcdTestCli, false, op11c, 0) c.Assert(err, IsNil) c.Assert(putted, IsTrue) op12c := op12 op12c.Done = true - _, putted, err = optimism.PutOperation(etcdTestCli, false, op12c, 0) + _, putted, err = optimism.PutOperation(t.etcdTestCli, false, op12c, 0) c.Assert(err, IsNil) c.Assert(putted, IsTrue) c.Assert(utils.WaitSomething(backOff, waitTime, func() bool { @@ -1001,7 +1009,7 @@ func (t *testOptimist) TestOptimistInitSchema(c *C) { }), IsTrue) // PUT i21 to create the lock again. - _, err = optimism.PutInfo(etcdTestCli, i21) + _, err = optimism.PutInfo(t.etcdTestCli, i21) c.Assert(err, IsNil) c.Assert(utils.WaitSomething(backOff, waitTime, func() bool { return len(o.Locks()) == 1 @@ -1079,7 +1087,7 @@ func (t *testOptimist) testSortInfos(c *C, cli *clientv3.Client) { } func (t *testOptimist) TestBuildLockJoinedAndTable(c *C) { - defer clearOptimistTestSourceInfoOperation(c) + defer t.clearOptimistTestSourceInfoOperation(c) var ( logger = log.L() @@ -1111,24 +1119,24 @@ func (t *testOptimist) TestBuildLockJoinedAndTable(c *C) { st1.AddTable("foo", "bar-1", downSchema, downTable) st2.AddTable("foo", "bar-1", downSchema, downTable) - c.Assert(o.Start(ctx, etcdTestCli), IsNil) - _, err := optimism.PutSourceTables(etcdTestCli, st1) + c.Assert(o.Start(ctx, t.etcdTestCli), IsNil) + _, err := optimism.PutSourceTables(t.etcdTestCli, st1) c.Assert(err, IsNil) - _, err = optimism.PutSourceTables(etcdTestCli, st2) + _, err = optimism.PutSourceTables(t.etcdTestCli, st2) c.Assert(err, IsNil) - _, err = optimism.PutInfo(etcdTestCli, i21) + _, err = optimism.PutInfo(t.etcdTestCli, i21) c.Assert(err, IsNil) - _, err = optimism.PutInfo(etcdTestCli, i11) + _, err = optimism.PutInfo(t.etcdTestCli, i11) c.Assert(err, IsNil) - stm, _, err := optimism.GetAllSourceTables(etcdTestCli) + stm, _, err := optimism.GetAllSourceTables(t.etcdTestCli) c.Assert(err, IsNil) o.tk.Init(stm) } func (t *testOptimist) TestBuildLockWithInitSchema(c *C) { - defer clearOptimistTestSourceInfoOperation(c) + defer t.clearOptimistTestSourceInfoOperation(c) var ( logger = log.L() @@ -1160,18 +1168,18 @@ func (t *testOptimist) TestBuildLockWithInitSchema(c *C) { st1.AddTable("foo", "bar-1", downSchema, downTable) st2.AddTable("foo", "bar-1", downSchema, downTable) - c.Assert(o.Start(ctx, etcdTestCli), IsNil) - _, err := optimism.PutSourceTables(etcdTestCli, st1) + c.Assert(o.Start(ctx, t.etcdTestCli), IsNil) + _, err := optimism.PutSourceTables(t.etcdTestCli, st1) c.Assert(err, IsNil) - _, err = optimism.PutSourceTables(etcdTestCli, st2) + _, err = optimism.PutSourceTables(t.etcdTestCli, st2) c.Assert(err, IsNil) - _, err = optimism.PutInfo(etcdTestCli, infoDropB) + _, err = optimism.PutInfo(t.etcdTestCli, infoDropB) c.Assert(err, IsNil) - _, err = optimism.PutInfo(etcdTestCli, infoDropC) + _, err = optimism.PutInfo(t.etcdTestCli, infoDropC) c.Assert(err, IsNil) - stm, _, err := optimism.GetAllSourceTables(etcdTestCli) + stm, _, err := optimism.GetAllSourceTables(t.etcdTestCli) c.Assert(err, IsNil) o.tk.Init(stm) } diff --git a/dm/dm/master/shardddl/pessimist.go b/dm/dm/master/shardddl/pessimist.go index 61f58a566ed..19dbedeccd7 100644 --- a/dm/dm/master/shardddl/pessimist.go +++ b/dm/dm/master/shardddl/pessimist.go @@ -19,7 +19,7 @@ import ( "sync" "time" - "go.etcd.io/etcd/clientv3" + clientv3 "go.etcd.io/etcd/client/v3" "go.uber.org/zap" "github.com/pingcap/failpoint" diff --git a/dm/dm/master/shardddl/pessimist_test.go b/dm/dm/master/shardddl/pessimist_test.go index b2daee93b4c..71b75634328 100644 --- a/dm/dm/master/shardddl/pessimist_test.go +++ b/dm/dm/master/shardddl/pessimist_test.go @@ -21,9 +21,9 @@ import ( "time" . "github.com/pingcap/check" - "go.etcd.io/etcd/clientv3" - v3rpc "go.etcd.io/etcd/etcdserver/api/v3rpc/rpctypes" - "go.etcd.io/etcd/integration" + v3rpc "go.etcd.io/etcd/api/v3/v3rpc/rpctypes" + clientv3 "go.etcd.io/etcd/client/v3" + "go.etcd.io/etcd/tests/v3/integration" "github.com/pingcap/tiflow/dm/dm/common" "github.com/pingcap/tiflow/dm/dm/config" @@ -35,7 +35,6 @@ import ( ) var ( - etcdTestCli *clientv3.Client etcdErrCompacted = v3rpc.ErrCompacted tt *testing.T ) @@ -46,7 +45,10 @@ const ( restartNewInstance // restart with build a new instance. mock progress restore from failure ) -type testPessimist struct{} +type testPessimist struct { + mockCluster *integration.ClusterV3 + etcdTestCli *clientv3.Client +} var _ = Suite(&testPessimist{}) @@ -57,22 +59,28 @@ func TestShardDDL(t *testing.T) { } tt = t - mockCluster := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1}) - defer mockCluster.Terminate(t) - - etcdTestCli = mockCluster.RandClient() + integration.BeforeTestExternal(t) TestingT(t) } // clear keys in etcd test cluster. -func clearTestInfoOperation(c *C) { +func (t *testPessimist) clearTestInfoOperation(c *C) { clearInfo := clientv3.OpDelete(common.ShardDDLPessimismInfoKeyAdapter.Path(), clientv3.WithPrefix()) clearOp := clientv3.OpDelete(common.ShardDDLPessimismOperationKeyAdapter.Path(), clientv3.WithPrefix()) - _, err := etcdTestCli.Txn(context.Background()).Then(clearInfo, clearOp).Commit() + _, err := t.etcdTestCli.Txn(context.Background()).Then(clearInfo, clearOp).Commit() c.Assert(err, IsNil) } +func (t *testPessimist) SetUpSuite(c *C) { + t.mockCluster = integration.NewClusterV3(tt, &integration.ClusterConfig{Size: 1}) + t.etcdTestCli = t.mockCluster.RandClient() +} + +func (t *testPessimist) TearDownSuite(c *C) { + t.mockCluster.Terminate(tt) +} + func (t *testPessimist) TestPessimist(c *C) { t.testPessimistProgress(c, noRestart) t.testPessimistProgress(c, restartOnly) @@ -80,7 +88,7 @@ func (t *testPessimist) TestPessimist(c *C) { } func (t *testPessimist) testPessimistProgress(c *C, restart int) { - defer clearTestInfoOperation(c) + defer t.clearTestInfoOperation(c) var ( watchTimeout = 3 * time.Second @@ -117,11 +125,11 @@ func (t *testPessimist) testPessimistProgress(c *C, restart int) { switch restart { case restartOnly: p.Close() - c.Assert(p.Start(ctx, etcdTestCli), IsNil) + c.Assert(p.Start(ctx, t.etcdTestCli), IsNil) case restartNewInstance: p.Close() p = NewPessimist(&logger, sources) - c.Assert(p.Start(ctx, etcdTestCli), IsNil) + c.Assert(p.Start(ctx, t.etcdTestCli), IsNil) } } ) @@ -130,17 +138,17 @@ func (t *testPessimist) testPessimistProgress(c *C, restart int) { defer cancel() // CASE 1: start without any previous kv and no etcd operation. - c.Assert(p.Start(ctx, etcdTestCli), IsNil) + c.Assert(p.Start(ctx, t.etcdTestCli), IsNil) c.Assert(p.Locks(), HasLen, 0) p.Close() p.Close() // close multiple times. // CASE 2: start again without any previous kv. - c.Assert(p.Start(ctx, etcdTestCli), IsNil) + c.Assert(p.Start(ctx, t.etcdTestCli), IsNil) c.Assert(p.Locks(), HasLen, 0) // PUT i11, will create a lock but not synced. - _, err := pessimism.PutInfo(etcdTestCli, i11) + _, err := pessimism.PutInfo(t.etcdTestCli, i11) c.Assert(err, IsNil) c.Assert(utils.WaitSomething(30, 100*time.Millisecond, func() bool { return len(p.Locks()) == 1 @@ -151,7 +159,7 @@ func (t *testPessimist) testPessimistProgress(c *C, restart int) { c.Assert(remain, Equals, 1) // PUT i12, the lock will be synced, then an operation PUT for the owner will be triggered. - rev1, err := pessimism.PutInfo(etcdTestCli, i12) + rev1, err := pessimism.PutInfo(t.etcdTestCli, i12) c.Assert(err, IsNil) c.Assert(utils.WaitSomething(30, 100*time.Millisecond, func() bool { synced, _ = p.Locks()[ID1].IsSynced() @@ -162,7 +170,7 @@ func (t *testPessimist) testPessimistProgress(c *C, restart int) { opCh := make(chan pessimism.Operation, 10) errCh := make(chan error, 10) ctx2, cancel2 := context.WithTimeout(ctx, watchTimeout) - pessimism.WatchOperationPut(ctx2, etcdTestCli, task1, source1, rev1+1, opCh, errCh) + pessimism.WatchOperationPut(ctx2, t.etcdTestCli, task1, source1, rev1+1, opCh, errCh) cancel2() close(opCh) close(errCh) @@ -175,7 +183,7 @@ func (t *testPessimist) testPessimistProgress(c *C, restart int) { // mark exec operation for the owner as `done` (and delete the info). op11c := op11 op11c.Done = true - done, rev2, err := pessimism.PutOperationDeleteExistInfo(etcdTestCli, op11c, i11) + done, rev2, err := pessimism.PutOperationDeleteExistInfo(t.etcdTestCli, op11c, i11) c.Assert(err, IsNil) c.Assert(done, IsTrue) c.Assert(utils.WaitSomething(30, 100*time.Millisecond, func() bool { @@ -186,7 +194,7 @@ func (t *testPessimist) testPessimistProgress(c *C, restart int) { opCh = make(chan pessimism.Operation, 10) errCh = make(chan error, 10) ctx2, cancel2 = context.WithTimeout(ctx, watchTimeout) - pessimism.WatchOperationPut(ctx2, etcdTestCli, task1, source2, rev2+1, opCh, errCh) + pessimism.WatchOperationPut(ctx2, t.etcdTestCli, task1, source2, rev2+1, opCh, errCh) cancel2() close(opCh) close(errCh) @@ -200,7 +208,7 @@ func (t *testPessimist) testPessimistProgress(c *C, restart int) { // the lock should become resolved and deleted. op12c := op12 op12c.Done = true - done, _, err = pessimism.PutOperationDeleteExistInfo(etcdTestCli, op12c, i12) + done, _, err = pessimism.PutOperationDeleteExistInfo(t.etcdTestCli, op12c, i12) c.Assert(err, IsNil) c.Assert(done, IsTrue) c.Assert(utils.WaitSomething(50, 100*time.Millisecond, func() bool { @@ -211,9 +219,9 @@ func (t *testPessimist) testPessimistProgress(c *C, restart int) { c.Assert(p.ShowLocks("", nil), HasLen, 0) // PUT i21, i22, this will create a lock. - _, err = pessimism.PutInfo(etcdTestCli, i21) + _, err = pessimism.PutInfo(t.etcdTestCli, i21) c.Assert(err, IsNil) - _, err = pessimism.PutInfo(etcdTestCli, i22) + _, err = pessimism.PutInfo(t.etcdTestCli, i22) c.Assert(err, IsNil) c.Assert(utils.WaitSomething(30, 100*time.Millisecond, func() bool { lock := p.Locks()[ID2] @@ -254,7 +262,7 @@ func (t *testPessimist) testPessimistProgress(c *C, restart int) { c.Assert(p.ShowLocks("", []string{"not-exist"}), HasLen, 0) // PUT i23, then the lock will become synced. - rev3, err := pessimism.PutInfo(etcdTestCli, i23) + rev3, err := pessimism.PutInfo(t.etcdTestCli, i23) c.Assert(err, IsNil) c.Assert(utils.WaitSomething(30, 100*time.Millisecond, func() bool { synced, _ = p.Locks()[ID2].IsSynced() @@ -267,7 +275,7 @@ func (t *testPessimist) testPessimistProgress(c *C, restart int) { ctx2, cancel2 = context.WithTimeout(ctx, watchTimeout) // both source1 and source2 have shard DDL info exist, and neither of them have operation exist. // we must ensure source1 always become the owner of the lock. - pessimism.WatchOperationPut(ctx2, etcdTestCli, task2, source1, rev3+1, opCh, errCh) + pessimism.WatchOperationPut(ctx2, t.etcdTestCli, task2, source1, rev3+1, opCh, errCh) cancel2() close(opCh) close(errCh) @@ -288,7 +296,7 @@ func (t *testPessimist) testPessimistProgress(c *C, restart int) { // mark exec operation for the owner as `done` (and delete the info). op21c := op21 op21c.Done = true - done, _, err = pessimism.PutOperationDeleteExistInfo(etcdTestCli, op21c, i21) + done, _, err = pessimism.PutOperationDeleteExistInfo(t.etcdTestCli, op21c, i21) c.Assert(err, IsNil) c.Assert(done, IsTrue) c.Assert(utils.WaitSomething(30, 100*time.Millisecond, func() bool { @@ -306,7 +314,7 @@ func (t *testPessimist) testPessimistProgress(c *C, restart int) { // mark exec operation for one non-owner as `done` (and delete the info). op22c := pessimism.NewOperation(ID2, task2, source2, DDLs, false, true) - done, _, err = pessimism.PutOperationDeleteExistInfo(etcdTestCli, op22c, i22) + done, _, err = pessimism.PutOperationDeleteExistInfo(t.etcdTestCli, op22c, i22) c.Assert(err, IsNil) c.Assert(done, IsTrue) c.Assert(utils.WaitSomething(30, 100*time.Millisecond, func() bool { @@ -326,7 +334,7 @@ func (t *testPessimist) testPessimistProgress(c *C, restart int) { // mark skip operation for the non-owner as `done` (and delete the info). // the lock should become resolved and deleted. op23c := pessimism.NewOperation(ID2, task2, source3, DDLs, false, true) - done, _, err = pessimism.PutOperationDeleteExistInfo(etcdTestCli, op23c, i23) + done, _, err = pessimism.PutOperationDeleteExistInfo(t.etcdTestCli, op23c, i23) c.Assert(err, IsNil) c.Assert(done, IsTrue) c.Assert(utils.WaitSomething(30, 100*time.Millisecond, func() bool { @@ -343,7 +351,7 @@ func (t *testPessimist) testPessimistProgress(c *C, restart int) { func (t *testPessimist) TestSourceReEntrant(c *C) { // sources (owner or non-owner) may be interrupted and re-run the sequence again. - defer clearTestInfoOperation(c) + defer t.clearTestInfoOperation(c) var ( watchTimeout = 3 * time.Second @@ -375,14 +383,14 @@ func (t *testPessimist) TestSourceReEntrant(c *C) { defer cancel() // 0. start the pessimist. - c.Assert(p.Start(ctx, etcdTestCli), IsNil) + c.Assert(p.Start(ctx, t.etcdTestCli), IsNil) c.Assert(p.Locks(), HasLen, 0) defer p.Close() // 1. PUT i11 and i12, will create a lock but not synced. - _, err := pessimism.PutInfo(etcdTestCli, i11) + _, err := pessimism.PutInfo(t.etcdTestCli, i11) c.Assert(err, IsNil) - _, err = pessimism.PutInfo(etcdTestCli, i12) + _, err = pessimism.PutInfo(t.etcdTestCli, i12) c.Assert(err, IsNil) c.Assert(utils.WaitSomething(30, 100*time.Millisecond, func() bool { lock := p.Locks()[ID] @@ -394,11 +402,11 @@ func (t *testPessimist) TestSourceReEntrant(c *C) { }), IsTrue) // 2. re-PUT i11, to simulate the re-entrant of the owner before the lock become synced. - rev1, err := pessimism.PutInfo(etcdTestCli, i11) + rev1, err := pessimism.PutInfo(t.etcdTestCli, i11) c.Assert(err, IsNil) // 3. re-PUT i12, to simulate the re-entrant of the non-owner before the lock become synced. - rev2, err := pessimism.PutInfo(etcdTestCli, i12) + rev2, err := pessimism.PutInfo(t.etcdTestCli, i12) c.Assert(err, IsNil) // 4. wait exec operation for the owner become available. @@ -409,7 +417,7 @@ func (t *testPessimist) TestSourceReEntrant(c *C) { opCh := make(chan pessimism.Operation, 10) errCh := make(chan error, 10) ctx2, cancel2 := context.WithTimeout(ctx, watchTimeout) - pessimism.WatchOperationPut(ctx2, etcdTestCli, task, source1, rev1+1, opCh, errCh) + pessimism.WatchOperationPut(ctx2, t.etcdTestCli, task, source1, rev1+1, opCh, errCh) cancel2() close(opCh) close(errCh) @@ -421,19 +429,19 @@ func (t *testPessimist) TestSourceReEntrant(c *C) { }() // 5. put i13, the lock will become synced, then an operation PUT for the owner will be triggered. - _, err = pessimism.PutInfo(etcdTestCli, i13) + _, err = pessimism.PutInfo(t.etcdTestCli, i13) c.Assert(err, IsNil) wg.Wait() // 6. re-PUT i11, to simulate the re-entrant of the owner after the lock become synced. - rev1, err = pessimism.PutInfo(etcdTestCli, i11) + rev1, err = pessimism.PutInfo(t.etcdTestCli, i11) c.Assert(err, IsNil) // 8. wait exec operation for the owner become available again (with new revision). opCh := make(chan pessimism.Operation, 10) errCh := make(chan error, 10) ctx2, cancel2 := context.WithTimeout(ctx, watchTimeout) - pessimism.WatchOperationPut(ctx2, etcdTestCli, task, source1, rev1+1, opCh, errCh) + pessimism.WatchOperationPut(ctx2, t.etcdTestCli, task, source1, rev1+1, opCh, errCh) cancel2() close(opCh) close(errCh) @@ -450,7 +458,7 @@ func (t *testPessimist) TestSourceReEntrant(c *C) { opCh = make(chan pessimism.Operation, 10) errCh = make(chan error, 10) ctx2, cancel2 = context.WithTimeout(ctx, watchTimeout) - pessimism.WatchOperationPut(ctx2, etcdTestCli, task, source2, rev2+1, opCh, errCh) + pessimism.WatchOperationPut(ctx2, t.etcdTestCli, task, source2, rev2+1, opCh, errCh) cancel2() close(opCh) close(errCh) @@ -464,7 +472,7 @@ func (t *testPessimist) TestSourceReEntrant(c *C) { // 10. mark exec operation for the owner as `done` (and delete the info). op11c := op11 op11c.Done = true - done, _, err := pessimism.PutOperationDeleteExistInfo(etcdTestCli, op11c, i11) + done, _, err := pessimism.PutOperationDeleteExistInfo(t.etcdTestCli, op11c, i11) c.Assert(err, IsNil) c.Assert(done, IsTrue) c.Assert(utils.WaitSomething(30, 100*time.Millisecond, func() bool { @@ -473,14 +481,14 @@ func (t *testPessimist) TestSourceReEntrant(c *C) { wg.Wait() // 11. re-PUT i12, to simulate the re-entrant of the non-owner after the lock become synced. - rev2, err = pessimism.PutInfo(etcdTestCli, i12) + rev2, err = pessimism.PutInfo(t.etcdTestCli, i12) c.Assert(err, IsNil) // 12. wait skip operation for the non-owner become available again (with new revision, without existing done). opCh = make(chan pessimism.Operation, 10) errCh = make(chan error, 10) ctx2, cancel2 = context.WithTimeout(ctx, watchTimeout) - pessimism.WatchOperationPut(ctx2, etcdTestCli, task, source2, rev2+1, opCh, errCh) + pessimism.WatchOperationPut(ctx2, t.etcdTestCli, task, source2, rev2+1, opCh, errCh) cancel2() close(opCh) close(errCh) @@ -493,7 +501,7 @@ func (t *testPessimist) TestSourceReEntrant(c *C) { // 13. mark skip operation for the non-owner as `done` (and delete the info). op12c := op12 op12c.Done = true - done, _, err = pessimism.PutOperationDeleteExistInfo(etcdTestCli, op12c, i12) + done, _, err = pessimism.PutOperationDeleteExistInfo(t.etcdTestCli, op12c, i12) c.Assert(err, IsNil) c.Assert(done, IsTrue) c.Assert(utils.WaitSomething(30, 100*time.Millisecond, func() bool { @@ -501,14 +509,14 @@ func (t *testPessimist) TestSourceReEntrant(c *C) { }), IsTrue) // 14. re-PUT i13, to simulate the re-entrant of the owner after the lock become synced. - rev3, err := pessimism.PutInfo(etcdTestCli, i13) + rev3, err := pessimism.PutInfo(t.etcdTestCli, i13) c.Assert(err, IsNil) // 15. wait skip operation for the non-owner become available again (with new revision, with existing done). opCh = make(chan pessimism.Operation, 10) errCh = make(chan error, 10) ctx2, cancel2 = context.WithTimeout(ctx, watchTimeout) - pessimism.WatchOperationPut(ctx2, etcdTestCli, task, source3, rev3+1, opCh, errCh) + pessimism.WatchOperationPut(ctx2, t.etcdTestCli, task, source3, rev3+1, opCh, errCh) cancel2() close(opCh) close(errCh) @@ -522,7 +530,7 @@ func (t *testPessimist) TestSourceReEntrant(c *C) { // the lock should become resolved now. op13c := op13 op13c.Done = true - done, _, err = pessimism.PutOperationDeleteExistInfo(etcdTestCli, op13c, i13) + done, _, err = pessimism.PutOperationDeleteExistInfo(t.etcdTestCli, op13c, i13) c.Assert(err, IsNil) c.Assert(done, IsTrue) c.Assert(utils.WaitSomething(30, 100*time.Millisecond, func() bool { @@ -534,7 +542,7 @@ func (t *testPessimist) TestSourceReEntrant(c *C) { func (t *testPessimist) TestUnlockSourceMissBeforeSynced(c *C) { // some sources may be deleted (miss) before the lock become synced. - defer clearTestInfoOperation(c) + defer t.clearTestInfoOperation(c) oriUnlockWaitOwnerInterval := unlockWaitInterval unlockWaitInterval = 100 * time.Millisecond @@ -572,7 +580,7 @@ func (t *testPessimist) TestUnlockSourceMissBeforeSynced(c *C) { // 0. start the pessimist. c.Assert(terror.ErrMasterPessimistNotStarted.Equal(p.UnlockLock(ctx, ID, "", false)), IsTrue) - c.Assert(p.Start(ctx, etcdTestCli), IsNil) + c.Assert(p.Start(ctx, t.etcdTestCli), IsNil) c.Assert(p.Locks(), HasLen, 0) defer p.Close() @@ -581,9 +589,9 @@ func (t *testPessimist) TestUnlockSourceMissBeforeSynced(c *C) { // 1. PUT i11 & i12, will create a lock but now synced. // not PUT info for source3 to simulate the deletion of it. - _, err := pessimism.PutInfo(etcdTestCli, i11) + _, err := pessimism.PutInfo(t.etcdTestCli, i11) c.Assert(err, IsNil) - rev1, err := pessimism.PutInfo(etcdTestCli, i12) + rev1, err := pessimism.PutInfo(t.etcdTestCli, i12) c.Assert(err, IsNil) c.Assert(utils.WaitSomething(30, 100*time.Millisecond, func() bool { if len(p.Locks()) != 1 { @@ -627,7 +635,7 @@ func (t *testPessimist) TestUnlockSourceMissBeforeSynced(c *C) { func (t *testPessimist) TestUnlockSourceInterrupt(c *C) { // operations may be done but not be deleted, and then interrupted. - defer clearTestInfoOperation(c) + defer t.clearTestInfoOperation(c) oriUnlockWaitOwnerInterval := unlockWaitInterval unlockWaitInterval = 100 * time.Millisecond @@ -663,15 +671,15 @@ func (t *testPessimist) TestUnlockSourceInterrupt(c *C) { defer cancel() // 0. start the pessimist. - c.Assert(p.Start(ctx, etcdTestCli), IsNil) + c.Assert(p.Start(ctx, t.etcdTestCli), IsNil) c.Assert(p.Locks(), HasLen, 0) defer p.Close() // CASE 1: owner interrupted. // 1. PUT i11 & i12, will create a lock and synced. - rev1, err := pessimism.PutInfo(etcdTestCli, i11) + rev1, err := pessimism.PutInfo(t.etcdTestCli, i11) c.Assert(err, IsNil) - _, err = pessimism.PutInfo(etcdTestCli, i12) + _, err = pessimism.PutInfo(t.etcdTestCli, i12) c.Assert(err, IsNil) c.Assert(utils.WaitSomething(30, 100*time.Millisecond, func() bool { if len(p.Locks()) != 1 { @@ -690,7 +698,7 @@ func (t *testPessimist) TestUnlockSourceInterrupt(c *C) { opCh := make(chan pessimism.Operation, 10) errCh := make(chan error, 10) ctx2, cancel2 := context.WithTimeout(ctx, watchTimeout) - pessimism.WatchOperationPut(ctx2, etcdTestCli, task, "", rev1+1, opCh, errCh) + pessimism.WatchOperationPut(ctx2, t.etcdTestCli, task, "", rev1+1, opCh, errCh) cancel2() close(opCh) close(errCh) @@ -712,9 +720,9 @@ func (t *testPessimist) TestUnlockSourceInterrupt(c *C) { // CASE 2: non-owner interrupted. // 1. PUT i11 & i12, will create a lock and synced. - rev1, err = pessimism.PutInfo(etcdTestCli, i11) + rev1, err = pessimism.PutInfo(t.etcdTestCli, i11) c.Assert(err, IsNil) - _, err = pessimism.PutInfo(etcdTestCli, i12) + _, err = pessimism.PutInfo(t.etcdTestCli, i12) c.Assert(err, IsNil) c.Assert(utils.WaitSomething(30, 100*time.Millisecond, func() bool { if len(p.Locks()) != 1 { @@ -743,7 +751,7 @@ func (t *testPessimist) TestUnlockSourceInterrupt(c *C) { func (t *testPessimist) TestUnlockSourceOwnerRemoved(c *C) { // the owner may be deleted before the lock become synced. - defer clearTestInfoOperation(c) + defer t.clearTestInfoOperation(c) oriUnlockWaitOwnerInterval := unlockWaitInterval unlockWaitInterval = 100 * time.Millisecond @@ -780,7 +788,7 @@ func (t *testPessimist) TestUnlockSourceOwnerRemoved(c *C) { defer cancel() // 0. start the pessimist. - c.Assert(p.Start(ctx, etcdTestCli), IsNil) + c.Assert(p.Start(ctx, t.etcdTestCli), IsNil) c.Assert(p.Locks(), HasLen, 0) defer p.Close() @@ -788,9 +796,9 @@ func (t *testPessimist) TestUnlockSourceOwnerRemoved(c *C) { c.Assert(terror.ErrMasterLockNotFound.Equal(p.UnlockLock(ctx, ID, "", false)), IsTrue) // 1. PUT i11 & i12, will create a lock but now synced. - _, err := pessimism.PutInfo(etcdTestCli, i11) + _, err := pessimism.PutInfo(t.etcdTestCli, i11) c.Assert(err, IsNil) - rev1, err := pessimism.PutInfo(etcdTestCli, i12) + rev1, err := pessimism.PutInfo(t.etcdTestCli, i12) c.Assert(err, IsNil) c.Assert(utils.WaitSomething(30, 100*time.Millisecond, func() bool { if len(p.Locks()) != 1 { @@ -830,7 +838,7 @@ func (t *testPessimist) TestUnlockSourceOwnerRemoved(c *C) { } func (t *testPessimist) TestMeetEtcdCompactError(c *C) { - defer clearTestInfoOperation(c) + defer t.clearTestInfoOperation(c) var ( watchTimeout = 3 * time.Second @@ -869,37 +877,37 @@ func (t *testPessimist) TestMeetEtcdCompactError(c *C) { ctx, cancel := context.WithCancel(context.Background()) defer cancel() - p.cli = etcdTestCli + p.cli = t.etcdTestCli for i := 0; i <= 1; i++ { // i == 0, watch info is compacted; i == 1, watch operation is compacted // step 1: trigger an etcd compaction if i == 0 { - revCompacted, err = pessimism.PutInfo(etcdTestCli, i11) + revCompacted, err = pessimism.PutInfo(t.etcdTestCli, i11) } else { var putted bool - revCompacted, putted, err = pessimism.PutOperations(etcdTestCli, false, op) + revCompacted, putted, err = pessimism.PutOperations(t.etcdTestCli, false, op) c.Assert(putted, IsTrue) } c.Assert(err, IsNil) if i == 0 { - _, err = pessimism.DeleteInfosOperations(etcdTestCli, []pessimism.Info{i11}, []pessimism.Operation{}) + _, err = pessimism.DeleteInfosOperations(t.etcdTestCli, []pessimism.Info{i11}, []pessimism.Operation{}) } else { - _, err = pessimism.DeleteOperations(etcdTestCli, op) + _, err = pessimism.DeleteOperations(t.etcdTestCli, op) } c.Assert(err, IsNil) - revThreshold, err := pessimism.PutInfo(etcdTestCli, i11) + revThreshold, err := pessimism.PutInfo(t.etcdTestCli, i11) c.Assert(err, IsNil) - _, err = etcdTestCli.Compact(ctx, revThreshold) + _, err = t.etcdTestCli.Compact(ctx, revThreshold) c.Assert(err, IsNil) infoCh = make(chan pessimism.Info, 10) errCh = make(chan error, 10) ctx1, cancel1 := context.WithTimeout(ctx, time.Second) if i == 0 { - pessimism.WatchInfoPut(ctx1, etcdTestCli, revCompacted, infoCh, errCh) + pessimism.WatchInfoPut(ctx1, t.etcdTestCli, revCompacted, infoCh, errCh) } else { - pessimism.WatchOperationPut(ctx1, etcdTestCli, "", "", revCompacted, opCh, errCh) + pessimism.WatchOperationPut(ctx1, t.etcdTestCli, "", "", revCompacted, opCh, errCh) } cancel1() select { @@ -919,7 +927,7 @@ func (t *testPessimist) TestMeetEtcdCompactError(c *C) { if i == 1 { rev1, rev2 = rev2, rev1 } - c.Assert(p.run(ctx2, etcdTestCli, rev1, rev2), IsNil) + c.Assert(p.run(ctx2, t.etcdTestCli, rev1, rev2), IsNil) }() // PUT i11, will create a lock but not synced. c.Assert(utils.WaitSomething(30, 100*time.Millisecond, func() bool { @@ -931,7 +939,7 @@ func (t *testPessimist) TestMeetEtcdCompactError(c *C) { c.Assert(remain, Equals, 1) // PUT i12, the lock will be synced, then an operation PUT for the owner will be triggered. - rev1, err := pessimism.PutInfo(etcdTestCli, i12) + rev1, err := pessimism.PutInfo(t.etcdTestCli, i12) c.Assert(err, IsNil) c.Assert(utils.WaitSomething(30, 100*time.Millisecond, func() bool { synced, _ = p.Locks()[ID1].IsSynced() @@ -942,7 +950,7 @@ func (t *testPessimist) TestMeetEtcdCompactError(c *C) { opCh = make(chan pessimism.Operation, 10) errCh = make(chan error, 10) ctx3, cancel3 := context.WithTimeout(ctx, watchTimeout) - pessimism.WatchOperationPut(ctx3, etcdTestCli, task1, source1, rev1+1, opCh, errCh) + pessimism.WatchOperationPut(ctx3, t.etcdTestCli, task1, source1, rev1+1, opCh, errCh) cancel3() close(opCh) close(errCh) @@ -955,7 +963,7 @@ func (t *testPessimist) TestMeetEtcdCompactError(c *C) { // mark exec operation for the owner as `done` (and delete the info). op11c := op11 op11c.Done = true - done, rev2, err := pessimism.PutOperationDeleteExistInfo(etcdTestCli, op11c, i11) + done, rev2, err := pessimism.PutOperationDeleteExistInfo(t.etcdTestCli, op11c, i11) c.Assert(err, IsNil) c.Assert(done, IsTrue) c.Assert(utils.WaitSomething(30, 100*time.Millisecond, func() bool { @@ -966,7 +974,7 @@ func (t *testPessimist) TestMeetEtcdCompactError(c *C) { opCh = make(chan pessimism.Operation, 10) errCh = make(chan error, 10) ctx3, cancel3 = context.WithTimeout(ctx, watchTimeout) - pessimism.WatchOperationPut(ctx3, etcdTestCli, task1, source2, rev2+1, opCh, errCh) + pessimism.WatchOperationPut(ctx3, t.etcdTestCli, task1, source2, rev2+1, opCh, errCh) cancel3() close(opCh) close(errCh) @@ -980,7 +988,7 @@ func (t *testPessimist) TestMeetEtcdCompactError(c *C) { // the lock should become resolved and deleted. op12c := op12 op12c.Done = true - done, _, err = pessimism.PutOperationDeleteExistInfo(etcdTestCli, op12c, i12) + done, _, err = pessimism.PutOperationDeleteExistInfo(t.etcdTestCli, op12c, i12) c.Assert(err, IsNil) c.Assert(done, IsTrue) c.Assert(utils.WaitSomething(30, 100*time.Millisecond, func() bool { @@ -1007,7 +1015,7 @@ func (t *testPessimist) putDoneForSource( wg.Add(2) go func() { defer wg.Done() - pessimism.WatchOperationPut(ctx2, etcdTestCli, task, source, watchRev, opCh, errCh) + pessimism.WatchOperationPut(ctx2, t.etcdTestCli, task, source, watchRev, opCh, errCh) close(opCh) close(errCh) }() @@ -1024,7 +1032,7 @@ func (t *testPessimist) putDoneForSource( c.Assert(op.Exec, Equals, exec) c.Assert(op.Done, IsFalse) op.Done = true - done, _, err := pessimism.PutOperationDeleteExistInfo(etcdTestCli, op, info) + done, _, err := pessimism.PutOperationDeleteExistInfo(t.etcdTestCli, op, info) c.Assert(err, IsNil) c.Assert(done, IsTrue) case err := <-errCh: @@ -1036,10 +1044,10 @@ func (t *testPessimist) putDoneForSource( func (t *testPessimist) noLockExist(c *C, p *Pessimist) { c.Assert(p.Locks(), HasLen, 0) - ifm, _, err := pessimism.GetAllInfo(etcdTestCli) + ifm, _, err := pessimism.GetAllInfo(t.etcdTestCli) c.Assert(err, IsNil) c.Assert(ifm, HasLen, 0) - opm, _, err := pessimism.GetAllOperations(etcdTestCli) + opm, _, err := pessimism.GetAllOperations(t.etcdTestCli) c.Assert(err, IsNil) c.Assert(opm, HasLen, 0) } diff --git a/dm/dm/worker/server.go b/dm/dm/worker/server.go index 6cfcbd481bc..532fd7825f2 100644 --- a/dm/dm/worker/server.go +++ b/dm/dm/worker/server.go @@ -36,7 +36,7 @@ import ( "github.com/pingcap/errors" toolutils "github.com/pingcap/tidb-tools/pkg/utils" "github.com/soheilhy/cmux" - "go.etcd.io/etcd/clientv3" + clientv3 "go.etcd.io/etcd/client/v3" "go.uber.org/atomic" "go.uber.org/zap" "google.golang.org/grpc" diff --git a/dm/dm/worker/server_test.go b/dm/dm/worker/server_test.go index 984240f3846..da1a6e5f022 100644 --- a/dm/dm/worker/server_test.go +++ b/dm/dm/worker/server_test.go @@ -26,9 +26,9 @@ import ( . "github.com/pingcap/check" "github.com/pingcap/failpoint" "github.com/tikv/pd/pkg/tempurl" - "go.etcd.io/etcd/clientv3" - "go.etcd.io/etcd/embed" - v3rpc "go.etcd.io/etcd/etcdserver/api/v3rpc/rpctypes" + v3rpc "go.etcd.io/etcd/api/v3/v3rpc/rpctypes" + clientv3 "go.etcd.io/etcd/client/v3" + "go.etcd.io/etcd/server/v3/embed" "google.golang.org/grpc" "github.com/pingcap/tiflow/dm/dm/config" @@ -45,9 +45,7 @@ import ( // do not forget to update this path if the file removed/renamed. const ( - sourceSampleFile = "./source.yaml" - subtaskSampleFile = "./subtask.toml" - mydumperPath = "../../bin/mydumper" + mydumperPath = "../../bin/mydumper" ) var etcdErrCompacted = v3rpc.ErrCompacted @@ -109,7 +107,6 @@ func (t *testServer) TestServer(c *C) { etcdDir := c.MkDir() ETCD, err := createMockETCD(etcdDir, "http://"+masterAddr) c.Assert(err, IsNil) - defer ETCD.Close() cfg := NewConfig() c.Assert(cfg.Parse([]string{"-config=./dm-worker.toml"}), IsNil) cfg.Join = masterAddr @@ -166,7 +163,7 @@ func (t *testServer) TestServer(c *C) { // start task subtaskCfg := config.SubTaskConfig{} - err = subtaskCfg.DecodeFile(subtaskSampleFile, true) + err = subtaskCfg.Decode(config.SampleSubtaskConfig, true) c.Assert(err, IsNil) subtaskCfg.MydumperPath = mydumperPath @@ -638,7 +635,7 @@ func checkRelayStatus(cli pb.WorkerClient, expect pb.Stage) bool { } func loadSourceConfigWithoutPassword(c *C) *config.SourceConfig { - sourceCfg, err := config.LoadFromFile(sourceSampleFile) + sourceCfg, err := config.ParseYamlAndVerify(config.SampleSourceConfig) c.Assert(err, IsNil) sourceCfg.From.Password = "" // no password set return sourceCfg diff --git a/dm/dm/worker/source_worker.go b/dm/dm/worker/source_worker.go index c6ec91c9a30..d5cf627c957 100644 --- a/dm/dm/worker/source_worker.go +++ b/dm/dm/worker/source_worker.go @@ -24,7 +24,7 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/failpoint" bf "github.com/pingcap/tidb-tools/pkg/binlog-filter" - "go.etcd.io/etcd/clientv3" + clientv3 "go.etcd.io/etcd/client/v3" "go.uber.org/atomic" "go.uber.org/zap" diff --git a/dm/dm/worker/source_worker_test.go b/dm/dm/worker/source_worker_test.go index 3c435149593..288b3b8fd97 100644 --- a/dm/dm/worker/source_worker_test.go +++ b/dm/dm/worker/source_worker_test.go @@ -24,7 +24,7 @@ import ( . "github.com/pingcap/check" "github.com/pingcap/failpoint" "github.com/tikv/pd/pkg/tempurl" - "go.etcd.io/etcd/clientv3" + clientv3 "go.etcd.io/etcd/client/v3" "github.com/pingcap/tiflow/dm/dm/config" "github.com/pingcap/tiflow/dm/dm/pb" @@ -201,7 +201,7 @@ func (t *testServer2) TestTaskAutoResume(c *C) { }), IsTrue) // start task var subtaskCfg config.SubTaskConfig - c.Assert(subtaskCfg.DecodeFile("./subtask.toml", true), IsNil) + c.Assert(subtaskCfg.Decode(config.SampleSubtaskConfig, true), IsNil) c.Assert(err, IsNil) subtaskCfg.Mode = "full" subtaskCfg.Timezone = "UTC" @@ -296,7 +296,7 @@ func (t *testWorkerFunctionalities) TestWorkerFunctionalities(c *C) { sourceCfg.EnableRelay = false subtaskCfg := config.SubTaskConfig{} - err = subtaskCfg.DecodeFile(subtaskSampleFile, true) + err = subtaskCfg.Decode(config.SampleSubtaskConfig, true) c.Assert(err, IsNil) // start worker @@ -483,7 +483,7 @@ func (t *testWorkerEtcdCompact) TestWatchSubtaskStageEtcdCompact(c *C) { }), IsTrue) // step 2: Put a subtask config and subtask stage to this source, then delete it subtaskCfg := config.SubTaskConfig{} - err = subtaskCfg.DecodeFile(subtaskSampleFile, true) + err = subtaskCfg.Decode(config.SampleSubtaskConfig, true) c.Assert(err, IsNil) subtaskCfg.MydumperPath = mydumperPath @@ -603,7 +603,7 @@ func (t *testWorkerEtcdCompact) TestWatchValidatorStageEtcdCompact(c *C) { // // step 2: Put a subtask config and subtask stage to this source, then delete it subtaskCfg := config.SubTaskConfig{} - err = subtaskCfg.DecodeFile(subtaskSampleFile, true) + err = subtaskCfg.Decode(config.SampleSubtaskConfig, true) c.Assert(err, IsNil) subtaskCfg.MydumperPath = mydumperPath subtaskCfg.ValidatorCfg = config.ValidatorConfig{Mode: config.ValidationNone} diff --git a/dm/dm/worker/subtask.go b/dm/dm/worker/subtask.go index ad4e26260b0..a2ade8ee8ee 100644 --- a/dm/dm/worker/subtask.go +++ b/dm/dm/worker/subtask.go @@ -22,7 +22,7 @@ import ( "github.com/go-mysql-org/go-mysql/mysql" "github.com/pingcap/failpoint" "github.com/prometheus/client_golang/prometheus" - "go.etcd.io/etcd/clientv3" + clientv3 "go.etcd.io/etcd/client/v3" "go.uber.org/atomic" "go.uber.org/zap" diff --git a/dm/dm/worker/subtask_test.go b/dm/dm/worker/subtask_test.go index 8fd1b6eb166..47ea2a7b4d7 100644 --- a/dm/dm/worker/subtask_test.go +++ b/dm/dm/worker/subtask_test.go @@ -30,7 +30,7 @@ import ( . "github.com/pingcap/check" "github.com/pingcap/errors" - "go.etcd.io/etcd/clientv3" + clientv3 "go.etcd.io/etcd/client/v3" ) const ( diff --git a/dm/dm/worker/utils.go b/dm/dm/worker/utils.go index 0b4aad256b8..9d569d3e56e 100644 --- a/dm/dm/worker/utils.go +++ b/dm/dm/worker/utils.go @@ -14,7 +14,7 @@ package worker import ( - "go.etcd.io/etcd/clientv3" + clientv3 "go.etcd.io/etcd/client/v3" "github.com/pingcap/tiflow/dm/dm/config" "github.com/pingcap/tiflow/dm/dm/pb" diff --git a/dm/dm/worker/utils_test.go b/dm/dm/worker/utils_test.go index 0a63f9eb3ff..c77416d27c3 100644 --- a/dm/dm/worker/utils_test.go +++ b/dm/dm/worker/utils_test.go @@ -17,7 +17,7 @@ import ( "testing" "github.com/stretchr/testify/require" - "go.etcd.io/etcd/integration" + "go.etcd.io/etcd/tests/v3/integration" "github.com/pingcap/tiflow/dm/dm/config" "github.com/pingcap/tiflow/dm/dm/pb" @@ -25,6 +25,7 @@ import ( ) func TestGetExpectValidatorStage(t *testing.T) { + integration.BeforeTestExternal(t) mockCluster := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1}) defer mockCluster.Terminate(t) @@ -33,7 +34,7 @@ func TestGetExpectValidatorStage(t *testing.T) { require.Nil(t, ha.ClearTestInfoOperation(etcdTestCli)) }() cfg := config.SubTaskConfig{} - require.Nil(t, cfg.DecodeFile("subtask.toml", true)) + require.Nil(t, cfg.Decode(config.SampleSubtaskConfig, true)) source := cfg.SourceID task := cfg.Name stage := ha.NewSubTaskStage(pb.Stage_Running, source, task) diff --git a/dm/loader/lightning.go b/dm/loader/lightning.go index 94aba624773..98eb68cd49e 100644 --- a/dm/loader/lightning.go +++ b/dm/loader/lightning.go @@ -24,7 +24,7 @@ import ( "github.com/pingcap/tidb/br/pkg/lightning" "github.com/pingcap/tidb/br/pkg/lightning/checkpoints" lcfg "github.com/pingcap/tidb/br/pkg/lightning/config" - "go.etcd.io/etcd/clientv3" + clientv3 "go.etcd.io/etcd/client/v3" "go.uber.org/atomic" "go.uber.org/zap" diff --git a/dm/loader/loader.go b/dm/loader/loader.go index 4761f1b22cd..3a66c268e09 100644 --- a/dm/loader/loader.go +++ b/dm/loader/loader.go @@ -25,7 +25,7 @@ import ( "sync" "time" - "go.etcd.io/etcd/clientv3" + clientv3 "go.etcd.io/etcd/client/v3" "golang.org/x/sync/errgroup" "github.com/pingcap/tiflow/dm/dm/config" diff --git a/dm/loader/util.go b/dm/loader/util.go index f132a2fab0a..db7d4669328 100644 --- a/dm/loader/util.go +++ b/dm/loader/util.go @@ -22,7 +22,7 @@ import ( "path/filepath" "strings" - "go.etcd.io/etcd/clientv3" + clientv3 "go.etcd.io/etcd/client/v3" "go.uber.org/zap" "github.com/pingcap/failpoint" diff --git a/dm/pkg/election/election.go b/dm/pkg/election/election.go index 6fd924deb49..7e610ecf1ff 100644 --- a/dm/pkg/election/election.go +++ b/dm/pkg/election/election.go @@ -23,9 +23,9 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/failpoint" - "go.etcd.io/etcd/clientv3" - "go.etcd.io/etcd/clientv3/concurrency" - "go.etcd.io/etcd/mvcc/mvccpb" + "go.etcd.io/etcd/api/v3/mvccpb" + clientv3 "go.etcd.io/etcd/client/v3" + "go.etcd.io/etcd/client/v3/concurrency" "go.uber.org/atomic" "go.uber.org/zap" diff --git a/dm/pkg/election/election_test.go b/dm/pkg/election/election_test.go index bac257439b6..22bd325962a 100644 --- a/dm/pkg/election/election_test.go +++ b/dm/pkg/election/election_test.go @@ -24,8 +24,8 @@ import ( . "github.com/pingcap/check" "github.com/pingcap/failpoint" "github.com/tikv/pd/pkg/tempurl" - "go.etcd.io/etcd/clientv3" - "go.etcd.io/etcd/embed" + clientv3 "go.etcd.io/etcd/client/v3" + "go.etcd.io/etcd/server/v3/embed" "github.com/pingcap/tiflow/dm/pkg/etcdutil" "github.com/pingcap/tiflow/dm/pkg/log" diff --git a/dm/pkg/etcdutil/etcdutil.go b/dm/pkg/etcdutil/etcdutil.go index f90b329e391..8ebc4e4a584 100644 --- a/dm/pkg/etcdutil/etcdutil.go +++ b/dm/pkg/etcdutil/etcdutil.go @@ -22,8 +22,8 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/failpoint" - "go.etcd.io/etcd/clientv3" - v3rpc "go.etcd.io/etcd/etcdserver/api/v3rpc/rpctypes" + v3rpc "go.etcd.io/etcd/api/v3/v3rpc/rpctypes" + clientv3 "go.etcd.io/etcd/client/v3" "go.uber.org/zap" tcontext "github.com/pingcap/tiflow/dm/pkg/context" diff --git a/dm/pkg/etcdutil/etcdutil_test.go b/dm/pkg/etcdutil/etcdutil_test.go index 5d2357c9a53..1f1a3c5ae80 100644 --- a/dm/pkg/etcdutil/etcdutil_test.go +++ b/dm/pkg/etcdutil/etcdutil_test.go @@ -24,12 +24,12 @@ import ( "github.com/pingcap/failpoint" "github.com/stretchr/testify/require" "github.com/tikv/pd/pkg/tempurl" - "go.etcd.io/etcd/clientv3" - "go.etcd.io/etcd/clientv3/clientv3util" - "go.etcd.io/etcd/embed" - v3rpc "go.etcd.io/etcd/etcdserver/api/v3rpc/rpctypes" - "go.etcd.io/etcd/etcdserver/etcdserverpb" - "go.etcd.io/etcd/integration" + "go.etcd.io/etcd/api/v3/etcdserverpb" + v3rpc "go.etcd.io/etcd/api/v3/v3rpc/rpctypes" + clientv3 "go.etcd.io/etcd/client/v3" + "go.etcd.io/etcd/client/v3/clientv3util" + "go.etcd.io/etcd/server/v3/embed" + "go.etcd.io/etcd/tests/v3/integration" "github.com/pingcap/tiflow/dm/pkg/log" "github.com/pingcap/tiflow/dm/pkg/terror" @@ -50,6 +50,7 @@ func (t *testEtcdUtilSuite) SetUpSuite(c *C) { } func TestSuite(t *testing.T) { + integration.BeforeTestExternal(t) // inject *testing.T to suite s := etcdTestSuite.(*testEtcdUtilSuite) s.testT = t diff --git a/dm/pkg/ha/bound.go b/dm/pkg/ha/bound.go index 1a6da3bd7af..11dce257773 100644 --- a/dm/pkg/ha/bound.go +++ b/dm/pkg/ha/bound.go @@ -19,8 +19,8 @@ import ( "time" "github.com/pingcap/failpoint" - "go.etcd.io/etcd/clientv3" - "go.etcd.io/etcd/mvcc/mvccpb" + "go.etcd.io/etcd/api/v3/mvccpb" + clientv3 "go.etcd.io/etcd/client/v3" "go.uber.org/zap" "github.com/pingcap/tiflow/dm/dm/common" diff --git a/dm/pkg/ha/keepalive.go b/dm/pkg/ha/keepalive.go index ada13ba86da..6eb5eea4905 100644 --- a/dm/pkg/ha/keepalive.go +++ b/dm/pkg/ha/keepalive.go @@ -19,8 +19,8 @@ import ( "sync/atomic" "time" - "go.etcd.io/etcd/clientv3" - "go.etcd.io/etcd/mvcc/mvccpb" + "go.etcd.io/etcd/api/v3/mvccpb" + clientv3 "go.etcd.io/etcd/client/v3" "go.uber.org/zap" "github.com/pingcap/tiflow/dm/dm/common" diff --git a/dm/pkg/ha/load_task.go b/dm/pkg/ha/load_task.go index 4d019600470..c26a5d22648 100644 --- a/dm/pkg/ha/load_task.go +++ b/dm/pkg/ha/load_task.go @@ -18,8 +18,8 @@ import ( "encoding/json" "fmt" - "go.etcd.io/etcd/clientv3" - "go.etcd.io/etcd/mvcc/mvccpb" + "go.etcd.io/etcd/api/v3/mvccpb" + clientv3 "go.etcd.io/etcd/client/v3" "github.com/pingcap/tiflow/dm/dm/common" "github.com/pingcap/tiflow/dm/pkg/etcdutil" diff --git a/dm/pkg/ha/openapi_task_config.go b/dm/pkg/ha/openapi_task_config.go index 009dc2304e3..8bb9f3721e4 100644 --- a/dm/pkg/ha/openapi_task_config.go +++ b/dm/pkg/ha/openapi_task_config.go @@ -16,8 +16,8 @@ package ha import ( "context" - "go.etcd.io/etcd/clientv3" - "go.etcd.io/etcd/clientv3/clientv3util" + clientv3 "go.etcd.io/etcd/client/v3" + "go.etcd.io/etcd/client/v3/clientv3util" "github.com/pingcap/tiflow/dm/dm/common" "github.com/pingcap/tiflow/dm/openapi" diff --git a/dm/pkg/ha/ops.go b/dm/pkg/ha/ops.go index b6e2d2eb519..bb826fa597c 100644 --- a/dm/pkg/ha/ops.go +++ b/dm/pkg/ha/ops.go @@ -14,8 +14,8 @@ package ha import ( - "go.etcd.io/etcd/clientv3" - "go.etcd.io/etcd/mvcc/mvccpb" + "go.etcd.io/etcd/api/v3/mvccpb" + clientv3 "go.etcd.io/etcd/client/v3" "github.com/pingcap/tiflow/dm/dm/config" "github.com/pingcap/tiflow/dm/pkg/etcdutil" diff --git a/dm/pkg/ha/ops_test.go b/dm/pkg/ha/ops_test.go index a4295a807a4..1818f6436f7 100644 --- a/dm/pkg/ha/ops_test.go +++ b/dm/pkg/ha/ops_test.go @@ -41,7 +41,7 @@ func (t *testForEtcd) TestOpsEtcd(c *C) { sourceCfg, err := config.LoadFromFile(sourceSampleFilePath) c.Assert(err, IsNil) sourceCfg.SourceID = source - c.Assert(subtaskCfg1.DecodeFile(subTaskSampleFile, true), IsNil) + c.Assert(subtaskCfg1.Decode(config.SampleSubtaskConfig, true), IsNil) subtaskCfg1.SourceID = source subtaskCfg1.Name = task1 c.Assert(subtaskCfg1.Adjust(true), IsNil) diff --git a/dm/pkg/ha/relay.go b/dm/pkg/ha/relay.go index 3b9bd029a60..0de8bd583a3 100644 --- a/dm/pkg/ha/relay.go +++ b/dm/pkg/ha/relay.go @@ -17,8 +17,8 @@ import ( "context" "time" - "go.etcd.io/etcd/clientv3" - "go.etcd.io/etcd/mvcc/mvccpb" + "go.etcd.io/etcd/api/v3/mvccpb" + clientv3 "go.etcd.io/etcd/client/v3" "go.uber.org/zap" "github.com/pingcap/tiflow/dm/dm/common" diff --git a/dm/pkg/ha/source.go b/dm/pkg/ha/source.go index 0901362c2c5..a96b6811d21 100644 --- a/dm/pkg/ha/source.go +++ b/dm/pkg/ha/source.go @@ -17,7 +17,7 @@ import ( "context" "github.com/pingcap/failpoint" - "go.etcd.io/etcd/clientv3" + clientv3 "go.etcd.io/etcd/client/v3" "github.com/pingcap/tiflow/dm/dm/common" "github.com/pingcap/tiflow/dm/dm/config" diff --git a/dm/pkg/ha/source_test.go b/dm/pkg/ha/source_test.go index 2dfd3120b21..7d7ea53f872 100644 --- a/dm/pkg/ha/source_test.go +++ b/dm/pkg/ha/source_test.go @@ -22,8 +22,8 @@ import ( "testing" . "github.com/pingcap/check" - "go.etcd.io/etcd/clientv3" - "go.etcd.io/etcd/integration" + clientv3 "go.etcd.io/etcd/client/v3" + "go.etcd.io/etcd/tests/v3/integration" "github.com/pingcap/tiflow/dm/dm/config" ) @@ -77,6 +77,7 @@ var ( ) func TestHA(t *testing.T) { + integration.BeforeTestExternal(t) mockCluster := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1}) defer mockCluster.Terminate(t) diff --git a/dm/pkg/ha/stage.go b/dm/pkg/ha/stage.go index 0cee6ce77ce..b56b806a304 100644 --- a/dm/pkg/ha/stage.go +++ b/dm/pkg/ha/stage.go @@ -17,8 +17,8 @@ import ( "context" "encoding/json" - "go.etcd.io/etcd/clientv3" - "go.etcd.io/etcd/mvcc/mvccpb" + "go.etcd.io/etcd/api/v3/mvccpb" + clientv3 "go.etcd.io/etcd/client/v3" "go.uber.org/zap" "github.com/pingcap/tiflow/dm/dm/common" diff --git a/dm/pkg/ha/stage_test.go b/dm/pkg/ha/stage_test.go index 946dea993fb..1fe6d2ecabb 100644 --- a/dm/pkg/ha/stage_test.go +++ b/dm/pkg/ha/stage_test.go @@ -218,7 +218,7 @@ func (t *testForEtcd) TestGetSubTaskStageConfigEtcd(c *C) { defer clearTestInfoOperation(c) cfg := config.SubTaskConfig{} - c.Assert(cfg.DecodeFile(subTaskSampleFile, true), IsNil) + c.Assert(cfg.Decode(config.SampleSubtaskConfig, true), IsNil) source := cfg.SourceID task := cfg.Name stage := NewSubTaskStage(pb.Stage_Running, source, task) diff --git a/dm/pkg/ha/subtask.go b/dm/pkg/ha/subtask.go index 80b3bdfd60e..1ca6b1b2194 100644 --- a/dm/pkg/ha/subtask.go +++ b/dm/pkg/ha/subtask.go @@ -16,7 +16,7 @@ package ha import ( "context" - "go.etcd.io/etcd/clientv3" + clientv3 "go.etcd.io/etcd/client/v3" "github.com/pingcap/tiflow/dm/dm/common" "github.com/pingcap/tiflow/dm/dm/config" diff --git a/dm/pkg/ha/subtask_test.go b/dm/pkg/ha/subtask_test.go index 5dc537bc131..0ea6b62c1e0 100644 --- a/dm/pkg/ha/subtask_test.go +++ b/dm/pkg/ha/subtask_test.go @@ -21,16 +21,11 @@ import ( "github.com/pingcap/tiflow/dm/dm/config" ) -const ( - // do not forget to update this path if the file removed/renamed. - subTaskSampleFile = "../../dm/worker/subtask.toml" -) - func (t *testForEtcd) TestSubTaskEtcd(c *C) { defer clearTestInfoOperation(c) cfg1 := config.SubTaskConfig{} - c.Assert(cfg1.DecodeFile(subTaskSampleFile, true), IsNil) + c.Assert(cfg1.Decode(config.SampleSubtaskConfig, true), IsNil) source := cfg1.SourceID taskName1 := cfg1.Name diff --git a/dm/pkg/ha/task_cli_args.go b/dm/pkg/ha/task_cli_args.go index 70f51e06484..2e3e0cf4d72 100644 --- a/dm/pkg/ha/task_cli_args.go +++ b/dm/pkg/ha/task_cli_args.go @@ -16,7 +16,7 @@ package ha import ( "context" - "go.etcd.io/etcd/clientv3" + clientv3 "go.etcd.io/etcd/client/v3" "github.com/pingcap/tiflow/dm/dm/common" "github.com/pingcap/tiflow/dm/dm/config" diff --git a/dm/pkg/ha/worker.go b/dm/pkg/ha/worker.go index e28f7663f19..52b89e57498 100644 --- a/dm/pkg/ha/worker.go +++ b/dm/pkg/ha/worker.go @@ -17,7 +17,7 @@ import ( "context" "encoding/json" - "go.etcd.io/etcd/clientv3" + clientv3 "go.etcd.io/etcd/client/v3" "github.com/pingcap/tiflow/dm/dm/common" "github.com/pingcap/tiflow/dm/pkg/etcdutil" diff --git a/dm/pkg/shardddl/optimism/column.go b/dm/pkg/shardddl/optimism/column.go index 396d4c05130..364ff04da3d 100644 --- a/dm/pkg/shardddl/optimism/column.go +++ b/dm/pkg/shardddl/optimism/column.go @@ -16,7 +16,7 @@ package optimism import ( "encoding/json" - "go.etcd.io/etcd/clientv3" + clientv3 "go.etcd.io/etcd/client/v3" "github.com/pingcap/tiflow/dm/dm/common" "github.com/pingcap/tiflow/dm/pkg/etcdutil" diff --git a/dm/pkg/shardddl/optimism/info.go b/dm/pkg/shardddl/optimism/info.go index bdb43c067ce..143b591b200 100644 --- a/dm/pkg/shardddl/optimism/info.go +++ b/dm/pkg/shardddl/optimism/info.go @@ -20,8 +20,8 @@ import ( "github.com/pingcap/tidb-tools/pkg/schemacmp" "github.com/pingcap/tidb/parser/model" - "go.etcd.io/etcd/clientv3" - "go.etcd.io/etcd/mvcc/mvccpb" + "go.etcd.io/etcd/api/v3/mvccpb" + clientv3 "go.etcd.io/etcd/client/v3" "github.com/pingcap/tiflow/dm/dm/common" "github.com/pingcap/tiflow/dm/pkg/etcdutil" diff --git a/dm/pkg/shardddl/optimism/info_test.go b/dm/pkg/shardddl/optimism/info_test.go index 44a35c1c41d..0035932400f 100644 --- a/dm/pkg/shardddl/optimism/info_test.go +++ b/dm/pkg/shardddl/optimism/info_test.go @@ -27,8 +27,8 @@ import ( "github.com/pingcap/tidb/parser/model" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/util/mock" - "go.etcd.io/etcd/clientv3" - "go.etcd.io/etcd/integration" + clientv3 "go.etcd.io/etcd/client/v3" + "go.etcd.io/etcd/tests/v3/integration" "github.com/pingcap/tiflow/dm/dm/common" "github.com/pingcap/tiflow/dm/pkg/etcdutil" @@ -37,6 +37,7 @@ import ( var etcdTestCli *clientv3.Client func TestInfo(t *testing.T) { + integration.BeforeTestExternal(t) mockCluster := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1}) defer mockCluster.Terminate(t) diff --git a/dm/pkg/shardddl/optimism/keeper.go b/dm/pkg/shardddl/optimism/keeper.go index f1f0d3dc57c..d796a1bdcb5 100644 --- a/dm/pkg/shardddl/optimism/keeper.go +++ b/dm/pkg/shardddl/optimism/keeper.go @@ -18,7 +18,7 @@ import ( "sync" "github.com/pingcap/tidb-tools/pkg/schemacmp" - "go.etcd.io/etcd/clientv3" + clientv3 "go.etcd.io/etcd/client/v3" "github.com/pingcap/tiflow/dm/dm/config" "github.com/pingcap/tiflow/dm/pkg/log" diff --git a/dm/pkg/shardddl/optimism/keeper_test.go b/dm/pkg/shardddl/optimism/keeper_test.go index 5aefe4173c2..d6d56fb4c39 100644 --- a/dm/pkg/shardddl/optimism/keeper_test.go +++ b/dm/pkg/shardddl/optimism/keeper_test.go @@ -20,7 +20,7 @@ import ( "github.com/pingcap/tidb/parser" "github.com/pingcap/tidb/parser/model" "github.com/pingcap/tidb/util/mock" - "go.etcd.io/etcd/integration" + "go.etcd.io/etcd/tests/v3/integration" "github.com/pingcap/tiflow/dm/dm/config" "github.com/pingcap/tiflow/dm/pkg/conn" @@ -32,6 +32,7 @@ type testKeeper struct{} var _ = Suite(&testKeeper{}) func TestKeeper(t *testing.T) { + integration.BeforeTestExternal(t) mockCluster := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1}) defer mockCluster.Terminate(t) diff --git a/dm/pkg/shardddl/optimism/lock.go b/dm/pkg/shardddl/optimism/lock.go index c08760cda87..4ed86860121 100644 --- a/dm/pkg/shardddl/optimism/lock.go +++ b/dm/pkg/shardddl/optimism/lock.go @@ -25,7 +25,7 @@ import ( "github.com/pingcap/tidb/parser" "github.com/pingcap/tidb/parser/ast" "github.com/pingcap/tidb/parser/model" - "go.etcd.io/etcd/clientv3" + clientv3 "go.etcd.io/etcd/client/v3" "go.uber.org/zap" "golang.org/x/net/context" diff --git a/dm/pkg/shardddl/optimism/lock_test.go b/dm/pkg/shardddl/optimism/lock_test.go index 40a590a7e11..c2f47c14774 100644 --- a/dm/pkg/shardddl/optimism/lock_test.go +++ b/dm/pkg/shardddl/optimism/lock_test.go @@ -25,7 +25,7 @@ import ( "github.com/pingcap/tidb/parser/ast" "github.com/pingcap/tidb/parser/model" "github.com/pingcap/tidb/util/mock" - "go.etcd.io/etcd/integration" + "go.etcd.io/etcd/tests/v3/integration" "github.com/pingcap/tiflow/dm/dm/config" "github.com/pingcap/tiflow/dm/pkg/conn" @@ -40,6 +40,7 @@ type testLock struct{} var _ = Suite(&testLock{}) func TestLock(t *testing.T) { + integration.BeforeTestExternal(t) mockCluster := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1}) defer mockCluster.Terminate(t) diff --git a/dm/pkg/shardddl/optimism/operation.go b/dm/pkg/shardddl/optimism/operation.go index 3c6b214c6f9..a7b72ea0f74 100644 --- a/dm/pkg/shardddl/optimism/operation.go +++ b/dm/pkg/shardddl/optimism/operation.go @@ -17,9 +17,9 @@ import ( "context" "encoding/json" - "go.etcd.io/etcd/clientv3" - "go.etcd.io/etcd/clientv3/clientv3util" - "go.etcd.io/etcd/mvcc/mvccpb" + "go.etcd.io/etcd/api/v3/mvccpb" + clientv3 "go.etcd.io/etcd/client/v3" + "go.etcd.io/etcd/client/v3/clientv3util" "github.com/pingcap/tiflow/dm/dm/common" "github.com/pingcap/tiflow/dm/pkg/etcdutil" diff --git a/dm/pkg/shardddl/optimism/ops.go b/dm/pkg/shardddl/optimism/ops.go index 28c92f3e495..d6322542fd8 100644 --- a/dm/pkg/shardddl/optimism/ops.go +++ b/dm/pkg/shardddl/optimism/ops.go @@ -14,7 +14,7 @@ package optimism import ( - "go.etcd.io/etcd/clientv3" + clientv3 "go.etcd.io/etcd/client/v3" "github.com/pingcap/tiflow/dm/dm/common" "github.com/pingcap/tiflow/dm/pkg/etcdutil" diff --git a/dm/pkg/shardddl/optimism/table.go b/dm/pkg/shardddl/optimism/table.go index 0951720fd77..c729c324ab1 100644 --- a/dm/pkg/shardddl/optimism/table.go +++ b/dm/pkg/shardddl/optimism/table.go @@ -18,8 +18,8 @@ import ( "encoding/json" "fmt" - "go.etcd.io/etcd/clientv3" - "go.etcd.io/etcd/mvcc/mvccpb" + "go.etcd.io/etcd/api/v3/mvccpb" + clientv3 "go.etcd.io/etcd/client/v3" "github.com/pingcap/tiflow/dm/dm/common" "github.com/pingcap/tiflow/dm/pkg/etcdutil" diff --git a/dm/pkg/shardddl/pessimism/info.go b/dm/pkg/shardddl/pessimism/info.go index 066ab514500..4aa614d801e 100644 --- a/dm/pkg/shardddl/pessimism/info.go +++ b/dm/pkg/shardddl/pessimism/info.go @@ -17,9 +17,9 @@ import ( "context" "encoding/json" - "go.etcd.io/etcd/clientv3" - "go.etcd.io/etcd/clientv3/clientv3util" - "go.etcd.io/etcd/mvcc/mvccpb" + "go.etcd.io/etcd/api/v3/mvccpb" + clientv3 "go.etcd.io/etcd/client/v3" + "go.etcd.io/etcd/client/v3/clientv3util" "github.com/pingcap/tiflow/dm/dm/common" "github.com/pingcap/tiflow/dm/pkg/etcdutil" diff --git a/dm/pkg/shardddl/pessimism/info_test.go b/dm/pkg/shardddl/pessimism/info_test.go index 43427197d3e..8c7eaebc9f1 100644 --- a/dm/pkg/shardddl/pessimism/info_test.go +++ b/dm/pkg/shardddl/pessimism/info_test.go @@ -22,8 +22,8 @@ import ( . "github.com/pingcap/check" "github.com/pingcap/tidb-tools/pkg/dbutil" - "go.etcd.io/etcd/clientv3" - "go.etcd.io/etcd/integration" + clientv3 "go.etcd.io/etcd/client/v3" + "go.etcd.io/etcd/tests/v3/integration" "github.com/pingcap/tiflow/dm/dm/common" "github.com/pingcap/tiflow/dm/pkg/utils" @@ -32,6 +32,7 @@ import ( var etcdTestCli *clientv3.Client func TestInfo(t *testing.T) { + integration.BeforeTestExternal(t) mockCluster := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1}) defer mockCluster.Terminate(t) diff --git a/dm/pkg/shardddl/pessimism/operation.go b/dm/pkg/shardddl/pessimism/operation.go index 7a7fc786c7d..05184e8945b 100644 --- a/dm/pkg/shardddl/pessimism/operation.go +++ b/dm/pkg/shardddl/pessimism/operation.go @@ -17,9 +17,9 @@ import ( "context" "encoding/json" - "go.etcd.io/etcd/clientv3" - "go.etcd.io/etcd/clientv3/clientv3util" - "go.etcd.io/etcd/mvcc/mvccpb" + "go.etcd.io/etcd/api/v3/mvccpb" + clientv3 "go.etcd.io/etcd/client/v3" + "go.etcd.io/etcd/client/v3/clientv3util" "github.com/pingcap/tiflow/dm/dm/common" "github.com/pingcap/tiflow/dm/pkg/etcdutil" diff --git a/dm/pkg/shardddl/pessimism/operation_test.go b/dm/pkg/shardddl/pessimism/operation_test.go index 74e86ab5102..b8c48f92cbd 100644 --- a/dm/pkg/shardddl/pessimism/operation_test.go +++ b/dm/pkg/shardddl/pessimism/operation_test.go @@ -19,8 +19,8 @@ import ( "time" . "github.com/pingcap/check" - "go.etcd.io/etcd/clientv3" - "go.etcd.io/etcd/mvcc/mvccpb" + "go.etcd.io/etcd/api/v3/mvccpb" + clientv3 "go.etcd.io/etcd/client/v3" ) func (t *testForEtcd) TestOperationJSON(c *C) { diff --git a/dm/pkg/shardddl/pessimism/ops.go b/dm/pkg/shardddl/pessimism/ops.go index 643afb06df5..0b45a3d085c 100644 --- a/dm/pkg/shardddl/pessimism/ops.go +++ b/dm/pkg/shardddl/pessimism/ops.go @@ -16,7 +16,7 @@ package pessimism import ( "context" - "go.etcd.io/etcd/clientv3" + clientv3 "go.etcd.io/etcd/client/v3" "github.com/pingcap/tiflow/dm/dm/common" "github.com/pingcap/tiflow/dm/pkg/etcdutil" diff --git a/dm/pkg/upgrade/upgrade.go b/dm/pkg/upgrade/upgrade.go index 79fe79aa2e8..a8de0f30914 100644 --- a/dm/pkg/upgrade/upgrade.go +++ b/dm/pkg/upgrade/upgrade.go @@ -19,7 +19,7 @@ import ( "time" "github.com/pingcap/tidb-tools/pkg/dbutil" - "go.etcd.io/etcd/clientv3" + clientv3 "go.etcd.io/etcd/client/v3" "go.uber.org/zap" "github.com/pingcap/tiflow/dm/dm/common" diff --git a/dm/pkg/upgrade/upgrade_test.go b/dm/pkg/upgrade/upgrade_test.go index 0504ccd79b5..69828ead9f1 100644 --- a/dm/pkg/upgrade/upgrade_test.go +++ b/dm/pkg/upgrade/upgrade_test.go @@ -19,27 +19,24 @@ import ( "testing" . "github.com/pingcap/check" - "go.etcd.io/etcd/clientv3" - "go.etcd.io/etcd/integration" + clientv3 "go.etcd.io/etcd/client/v3" + "go.etcd.io/etcd/tests/v3/integration" "github.com/pingcap/tiflow/dm/dm/common" ) var ( + mockCluster *integration.ClusterV3 + bigTxnCluster *integration.ClusterV3 etcdTestCli *clientv3.Client bigTxnTestCli *clientv3.Client ) func TestUpgrade(t *testing.T) { - mockCluster := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1}) - defer mockCluster.Terminate(t) + integration.BeforeTestExternal(t) - etcdTestCli = mockCluster.RandClient() - - bigCluster := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1, MaxTxnOps: 2048}) - defer bigCluster.Terminate(t) - - bigTxnTestCli = bigCluster.RandClient() + suite.(*testForEtcd).testT = t + suiteForBigTxn.(*testForBigTxn).testT = t TestingT(t) } @@ -50,9 +47,36 @@ func clearTestData(c *C) { c.Assert(err, IsNil) } -type testForEtcd struct{} +type testForEtcd struct { + testT *testing.T +} + +type testForBigTxn struct { + testT *testing.T +} + +var ( + suite = SerialSuites(&testForEtcd{}) + suiteForBigTxn = SerialSuites(&testForBigTxn{}) +) + +func (t *testForEtcd) SetUpSuite(c *C) { + mockCluster = integration.NewClusterV3(t.testT, &integration.ClusterConfig{Size: 1}) + etcdTestCli = mockCluster.RandClient() +} + +func (t *testForEtcd) TearDownSuite(c *C) { + mockCluster.Terminate(t.testT) +} + +func (t *testForBigTxn) SetUpSuite(c *C) { + bigTxnCluster = integration.NewClusterV3(t.testT, &integration.ClusterConfig{Size: 1, MaxTxnOps: 2048}) + bigTxnTestCli = bigTxnCluster.RandClient() +} -var _ = SerialSuites(&testForEtcd{}) +func (t *testForBigTxn) TearDownSuite(c *C) { + bigTxnCluster.Terminate(t.testT) +} func (t *testForEtcd) TestTryUpgrade(c *C) { defer clearTestData(c) @@ -153,6 +177,12 @@ func (t *testForEtcd) TestUpgradeToVer3(c *C) { c.Assert(err, IsNil) } c.Assert(upgradeToVer3(ctx, etcdTestCli), ErrorMatches, ".*too many operations in txn request.*") +} + +func (t *testForBigTxn) TestUpgradeToVer3(c *C) { + source := "source-1" + oldVal := "test" + ctx := context.Background() for i := 0; i < 1000; i++ { key := common.UpstreamConfigKeyAdapterV1.Encode(fmt.Sprintf("%s-%d", source, i)) diff --git a/dm/pkg/upgrade/version.go b/dm/pkg/upgrade/version.go index 6d6d6f249a8..1c941d229e4 100644 --- a/dm/pkg/upgrade/version.go +++ b/dm/pkg/upgrade/version.go @@ -16,7 +16,7 @@ package upgrade import ( "encoding/json" - "go.etcd.io/etcd/clientv3" + clientv3 "go.etcd.io/etcd/client/v3" "github.com/pingcap/tiflow/dm/dm/common" "github.com/pingcap/tiflow/dm/pkg/etcdutil" diff --git a/dm/syncer/safe_mode_test.go b/dm/syncer/safe_mode_test.go index ed677525b6b..01814b7e0aa 100644 --- a/dm/syncer/safe_mode_test.go +++ b/dm/syncer/safe_mode_test.go @@ -17,14 +17,15 @@ import ( "time" "github.com/go-mysql-org/go-mysql/mysql" + "github.com/stretchr/testify/require" + "go.etcd.io/etcd/tests/v3/integration" + "go.uber.org/zap" + "github.com/pingcap/tiflow/dm/dm/config" "github.com/pingcap/tiflow/dm/pkg/binlog" tcontext "github.com/pingcap/tiflow/dm/pkg/context" "github.com/pingcap/tiflow/dm/pkg/log" mode "github.com/pingcap/tiflow/dm/syncer/safe-mode" - "github.com/stretchr/testify/require" - "go.etcd.io/etcd/integration" - "go.uber.org/zap" ) type mockCheckpointForSafeMode struct { @@ -38,6 +39,7 @@ func (c *mockCheckpointForSafeMode) SafeModeExitPoint() *binlog.Location { } func TestEnableSafeModeInitializationPhase(t *testing.T) { + integration.BeforeTestExternal(t) mockCluster := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1}) defer mockCluster.Terminate(t) etcdTestCli := mockCluster.RandClient() diff --git a/dm/syncer/shardddl/optimist.go b/dm/syncer/shardddl/optimist.go index 4b076e96008..f940e28cf7c 100644 --- a/dm/syncer/shardddl/optimist.go +++ b/dm/syncer/shardddl/optimist.go @@ -19,7 +19,7 @@ import ( filter "github.com/pingcap/tidb-tools/pkg/table-filter" "github.com/pingcap/tidb/parser/model" - "go.etcd.io/etcd/clientv3" + clientv3 "go.etcd.io/etcd/client/v3" "go.uber.org/zap" "github.com/pingcap/tiflow/dm/pkg/log" diff --git a/dm/syncer/shardddl/pessimist.go b/dm/syncer/shardddl/pessimist.go index 4b7d104b68c..bdc31d720f7 100644 --- a/dm/syncer/shardddl/pessimist.go +++ b/dm/syncer/shardddl/pessimist.go @@ -17,7 +17,7 @@ import ( "context" "sync" - "go.etcd.io/etcd/clientv3" + clientv3 "go.etcd.io/etcd/client/v3" "go.uber.org/zap" "github.com/pingcap/tiflow/dm/pkg/log" diff --git a/dm/syncer/shardddl/pessimist_test.go b/dm/syncer/shardddl/pessimist_test.go index a36450f8aa7..70b04d21207 100644 --- a/dm/syncer/shardddl/pessimist_test.go +++ b/dm/syncer/shardddl/pessimist_test.go @@ -19,8 +19,8 @@ import ( "time" . "github.com/pingcap/check" - "go.etcd.io/etcd/clientv3" - "go.etcd.io/etcd/integration" + clientv3 "go.etcd.io/etcd/client/v3" + "go.etcd.io/etcd/tests/v3/integration" "github.com/pingcap/tiflow/dm/dm/common" "github.com/pingcap/tiflow/dm/pkg/log" @@ -40,6 +40,7 @@ func TestShardDDL(t *testing.T) { t.Fatal(err) } + integration.BeforeTestExternal(t) mockCluster := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1}) defer mockCluster.Terminate(t) diff --git a/dm/syncer/syncer.go b/dm/syncer/syncer.go index bde89f5472f..30327c6e938 100644 --- a/dm/syncer/syncer.go +++ b/dm/syncer/syncer.go @@ -38,7 +38,7 @@ import ( "github.com/pingcap/tidb/parser/format" "github.com/pingcap/tidb/parser/model" "github.com/pingcap/tidb/sessionctx" - "go.etcd.io/etcd/clientv3" + clientv3 "go.etcd.io/etcd/client/v3" "go.uber.org/atomic" "go.uber.org/zap" diff --git a/dm/tests/_utils/run_dm_master_info_log b/dm/tests/_utils/run_dm_master_info_log new file mode 100755 index 00000000000..e4052b79947 --- /dev/null +++ b/dm/tests/_utils/run_dm_master_info_log @@ -0,0 +1,28 @@ +#!/bin/sh +# parameter 1: work directory +# parameter 2: master-addr port +# parameter 3: config file for DM-master +# parameter 4: binary version, previous or current, can be omitted + +set -eu + +workdir=$1 +port=$2 +conf=$3 + +binary=$PWD/bin/dm-master.test +if [ "$#" -ge 4 ]; then + version=$4 + binary=$PWD/bin/dm-master.test.$version +fi + +PWD=$(pwd) + +mkdir -p $workdir/log + +echo "[$(date)] <<<<<< START DM-MASTER on port $port, config: $conf >>>>>>" +cd $workdir +$binary -test.coverprofile="$TEST_DIR/cov.$TEST_NAME.master.out" DEVEL \ + --master-addr=:$port --log-file="$workdir/log/dm-master.log" -L=info \ + --config="$conf" >>$workdir/log/stdout.log 2>&1 & +cd $PWD diff --git a/dm/tests/adjust_gtid/data/db2.increment.sql b/dm/tests/adjust_gtid/data/db2.increment.sql index 9be41c3a531..bfb3d4f05a4 100644 --- a/dm/tests/adjust_gtid/data/db2.increment.sql +++ b/dm/tests/adjust_gtid/data/db2.increment.sql @@ -2,4 +2,7 @@ use adjust_gtid; delete from t2 where name = 'Sansa'; -- test sql_mode=NO_AUTO_VALUE_ON_ZERO -insert into t2 (id, name) values (0,'haha') \ No newline at end of file +insert into t2 (id, name) values (0,'haha'); + +create table t3(c int primary key, t time(2)); +insert into t3 values(1, '-00:00:00.01'); diff --git a/dm/tests/all_mode/run.sh b/dm/tests/all_mode/run.sh index e3f23df4231..e13d886b0a9 100755 --- a/dm/tests/all_mode/run.sh +++ b/dm/tests/all_mode/run.sh @@ -406,7 +406,8 @@ function run() { check_contains 'Query OK, 3 rows affected' # start DM worker and master - run_dm_master $WORK_DIR/master $MASTER_PORT $cur/conf/dm-master.toml + # set log level of DM-master to info, because debug level will let etcd print KV, thus expose the password in task config + run_dm_master_info_log $WORK_DIR/master $MASTER_PORT $cur/conf/dm-master.toml check_rpc_alive $cur/../bin/check_master_online 127.0.0.1:$MASTER_PORT check_metric $MASTER_PORT 'start_leader_counter' 3 0 2 run_dm_worker $WORK_DIR/worker1 $WORKER1_PORT $cur/conf/dm-worker1.toml diff --git a/dm/tests/lightning_mode/run.sh b/dm/tests/lightning_mode/run.sh index 5f49f3bfa1e..67dbd9c4193 100755 --- a/dm/tests/lightning_mode/run.sh +++ b/dm/tests/lightning_mode/run.sh @@ -23,7 +23,7 @@ function run() { check_contains 'Query OK, 3 rows affected' # start DM worker and master - run_dm_master $WORK_DIR/master $MASTER_PORT $cur/conf/dm-master.toml + run_dm_master_info_log $WORK_DIR/master $MASTER_PORT $cur/conf/dm-master.toml check_rpc_alive $cur/../bin/check_master_online 127.0.0.1:$MASTER_PORT check_metric $MASTER_PORT 'start_leader_counter' 3 0 2 run_dm_worker $WORK_DIR/worker1 $WORKER1_PORT $cur/conf/dm-worker1.toml diff --git a/dm/tests/tls/run.sh b/dm/tests/tls/run.sh index 9e7f08f6fd7..f05f59cfb93 100644 --- a/dm/tests/tls/run.sh +++ b/dm/tests/tls/run.sh @@ -372,7 +372,8 @@ function test_master_ha_when_enable_tidb_tls() { check_sync_diff $WORK_DIR $cur/conf/diff_config.toml # https://github.com/pingcap/dm/issues/1458 - check_log_not_contains $WORK_DIR/master1/log/dm-master.log "remote error: tls: bad certificate" + # check the log is not repeatedly printed + check_log_contains $WORK_DIR/master1/log/dm-master.log "remote error: tls: bad certificate" 1 echo "============================== test_master_ha_when_enable_tidb_tls success ==================================" } diff --git a/go.mod b/go.mod index 58a4fcab58a..418a079cc91 100644 --- a/go.mod +++ b/go.mod @@ -25,18 +25,18 @@ require ( github.com/frankban/quicktest v1.11.1 // indirect github.com/getkin/kin-openapi v0.80.0 github.com/gin-gonic/gin v1.7.4 - github.com/go-mysql-org/go-mysql v1.4.1-0.20211217061939-06f932768788 + github.com/go-mysql-org/go-mysql v1.4.1-0.20220221114137-89145541e0d4 github.com/go-sql-driver/mysql v1.6.0 github.com/gogo/gateway v1.1.0 github.com/gogo/protobuf v1.3.2 github.com/golang-jwt/jwt v3.2.2+incompatible // indirect github.com/golang/mock v1.6.0 github.com/golang/protobuf v1.5.2 - github.com/google/btree v1.0.0 + github.com/google/btree v1.0.1 github.com/google/go-cmp v0.5.6 github.com/google/shlex v0.0.0-20191202100458-e7afc7fbc510 github.com/google/uuid v1.1.2 - github.com/grpc-ecosystem/go-grpc-middleware v1.1.0 + github.com/grpc-ecosystem/go-grpc-middleware v1.3.0 github.com/grpc-ecosystem/go-grpc-prometheus v1.2.0 github.com/grpc-ecosystem/grpc-gateway v1.16.0 github.com/integralist/go-findroot v0.0.0-20160518114804-ac90681525dc @@ -53,12 +53,12 @@ require ( github.com/pingcap/check v0.0.0-20211026125417-57bd13f7b5f0 github.com/pingcap/errors v0.11.5-0.20211224045212-9687c2b0f87c github.com/pingcap/failpoint v0.0.0-20210918120811-547c13e3eb00 - github.com/pingcap/kvproto v0.0.0-20211224055123-d1a140660c39 - github.com/pingcap/log v0.0.0-20211207084639-71a2e5860834 - github.com/pingcap/tidb v1.1.0-beta.0.20220210065735-92f5a6340ca2 - github.com/pingcap/tidb-tools v5.2.3-0.20211105044302-2dabb6641a6e+incompatible - github.com/pingcap/tidb/parser v0.0.0-20220210065735-92f5a6340ca2 - github.com/prometheus/client_golang v1.7.1 + github.com/pingcap/kvproto v0.0.0-20220303025110-d5eb908517b2 + github.com/pingcap/log v0.0.0-20211215031037-e024ba4eb0ee + github.com/pingcap/tidb v1.1.0-beta.0.20220303050146-f5a52909b6af + github.com/pingcap/tidb-tools v5.4.1-0.20220302060346-b6d9cbe5dff1+incompatible + github.com/pingcap/tidb/parser v0.0.0-20220303050146-f5a52909b6af + github.com/prometheus/client_golang v1.11.0 github.com/prometheus/client_model v0.2.0 github.com/r3labs/diff v1.1.0 github.com/rcrowley/go-metrics v0.0.0-20200313005456-10cdbea86bc0 @@ -73,35 +73,35 @@ require ( github.com/syndtr/goleveldb v1.0.1-0.20210305035536-64b5b1c73954 github.com/tidwall/gjson v1.9.1 github.com/tidwall/sjson v1.2.2 - github.com/tikv/client-go/v2 v2.0.0-rc.0.20220107040026-d22815099720 - github.com/tikv/pd v1.1.0-beta.0.20220207063535-9268bed87199 + github.com/tikv/client-go/v2 v2.0.0-rc.0.20220217140116-589cf79b2ab5 + github.com/tikv/pd v1.1.0-beta.0.20220303060546-3695d8164800 + github.com/tikv/pd/client v0.0.0-20220216080339-1b8f82378ee7 github.com/tinylib/msgp v1.1.0 github.com/uber-go/atomic v1.4.0 github.com/vmihailenco/msgpack/v5 v5.3.5 github.com/xdg/scram v0.0.0-20180814205039-7eeb5667e42c github.com/xitongsys/parquet-go v1.6.0 // indirect - go.etcd.io/etcd v0.5.0-alpha.5.0.20210512015243-d19fbe541bf9 + go.etcd.io/etcd/api/v3 v3.5.2 + go.etcd.io/etcd/client/pkg/v3 v3.5.2 + go.etcd.io/etcd/client/v3 v3.5.2 + go.etcd.io/etcd/pkg/v3 v3.5.2 + go.etcd.io/etcd/server/v3 v3.5.2 + go.etcd.io/etcd/tests/v3 v3.5.2 go.uber.org/atomic v1.9.0 go.uber.org/goleak v1.1.12 go.uber.org/multierr v1.7.0 - go.uber.org/zap v1.19.1 + go.uber.org/zap v1.20.0 golang.org/x/net v0.0.0-20211020060615-d418f374d309 golang.org/x/sync v0.0.0-20210220032951-036812b2e83c golang.org/x/sys v0.0.0-20211216021012-1d35b9e2eb4e golang.org/x/text v0.3.7 golang.org/x/time v0.0.0-20210220033141-f8bda1e9f3ba google.golang.org/genproto v0.0.0-20210825212027-de86158e7fda - google.golang.org/grpc v1.40.0 + google.golang.org/grpc v1.43.0 gopkg.in/mgo.v2 v2.0.0-20190816093944-a6b53ec6cb22 // indirect gopkg.in/yaml.v2 v2.4.0 - sigs.k8s.io/yaml v1.2.0 // indirect upper.io/db.v3 v3.7.1+incompatible ) -replace ( - // Fix CVE-2020-26160. - github.com/dgrijalva/jwt-go v3.2.0+incompatible => github.com/golang-jwt/jwt v3.2.1+incompatible - // cloud.google.com/go/storage will upgrade grpc to v1.40.0 - // we need keep the replacement until go.etcd.io supports the higher version of grpc. - google.golang.org/grpc v1.40.0 => google.golang.org/grpc v1.29.1 -) +// Fix CVE-2020-26160. +replace github.com/dgrijalva/jwt-go v3.2.0+incompatible => github.com/golang-jwt/jwt v3.2.1+incompatible diff --git a/go.sum b/go.sum index ee42888fbda..3908a3a6936 100644 --- a/go.sum +++ b/go.sum @@ -104,6 +104,7 @@ github.com/alecthomas/template v0.0.0-20160405071501-a0175ee3bccc/go.mod h1:LOuy github.com/alecthomas/template v0.0.0-20190718012654-fb15b899a751/go.mod h1:LOuyumcjzFXgccqObfd/Ljyb9UuFJ6TxHnclSeseNhc= github.com/alecthomas/units v0.0.0-20151022065526-2efee857e7cf/go.mod h1:ybxpYRFXyAe+OPACYpWeL0wqObRcbAqCMya13uyzqw0= github.com/alecthomas/units v0.0.0-20190717042225-c3de453c63f4/go.mod h1:ybxpYRFXyAe+OPACYpWeL0wqObRcbAqCMya13uyzqw0= +github.com/alecthomas/units v0.0.0-20190924025748-f65c72e2690d/go.mod h1:rBZYJk541a8SKzHPHnH3zbiI+7dagKZ0cgpgrD7Fyho= github.com/alvaroloes/enumer v1.1.2/go.mod h1:FxrjvuXoDAx9isTJrv4c+T410zFi0DtXIT0m65DJ+Wo= github.com/antihax/optional v0.0.0-20180407024304-ca021399b1a6/go.mod h1:V8iCPQYkqmusNa815XgQio277wI47sdRh1dUOLdyC6Q= github.com/antihax/optional v1.0.0/go.mod h1:uupD/76wgC+ih3iEmQUL+0Ugr19nfwCT1kdvxnR2qWY= @@ -131,6 +132,7 @@ github.com/aws/aws-sdk-go v1.35.3/go.mod h1:H7NKnBqNVzoTJpGfLrQkkD+ytBA93eiDYi/+ github.com/axw/gocov v1.0.0/go.mod h1:LvQpEYiwwIb2nYkXY2fDWhg9/AsYqkhmrCshjlUJECE= github.com/aymerick/raymond v2.0.3-0.20180322193309-b565731e1464+incompatible/go.mod h1:osfaiScAUVup+UC9Nfq76eWqDhXlp+4UYaA8uhTBO6g= github.com/beefsack/go-rate v0.0.0-20180408011153-efa7637bb9b6/go.mod h1:6YNgTHLutezwnBvyneBbwvB8C82y3dcoOj5EQJIdGXA= +github.com/benbjohnson/clock v1.0.3/go.mod h1:bGMdMPoPVvcYyt1gHDf4J2KE153Yf9BuiUKYMaxlTDM= github.com/benbjohnson/clock v1.1.0 h1:Q92kusRqC1XV2MjkWETPvjJVqKetz1OzxZB7mHJLju8= github.com/benbjohnson/clock v1.1.0/go.mod h1:J11/hYXuz8f4ySSvYwY0FKfm+ezbsZBKZxNJlLklBHA= github.com/beorn7/perks v0.0.0-20180321164747-3a771d992973/go.mod h1:Dwedo/Wpr24TaqPxmxbtue+5NUziq4I4S80YR8gNf3Q= @@ -138,6 +140,7 @@ github.com/beorn7/perks v1.0.0/go.mod h1:KWe93zE9D1o94FZ5RNwFwVgaQK1VOXiVxmqh+Ce github.com/beorn7/perks v1.0.1 h1:VlbKKnNfV8bJzeqoa4cOKqO6bYr3WgKZxO8Z16+hsOM= github.com/beorn7/perks v1.0.1/go.mod h1:G2ZrVWU2WbWT9wwq4/hrbKbnv/1ERSJQ0ibhJ6rlkpw= github.com/bgentry/speakeasy v0.1.0/go.mod h1:+zsyZBPWlz7T6j88CTgSN5bM796AkVf0kBD4zp0CCIs= +github.com/bketelsen/crypt v0.0.3-0.20200106085610-5cbc8cc4026c/go.mod h1:MKsuJmJgSg28kpZDP6UIiPt0e0Oz0kqKNGyRaWEPv84= github.com/bketelsen/crypt v0.0.4/go.mod h1:aI6NrJ0pMGgvZKL1iVgXLnfIFJtfV+bKCoqOes/6LfM= github.com/blacktear23/go-proxyprotocol v0.0.0-20180807104634-af7a81e8dd0d h1:rQlvB2AYWme2bIB18r/SipGiMEVJYE9U0z+MGoU/LtQ= github.com/blacktear23/go-proxyprotocol v0.0.0-20180807104634-af7a81e8dd0d/go.mod h1:VKt7CNAQxpFpSDz3sXyj9hY/GbVsQCr0sB3w59nE7lU= @@ -153,6 +156,8 @@ github.com/carlmjohnson/flagext v0.21.0/go.mod h1:Eenv0epIUAr4NuedNmkzI8WmBmjIxZ github.com/cenkalti/backoff/v4 v4.0.2 h1:JIufpQLbh4DkbQoii76ItQIUFzevQSqOLZca4eamEDs= github.com/cenkalti/backoff/v4 v4.0.2/go.mod h1:eEew/i+1Q6OrCDZh3WiXYv3+nJwBASZ8Bog/87DQnVg= github.com/census-instrumentation/opencensus-proto v0.2.1/go.mod h1:f6KPmirojxKA12rnyqOA5BBL4O983OfeGPqjHWSTneU= +github.com/certifi/gocertifi v0.0.0-20191021191039-0944d244cd40/go.mod h1:sGbDF6GwGcLpkNXPUTkMRoywsNa/ol15pxFe6ERfguA= +github.com/certifi/gocertifi v0.0.0-20200922220541-2c3bb06c6054/go.mod h1:sGbDF6GwGcLpkNXPUTkMRoywsNa/ol15pxFe6ERfguA= github.com/cespare/xxhash v1.1.0 h1:a6HrQnmkObjyL+Gs60czilIUGqrzKutQD6XZog3p+ko= github.com/cespare/xxhash v1.1.0/go.mod h1:XrSqR1VqqWfGrhpAt58auRo0WTKS1nRRg3ghfAqPWnc= github.com/cespare/xxhash/v2 v2.1.0/go.mod h1:dgIUBU3pDso/gPgZ1osOZ0iQf77oPR28Tjxl5dIMyVM= @@ -173,11 +178,17 @@ github.com/client9/misspell v0.3.4/go.mod h1:qj6jICC3Q7zFZvVWo7KLAzC3yx5G7kyvSDk github.com/cncf/udpa/go v0.0.0-20191209042840-269d4d468f6f/go.mod h1:M8M6+tZqaGXZJjfX53e64911xZQV5JYwmTeXPW+k8Sc= github.com/cncf/udpa/go v0.0.0-20200629203442-efcf912fb354/go.mod h1:WmhPx2Nbnhtbo57+VJT5O0JRkEi1Wbu0z5j0R8u5Hbk= github.com/cncf/udpa/go v0.0.0-20201120205902-5459f2c99403/go.mod h1:WmhPx2Nbnhtbo57+VJT5O0JRkEi1Wbu0z5j0R8u5Hbk= +github.com/cncf/udpa/go v0.0.0-20210930031921-04548b0d99d4/go.mod h1:6pvJx4me5XPnfI9Z40ddWsdw2W/uZgQLFXToKeRcDiI= github.com/cncf/xds/go v0.0.0-20210312221358-fbca930ec8ed/go.mod h1:eXthEFrGJvWHgFFCl3hGmgk+/aYT6PnTQLykKQRLhEs= +github.com/cncf/xds/go v0.0.0-20210805033703-aa0b78936158/go.mod h1:eXthEFrGJvWHgFFCl3hGmgk+/aYT6PnTQLykKQRLhEs= +github.com/cncf/xds/go v0.0.0-20210922020428-25de7278fc84/go.mod h1:eXthEFrGJvWHgFFCl3hGmgk+/aYT6PnTQLykKQRLhEs= +github.com/cncf/xds/go v0.0.0-20211011173535-cb28da3451f1/go.mod h1:eXthEFrGJvWHgFFCl3hGmgk+/aYT6PnTQLykKQRLhEs= github.com/cockroachdb/apd v1.1.0/go.mod h1:8Sl8LxpKi29FqWXR16WEFZRNSz3SoPzUzeMeY4+DwBQ= github.com/cockroachdb/datadriven v0.0.0-20190809214429-80d97fb3cbaa/go.mod h1:zn76sxSg3SzpJ0PPJaLDCu+Bu0Lg3sKTORVIj19EIF8= +github.com/cockroachdb/datadriven v0.0.0-20200714090401-bf6692d28da5/go.mod h1:h6jFvWxBdQXxjopDMZyH2UVceIRfR84bdzbkoKrsWNo= github.com/cockroachdb/datadriven v1.0.0 h1:uhZrAfEayBecH2w2tZmhe20HJ7hDvrrA4x2Bg9YdZKM= github.com/cockroachdb/datadriven v1.0.0/go.mod h1:5Ib8Meh+jk1RlHIXej6Pzevx/NLlNvQB9pmSBZErGA4= +github.com/cockroachdb/errors v1.2.4/go.mod h1:rQD95gz6FARkaKkQXUksEje/d9a6wBJoCr5oaCLELYA= github.com/cockroachdb/errors v1.6.1/go.mod h1:tm6FTP5G81vwJ5lC0SizQo374JNCOPrHyXGitRJoDqM= github.com/cockroachdb/errors v1.8.1 h1:A5+txlVZfOqFBDa4mGz2bUWSp0aHElvHX2bKkdbQu+Y= github.com/cockroachdb/errors v1.8.1/go.mod h1:qGwQn6JmZ+oMjuLwjWzUNqblqk0xl4CVV3SQbGwK7Ac= @@ -201,6 +212,7 @@ github.com/coocood/rtutil v0.0.0-20190304133409-c84515f646f2 h1:NnLfQ77q0G4k2Of2 github.com/coocood/rtutil v0.0.0-20190304133409-c84515f646f2/go.mod h1:7qG7YFnOALvsx6tKTNmQot8d7cGFXM9TidzvRFLWYwM= github.com/coreos/bbolt v1.3.2/go.mod h1:iRUV2dpdMOn7Bo10OQBFzIJO9kkE559Wcmn+qkEiiKk= github.com/coreos/etcd v3.3.10+incompatible/go.mod h1:uF7uidLiAD3TWHmW31ZFd/JWoc32PjwdhPthX9715RE= +github.com/coreos/etcd v3.3.13+incompatible/go.mod h1:uF7uidLiAD3TWHmW31ZFd/JWoc32PjwdhPthX9715RE= github.com/coreos/go-etcd v2.0.0+incompatible/go.mod h1:Jez6KQU2B/sWsbdaef3ED8NzMklzPG4d5KIOhIy30Tk= github.com/coreos/go-semver v0.2.0/go.mod h1:nnelYz7RCh+5ahJtPPxZlU+153eP4D4r3EedlOD2RNk= github.com/coreos/go-semver v0.3.0 h1:wkHLiw0WNATZnSG7epLsujiMCgPAc9xhjJ4tgnAxmfM= @@ -209,9 +221,9 @@ github.com/coreos/go-systemd v0.0.0-20180511133405-39ca1b05acc7/go.mod h1:F5haX7 github.com/coreos/go-systemd v0.0.0-20190321100706-95778dfbb74e/go.mod h1:F5haX7vjVVG0kc13fIWeqUViNPyEJxv/OmvnBo0Yme4= github.com/coreos/go-systemd v0.0.0-20190719114852-fd7a80b32e1f h1:JOrtw2xFKzlg+cbHpyrpLDmnN1HqhBfnX7WDiW7eG2c= github.com/coreos/go-systemd v0.0.0-20190719114852-fd7a80b32e1f/go.mod h1:F5haX7vjVVG0kc13fIWeqUViNPyEJxv/OmvnBo0Yme4= +github.com/coreos/go-systemd/v22 v22.3.2 h1:D9/bQk5vlXQFZ6Kwuu6zaiXJ9oTPe68++AzAJc1DzSI= github.com/coreos/go-systemd/v22 v22.3.2/go.mod h1:Y58oyj3AT4RCenI/lSvhwexgC+NSVTIJ3seZv2GcEnc= github.com/coreos/pkg v0.0.0-20160727233714-3ac0863d7acf/go.mod h1:E3G3o1h8I7cfcXa63jLwjI0eiQQMgzzUDFVpN/nH/eA= -github.com/coreos/pkg v0.0.0-20180928190104-399ea9e2e55f h1:lBNOc5arjvs8E5mO2tbpBpLoyyu8B6e44T7hJy6potg= github.com/coreos/pkg v0.0.0-20180928190104-399ea9e2e55f/go.mod h1:E3G3o1h8I7cfcXa63jLwjI0eiQQMgzzUDFVpN/nH/eA= github.com/corona10/goimagehash v1.0.2/go.mod h1:/l9umBhvcHQXVtQO1V6Gp1yD20STawkhRnnX0D1bvVI= github.com/cpuguy83/go-md2man v1.0.10/go.mod h1:SmD6nW6nTyfqj6ABTjUi3V3JVMnlJmwcJI5acqYI6dE= @@ -273,7 +285,6 @@ github.com/eapache/queue v1.1.0 h1:YOEu7KNc61ntiQlcEeUIoDTJ2o8mQznoNvUhiigpIqc= github.com/eapache/queue v1.1.0/go.mod h1:6eCeP0CKFpHLu8blIFXhExK/dRa7WDZfr6jVFPTqq+I= github.com/edwingeng/deque v0.0.0-20191220032131-8596380dee17 h1:8i9x3Q4hW1kLE4ScsOtUlwVHT76LKhkmOw9zbDxnyUc= github.com/edwingeng/deque v0.0.0-20191220032131-8596380dee17/go.mod h1:3Ys1pJhyVaB6iWigv4o2r6Ug1GZmfDWqvqmO6bjojg0= -github.com/eknkc/amber v0.0.0-20171010120322-cdade1c07385 h1:clC1lXBpe2kTj2VHdaIu9ajZQe4kcEY9j0NsnDDBZ3o= github.com/eknkc/amber v0.0.0-20171010120322-cdade1c07385/go.mod h1:0vRUJqYpeSZifjYj7uP3BG/gKcuzL9xWVV/Y+cK33KM= github.com/envoyproxy/go-control-plane v0.9.0/go.mod h1:YTl/9mNaCwkRvm6d1a2C3ymFceY/DCBVvsKhRF0iEA4= github.com/envoyproxy/go-control-plane v0.9.1-0.20191026205805-5f8ba28d4473/go.mod h1:YTl/9mNaCwkRvm6d1a2C3ymFceY/DCBVvsKhRF0iEA4= @@ -282,9 +293,11 @@ github.com/envoyproxy/go-control-plane v0.9.7/go.mod h1:cwu0lG7PUMfa9snN8LXBig5y github.com/envoyproxy/go-control-plane v0.9.9-0.20201210154907-fd9021fe5dad/go.mod h1:cXg6YxExXjJnVBQHBLXeUAgxn2UodCpnH306RInaBQk= github.com/envoyproxy/go-control-plane v0.9.9-0.20210217033140-668b12f5399d/go.mod h1:cXg6YxExXjJnVBQHBLXeUAgxn2UodCpnH306RInaBQk= github.com/envoyproxy/go-control-plane v0.9.9-0.20210512163311-63b5d3c536b0/go.mod h1:hliV/p42l8fGbc6Y9bQ70uLwIvmJyVE5k4iMKlh8wCQ= +github.com/envoyproxy/go-control-plane v0.9.10-0.20210907150352-cf90f659a021/go.mod h1:AFq3mo9L8Lqqiid3OhADV3RfLJnjiw63cSpi+fDTRC0= github.com/envoyproxy/protoc-gen-validate v0.1.0/go.mod h1:iSmxcyjqTsJpI2R4NaDN7+kN2VEUnK/pcBlmesArF7c= github.com/erikstmartin/go-testdb v0.0.0-20160219214506-8d10e4a1bae5/go.mod h1:a2zkGnVExMxdzMo3M0Hi/3sEU+cWnZpSni0O6/Yb/P0= github.com/etcd-io/bbolt v1.3.3/go.mod h1:ZF2nL25h33cCyBtcyWeZ2/I3HQOfTP+0PIEvHjkjCrw= +github.com/etcd-io/gofail v0.0.0-20190801230047-ad7f989257ca/go.mod h1:49H/RkXP8pKaZy4h0d+NW16rSLhyVBt4o6VLJbmOqDE= github.com/facebookgo/subset v0.0.0-20200203212716-c811ad88dec4 h1:7HZCaLC5+BZpmbhCOZJ293Lz68O7PYrF2EzeiFMwCLk= github.com/facebookgo/subset v0.0.0-20200203212716-c811ad88dec4/go.mod h1:5tD+neXqOorC30/tWg0LCSkrqj/AR6gu8yY8/fpw1q0= github.com/fasthttp-contrib/websocket v0.0.0-20160511215533-1f3b11f56072/go.mod h1:duJ4Jxv5lDcvg4QuQr0oowTf7dz4/CR8NtyCooz9HL8= @@ -299,6 +312,8 @@ github.com/felixge/httpsnoop v1.0.1/go.mod h1:m8KPJKqk1gH5J9DgRY2ASl2lWCfGKXixSw github.com/flosch/pongo2 v0.0.0-20190707114632-bbf5a6c351f4/go.mod h1:T9YF2M40nIgbVgp3rreNmTged+9HrbNTIQf1PsaIiTA= github.com/fogleman/gg v1.2.1-0.20190220221249-0403632d5b90/go.mod h1:R/bRT+9gY/C5z7JzPU0zXsXHKM4/ayA+zqcVNZzPa1k= github.com/fogleman/gg v1.3.0/go.mod h1:R/bRT+9gY/C5z7JzPU0zXsXHKM4/ayA+zqcVNZzPa1k= +github.com/form3tech-oss/jwt-go v3.2.3+incompatible/go.mod h1:pbq4aXjuKjdthFRnoDwaVPLA+WlJuPGy+QneDUgJi2k= +github.com/form3tech-oss/jwt-go v3.2.5+incompatible h1:/l4kBbb4/vGSsdtB5nUe8L7B9mImVMaBPw9L/0TBHU8= github.com/form3tech-oss/jwt-go v3.2.5+incompatible/go.mod h1:pbq4aXjuKjdthFRnoDwaVPLA+WlJuPGy+QneDUgJi2k= github.com/fortytw2/leaktest v1.3.0 h1:u8491cBMTQ8ft8aeV+adlcytMZylmA5nnwwkRZjI8vw= github.com/fortytw2/leaktest v1.3.0/go.mod h1:jDsjWgpAGjm2CA7WthBh/CdZYEPF31XHquHwclZch5g= @@ -316,6 +331,7 @@ github.com/gdamore/encoding v1.0.0/go.mod h1:alR0ol34c49FCSBLjhosxzcPHQbf2trDkoo github.com/gdamore/tcell v1.3.0/go.mod h1:Hjvr+Ofd+gLglo7RYKxxnzCBmev3BzsS67MebKS4zMM= github.com/getkin/kin-openapi v0.80.0 h1:W/s5/DNnDCR8P+pYyafEWlGk4S7/AfQUWXgrRSSAzf8= github.com/getkin/kin-openapi v0.80.0/go.mod h1:660oXbgy5JFMKreazJaQTw7o+X00qeSyhcnluiMv+Xg= +github.com/getsentry/raven-go v0.2.0/go.mod h1:KungGk8q33+aIAZUIVWZDr2OfAEBsO49PX4NzFV5kcQ= github.com/ghemawat/stream v0.0.0-20171120220530-696b145b53b9/go.mod h1:106OIgooyS7OzLDOpUGgm9fA3bQENb/cFSyyBmMoJDs= github.com/ghodss/yaml v1.0.0 h1:wQHKEahhL6wmXdzwWG11gIVCkOv05bNOh+Rxn0yngAk= github.com/ghodss/yaml v1.0.0/go.mod h1:4dBDuWmgqj2HViK6kFavaiC9ZROes6MMH2rRYeMEF04= @@ -342,12 +358,14 @@ github.com/go-gl/glfw/v3.3/glfw v0.0.0-20191125211704-12ad95a8df72/go.mod h1:tQ2 github.com/go-gl/glfw/v3.3/glfw v0.0.0-20200222043503-6f7a984d4dc4/go.mod h1:tQ2UAYgL5IevRw8kRxooKSPJfGvJ9fJQFa0TUsXzTg8= github.com/go-kit/kit v0.8.0/go.mod h1:xBxKIO96dXMWWy0MnWVtmwkA9/13aqxPnvrjFYMA2as= github.com/go-kit/kit v0.9.0/go.mod h1:xBxKIO96dXMWWy0MnWVtmwkA9/13aqxPnvrjFYMA2as= +github.com/go-kit/log v0.1.0/go.mod h1:zbhenjAZHb184qTLMA9ZjW7ThYL0H2mk7Q6pNt4vbaY= github.com/go-logfmt/logfmt v0.3.0/go.mod h1:Qt1PoO58o5twSAckw1HlFXLmHsOX5/0LbT9GBnD5lWE= github.com/go-logfmt/logfmt v0.4.0/go.mod h1:3RMwSq7FuexP4Kalkev3ejPJsZTpXXBr9+V4qmtdjCk= +github.com/go-logfmt/logfmt v0.5.0/go.mod h1:wCYkCAKZfumFQihp8CzCvQ3paCTfi41vtzG1KdI/P7A= github.com/go-logr/logr v0.1.0/go.mod h1:ixOQHD9gLJUVQQ2ZOR7zLEifBX6tGkNJF4QyIY7sIas= github.com/go-martini/martini v0.0.0-20170121215854-22fa46961aab/go.mod h1:/P9AEU963A2AYjv4d1V5eVL1CQbEJq6aCNHDDjibzu8= -github.com/go-mysql-org/go-mysql v1.4.1-0.20211217061939-06f932768788 h1:0IsP4ViNmA7ZElbCE4/lINdTppdw3jdcAiJaPDyeHx8= -github.com/go-mysql-org/go-mysql v1.4.1-0.20211217061939-06f932768788/go.mod h1:3lFZKf7l95Qo70+3XB2WpiSf9wu2s3na3geLMaIIrqQ= +github.com/go-mysql-org/go-mysql v1.4.1-0.20220221114137-89145541e0d4 h1:6AopLIKxyLXWY8PRxIq0q33BedlkNNcJfgJSBZh6Bks= +github.com/go-mysql-org/go-mysql v1.4.1-0.20220221114137-89145541e0d4/go.mod h1:TRs381neMzw+J5+bobjUY2ZsIMgvp4wBCRBW274gc68= github.com/go-ole/go-ole v1.2.4/go.mod h1:XCwSNxSkXRo4vlyPy93sltvi/qJq0jqQhjqQNIwKuxM= github.com/go-ole/go-ole v1.2.6 h1:/Fpf6oFPoeFik9ty7siob0G6Ke8QvQEuVcuChpwXzpY= github.com/go-ole/go-ole v1.2.6/go.mod h1:pprOEPIfldk/42T2oK7lQ4v4JSDwmV0As9GaiUsvbm0= @@ -390,6 +408,7 @@ github.com/go-sql-driver/mysql v1.5.0/go.mod h1:DCzpHaOWr8IXmIStZouvnhqoel9Qv2LB github.com/go-sql-driver/mysql v1.6.0 h1:BCTh4TKNUYmOmMUcQ3IipzF5prigylS7XXjEkfCHuOE= github.com/go-sql-driver/mysql v1.6.0/go.mod h1:DCzpHaOWr8IXmIStZouvnhqoel9Qv2LBy8hT2VhHyBg= github.com/go-stack/stack v1.8.0/go.mod h1:v0f6uXyyMGvRgIKkXu+yp6POWl0qKG85gN/melR3HDY= +github.com/go-task/slim-sprig v0.0.0-20210107165309-348f09dbbbc0/go.mod h1:fyg7847qk6SyHyPtNmDHnmrv/HOrqktSC+C9fM+CJOE= github.com/gobwas/httphead v0.0.0-20180130184737-2c6c146eadee/go.mod h1:L0fX3K22YWvt/FAX9NnzrNzcI4wNYi9Yku4O0LKYflo= github.com/gobwas/pool v0.2.0/go.mod h1:q8bcK0KcYlCgd9e7WYLm9LpyS+YeLd8JVDW6WezmKEw= github.com/gobwas/ws v1.0.2/go.mod h1:szmBTxLgaFppYjEmNtny/v3w89xOydFnnZMcgRRu/EM= @@ -423,8 +442,9 @@ github.com/golang/groupcache v0.0.0-20160516000752-02826c3e7903/go.mod h1:cIg4er github.com/golang/groupcache v0.0.0-20190129154638-5b532d6fd5ef/go.mod h1:cIg4eruTrX1D+g88fzRXU5OdNfaM+9IcxsU14FzY7Hc= github.com/golang/groupcache v0.0.0-20190702054246-869f871628b6/go.mod h1:cIg4eruTrX1D+g88fzRXU5OdNfaM+9IcxsU14FzY7Hc= github.com/golang/groupcache v0.0.0-20191227052852-215e87163ea7/go.mod h1:cIg4eruTrX1D+g88fzRXU5OdNfaM+9IcxsU14FzY7Hc= -github.com/golang/groupcache v0.0.0-20200121045136-8c9f03a8e57e h1:1r7pUrabqp18hOBcwBwiTsbnFeTZHV9eER/QT5JVZxY= github.com/golang/groupcache v0.0.0-20200121045136-8c9f03a8e57e/go.mod h1:cIg4eruTrX1D+g88fzRXU5OdNfaM+9IcxsU14FzY7Hc= +github.com/golang/groupcache v0.0.0-20210331224755-41bb18bfe9da h1:oI5xCqsCo564l8iNU+DwB5epxmsaqB+rhGL0m5jtYqE= +github.com/golang/groupcache v0.0.0-20210331224755-41bb18bfe9da/go.mod h1:cIg4eruTrX1D+g88fzRXU5OdNfaM+9IcxsU14FzY7Hc= github.com/golang/mock v1.1.1/go.mod h1:oTYuIxOrZwtPieC+H1uAHpcLFnEyAGVDL/k47Jfbm0A= github.com/golang/mock v1.2.0/go.mod h1:oTYuIxOrZwtPieC+H1uAHpcLFnEyAGVDL/k47Jfbm0A= github.com/golang/mock v1.3.1/go.mod h1:sBzyDLLjw3U8JLTeZvSv8jJB+tU5PVekmnlKIyFUx0Y= @@ -463,8 +483,9 @@ github.com/golang/snappy v0.0.3/go.mod h1:/XxbfmMg8lxefKM7IXC3fBNl/7bRcc72aCRzEW github.com/golangci/lint-1 v0.0.0-20181222135242-d2cdd8c08219/go.mod h1:/X8TswGSh1pIozq4ZwCfxS0WA5JGXguxk94ar/4c87Y= github.com/gomodule/redigo v1.7.1-0.20190724094224-574c33c3df38/go.mod h1:B4C85qUVwatsJoIUNIfCRsp7qO0iAmpGFZ4EELWSbC4= github.com/google/btree v0.0.0-20180813153112-4030bb1f1f0c/go.mod h1:lNA+9X1NB3Zf8V7Ke586lFgjr2dZNuvo3lPJSGZ5JPQ= -github.com/google/btree v1.0.0 h1:0udJVsspx3VBr5FwtLhQQtuAsVc79tTq0ocGIPAU6qo= github.com/google/btree v1.0.0/go.mod h1:lNA+9X1NB3Zf8V7Ke586lFgjr2dZNuvo3lPJSGZ5JPQ= +github.com/google/btree v1.0.1 h1:gK4Kx5IaGY9CD5sPJ36FHiBJ6ZXl0kilRiiCj+jdYp4= +github.com/google/btree v1.0.1/go.mod h1:xXMiIv4Fb/0kKde4SpL7qlzvu5cMJDRkFDxJfI9uaxA= github.com/google/go-cmp v0.2.0/go.mod h1:oXzfMopK8JAjlY9xF4vHSVASa0yLyX7SntLO5aqRK0M= github.com/google/go-cmp v0.3.0/go.mod h1:8QqcDgzrUqlUb/G2PQTWiueGozuR1884gddMywk6iLU= github.com/google/go-cmp v0.3.1/go.mod h1:8QqcDgzrUqlUb/G2PQTWiueGozuR1884gddMywk6iLU= @@ -498,6 +519,7 @@ github.com/google/pprof v0.0.0-20201023163331-3e6fc7fc9c4c/go.mod h1:kpwsk12EmLe github.com/google/pprof v0.0.0-20201203190320-1bf35d6f28c2/go.mod h1:kpwsk12EmLew5upagYY7GY0pfYCcupk39gWOCRROcvE= github.com/google/pprof v0.0.0-20210122040257-d980be63207e/go.mod h1:kpwsk12EmLew5upagYY7GY0pfYCcupk39gWOCRROcvE= github.com/google/pprof v0.0.0-20210226084205-cbba55b83ad5/go.mod h1:kpwsk12EmLew5upagYY7GY0pfYCcupk39gWOCRROcvE= +github.com/google/pprof v0.0.0-20210407192527-94a9f03dee38/go.mod h1:kpwsk12EmLew5upagYY7GY0pfYCcupk39gWOCRROcvE= github.com/google/pprof v0.0.0-20210601050228-01bbb1931b22/go.mod h1:kpwsk12EmLew5upagYY7GY0pfYCcupk39gWOCRROcvE= github.com/google/pprof v0.0.0-20210609004039-a478d1d731e9/go.mod h1:kpwsk12EmLew5upagYY7GY0pfYCcupk39gWOCRROcvE= github.com/google/pprof v0.0.0-20210720184732-4bb14d4b1be1/go.mod h1:kpwsk12EmLew5upagYY7GY0pfYCcupk39gWOCRROcvE= @@ -529,8 +551,9 @@ github.com/gorilla/websocket v1.4.2 h1:+/TMaTYc4QFitKJxsQ7Yye35DkWvkdLcvGKqM+x0U github.com/gorilla/websocket v1.4.2/go.mod h1:YR8l580nyteQvAITg2hZ9XVh4b55+EU/adAjf1fMHhE= github.com/grpc-ecosystem/go-grpc-middleware v1.0.0/go.mod h1:FiyG127CGDf3tlThmgyCl78X/SZQqEOJBCDaAfeWzPs= github.com/grpc-ecosystem/go-grpc-middleware v1.0.1-0.20190118093823-f849b5445de4/go.mod h1:FiyG127CGDf3tlThmgyCl78X/SZQqEOJBCDaAfeWzPs= -github.com/grpc-ecosystem/go-grpc-middleware v1.1.0 h1:THDBEeQ9xZ8JEaCLyLQqXMMdRqNr0QAUJTIkQAUtFjg= github.com/grpc-ecosystem/go-grpc-middleware v1.1.0/go.mod h1:f5nM7jw/oeRSadq3xCzHAvxcr8HZnzsqU6ILg/0NiiE= +github.com/grpc-ecosystem/go-grpc-middleware v1.3.0 h1:+9834+KizmvFV7pXQGSXQTsaWhq2GjuNUt0aUU0YBYw= +github.com/grpc-ecosystem/go-grpc-middleware v1.3.0/go.mod h1:z0ButlSOZa5vEBq9m2m2hlwIgKw+rp3sdCBRoJY+30Y= github.com/grpc-ecosystem/go-grpc-prometheus v1.2.0 h1:Ovs26xHkKqVztRpIrF/92BcuyuQ/YW4NSIpoGtfXNho= github.com/grpc-ecosystem/go-grpc-prometheus v1.2.0/go.mod h1:8NvIoxWQoOIhqOTXgfV/d3M/q6VIi02HzZEHgUlZvzk= github.com/grpc-ecosystem/grpc-gateway v1.8.5/go.mod h1:vNeuVxBJEsws4ogUvrchl83t/GYV9WGTSLVdBhOQFDY= @@ -580,7 +603,6 @@ github.com/improbable-eng/grpc-web v0.12.0 h1:GlCS+lMZzIkfouf7CNqY+qqpowdKuJLSLL github.com/improbable-eng/grpc-web v0.12.0/go.mod h1:6hRR09jOEG81ADP5wCQju1z71g6OL4eEvELdran/3cs= github.com/inconshreveable/mousetrap v1.0.0 h1:Z8tu5sraLXCXIcARxBp/8cbvlwVa7Z1NHg9XEKhtSvM= github.com/inconshreveable/mousetrap v1.0.0/go.mod h1:PxqpIevigyE2G7u3NXJIT2ANytuPF1OarO4DADm73n8= -github.com/influxdata/tdigest v0.0.1/go.mod h1:Z0kXnxzbTC2qrx4NaIzYkE1k66+6oEDQTvL95hQFh5Y= github.com/integralist/go-findroot v0.0.0-20160518114804-ac90681525dc h1:4IZpk3M4m6ypx0IlRoEyEyY1gAdicWLMQ0NcG/gBnnA= github.com/integralist/go-findroot v0.0.0-20160518114804-ac90681525dc/go.mod h1:UlaC6ndby46IJz9m/03cZPKKkR9ykeIVBBDE3UDBdJk= github.com/iris-contrib/blackfriday v2.0.0+incompatible/go.mod h1:UzZ2bDEoaSGPbkg6SAB4att1aAwTmVIx/5gCVqeyUdI= @@ -647,6 +669,8 @@ github.com/jonboulle/clockwork v0.1.0/go.mod h1:Ii8DK3G1RaLaWxj9trq07+26W01tbo22 github.com/jonboulle/clockwork v0.2.2 h1:UOGuzwb1PwsrDAObMuhUnj0p5ULPj8V/xJ7Kx9qUBdQ= github.com/jonboulle/clockwork v0.2.2/go.mod h1:Pkfl5aHPm1nk2H9h0bjmnJD/BcgbGXUBGnn1kMkgxc8= github.com/joomcode/errorx v1.0.1/go.mod h1:kgco15ekB6cs+4Xjzo7SPeXzx38PbJzBwbnu9qfVNHQ= +github.com/jpillora/backoff v1.0.0 h1:uvFg412JmmHBHw7iwprIxkPMI+sGQ4kzOWsMeHnm2EA= +github.com/jpillora/backoff v1.0.0/go.mod h1:J/6gKK9jxlEcS3zixgDgUAsiuZ7yrSoa/FX5e0EB2j4= github.com/json-iterator/go v1.1.5/go.mod h1:+SdeFBvtyEkXs7REEP0seUULqWtbJapLOCVDaaPEHmU= github.com/json-iterator/go v1.1.6/go.mod h1:+SdeFBvtyEkXs7REEP0seUULqWtbJapLOCVDaaPEHmU= github.com/json-iterator/go v1.1.7/go.mod h1:KdQUCv79m/52Kvf8AW2vK1V8akMuk1QjK/uOdHXbAo4= @@ -660,10 +684,10 @@ github.com/jstemmer/go-junit-report v0.9.1/go.mod h1:Brl9GWCQeLvo8nXZwPNNblvFj/X github.com/jtolds/gls v4.20.0+incompatible/go.mod h1:QJZ7F/aHp+rZTRtaJ1ow/lLfFfVYBRgL+9YlvaHOwJU= github.com/juju/errors v0.0.0-20181118221551-089d3ea4e4d5/go.mod h1:W54LbzXuIE0boCoNJfwqpmkKJ1O4TCTZMetAt6jGk7Q= github.com/juju/loggo v0.0.0-20180524022052-584905176618/go.mod h1:vgyd7OREkbtVEN/8IXZe5Ooef3LQePvuBm9UWj6ZL8U= -github.com/juju/ratelimit v1.0.1 h1:+7AIFJVQ0EQgq/K9+0Krm7m530Du7tIz0METWzN0RgY= github.com/juju/ratelimit v1.0.1/go.mod h1:qapgC/Gy+xNh9UxzV13HGGl/6UXNN+ct+vwSgWNm/qk= github.com/juju/testing v0.0.0-20180920084828-472a3e8b2073/go.mod h1:63prj8cnj0tU0S9OHjGJn+b1h0ZghCndfnbQolrYTwA= github.com/julienschmidt/httprouter v1.2.0/go.mod h1:SYymIcj16QtmaHHD7aYtjjsJG7VTCxuUUipMqKk8s4w= +github.com/julienschmidt/httprouter v1.3.0/go.mod h1:JR6WtHb+2LUe8TCKY3cZOxFyyO8IZAc4RVcycCCAKdM= github.com/jung-kurt/gofpdf v1.0.3-0.20190309125859-24315acbbda5/go.mod h1:7Id9E/uU8ce6rXgefFLlgrJj/GYY22cpxn+r32jIOes= github.com/k0kubun/colorstring v0.0.0-20150214042306-9440f1994b88/go.mod h1:3w7q1U84EfirKl04SVQ/s7nPm1ZPhiXd34z40TNz36k= github.com/kami-zh/go-capturer v0.0.0-20171211120116-e492ea43421d h1:cVtBfNW5XTHiKQe7jDaDBSh/EVM4XLPutLAGboIXuM0= @@ -734,6 +758,7 @@ github.com/lucasb-eyer/go-colorful v1.0.3/go.mod h1:R4dSotOR9KMtayYi1e77YzuveK+i github.com/lufia/plan9stats v0.0.0-20211012122336-39d0f177ccd0 h1:6E+4a0GO5zZEnZ81pIr0yLvtUWk2if982qA3F3QD6H4= github.com/lufia/plan9stats v0.0.0-20211012122336-39d0f177ccd0/go.mod h1:zJYVVT2jmtg6P3p1VtQj7WsuWi/y4VnjVBn7F8KPB3I= github.com/magiconair/properties v1.8.0/go.mod h1:PppfXfuXeibc/6YijjN8zIbojt8czPbwD3XqdrwzmxQ= +github.com/magiconair/properties v1.8.1/go.mod h1:PppfXfuXeibc/6YijjN8zIbojt8czPbwD3XqdrwzmxQ= github.com/magiconair/properties v1.8.5/go.mod h1:y3VJvCyxH9uVvJTWEGAELF3aiYNyPKd5NZ3oSwXrF60= github.com/mailru/easyjson v0.0.0-20180823135443-60711f1a8329/go.mod h1:C1wdFJiN94OJF2b5HbByQZoLdCWB1Yqtg26g4irojpc= github.com/mailru/easyjson v0.0.0-20190614124828-94de47d64c63/go.mod h1:C1wdFJiN94OJF2b5HbByQZoLdCWB1Yqtg26g4irojpc= @@ -802,7 +827,6 @@ github.com/modern-go/reflect2 v1.0.1/go.mod h1:bx2lNnkwVCuqBIxFjflWJWanXIb3Rllmb github.com/modern-go/reflect2 v1.0.2 h1:xBagoLtFs94CBntxluKeaWgTMpvLxC4ur3nMaC9Gz0M= github.com/modern-go/reflect2 v1.0.2/go.mod h1:yWuevngMOJpCy52FWWMvUC8ws7m/LJsjYzDa0/r8luk= github.com/modocache/gover v0.0.0-20171022184752-b58185e213c5/go.mod h1:caMODM3PzxT8aQXRPkAt8xlV/e7d7w8GM5g0fa5F0D8= -github.com/montanaflynn/stats v0.5.0 h1:2EkzeTSqBB4V4bJwWrt5gIIrZmpJBcoIRGS2kWLgzmk= github.com/montanaflynn/stats v0.5.0/go.mod h1:wL8QJuTMNUDYhXwkmfOly8iTdp5TEcJFWZD2D7SIkUc= github.com/moul/http2curl v1.0.0/go.mod h1:8UbvGypXm98wA/IqH45anm5Y2Z6ep6O31QGOAZ3H0fQ= github.com/mtibben/percent v0.2.1 h1:5gssi8Nqo8QU/r2pynCm+hBQHpkB/uNK7BJCFogWdzs= @@ -824,22 +848,29 @@ github.com/ngaut/sync2 v0.0.0-20141008032647-7a24ed77b2ef h1:K0Fn+DoFqNqktdZtdV3 github.com/ngaut/sync2 v0.0.0-20141008032647-7a24ed77b2ef/go.mod h1:7WjlapSfwQyo6LNmIvEWzsW1hbBQfpUO4JWnuQRmva8= github.com/nicksnyder/go-i18n v1.10.0/go.mod h1:HrK7VCrbOvQoUAQ7Vpy7i87N7JZZZ7R2xBGjv0j365Q= github.com/niemeyer/pretty v0.0.0-20200227124842-a10e7caefd8e/go.mod h1:zD1mROLANZcx1PVRCS0qkT7pwLkGfwJo4zjcN/Tysno= -github.com/nxadm/tail v1.4.4 h1:DQuhQpB1tVlglWS2hLQ5OV6B5r8aGxSrPc5Qo6uTN78= github.com/nxadm/tail v1.4.4/go.mod h1:kenIhsEOeOJmVchQTgglprH7qJGnHDVpk1VPCcaMI8A= +github.com/nxadm/tail v1.4.8 h1:nPr65rt6Y5JFSKQO7qToXr7pePgD6Gwiw05lkbyAQTE= +github.com/nxadm/tail v1.4.8/go.mod h1:+ncqLTQzXmGhMZNUePPaPqPvBxHAIsmXswZKocGu+AU= github.com/oklog/ulid v1.3.1/go.mod h1:CirwcVhetQ6Lv90oh/F+FBtV6XMibvdAFo93nm5qn4U= github.com/oleiade/reflections v1.0.1/go.mod h1:rdFxbxq4QXVZWj0F+e9jqjDkc7dbp97vkRixKo2JR60= github.com/olekukonko/tablewriter v0.0.0-20170122224234-a0225b3f23b5/go.mod h1:vsDQFd/mU46D+Z4whnwzcISnGGzXWMclvtLoiIKAKIo= github.com/olekukonko/tablewriter v0.0.4/go.mod h1:zq6QwlOf5SlnkVbMSr5EoBv3636FWnp+qbPhuoO21uA= +github.com/olekukonko/tablewriter v0.0.5/go.mod h1:hPp6KlRPjbx+hW8ykQs1w3UBbZlj6HuIJcUGPhkA7kY= github.com/onsi/ginkgo v1.6.0/go.mod h1:lLunBs/Ym6LB5Z9jYTR76FiuTmxDTDusOGeTQH+WWjE= github.com/onsi/ginkgo v1.7.0/go.mod h1:lLunBs/Ym6LB5Z9jYTR76FiuTmxDTDusOGeTQH+WWjE= github.com/onsi/ginkgo v1.12.1/go.mod h1:zj2OWP4+oCPe1qIXoGWkgMRwljMUYCdkwsT2108oapk= github.com/onsi/ginkgo v1.13.0/go.mod h1:+REjRxOmWfHCjfv9TTWB1jD1Frx4XydAD3zm1lskyM0= -github.com/onsi/ginkgo v1.14.0 h1:2mOpI4JVVPBN+WQRa0WKH2eXR+Ey+uK4n7Zj0aYpIQA= github.com/onsi/ginkgo v1.14.0/go.mod h1:iSB4RoI2tjJc9BBv4NKIKWKya62Rps+oPG/Lv9klQyY= +github.com/onsi/ginkgo v1.16.4/go.mod h1:dX+/inL/fNMqNlz0e9LfyB9TswhZpCVdJM/Z6Vvnwo0= +github.com/onsi/ginkgo v1.16.5 h1:8xi0RTUf59SOSfEtZMvwTvXYMzG4gV23XVHOZiXNtnE= +github.com/onsi/ginkgo v1.16.5/go.mod h1:+E8gABHa3K6zRBolWtd+ROzc/U5bkGt0FwiG042wbpU= +github.com/onsi/ginkgo/v2 v2.0.0/go.mod h1:vw5CSIxN1JObi/U8gcbwft7ZxR2dgaR70JSE3/PpL4c= github.com/onsi/gomega v1.4.3/go.mod h1:ex+gbHU/CVuBBDIJjb2X0qEXbFg53c61hWP/1CpauHY= github.com/onsi/gomega v1.7.1/go.mod h1:XdKZgCCFLUoM/7CFJVPcG8C1xQ1AJ0vpAezJrB7JYyY= -github.com/onsi/gomega v1.10.1 h1:o0+MgICZLuZ7xjH7Vx6zS/zcu93/BEp1VwkIW1mEXCE= github.com/onsi/gomega v1.10.1/go.mod h1:iN09h71vgCQne3DLsj+A5owkum+a2tYe+TOCB1ybHNo= +github.com/onsi/gomega v1.17.0/go.mod h1:HnhC7FXeEQY45zxNK3PPoIUhzk/80Xly9PcubAlGdZY= +github.com/onsi/gomega v1.18.1 h1:M1GfJqGRrBrrGGsbxzV5dqM2U2ApXefZCQpkukxYRLE= +github.com/onsi/gomega v1.18.1/go.mod h1:0q+aL8jAiMXy9hbwj2mr5GziHiwhAIQpFmmtT5hitRs= github.com/opentracing/basictracer-go v1.0.0 h1:YyUAhaEfjoWXclZVJ9sGoNct7j4TVk7lZWlQw5UXuoo= github.com/opentracing/basictracer-go v1.0.0/go.mod h1:QfBfYuafItcjQuMwinw9GhYKwFXS9KnPs5lxoYwgW74= github.com/opentracing/opentracing-go v1.1.0/go.mod h1:UkNAQd3GIcIGf0SeVgPpRdFStlNbqXla1AfSYxPUl2o= @@ -855,7 +886,6 @@ github.com/petermattis/goid v0.0.0-20180202154549-b0b1615b78e5/go.mod h1:jvVRKCr github.com/petermattis/goid v0.0.0-20211229010228-4d14c490ee36/go.mod h1:pxMtw7cyUw6B2bRH0ZBANSPg+AoSud1I1iyJHI69jH4= github.com/phayes/freeport v0.0.0-20180830031419-95f893ade6f2 h1:JhzVVoYvbOACxoUmOs6V/G4D5nPVUW73rKvXxP4XUJc= github.com/phayes/freeport v0.0.0-20180830031419-95f893ade6f2/go.mod h1:iIss55rKnNBTvrwdmkUpLnDpZoAHvWaiq5+iMmen4AE= -github.com/phf/go-queue v0.0.0-20170504031614-9abe38d0371d h1:U+PMnTlV2tu7RuMK5etusZG3Cf+rpow5hqQByeCzJ2g= github.com/phf/go-queue v0.0.0-20170504031614-9abe38d0371d/go.mod h1:lXfE4PvvTW5xOjO6Mba8zDPyw8M93B6AQ7frTGnMlA8= github.com/philhofer/fwd v1.0.0 h1:UbZqGr5Y38ApvM/V/jEljVxwocdweyH+vmYvRPBnbqQ= github.com/philhofer/fwd v1.0.0/go.mod h1:gk3iGcWd9+svBvR0sR+KPcfE+RNWozjowpeBVG3ZVNU= @@ -870,7 +900,6 @@ github.com/pingcap/check v0.0.0-20191216031241-8a5a85928f12/go.mod h1:PYMCGwN0JH github.com/pingcap/check v0.0.0-20200212061837-5e12011dc712/go.mod h1:PYMCGwN0JHjoqGr3HrZoD+b8Tgx8bKnArhSq8YVzUMc= github.com/pingcap/check v0.0.0-20211026125417-57bd13f7b5f0 h1:HVl5539r48eA+uDuX/ziBmQCxzT1pGrzWbKuXT46Bq0= github.com/pingcap/check v0.0.0-20211026125417-57bd13f7b5f0/go.mod h1:PYMCGwN0JHjoqGr3HrZoD+b8Tgx8bKnArhSq8YVzUMc= -github.com/pingcap/errcode v0.3.0 h1:IF6LC/4+b1KNwrMlr2rBTUrojFPMexXBcDWZSpNwxjg= github.com/pingcap/errcode v0.3.0/go.mod h1:4b2X8xSqxIroj/IZ9MX/VGZhAwc11wB9wRIzHvz6SeM= github.com/pingcap/errors v0.11.0/go.mod h1:Oi8TUi2kEtXXLMJk9l1cGmz20kV3TaQ0usTwv5KuLY8= github.com/pingcap/errors v0.11.4/go.mod h1:Oi8TUi2kEtXXLMJk9l1cGmz20kV3TaQ0usTwv5KuLY8= @@ -896,17 +925,19 @@ github.com/pingcap/kvproto v0.0.0-20200411081810-b85805c9476c/go.mod h1:IOdRDPLy github.com/pingcap/kvproto v0.0.0-20210219064844-c1844a4775d6/go.mod h1:IOdRDPLyda8GX2hE/jO7gqaCV/PNFh8BZQCQZXfIOqI= github.com/pingcap/kvproto v0.0.0-20210805052247-76981389e818/go.mod h1:IOdRDPLyda8GX2hE/jO7gqaCV/PNFh8BZQCQZXfIOqI= github.com/pingcap/kvproto v0.0.0-20210915062418-0f5764a128ad/go.mod h1:IOdRDPLyda8GX2hE/jO7gqaCV/PNFh8BZQCQZXfIOqI= -github.com/pingcap/kvproto v0.0.0-20211213085605-3329b3c5404c/go.mod h1:IOdRDPLyda8GX2hE/jO7gqaCV/PNFh8BZQCQZXfIOqI= -github.com/pingcap/kvproto v0.0.0-20211224055123-d1a140660c39 h1:ygiBX0nLWmpN3tD2I0OHhRZ7pOlh8PdjqvuzoaCJEEg= -github.com/pingcap/kvproto v0.0.0-20211224055123-d1a140660c39/go.mod h1:IOdRDPLyda8GX2hE/jO7gqaCV/PNFh8BZQCQZXfIOqI= +github.com/pingcap/kvproto v0.0.0-20220106070556-3fa8fa04f898/go.mod h1:IOdRDPLyda8GX2hE/jO7gqaCV/PNFh8BZQCQZXfIOqI= +github.com/pingcap/kvproto v0.0.0-20220215045702-d229fcc888c8/go.mod h1:IOdRDPLyda8GX2hE/jO7gqaCV/PNFh8BZQCQZXfIOqI= +github.com/pingcap/kvproto v0.0.0-20220228094105-9bb22e5a97fc/go.mod h1:IOdRDPLyda8GX2hE/jO7gqaCV/PNFh8BZQCQZXfIOqI= +github.com/pingcap/kvproto v0.0.0-20220303025110-d5eb908517b2 h1:l+d3TYHO1ohk/li9YlOH0efNecAPdI4s4Ze5jbS6oLc= +github.com/pingcap/kvproto v0.0.0-20220303025110-d5eb908517b2/go.mod h1:IOdRDPLyda8GX2hE/jO7gqaCV/PNFh8BZQCQZXfIOqI= github.com/pingcap/log v0.0.0-20191012051959-b742a5d432e9/go.mod h1:4rbK1p9ILyIfb6hU7OG2CiWSqMXnp3JMbiaVJ6mvoY8= github.com/pingcap/log v0.0.0-20200511115504-543df19646ad/go.mod h1:4rbK1p9ILyIfb6hU7OG2CiWSqMXnp3JMbiaVJ6mvoY8= github.com/pingcap/log v0.0.0-20201112100606-8f1e84a3abc8/go.mod h1:4rbK1p9ILyIfb6hU7OG2CiWSqMXnp3JMbiaVJ6mvoY8= github.com/pingcap/log v0.0.0-20210317133921-96f4fcab92a4/go.mod h1:4rbK1p9ILyIfb6hU7OG2CiWSqMXnp3JMbiaVJ6mvoY8= github.com/pingcap/log v0.0.0-20210625125904-98ed8e2eb1c7/go.mod h1:8AanEdAHATuRurdGxZXBz0At+9avep+ub7U1AGYLIMM= github.com/pingcap/log v0.0.0-20210906054005-afc726e70354/go.mod h1:DWQW5jICDR7UJh4HtxXSM20Churx4CQL0fwL/SoOSA4= -github.com/pingcap/log v0.0.0-20211207084639-71a2e5860834 h1:/wKEagMl3KDpt7758/4Z+31yY3uxQCygOZ1xSbRXCP0= -github.com/pingcap/log v0.0.0-20211207084639-71a2e5860834/go.mod h1:DWQW5jICDR7UJh4HtxXSM20Churx4CQL0fwL/SoOSA4= +github.com/pingcap/log v0.0.0-20211215031037-e024ba4eb0ee h1:VO2t6IBpfvW34TdtD/G10VvnGqjLic1jzOuHjUb5VqM= +github.com/pingcap/log v0.0.0-20211215031037-e024ba4eb0ee/go.mod h1:DWQW5jICDR7UJh4HtxXSM20Churx4CQL0fwL/SoOSA4= github.com/pingcap/parser v0.0.0-20210415081931-48e7f467fd74/go.mod h1:xZC8I7bug4GJ5KtHhgAikjTfU4kBv1Sbo3Pf1MZ6lVw= github.com/pingcap/parser v0.0.0-20210525032559-c37778aff307/go.mod h1:xZC8I7bug4GJ5KtHhgAikjTfU4kBv1Sbo3Pf1MZ6lVw= github.com/pingcap/sysutil v0.0.0-20200206130906-2bfa6dc40bcd/go.mod h1:EB/852NMQ+aRKioCpToQ94Wl7fktV+FNnxf3CX/TTXI= @@ -916,23 +947,22 @@ github.com/pingcap/sysutil v0.0.0-20211208032423-041a72e5860d/go.mod h1:7j18ezaW github.com/pingcap/sysutil v0.0.0-20220114020952-ea68d2dbf5b4 h1:HYbcxtnkN3s5tqrZ/z3eJS4j3Db8wMphEm1q10lY/TM= github.com/pingcap/sysutil v0.0.0-20220114020952-ea68d2dbf5b4/go.mod h1:sDCsM39cGiv2vwunZkaFA917vVkqDTGSPbbV7z4Oops= github.com/pingcap/tidb v1.1.0-beta.0.20211023132847-efa94595c071/go.mod h1:Ci7ABF58a4jn6YtaHi7655jP409edqC2JxWWFRqOubg= -github.com/pingcap/tidb v1.1.0-beta.0.20220210065735-92f5a6340ca2 h1:0GZfSPfQ5zSXPo+mn8mqSsm6G/JerW8L+KZXGLT8pLk= -github.com/pingcap/tidb v1.1.0-beta.0.20220210065735-92f5a6340ca2/go.mod h1:1ATxD29hPy7OXNIxyZWb9PqA12F5YjmYoO9uh/655OY= +github.com/pingcap/tidb v1.1.0-beta.0.20220303050146-f5a52909b6af h1:XZfzhsveemMcFOnltF+V6ZwzQ8p1iczdHXwbZnnXuaE= +github.com/pingcap/tidb v1.1.0-beta.0.20220303050146-f5a52909b6af/go.mod h1:tXnQq3ZPgehlgpeZ8X+Mr/dvwjiAlmE6nt3H8pOY4zI= github.com/pingcap/tidb-dashboard v0.0.0-20210312062513-eef5d6404638/go.mod h1:OzFN8H0EDMMqeulPhPMw2i2JaiZWOKFQ7zdRPhENNgo= github.com/pingcap/tidb-dashboard v0.0.0-20210716172320-2226872e3296/go.mod h1:OCXbZTBTIMRcIt0jFsuCakZP+goYRv6IjawKbwLS2TQ= -github.com/pingcap/tidb-dashboard v0.0.0-20211206031355-bcc43a01d537/go.mod h1:OCXbZTBTIMRcIt0jFsuCakZP+goYRv6IjawKbwLS2TQ= github.com/pingcap/tidb-dashboard v0.0.0-20220117082709-e8076b5c79ba/go.mod h1:4hk/3owVGWdvI9Kx6yCqqvM1T5PVgwyQNyMQxD3rwfc= github.com/pingcap/tidb-tools v5.0.3+incompatible/go.mod h1:XGdcy9+yqlDSEMTpOXnwf3hiTeqrV6MN/u1se9N8yIM= -github.com/pingcap/tidb-tools v5.2.2-0.20211019062242-37a8bef2fa17+incompatible/go.mod h1:XGdcy9+yqlDSEMTpOXnwf3hiTeqrV6MN/u1se9N8yIM= -github.com/pingcap/tidb-tools v5.2.3-0.20211105044302-2dabb6641a6e+incompatible h1:acgn+tiufY2FnubfxaBuRjqvrknesyP8tXOTi2Otkwg= -github.com/pingcap/tidb-tools v5.2.3-0.20211105044302-2dabb6641a6e+incompatible/go.mod h1:XGdcy9+yqlDSEMTpOXnwf3hiTeqrV6MN/u1se9N8yIM= +github.com/pingcap/tidb-tools v5.4.1-0.20220221041303-00998a9a4bfd+incompatible/go.mod h1:XGdcy9+yqlDSEMTpOXnwf3hiTeqrV6MN/u1se9N8yIM= +github.com/pingcap/tidb-tools v5.4.1-0.20220302060346-b6d9cbe5dff1+incompatible h1:DLtjKK4k9++IE4zJBjfPbA0kAncx9uVQVJiamT/Cglc= +github.com/pingcap/tidb-tools v5.4.1-0.20220302060346-b6d9cbe5dff1+incompatible/go.mod h1:XGdcy9+yqlDSEMTpOXnwf3hiTeqrV6MN/u1se9N8yIM= github.com/pingcap/tidb/parser v0.0.0-20211011031125-9b13dc409c5e/go.mod h1:e1MGCA9Sg3T8jid8PKAEq5eYVuMMCq4n8gJ+Kqp4Plg= github.com/pingcap/tidb/parser v0.0.0-20211023132847-efa94595c071/go.mod h1:e1MGCA9Sg3T8jid8PKAEq5eYVuMMCq4n8gJ+Kqp4Plg= -github.com/pingcap/tidb/parser v0.0.0-20220210065735-92f5a6340ca2 h1:qbcHdNrYV+9kLFfF5Rc5vFI4wDPRdJtaFUbKDK6LD88= -github.com/pingcap/tidb/parser v0.0.0-20220210065735-92f5a6340ca2/go.mod h1:ElJiub4lRy6UZDb+0JHDkGEdr6aOli+ykhyej7VCLoI= +github.com/pingcap/tidb/parser v0.0.0-20220303050146-f5a52909b6af h1:4ZapyLBNNKE4byEp+BekyIP0PV8e2UzIwci6TtIf2Pg= +github.com/pingcap/tidb/parser v0.0.0-20220303050146-f5a52909b6af/go.mod h1:ElJiub4lRy6UZDb+0JHDkGEdr6aOli+ykhyej7VCLoI= github.com/pingcap/tipb v0.0.0-20211008080435-3fd327dfce0e/go.mod h1:A7mrd7WHBl1o63LE2bIBGEJMTNWXqhgmYiOvMLxozfs= -github.com/pingcap/tipb v0.0.0-20220107024056-3b91949a18a7 h1:DHU4vw0o15qdKsf7d/Pyhun4YtX8FwoDQxG0plPByUg= -github.com/pingcap/tipb v0.0.0-20220107024056-3b91949a18a7/go.mod h1:A7mrd7WHBl1o63LE2bIBGEJMTNWXqhgmYiOvMLxozfs= +github.com/pingcap/tipb v0.0.0-20220110031732-29e23c62eeac h1:bVklq/Np5uHUylW4Htyhi92TNrvIqVwht/+bHk0R/Tk= +github.com/pingcap/tipb v0.0.0-20220110031732-29e23c62eeac/go.mod h1:A7mrd7WHBl1o63LE2bIBGEJMTNWXqhgmYiOvMLxozfs= github.com/pkg/browser v0.0.0-20180916011732-0a3d74bf9ce4 h1:49lOXmGaUpV9Fz3gd7TFZY106KVlPVa5jcYD1gaQf98= github.com/pkg/browser v0.0.0-20180916011732-0a3d74bf9ce4/go.mod h1:4OwLy04Bl9Ef3GJJCoec+30X3LQs/0/m4HFRt/2LUSA= github.com/pkg/diff v0.0.0-20210226163009-20ebb0f2a09e/go.mod h1:pJLUxLENpZxwdsKMEsNbx1VGcRFpLqf3715MtcvvzbA= @@ -955,8 +985,9 @@ github.com/prometheus/client_golang v1.0.0/go.mod h1:db9x61etRT2tGnBNRi70OPL5Fsn github.com/prometheus/client_golang v1.1.0/go.mod h1:I1FGZT9+L76gKKOs5djB6ezCbFQP1xR9D75/vuwEF3g= github.com/prometheus/client_golang v1.2.1/go.mod h1:XMU6Z2MjaRKVu/dC1qupJI9SiNkDYzz3xecMgSW/F+U= github.com/prometheus/client_golang v1.5.1/go.mod h1:e9GMxYsXl05ICDXkRhurwBS4Q3OK1iX/F2sw+iXX5zU= -github.com/prometheus/client_golang v1.7.1 h1:NTGy1Ja9pByO+xAeH/qiWnLrKtr3hJPNjaVUwnjpdpA= github.com/prometheus/client_golang v1.7.1/go.mod h1:PY5Wy2awLA44sXw4AOSfFBetzPP4j5+D6mVACh+pe2M= +github.com/prometheus/client_golang v1.11.0 h1:HNkLOAEQMIDv/K+04rukrLx6ch7msSRwf3/SASFAGtQ= +github.com/prometheus/client_golang v1.11.0/go.mod h1:Z6t4BnS23TR94PD6BsDNk8yVqroYurpAkEiz0P2BEV0= github.com/prometheus/client_model v0.0.0-20180712105110-5c3871d89910/go.mod h1:MbSGuTsp3dbXC40dX6PRTWyKYBIrTGTE9sqQNg2J8bo= github.com/prometheus/client_model v0.0.0-20190129233127-fd36f4220a90/go.mod h1:xMI15A0UPsDsEKsMN9yxemIoYk6Tm2C1GtYGdfGttqA= github.com/prometheus/client_model v0.0.0-20190812154241-14fe0d1b01d4/go.mod h1:xMI15A0UPsDsEKsMN9yxemIoYk6Tm2C1GtYGdfGttqA= @@ -969,16 +1000,18 @@ github.com/prometheus/common v0.4.1/go.mod h1:TNfzLD0ON7rHzMJeJkieUDPYmFC7Snx/y8 github.com/prometheus/common v0.6.0/go.mod h1:eBmuwkDJBwy6iBfxCBob6t6dR6ENT/y+J+Zk0j9GMYc= github.com/prometheus/common v0.7.0/go.mod h1:DjGbpBbp5NYNiECxcL/VnbXCCaQpKd3tt26CguLLsqA= github.com/prometheus/common v0.9.1/go.mod h1:yhUN8i9wzaXS3w1O07YhxHEBxD+W35wd8bs7vj7HSQ4= -github.com/prometheus/common v0.10.0 h1:RyRA7RzGXQZiW+tGMr7sxa85G1z0yOpM1qq5c8lNawc= github.com/prometheus/common v0.10.0/go.mod h1:Tlit/dnDKsSWFlCLTWaA1cyBgKHSMdTB80sz/V91rCo= +github.com/prometheus/common v0.26.0 h1:iMAkS2TDoNWnKM+Kopnx/8tnEStIfpYA0ur0xQzzhMQ= +github.com/prometheus/common v0.26.0/go.mod h1:M7rCNAaPfAosfx8veZJCuw84e35h3Cfd9VFqTh1DIvc= github.com/prometheus/procfs v0.0.0-20181005140218-185b4288413d/go.mod h1:c3At6R/oaqEKCNdg8wHV1ftS6bRYblBhIjjI8uT2IGk= github.com/prometheus/procfs v0.0.0-20190507164030-5867b95ac084/go.mod h1:TjEm7ze935MbeOT/UhFTIMYKhuLP4wbCsTZCD3I8kEA= github.com/prometheus/procfs v0.0.2/go.mod h1:TjEm7ze935MbeOT/UhFTIMYKhuLP4wbCsTZCD3I8kEA= github.com/prometheus/procfs v0.0.3/go.mod h1:4A/X28fw3Fc593LaREMrKMqOKvUAntwMDaekg4FpcdQ= github.com/prometheus/procfs v0.0.5/go.mod h1:4A/X28fw3Fc593LaREMrKMqOKvUAntwMDaekg4FpcdQ= github.com/prometheus/procfs v0.0.8/go.mod h1:7Qr8sr6344vo1JqZ6HhLceV9o3AJ1Ff+GxbHq6oeK9A= -github.com/prometheus/procfs v0.1.3 h1:F0+tqvhOksq22sc6iCHF5WGlWjdwj92p0udFh1VFBS8= github.com/prometheus/procfs v0.1.3/go.mod h1:lV6e/gmhEcM9IjHGsFOCxxuZ+z1YqCvr4OA4YeYWdaU= +github.com/prometheus/procfs v0.6.0 h1:mxy4L2jP6qMonqmq+aTtOx1ifVWUgG/TAmntgbh3xv4= +github.com/prometheus/procfs v0.6.0/go.mod h1:cz+aTbrPOrUb4q7XlbU9ygM+/jj0fzG6c1xBZuNvfVA= github.com/prometheus/tsdb v0.7.1/go.mod h1:qhTCs0VvXwvX/y3TZrWD7rabWM+ijKTux40TwIPHuXU= github.com/r3labs/diff v1.1.0 h1:V53xhrbTHrWFWq3gI4b94AjgEJOerO1+1l0xyHOBi8M= github.com/r3labs/diff v1.1.0/go.mod h1:7WjXasNzi0vJetRcB/RqNl5dlIsmXcTTLmF5IoH6Xig= @@ -1030,8 +1063,9 @@ github.com/shurcooL/httpfs v0.0.0-20190707220628-8d4bc4ba7749/go.mod h1:ZY1cvUeJ github.com/shurcooL/httpgzip v0.0.0-20190720172056-320755c1c1b0 h1:mj/nMDAwTBiaCqMEs4cYCqF7pO6Np7vhy1D1wcQGz+E= github.com/shurcooL/httpgzip v0.0.0-20190720172056-320755c1c1b0/go.mod h1:919LwcH0M7/W4fcZ0/jy0qGght1GIhqyS/EgWGH2j5Q= github.com/shurcooL/sanitized_anchor_name v1.0.0/go.mod h1:1NzhyTcUVG4SuEtjjoZeVRXNmyL/1OwPU0+IJeTBvfc= -github.com/shurcooL/vfsgen v0.0.0-20181202132449-6a9ea43bcacd h1:ug7PpSOB5RBPK1Kg6qskGBoP3Vnj/aNYFTznWvlkGo0= github.com/shurcooL/vfsgen v0.0.0-20181202132449-6a9ea43bcacd/go.mod h1:TrYk7fJVaAttu97ZZKrO9UbRa8izdowaMIZcxYMbVaw= +github.com/shurcooL/vfsgen v0.0.0-20200824052919-0d455de96546 h1:pXY9qYc/MP5zdvqWEUH6SjNiu7VhSjuVFTFiTcphaLU= +github.com/shurcooL/vfsgen v0.0.0-20200824052919-0d455de96546/go.mod h1:TrYk7fJVaAttu97ZZKrO9UbRa8izdowaMIZcxYMbVaw= github.com/siddontang/go v0.0.0-20180604090527-bdc77568d726 h1:xT+JlYxNGqyT+XcU8iUrN18JYed2TvG9yN5ULG2jATM= github.com/siddontang/go v0.0.0-20180604090527-bdc77568d726/go.mod h1:3yhqj7WBBfRhbBlzyOC3gUxftwsU0u8gqevxwIHQpMw= github.com/siddontang/go-log v0.0.0-20180807004314-8d05993dda07 h1:oI+RNwuC9jF2g2lP0u0cVEEZrc/AYBCuFdvwrLWM/6Q= @@ -1040,6 +1074,7 @@ github.com/sirupsen/logrus v1.2.0/go.mod h1:LxeOpSwHxABJmUn/MG1IvRgCAasNZTLOkJPx github.com/sirupsen/logrus v1.4.1/go.mod h1:ni0Sbl8bgC9z8RoU9G6nDWqqs/fq4eDPysMBDgk/93Q= github.com/sirupsen/logrus v1.4.2/go.mod h1:tLMulIdttU9McNUspp0xgXVQah82FyeX6MwdIuYE2rE= github.com/sirupsen/logrus v1.6.0/go.mod h1:7uNnSEd1DgxDLC74fIahvMZmmYsHGZGEOFrfsX/uA88= +github.com/sirupsen/logrus v1.7.0/go.mod h1:yWOB1SBYBC5VeMP7gHvWumXLIWorT60ONWic61uBYv0= github.com/sirupsen/logrus v1.8.1 h1:dJKuHgqk1NNQlqoA6BTlM1Wf9DOH3NBjQyu0h9+AZZE= github.com/sirupsen/logrus v1.8.1/go.mod h1:yWOB1SBYBC5VeMP7gHvWumXLIWorT60ONWic61uBYv0= github.com/smartystreets/assertions v0.0.0-20180927180507-b2de0cb4f26d/go.mod h1:OnSkiWE9lh6wB0YB77sQom3nweQdgAjqCqsofrRNTgc= @@ -1058,6 +1093,7 @@ github.com/spf13/cast v1.3.1/go.mod h1:Qx5cxh0v+4UWYiBimWS+eyWzqEqokIECu5etghLkU github.com/spf13/cobra v0.0.3/go.mod h1:1l0Ry5zgKvJasoi3XT1TypsSe7PqH0Sj9dhYf7v3XqQ= github.com/spf13/cobra v0.0.5/go.mod h1:3K3wKZymM7VvHMDS9+Akkh4K60UwM26emMESw8tLCHU= github.com/spf13/cobra v1.0.0/go.mod h1:/6GTrnGXV9HjY+aR4k0oJ5tcvakLuG6EuKReYlHNrgE= +github.com/spf13/cobra v1.1.3/go.mod h1:pGADOWyqRD/YMrPZigI/zbliZ2wVD/23d+is3pSWzOo= github.com/spf13/cobra v1.2.1 h1:+KmjbUw1hriSNMF55oPrkZcb27aECyrj8V2ytv7kWDw= github.com/spf13/cobra v1.2.1/go.mod h1:ExllRjgxM/piMAM+3tAZvg8fsklGAf3tPfi+i8t68Nk= github.com/spf13/jwalterweatherman v1.0.0/go.mod h1:cQK4TGJAtQXfYWX+Ddv3mKDzgVb68N+wFjFa4jdeBTo= @@ -1068,6 +1104,7 @@ github.com/spf13/pflag v1.0.5 h1:iy+VFUOCP1a+8yFto/drg2CJ5u0yRoB7fZw3DKv/JXA= github.com/spf13/pflag v1.0.5/go.mod h1:McXfInJRrz4CZXVZOBLb0bTZqETkiAhM9Iw0y3An2Bg= github.com/spf13/viper v1.3.2/go.mod h1:ZiWeW+zYFKm7srdB9IoDzzZXaJaI5eL9QjNiN/DMA2s= github.com/spf13/viper v1.4.0/go.mod h1:PTJ7Z/lr49W6bUbkmS1V3by4uWynFiR9p7+dSq/yZzE= +github.com/spf13/viper v1.7.0/go.mod h1:8WkrPz2fc9jxqZNCJI/76HCieCp4Q8HaLFoCha5qpdg= github.com/spf13/viper v1.8.1/go.mod h1:o0Pch8wJ9BVSWGQMbra6iw0oQ5oktSIBaujf1rJH9Ns= github.com/stretchr/objx v0.1.0/go.mod h1:HFkY916IF+rwdDfMAkV7OtwuqBVzrE8GR6GFx+wExME= github.com/stretchr/objx v0.1.1/go.mod h1:HFkY916IF+rwdDfMAkV7OtwuqBVzrE8GR6GFx+wExME= @@ -1112,13 +1149,15 @@ github.com/tidwall/pretty v1.2.0/go.mod h1:ITEVvHYasfjBbM0u2Pg8T2nJnzm8xPwvNhhso github.com/tidwall/sjson v1.2.2 h1:H1Llj/C9G+BoUN2DsybLHjWvr9dx4Uazavf0sXQ+rOs= github.com/tidwall/sjson v1.2.2/go.mod h1:jmW2RZpbKuExPFUHeFSBMiovT9ZyOziEHDRkbsdp0B0= github.com/tikv/client-go/v2 v2.0.0-alpha.0.20211011083157-49c8dd23f1f0/go.mod h1:00plYwQsQ5kBUmafHO+JkjznGgFaBokMZl82TZIbsQk= -github.com/tikv/client-go/v2 v2.0.0-rc.0.20220107040026-d22815099720 h1:GgXeSpvljDDkwScjOXM4NSVrR6hWnQthM8ZiHGHyBf8= -github.com/tikv/client-go/v2 v2.0.0-rc.0.20220107040026-d22815099720/go.mod h1:Yn4EKNn2y8XrudACiBGnVpDwfIXqi5oLPADpIoAcPyg= +github.com/tikv/client-go/v2 v2.0.0-rc.0.20220217140116-589cf79b2ab5 h1:xfhhUvnueHqt+mYfRR5ONAVezrqNBCvcWSh4Ml6Isjc= +github.com/tikv/client-go/v2 v2.0.0-rc.0.20220217140116-589cf79b2ab5/go.mod h1:gaHSp8rnxZ0w36qb6QPPNPh9P0Mu5vAEwCQcc0Brni4= github.com/tikv/pd v1.1.0-beta.0.20210323121136-78679e5e209d/go.mod h1:Jw9KG11C/23Rr7DW4XWQ7H5xOgGZo6DFL1OKAF4+Igw= github.com/tikv/pd v1.1.0-beta.0.20210818082359-acba1da0018d/go.mod h1:rammPjeZgpvfrQRPkijcx8tlxF1XM5+m6kRXrkDzCAA= -github.com/tikv/pd v1.1.0-beta.0.20211214024235-3c626f28bd32/go.mod h1:v9lIYxM8VeWZl4aF7jqsDXvqQoRjXdi7vYo5cO1g/2w= -github.com/tikv/pd v1.1.0-beta.0.20220207063535-9268bed87199 h1:54W4oA8BhAEyxliOZgWeIiOZQB6Oc25EXXRN3/hOSEE= -github.com/tikv/pd v1.1.0-beta.0.20220207063535-9268bed87199/go.mod h1:PWnQYE+JxAk4A31CtmZiBAhBMLXsAc9JaHCHpakX9Is= +github.com/tikv/pd v1.1.0-beta.0.20220303060546-3695d8164800 h1:lIfIwqe1HPa0suhMpiI200nYxau+rXWXTqZxSGg1HS4= +github.com/tikv/pd v1.1.0-beta.0.20220303060546-3695d8164800/go.mod h1:J/dj1zpEE9b7idgONGFttnXM+ncl88LmnkD/xDbq0hA= +github.com/tikv/pd/client v0.0.0-20220216070739-26c668271201/go.mod h1:fEvI5fhAuJn1Fn87VJF8ByE9Vc16EzWGoePZB21/nL8= +github.com/tikv/pd/client v0.0.0-20220216080339-1b8f82378ee7 h1:RRiYmyzHgTgV0mrp6Ue6cGn0EAB7U1YHnEfVWEGCKk8= +github.com/tikv/pd/client v0.0.0-20220216080339-1b8f82378ee7/go.mod h1:fEvI5fhAuJn1Fn87VJF8ByE9Vc16EzWGoePZB21/nL8= github.com/tinylib/msgp v1.1.0 h1:9fQd+ICuRIu/ue4vxJZu6/LzxN0HwMds2nq/0cFvxHU= github.com/tinylib/msgp v1.1.0/go.mod h1:+d+yLhGm8mzTaHzB+wgMYrodPfmZrzkirds8fDWklFE= github.com/tklauser/go-sysconf v0.3.4/go.mod h1:Cl2c8ZRWfHD5IrfHo9VN+FX9kCFjIOyVklgXycLB6ek= @@ -1129,8 +1168,9 @@ github.com/tklauser/numcpus v0.3.0 h1:ILuRUQBtssgnxw0XXIjKUC56fgnOrFoQQ/4+DeU2bi github.com/tklauser/numcpus v0.3.0/go.mod h1:yFGUr7TUHQRAhyqBcEg0Ge34zDBAsIvJJcyE6boqnA8= github.com/tmc/grpc-websocket-proxy v0.0.0-20170815181823-89b8d40f7ca8/go.mod h1:ncp9v5uamzpCO7NfCPTXjqaC+bZgJeR0sMTm6dMHP7U= github.com/tmc/grpc-websocket-proxy v0.0.0-20190109142713-0ad062ec5ee5/go.mod h1:ncp9v5uamzpCO7NfCPTXjqaC+bZgJeR0sMTm6dMHP7U= -github.com/tmc/grpc-websocket-proxy v0.0.0-20200427203606-3cfed13b9966 h1:j6JEOq5QWFker+d7mFQYOhjTZonQ7YkLTHm56dbn+yM= github.com/tmc/grpc-websocket-proxy v0.0.0-20200427203606-3cfed13b9966/go.mod h1:ncp9v5uamzpCO7NfCPTXjqaC+bZgJeR0sMTm6dMHP7U= +github.com/tmc/grpc-websocket-proxy v0.0.0-20201229170055-e5319fda7802 h1:uruHq4dN7GR16kFc5fp3d1RIYzJW5onx8Ybykw2YQFA= +github.com/tmc/grpc-websocket-proxy v0.0.0-20201229170055-e5319fda7802/go.mod h1:ncp9v5uamzpCO7NfCPTXjqaC+bZgJeR0sMTm6dMHP7U= github.com/twmb/murmur3 v1.1.3 h1:D83U0XYKcHRYwYIpBKf3Pks91Z0Byda/9SJ8B6EMRcA= github.com/twmb/murmur3 v1.1.3/go.mod h1:Qq/R7NUyOfr65zD+6Q5IHKsJLwP7exErjN6lyyq3OSQ= github.com/uber-go/atomic v1.4.0 h1:yOuPqEq4ovnhEjpHmfFwsqBXDYbQeT6Nb0bwD6XnD5o= @@ -1150,12 +1190,10 @@ github.com/ugorji/go/codec v1.1.5-pre/go.mod h1:tULtS6Gy1AE1yCENaw4Vb//HLH5njI2t github.com/ugorji/go/codec v1.1.7/go.mod h1:Ax+UKWsSmolVDwsd+7N3ZtXu+yMGCf907BLYF3GoBXY= github.com/ugorji/go/codec v1.2.6 h1:7kbGefxLoDBuYXOms4yD7223OpNMMPNPZxXk5TvFcyQ= github.com/ugorji/go/codec v1.2.6/go.mod h1:V6TCNZ4PHqoHGFZuSG1W8nrCzzdgA2DozYxWFFpvxTw= -github.com/unrolled/render v1.0.1 h1:VDDnQQVfBMsOsp3VaCJszSO0nkBIVEYoPWeRThk9spY= github.com/unrolled/render v1.0.1/go.mod h1:gN9T0NhL4Bfbwu8ann7Ry/TGHYfosul+J0obPf6NBdM= github.com/urfave/cli v1.20.0/go.mod h1:70zkFmudgCuE/ngEzBv17Jvp/497gISqfk5gWijbERA= github.com/urfave/cli/v2 v2.1.1/go.mod h1:SE9GqnLQmjVa0iPEY0f1w3ygNIYcIJ0OKPMoW2caLfQ= github.com/urfave/negroni v0.3.0/go.mod h1:Meg73S6kFm/4PpbYdq35yYWoCZ9mS/YSx+lKnmiohz4= -github.com/urfave/negroni v1.0.0 h1:kIimOitoypq34K7TG7DUaJ9kq/N4Ofuwi1sjz0KipXc= github.com/urfave/negroni v1.0.0/go.mod h1:Meg73S6kFm/4PpbYdq35yYWoCZ9mS/YSx+lKnmiohz4= github.com/valyala/bytebufferpool v1.0.0 h1:GqA5TC/0021Y/b9FG4Oi9Mr3q7XYx6KllzawFIhcdPw= github.com/valyala/bytebufferpool v1.0.0/go.mod h1:6bBcMArwyJ5K/AmCkWv1jt77kVWyCJ6HpOuEn7z0Csc= @@ -1207,16 +1245,34 @@ github.com/yusufpapurcu/wmi v1.2.2/go.mod h1:SBZ9tNy3G9/m5Oi98Zks0QjeHVDvuK0qfxQ github.com/zenazn/goji v0.9.0/go.mod h1:7S9M489iMyHBNxwZnk9/EHS098H4/F6TATF2mIxtB1Q= go.etcd.io/bbolt v1.3.2/go.mod h1:IbVyRI1SCnLcuJnV2u8VeU0CEYM7e686BmAb1XKL+uU= go.etcd.io/bbolt v1.3.3/go.mod h1:IbVyRI1SCnLcuJnV2u8VeU0CEYM7e686BmAb1XKL+uU= -go.etcd.io/bbolt v1.3.5 h1:XAzx9gjCb0Rxj7EoqcClPD1d5ZBxZJk0jbuoPHenBt0= go.etcd.io/bbolt v1.3.5/go.mod h1:G5EMThwa9y8QZGBClrRx5EY+Yw9kAhnjy3bSjsnlVTQ= +go.etcd.io/bbolt v1.3.6 h1:/ecaJf0sk1l4l6V4awd65v2C3ILy7MSj+s/x1ADCIMU= +go.etcd.io/bbolt v1.3.6/go.mod h1:qXsaaIqmgQH0T+OPdb99Bf+PKfBBQVAdyD6TY9G8XM4= go.etcd.io/etcd v0.0.0-20191023171146-3cf2f69b5738/go.mod h1:dnLIgRNXwCJa5e+c6mIZCrds/GIG4ncV9HhK5PX7jPg= go.etcd.io/etcd v0.5.0-alpha.5.0.20191023171146-3cf2f69b5738/go.mod h1:dnLIgRNXwCJa5e+c6mIZCrds/GIG4ncV9HhK5PX7jPg= go.etcd.io/etcd v0.5.0-alpha.5.0.20200824191128-ae9734ed278b/go.mod h1:yVHk9ub3CSBatqGNg7GRmsnfLWtoW60w4eDYfh7vHDg= go.etcd.io/etcd v0.5.0-alpha.5.0.20210512015243-d19fbe541bf9 h1:MNsY1TIsWLNCMT4DzZjFOxbDKfSoULYP0OFjJ8dSxts= go.etcd.io/etcd v0.5.0-alpha.5.0.20210512015243-d19fbe541bf9/go.mod h1:q+i20RPAmay+xq8LJ3VMOhXCNk4YCk3V7QP91meFavw= go.etcd.io/etcd/api/v3 v3.5.0/go.mod h1:cbVKeC6lCfl7j/8jBhAK6aIYO9XOjdptoxU/nLQcPvs= +go.etcd.io/etcd/api/v3 v3.5.2 h1:tXok5yLlKyuQ/SXSjtqHc4uzNaMqZi2XsoSPr/LlJXI= +go.etcd.io/etcd/api/v3 v3.5.2/go.mod h1:5GB2vv4A4AOn3yk7MftYGHkUfGtDHnEraIjym4dYz5A= go.etcd.io/etcd/client/pkg/v3 v3.5.0/go.mod h1:IJHfcCEKxYu1Os13ZdwCwIUTUVGYTSAM3YSwc9/Ac1g= +go.etcd.io/etcd/client/pkg/v3 v3.5.2 h1:4hzqQ6hIb3blLyQ8usCU4h3NghkqcsohEQ3o3VetYxE= +go.etcd.io/etcd/client/pkg/v3 v3.5.2/go.mod h1:IJHfcCEKxYu1Os13ZdwCwIUTUVGYTSAM3YSwc9/Ac1g= go.etcd.io/etcd/client/v2 v2.305.0/go.mod h1:h9puh54ZTgAKtEbut2oe9P4L/oqKCVB6xsXlzd7alYQ= +go.etcd.io/etcd/client/v2 v2.305.2 h1:ymrVwTkefuqA/rPkSW7/B4ApijbPVefRumkY+stNfS0= +go.etcd.io/etcd/client/v2 v2.305.2/go.mod h1:2D7ZejHVMIfog1221iLSYlQRzrtECw3kz4I4VAQm3qI= +go.etcd.io/etcd/client/v3 v3.5.2 h1:WdnejrUtQC4nCxK0/dLTMqKOB+U5TP/2Ya0BJL+1otA= +go.etcd.io/etcd/client/v3 v3.5.2/go.mod h1:kOOaWFFgHygyT0WlSmL8TJiXmMysO/nNUlEsSsN6W4o= +go.etcd.io/etcd/etcdutl/v3 v3.5.2/go.mod h1:f+KEUNxRzqQGq1Y/SsaDN5cmlOGRWgfE3lXEDi5F1Ys= +go.etcd.io/etcd/pkg/v3 v3.5.2 h1:YZUojdoPhOyl5QILYnR8LTUbbNefu/sV4ma+ZMr2tto= +go.etcd.io/etcd/pkg/v3 v3.5.2/go.mod h1:zsXz+9D/kijzRiG/UnFGDTyHKcVp0orwiO8iMLAi+k0= +go.etcd.io/etcd/raft/v3 v3.5.2 h1:uCC37qOXqBvKqTGHGyhASsaCsnTuJugl1GvneJNwHWo= +go.etcd.io/etcd/raft/v3 v3.5.2/go.mod h1:G6pCP1sFgbjod7/KnEHY0vHUViqxjkdt6AiKsD0GRr8= +go.etcd.io/etcd/server/v3 v3.5.2 h1:B6ytJvS4Fmt8nkjzS2/8POf4tuPhFMluE0lWd4dx/7U= +go.etcd.io/etcd/server/v3 v3.5.2/go.mod h1:mlG8znIEz4N/28GABrohZCBM11FqgGVQcpbcyJgh0j0= +go.etcd.io/etcd/tests/v3 v3.5.2 h1:uk7/uMGVebpBDl+roivowHt6gJ5Fnqwik3syDkoSKdo= +go.etcd.io/etcd/tests/v3 v3.5.2/go.mod h1:Jdzbei4uFi9C3xDBfCwckRXjlX0UPooiP4g/zXgBMgQ= go.opencensus.io v0.21.0/go.mod h1:mSImk1erAIZhrmZN+AvHh14ztQfjbGwt4TtuofqLduU= go.opencensus.io v0.22.0/go.mod h1:+kGneAE2xo2IficOXnaByMWTGM9T73dGwxeWcUqIpI8= go.opencensus.io v0.22.2/go.mod h1:yxeiOL68Rb0Xd1ddK5vPZ/oVn4vY4Ynel7k9FzqtOIw= @@ -1225,6 +1281,27 @@ go.opencensus.io v0.22.4/go.mod h1:yxeiOL68Rb0Xd1ddK5vPZ/oVn4vY4Ynel7k9FzqtOIw= go.opencensus.io v0.22.5/go.mod h1:5pWMHQbX5EPX2/62yrJeAkowc+lfs/XD7Uxpq3pI6kk= go.opencensus.io v0.23.0 h1:gqCw0LfLxScz8irSi8exQc7fyQ0fKQU/qnC/X8+V/1M= go.opencensus.io v0.23.0/go.mod h1:XItmlyltB5F7CS4xOC1DcqMoFqwtC6OG2xF7mCv7P7E= +go.opentelemetry.io/contrib v0.20.0 h1:ubFQUn0VCZ0gPwIoJfBJVpeBlyRMxu8Mm/huKWYd9p0= +go.opentelemetry.io/contrib v0.20.0/go.mod h1:G/EtFaa6qaN7+LxqfIAT3GiZa7Wv5DTBUzl5H4LY0Kc= +go.opentelemetry.io/contrib/instrumentation/google.golang.org/grpc/otelgrpc v0.20.0 h1:sO4WKdPAudZGKPcpZT4MJn6JaDmpyLrMPDGGyA1SttE= +go.opentelemetry.io/contrib/instrumentation/google.golang.org/grpc/otelgrpc v0.20.0/go.mod h1:oVGt1LRbBOBq1A5BQLlUg9UaU/54aiHw8cgjV3aWZ/E= +go.opentelemetry.io/otel v0.20.0 h1:eaP0Fqu7SXHwvjiqDq83zImeehOHX8doTvU9AwXON8g= +go.opentelemetry.io/otel v0.20.0/go.mod h1:Y3ugLH2oa81t5QO+Lty+zXf8zC9L26ax4Nzoxm/dooo= +go.opentelemetry.io/otel/exporters/otlp v0.20.0 h1:PTNgq9MRmQqqJY0REVbZFvwkYOA85vbdQU/nVfxDyqg= +go.opentelemetry.io/otel/exporters/otlp v0.20.0/go.mod h1:YIieizyaN77rtLJra0buKiNBOm9XQfkPEKBeuhoMwAM= +go.opentelemetry.io/otel/metric v0.20.0 h1:4kzhXFP+btKm4jwxpjIqjs41A7MakRFUS86bqLHTIw8= +go.opentelemetry.io/otel/metric v0.20.0/go.mod h1:598I5tYlH1vzBjn+BTuhzTCSb/9debfNp6R3s7Pr1eU= +go.opentelemetry.io/otel/oteltest v0.20.0 h1:HiITxCawalo5vQzdHfKeZurV8x7ljcqAgiWzF6Vaeaw= +go.opentelemetry.io/otel/oteltest v0.20.0/go.mod h1:L7bgKf9ZB7qCwT9Up7i9/pn0PWIa9FqQ2IQ8LoxiGnw= +go.opentelemetry.io/otel/sdk v0.20.0 h1:JsxtGXd06J8jrnya7fdI/U/MR6yXA5DtbZy+qoHQlr8= +go.opentelemetry.io/otel/sdk v0.20.0/go.mod h1:g/IcepuwNsoiX5Byy2nNV0ySUF1em498m7hBWC279Yc= +go.opentelemetry.io/otel/sdk/export/metric v0.20.0 h1:c5VRjxCXdQlx1HjzwGdQHzZaVI82b5EbBgOu2ljD92g= +go.opentelemetry.io/otel/sdk/export/metric v0.20.0/go.mod h1:h7RBNMsDJ5pmI1zExLi+bJK+Dr8NQCh0qGhm1KDnNlE= +go.opentelemetry.io/otel/sdk/metric v0.20.0 h1:7ao1wpzHRVKf0OQ7GIxiQJA6X7DLX9o14gmVon7mMK8= +go.opentelemetry.io/otel/sdk/metric v0.20.0/go.mod h1:knxiS8Xd4E/N+ZqKmUPf3gTTZ4/0TjTXukfxjzSTpHE= +go.opentelemetry.io/otel/trace v0.20.0 h1:1DL6EXUdcg95gukhuRRvLDO/4X5THh/5dIV52lqtnbw= +go.opentelemetry.io/otel/trace v0.20.0/go.mod h1:6GjCW8zgDjwGHGa6GkyeB8+/5vjT16gUEi0Nf1iBdgw= +go.opentelemetry.io/proto/otlp v0.7.0 h1:rwOQPCuKAKmwGKq2aVNnYIibI6wnV7EvzgfTCzcdGg8= go.opentelemetry.io/proto/otlp v0.7.0/go.mod h1:PqfVotwruBrMGOCsRd/89rSnXhoiJIqeYNgFYFoEGnI= go.uber.org/atomic v1.3.2/go.mod h1:gD2HeocX3+yG+ygLZcrzQJaqmWj9AIm7n08wl/qW/PE= go.uber.org/atomic v1.4.0/go.mod h1:gD2HeocX3+yG+ygLZcrzQJaqmWj9AIm7n08wl/qW/PE= @@ -1241,6 +1318,7 @@ go.uber.org/fx v1.12.0/go.mod h1:egT3Kyg1JFYQkvKLZ3EsykxkNrZxgXS+gKoKo7abERY= go.uber.org/goleak v0.10.0/go.mod h1:VCZuO8V8mFPlL0F5J5GK1rtHV3DrFcQ1R8ryq7FK0aI= go.uber.org/goleak v1.1.10/go.mod h1:8a7PlsEVH3e/a/GLqe5IIrQx6GzcnRmZEufDUTk4A7A= go.uber.org/goleak v1.1.11-0.20210813005559-691160354723/go.mod h1:cwTWslyiVhfpKIDGSZEM2HlOvcqm+tG4zioyIeLoqMQ= +go.uber.org/goleak v1.1.11/go.mod h1:cwTWslyiVhfpKIDGSZEM2HlOvcqm+tG4zioyIeLoqMQ= go.uber.org/goleak v1.1.12 h1:gZAh5/EyT/HQwlpkCy6wTpqfH9H8Lz8zbm3dZh+OyzA= go.uber.org/goleak v1.1.12/go.mod h1:cwTWslyiVhfpKIDGSZEM2HlOvcqm+tG4zioyIeLoqMQ= go.uber.org/multierr v1.1.0/go.mod h1:wR5kodmAFQ0UK8QlbwjlSNy0Z68gJhDJUG5sjR94q/0= @@ -1260,8 +1338,9 @@ go.uber.org/zap v1.16.0/go.mod h1:MA8QOfq0BHJwdXa996Y4dYkAqRKB8/1K1QMMZVaNZjQ= go.uber.org/zap v1.17.0/go.mod h1:MXVU+bhUf/A7Xi2HNOnopQOrmycQ5Ih87HtOu4q5SSo= go.uber.org/zap v1.18.1/go.mod h1:xg/QME4nWcxGxrpdeYfq7UvYrLh66cuVKdrbD1XF/NI= go.uber.org/zap v1.19.0/go.mod h1:xg/QME4nWcxGxrpdeYfq7UvYrLh66cuVKdrbD1XF/NI= -go.uber.org/zap v1.19.1 h1:ue41HOKd1vGURxrmeKIgELGb3jPW9DMUDGtsinblHwI= go.uber.org/zap v1.19.1/go.mod h1:j3DNczoxDZroyBnOT1L/Q79cfUMGZxlv/9dzN7SM1rI= +go.uber.org/zap v1.20.0 h1:N4oPlghZwYG55MlU6LXk/Zp00FVNE9X9wrYO8CEs4lc= +go.uber.org/zap v1.20.0/go.mod h1:wjWOCqI0f2ZZrJF/UufIOkiC8ii6tm1iqIsLo76RfJw= golang.org/x/crypto v0.0.0-20180723164146-c126467f60eb/go.mod h1:6SG95UA2DQfeDnfUPMdvaQW0Q7yPrPDi9nlGo2tz2b4= golang.org/x/crypto v0.0.0-20180904163835-0709b304e793/go.mod h1:6SG95UA2DQfeDnfUPMdvaQW0Q7yPrPDi9nlGo2tz2b4= golang.org/x/crypto v0.0.0-20181029021203-45a5f77698d3/go.mod h1:6SG95UA2DQfeDnfUPMdvaQW0Q7yPrPDi9nlGo2tz2b4= @@ -1281,6 +1360,7 @@ golang.org/x/crypto v0.0.0-20200204104054-c9f3fb736b72/go.mod h1:LzIPMQfyMNhhGPh golang.org/x/crypto v0.0.0-20200323165209-0ec3e9974c59/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto= golang.org/x/crypto v0.0.0-20200622213623-75b288015ac9/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto= golang.org/x/crypto v0.0.0-20200820211705-5c72a883971a/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto= +golang.org/x/crypto v0.0.0-20201002170205-7f63de1d35b0/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto= golang.org/x/crypto v0.0.0-20201016220609-9e8e0b390897/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto= golang.org/x/crypto v0.0.0-20201217014255-9d1352758620/go.mod h1:jdWPYTVW3xRLrWPugEBEK3UY2ZEsg3UU495nc5E+M+I= golang.org/x/crypto v0.0.0-20210711020723-a769d52b0f97/go.mod h1:GvvjBRRGRdwPK5ydBHafDWAxML/pGHZbMvKqRZ5+Abc= @@ -1387,6 +1467,7 @@ golang.org/x/net v0.0.0-20210119194325-5f4716e94777/go.mod h1:m0MpNAwzfU5UDzcl9v golang.org/x/net v0.0.0-20210226172049-e18ecbb05110/go.mod h1:m0MpNAwzfU5UDzcl9v0D8zg8gWTRqZa9RBIspLL5mdg= golang.org/x/net v0.0.0-20210316092652-d523dce5a7f4/go.mod h1:RBQZq4jEuRlivfhVLdyRGr576XBO4/greRjx4P4O3yc= golang.org/x/net v0.0.0-20210405180319-a5a99cb37ef4/go.mod h1:p54w0d4576C0XHj96bSt6lcn1PtDYWL6XObtHCRCNQM= +golang.org/x/net v0.0.0-20210428140749-89ef3d95e781/go.mod h1:OJAsFXCWl8Ukc7SiCT/9KSuxbyM7479/AVlXFRxuMCk= golang.org/x/net v0.0.0-20210503060351-7fd8e65b6420/go.mod h1:9nx3DQGgdP8bBQD5qxJ1jj9UTztislL4KSBs9R2vV5Y= golang.org/x/net v0.0.0-20210610132358-84b48f89b13b/go.mod h1:9nx3DQGgdP8bBQD5qxJ1jj9UTztislL4KSBs9R2vV5Y= golang.org/x/net v0.0.0-20210805182204-aaa1db679c0d/go.mod h1:9nx3DQGgdP8bBQD5qxJ1jj9UTztislL4KSBs9R2vV5Y= @@ -1475,16 +1556,20 @@ golang.org/x/sys v0.0.0-20200515095857-1151b9dac4a9/go.mod h1:h1NjWce9XRLGQEsW7w golang.org/x/sys v0.0.0-20200519105757-fe76b779f299/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20200523222454-059865788121/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20200615200032-f1bc736245b1/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20200625212154-ddb9806d33ae/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20200803210538-64077c9b5642/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20200814200057-3d37ad5750ed/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20200826173525-f9321e4c35a6/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20200905004654-be1d3432aa8f/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20200923182605-d9f96fdee20d/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20200930185726-fdedc70b468f/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20201119102817-f84b799fce68/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20201201145000-ef89a241ccb3/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20201204225414-ed752295db88/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20210104204734-6f8348627aad/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20210112080510-489259a85091/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20210119212857-b64e53b001e4/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20210124154548-22da62e12c0c/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20210217105451-b926d437f341/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20210220050731-9a76102bfb43/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20210305230114-8fe3ee5dd75b/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= @@ -1495,6 +1580,7 @@ golang.org/x/sys v0.0.0-20210403161142-5e06dd20ab57/go.mod h1:h1NjWce9XRLGQEsW7w golang.org/x/sys v0.0.0-20210423082822-04245dca01da/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20210510120138-977fb7262007/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20210514084401-e8d321eab015/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= +golang.org/x/sys v0.0.0-20210603081109-ebe580a85c40/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20210603125802-9665404d3644/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20210615035016-665e8c7367d1/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20210616094352-59db8d763f22/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= @@ -1599,6 +1685,7 @@ golang.org/x/tools v0.0.0-20201110124207-079ba7bd75cd/go.mod h1:emZCQorbCU4vsT4f golang.org/x/tools v0.0.0-20201125231158-b5590deeca9b/go.mod h1:emZCQorbCU4vsT4fOWvOPXz4eW1wZW4PmDk9uLelYpA= golang.org/x/tools v0.0.0-20201201161351-ac6f37ff4c2a/go.mod h1:emZCQorbCU4vsT4fOWvOPXz4eW1wZW4PmDk9uLelYpA= golang.org/x/tools v0.0.0-20201208233053-a543418bbed2/go.mod h1:emZCQorbCU4vsT4fOWvOPXz4eW1wZW4PmDk9uLelYpA= +golang.org/x/tools v0.0.0-20201224043029-2b0845dc783e/go.mod h1:emZCQorbCU4vsT4fOWvOPXz4eW1wZW4PmDk9uLelYpA= golang.org/x/tools v0.0.0-20210105154028-b0ab187a4818/go.mod h1:emZCQorbCU4vsT4fOWvOPXz4eW1wZW4PmDk9uLelYpA= golang.org/x/tools v0.0.0-20210106214847-113979e3529a/go.mod h1:emZCQorbCU4vsT4fOWvOPXz4eW1wZW4PmDk9uLelYpA= golang.org/x/tools v0.0.0-20210112230658-8b4aab62c064/go.mod h1:emZCQorbCU4vsT4fOWvOPXz4eW1wZW4PmDk9uLelYpA= @@ -1619,9 +1706,7 @@ golang.org/x/xerrors v0.0.0-20191204190536-9bdfabe68543/go.mod h1:I/5z698sn9Ka8T golang.org/x/xerrors v0.0.0-20200804184101-5ec99f83aff1 h1:go1bK/D/BFZV2I8cIQd1NKEZ+0owSTG1fDTci4IqFcE= golang.org/x/xerrors v0.0.0-20200804184101-5ec99f83aff1/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= gonum.org/v1/gonum v0.0.0-20180816165407-929014505bf4/go.mod h1:Y+Yx5eoAFn32cQvJDxZx5Dpnq+c3wtXuadVZAcxbbBo= -gonum.org/v1/gonum v0.0.0-20181121035319-3f7ecaa7e8ca/go.mod h1:Y+Yx5eoAFn32cQvJDxZx5Dpnq+c3wtXuadVZAcxbbBo= gonum.org/v1/gonum v0.8.2/go.mod h1:oe/vMfY3deqTw+1EZJhuvEW2iwGF1bW9wwu7XCu0+v0= -gonum.org/v1/netlib v0.0.0-20181029234149-ec6d1f5cefe6/go.mod h1:wa6Ws7BG/ESfp6dHfk7C6KdzKA7wR7u/rKwOGE66zvw= gonum.org/v1/netlib v0.0.0-20190313105609-8cb42192e0e0/go.mod h1:wa6Ws7BG/ESfp6dHfk7C6KdzKA7wR7u/rKwOGE66zvw= gonum.org/v1/plot v0.0.0-20190515093506-e2840ee46a6b/go.mod h1:Wt8AAjI+ypCyYX3nZBvf6cAIx93T+c/OS2HFAYskSZc= google.golang.org/api v0.4.0/go.mod h1:8k5glujaEP+g9n7WNsDg8QP6cUVNI86fCNMcbazEtwE= @@ -1685,6 +1770,7 @@ google.golang.org/genproto v0.0.0-20200228133532-8c2c7df3a383/go.mod h1:55QSHmfG google.golang.org/genproto v0.0.0-20200305110556-506484158171/go.mod h1:55QSHmfGQM9UVYDPBsyGGes0y52j32PQ3BqQfXhyH3c= google.golang.org/genproto v0.0.0-20200312145019-da6875a35672/go.mod h1:55QSHmfGQM9UVYDPBsyGGes0y52j32PQ3BqQfXhyH3c= google.golang.org/genproto v0.0.0-20200331122359-1ee6d9798940/go.mod h1:55QSHmfGQM9UVYDPBsyGGes0y52j32PQ3BqQfXhyH3c= +google.golang.org/genproto v0.0.0-20200423170343-7949de9c1215/go.mod h1:55QSHmfGQM9UVYDPBsyGGes0y52j32PQ3BqQfXhyH3c= google.golang.org/genproto v0.0.0-20200430143042-b979b6f78d84/go.mod h1:55QSHmfGQM9UVYDPBsyGGes0y52j32PQ3BqQfXhyH3c= google.golang.org/genproto v0.0.0-20200511104702-f5ebc3bea380/go.mod h1:55QSHmfGQM9UVYDPBsyGGes0y52j32PQ3BqQfXhyH3c= google.golang.org/genproto v0.0.0-20200513103714-09dca8ec2884/go.mod h1:55QSHmfGQM9UVYDPBsyGGes0y52j32PQ3BqQfXhyH3c= @@ -1732,7 +1818,6 @@ google.golang.org/grpc v1.26.0/go.mod h1:qbnxyOmOxrQa7FizSgH+ReBfzJrCY1pSN7KXBS8 google.golang.org/grpc v1.27.0/go.mod h1:qbnxyOmOxrQa7FizSgH+ReBfzJrCY1pSN7KXBS8abTk= google.golang.org/grpc v1.27.1/go.mod h1:qbnxyOmOxrQa7FizSgH+ReBfzJrCY1pSN7KXBS8abTk= google.golang.org/grpc v1.28.0/go.mod h1:rpkK4SK4GF4Ach/+MFLZUBavHOvF2JJB5uozKKal+60= -google.golang.org/grpc v1.29.1 h1:EC2SB8S04d2r73uptxphDSUG+kTKVgjRPF+N3xpxRB4= google.golang.org/grpc v1.29.1/go.mod h1:itym6AZVZYACWQqET3MqgPpjcuV5QH3BxFS3IjizoKk= google.golang.org/grpc v1.30.0/go.mod h1:N36X2cJ7JwdamYAgDz+s+rVMFjt3numwzf/HckM8pak= google.golang.org/grpc v1.31.0/go.mod h1:N36X2cJ7JwdamYAgDz+s+rVMFjt3numwzf/HckM8pak= @@ -1748,6 +1833,9 @@ google.golang.org/grpc v1.37.1/go.mod h1:NREThFqKR1f3iQ6oBuvc5LadQuXVGo9rkm5ZGrQ google.golang.org/grpc v1.38.0/go.mod h1:NREThFqKR1f3iQ6oBuvc5LadQuXVGo9rkm5ZGrQdJfM= google.golang.org/grpc v1.39.0/go.mod h1:PImNr+rS9TWYb2O4/emRugxiyHZ5JyHW5F+RPnDzfrE= google.golang.org/grpc v1.39.1/go.mod h1:PImNr+rS9TWYb2O4/emRugxiyHZ5JyHW5F+RPnDzfrE= +google.golang.org/grpc v1.40.0/go.mod h1:ogyxbiOoUXAkP+4+xa6PZSE9DZgIHtSpzjDTB9KAK34= +google.golang.org/grpc v1.43.0 h1:Eeu7bZtDZ2DpRCsLhUlcrLnvYaMK1Gz86a+hMVvELmM= +google.golang.org/grpc v1.43.0/go.mod h1:k+4IHHFw41K8+bbowsex27ge2rCb65oeWqe4jJ590SU= google.golang.org/grpc/cmd/protoc-gen-go-grpc v1.1.0/go.mod h1:6Kw0yEErY5E/yWrBtf03jp27GLLJujG4z/JK95pnjjw= google.golang.org/protobuf v0.0.0-20200109180630-ec00e32a8dfd/go.mod h1:DFci5gLYBciE7Vtevhsrf46CRTquxDuWsQurQQe4oz8= google.golang.org/protobuf v0.0.0-20200221191635-4d8936d0db64/go.mod h1:kwYJMbMJ01Woi6D6+Kah6886xMZcty6N08ah7+eCXa0= @@ -1780,6 +1868,7 @@ gopkg.in/go-playground/assert.v1 v1.2.1/go.mod h1:9RXL0bg/zibRAgZUYszZSwO/z8Y/a8 gopkg.in/go-playground/validator.v8 v8.18.2/go.mod h1:RX2a/7Ha8BgOhfk7j780h4/u/RRjR0eouCJSH80/M2Y= gopkg.in/go-playground/validator.v9 v9.29.1/go.mod h1:+c9/zcJMFNgbLvly1L1V+PpxWdVbfP1avr/N00E2vyQ= gopkg.in/inconshreveable/log15.v2 v2.0.0-20180818164646-67afb5ed74ec/go.mod h1:aPpfJ7XW+gOuirDoZ8gHhLh3kZ1B08FtV2bbmy7Jv3s= +gopkg.in/ini.v1 v1.51.0/go.mod h1:pNLf8WUiyNEtQjuu5G5vTm06TEv9tsIgeAvK8hOrP4k= gopkg.in/ini.v1 v1.62.0/go.mod h1:pNLf8WUiyNEtQjuu5G5vTm06TEv9tsIgeAvK8hOrP4k= gopkg.in/jcmturner/aescts.v1 v1.0.1 h1:cVVZBK2b1zY26haWB4vbBiZrfFQnfbTVrE3xZq6hrEw= gopkg.in/jcmturner/aescts.v1 v1.0.1/go.mod h1:nsR8qBOg+OucoIW+WMhB3GspUQXq9XorLnQb9XtvcOo= diff --git a/pkg/cmd/cli/cli_capture_list.go b/pkg/cmd/cli/cli_capture_list.go index 9f122b5b290..1f62a4491e1 100644 --- a/pkg/cmd/cli/cli_capture_list.go +++ b/pkg/cmd/cli/cli_capture_list.go @@ -17,13 +17,14 @@ import ( "context" "github.com/pingcap/errors" + "github.com/spf13/cobra" + "go.etcd.io/etcd/client/v3/concurrency" + cmdcontext "github.com/pingcap/tiflow/pkg/cmd/context" "github.com/pingcap/tiflow/pkg/cmd/factory" "github.com/pingcap/tiflow/pkg/cmd/util" cerror "github.com/pingcap/tiflow/pkg/errors" "github.com/pingcap/tiflow/pkg/etcd" - "github.com/spf13/cobra" - "go.etcd.io/etcd/clientv3/concurrency" ) // capture holds capture information. diff --git a/pkg/cmd/factory/factory_impl.go b/pkg/cmd/factory/factory_impl.go index b35274ffcd5..9761cd9b38a 100644 --- a/pkg/cmd/factory/factory_impl.go +++ b/pkg/cmd/factory/factory_impl.go @@ -20,17 +20,18 @@ import ( "github.com/pingcap/errors" tidbkv "github.com/pingcap/tidb/kv" + pd "github.com/tikv/pd/client" + etcdlogutil "go.etcd.io/etcd/client/pkg/v3/logutil" + clientv3 "go.etcd.io/etcd/client/v3" + "go.uber.org/zap" + "google.golang.org/grpc" + "google.golang.org/grpc/backoff" + "github.com/pingcap/tiflow/cdc/kv" cmdconetxt "github.com/pingcap/tiflow/pkg/cmd/context" "github.com/pingcap/tiflow/pkg/etcd" "github.com/pingcap/tiflow/pkg/security" "github.com/pingcap/tiflow/pkg/version" - pd "github.com/tikv/pd/client" - "go.etcd.io/etcd/clientv3" - etcdlogutil "go.etcd.io/etcd/pkg/logutil" - "go.uber.org/zap" - "google.golang.org/grpc" - "google.golang.org/grpc/backoff" ) type factoryImpl struct { diff --git a/pkg/errorutil/ignore.go b/pkg/errorutil/ignore.go index 3de2b82d311..81c6faf3352 100644 --- a/pkg/errorutil/ignore.go +++ b/pkg/errorutil/ignore.go @@ -19,7 +19,7 @@ import ( tddl "github.com/pingcap/tidb/ddl" "github.com/pingcap/tidb/infoschema" "github.com/pingcap/tidb/parser/mysql" - v3rpc "go.etcd.io/etcd/etcdserver/api/v3rpc/rpctypes" + v3rpc "go.etcd.io/etcd/api/v3/v3rpc/rpctypes" ) // IsIgnorableMySQLDDLError is used to check what error can be ignored diff --git a/pkg/errorutil/ignore_test.go b/pkg/errorutil/ignore_test.go index 825bf7d91b6..569c4270bd3 100644 --- a/pkg/errorutil/ignore_test.go +++ b/pkg/errorutil/ignore_test.go @@ -21,7 +21,7 @@ import ( "github.com/pingcap/tidb/infoschema" tmysql "github.com/pingcap/tidb/parser/mysql" "github.com/stretchr/testify/require" - v3rpc "go.etcd.io/etcd/etcdserver/api/v3rpc/rpctypes" + v3rpc "go.etcd.io/etcd/api/v3/v3rpc/rpctypes" ) func newMysqlErr(number uint16, message string) *mysql.MySQLError { diff --git a/pkg/etcd/client.go b/pkg/etcd/client.go index b8cb028216b..42932ec81d0 100644 --- a/pkg/etcd/client.go +++ b/pkg/etcd/client.go @@ -20,14 +20,15 @@ import ( "github.com/benbjohnson/clock" "github.com/pingcap/errors" "github.com/pingcap/log" - cerrors "github.com/pingcap/tiflow/pkg/errors" - "github.com/pingcap/tiflow/pkg/errorutil" - "github.com/pingcap/tiflow/pkg/retry" "github.com/prometheus/client_golang/prometheus" - "go.etcd.io/etcd/clientv3" - v3rpc "go.etcd.io/etcd/etcdserver/api/v3rpc/rpctypes" + v3rpc "go.etcd.io/etcd/api/v3/v3rpc/rpctypes" + clientv3 "go.etcd.io/etcd/client/v3" "go.uber.org/zap" "google.golang.org/grpc/codes" + + cerrors "github.com/pingcap/tiflow/pkg/errors" + "github.com/pingcap/tiflow/pkg/errorutil" + "github.com/pingcap/tiflow/pkg/retry" ) // etcd operation names diff --git a/pkg/etcd/client_test.go b/pkg/etcd/client_test.go index b8fe6bcbbee..faf155a1445 100644 --- a/pkg/etcd/client_test.go +++ b/pkg/etcd/client_test.go @@ -24,8 +24,8 @@ import ( "github.com/benbjohnson/clock" "github.com/pingcap/errors" "github.com/stretchr/testify/require" - "go.etcd.io/etcd/clientv3" - "go.etcd.io/etcd/etcdserver/api/v3rpc/rpctypes" + "go.etcd.io/etcd/api/v3/v3rpc/rpctypes" + clientv3 "go.etcd.io/etcd/client/v3" ) type mockClient struct { diff --git a/pkg/etcd/etcd.go b/pkg/etcd/etcd.go index 5c6a4209645..907595eaa7b 100644 --- a/pkg/etcd/etcd.go +++ b/pkg/etcd/etcd.go @@ -19,20 +19,20 @@ import ( "net/url" "time" + "github.com/pingcap/errors" "github.com/pingcap/log" - "github.com/pingcap/tiflow/cdc/model" - cerror "github.com/pingcap/tiflow/pkg/errors" "github.com/prometheus/client_golang/prometheus" - "go.etcd.io/etcd/clientv3" - "go.etcd.io/etcd/clientv3/concurrency" - "go.etcd.io/etcd/etcdserver/api/v3rpc/rpctypes" - "go.etcd.io/etcd/mvcc/mvccpb" + "github.com/tikv/pd/pkg/tempurl" + "go.etcd.io/etcd/api/v3/mvccpb" + "go.etcd.io/etcd/api/v3/v3rpc/rpctypes" + clientv3 "go.etcd.io/etcd/client/v3" + "go.etcd.io/etcd/client/v3/concurrency" + "go.etcd.io/etcd/server/v3/embed" "go.uber.org/zap" "google.golang.org/grpc/codes" - "github.com/pingcap/errors" - "github.com/tikv/pd/pkg/tempurl" - "go.etcd.io/etcd/embed" + "github.com/pingcap/tiflow/cdc/model" + cerror "github.com/pingcap/tiflow/pkg/errors" ) const ( diff --git a/pkg/etcd/etcd_test.go b/pkg/etcd/etcd_test.go index 36b604d44cf..bd0399189f3 100644 --- a/pkg/etcd/etcd_test.go +++ b/pkg/etcd/etcd_test.go @@ -25,17 +25,18 @@ import ( "testing" "time" - "github.com/pingcap/tiflow/cdc/model" - cerror "github.com/pingcap/tiflow/pkg/errors" - "github.com/pingcap/tiflow/pkg/util" "github.com/stretchr/testify/require" - "go.etcd.io/etcd/clientv3" - "go.etcd.io/etcd/clientv3/concurrency" - "go.etcd.io/etcd/embed" - "go.etcd.io/etcd/pkg/logutil" + "go.etcd.io/etcd/client/pkg/v3/logutil" + clientv3 "go.etcd.io/etcd/client/v3" + "go.etcd.io/etcd/client/v3/concurrency" + "go.etcd.io/etcd/server/v3/embed" "go.uber.org/zap" "go.uber.org/zap/zapcore" "golang.org/x/sync/errgroup" + + "github.com/pingcap/tiflow/cdc/model" + cerror "github.com/pingcap/tiflow/pkg/errors" + "github.com/pingcap/tiflow/pkg/util" ) type Captures []*model.CaptureInfo @@ -79,7 +80,10 @@ func (s *etcdTester) tearDownTest(t *testing.T) { logEtcdError: for { select { - case err := <-s.etcd.Err(): + case err, ok := <-s.etcd.Err(): + if !ok { + break logEtcdError + } t.Logf("etcd server error: %v", err) default: break logEtcdError diff --git a/pkg/etcd/util.go b/pkg/etcd/util.go index e70ec080edf..3707089cb65 100644 --- a/pkg/etcd/util.go +++ b/pkg/etcd/util.go @@ -13,7 +13,7 @@ package etcd -import "go.etcd.io/etcd/clientv3" +import clientv3 "go.etcd.io/etcd/client/v3" func getRevisionFromWatchOpts(opts ...clientv3.OpOption) int64 { op := &clientv3.Op{} diff --git a/pkg/etcd/util_test.go b/pkg/etcd/util_test.go index 210202c4f32..39ee4b5f91d 100644 --- a/pkg/etcd/util_test.go +++ b/pkg/etcd/util_test.go @@ -18,7 +18,7 @@ import ( "testing" "github.com/stretchr/testify/require" - "go.etcd.io/etcd/clientv3" + clientv3 "go.etcd.io/etcd/client/v3" ) func TestGetRevisionFromWatchOpts(t *testing.T) { diff --git a/pkg/leakutil/leak_helper.go b/pkg/leakutil/leak_helper.go index accfde33667..622565554e0 100644 --- a/pkg/leakutil/leak_helper.go +++ b/pkg/leakutil/leak_helper.go @@ -23,7 +23,7 @@ import ( // options can be used to implement other ignore items func SetUpLeakTest(m *testing.M, options ...goleak.Option) { opts := []goleak.Option{ - goleak.IgnoreTopFunction("go.etcd.io/etcd/pkg/logutil.(*MergeLogger).outputLoop"), + goleak.IgnoreTopFunction("go.etcd.io/etcd/client/pkg/v3/logutil.(*MergeLogger).outputLoop"), goleak.IgnoreTopFunction("go.opencensus.io/stats/view.(*worker).start"), } diff --git a/pkg/orchestrator/etcd_worker.go b/pkg/orchestrator/etcd_worker.go index 8c60a853a00..d7b23a3c760 100644 --- a/pkg/orchestrator/etcd_worker.go +++ b/pkg/orchestrator/etcd_worker.go @@ -22,18 +22,19 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/failpoint" "github.com/pingcap/log" - cerrors "github.com/pingcap/tiflow/pkg/errors" - "github.com/pingcap/tiflow/pkg/etcd" - "github.com/pingcap/tiflow/pkg/orchestrator/util" "github.com/prometheus/client_golang/prometheus" - "go.etcd.io/etcd/clientv3" - "go.etcd.io/etcd/clientv3/concurrency" - "go.etcd.io/etcd/etcdserver/api/v3rpc/rpctypes" - "go.etcd.io/etcd/etcdserver/etcdserverpb" - "go.etcd.io/etcd/mvcc/mvccpb" + "go.etcd.io/etcd/api/v3/etcdserverpb" + "go.etcd.io/etcd/api/v3/mvccpb" + "go.etcd.io/etcd/api/v3/v3rpc/rpctypes" + clientv3 "go.etcd.io/etcd/client/v3" + "go.etcd.io/etcd/client/v3/concurrency" "go.uber.org/zap" "go.uber.org/zap/zapcore" "golang.org/x/time/rate" + + cerrors "github.com/pingcap/tiflow/pkg/errors" + "github.com/pingcap/tiflow/pkg/etcd" + "github.com/pingcap/tiflow/pkg/orchestrator/util" ) const ( diff --git a/pkg/orchestrator/etcd_worker_test.go b/pkg/orchestrator/etcd_worker_test.go index a46eb83cab9..be5c6d3da0c 100644 --- a/pkg/orchestrator/etcd_worker_test.go +++ b/pkg/orchestrator/etcd_worker_test.go @@ -27,15 +27,16 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/log" - cerrors "github.com/pingcap/tiflow/pkg/errors" - "github.com/pingcap/tiflow/pkg/etcd" - "github.com/pingcap/tiflow/pkg/orchestrator/util" "github.com/prometheus/client_golang/prometheus" "github.com/stretchr/testify/require" - "go.etcd.io/etcd/clientv3" - "go.etcd.io/etcd/etcdserver/api/v3rpc/rpctypes" + "go.etcd.io/etcd/api/v3/v3rpc/rpctypes" + clientv3 "go.etcd.io/etcd/client/v3" "go.uber.org/zap" "golang.org/x/sync/errgroup" + + cerrors "github.com/pingcap/tiflow/pkg/errors" + "github.com/pingcap/tiflow/pkg/etcd" + "github.com/pingcap/tiflow/pkg/orchestrator/util" ) const ( diff --git a/pkg/util/testleak/leaktest.go b/pkg/util/testleak/leaktest.go index 9bbb7a6122c..5b7ad55bae3 100644 --- a/pkg/util/testleak/leaktest.go +++ b/pkg/util/testleak/leaktest.go @@ -48,7 +48,7 @@ func interestingGoroutines() (gs []string) { // false positive leak failures "google.golang.org/grpc.(*addrConn).resetTransport", "google.golang.org/grpc.(*ccBalancerWrapper).watcher", - "go.etcd.io/etcd/pkg/logutil.(*MergeLogger).outputLoop", + "go.etcd.io/etcd/client/pkg/v3/logutil.(*MergeLogger).outputLoop", "go.etcd.io/etcd/v3/pkg/logutil.(*MergeLogger).outputLoop", // library used by sarama, ref: https://github.com/rcrowley/go-metrics/pull/266 "github.com/rcrowley/go-metrics.(*meterArbiter).tick", diff --git a/tests/integration_tests/move_table/main.go b/tests/integration_tests/move_table/main.go index 0dfcb569491..102d166b4fd 100644 --- a/tests/integration_tests/move_table/main.go +++ b/tests/integration_tests/move_table/main.go @@ -29,18 +29,19 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/log" + "go.etcd.io/etcd/client/pkg/v3/logutil" + clientv3 "go.etcd.io/etcd/client/v3" + "go.uber.org/zap" + "go.uber.org/zap/zapcore" + "google.golang.org/grpc" + "google.golang.org/grpc/backoff" + "github.com/pingcap/tiflow/cdc/model" cerrors "github.com/pingcap/tiflow/pkg/errors" "github.com/pingcap/tiflow/pkg/etcd" "github.com/pingcap/tiflow/pkg/httputil" "github.com/pingcap/tiflow/pkg/retry" "github.com/pingcap/tiflow/pkg/security" - "go.etcd.io/etcd/clientv3" - "go.etcd.io/etcd/pkg/logutil" - "go.uber.org/zap" - "go.uber.org/zap/zapcore" - "google.golang.org/grpc" - "google.golang.org/grpc/backoff" ) var ( diff --git a/tests/utils/cdc_state_checker/cdc_monitor.go b/tests/utils/cdc_state_checker/cdc_monitor.go index b5bc997860e..0c3480c9498 100644 --- a/tests/utils/cdc_state_checker/cdc_monitor.go +++ b/tests/utils/cdc_state_checker/cdc_monitor.go @@ -19,16 +19,17 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/log" - "github.com/pingcap/tiflow/pkg/etcd" - "github.com/pingcap/tiflow/pkg/orchestrator" - "github.com/pingcap/tiflow/pkg/security" "github.com/prometheus/client_golang/prometheus" - "go.etcd.io/etcd/clientv3" - "go.etcd.io/etcd/pkg/logutil" + "go.etcd.io/etcd/client/pkg/v3/logutil" + clientv3 "go.etcd.io/etcd/client/v3" "go.uber.org/zap" "go.uber.org/zap/zapcore" "google.golang.org/grpc" "google.golang.org/grpc/backoff" + + "github.com/pingcap/tiflow/pkg/etcd" + "github.com/pingcap/tiflow/pkg/orchestrator" + "github.com/pingcap/tiflow/pkg/security" ) type cdcMonitor struct { From 7db03434308d2f6b289a20672e54bfaf8a5b3f3d Mon Sep 17 00:00:00 2001 From: Ling Jin <7138436+3AceShowHand@users.noreply.github.com> Date: Mon, 7 Mar 2022 17:25:49 +0800 Subject: [PATCH 5/7] cdc/test: integration test add default null for unsigned type (#4751) close pingcap/tiflow#4787 --- .../integration_tests/canal_json_basic/data/data.sql | 3 +++ .../integration_tests/clustered_index/data/test.sql | 1 - tests/integration_tests/common_1/data/test.sql | 3 +++ tests/integration_tests/common_1/data/test_v5.sql | 4 +++- .../ddl_attributes/data/prepare.sql | 2 ++ .../integration_tests/ddl_sequence/data/prepare.sql | 12 +++++++++++- .../data/test1.sql | 7 +++++++ tests/integration_tests/row_format/data/step1.sql | 3 +++ tests/integration_tests/tiflash/data/prepare.sql | 8 +++++++- 9 files changed, 39 insertions(+), 4 deletions(-) diff --git a/tests/integration_tests/canal_json_basic/data/data.sql b/tests/integration_tests/canal_json_basic/data/data.sql index 84c18a78b73..361ca7ad317 100644 --- a/tests/integration_tests/canal_json_basic/data/data.sql +++ b/tests/integration_tests/canal_json_basic/data/data.sql @@ -43,6 +43,9 @@ CREATE TABLE multi_data_type PRIMARY KEY (id) ); +-- make sure `nullable` can be handled by the mounter and mq encoding protocol +INSERT INTO multi_data_type() VALUES (); + INSERT INTO multi_data_type( t_tinyint, t_tinyint_unsigned, t_smallint, t_smallint_unsigned, t_mediumint , t_mediumint_unsigned, t_int, t_int_unsigned, t_bigint, t_bigint_unsigned , t_boolean, t_float, t_double, t_decimal diff --git a/tests/integration_tests/clustered_index/data/test.sql b/tests/integration_tests/clustered_index/data/test.sql index adeb694e79a..02d913b33ee 100644 --- a/tests/integration_tests/clustered_index/data/test.sql +++ b/tests/integration_tests/clustered_index/data/test.sql @@ -176,7 +176,6 @@ insert into t11 values(1, 1.1, 1),(2, 2.2, 2),(3, 3.3, 3); update t11 set b = 4.4 where c = 3; delete from t11 where b = 2; - create table t12(name char(255) primary key, b int, c int, index idx(name), unique index uidx(name)); insert into t12 values("aaaa", 1, 1), ("bbb", 2, 2), ("ccc", 3, 3); update t12 set name = 'ddd' where c = 3; diff --git a/tests/integration_tests/common_1/data/test.sql b/tests/integration_tests/common_1/data/test.sql index 5a3c55c3df9..a38cd5c182b 100644 --- a/tests/integration_tests/common_1/data/test.sql +++ b/tests/integration_tests/common_1/data/test.sql @@ -29,6 +29,9 @@ CREATE TABLE cdc_multi_data_type DEFAULT CHARSET = utf8 COLLATE = utf8_bin; +-- make sure `nullable` can be handled by the mounter and mq encoding protocol +INSERT INTO cdc_multi_data_type() VALUES (); + INSERT INTO cdc_multi_data_type( t_boolean, t_bigint, t_double, t_decimal, t_bit , t_date, t_datetime, t_timestamp, t_time, t_year , t_char, t_varchar, t_blob, t_text, t_enum diff --git a/tests/integration_tests/common_1/data/test_v5.sql b/tests/integration_tests/common_1/data/test_v5.sql index 529561f012a..caa682bc74c 100644 --- a/tests/integration_tests/common_1/data/test_v5.sql +++ b/tests/integration_tests/common_1/data/test_v5.sql @@ -17,9 +17,11 @@ alter table `add_and_drop_columns` insert into `add_and_drop_columns` (id, col1, col2, col3) values (2, 3, 4, 5); +insert into `add_and_drop_columns` (id) values (3); + alter table `add_and_drop_columns` drop col1, drop col2; insert into `add_and_drop_columns` (id, col3) -values (3, 4); +values (4, 5); diff --git a/tests/integration_tests/ddl_attributes/data/prepare.sql b/tests/integration_tests/ddl_attributes/data/prepare.sql index 9fb8931b8ce..a229e38e78b 100644 --- a/tests/integration_tests/ddl_attributes/data/prepare.sql +++ b/tests/integration_tests/ddl_attributes/data/prepare.sql @@ -5,11 +5,13 @@ USE `ddl_attributes`; CREATE TABLE attributes_t1 (id INT PRIMARY KEY, name VARCHAR(50)); ALTER TABLE attributes_t1 ATTRIBUTES='merge_option=deny'; INSERT INTO attributes_t1 (id, name) VALUES (1, "test1"); +INSERT INTO attributes_t1 (id) VALUES (2); CREATE TABLE attributes_t2 (id INT PRIMARY KEY, name VARCHAR(50)) PARTITION BY RANGE (id) (PARTITION p0 VALUES LESS THAN (10000), PARTITION p1 VALUES LESS THAN (MAXVALUE)); ALTER TABLE attributes_t2 ATTRIBUTES='merge_option=deny'; ALTER TABLE attributes_t2 PARTITION p0 ATTRIBUTES='merge_option=allow'; INSERT INTO attributes_t2 (id, name) VALUES (2, "test2"); +INSERT INTO attributes_t2 (id) VALUES (3); DROP TABLE attributes_t1; RECOVER TABLE attributes_t1; diff --git a/tests/integration_tests/ddl_sequence/data/prepare.sql b/tests/integration_tests/ddl_sequence/data/prepare.sql index fbff85bacb5..50b7ae63f58 100644 --- a/tests/integration_tests/ddl_sequence/data/prepare.sql +++ b/tests/integration_tests/ddl_sequence/data/prepare.sql @@ -7,6 +7,9 @@ CREATE TABLE many_cols1 ( val INT DEFAULT 0, col0 INT NOT NULL ); + +INSERT INTO many_cols1(col0) VALUES (1); + ALTER TABLE many_cols1 DROP COLUMN col0; INSERT INTO many_cols1 (val) VALUES (1); @@ -15,6 +18,9 @@ CREATE TABLE many_cols2 ( val INT DEFAULT 0, col0 INT NOT NULL ); + +INSERT INTO many_cols2(col0) VALUES (1); + ALTER TABLE many_cols2 DROP COLUMN col0; INSERT INTO many_cols2 (val) VALUES (1); @@ -23,6 +29,8 @@ CREATE TABLE many_cols3 ( val INT DEFAULT 0, col0 INT NOT NULL ); + +INSERT INTO many_cols3(col0) VALUES (1); ALTER TABLE many_cols3 DROP COLUMN col0; INSERT INTO many_cols3 (val) VALUES (1); @@ -31,6 +39,7 @@ CREATE TABLE many_cols4 ( val INT DEFAULT 0, col0 INT NOT NULL ); +INSERT INTO many_cols4(col0) VALUES (1); ALTER TABLE many_cols4 DROP COLUMN col0; INSERT INTO many_cols4 (val) VALUES (1); @@ -39,7 +48,8 @@ CREATE TABLE many_cols5 ( val INT DEFAULT 0, col0 INT NOT NULL ); +INSERT INTO many_cols5(col0) VALUES (1); ALTER TABLE many_cols5 DROP COLUMN col0; INSERT INTO many_cols5 (val) VALUES (1); -CREATE TABLE finish_mark(a int primary key) \ No newline at end of file +CREATE TABLE finish_mark(a int primary key) diff --git a/tests/integration_tests/new_ci_collation_without_old_value/data/test1.sql b/tests/integration_tests/new_ci_collation_without_old_value/data/test1.sql index 819d1bfe0b5..7332a3c7623 100644 --- a/tests/integration_tests/new_ci_collation_without_old_value/data/test1.sql +++ b/tests/integration_tests/new_ci_collation_without_old_value/data/test1.sql @@ -34,14 +34,21 @@ CREATE TABLE t5 ( insert into t1 (a) values ('A'),(' A'),('A\t'),('b'),('bA'),('bac'),('ab'); insert into t1 (a) values ('😉'); + insert into t2 (a) values ('A'),(' A'),('A\t'),('b'),('bA'),('bac'),('ab'); + +insert into t3() values(); insert into t3 (a) values ('A'),('A '),('A '),(' A'),('A\t'),('A\t '); insert into t3 (a) values ('a'),('a '),('a '),(' a'),('a\t'),('a\t '); insert into t3 (a) values ('B'),('B '),('B '),(' B'),('B\t'),('B\t '); insert into t3 (a) values ('b'),('b '),('b '),(' b'),('b\t'),('b\t '); + insert into t4 values (1,'A','A','1'),(2,'a\t','a\t','2'),(3,'ab','ab','3'),(4,'abc','abc','4'); +insert into t4(a, d) values(5, "a"); + insert into t5 (a) values ('😉'); insert into t5 (a) values ('a'),('A'),(' a'),(' A'),('a\t'),('ab'),('Ab'); + update t1 set b = b + 1; update t2 set b = 13; update t3 set b = 11 where a > 'A'; diff --git a/tests/integration_tests/row_format/data/step1.sql b/tests/integration_tests/row_format/data/step1.sql index 950549078d3..db113d36034 100644 --- a/tests/integration_tests/row_format/data/step1.sql +++ b/tests/integration_tests/row_format/data/step1.sql @@ -27,6 +27,9 @@ CREATE TABLE multi_data_type DEFAULT CHARSET = utf8 COLLATE = utf8_bin; +-- make sure `nullable` can be handled by the mounter and mq encoding protocol +INSERT INTO multi_data_type() VALUES (); + INSERT INTO multi_data_type( t_boolean, t_bigint, t_double, t_decimal, t_bit , t_date, t_datetime, t_timestamp, t_time, t_year , t_char, t_varchar, t_blob, t_text, t_enum diff --git a/tests/integration_tests/tiflash/data/prepare.sql b/tests/integration_tests/tiflash/data/prepare.sql index aadce4d60e2..c10dce39def 100644 --- a/tests/integration_tests/tiflash/data/prepare.sql +++ b/tests/integration_tests/tiflash/data/prepare.sql @@ -27,6 +27,8 @@ CREATE TABLE multi_data_type DEFAULT CHARSET = utf8 COLLATE = utf8_bin; +INSERT INTO multi_data_type() VALUES (); + INSERT INTO multi_data_type( t_boolean, t_bigint, t_double, t_decimal, t_bit , t_date, t_datetime, t_timestamp, t_time, t_year , t_char, t_varchar, t_blob, t_text, t_enum @@ -38,6 +40,8 @@ VALUES ( true, 9223372036854775807, 123.123, 123456789012.123456789012, b'100000 ALTER TABLE multi_data_type SET TIFLASH REPLICA 1; +INSERT INTO multi_data_type() VALUES (); + INSERT INTO multi_data_type( t_boolean, t_bigint, t_double, t_decimal, t_bit , t_date, t_datetime, t_timestamp, t_time, t_year , t_char, t_varchar, t_blob, t_text, t_enum @@ -53,6 +57,8 @@ WHERE id = 1; SET GLOBAL tidb_row_format_version = 2; +INSERT INTO multi_data_type() VALUES (); + INSERT INTO multi_data_type( t_boolean, t_bigint, t_double, t_decimal, t_bit , t_date, t_datetime, t_timestamp, t_time, t_year , t_char, t_varchar, t_blob, t_text, t_enum @@ -71,4 +77,4 @@ SET t_bigint = 888, "key0": "value0", "key2": "value2" }' -WHERE id = 2; \ No newline at end of file +WHERE id = 2; From 49f12dce9edebf44c0451dfd4a4c8d1f2df60ec5 Mon Sep 17 00:00:00 2001 From: Neil Shen Date: Mon, 7 Mar 2022 18:13:49 +0800 Subject: [PATCH 6/7] sorter/leveldb(ticdc): fix duplicate events (#4783) close pingcap/tiflow#3381, ref pingcap/tiflow#4631 --- cdc/sorter/leveldb/table_sorter.go | 65 ++++++++++--------------- cdc/sorter/leveldb/table_sorter_test.go | 59 +++++++++++----------- 2 files changed, 57 insertions(+), 67 deletions(-) diff --git a/cdc/sorter/leveldb/table_sorter.go b/cdc/sorter/leveldb/table_sorter.go index 01391fde7b7..b87842e92de 100644 --- a/cdc/sorter/leveldb/table_sorter.go +++ b/cdc/sorter/leveldb/table_sorter.go @@ -98,9 +98,7 @@ func (r *reader) outputResolvedTs(rts model.Ts) { // outputBufferedResolvedEvents nonblocking output resolved events and // resolved ts that are buffered in outputBuffer. // It pops outputted events in the buffer and append their key to deleteKeys. -func (r *reader) outputBufferedResolvedEvents( - buffer *outputBuffer, sendResolvedTsHint bool, -) { +func (r *reader) outputBufferedResolvedEvents(buffer *outputBuffer) { hasRemainEvents := false // Index of remaining output events remainIdx := 0 @@ -124,7 +122,7 @@ func (r *reader) outputBufferedResolvedEvents( buffer.shiftResolvedEvents(remainIdx) // If all buffered resolved events are sent, send its resolved ts too. - if sendResolvedTsHint && lastCommitTs != 0 && !hasRemainEvents { + if lastCommitTs != 0 && !hasRemainEvents { r.outputResolvedTs(lastCommitTs) } } @@ -143,8 +141,7 @@ func (r *reader) outputBufferedResolvedEvents( // // Note: outputBuffer must be empty. func (r *reader) outputIterEvents( - iter db.Iterator, hasReadLastNext bool, buffer *outputBuffer, - resolvedTs uint64, + iter db.Iterator, hasReadLastNext bool, buffer *outputBuffer, resolvedTs uint64, ) (bool, uint64, error) { lenResolvedEvents, lenDeleteKeys := buffer.len() if lenDeleteKeys > 0 || lenResolvedEvents > 0 { @@ -186,14 +183,15 @@ func (r *reader) outputIterEvents( if commitTs == 0 { commitTs = event.CRTs } - // Group resolved events that has the same commit ts. + // Read all resolved events that have the same commit ts. if commitTs == event.CRTs { buffer.appendResolvedEvent(event) continue } - // As a new event belongs to a new txn group, we need to output all - // buffered events before append the event. - r.outputBufferedResolvedEvents(buffer, true) + + // Commit ts has changed, the new event belongs to a new txn group, + // we need to output all buffered events before append the event. + r.outputBufferedResolvedEvents(buffer) lenResolvedEvents, _ = buffer.len() if lenResolvedEvents > 0 { // Output blocked, skip append new event. @@ -209,35 +207,25 @@ func (r *reader) outputIterEvents( elapsed := time.Since(start) r.metricIterReadDuration.Observe(elapsed.Seconds()) + // When iter exhausts, buffer may never get a chance to output in the above + // for loop. We retry output buffer again. + r.outputBufferedResolvedEvents(buffer) + // Try shrink buffer to release memory. buffer.maybeShrink() - // Events have not been sent, buffer them and output them later. - // Do not let outputBufferedResolvedEvents output resolved ts, instead we - // output resolved ts here. - sendResolvedTsHint := false - r.outputBufferedResolvedEvents(buffer, sendResolvedTsHint) - lenResolvedEvents, _ = buffer.len() - - // Skip output resolved ts if there is any buffered resolved event. - if lenResolvedEvents != 0 { - return hasReadNext, 0, nil - } - - if !hasNext && resolvedTs != 0 { - // Iter is exhausted and there is no resolved event (up to max - // resolved ts), output max resolved ts and return an exhausted - // resolved ts. - r.outputResolvedTs(resolvedTs) - return hasReadNext, resolvedTs, nil - } - if commitTs != 0 { - // All buffered resolved events are outputted, - // output last commit ts. - r.outputResolvedTs(commitTs) + // All resolved events whose commit ts are less or equal to the commitTs + // have read into buffer. + exhaustedResolvedTs := commitTs + if !hasNext { + // Iter is exhausted, it means resolved events whose commit ts are + // less or equal to the commitTs have read into buffer. + if resolvedTs != 0 { + exhaustedResolvedTs = resolvedTs + } } - return hasReadNext, 0, nil + return hasReadNext, exhaustedResolvedTs, nil } // TODO: inline the struct to reader. @@ -248,7 +236,7 @@ type pollState struct { maxCommitTs uint64 // The maximum commit ts for all resolved ts events. maxResolvedTs uint64 - // All resolved events before the resolved ts are outputted. + // All resolved events before the resolved ts are read into buffer. exhaustedResolvedTs uint64 // ID and router of the reader itself. @@ -426,9 +414,8 @@ func (r *reader) Poll(ctx context.Context, msgs []actormsg.Message) (running boo // Length of buffered resolved events. lenResolvedEvents, _ := r.state.outputBuf.len() if lenResolvedEvents != 0 { - // No new received events, it means output channel is available. - // output resolved events as much as possible. - r.outputBufferedResolvedEvents(r.state.outputBuf, true) + // Try output buffered resolved events. + r.outputBufferedResolvedEvents(r.state.outputBuf) lenResolvedEvents, _ = r.state.outputBuf.len() } // Build task for new events and delete sent keys. @@ -446,7 +433,7 @@ func (r *reader) Poll(ctx context.Context, msgs []actormsg.Message) (running boo // 2. There are some events that can be resolved. readIter = readIter && r.state.hasResolvedEvents() if !readIter { - // No new events and no resolved events. + // No buffered resolved events, try to send resolved ts. if !r.state.hasResolvedEvents() && r.state.maxResolvedTs != 0 { // To avoid ping-pong busy loop, we only send resolved ts // when it advances. diff --git a/cdc/sorter/leveldb/table_sorter_test.go b/cdc/sorter/leveldb/table_sorter_test.go index a2986b1644a..5a313952986 100644 --- a/cdc/sorter/leveldb/table_sorter_test.go +++ b/cdc/sorter/leveldb/table_sorter_test.go @@ -182,10 +182,9 @@ func TestReaderOutputBufferedResolvedEvents(t *testing.T) { buf := newOutputBuffer(capacity) cases := []struct { - outputChCap int - inputEvents []*model.PolymorphicEvent - inputDeleteKeys []message.Key - inputSendResolvedTsHint bool + outputChCap int + inputEvents []*model.PolymorphicEvent + inputDeleteKeys []message.Key expectEvents []*model.PolymorphicEvent expectDeleteKeys []message.Key @@ -193,10 +192,9 @@ func TestReaderOutputBufferedResolvedEvents(t *testing.T) { }{ // Empty buffer. { - outputChCap: 1, - inputEvents: []*model.PolymorphicEvent{}, - inputDeleteKeys: []message.Key{}, - inputSendResolvedTsHint: true, + outputChCap: 1, + inputEvents: []*model.PolymorphicEvent{}, + inputDeleteKeys: []message.Key{}, expectEvents: []*model.PolymorphicEvent{}, expectDeleteKeys: []message.Key{}, @@ -208,8 +206,7 @@ func TestReaderOutputBufferedResolvedEvents(t *testing.T) { inputEvents: []*model.PolymorphicEvent{ model.NewPolymorphicEvent(&model.RawKVEntry{CRTs: 1}), }, - inputDeleteKeys: []message.Key{}, - inputSendResolvedTsHint: true, + inputDeleteKeys: []message.Key{}, expectEvents: []*model.PolymorphicEvent{}, expectDeleteKeys: []message.Key{ @@ -230,7 +227,6 @@ func TestReaderOutputBufferedResolvedEvents(t *testing.T) { message.Key(encoding.EncodeKey(r.uid, r.tableID, model.NewPolymorphicEvent(&model.RawKVEntry{CRTs: 1}))), }, - inputSendResolvedTsHint: true, expectEvents: []*model.PolymorphicEvent{}, expectDeleteKeys: []message.Key{ @@ -249,7 +245,6 @@ func TestReaderOutputBufferedResolvedEvents(t *testing.T) { message.Key(encoding.EncodeKey(r.uid, r.tableID, model.NewPolymorphicEvent(&model.RawKVEntry{CRTs: 1}))), }, - inputSendResolvedTsHint: true, expectEvents: []*model.PolymorphicEvent{}, expectDeleteKeys: []message.Key{ @@ -272,8 +267,7 @@ func TestReaderOutputBufferedResolvedEvents(t *testing.T) { model.NewPolymorphicEvent(&model.RawKVEntry{CRTs: 3, RegionID: 2}), model.NewPolymorphicEvent(&model.RawKVEntry{CRTs: 3, RegionID: 3}), }, - inputDeleteKeys: []message.Key{}, - inputSendResolvedTsHint: true, + inputDeleteKeys: []message.Key{}, expectEvents: []*model.PolymorphicEvent{ model.NewPolymorphicEvent(&model.RawKVEntry{CRTs: 3, RegionID: 3}), @@ -297,8 +291,7 @@ func TestReaderOutputBufferedResolvedEvents(t *testing.T) { model.NewPolymorphicEvent(&model.RawKVEntry{CRTs: 4, RegionID: 1}), model.NewPolymorphicEvent(&model.RawKVEntry{CRTs: 4, RegionID: 2}), }, - inputDeleteKeys: []message.Key{}, - inputSendResolvedTsHint: true, + inputDeleteKeys: []message.Key{}, expectEvents: []*model.PolymorphicEvent{ model.NewPolymorphicEvent(&model.RawKVEntry{CRTs: 4, RegionID: 1}), @@ -314,7 +307,7 @@ func TestReaderOutputBufferedResolvedEvents(t *testing.T) { buf.resolvedEvents = append([]*model.PolymorphicEvent{}, cs.inputEvents...) buf.deleteKeys = append([]message.Key{}, cs.inputDeleteKeys...) - r.outputBufferedResolvedEvents(buf, cs.inputSendResolvedTsHint) + r.outputBufferedResolvedEvents(buf) require.EqualValues(t, cs.expectDeleteKeys, buf.deleteKeys, "case #%d, %v", i, cs) require.EqualValues(t, cs.expectEvents, buf.resolvedEvents, "case #%d, %v", i, cs) @@ -389,9 +382,9 @@ func TestReaderOutputIterEvents(t *testing.T) { expectExhaustedRTs: 0, expectHasReadNext: true, }, - // Nonblocking output three events and one resolved ts. + // Nonblocking output three events. { - outputChCap: 4, + outputChCap: 3, maxResolvedTs: 3, // CRTs 3 has 3 events. expectEvents: []*model.PolymorphicEvent{}, @@ -404,8 +397,6 @@ func TestReaderOutputIterEvents(t *testing.T) { newTestEvent(3, 1, 0), newTestEvent(3, 1, 1), newTestEvent(3, 1, 2), - // No buffered resolved events, it outputs a resolved ts event. - model.NewResolvedPolymorphicEvent(0, 3), }, expectExhaustedRTs: 3, // Iter is exhausted and no buffered resolved events. expectHasReadNext: true, @@ -425,7 +416,7 @@ func TestReaderOutputIterEvents(t *testing.T) { newTestEvent(4, 2, 1), }, // Events of CRTs 4 have been read and buffered. - expectExhaustedRTs: 0, + expectExhaustedRTs: 4, expectHasReadNext: true, }, // Output remaining event of CRTs 4. @@ -467,7 +458,9 @@ func TestReaderOutputIterEvents(t *testing.T) { model.NewResolvedPolymorphicEvent(0, 5), newTestEvent(6, 4, 0), }, - expectExhaustedRTs: 0, // Iter is not exhausted. + // Iter is not exhausted, but all events with commit ts 6 have been + // read into buffer. + expectExhaustedRTs: 6, expectHasReadNext: false, // (6, 4, 1) is neither output nor buffered. }, // Resolved ts covers all resolved events, nonblocking output all events. @@ -495,6 +488,18 @@ func TestReaderOutputIterEvents(t *testing.T) { expectExhaustedRTs: 7, // Iter is exhausted and no buffered resolved events. expectHasReadNext: true, }, + // All resolved events outputted, as resolved ts continues advance, + // exhausted resolved ts advances too. + { + outputChCap: 1, + maxResolvedTs: 8, + + expectEvents: []*model.PolymorphicEvent{}, + expectDeleteKeys: []message.Key{}, + expectOutputs: []*model.PolymorphicEvent{}, + expectExhaustedRTs: 8, + expectHasReadNext: true, + }, } for i, cs := range cases { @@ -715,11 +720,9 @@ func TestReaderPoll(t *testing.T) { // state is inherited from the first poll. inputIter: nil, // no need to make an iterator. - expectEvents: []*model.PolymorphicEvent{}, - expectDeleteKeys: []message.Key{}, - expectOutputs: []*model.PolymorphicEvent{ - model.NewResolvedPolymorphicEvent(0, 2), - }, + expectEvents: []*model.PolymorphicEvent{}, + expectDeleteKeys: []message.Key{}, + expectOutputs: []*model.PolymorphicEvent{}, expectMaxCommitTs: 3, expectMaxResolvedTs: 2, // exhaustedResolvedTs must advance if there is no resolved event. From 643ac189993de0e054b19e78534821e43c9980b0 Mon Sep 17 00:00:00 2001 From: sdojjy Date: Mon, 7 Mar 2022 18:51:49 +0800 Subject: [PATCH 7/7] tests(ticdc): add table actor integration test (#4738) ref pingcap/tiflow#3881 --- cdc/processor/pipeline/sorter.go | 11 +++-- cdc/processor/pipeline/sorter_test.go | 6 +-- .../_utils/start_tidb_cluster_impl | 2 +- .../conf/ticdc_table_actor.toml | 2 + .../integration_tests/bank_table_actor/run.sh | 43 +++++++++++++++++++ 5 files changed, 57 insertions(+), 7 deletions(-) create mode 100644 tests/integration_tests/bank_table_actor/conf/ticdc_table_actor.toml create mode 100644 tests/integration_tests/bank_table_actor/run.sh diff --git a/cdc/processor/pipeline/sorter.go b/cdc/processor/pipeline/sorter.go index e237654be78..81d279d80b1 100644 --- a/cdc/processor/pipeline/sorter.go +++ b/cdc/processor/pipeline/sorter.go @@ -267,7 +267,7 @@ func (n *sorterNode) handleRawEvent(ctx context.Context, event *model.Polymorphi } atomic.StoreUint64(&n.resolvedTs, rawKV.CRTs) - if resolvedTs > n.barrierTs && + if resolvedTs > n.BarrierTs() && !redo.IsConsistentEnabled(n.replConfig.Consistent.Level) { // Do not send resolved ts events that is larger than // barrier ts. @@ -278,7 +278,7 @@ func (n *sorterNode) handleRawEvent(ctx context.Context, event *model.Polymorphi // resolved ts, conflicts to this change. // TODO: Remove redolog check once redolog decouples for global // resolved ts. - event = model.NewResolvedPolymorphicEvent(0, n.barrierTs) + event = model.NewResolvedPolymorphicEvent(0, n.BarrierTs()) } } n.sorter.AddEntry(ctx, event) @@ -299,7 +299,7 @@ func (n *sorterNode) TryHandleDataMessage(ctx context.Context, msg pipeline.Mess } func (n *sorterNode) updateBarrierTs(barrierTs model.Ts) { - if barrierTs > atomic.LoadUint64(&n.barrierTs) { + if barrierTs > n.BarrierTs() { atomic.StoreUint64(&n.barrierTs, barrierTs) } } @@ -328,3 +328,8 @@ func (n *sorterNode) Destroy(ctx pipeline.NodeContext) error { func (n *sorterNode) ResolvedTs() model.Ts { return atomic.LoadUint64(&n.resolvedTs) } + +// BarrierTs returns the sorter barrierTs +func (n *sorterNode) BarrierTs() model.Ts { + return atomic.LoadUint64(&n.barrierTs) +} diff --git a/cdc/processor/pipeline/sorter_test.go b/cdc/processor/pipeline/sorter_test.go index 06cc897bc64..6741b0abf6f 100644 --- a/cdc/processor/pipeline/sorter_test.go +++ b/cdc/processor/pipeline/sorter_test.go @@ -120,7 +120,7 @@ func TestSorterResolvedTsLessEqualBarrierTs(t *testing.T) { ) err = sn.Receive(nctx) require.Nil(t, err) - require.EqualValues(t, 2, sn.barrierTs) + require.EqualValues(t, 2, sn.BarrierTs()) // Barrier message must be passed to the next node. require.EqualValues(t, pipeline.BarrierMessage(2), <-ch) @@ -152,7 +152,7 @@ func TestSorterUpdateBarrierTs(t *testing.T) { t.Parallel() s := &sorterNode{barrierTs: 1} s.updateBarrierTs(model.Ts(2)) - require.Equal(t, model.Ts(2), s.barrierTs) + require.Equal(t, model.Ts(2), s.BarrierTs()) s.updateBarrierTs(model.Ts(1)) - require.Equal(t, model.Ts(2), s.barrierTs) + require.Equal(t, model.Ts(2), s.BarrierTs()) } diff --git a/tests/integration_tests/_utils/start_tidb_cluster_impl b/tests/integration_tests/_utils/start_tidb_cluster_impl index e127bfa290b..4339b572faa 100755 --- a/tests/integration_tests/_utils/start_tidb_cluster_impl +++ b/tests/integration_tests/_utils/start_tidb_cluster_impl @@ -17,7 +17,7 @@ random_file_name= randomGenSocketsConf() { random_str=$(date '+%s%N') if [ "$(uname)" == "Darwin" ]; then - random_str=$(cat /dev/random | LC_CTYPE=C tr -dc "a-zA-Z0-9" | head -c 10) + random_str=$(cat /dev/random | LC_ALL=C tr -dc "a-zA-Z0-9" | head -c 10) fi random_file_name="$OUT_DIR/tidb-config-$random_str.toml" diff --git a/tests/integration_tests/bank_table_actor/conf/ticdc_table_actor.toml b/tests/integration_tests/bank_table_actor/conf/ticdc_table_actor.toml new file mode 100644 index 00000000000..4baed7a5c84 --- /dev/null +++ b/tests/integration_tests/bank_table_actor/conf/ticdc_table_actor.toml @@ -0,0 +1,2 @@ +[debug] +enable-table-actor=true diff --git a/tests/integration_tests/bank_table_actor/run.sh b/tests/integration_tests/bank_table_actor/run.sh new file mode 100644 index 00000000000..69458ac29fc --- /dev/null +++ b/tests/integration_tests/bank_table_actor/run.sh @@ -0,0 +1,43 @@ +#!/bin/bash + +# TODO: remove this integration test once table actor is enabled by default. + +set -eu + +CUR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +source $CUR/../_utils/test_prepare +WORK_DIR=$OUT_DIR/$TEST_NAME +CDC_BINARY=cdc.test +SINK_TYPE=$1 + +function prepare() { + + rm -rf $WORK_DIR && mkdir -p $WORK_DIR + + start_tidb_cluster --workdir $WORK_DIR + + cd $WORK_DIR + + # create table to upstream. + run_sql "CREATE DATABASE bank" ${UP_TIDB_HOST} ${UP_TIDB_PORT} + run_sql "CREATE DATABASE bank" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} + + run_cdc_server --workdir $WORK_DIR --binary $CDC_BINARY --config $CUR/conf/ticdc_table_actor.toml + + run_cdc_cli changefeed create --sink-uri="mysql://root@${DOWN_TIDB_HOST}:${DOWN_TIDB_PORT}/" +} + +trap stop_tidb_cluster EXIT +# kafka is not supported yet. +if [ "$SINK_TYPE" != "kafka" ]; then + prepare $* + + cd "$(dirname "$0")" + set -euxo pipefail + + GO111MODULE=on go run ../bank/bank.go ../bank/case.go -u "root@tcp(${UP_TIDB_HOST}:${UP_TIDB_PORT})/bank" \ + -d "root@tcp(${DOWN_TIDB_HOST}:${DOWN_TIDB_PORT})/bank" --test-round=20000 + + cleanup_process $CDC_BINARY + echo "[$(date)] <<<<<< run test case $TEST_NAME success! >>>>>>" +fi