diff --git a/cdc/capture/capture.go b/cdc/capture/capture.go index 3d09c741779..97d68aff8ff 100644 --- a/cdc/capture/capture.go +++ b/cdc/capture/capture.go @@ -28,6 +28,7 @@ import ( "github.com/pingcap/ticdc/cdc/model" "github.com/pingcap/ticdc/cdc/owner" "github.com/pingcap/ticdc/cdc/processor" + "github.com/pingcap/ticdc/cdc/processor/pipeline/system" "github.com/pingcap/ticdc/pkg/config" cdcContext "github.com/pingcap/ticdc/pkg/context" cerror "github.com/pingcap/ticdc/pkg/errors" @@ -60,6 +61,8 @@ type Capture struct { etcdClient *etcd.CDCEtcdClient grpcPool kv.GrpcPool + tableActorSystem *system.System + cancel context.CancelFunc newProcessorManager func() *processor.Manager @@ -103,6 +106,19 @@ func (c *Capture) reset(ctx context.Context) error { if c.grpcPool != nil { c.grpcPool.Close() } + if c.tableActorSystem != nil { + err := c.tableActorSystem.Stop() + if err != nil { + log.Warn("stop table actor system failed", zap.Error(err)) + } + } + if conf.Debug.EnableTableActor { + c.tableActorSystem = system.NewSystem() + err = c.tableActorSystem.Start(ctx) + if err != nil { + return errors.Annotate(cerror.WrapError(cerror.ErrNewCaptureFailed, err), "create capture session") + } + } c.grpcPool = kv.NewGrpcPoolImpl(ctx, conf.Security) log.Info("init capture", zap.String("capture-id", c.info.ID), zap.String("capture-addr", c.info.AdvertiseAddr)) return nil @@ -148,11 +164,12 @@ func (c *Capture) Run(ctx context.Context) error { func (c *Capture) run(stdCtx context.Context) error { ctx := cdcContext.NewContext(stdCtx, &cdcContext.GlobalVars{ - PDClient: c.pdClient, - KVStorage: c.kvStorage, - CaptureInfo: c.info, - EtcdClient: c.etcdClient, - GrpcPool: c.grpcPool, + PDClient: c.pdClient, + KVStorage: c.kvStorage, + CaptureInfo: c.info, + EtcdClient: c.etcdClient, + GrpcPool: c.grpcPool, + TableActorSystem: c.tableActorSystem, }) err := c.register(ctx) if err != nil { @@ -354,6 +371,13 @@ func (c *Capture) AsyncClose() { if c.grpcPool != nil { c.grpcPool.Close() } + if c.tableActorSystem != nil { + err := c.tableActorSystem.Stop() + if err != nil { + log.Warn("stop table actor system failed", zap.Error(err)) + } + c.tableActorSystem = nil + } } // WriteDebugInfo writes the debug info into writer. diff --git a/cdc/metrics.go b/cdc/metrics.go index ae9d87159dc..0a4d0b36d12 100644 --- a/cdc/metrics.go +++ b/cdc/metrics.go @@ -23,6 +23,7 @@ import ( "github.com/pingcap/ticdc/cdc/sink" "github.com/pingcap/ticdc/cdc/sorter" "github.com/pingcap/ticdc/cdc/sorter/leveldb" + "github.com/pingcap/ticdc/cdc/sorter/leveldb/system" "github.com/pingcap/ticdc/cdc/sorter/memory" "github.com/pingcap/ticdc/cdc/sorter/unified" "github.com/pingcap/ticdc/pkg/actor" @@ -53,4 +54,5 @@ func init() { memory.InitMetrics(registry) unified.InitMetrics(registry) leveldb.InitMetrics(registry) + system.InitMetrics(registry) } diff --git a/cdc/processor/pipeline/system/system.go b/cdc/processor/pipeline/system/system.go new file mode 100644 index 00000000000..a08524da482 --- /dev/null +++ b/cdc/processor/pipeline/system/system.go @@ -0,0 +1,44 @@ +// Copyright 2021 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package system + +import ( + "context" + + "github.com/pingcap/ticdc/pkg/actor" +) + +// System manages table pipeline global resource. +type System struct { + tableActorSystem *actor.System + tableActorRouter *actor.Router +} + +// NewSystem returns a system. +func NewSystem() *System { + return &System{} +} + +// Start starts a system. +func (s *System) Start(ctx context.Context) error { + // todo: make the table actor system configurable + s.tableActorSystem, s.tableActorRouter = actor.NewSystemBuilder("table").Build() + s.tableActorSystem.Start(ctx) + return nil +} + +// Stop stops a system. +func (s *System) Stop() error { + return s.tableActorSystem.Stop() +} diff --git a/cdc/processor/pipeline/system/system_test.go b/cdc/processor/pipeline/system/system_test.go new file mode 100644 index 00000000000..812951cc312 --- /dev/null +++ b/cdc/processor/pipeline/system/system_test.go @@ -0,0 +1,29 @@ +// Copyright 2021 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package system + +import ( + "context" + "testing" + + "github.com/stretchr/testify/require" +) + +func TestStartAndStopSystem(t *testing.T) { + t.Parallel() + + s := NewSystem() + require.Nil(t, s.Start(context.TODO())) + require.Nil(t, s.Stop()) +} diff --git a/cdc/scheduler/agent.go b/cdc/scheduler/agent.go new file mode 100644 index 00000000000..35e68a20f78 --- /dev/null +++ b/cdc/scheduler/agent.go @@ -0,0 +1,454 @@ +// Copyright 2021 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package scheduler + +import ( + "sync" + "time" + + "github.com/edwingeng/deque" + "github.com/pingcap/errors" + "github.com/pingcap/log" + "github.com/pingcap/ticdc/cdc/model" + "github.com/pingcap/ticdc/cdc/scheduler/util" + "github.com/pingcap/ticdc/pkg/context" + cerrors "github.com/pingcap/ticdc/pkg/errors" + "github.com/uber-go/atomic" + "go.uber.org/zap" +) + +// Agent is an interface for an object inside Processor that is responsible +// for receiving commands from the Owner. +// Ideally the processor should drive the Agent by Tick. +type Agent interface { + // Tick is called periodically by the processor to drive the Agent's internal logic. + Tick(ctx context.Context) error + + // GetLastSentCheckpointTs returns the last checkpoint-ts already sent to the Owner. + GetLastSentCheckpointTs() (checkpointTs model.Ts) +} + +// TableExecutor is an abstraction for "Processor". +// +// This interface is so designed that it would be the least problematic +// to adapt the current Processor implementation to it. +// TODO find a way to make the semantics easier to understand. +type TableExecutor interface { + AddTable(ctx context.Context, tableID model.TableID) error + RemoveTable(ctx context.Context, tableID model.TableID) (done bool, err error) + IsAddTableFinished(ctx context.Context, tableID model.TableID) (done bool) + IsRemoveTableFinished(ctx context.Context, tableID model.TableID) (done bool) + + // GetAllCurrentTables should return all tables that are being run, + // being added and being removed. + // + // NOTE: two subsequent calls to the method should return the same + // result, unless there is a call to AddTable, RemoveTable, IsAddTableFinished + // or IsRemoveTableFinished in between two calls to this method. + GetAllCurrentTables() []model.TableID + + // GetCheckpoint returns the local checkpoint-ts and resolved-ts of + // the processor. Its calculation should take into consideration all + // tables that would have been returned if GetAllCurrentTables had been + // called immediately before. + GetCheckpoint() (checkpointTs, resolvedTs model.Ts) +} + +// ProcessorMessenger implements how messages should be sent to the owner, +// and should be able to know whether there are any messages not yet acknowledged +// 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) + // SyncTaskStatuses informs the owner of the processor's current internal state. + SyncTaskStatuses(ctx context.Context, running, adding, removing []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. + OnOwnerChanged(ctx context.Context, newOwnerCaptureID model.CaptureID) + // Close closes the messenger and does the necessary cleanup. + Close() error +} + +// BaseAgentConfig stores configurations for BaseAgent +type BaseAgentConfig struct { + // SendCheckpointTsInterval is the interval to send checkpoint-ts to the owner. + SendCheckpointTsInterval time.Duration +} + +// BaseAgent is an implementation of Agent. +// It implements the basic logic and is useful only if the Processor +// implements its own TableExecutor and ProcessorMessenger. +type BaseAgent struct { + executor TableExecutor + communicator ProcessorMessenger + + // pendingOpsMu protects pendingOps. + // Note that we need a mutex because some methods are expected + // to be called from a message handler goroutine. + pendingOpsMu sync.Mutex + // pendingOps is a queue of operations yet to be processed. + // the Deque stores *agentOperation. + pendingOps deque.Deque + + // tableOperations is a map from tableID to the operation + // that is currently being processed. + tableOperations map[model.TableID]*agentOperation + + // needSyncNow indicates that the agent needs to send the + // current owner a sync message as soon as possible. + needSyncNow *atomic.Bool + + // checkpointSender is used to send checkpoint-ts to the owner. + checkpointSender checkpointSender + + ownerInfoMu sync.RWMutex + ownerInfo *ownerInfo + + // ownerHasChanged indicates that the owner has changed and + // the communicator needs to be reset. + ownerHasChanged *atomic.Bool + + // read-only fields + config *BaseAgentConfig + logger *zap.Logger +} + +// NewBaseAgent creates a new BaseAgent. +func NewBaseAgent( + changeFeedID model.ChangeFeedID, + executor TableExecutor, + messenger ProcessorMessenger, + config *BaseAgentConfig, +) *BaseAgent { + logger := log.L().With(zap.String("changefeed-id", changeFeedID)) + return &BaseAgent{ + pendingOps: deque.NewDeque(), + tableOperations: map[model.TableID]*agentOperation{}, + logger: logger, + executor: executor, + ownerInfo: &ownerInfo{}, + communicator: messenger, + needSyncNow: atomic.NewBool(true), + checkpointSender: newCheckpointSender(messenger, logger, config.SendCheckpointTsInterval), + ownerHasChanged: atomic.NewBool(false), + config: config, + } +} + +type agentOperationStatus int32 + +const ( + operationReceived = agentOperationStatus(iota + 1) + operationProcessed + operationFinished +) + +type agentOperation struct { + TableID model.TableID + IsDelete bool + + status agentOperationStatus +} + +type ownerInfo struct { + OwnerCaptureID model.CaptureID + // OwnerRev is needed in order to know who is the latest owner + // whenever there is a possibility of confusion, usually when the + // old owner has just gone down but its gRPC messages have yet to be + // processed. Since messages from the old and new owner could interleave, + // we need a way to tell. + OwnerRev int64 +} + +// Tick implements the interface Agent. +func (a *BaseAgent) Tick(ctx context.Context) error { + if a.ownerHasChanged.Swap(false) { + // We need to notify the communicator if the owner has changed. + // This is necessary because the communicator might be waiting for + // messages to be received by the previous owner. + a.communicator.OnOwnerChanged(ctx, a.currentOwner()) + } + + if a.needSyncNow.Load() { + done, err := a.sendSync(ctx) + if err != nil { + return errors.Trace(err) + } + if !done { + // We need to send a sync successfully before proceeding. + return nil + } + a.needSyncNow.Store(false) + } + + // We send checkpoints only after a required Sync to make the protocol + // easier to reason about. + if err := a.sendCheckpoint(ctx); err != nil { + return errors.Trace(err) + } + + opsToApply := a.popPendingOps() + for _, op := range opsToApply { + if _, ok := a.tableOperations[op.TableID]; ok { + a.logger.DPanic("duplicate operation", zap.Any("op", op)) + return cerrors.ErrProcessorDuplicateOperations.GenWithStackByArgs(op.TableID) + } + a.tableOperations[op.TableID] = op + } + + if err := a.processOperations(ctx); err != nil { + return errors.Trace(err) + } + + return nil +} + +// GetLastSentCheckpointTs implements the interface Agent. +func (a *BaseAgent) GetLastSentCheckpointTs() model.Ts { + return a.checkpointSender.LastSentCheckpointTs() +} + +func (a *BaseAgent) popPendingOps() (opsToApply []*agentOperation) { + a.pendingOpsMu.Lock() + defer a.pendingOpsMu.Unlock() + + for !a.pendingOps.Empty() { + opsBatch := a.pendingOps.PopManyFront(128 /* batch size */) + for _, op := range opsBatch { + opsToApply = append(opsToApply, op.(*agentOperation)) + } + } + return +} + +// sendSync needs to be called with a.pendingOpsMu held. +func (a *BaseAgent) sendSync(ctx context.Context) (bool, error) { + var adding, removing, running []model.TableID + for _, op := range a.tableOperations { + if !op.IsDelete { + adding = append(adding, op.TableID) + } else { + removing = append(removing, op.TableID) + } + } + for _, tableID := range a.executor.GetAllCurrentTables() { + if _, ok := a.tableOperations[tableID]; ok { + // Tables with a pending operation is not in the Running state. + continue + } + running = append(running, tableID) + } + + // We are sorting these so that there content can be predictable in tests. + // TODO try to find a better way. + util.SortTableIDs(running) + util.SortTableIDs(adding) + util.SortTableIDs(removing) + done, err := a.communicator.SyncTaskStatuses(ctx, running, adding, removing) + if err != nil { + return false, errors.Trace(err) + } + return done, nil +} + +// processOperations tries to make progress on each pending table operations. +// It queries the executor for the current status of each table. +func (a *BaseAgent) processOperations(ctx context.Context) error { + for tableID, op := range a.tableOperations { + switch op.status { + case operationReceived: + if !op.IsDelete { + // add table + if err := a.executor.AddTable(ctx, op.TableID); err != nil { + return errors.Trace(err) + } + } else { + // delete table + done, err := a.executor.RemoveTable(ctx, op.TableID) + if err != nil { + return errors.Trace(err) + } + if !done { + break + } + } + op.status = operationProcessed + fallthrough + case operationProcessed: + var done bool + if !op.IsDelete { + done = a.executor.IsAddTableFinished(ctx, op.TableID) + } else { + done = a.executor.IsRemoveTableFinished(ctx, op.TableID) + } + if !done { + break + } + op.status = operationFinished + fallthrough + case operationFinished: + done, err := a.communicator.FinishTableOperation(ctx, op.TableID) + if err != nil { + return errors.Trace(err) + } + if done { + delete(a.tableOperations, tableID) + } + } + } + return nil +} + +func (a *BaseAgent) sendCheckpoint(ctx context.Context) error { + checkpointProvider := func() (checkpointTs, resolvedTs model.Ts, ok bool) { + // We cannot have a meaningful checkpoint for a processor running NO table. + if len(a.executor.GetAllCurrentTables()) == 0 { + a.logger.Debug("no table is running, skip sending checkpoint") + return 0, 0, false // false indicates no available checkpoint + } + checkpointTs, resolvedTs = a.executor.GetCheckpoint() + ok = true + return + } + + if err := a.checkpointSender.SendCheckpoint(ctx, checkpointProvider); err != nil { + return errors.Trace(err) + } + return nil +} + +// OnOwnerDispatchedTask should be called when the Owner sent a new dispatched task. +// The Processor is responsible for calling this function when appropriate. +func (a *BaseAgent) OnOwnerDispatchedTask( + ownerCaptureID model.CaptureID, + ownerRev int64, + tableID model.TableID, + isDelete bool, +) { + if !a.updateOwnerInfo(ownerCaptureID, ownerRev) { + a.logger.Info("task from stale owner ignored", + zap.Int64("table-id", tableID), + zap.Bool("is-delete", isDelete)) + return + } + + a.pendingOpsMu.Lock() + defer a.pendingOpsMu.Unlock() + + op := &agentOperation{ + TableID: tableID, + IsDelete: isDelete, + status: operationReceived, + } + a.pendingOps.PushBack(op) + + a.logger.Debug("OnOwnerDispatchedTask", + zap.String("owner-capture-id", ownerCaptureID), + zap.Int64("owner-rev", ownerRev), + zap.Any("op", op)) +} + +// OnOwnerAnnounce should be called when a new Owner announces its ownership. +// The Processor is responsible for calling this function when appropriate. +// +// ownerRev is the revision number generated by the election mechanism to +// indicate the order in which owners are elected. +func (a *BaseAgent) OnOwnerAnnounce( + ownerCaptureID model.CaptureID, + ownerRev int64, +) { + if !a.updateOwnerInfo(ownerCaptureID, ownerRev) { + a.logger.Info("sync request from stale owner ignored", + zap.String("owner-capture-id", ownerCaptureID), + zap.Int64("owner-rev", ownerRev)) + return + } + + // Sets the needSyncNow flag so that in the next tick, + // we will try to send a Sync to the Owner. + a.needSyncNow.Store(true) + + a.logger.Info("OnOwnerAnnounce", + zap.String("owner-capture-id", ownerCaptureID), + zap.Int64("owner-rev", ownerRev)) +} + +// updateOwnerInfo tries to update the stored ownerInfo, and returns false if the +// owner is stale, in which case the incoming message should be ignored since +// it has come from an owner that for sure is dead. +// +// ownerCaptureID: the incoming owner's capture ID +// ownerRev: the incoming owner's revision as generated by Etcd election. +func (a *BaseAgent) updateOwnerInfo(ownerCaptureID model.CaptureID, ownerRev int64) bool { + a.ownerInfoMu.Lock() + defer a.ownerInfoMu.Unlock() + + if a.ownerInfo.OwnerRev < ownerRev { + // the stored ownerInfo is stale, we update it + a.ownerInfo.OwnerRev = ownerRev + a.ownerInfo.OwnerCaptureID = ownerCaptureID + + // We set a flag to indicate that the owner has changed. + // This flag is needed so that the communicator can be reset in time. + // It is difficult to reset the communicator here, because this function + // is called in a separate goroutine (possibly in the message handler), + // so blocking it for lock will increase the risk of deadlock. + a.ownerHasChanged.Store(true) + + a.logger.Info("owner updated", + zap.Any("new-owner-info", a.ownerInfo)) + + // Resets the deque so that pending operations from the previous owner + // will not be processed. + // Note: these pending operations have not yet been processed by the agent, + // so it is okay to lose them. + a.pendingOpsMu.Lock() + a.pendingOps = deque.NewDeque() + a.pendingOpsMu.Unlock() + return true + } + if a.ownerInfo.OwnerRev > ownerRev { + // the owner where the message just came from is stale. + a.logger.Info("message received from stale owner", + zap.Any("old-owner", ownerInfo{ + OwnerCaptureID: ownerCaptureID, + OwnerRev: ownerRev, + }), + zap.Any("current-owner", a.ownerInfo)) + + // Returning false indicates that we should reject the owner, + // because it is stale. + return false + } + if a.ownerInfo.OwnerCaptureID != ownerCaptureID { + // This panic will happen only if two messages have been received + // with the same ownerRev but with different ownerIDs. + // This should never happen unless the election via Etcd is buggy. + a.logger.Panic("owner IDs do not match", + zap.String("expected", a.ownerInfo.OwnerCaptureID), + zap.String("actual", ownerCaptureID)) + } + return true +} + +func (a *BaseAgent) currentOwner() model.CaptureID { + a.ownerInfoMu.RLock() + defer a.ownerInfoMu.RUnlock() + + return a.ownerInfo.OwnerCaptureID +} diff --git a/cdc/scheduler/agent_mock.go b/cdc/scheduler/agent_mock.go deleted file mode 100644 index 9c3c248a886..00000000000 --- a/cdc/scheduler/agent_mock.go +++ /dev/null @@ -1,38 +0,0 @@ -// Copyright 2021 PingCAP, Inc. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// See the License for the specific language governing permissions and -// limitations under the License. - -package scheduler - -import ( - "github.com/pingcap/ticdc/cdc/model" - "github.com/pingcap/ticdc/pkg/context" -) - -// ProcessorMessenger implements how messages should be sent to the owner, -// and should be able to know whether there are any messages not yet acknowledged -// by the owner. -type ProcessorMessenger interface { - // FinishTableOperation notifies the owner that a table operation has finished. - FinishTableOperation(ctx context.Context, tableID model.TableID) (bool, error) - // SyncTaskStatuses informs the owner of the processor's current internal state. - SyncTaskStatuses(ctx context.Context, running, adding, removing []model.TableID) (bool, 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) (bool, 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. - OnOwnerChanged(ctx context.Context, newOwnerCaptureID model.CaptureID) - // Close closes the messenger and does the necessary cleanup. - Close() error -} diff --git a/cdc/scheduler/agent_mock_test.go b/cdc/scheduler/agent_mock_test.go index 91b3cf5111c..bce1f64c8e3 100644 --- a/cdc/scheduler/agent_mock_test.go +++ b/cdc/scheduler/agent_mock_test.go @@ -14,9 +14,12 @@ package scheduler import ( + "testing" + "github.com/pingcap/ticdc/cdc/model" cdcContext "github.com/pingcap/ticdc/pkg/context" "github.com/stretchr/testify/mock" + "github.com/stretchr/testify/require" ) type mockProcessorMessenger struct { @@ -51,3 +54,87 @@ func (m *mockProcessorMessenger) Close() error { args := m.Called() return args.Error(0) } + +type mockCheckpointSender struct { + lastSentCheckpointTs model.Ts + lastSentResolvedTs model.Ts +} + +func (s *mockCheckpointSender) SendCheckpoint(_ cdcContext.Context, provider checkpointProviderFunc) error { + checkpointTs, resolvedTs, ok := provider() + if !ok { + return nil + } + s.lastSentCheckpointTs = checkpointTs + s.lastSentResolvedTs = resolvedTs + return nil +} + +func (s *mockCheckpointSender) LastSentCheckpointTs() model.Ts { + return s.lastSentCheckpointTs +} + +type mockTableExecutor struct { + mock.Mock + + t *testing.T + + adding, running, removing map[model.TableID]struct{} +} + +func newMockTableExecutor(t *testing.T) *mockTableExecutor { + return &mockTableExecutor{ + t: t, + adding: map[model.TableID]struct{}{}, + running: map[model.TableID]struct{}{}, + removing: map[model.TableID]struct{}{}, + } +} + +func (e *mockTableExecutor) AddTable(ctx cdcContext.Context, tableID model.TableID) error { + args := e.Called(ctx, tableID) + require.NotContains(e.t, e.adding, tableID) + require.NotContains(e.t, e.running, tableID) + require.NotContains(e.t, e.removing, tableID) + e.adding[tableID] = struct{}{} + return args.Error(0) +} + +func (e *mockTableExecutor) RemoveTable(ctx cdcContext.Context, tableID model.TableID) (bool, error) { + args := e.Called(ctx, tableID) + require.Contains(e.t, e.running, tableID) + require.NotContains(e.t, e.removing, tableID) + delete(e.running, tableID) + e.removing[tableID] = struct{}{} + return args.Bool(0), args.Error(1) +} + +func (e *mockTableExecutor) IsAddTableFinished(ctx cdcContext.Context, tableID model.TableID) bool { + _, ok := e.running[tableID] + return ok +} + +func (e *mockTableExecutor) IsRemoveTableFinished(ctx cdcContext.Context, tableID model.TableID) bool { + _, ok := e.removing[tableID] + return !ok +} + +func (e *mockTableExecutor) GetAllCurrentTables() []model.TableID { + var ret []model.TableID + for tableID := range e.adding { + ret = append(ret, tableID) + } + for tableID := range e.running { + ret = append(ret, tableID) + } + for tableID := range e.removing { + ret = append(ret, tableID) + } + + return ret +} + +func (e *mockTableExecutor) GetCheckpoint() (checkpointTs, resolvedTs model.Ts) { + args := e.Called() + return args.Get(0).(model.Ts), args.Get(1).(model.Ts) +} diff --git a/cdc/scheduler/agent_test.go b/cdc/scheduler/agent_test.go new file mode 100644 index 00000000000..4eb06f37611 --- /dev/null +++ b/cdc/scheduler/agent_test.go @@ -0,0 +1,241 @@ +// Copyright 2021 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package scheduler + +import ( + "testing" + + "github.com/pingcap/ticdc/cdc/model" + cdcContext "github.com/pingcap/ticdc/pkg/context" + "github.com/stretchr/testify/mock" + "github.com/stretchr/testify/require" +) + +// read only +var agentConfigForTesting = &BaseAgentConfig{SendCheckpointTsInterval: 0} + +func TestAgentAddTable(t *testing.T) { + ctx := cdcContext.NewBackendContext4Test(false) + + 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) + 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) + executor.On("AddTable", mock.Anything, model.TableID(1)).Return(nil) + messenger.On("OnOwnerChanged", mock.Anything, "capture-1") + + err = agent.Tick(ctx) + require.NoError(t, err) + messenger.AssertExpectations(t) + + executor.ExpectedCalls = nil + messenger.ExpectedCalls = nil + delete(executor.adding, model.TableID(1)) + 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) + + err = agent.Tick(ctx) + require.NoError(t, err) + messenger.AssertExpectations(t) + + executor.ExpectedCalls = nil + messenger.ExpectedCalls = nil + + messenger.On("Barrier", mock.Anything).Return(true) + executor.On("GetCheckpoint").Return(model.Ts(1003), model.Ts(1005)) + messenger.On("SendCheckpoint", mock.Anything, model.Ts(1003), model.Ts(1005)).Return(true, nil) + + err = agent.Tick(ctx) + require.NoError(t, err) + messenger.AssertExpectations(t) +} + +func TestAgentRemoveTable(t *testing.T) { + ctx := cdcContext.NewBackendContext4Test(false) + + executor := newMockTableExecutor(t) + executor.running[model.TableID(1)] = struct{}{} + executor.running[model.TableID(2)] = struct{}{} + + 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) + 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) + err := agent.Tick(ctx) + require.NoError(t, err) + messenger.AssertExpectations(t) + + executor.ExpectedCalls = nil + messenger.ExpectedCalls = nil + agent.OnOwnerDispatchedTask("capture-2", 1, model.TableID(1), true) + 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) + messenger.On("Barrier", mock.Anything).Return(true) + err = agent.Tick(ctx) + require.NoError(t, err) + messenger.AssertExpectations(t) + + // Inject an owner change + 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("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) + agent.OnOwnerAnnounce("capture-3", 2) + err = agent.Tick(ctx) + require.NoError(t, err) + messenger.AssertExpectations(t) + + executor.ExpectedCalls = nil + messenger.ExpectedCalls = nil + 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("SendCheckpoint", mock.Anything, model.Ts(1002), model.Ts(1000)).Return(true, nil) + + err = agent.Tick(ctx) + require.NoError(t, err) + messenger.AssertExpectations(t) +} + +func TestAgentOwnerChangedWhileAddingTable(t *testing.T) { + ctx := cdcContext.NewBackendContext4Test(false) + + 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) + err := agent.Tick(ctx) + require.NoError(t, err) + messenger.AssertExpectations(t) + + agent.OnOwnerDispatchedTask("capture-1", 1, model.TableID(1), false) + executor.On("AddTable", mock.Anything, model.TableID(1)).Return(nil) + messenger.On("OnOwnerChanged", mock.Anything, "capture-1") + + err = agent.Tick(ctx) + require.NoError(t, err) + messenger.AssertExpectations(t) + + executor.ExpectedCalls = nil + messenger.ExpectedCalls = nil + executor.On("GetCheckpoint").Return(model.Ts(1002), model.Ts(1000)) + messenger.On("SendCheckpoint", mock.Anything, model.Ts(1002), model.Ts(1000)).Return(true, nil) + + err = agent.Tick(ctx) + require.NoError(t, err) + messenger.AssertExpectations(t) + + executor.ExpectedCalls = nil + 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("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) + + err = agent.Tick(ctx) + require.NoError(t, err) + messenger.AssertExpectations(t) +} + +func TestAgentReceiveFromStaleOwner(t *testing.T) { + ctx := cdcContext.NewBackendContext4Test(false) + + executor := newMockTableExecutor(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) + err := agent.Tick(ctx) + require.NoError(t, err) + messenger.AssertExpectations(t) + + agent.OnOwnerDispatchedTask("capture-1", 1, model.TableID(1), false) + executor.On("AddTable", mock.Anything, model.TableID(1)).Return(nil) + messenger.On("OnOwnerChanged", mock.Anything, "capture-1") + + err = agent.Tick(ctx) + require.NoError(t, err) + messenger.AssertExpectations(t) + + executor.ExpectedCalls = nil + messenger.ExpectedCalls = nil + executor.On("GetCheckpoint").Return(model.Ts(1002), model.Ts(1000)) + // Stale owner + agent.OnOwnerDispatchedTask("capture-2", 0, model.TableID(2), false) + + err = agent.Tick(ctx) + require.NoError(t, err) + messenger.AssertExpectations(t) + + executor.ExpectedCalls = nil + messenger.ExpectedCalls = nil + // Stale owner announce + executor.On("GetCheckpoint").Return(model.Ts(1002), model.Ts(1000)) + agent.OnOwnerAnnounce("capture-2", 0) + err = agent.Tick(ctx) + require.NoError(t, err) + messenger.AssertExpectations(t) +} + +func TestOwnerMismatchShouldPanic(t *testing.T) { + ctx := cdcContext.NewBackendContext4Test(false) + + executor := newMockTableExecutor(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) + err := agent.Tick(ctx) + require.NoError(t, err) + messenger.AssertExpectations(t) + + // capture-1 becomes owner with ownerRev == 1 + agent.OnOwnerAnnounce("capture-1", 1) + messenger.On("OnOwnerChanged", mock.Anything, "capture-1") + + err = agent.Tick(ctx) + require.NoError(t, err) + messenger.AssertExpectations(t) + + // capture-2 claims to be the owner with ownerRev == 1 + require.Panics(t, func() { + agent.OnOwnerAnnounce("capture-2", 1) + }, "should have panicked") +} diff --git a/cdc/scheduler/balancer.go b/cdc/scheduler/balancer.go new file mode 100644 index 00000000000..1a102a6353f --- /dev/null +++ b/cdc/scheduler/balancer.go @@ -0,0 +1,157 @@ +// Copyright 2021 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package scheduler + +import ( + "math" + + "github.com/pingcap/ticdc/cdc/model" + "github.com/pingcap/ticdc/cdc/scheduler/util" + "go.uber.org/zap" +) + +// balancer is used to support the rebalance process, in which +// some victims are chosen and de-scheduled. Later, the victims +// will be automatically rescheduled, during which the target captures +// will be chosen so that the workload is the most balanced. +// +// The FindTarget method is also used when we need to schedule any table, +// not only when we need to rebalance. +type balancer interface { + // FindVictims returns a set of possible victim tables. + // Removing these tables will make the workload more balanced. + FindVictims( + tables *util.TableSet, + captures map[model.CaptureID]*model.CaptureInfo, + ) (tablesToRemove []*util.TableRecord) + + // FindTarget returns a target capture to add a table to. + FindTarget( + tables *util.TableSet, + captures map[model.CaptureID]*model.CaptureInfo, + ) (minLoadCapture model.CaptureID, ok bool) +} + +// tableNumberBalancer implements a balance strategy based on the +// current number of tables replicated by each capture. +// TODO: Implement finer-grained balance strategy based on the actual +// workload of each table. +type tableNumberBalancer struct { + logger *zap.Logger +} + +func newTableNumberRebalancer(logger *zap.Logger) balancer { + return &tableNumberBalancer{ + logger: logger, + } +} + +// FindTarget returns the capture with the smallest workload (in table count). +func (r *tableNumberBalancer) FindTarget( + tables *util.TableSet, + captures map[model.CaptureID]*model.CaptureInfo, +) (minLoadCapture model.CaptureID, ok bool) { + if len(captures) == 0 { + return "", false + } + + captureWorkload := make(map[model.CaptureID]int) + for captureID := range captures { + captureWorkload[captureID] = 0 + } + + for captureID, tables := range tables.GetAllTablesGroupedByCaptures() { + // We use the number of tables as workload + captureWorkload[captureID] = len(tables) + } + + candidate := "" + minWorkload := math.MaxInt64 + + for captureID, workload := range captureWorkload { + if workload < minWorkload { + minWorkload = workload + candidate = captureID + } + } + + if minWorkload == math.MaxInt64 { + r.logger.Panic("unexpected minWorkerload == math.MaxInt64") + } + + return candidate, true +} + +// FindVictims returns some victims to remove. +// Read the comment in the function body on the details of the victim selection. +func (r *tableNumberBalancer) FindVictims( + tables *util.TableSet, + captures map[model.CaptureID]*model.CaptureInfo, +) []*util.TableRecord { + // Algorithm overview: We try to remove some tables as the victims so that + // no captures are assigned more tables than the average workload measured in table number, + // modulo the necessary margin due to the fraction part of the average. + // + // In formula, we try to maintain the invariant: + // + // num(tables assigned to any capture) < num(tables) / num(captures) + 1 + + totalTableNum := len(tables.GetAllTables()) + captureNum := len(captures) + + if captureNum == 0 { + return nil + } + + upperLimitPerCapture := int(math.Ceil(float64(totalTableNum) / float64(captureNum))) + + r.logger.Info("Start rebalancing", + zap.Int("table-num", totalTableNum), + zap.Int("capture-num", captureNum), + zap.Int("target-limit", upperLimitPerCapture)) + + var victims []*util.TableRecord + for _, tables := range tables.GetAllTablesGroupedByCaptures() { + var tableList []model.TableID + for tableID := range tables { + tableList = append(tableList, tableID) + } + // We sort the tableIDs here so that the result is deterministic, + // which would aid testing and debugging. + util.SortTableIDs(tableList) + + tableNum2Remove := len(tables) - upperLimitPerCapture + if tableNum2Remove <= 0 { + continue + } + + // here we pick `tableNum2Remove` tables to delete, + for _, tableID := range tableList { + if tableNum2Remove <= 0 { + break + } + + record := tables[tableID] + if record == nil { + panic("unreachable") + } + + r.logger.Info("Rebalance: find victim table", + zap.Any("table-record", record)) + victims = append(victims, record) + tableNum2Remove-- + } + } + return victims +} diff --git a/cdc/scheduler/balancer_test.go b/cdc/scheduler/balancer_test.go new file mode 100644 index 00000000000..08e398a742b --- /dev/null +++ b/cdc/scheduler/balancer_test.go @@ -0,0 +1,130 @@ +// Copyright 2021 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package scheduler + +import ( + "testing" + + "github.com/pingcap/ticdc/cdc/model" + "github.com/pingcap/ticdc/cdc/scheduler/util" + "github.com/stretchr/testify/require" + "go.uber.org/zap" +) + +func TestBalancerFindVictims(t *testing.T) { + balancer := newTableNumberRebalancer(zap.L()) + tables := util.NewTableSet() + + tables.AddTableRecord(&util.TableRecord{ + TableID: 1, + CaptureID: "capture-1", + }) + tables.AddTableRecord(&util.TableRecord{ + TableID: 2, + CaptureID: "capture-1", + }) + tables.AddTableRecord(&util.TableRecord{ + TableID: 3, + CaptureID: "capture-1", + }) + tables.AddTableRecord(&util.TableRecord{ + TableID: 4, + CaptureID: "capture-1", + }) + tables.AddTableRecord(&util.TableRecord{ + TableID: 5, + CaptureID: "capture-2", + }) + tables.AddTableRecord(&util.TableRecord{ + TableID: 6, + CaptureID: "capture-2", + }) + + mockCaptureInfos := map[model.CaptureID]*model.CaptureInfo{ + "capture-1": { + ID: "capture-1", + }, + "capture-2": { + ID: "capture-2", + }, + "capture-3": { + ID: "capture-3", + }, + } + + victims := balancer.FindVictims(tables, mockCaptureInfos) + require.Len(t, victims, 2) + require.Contains(t, victims, &util.TableRecord{ + TableID: 1, + CaptureID: "capture-1", + }) + require.Contains(t, victims, &util.TableRecord{ + TableID: 2, + CaptureID: "capture-1", + }) +} + +func TestBalancerFindTarget(t *testing.T) { + balancer := newTableNumberRebalancer(zap.L()) + tables := util.NewTableSet() + + tables.AddTableRecord(&util.TableRecord{ + TableID: 1, + CaptureID: "capture-1", + }) + tables.AddTableRecord(&util.TableRecord{ + TableID: 2, + CaptureID: "capture-1", + }) + tables.AddTableRecord(&util.TableRecord{ + TableID: 3, + CaptureID: "capture-1", + }) + tables.AddTableRecord(&util.TableRecord{ + TableID: 4, + CaptureID: "capture-2", + }) + tables.AddTableRecord(&util.TableRecord{ + TableID: 5, + CaptureID: "capture-2", + }) + tables.AddTableRecord(&util.TableRecord{ + TableID: 6, + CaptureID: "capture-3", + }) + + mockCaptureInfos := map[model.CaptureID]*model.CaptureInfo{ + "capture-1": { + ID: "capture-1", + }, + "capture-2": { + ID: "capture-2", + }, + "capture-3": { + ID: "capture-3", + }, + } + + target, ok := balancer.FindTarget(tables, mockCaptureInfos) + require.True(t, ok) + require.Equal(t, "capture-3", target) +} + +func TestBalancerNoCaptureAvailable(t *testing.T) { + balancer := newTableNumberRebalancer(zap.L()) + tables := util.NewTableSet() + + _, ok := balancer.FindTarget(tables, map[model.CaptureID]*model.CaptureInfo{}) + require.False(t, ok) +} diff --git a/cdc/scheduler/util/sort_table_ids.go b/cdc/scheduler/util/sort_table_ids.go new file mode 100644 index 00000000000..052151e9d1f --- /dev/null +++ b/cdc/scheduler/util/sort_table_ids.go @@ -0,0 +1,27 @@ +// Copyright 2021 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package util + +import ( + "sort" + + "github.com/pingcap/ticdc/cdc/model" +) + +// SortTableIDs sorts a slice of table IDs in ascending order. +func SortTableIDs(tableIDs []model.TableID) { + sort.Slice(tableIDs, func(i, j int) bool { + return tableIDs[i] < tableIDs[j] + }) +} diff --git a/cdc/sink/codec/canal_flat.go b/cdc/sink/codec/canal_flat.go index 664ffe59022..5e7813b7fb9 100644 --- a/cdc/sink/codec/canal_flat.go +++ b/cdc/sink/codec/canal_flat.go @@ -16,7 +16,9 @@ package codec import ( "context" "encoding/json" + "sort" "strconv" + "strings" "time" "github.com/pingcap/errors" @@ -24,6 +26,7 @@ import ( "github.com/pingcap/ticdc/cdc/model" cerrors "github.com/pingcap/ticdc/pkg/errors" canal "github.com/pingcap/ticdc/proto/canal" + "github.com/pingcap/tidb/parser/types" "go.uber.org/zap" ) @@ -73,6 +76,11 @@ type canalFlatMessageInterface interface { getTikvTs() uint64 getSchema() *string getTable() *string + getCommitTs() uint64 + getQuery() string + getOld() map[string]interface{} + getData() map[string]interface{} + getMySQLType() map[string]string } // adapted from https://github.com/alibaba/canal/blob/master/protocol/src/main/java/com/alibaba/otter/canal/protocol/FlatMessage.java @@ -113,6 +121,27 @@ func (c *canalFlatMessage) getTable() *string { return &c.Table } +// for canalFlatMessage, we lost the commit-ts +func (c *canalFlatMessage) getCommitTs() uint64 { + return 0 +} + +func (c *canalFlatMessage) getQuery() string { + return c.Query +} + +func (c *canalFlatMessage) getOld() map[string]interface{} { + return c.Old[0] +} + +func (c *canalFlatMessage) getData() map[string]interface{} { + return c.Data[0] +} + +func (c *canalFlatMessage) getMySQLType() map[string]string { + return c.MySQLType +} + type tidbExtension struct { CommitTs uint64 `json:"commit-ts"` WatermarkTs uint64 `json:"watermark-ts"` @@ -139,6 +168,26 @@ func (c *canalFlatMessageWithTiDBExtension) getTable() *string { return &c.Table } +func (c *canalFlatMessageWithTiDBExtension) getCommitTs() uint64 { + return c.Extensions.CommitTs +} + +func (c *canalFlatMessageWithTiDBExtension) getQuery() string { + return c.Query +} + +func (c *canalFlatMessageWithTiDBExtension) getOld() map[string]interface{} { + return c.Old[0] +} + +func (c *canalFlatMessageWithTiDBExtension) getData() map[string]interface{} { + return c.Data[0] +} + +func (c *canalFlatMessageWithTiDBExtension) getMySQLType() map[string]string { + return c.MySQLType +} + func (c *CanalFlatEventBatchEncoder) newFlatMessageForDML(e *model.RowChangedEvent) (canalFlatMessageInterface, error) { eventType := convertRowEventType(e) header := c.builder.buildHeader(e.CommitTs, e.Table.Schema, e.Table.Table, eventType, 1) @@ -362,3 +411,152 @@ func (c *CanalFlatEventBatchEncoder) SetParams(params map[string]string) error { } return nil } + +// CanalFlatEventBatchDecoder decodes the byte into the original message. +type CanalFlatEventBatchDecoder struct { + data []byte + msg *MQMessage + enableTiDBExtension bool +} + +func NewCanalFlatEventBatchDecoder(data []byte, enableTiDBExtension bool) EventBatchDecoder { + return &CanalFlatEventBatchDecoder{ + data: data, + msg: nil, + enableTiDBExtension: enableTiDBExtension, + } +} + +// HasNext implements the EventBatchDecoder interface +func (b *CanalFlatEventBatchDecoder) HasNext() (model.MqMessageType, bool, error) { + if len(b.data) == 0 { + return model.MqMessageTypeUnknown, false, nil + } + msg := &MQMessage{} + if err := json.Unmarshal(b.data, msg); err != nil { + return model.MqMessageTypeUnknown, false, err + } + b.msg = msg + b.data = nil + if b.msg.Type == model.MqMessageTypeUnknown { + return model.MqMessageTypeUnknown, false, nil + } + return b.msg.Type, true, nil +} + +// NextRowChangedEvent implements the EventBatchDecoder interface +// `HasNext` should be called before this. +func (b *CanalFlatEventBatchDecoder) NextRowChangedEvent() (*model.RowChangedEvent, error) { + if b.msg == nil || b.msg.Type != model.MqMessageTypeRow { + return nil, cerrors.ErrCanalDecodeFailed.GenWithStack("not found row changed event message") + } + + var data canalFlatMessageInterface = &canalFlatMessage{} + if b.enableTiDBExtension { + data = &canalFlatMessageWithTiDBExtension{canalFlatMessage: &canalFlatMessage{}, Extensions: &tidbExtension{}} + } + + if err := json.Unmarshal(b.msg.Value, data); err != nil { + return nil, errors.Trace(err) + } + b.msg = nil + return canalFlatMessage2RowChangedEvent(data) +} + +// NextDDLEvent implements the EventBatchDecoder interface +// `HasNext` should be called before this. +func (b *CanalFlatEventBatchDecoder) NextDDLEvent() (*model.DDLEvent, error) { + if b.msg == nil || b.msg.Type != model.MqMessageTypeDDL { + return nil, cerrors.ErrCanalDecodeFailed.GenWithStack("not found ddl event message") + } + + var data canalFlatMessageInterface = &canalFlatMessage{} + if b.enableTiDBExtension { + data = &canalFlatMessageWithTiDBExtension{canalFlatMessage: &canalFlatMessage{}, Extensions: &tidbExtension{}} + } + + if err := json.Unmarshal(b.msg.Value, data); err != nil { + return nil, errors.Trace(err) + } + b.msg = nil + return canalFlatMessage2DDLEvent(data), nil +} + +// NextResolvedEvent implements the EventBatchDecoder interface +// `HasNext` should be called before this. +func (b *CanalFlatEventBatchDecoder) NextResolvedEvent() (uint64, error) { + if b.msg == nil || b.msg.Type != model.MqMessageTypeResolved { + return 0, cerrors.ErrCanalDecodeFailed.GenWithStack("not found resolved event message") + } + + message := &canalFlatMessageWithTiDBExtension{ + canalFlatMessage: &canalFlatMessage{}, + } + if err := json.Unmarshal(b.msg.Value, message); err != nil { + return 0, errors.Trace(err) + } + b.msg = nil + return message.Extensions.WatermarkTs, nil +} + +func canalFlatMessage2RowChangedEvent(flatMessage canalFlatMessageInterface) (*model.RowChangedEvent, error) { + result := new(model.RowChangedEvent) + result.CommitTs = flatMessage.getCommitTs() + result.Table = &model.TableName{ + Schema: *flatMessage.getSchema(), + Table: *flatMessage.getTable(), + } + + var err error + result.Columns, err = canalFlatJSONColumnMap2SinkColumns(flatMessage.getData(), flatMessage.getMySQLType()) + if err != nil { + return nil, err + } + result.PreColumns, err = canalFlatJSONColumnMap2SinkColumns(flatMessage.getOld(), flatMessage.getMySQLType()) + if err != nil { + return nil, err + } + + return result, nil +} + +func canalFlatJSONColumnMap2SinkColumns(cols map[string]interface{}, mysqlType map[string]string) ([]*model.Column, error) { + result := make([]*model.Column, 0, len(cols)) + for name, value := range cols { + typeStr, ok := mysqlType[name] + if !ok { + // this should not happen, else we have to check encoding for mysqlType. + return nil, cerrors.ErrCanalDecodeFailed.GenWithStack( + "mysql type does not found, column: %+v, mysqlType: %+v", name, mysqlType) + } + tp := types.StrToType(typeStr) + if !ok { + return nil, cerrors.ErrCanalDecodeFailed.GenWithStack( + "mysql type does not found, column: %+v, type: %+v", name, tp) + } + col := NewColumn(value, tp).ToSinkColumn(name) + result = append(result, col) + } + if len(result) == 0 { + return nil, nil + } + sort.Slice(result, func(i, j int) bool { + return strings.Compare(result[i].Name, result[j].Name) > 0 + }) + return result, nil +} + +func canalFlatMessage2DDLEvent(flatDDL canalFlatMessageInterface) *model.DDLEvent { + result := new(model.DDLEvent) + // we lost the startTs from kafka message + result.CommitTs = flatDDL.getCommitTs() + + result.TableInfo = new(model.SimpleTableInfo) + result.TableInfo.Schema = *flatDDL.getSchema() + result.TableInfo.Table = *flatDDL.getTable() + + // we lost DDL type from canal flat json format, only got the DDL SQL. + result.Query = flatDDL.getQuery() + + return result +} diff --git a/cdc/sink/codec/canal_flat_test.go b/cdc/sink/codec/canal_flat_test.go index f031fc7ff32..aecf7fe35d5 100644 --- a/cdc/sink/codec/canal_flat_test.go +++ b/cdc/sink/codec/canal_flat_test.go @@ -44,37 +44,45 @@ func (s *canalFlatSuite) TestNewCanalFlatMessageFromDML(c *check.C) { c.Assert(msg.Table, check.Equals, "person") c.Assert(msg.IsDDL, check.IsFalse) c.Assert(msg.SQLType, check.DeepEquals, map[string]int32{ - "id": int32(JavaSQLTypeBIGINT), - "name": int32(JavaSQLTypeVARCHAR), - "tiny": int32(JavaSQLTypeSMALLINT), - "comment": int32(JavaSQLTypeVARCHAR), - "blob": int32(JavaSQLTypeBLOB), + "id": int32(JavaSQLTypeBIGINT), + "name": int32(JavaSQLTypeVARCHAR), + "tiny": int32(JavaSQLTypeSMALLINT), + "comment": int32(JavaSQLTypeVARCHAR), + "blob": int32(JavaSQLTypeBLOB), + "binaryString": int32(JavaSQLTypeCHAR), + "binaryBlob": int32(JavaSQLTypeVARCHAR), }) c.Assert(msg.MySQLType, check.DeepEquals, map[string]string{ - "id": "int", - "name": "varchar", - "tiny": "tinyint", - "comment": "text", - "blob": "blob", + "id": "int", + "name": "varchar", + "tiny": "tinyint", + "comment": "text", + "blob": "blob", + "binaryString": "binary", + "binaryBlob": "varbinary", }) encodedBytes, err := charmap.ISO8859_1.NewDecoder().Bytes([]byte("测试blob")) c.Assert(err, check.IsNil) c.Assert(msg.Data, check.DeepEquals, []map[string]interface{}{ { - "id": "1", - "name": "Bob", - "tiny": "255", - "comment": "测试", - "blob": string(encodedBytes), + "id": "1", + "name": "Bob", + "tiny": "255", + "comment": "测试", + "blob": string(encodedBytes), + "binaryString": "Chengdu International Airport", + "binaryBlob": "你好,世界", }, }) c.Assert(msg.Old, check.DeepEquals, []map[string]interface{}{ { - "id": "1", - "name": "Alice", - "tiny": "255", - "comment": "测试", - "blob": string(encodedBytes), + "id": "1", + "name": "Alice", + "tiny": "255", + "comment": "测试", + "blob": string(encodedBytes), + "binaryString": "Chengdu International Airport", + "binaryBlob": "你好,世界", }, }) @@ -90,6 +98,84 @@ func (s *canalFlatSuite) TestNewCanalFlatMessageFromDML(c *check.C) { c.Assert(withExtension.Extensions.CommitTs, check.Equals, testCaseUpdate.CommitTs) } +func (s *canalFlatSuite) TestNewCanalFlatEventBatchDecoder4RowMessage(c *check.C) { + defer testleak.AfterTest(c)() + + encodedBytes, err := charmap.ISO8859_1.NewDecoder().Bytes([]byte("测试blob")) + c.Assert(err, check.IsNil) + expected := map[string]interface{}{ + "id": "1", + "name": "Bob", + "tiny": "255", + "comment": "测试", + "blob": string(encodedBytes), + "binaryString": "Chengdu International Airport", + "binaryBlob": "你好,世界", + } + + for _, encodeEnable := range []bool{false, true} { + encoder := &CanalFlatEventBatchEncoder{builder: NewCanalEntryBuilder(), enableTiDBExtension: encodeEnable} + c.Assert(encoder, check.NotNil) + + result, err := encoder.AppendRowChangedEvent(testCaseUpdate) + c.Assert(err, check.IsNil) + c.Assert(result, check.Equals, EncoderNoOperation) + + result, err = encoder.AppendResolvedEvent(417318403368288260) + c.Assert(err, check.IsNil) + c.Assert(result, check.Equals, EncoderNeedAsyncWrite) + + mqMessages := encoder.Build() + c.Assert(len(mqMessages), check.Equals, 1) + + rawBytes, err := json.Marshal(mqMessages[0]) + c.Assert(err, check.IsNil) + + for _, decodeEnable := range []bool{false, true} { + decoder := NewCanalFlatEventBatchDecoder(rawBytes, decodeEnable) + + ty, hasNext, err := decoder.HasNext() + c.Assert(err, check.IsNil) + c.Assert(hasNext, check.IsTrue) + c.Assert(ty, check.Equals, model.MqMessageTypeRow) + + consumed, err := decoder.NextRowChangedEvent() + c.Assert(err, check.IsNil) + + c.Assert(consumed.Table, check.DeepEquals, testCaseUpdate.Table) + if encodeEnable && decodeEnable { + c.Assert(consumed.CommitTs, check.Equals, testCaseUpdate.CommitTs) + } else { + c.Assert(consumed.CommitTs, check.Equals, uint64(0)) + } + + for _, col := range consumed.Columns { + value, ok := expected[col.Name] + c.Assert(ok, check.IsTrue) + + if val, ok := col.Value.([]byte); ok { + c.Assert(string(val), check.Equals, value) + } else { + c.Assert(col.Value, check.Equals, value) + } + + for _, item := range testCaseUpdate.Columns { + if item.Name == col.Name { + c.Assert(col.Type, check.Equals, item.Type) + } + } + } + + _, hasNext, _ = decoder.HasNext() + c.Assert(hasNext, check.IsFalse) + + consumed, err = decoder.NextRowChangedEvent() + c.Assert(err, check.NotNil) + c.Assert(consumed, check.IsNil) + } + } +} + func (s *canalFlatSuite) TestNewCanalFlatMessageFromDDL(c *check.C) { defer testleak.AfterTest(c)() encoder := &CanalFlatEventBatchEncoder{builder: NewCanalEntryBuilder()} @@ -121,6 +207,51 @@ func (s *canalFlatSuite) TestNewCanalFlatMessageFromDDL(c *check.C) { c.Assert(withExtension.Extensions.CommitTs, check.Equals, testCaseDdl.CommitTs) } +func (s *canalFlatSuite) TestNewCanalFlatEventBatchDecoder4DDLMessage(c *check.C) { + defer testleak.AfterTest(c)() + for _, encodeEnable := range []bool{false, true} { + encoder := &CanalFlatEventBatchEncoder{builder: NewCanalEntryBuilder(), enableTiDBExtension: encodeEnable} + c.Assert(encoder, check.NotNil) + + result, err := encoder.EncodeDDLEvent(testCaseDdl) + c.Assert(err, check.IsNil) + c.Assert(result, check.NotNil) + + rawBytes, err := json.Marshal(result) + c.Assert(err, check.IsNil) + + for _, decodeEnable := range []bool{false, true} { + decoder := NewCanalFlatEventBatchDecoder(rawBytes, decodeEnable) + + ty, hasNext, err := decoder.HasNext() + c.Assert(err, check.IsNil) + c.Assert(hasNext, check.IsTrue) + c.Assert(ty, check.Equals, model.MqMessageTypeDDL) + + consumed, err := decoder.NextDDLEvent() + c.Assert(err, check.IsNil) + + if encodeEnable && decodeEnable { + c.Assert(consumed.CommitTs, check.Equals, testCaseDdl.CommitTs) + } else { + c.Assert(consumed.CommitTs, check.Equals, uint64(0)) + } + + c.Assert(consumed.TableInfo, check.DeepEquals, testCaseDdl.TableInfo) + c.Assert(consumed.Query, check.Equals, testCaseDdl.Query) + + ty, hasNext, err = decoder.HasNext() + c.Assert(err, check.IsNil) + c.Assert(hasNext, check.IsFalse) + c.Assert(ty, check.Equals, model.MqMessageTypeUnknown) + + consumed, err = decoder.NextDDLEvent() + c.Assert(err, check.NotNil) + c.Assert(consumed, check.IsNil) + } + } +} + func (s *canalFlatSuite) TestBatching(c *check.C) { defer testleak.AfterTest(c)() encoder := &CanalFlatEventBatchEncoder{builder: NewCanalEntryBuilder()} @@ -167,17 +298,43 @@ func (s *canalFlatSuite) TestBatching(c *check.C) { func (s *canalFlatSuite) TestEncodeCheckpointEvent(c *check.C) { defer testleak.AfterTest(c)() - encoder := &CanalFlatEventBatchEncoder{builder: NewCanalEntryBuilder(), enableTiDBExtension: false} - c.Assert(encoder, check.NotNil) + var watermark uint64 = 2333 + for _, enable := range []bool{false, true} { + encoder := &CanalFlatEventBatchEncoder{builder: NewCanalEntryBuilder(), enableTiDBExtension: enable} + c.Assert(encoder, check.NotNil) - msg, err := encoder.EncodeCheckpointEvent(2333) - c.Assert(err, check.IsNil) - c.Assert(msg, check.IsNil) + msg, err := encoder.EncodeCheckpointEvent(watermark) + c.Assert(err, check.IsNil) + if enable { + c.Assert(msg, check.NotNil) + } else { + c.Assert(msg, check.IsNil) + } - encoder.enableTiDBExtension = true - msg, err = encoder.EncodeCheckpointEvent(2333) - c.Assert(err, check.IsNil) - c.Assert(msg, check.NotNil) + rawBytes, err := json.Marshal(msg) + c.Assert(err, check.IsNil) + c.Assert(rawBytes, check.NotNil) + + decoder := NewCanalFlatEventBatchDecoder(rawBytes, enable) + + ty, hasNext, err := decoder.HasNext() + c.Assert(err, check.IsNil) + if enable { + c.Assert(hasNext, check.IsTrue) + c.Assert(ty, check.Equals, model.MqMessageTypeResolved) + consumed, err := decoder.NextResolvedEvent() + c.Assert(err, check.IsNil) + c.Assert(consumed, check.Equals, watermark) + } else { + c.Assert(hasNext, check.IsFalse) + c.Assert(ty, check.Equals, model.MqMessageTypeUnknown) + } + + ty, hasNext, err = decoder.HasNext() + c.Assert(err, check.IsNil) + c.Assert(hasNext, check.IsFalse) + c.Assert(ty, check.Equals, model.MqMessageTypeUnknown) + } } var testCaseUpdate = &model.RowChangedEvent{ @@ -192,6 +349,8 @@ var testCaseUpdate = &model.RowChangedEvent{ {Name: "tiny", Type: mysql.TypeTiny, Value: 255}, {Name: "comment", Type: mysql.TypeBlob, Value: []byte("测试")}, {Name: "blob", Type: mysql.TypeBlob, Value: []byte("测试blob"), Flag: model.BinaryFlag}, + {Name: "binaryString", Type: mysql.TypeString, Value: "Chengdu International Airport", Flag: model.BinaryFlag}, + {Name: "binaryBlob", Type: mysql.TypeVarchar, Value: []byte("你好,世界"), Flag: model.BinaryFlag}, }, PreColumns: []*model.Column{ {Name: "id", Type: mysql.TypeLong, Flag: model.HandleKeyFlag, Value: 1}, @@ -199,6 +358,8 @@ var testCaseUpdate = &model.RowChangedEvent{ {Name: "tiny", Type: mysql.TypeTiny, Value: 255}, {Name: "comment", Type: mysql.TypeBlob, Value: []byte("测试")}, {Name: "blob", Type: mysql.TypeBlob, Value: []byte("测试blob"), Flag: model.BinaryFlag}, + {Name: "binaryString", Type: mysql.TypeString, Value: "Chengdu International Airport", Flag: model.BinaryFlag}, + {Name: "binaryBlob", Type: mysql.TypeVarchar, Value: []byte("你好,世界"), Flag: model.BinaryFlag}, }, } diff --git a/cdc/sink/codec/json.go b/cdc/sink/codec/json.go index 723ef7853dd..e017f29ceeb 100644 --- a/cdc/sink/codec/json.go +++ b/cdc/sink/codec/json.go @@ -52,6 +52,13 @@ type column struct { Value interface{} `json:"v"` } +func NewColumn(value interface{}, tp byte) *column { + return &column{ + Value: value, + Type: tp, + } +} + func (c *column) FromSinkColumn(col *model.Column) { c.Type = col.Type c.Flag = col.Flag diff --git a/cdc/sink/producer/kafka/kafka.go b/cdc/sink/producer/kafka/kafka.go index 5cfd5b69ef8..4f017fe58c6 100644 --- a/cdc/sink/producer/kafka/kafka.go +++ b/cdc/sink/producer/kafka/kafka.go @@ -457,18 +457,20 @@ func topicPreProcess(topic string, protocol codec.Protocol, config *Config, sara // when try to create the topic, we don't know how to set the `max.message.bytes` for the topic. // Kafka would create the topic with broker's `message.max.bytes`, - // we have to make sure it's not greater than `max-message-bytes` - brokerMessageMaxBytes, err := getBrokerMessageMaxBytes(admin) - if err != nil { - log.Warn("TiCDC cannot find `message.max.bytes` from broker's configuration") - return errors.Trace(err) - } + // we have to make sure it's not greater than `max-message-bytes` for the default open protocol & craft protocol. + if protocol == codec.ProtocolDefault || protocol == codec.ProtocolCraft { + brokerMessageMaxBytes, err := getBrokerMessageMaxBytes(admin) + if err != nil { + log.Warn("TiCDC cannot find `message.max.bytes` from broker's configuration") + return errors.Trace(err) + } - if brokerMessageMaxBytes < config.MaxMessageBytes { - return cerror.ErrKafkaInvalidConfig.GenWithStack( - "broker's message.max.bytes(%d) less than max-message-bytes(%d)"+ - "Please make sure `max-message-bytes` not greater than broker's `message.max.bytes`", - brokerMessageMaxBytes, config.MaxMessageBytes) + if brokerMessageMaxBytes < config.MaxMessageBytes { + return cerror.ErrKafkaInvalidConfig.GenWithStack( + "broker's message.max.bytes(%d) less than max-message-bytes(%d)"+ + "Please make sure `max-message-bytes` not greater than broker's `message.max.bytes`", + brokerMessageMaxBytes, config.MaxMessageBytes) + } } // topic not created yet, and user does not specify the `partition-num` in the sink uri. diff --git a/cdc/sorter/leveldb/buffer.go b/cdc/sorter/leveldb/buffer.go new file mode 100644 index 00000000000..fe8d9330b9b --- /dev/null +++ b/cdc/sorter/leveldb/buffer.go @@ -0,0 +1,118 @@ +// Copyright 2021 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package leveldb + +import ( + "github.com/pingcap/log" + "github.com/pingcap/ticdc/cdc/model" + "github.com/pingcap/ticdc/cdc/sorter/leveldb/message" + "go.uber.org/zap" +) + +// outputBuffer a struct that facilitate leveldb table sorter. +// TODO remove unused lint +//nolint:unused +type outputBuffer struct { + // A slice of keys need to be deleted. + deleteKeys []message.Key + // A slice of resolved events that have the same commit ts. + resolvedEvents []*model.PolymorphicEvent + + advisedCapacity int +} + +// TODO remove unused lint +//nolint:unused +func newOutputBuffer(advisedCapacity int) *outputBuffer { + return &outputBuffer{ + deleteKeys: make([]message.Key, 0, advisedCapacity), + resolvedEvents: make([]*model.PolymorphicEvent, 0, advisedCapacity), + advisedCapacity: advisedCapacity, + } +} + +// maybeShrink try to shrink slices to the advised capacity. +// TODO remove unused lint +//nolint:unused +func (b *outputBuffer) maybeShrink() { + if len(b.deleteKeys) < b.advisedCapacity { + if cap(b.deleteKeys) > b.advisedCapacity { + buf := make([]message.Key, 0, b.advisedCapacity) + buf = append(buf, b.deleteKeys...) + b.deleteKeys = buf + } + } + if len(b.resolvedEvents) < b.advisedCapacity { + if cap(b.resolvedEvents) > b.advisedCapacity { + buf := make([]*model.PolymorphicEvent, 0, b.advisedCapacity) + buf = append(buf, b.resolvedEvents...) + b.resolvedEvents = buf + } + } +} + +// In place left shift resolved events slice. After the call, +// `index` will become the first element in the slice +// TODO remove unused lint +//nolint:unused +func (b *outputBuffer) shiftResolvedEvents(index int) { + if index > len(b.resolvedEvents) { + log.Panic("index out of range", zap.Int("len", len(b.resolvedEvents))) + } + if index != 0 { + length := len(b.resolvedEvents) + for left, right := 0, index; right < length; right++ { + b.resolvedEvents[left] = b.resolvedEvents[right] + // Set original element to nil to help GC. + b.resolvedEvents[right] = nil + left++ + } + b.resolvedEvents = b.resolvedEvents[:length-index] + } +} + +// appendResolvedEvent appends resolved events to the buffer. +// TODO remove unused lint +//nolint:unused +func (b *outputBuffer) appendResolvedEvent(event *model.PolymorphicEvent) { + if len(b.resolvedEvents) > 0 { + if b.resolvedEvents[0].CRTs != event.CRTs { + log.Panic("commit ts must be equal", + zap.Uint64("newCommitTs", event.CRTs), + zap.Uint64("commitTs", b.resolvedEvents[0].CRTs)) + } + } + b.resolvedEvents = append(b.resolvedEvents, event) +} + +// appendDeleteKey appends to-be-deleted keys to the buffer. +// TODO remove unused lint +//nolint:unused +func (b *outputBuffer) appendDeleteKey(key message.Key) { + b.deleteKeys = append(b.deleteKeys, key) +} + +// resetDeleteKey reset deleteKeys to a zero len slice. +// TODO remove unused lint +//nolint:unused +func (b *outputBuffer) resetDeleteKey() { + b.deleteKeys = b.deleteKeys[:0] +} + +// len returns the length of resolvedEvents and delete keys. +// TODO remove unused lint +//nolint:unused +func (b *outputBuffer) len() (int, int) { + return len(b.resolvedEvents), len(b.deleteKeys) +} diff --git a/cdc/sorter/leveldb/buffer_test.go b/cdc/sorter/leveldb/buffer_test.go new file mode 100644 index 00000000000..e01f0e0d267 --- /dev/null +++ b/cdc/sorter/leveldb/buffer_test.go @@ -0,0 +1,91 @@ +// Copyright 2021 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package leveldb + +import ( + "testing" + + "github.com/pingcap/ticdc/cdc/model" + "github.com/pingcap/ticdc/cdc/sorter/leveldb/message" + "github.com/stretchr/testify/require" +) + +func TestOutputBufferMaybeShrink(t *testing.T) { + t.Parallel() + advisedCapacity := 4 + buf := newOutputBuffer(advisedCapacity) + require.Equal(t, 0, len(buf.resolvedEvents)) + require.Equal(t, 0, len(buf.deleteKeys)) + require.Equal(t, advisedCapacity, cap(buf.resolvedEvents)) + require.Equal(t, advisedCapacity, cap(buf.deleteKeys)) + + // len == cap == advisedCapacity. + buf.resolvedEvents = make([]*model.PolymorphicEvent, advisedCapacity) + buf.resolvedEvents[0] = model.NewResolvedPolymorphicEvent(0, 1) + buf.deleteKeys = make([]message.Key, advisedCapacity) + buf.deleteKeys[0] = message.Key([]byte{1}) + resolvedEvents := append([]*model.PolymorphicEvent{}, buf.resolvedEvents...) + deleteKeys := append([]message.Key{}, buf.deleteKeys...) + + buf.maybeShrink() + require.Equal(t, advisedCapacity, len(buf.resolvedEvents)) + require.Equal(t, advisedCapacity, cap(buf.resolvedEvents)) + require.EqualValues(t, resolvedEvents, buf.resolvedEvents) + require.EqualValues(t, deleteKeys, buf.deleteKeys) + + // len < cap == 2*advisedCapacity. + buf.resolvedEvents = make([]*model.PolymorphicEvent, 2*advisedCapacity-1, 2*advisedCapacity) + buf.resolvedEvents[0] = model.NewResolvedPolymorphicEvent(0, 1) + buf.deleteKeys = make([]message.Key, 2*advisedCapacity-1, 2*advisedCapacity) + buf.deleteKeys[0] = message.Key([]byte{1}) + resolvedEvents = append([]*model.PolymorphicEvent{}, buf.resolvedEvents...) + deleteKeys = append([]message.Key{}, buf.deleteKeys...) + + buf.maybeShrink() + require.Equal(t, 2*advisedCapacity-1, len(buf.resolvedEvents)) + require.Equal(t, 2*advisedCapacity-1, len(buf.deleteKeys)) + require.EqualValues(t, resolvedEvents, buf.resolvedEvents) + require.EqualValues(t, deleteKeys, buf.deleteKeys) + + // len < cap/2 == advisedCapacity. + buf.resolvedEvents = make([]*model.PolymorphicEvent, advisedCapacity-1, 2*advisedCapacity) + buf.resolvedEvents[0] = model.NewResolvedPolymorphicEvent(0, 1) + buf.deleteKeys = make([]message.Key, advisedCapacity-1, 2*advisedCapacity) + buf.deleteKeys[0] = message.Key([]byte{1}) + resolvedEvents = append([]*model.PolymorphicEvent{}, buf.resolvedEvents...) + deleteKeys = append([]message.Key{}, buf.deleteKeys...) + + buf.maybeShrink() + require.Equal(t, advisedCapacity-1, len(buf.resolvedEvents)) + require.Equal(t, advisedCapacity-1, len(buf.deleteKeys)) + require.EqualValues(t, resolvedEvents, buf.resolvedEvents) + require.EqualValues(t, deleteKeys, buf.deleteKeys) +} + +func TestOutputBufferShiftResolvedEvents(t *testing.T) { + t.Parallel() + advisedCapacity := 64 + buf := newOutputBuffer(advisedCapacity) + + events := make([]*model.PolymorphicEvent, advisedCapacity) + for i := range events { + events[i] = &model.PolymorphicEvent{CRTs: uint64(1)} + } + + for i := 0; i < advisedCapacity; i++ { + buf.resolvedEvents = append([]*model.PolymorphicEvent{}, events...) + buf.shiftResolvedEvents(i) + require.EqualValues(t, buf.resolvedEvents, events[i:]) + } +} diff --git a/cdc/sorter/leveldb/system/metrics.go b/cdc/sorter/leveldb/system/metrics.go new file mode 100644 index 00000000000..d51752eca81 --- /dev/null +++ b/cdc/sorter/leveldb/system/metrics.go @@ -0,0 +1,80 @@ +// Copyright 2021 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package system + +import ( + "github.com/prometheus/client_golang/prometheus" +) + +var ( + sorterDBWriteBytes = prometheus.NewGaugeVec(prometheus.GaugeOpts{ + Namespace: "ticdc", + Subsystem: "sorter", + Name: "leveldb_write_bytes_total", + Help: "The total number of write bytes by the leveldb", + }, []string{"capture", "id"}) + + sorterDBReadBytes = prometheus.NewGaugeVec(prometheus.GaugeOpts{ + Namespace: "ticdc", + Subsystem: "sorter", + Name: "leveldb_read_bytes_total", + Help: "The total number of read bytes by the leveldb", + }, []string{"capture", "id"}) + + sorterDBSnapshotGauge = prometheus.NewGaugeVec(prometheus.GaugeOpts{ + Namespace: "ticdc", + Subsystem: "sorter", + Name: "leveldb_snapshot_count_gauge", + Help: "The number of snapshot by the sorter", + }, []string{"capture", "id"}) + + sorterDBIteratorGauge = prometheus.NewGaugeVec(prometheus.GaugeOpts{ + Namespace: "ticdc", + Subsystem: "sorter", + Name: "leveldb_iterator_count_gauge", + Help: "The number of iterator by the sorter", + }, []string{"capture", "id"}) + + sorterDBLevelCount = prometheus.NewGaugeVec(prometheus.GaugeOpts{ + Namespace: "ticdc", + Subsystem: "sorter", + Name: "leveldb_level_count", + Help: "The number of files in each level by the sorter", + }, []string{"capture", "level", "id"}) + + sorterDBWriteDelayDuration = prometheus.NewGaugeVec(prometheus.GaugeOpts{ + Namespace: "ticdc", + Subsystem: "sorter", + Name: "leveldb_write_delay_seconds", + Help: "The duration of leveldb write delay seconds", + }, []string{"capture", "id"}) + + sorterDBWriteDelayCount = prometheus.NewGaugeVec(prometheus.GaugeOpts{ + Namespace: "ticdc", + Subsystem: "sorter", + Name: "leveldb_write_delay_total", + Help: "The total number of leveldb delay", + }, []string{"capture", "id"}) +) + +// InitMetrics registers all metrics in this file +func InitMetrics(registry *prometheus.Registry) { + registry.MustRegister(sorterDBSnapshotGauge) + registry.MustRegister(sorterDBIteratorGauge) + registry.MustRegister(sorterDBLevelCount) + registry.MustRegister(sorterDBWriteBytes) + registry.MustRegister(sorterDBReadBytes) + registry.MustRegister(sorterDBWriteDelayDuration) + registry.MustRegister(sorterDBWriteDelayCount) +} diff --git a/cdc/sorter/leveldb/system/system.go b/cdc/sorter/leveldb/system/system.go new file mode 100644 index 00000000000..f95b64297ce --- /dev/null +++ b/cdc/sorter/leveldb/system/system.go @@ -0,0 +1,257 @@ +// Copyright 2021 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package system + +import ( + "context" + "encoding/binary" + "hash/fnv" + "strconv" + "sync" + "time" + + "github.com/pingcap/errors" + "github.com/pingcap/log" + "github.com/pingcap/ticdc/cdc/sorter" + lsorter "github.com/pingcap/ticdc/cdc/sorter/leveldb" + "github.com/pingcap/ticdc/pkg/actor" + "github.com/pingcap/ticdc/pkg/actor/message" + "github.com/pingcap/ticdc/pkg/config" + cerrors "github.com/pingcap/ticdc/pkg/errors" + "github.com/syndtr/goleveldb/leveldb" + "go.uber.org/zap" +) + +// The interval of collecting leveldb metrics. +const defaultMetricInterval = 15 * time.Second + +// State of a system. +type sysState int + +const ( + sysStateInit sysState = iota + sysStateStarted + sysStateStopped +) + +// System manages leveldb sorter resource. +type System struct { + dbs []*leveldb.DB + dbSystem *actor.System + dbRouter *actor.Router + cleanSystem *actor.System + cleanRouter *actor.Router + cfg *config.SorterConfig + closedCh chan struct{} + closedWg *sync.WaitGroup + + state sysState + stateMu *sync.Mutex +} + +// NewSystem returns a system. +func NewSystem(cfg *config.SorterConfig) *System { + dbSystem, dbRouter := actor.NewSystemBuilder("sorter"). + WorkerNumber(cfg.LevelDB.Count).Build() + cleanSystem, cleanRouter := actor.NewSystemBuilder("cleaner"). + WorkerNumber(cfg.LevelDB.Count).Build() + return &System{ + dbSystem: dbSystem, + dbRouter: dbRouter, + cleanSystem: cleanSystem, + cleanRouter: cleanRouter, + cfg: cfg, + closedCh: make(chan struct{}), + closedWg: new(sync.WaitGroup), + state: sysStateInit, + stateMu: new(sync.Mutex), + } +} + +// ActorID returns an ActorID correspond with tableID. +func (s *System) ActorID(tableID uint64) actor.ID { + h := fnv.New64() + b := [8]byte{} + binary.LittleEndian.PutUint64(b[:], tableID) + h.Write(b[:]) + return actor.ID(h.Sum64() % uint64(s.cfg.LevelDB.Count)) +} + +// Router returns leveldb actors router. +func (s *System) Router() *actor.Router { + return s.dbRouter +} + +// CleanerRouter returns cleaner actors router. +func (s *System) CleanerRouter() *actor.Router { + return s.cleanRouter +} + +// broadcase messages to actors in the router. +// Caveats it may lose messages quietly. +func (s *System) broadcast(ctx context.Context, router *actor.Router, msg message.Message) { + dbCount := s.cfg.LevelDB.Count + for id := 0; id < dbCount; id++ { + err := router.SendB(ctx, actor.ID(id), msg) + if err != nil { + log.Warn("broadcast message failed", + zap.Int("ID", id), zap.Any("message", msg)) + } + } +} + +// Start starts a system. +func (s *System) Start(ctx context.Context) error { + s.stateMu.Lock() + defer s.stateMu.Unlock() + if s.state == sysStateStarted { + // Already started. + return nil + } else if s.state == sysStateStopped { + return cerrors.ErrStartAStoppedLevelDBSystem.GenWithStackByArgs() + } + s.state = sysStateStarted + + s.dbSystem.Start(ctx) + s.cleanSystem.Start(ctx) + captureAddr := config.GetGlobalServerConfig().AdvertiseAddr + dbCount := s.cfg.LevelDB.Count + for id := 0; id < dbCount; id++ { + // Open leveldb. + db, err := lsorter.OpenDB(ctx, id, s.cfg) + if err != nil { + return errors.Trace(err) + } + s.dbs = append(s.dbs, db) + // Create and spawn leveldb actor. + dbac, dbmb, err := lsorter.NewLevelDBActor( + ctx, id, db, s.cfg, s.closedWg, captureAddr) + if err != nil { + return errors.Trace(err) + } + err = s.dbSystem.Spawn(dbmb, dbac) + if err != nil { + return errors.Trace(err) + } + // Create and spawn cleaner actor. + clac, clmb, err := lsorter.NewCleanerActor( + id, db, s.cleanRouter, s.cfg, s.closedWg) + if err != nil { + return errors.Trace(err) + } + err = s.cleanSystem.Spawn(clmb, clac) + if err != nil { + return errors.Trace(err) + } + } + s.closedWg.Add(1) + go func() { + defer s.closedWg.Done() + metricsTimer := time.NewTimer(defaultMetricInterval) + defer metricsTimer.Stop() + for { + select { + case <-ctx.Done(): + return + case <-s.closedCh: + return + case <-metricsTimer.C: + collectMetrics(s.dbs, captureAddr) + metricsTimer.Reset(defaultMetricInterval) + } + } + }() + return nil +} + +// Stop stops a system. +func (s *System) Stop() error { + s.stateMu.Lock() + defer s.stateMu.Unlock() + switch s.state { + case sysStateStopped: + // Already stopped. + return nil + case sysStateInit: + // Not started. + return nil + } + s.state = sysStateStopped + + // TODO caller should pass context. + deadline := time.Now().Add(1 * time.Second) + ctx, cancel := context.WithDeadline(context.Background(), deadline) + defer cancel() + // Close actors + s.broadcast(ctx, s.dbRouter, message.StopMessage()) + s.broadcast(ctx, s.cleanRouter, message.StopMessage()) + // Close metrics goroutine. + close(s.closedCh) + // Wait actors and metrics goroutine. + s.closedWg.Wait() + + // Stop systems. + err := s.dbSystem.Stop() + if err != nil { + return errors.Trace(err) + } + err = s.cleanSystem.Stop() + if err != nil { + return errors.Trace(err) + } + + // Close leveldbs. + for _, db := range s.dbs { + err = db.Close() + if err != nil { + log.Warn("leveldb close error", zap.Error(err)) + } + } + return nil +} + +func collectMetrics(dbs []*leveldb.DB, captureAddr string) { + for i := range dbs { + db := dbs[i] + stats := leveldb.DBStats{} + err := db.Stats(&stats) + if err != nil { + log.Panic("leveldb error", zap.Error(err), zap.Int("db", i)) + } + id := strconv.Itoa(i) + sorter.OnDiskDataSizeGauge. + WithLabelValues(captureAddr, id).Set(float64(stats.LevelSizes.Sum())) + sorter.InMemoryDataSizeGauge. + WithLabelValues(captureAddr, id).Set(float64(stats.BlockCacheSize)) + sorter.OpenFileCountGauge. + WithLabelValues(captureAddr, id).Set(float64(stats.OpenedTablesCount)) + sorterDBSnapshotGauge. + WithLabelValues(captureAddr, id).Set(float64(stats.AliveSnapshots)) + sorterDBIteratorGauge. + WithLabelValues(captureAddr, id).Set(float64(stats.AliveIterators)) + sorterDBReadBytes. + WithLabelValues(captureAddr, id).Set(float64(stats.IORead)) + sorterDBWriteBytes. + WithLabelValues(captureAddr, id).Set(float64(stats.IOWrite)) + sorterDBWriteDelayCount. + WithLabelValues(captureAddr, id).Set(float64(stats.WriteDelayCount)) + sorterDBWriteDelayDuration. + WithLabelValues(captureAddr, id).Set(stats.WriteDelayDuration.Seconds()) + metricLevelCount := sorterDBLevelCount. + MustCurryWith(map[string]string{"capture": captureAddr, "id": id}) + for level, count := range stats.LevelTablesCounts { + metricLevelCount.WithLabelValues(strconv.Itoa(level)).Set(float64(count)) + } + } +} diff --git a/cdc/sorter/leveldb/system/system_test.go b/cdc/sorter/leveldb/system/system_test.go new file mode 100644 index 00000000000..fb64e6e5b46 --- /dev/null +++ b/cdc/sorter/leveldb/system/system_test.go @@ -0,0 +1,78 @@ +// Copyright 2021 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package system + +import ( + "context" + "testing" + + "github.com/pingcap/ticdc/pkg/config" + "github.com/stretchr/testify/require" +) + +func TestSystemStartStop(t *testing.T) { + t.Parallel() + ctx := context.Background() + cfg := config.GetDefaultServerConfig().Clone().Sorter + cfg.SortDir = t.TempDir() + cfg.LevelDB.Count = 1 + + sys := NewSystem(cfg) + require.Nil(t, sys.Start(ctx)) + require.Nil(t, sys.Stop()) + + // Close it again. + require.Nil(t, sys.Stop()) + // Start a closed system. + require.Error(t, sys.Start(ctx)) +} + +func TestSystemStopUnstarted(t *testing.T) { + t.Parallel() + cfg := config.GetDefaultServerConfig().Clone().Sorter + cfg.SortDir = t.TempDir() + cfg.LevelDB.Count = 1 + + sys := NewSystem(cfg) + require.Nil(t, sys.Stop()) +} + +func TestCollectMetrics(t *testing.T) { + t.Parallel() + ctx := context.Background() + cfg := config.GetDefaultServerConfig().Clone().Sorter + cfg.SortDir = t.TempDir() + cfg.LevelDB.Count = 2 + + sys := NewSystem(cfg) + require.Nil(t, sys.Start(ctx)) + collectMetrics(sys.dbs, "") + require.Nil(t, sys.Stop()) +} + +func TestActorID(t *testing.T) { + t.Parallel() + ctx := context.Background() + cfg := config.GetDefaultServerConfig().Clone().Sorter + cfg.SortDir = t.TempDir() + cfg.LevelDB.Count = 2 + + sys := NewSystem(cfg) + require.Nil(t, sys.Start(ctx)) + id1 := sys.ActorID(1) + id2 := sys.ActorID(1) + // tableID to actor ID must be deterministic. + require.Equal(t, id1, id2) + require.Nil(t, sys.Stop()) +} diff --git a/cdc/sorter/memory/entry_sorter.go b/cdc/sorter/memory/entry_sorter.go index f8132088dcf..5d1a8a2789c 100644 --- a/cdc/sorter/memory/entry_sorter.go +++ b/cdc/sorter/memory/entry_sorter.go @@ -23,6 +23,7 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/log" "github.com/pingcap/ticdc/cdc/model" + cerror "github.com/pingcap/ticdc/pkg/errors" "github.com/pingcap/ticdc/pkg/notify" "github.com/pingcap/ticdc/pkg/util" "go.uber.org/zap" @@ -59,35 +60,6 @@ func (es *EntrySorter) Run(ctx context.Context) error { metricEntrySorterSortDuration := entrySorterSortDuration.WithLabelValues(captureAddr, changefeedID, tableName) metricEntrySorterMergeDuration := entrySorterMergeDuration.WithLabelValues(captureAddr, changefeedID, tableName) - lessFunc := func(i *model.PolymorphicEvent, j *model.PolymorphicEvent) bool { - if i.CRTs == j.CRTs { - if i.RawKV.OpType == model.OpTypeDelete { - return true - } - if j.RawKV.OpType == model.OpTypeResolved { - return true - } - } - return i.CRTs < j.CRTs - } - mergeFunc := func(kvsA []*model.PolymorphicEvent, kvsB []*model.PolymorphicEvent, output func(*model.PolymorphicEvent)) { - var i, j int - for i < len(kvsA) && j < len(kvsB) { - if lessFunc(kvsA[i], kvsB[j]) { - output(kvsA[i]) - i++ - } else { - output(kvsB[j]) - j++ - } - } - for ; i < len(kvsA); i++ { - output(kvsA[i]) - } - for ; j < len(kvsB); j++ { - output(kvsB[j]) - } - } output := func(ctx context.Context, entry *model.PolymorphicEvent) { select { case <-ctx.Done(): @@ -136,14 +108,14 @@ func (es *EntrySorter) Run(ctx context.Context) error { toSort = append(toSort, resEvents...) startTime := time.Now() sort.Slice(toSort, func(i, j int) bool { - return lessFunc(toSort[i], toSort[j]) + return eventLess(toSort[i], toSort[j]) }) metricEntrySorterSortDuration.Observe(time.Since(startTime).Seconds()) maxResolvedTs := resolvedTsGroup[len(resolvedTsGroup)-1] startTime = time.Now() var merged []*model.PolymorphicEvent - mergeFunc(toSort, sorted, func(entry *model.PolymorphicEvent) { + mergeEvents(toSort, sorted, func(entry *model.PolymorphicEvent) { if entry.CRTs <= maxResolvedTs { output(ctx, entry) } else { @@ -159,18 +131,26 @@ func (es *EntrySorter) Run(ctx context.Context) error { } // AddEntry adds an RawKVEntry to the EntryGroup -func (es *EntrySorter) AddEntry(ctx context.Context, entry *model.PolymorphicEvent) { +func (es *EntrySorter) AddEntry(_ context.Context, entry *model.PolymorphicEvent) { if atomic.LoadInt32(&es.closed) != 0 { return } es.lock.Lock() + defer es.lock.Unlock() if entry.RawKV.OpType == model.OpTypeResolved { es.resolvedTsGroup = append(es.resolvedTsGroup, entry.CRTs) es.resolvedNotifier.Notify() } else { es.unsorted = append(es.unsorted, entry) } - es.lock.Unlock() +} + +func (es *EntrySorter) TryAddEntry(ctx context.Context, entry *model.PolymorphicEvent) (bool, error) { + if atomic.LoadInt32(&es.closed) != 0 { + return false, cerror.ErrSorterClosed.GenWithStackByArgs() + } + es.AddEntry(ctx, entry) + return true, nil } // Output returns the sorted raw kv output channel @@ -178,6 +158,38 @@ func (es *EntrySorter) Output() <-chan *model.PolymorphicEvent { return es.outputCh } +func eventLess(i *model.PolymorphicEvent, j *model.PolymorphicEvent) bool { + if i.CRTs == j.CRTs { + if i.RawKV.OpType == model.OpTypeDelete { + return true + } + + if j.RawKV.OpType == model.OpTypeResolved { + return true + } + } + return i.CRTs < j.CRTs +} + +func mergeEvents(kvsA []*model.PolymorphicEvent, kvsB []*model.PolymorphicEvent, output func(*model.PolymorphicEvent)) { + var i, j int + for i < len(kvsA) && j < len(kvsB) { + if eventLess(kvsA[i], kvsB[j]) { + output(kvsA[i]) + i++ + } else { + output(kvsB[j]) + j++ + } + } + for ; i < len(kvsA); i++ { + output(kvsA[i]) + } + for ; j < len(kvsB); j++ { + output(kvsB[j]) + } +} + // SortOutput receives a channel from a puller, then sort event and output to the channel returned. func SortOutput(ctx context.Context, input <-chan *model.RawKVEntry) <-chan *model.RawKVEntry { ctx, cancel := context.WithCancel(ctx) diff --git a/cdc/sorter/memory/entry_sorter_test.go b/cdc/sorter/memory/entry_sorter_test.go index 55e3229bc7a..def16842337 100644 --- a/cdc/sorter/memory/entry_sorter_test.go +++ b/cdc/sorter/memory/entry_sorter_test.go @@ -16,12 +16,15 @@ package memory import ( "context" "math/rand" + "sort" "sync" + "sync/atomic" "testing" "github.com/pingcap/check" "github.com/pingcap/errors" "github.com/pingcap/ticdc/cdc/model" + cerror "github.com/pingcap/ticdc/pkg/errors" "github.com/pingcap/ticdc/pkg/util/testleak" ) @@ -132,6 +135,109 @@ func (s *mockEntrySorterSuite) TestEntrySorter(c *check.C) { wg.Wait() } +func (s *mockEntrySorterSuite) TestEntrySorterNonBlocking(c *check.C) { + defer testleak.AfterTest(c)() + testCases := []struct { + input []*model.RawKVEntry + resolvedTs uint64 + expect []*model.RawKVEntry + }{ + { + input: []*model.RawKVEntry{ + {CRTs: 1, OpType: model.OpTypePut}, + {CRTs: 2, OpType: model.OpTypePut}, + {CRTs: 4, OpType: model.OpTypeDelete}, + {CRTs: 2, OpType: model.OpTypeDelete}, + }, + resolvedTs: 0, + expect: []*model.RawKVEntry{ + {CRTs: 0, OpType: model.OpTypeResolved}, + }, + }, + { + input: []*model.RawKVEntry{ + {CRTs: 3, OpType: model.OpTypePut}, + {CRTs: 2, OpType: model.OpTypePut}, + {CRTs: 5, OpType: model.OpTypePut}, + }, + resolvedTs: 3, + expect: []*model.RawKVEntry{ + {CRTs: 1, OpType: model.OpTypePut}, + {CRTs: 2, OpType: model.OpTypeDelete}, + {CRTs: 2, OpType: model.OpTypePut}, + {CRTs: 2, OpType: model.OpTypePut}, + {CRTs: 3, OpType: model.OpTypePut}, + {CRTs: 3, OpType: model.OpTypeResolved}, + }, + }, + { + input: []*model.RawKVEntry{}, + resolvedTs: 3, + expect: []*model.RawKVEntry{{CRTs: 3, OpType: model.OpTypeResolved}}, + }, + { + input: []*model.RawKVEntry{ + {CRTs: 7, OpType: model.OpTypePut}, + }, + resolvedTs: 6, + expect: []*model.RawKVEntry{ + {CRTs: 4, OpType: model.OpTypeDelete}, + {CRTs: 5, OpType: model.OpTypePut}, + {CRTs: 6, OpType: model.OpTypeResolved}, + }, + }, + { + input: []*model.RawKVEntry{{CRTs: 7, OpType: model.OpTypeDelete}}, + resolvedTs: 6, + expect: []*model.RawKVEntry{ + {CRTs: 6, OpType: model.OpTypeResolved}, + }, + }, + { + input: []*model.RawKVEntry{{CRTs: 7, OpType: model.OpTypeDelete}}, + resolvedTs: 8, + expect: []*model.RawKVEntry{ + {CRTs: 7, OpType: model.OpTypeDelete}, + {CRTs: 7, OpType: model.OpTypeDelete}, + {CRTs: 7, OpType: model.OpTypePut}, + {CRTs: 8, OpType: model.OpTypeResolved}, + }, + }, + { + input: []*model.RawKVEntry{}, + resolvedTs: 15, + expect: []*model.RawKVEntry{ + {CRTs: 15, OpType: model.OpTypeResolved}, + }, + }, + } + es := NewEntrySorter() + ctx, cancel := context.WithCancel(context.Background()) + var wg sync.WaitGroup + wg.Add(1) + go func() { + defer wg.Done() + err := es.Run(ctx) + c.Assert(errors.Cause(err), check.Equals, context.Canceled) + }() + for _, tc := range testCases { + for _, entry := range tc.input { + added, err := es.TryAddEntry(ctx, model.NewPolymorphicEvent(entry)) + c.Assert(added, check.IsTrue) + c.Assert(err, check.IsNil) + } + added, err := es.TryAddEntry(ctx, model.NewResolvedPolymorphicEvent(0, tc.resolvedTs)) + c.Assert(added, check.IsTrue) + c.Assert(err, check.IsNil) + for i := 0; i < len(tc.expect); i++ { + e := <-es.Output() + c.Check(e.RawKV, check.DeepEquals, tc.expect[i]) + } + } + cancel() + wg.Wait() +} + func (s *mockEntrySorterSuite) TestEntrySorterRandomly(c *check.C) { defer testleak.AfterTest(c)() es := NewEntrySorter() @@ -189,6 +295,179 @@ func (s *mockEntrySorterSuite) TestEntrySorterRandomly(c *check.C) { wg.Wait() } +func (s *mockEntrySorterSuite) TestEventLess(c *check.C) { + defer testleak.AfterTest(c)() + testCases := []struct { + i *model.PolymorphicEvent + j *model.PolymorphicEvent + expected bool + }{ + { + &model.PolymorphicEvent{ + CRTs: 1, + }, + &model.PolymorphicEvent{ + CRTs: 2, + }, + true, + }, + { + &model.PolymorphicEvent{ + CRTs: 2, + RawKV: &model.RawKVEntry{ + OpType: model.OpTypeDelete, + }, + }, + &model.PolymorphicEvent{ + CRTs: 2, + RawKV: &model.RawKVEntry{ + OpType: model.OpTypeDelete, + }, + }, + true, + }, + { + &model.PolymorphicEvent{ + CRTs: 2, + RawKV: &model.RawKVEntry{ + OpType: model.OpTypeResolved, + }, + }, + &model.PolymorphicEvent{ + CRTs: 2, + RawKV: &model.RawKVEntry{ + OpType: model.OpTypeResolved, + }, + }, + true, + }, + { + &model.PolymorphicEvent{ + CRTs: 2, + RawKV: &model.RawKVEntry{ + OpType: model.OpTypeResolved, + }, + }, + &model.PolymorphicEvent{ + CRTs: 2, + RawKV: &model.RawKVEntry{ + OpType: model.OpTypeDelete, + }, + }, + false, + }, + { + &model.PolymorphicEvent{ + CRTs: 3, + RawKV: &model.RawKVEntry{ + OpType: model.OpTypeDelete, + }, + }, + &model.PolymorphicEvent{ + CRTs: 2, + RawKV: &model.RawKVEntry{ + OpType: model.OpTypeResolved, + }, + }, + false, + }, + } + + for _, tc := range testCases { + c.Assert(eventLess(tc.i, tc.j), check.Equals, tc.expected) + } +} + +func (s *mockEntrySorterSuite) TestMergeEvents(c *check.C) { + defer testleak.AfterTest(c)() + events1 := []*model.PolymorphicEvent{ + { + CRTs: 1, + RawKV: &model.RawKVEntry{ + OpType: model.OpTypeDelete, + }, + }, + { + CRTs: 2, + RawKV: &model.RawKVEntry{ + OpType: model.OpTypePut, + }, + }, + { + CRTs: 3, + RawKV: &model.RawKVEntry{ + OpType: model.OpTypePut, + }, + }, + { + CRTs: 4, + RawKV: &model.RawKVEntry{ + OpType: model.OpTypePut, + }, + }, + { + CRTs: 5, + RawKV: &model.RawKVEntry{ + OpType: model.OpTypeDelete, + }, + }, + } + events2 := []*model.PolymorphicEvent{ + { + CRTs: 3, + RawKV: &model.RawKVEntry{ + OpType: model.OpTypeResolved, + }, + }, + { + CRTs: 4, + RawKV: &model.RawKVEntry{ + OpType: model.OpTypePut, + }, + }, + { + CRTs: 4, + RawKV: &model.RawKVEntry{ + OpType: model.OpTypeResolved, + }, + }, + { + CRTs: 7, + RawKV: &model.RawKVEntry{ + OpType: model.OpTypePut, + }, + }, + { + CRTs: 9, + RawKV: &model.RawKVEntry{ + OpType: model.OpTypeDelete, + }, + }, + } + + var outputResults []*model.PolymorphicEvent + output := func(event *model.PolymorphicEvent) { + outputResults = append(outputResults, event) + } + + expectedResults := append(events1, events2...) + sort.Slice(expectedResults, func(i, j int) bool { + return eventLess(expectedResults[i], expectedResults[j]) + }) + + mergeEvents(events1, events2, output) + c.Assert(outputResults, check.DeepEquals, expectedResults) +} + +func (s *mockEntrySorterSuite) TestEntrySorterClosed(c *check.C) { + defer testleak.AfterTest(c)() + es := NewEntrySorter() + atomic.StoreInt32(&es.closed, 1) + added, err := es.TryAddEntry(context.TODO(), model.NewResolvedPolymorphicEvent(0, 1)) + c.Assert(added, check.IsFalse) + c.Assert(cerror.ErrSorterClosed.Equal(err), check.IsTrue) +} + func BenchmarkSorter(b *testing.B) { es := NewEntrySorter() ctx, cancel := context.WithCancel(context.Background()) diff --git a/cdc/sorter/sorter.go b/cdc/sorter/sorter.go index 9145d5534fc..088fd308297 100644 --- a/cdc/sorter/sorter.go +++ b/cdc/sorter/sorter.go @@ -24,5 +24,9 @@ import ( type EventSorter interface { Run(ctx context.Context) error AddEntry(ctx context.Context, entry *model.PolymorphicEvent) + // TryAddEntry tries to add and entry to the sorter. + // Returns false if the entry can not be added; otherwise it returns true + // Returns error if the sorter is closed or context is done + TryAddEntry(ctx context.Context, entry *model.PolymorphicEvent) (bool, error) Output() <-chan *model.PolymorphicEvent } diff --git a/cdc/sorter/unified/unified_sorter.go b/cdc/sorter/unified/unified_sorter.go index 9480fa9a2c9..3f84c831ddb 100644 --- a/cdc/sorter/unified/unified_sorter.go +++ b/cdc/sorter/unified/unified_sorter.go @@ -281,6 +281,24 @@ func (s *Sorter) AddEntry(ctx context.Context, entry *model.PolymorphicEvent) { } } +// TryAddEntry implements the EventSorter interface +func (s *Sorter) TryAddEntry(ctx context.Context, entry *model.PolymorphicEvent) (bool, error) { + // add two select to guarantee the done/close condition is checked first. + select { + case <-ctx.Done(): + return false, ctx.Err() + case <-s.closeCh: + return false, cerror.ErrSorterClosed.GenWithStackByArgs() + default: + } + select { + case s.inputCh <- entry: + return true, nil + default: + return false, nil + } +} + // Output implements the EventSorter interface func (s *Sorter) Output() <-chan *model.PolymorphicEvent { return s.outputCh diff --git a/cdc/sorter/unified/unified_sorter_test.go b/cdc/sorter/unified/unified_sorter_test.go new file mode 100644 index 00000000000..eab4b2075ef --- /dev/null +++ b/cdc/sorter/unified/unified_sorter_test.go @@ -0,0 +1,59 @@ +// Copyright 2021 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package unified + +import ( + "context" + "testing" + + "github.com/pingcap/ticdc/cdc/model" + cerror "github.com/pingcap/ticdc/pkg/errors" + "github.com/stretchr/testify/require" +) + +func TestUnifiedSorterTryAddEntry(t *testing.T) { + t.Parallel() + + events := []*model.PolymorphicEvent{ + model.NewPolymorphicEvent(&model.RawKVEntry{OpType: model.OpTypePut, StartTs: 1, CRTs: 0, RegionID: 0}), + model.NewResolvedPolymorphicEvent(0, 1), + } + for _, event := range events { + s := &Sorter{inputCh: make(chan *model.PolymorphicEvent, 2), closeCh: make(chan struct{}, 2)} + added, err := s.TryAddEntry(context.TODO(), event) + require.True(t, added) + require.Nil(t, err) + added, err = s.TryAddEntry(context.TODO(), event) + require.True(t, added) + require.Nil(t, err) + added, err = s.TryAddEntry(context.TODO(), event) + require.False(t, added) + require.Nil(t, err) + <-s.inputCh + added, err = s.TryAddEntry(context.TODO(), event) + require.True(t, added) + require.Nil(t, err) + <-s.inputCh + ctx, cancel := context.WithCancel(context.TODO()) + cancel() + added, err = s.TryAddEntry(ctx, event) + require.False(t, added) + require.False(t, cerror.ErrSorterClosed.Equal(err)) + <-s.inputCh + s.closeCh <- struct{}{} + added, err = s.TryAddEntry(context.TODO(), event) + require.False(t, added) + require.True(t, cerror.ErrSorterClosed.Equal(err)) + } +} diff --git a/dm/cmd/dm-worker/main.go b/dm/cmd/dm-worker/main.go index 410f14553bf..ce567ba97e3 100644 --- a/dm/cmd/dm-worker/main.go +++ b/dm/cmd/dm-worker/main.go @@ -55,7 +55,6 @@ func main() { common.PrintLinesf("init logger error %s", terror.Message(err)) os.Exit(2) } - lightningLog.SetAppLogger(log.L().Logger) utils.LogHTTPProxies(true) @@ -65,6 +64,8 @@ func main() { lg, r, _ := globalLog.InitLogger(conf) lg = lg.With(zap.String("component", "ddl tracker")) globalLog.ReplaceGlobals(lg, r) + lightningLogger := lg.With(zap.String("component", "lightning")) + lightningLog.SetAppLogger(lightningLogger) utils.PrintInfo("dm-worker", func() { log.L().Info("", zap.Stringer("dm-worker config", cfg)) diff --git a/dm/dm/master/scheduler/scheduler.go b/dm/dm/master/scheduler/scheduler.go index c019021f248..3d2a7bca6fc 100644 --- a/dm/dm/master/scheduler/scheduler.go +++ b/dm/dm/master/scheduler/scheduler.go @@ -1753,6 +1753,16 @@ func (s *Scheduler) handleWorkerOnline(ev ha.WorkerEvent, toLock bool) error { // 3. change the stage (from Offline) to Free or Relay. lastRelaySource := w.RelaySourceID() + if lastRelaySource == "" { + // when worker is removed (for example lost keepalive when master scheduler boots up), w.RelaySourceID() is + // of course nothing, so we find the relay source from a better place + for source, workerM := range s.relayWorkers { + if _, ok2 := workerM[w.BaseInfo().Name]; ok2 { + lastRelaySource = source + break + } + } + } w.ToFree() // TODO: rename ToFree to Online and move below logic inside it if lastRelaySource != "" { diff --git a/dm/dm/master/server.go b/dm/dm/master/server.go index dd1c9e569fa..4452cc90855 100644 --- a/dm/dm/master/server.go +++ b/dm/dm/master/server.go @@ -1418,6 +1418,7 @@ func withHost(addr string) string { func (s *Server) removeMetaData(ctx context.Context, taskName, metaSchema string, toDBCfg *config.DBConfig) error { toDBCfg.Adjust() + // clear shard meta data for pessimistic/optimist err := s.pessimist.RemoveMetaData(taskName) if err != nil { diff --git a/dm/dm/master/shardddl/optimist.go b/dm/dm/master/shardddl/optimist.go index 2c1ca3c568b..eda57ea0332 100644 --- a/dm/dm/master/shardddl/optimist.go +++ b/dm/dm/master/shardddl/optimist.go @@ -698,7 +698,7 @@ func (o *Optimist) handleLock(info optimism.Info, tts []optimism.TargetTable, sk func (o *Optimist) removeLock(lock *optimism.Lock) (bool, error) { failpoint.Inject("SleepWhenRemoveLock", func(val failpoint.Value) { t := val.(int) - log.L().Info("wait new ddl info putted into etcd", + log.L().Info("wait new ddl info putted into etcd in optimistic", zap.String("failpoint", "SleepWhenRemoveLock"), zap.Int("max wait second", t)) diff --git a/dm/dm/master/shardddl/pessimist.go b/dm/dm/master/shardddl/pessimist.go index e20d228097c..e1f44df1e08 100644 --- a/dm/dm/master/shardddl/pessimist.go +++ b/dm/dm/master/shardddl/pessimist.go @@ -645,7 +645,7 @@ func (p *Pessimist) removeLock(lock *pessimism.Lock) error { failpoint.Inject("SleepWhenRemoveLock", func(val failpoint.Value) { t := val.(int) - log.L().Info("wait new ddl info putted into etcd", + log.L().Info("wait new ddl info putted into etcd in pessimistic", zap.String("failpoint", "SleepWhenRemoveLock"), zap.Int("max wait second", t)) diff --git a/dm/dm/worker/task_checker_test.go b/dm/dm/worker/task_checker_test.go index b35fbe4b117..89309cb34a1 100644 --- a/dm/dm/worker/task_checker_test.go +++ b/dm/dm/worker/task_checker_test.go @@ -306,6 +306,7 @@ func (s *testTaskCheckerSuite) TestIsResumableError(c *check.C) { {nil, true}, {errors.New("unknown error"), true}, {terror.ErrNotSet.Delegate(&tmysql.SQLError{Code: 1236, Message: "Could not find first log file name in binary log index file", State: tmysql.DefaultMySQLState}), false}, + {terror.ErrNotSet.Delegate(&tmysql.SQLError{Code: 1236, Message: "The slave is connecting using CHANGE MASTER TO MASTER_AUTO_POSITION = 1, but the master has purged binary logs containing GTIDs that the slave requires", State: tmysql.DefaultMySQLState}), false}, } for _, tc := range testCases { diff --git a/dm/loader/checkpoint.go b/dm/loader/checkpoint.go index c3f380ce1b7..c1387cbee72 100644 --- a/dm/loader/checkpoint.go +++ b/dm/loader/checkpoint.go @@ -14,6 +14,7 @@ package loader import ( + "context" "encoding/json" "fmt" "strings" @@ -31,6 +32,10 @@ import ( "go.uber.org/zap" ) +const ( + LightningCheckpointListName = "lightning_checkpoint_list" +) + // CheckPoint represents checkpoint status. type CheckPoint interface { // Load loads all checkpoints recorded before. @@ -100,7 +105,7 @@ type RemoteCheckPoint struct { } func newRemoteCheckPoint(tctx *tcontext.Context, cfg *config.SubTaskConfig, id string) (CheckPoint, error) { - db, dbConns, err := createConns(tctx, cfg, 1) + db, dbConns, err := createConns(tctx, cfg, cfg.Name, cfg.SourceID, 1) if err != nil { return nil, err } @@ -464,3 +469,99 @@ func (cp *RemoteCheckPoint) String() string { } return string(bytes) } + +type LightningCheckpointList struct { + db *conn.BaseDB + schema string + tableName string + logger log.Logger +} + +func NewLightningCheckpointList(db *conn.BaseDB, metaSchema string) *LightningCheckpointList { + return &LightningCheckpointList{ + db: db, + schema: dbutil.ColumnName(metaSchema), + tableName: dbutil.TableName(metaSchema, LightningCheckpointListName), + logger: log.L().WithFields(zap.String("component", "lightning checkpoint database list")), + } +} + +func (cp *LightningCheckpointList) Prepare(ctx context.Context) error { + connection, err := cp.db.GetBaseConn(ctx) + if err != nil { + return terror.WithScope(terror.Annotate(err, "initialize connection when prepare"), terror.ScopeDownstream) + } + createSchema := fmt.Sprintf("CREATE SCHEMA IF NOT EXISTS %s", cp.schema) + tctx := tcontext.NewContext(ctx, log.With(zap.String("job", "lightning-checkpoint"))) + _, err = connection.ExecuteSQL(tctx, nil, "lightning-checkpoint", []string{createSchema}) + if err != nil { + return err + } + createTable := `CREATE TABLE IF NOT EXISTS %s ( + worker_name varchar(255) NOT NULL, + task_name varchar(255) NOT NULL, + PRIMARY KEY (task_name, worker_name) + ); +` + sql2 := fmt.Sprintf(createTable, cp.tableName) + _, err = connection.ExecuteSQL(tctx, nil, "lightning-checkpoint", []string{sql2}) + return terror.WithScope(err, terror.ScopeDownstream) +} + +func (cp *LightningCheckpointList) RegisterCheckPoint(ctx context.Context, workerName, taskName string) error { + connection, err := cp.db.GetBaseConn(ctx) + if err != nil { + return terror.WithScope(terror.Annotate(err, "initialize connection"), terror.ScopeDownstream) + } + + sql := fmt.Sprintf("INSERT IGNORE INTO %s (`worker_name`, `task_name`) VALUES(?,?)", cp.tableName) + cp.logger.Info("initial checkpoint record", + zap.String("sql", sql), + zap.String("worker-name", workerName), + zap.String("task-name", taskName)) + args := []interface{}{workerName, taskName} + tctx := tcontext.NewContext(ctx, log.With(zap.String("job", "lightning-checkpoint"))) + _, err = connection.ExecuteSQL(tctx, nil, "lightning-checkpoint", []string{sql}, args) + if err != nil { + return terror.WithScope(terror.Annotate(err, "initialize checkpoint"), terror.ScopeDownstream) + } + return nil +} + +func (cp *LightningCheckpointList) RemoveTaskCheckPoint(ctx context.Context, taskName string) error { + connection, err := cp.db.GetBaseConn(ctx) + if err != nil { + return terror.WithScope(terror.Annotate(err, "initialize connection"), terror.ScopeDownstream) + } + + tctx := tcontext.NewContext(ctx, log.With(zap.String("job", "lightning-checkpoint"))) + query := fmt.Sprintf("SELECT `worker_name` from %s where `task_name`=?", cp.tableName) + rows, err := connection.QuerySQL(tctx, query, taskName) + if err != nil { + return terror.WithScope(err, terror.ScopeDownstream) + } + defer rows.Close() + var workerName string + for rows.Next() { + err = rows.Scan(&workerName) + if err != nil { + return terror.WithScope(terror.DBErrorAdapt(err, terror.ErrDBDriverError), terror.ScopeDownstream) + } + cpdb := config.TiDBLightningCheckpointPrefix + dbutil.TableName(workerName, taskName) + sql := fmt.Sprintf("DROP DATABASE IF NOT EXISTS %s", cpdb) + _, err = connection.ExecuteSQL(tctx, nil, "lightning-checkpoint", []string{sql}) + if err != nil { + return terror.WithScope(err, terror.ScopeDownstream) + } + } + query = fmt.Sprintf("DELETE from %s where `task_name`=?", cp.tableName) + _, err = connection.ExecuteSQL(tctx, nil, "lightning-checkpoint", []string{query}, []interface{}{taskName}) + return terror.WithScope(err, terror.ScopeDownstream) +} + +// Close implements CheckPoint.Close. +func (cp *LightningCheckpointList) Close() { + if err := cp.db.Close(); err != nil { + cp.logger.Error("close checkpoint list db", log.ShortError(err)) + } +} diff --git a/dm/loader/db.go b/dm/loader/db.go index d588c6a3626..a9a919aa8a9 100644 --- a/dm/loader/db.go +++ b/dm/loader/db.go @@ -37,7 +37,8 @@ import ( // DBConn represents a live DB connection // it's not thread-safe. type DBConn struct { - cfg *config.SubTaskConfig + name string + sourceID string baseConn *conn.BaseConn // generate new BaseConn and close old one @@ -89,7 +90,7 @@ func (conn *DBConn) querySQL(ctx *tcontext.Context, query string, args ...interf cost := time.Since(startTime) // duration seconds ds := cost.Seconds() - queryHistogram.WithLabelValues(conn.cfg.Name, conn.cfg.SourceID).Observe(ds) + queryHistogram.WithLabelValues(conn.name, conn.sourceID).Observe(ds) if ds > 1 { ctx.L().Warn("query statement too slow", zap.Duration("cost time", cost), @@ -123,7 +124,7 @@ func (conn *DBConn) executeSQL(ctx *tcontext.Context, queries []string, args ... FirstRetryDuration: 2 * time.Second, BackoffStrategy: retry.LinearIncrease, IsRetryableFn: func(retryTime int, err error) bool { - tidbExecutionErrorCounter.WithLabelValues(conn.cfg.Name, conn.cfg.SourceID).Inc() + tidbExecutionErrorCounter.WithLabelValues(conn.name, conn.sourceID).Inc() if retry.IsConnectionError(err) { err = conn.resetConn(ctx) if err != nil { @@ -151,7 +152,7 @@ func (conn *DBConn) executeSQL(ctx *tcontext.Context, queries []string, args ... params, func(ctx *tcontext.Context) (interface{}, error) { startTime := time.Now() - _, err := conn.baseConn.ExecuteSQL(ctx, stmtHistogram, conn.cfg.Name, queries, args...) + _, err := conn.baseConn.ExecuteSQL(ctx, stmtHistogram, conn.name, queries, args...) failpoint.Inject("LoadExecCreateTableFailed", func(val failpoint.Value) { errCode, err1 := strconv.ParseUint(val.(string), 10, 16) if err1 != nil { @@ -196,7 +197,9 @@ func (conn *DBConn) resetConn(tctx *tcontext.Context) error { return nil } -func createConns(tctx *tcontext.Context, cfg *config.SubTaskConfig, workerCount int) (*conn.BaseDB, []*DBConn, error) { +func createConns(tctx *tcontext.Context, cfg *config.SubTaskConfig, + name, sourceID string, + workerCount int) (*conn.BaseDB, []*DBConn, error) { baseDB, err := conn.DefaultDBProvider.Apply(&cfg.To) if err != nil { return nil, nil, terror.WithScope(err, terror.ScopeDownstream) @@ -218,7 +221,7 @@ func createConns(tctx *tcontext.Context, cfg *config.SubTaskConfig, workerCount } return baseDB.GetBaseConn(tctx.Context()) } - conns = append(conns, &DBConn{baseConn: baseConn, cfg: cfg, resetBaseConnFn: resetBaseConnFn}) + conns = append(conns, &DBConn{baseConn: baseConn, name: name, sourceID: sourceID, resetBaseConnFn: resetBaseConnFn}) } return baseDB, conns, nil } diff --git a/dm/loader/lightning.go b/dm/loader/lightning.go index 8ef21d367a6..8f6265ce102 100644 --- a/dm/loader/lightning.go +++ b/dm/loader/lightning.go @@ -16,6 +16,7 @@ package loader import ( "context" "path/filepath" + "strings" "sync" "github.com/docker/go-units" @@ -53,6 +54,7 @@ type LightningLoader struct { cfg *config.SubTaskConfig cli *clientv3.Client checkPoint CheckPoint + checkPointList *LightningCheckpointList workerName string logger log.Logger core *lightning.Lightning @@ -114,13 +116,33 @@ func (l *LightningLoader) Type() pb.UnitType { // if fail, it should not call l.Close. func (l *LightningLoader) Init(ctx context.Context) (err error) { tctx := tcontext.NewContext(ctx, l.logger) + toCfg, err := l.cfg.Clone() + if err != nil { + return err + } + l.toDB, l.toDBConns, err = createConns(tctx, l.cfg, toCfg.Name, toCfg.SourceID, 1) + if err != nil { + return err + } + checkpoint, err := newRemoteCheckPoint(tctx, l.cfg, l.checkpointID()) + if err == nil { + l.checkPoint = checkpoint + checkpointList := NewLightningCheckpointList(l.toDB, l.cfg.MetaSchema) + err1 := checkpointList.Prepare(ctx) + if err1 == nil { + l.checkPointList = checkpointList + } + err = err1 + } failpoint.Inject("ignoreLoadCheckpointErr", func(_ failpoint.Value) { l.logger.Info("", zap.String("failpoint", "ignoreLoadCheckpointErr")) err = nil }) - l.checkPoint = checkpoint - l.toDB, l.toDBConns, err = createConns(tctx, l.cfg, 1) + if err != nil { + return err + } + timeZone := l.cfg.Timezone if len(timeZone) == 0 { var err1 error @@ -130,6 +152,27 @@ func (l *LightningLoader) Init(ctx context.Context) (err error) { } } l.timeZone = timeZone + return nil +} + +func (l *LightningLoader) runLightning(ctx context.Context, cfg *lcfg.Config) error { + l.Lock() + taskCtx, cancel := context.WithCancel(ctx) + l.cancel = cancel + l.Unlock() + err := l.core.RunOnce(taskCtx, cfg, nil) + failpoint.Inject("LightningLoadDataSlowDown", nil) + failpoint.Inject("LightningLoadDataSlowDownByTask", func(val failpoint.Value) { + tasks := val.(string) + taskNames := strings.Split(tasks, ",") + for _, taskName := range taskNames { + if l.cfg.Name == taskName { + l.logger.Info("inject failpoint LightningLoadDataSlowDownByTask", zap.String("task", taskName)) + <-taskCtx.Done() + } + } + }) + l.logger.Info("end runLightning") return err } @@ -155,13 +198,19 @@ func (l *LightningLoader) restore(ctx context.Context) error { return err } if !l.checkPoint.IsTableFinished(lightningCheckpointDB, lightningCheckpointTable) { + if l.checkPointList != nil { + if err = l.checkPointList.RegisterCheckPoint(ctx, l.workerName, l.cfg.Name); err != nil { + return err + } + } cfg := lcfg.NewConfig() if err = cfg.LoadFromGlobal(l.lightningConfig); err != nil { return err } cfg.Routes = l.cfg.RouteRules cfg.Checkpoint.Driver = lcfg.CheckpointDriverMySQL - cfg.Checkpoint.Schema = config.TiDBLightningCheckpointPrefix + dbutil.ColumnName(l.workerName) + cfg.Checkpoint.Schema = config.TiDBLightningCheckpointPrefix + dbutil.TableName(l.workerName, l.cfg.Name) + cfg.Checkpoint.KeepAfterSuccess = lcfg.CheckpointOrigin param := common.MySQLConnectParam{ Host: cfg.TiDB.Host, Port: cfg.TiDB.Port, @@ -172,6 +221,13 @@ func (l *LightningLoader) restore(ctx context.Context) error { TLS: cfg.TiDB.TLS, } cfg.Checkpoint.DSN = param.ToDSN() + cfg.TiDB.Vars = make(map[string]string) + if l.cfg.To.Session != nil { + for k, v := range l.cfg.To.Session { + cfg.TiDB.Vars[k] = v + } + } + cfg.TiDB.StrSQLMode = l.cfg.LoaderConfig.SQLMode cfg.TiDB.Vars = map[string]string{ "time_zone": l.timeZone, @@ -179,21 +235,22 @@ func (l *LightningLoader) restore(ctx context.Context) error { if err = cfg.Adjust(ctx); err != nil { return err } - l.Lock() - taskCtx, cancel := context.WithCancel(ctx) - l.cancel = cancel - l.Unlock() - err = l.core.RunOnce(taskCtx, cfg, nil) + err = l.runLightning(ctx, cfg) + if err == nil { + err = lightning.CheckpointRemove(ctx, cfg, "all") + } if err == nil { l.finish.Store(true) offsetSQL := l.checkPoint.GenSQL(lightningCheckpointFile, 1) err = l.toDBConns[0].executeSQL(tctx, []string{offsetSQL}) _ = l.checkPoint.UpdateOffset(lightningCheckpointFile, 1) + } else { + l.logger.Error("failed to runlightning", zap.Error(err)) } } else { l.finish.Store(true) } - if l.cfg.Mode == config.ModeFull { + if err == nil && l.finish.Load() && l.cfg.Mode == config.ModeFull { if err = delLoadTask(l.cli, l.cfg, l.workerName); err != nil { return err } @@ -233,6 +290,7 @@ func (l *LightningLoader) Process(ctx context.Context, pr chan pb.ProcessResult) } if err := l.restore(ctx); err != nil && !utils.IsContextCanceledError(err) { + l.logger.Error("process error", zap.Error(err)) errs = append(errs, unit.NewProcessError(err)) } isCanceled := false @@ -241,7 +299,7 @@ func (l *LightningLoader) Process(ctx context.Context, pr chan pb.ProcessResult) isCanceled = true default: } - l.logger.Info("lightning load end") + l.logger.Info("lightning load end", zap.Bool("IsCanceled", isCanceled)) pr <- pb.ProcessResult{ IsCanceled: isCanceled, Errors: errs, @@ -261,6 +319,8 @@ func (l *LightningLoader) IsFreshTask(ctx context.Context) (bool, error) { // Close does graceful shutdown. func (l *LightningLoader) Close() { l.Pause() + l.checkPoint.Close() + l.checkPointList.Close() l.closed.Store(true) } diff --git a/dm/loader/loader.go b/dm/loader/loader.go index 2b80fba6ac7..38fd086dd64 100644 --- a/dm/loader/loader.go +++ b/dm/loader/loader.go @@ -562,7 +562,7 @@ func (l *Loader) Init(ctx context.Context) (err error) { l.logger.Info("loader's sql_mode is", zap.String("sqlmode", lcfg.To.Session["sql_mode"])) - l.toDB, l.toDBConns, err = createConns(tctx, lcfg, l.cfg.PoolSize) + l.toDB, l.toDBConns, err = createConns(tctx, lcfg, lcfg.Name, lcfg.SourceID, l.cfg.PoolSize) if err != nil { return err } @@ -1328,7 +1328,8 @@ func (q *jobQueue) startConsumers(handler func(ctx context.Context, job *restore } }(baseConn) session = &DBConn{ - cfg: job.loader.cfg, + name: job.loader.cfg.Name, + sourceID: job.loader.cfg.SourceID, baseConn: baseConn, resetBaseConnFn: func(*tcontext.Context, *conn.BaseConn) (*conn.BaseConn, error) { return nil, terror.ErrDBBadConn.Generate("bad connection error restoreData") diff --git a/dm/pkg/retry/errors.go b/dm/pkg/retry/errors.go index 3bbbed36fb1..9fba39061b6 100644 --- a/dm/pkg/retry/errors.go +++ b/dm/pkg/retry/errors.go @@ -52,6 +52,7 @@ var ( // ReplicationErrMsgs list the error message of un-recoverable replication error. ReplicationErrMsgs = []string{ "Could not find first log file name in binary log index file", + "The slave is connecting using CHANGE MASTER TO MASTER_AUTO_POSITION = 1, but the master has purged binary logs containing GTIDs that the slave requires", } // ParseRelayLogErrMsgs list the error messages of some un-recoverable relay log parsing error, which is used in task auto recovery. diff --git a/dm/syncer/causality_test.go b/dm/syncer/causality_test.go index 13e99125ca9..30e426f883a 100644 --- a/dm/syncer/causality_test.go +++ b/dm/syncer/causality_test.go @@ -133,3 +133,77 @@ func (s *testSyncerSuite) TestCasuality(c *C) { c.Assert(job.tp, Equals, op) } } + +func (s *testSyncerSuite) TestCasualityWithPrefixIndex(c *C) { + p := parser.New() + se := mock.NewContext() + schemaStr := "create table t (c1 text, c2 int unique, unique key c1(c1(3)));" + ti, err := createTableInfo(p, se, int64(0), schemaStr) + c.Assert(err, IsNil) + downTi := schema.GetDownStreamTi(ti, ti) + c.Assert(downTi, NotNil) + c.Assert(len(downTi.AvailableUKIndexList) == 2, IsTrue) + tiIndex := downTi.AvailableUKIndexList[0] + + jobCh := make(chan *job, 10) + syncer := &Syncer{ + cfg: &config.SubTaskConfig{ + SyncerConfig: config.SyncerConfig{ + QueueSize: 1024, + }, + Name: "task", + SourceID: "source", + }, + tctx: tcontext.Background().WithLogger(log.L()), + } + causalityCh := causalityWrap(jobCh, syncer) + testCases := []struct { + op opType + oldVals []interface{} + vals []interface{} + }{ + { + op: insert, + vals: []interface{}{"1234", 1}, + }, + { + op: insert, + vals: []interface{}{"2345", 2}, + }, + { + op: update, + oldVals: []interface{}{"2345", 2}, + vals: []interface{}{"2345", 3}, + }, + { + op: del, + vals: []interface{}{"1234", 1}, + }, + { + op: insert, + vals: []interface{}{"2345", 1}, + }, + } + results := []opType{insert, insert, update, del, conflict, insert} + resultKeys := []string{"123.c1.", "234.c1.", "234.c1.", "123.c1.", "conflict", "234.c1."} + table := &filter.Table{Schema: "test", Name: "t1"} + location := binlog.NewLocation("") + ec := &eventContext{startLocation: &location, currentLocation: &location, lastLocation: &location} + + for _, tc := range testCases { + job := newDMLJob(tc.op, table, table, newDML(tc.op, false, "", table, tc.oldVals, tc.vals, tc.oldVals, tc.vals, ti.Columns, ti, tiIndex, downTi), ec) + jobCh <- job + } + + c.Assert(utils.WaitSomething(30, 100*time.Millisecond, func() bool { + return len(causalityCh) == len(results) + }), IsTrue) + + for i, op := range results { + job := <-causalityCh + if job.tp != conflict { + c.Assert(job.dml.key, Equals, resultKeys[i]) + } + c.Assert(job.tp, Equals, op) + } +} diff --git a/dm/syncer/dml.go b/dm/syncer/dml.go index b8df3b4b63a..23427e70c4d 100644 --- a/dm/syncer/dml.go +++ b/dm/syncer/dml.go @@ -26,6 +26,8 @@ import ( "github.com/pingcap/tidb/parser/model" "github.com/pingcap/tidb/parser/mysql" "github.com/pingcap/tidb/parser/types" + "github.com/pingcap/tidb/tablecodec" + ttypes "github.com/pingcap/tidb/types" "go.uber.org/zap" tcontext "github.com/pingcap/ticdc/dm/pkg/context" @@ -683,6 +685,22 @@ func genKeyList(table string, columns []*model.ColumnInfo, dataSeq []interface{} return buf.String() } +// truncateIndexValues truncate prefix index from data. +func truncateIndexValues(indexColumns *model.IndexInfo, tiColumns []*model.ColumnInfo, data []interface{}) []interface{} { + values := make([]interface{}, 0, len(indexColumns.Columns)) + for i, iColumn := range indexColumns.Columns { + tcolumn := tiColumns[i] + if data[i] != nil { + datum := ttypes.NewDatum(data[i]) + tablecodec.TruncateIndexValue(&datum, iColumn, tcolumn) + values = append(values, datum.GetValue()) + } else { + values = append(values, data[i]) + } + } + return values +} + // genMultipleKeys gens keys with UNIQUE NOT NULL value. // if not UNIQUE NOT NULL value, use table name instead. func genMultipleKeys(downstreamTableInfo *schema.DownstreamTableInfo, ti *model.TableInfo, value []interface{}, table string) []string { @@ -690,7 +708,9 @@ func genMultipleKeys(downstreamTableInfo *schema.DownstreamTableInfo, ti *model. for _, indexCols := range downstreamTableInfo.AvailableUKIndexList { cols, vals := getColumnData(ti.Columns, indexCols, value) - key := genKeyList(table, cols, vals) + // handle prefix index + truncVals := truncateIndexValues(indexCols, cols, vals) + key := genKeyList(table, cols, truncVals) if len(key) > 0 { // ignore `null` value. multipleKeys = append(multipleKeys, key) } else { diff --git a/dm/syncer/dml_test.go b/dm/syncer/dml_test.go index fd0d3bd43a2..495e57b33f3 100644 --- a/dm/syncer/dml_test.go +++ b/dm/syncer/dml_test.go @@ -544,3 +544,59 @@ func (s *testSyncerSuite) TestGenDMLWithSameOp(c *C) { c.Assert(queries, DeepEquals, expectQueries) c.Assert(args, DeepEquals, expectArgs) } + +func (s *testSyncerSuite) TestTruncateIndexValues(c *C) { + p := parser.New() + se := mock.NewContext() + + testCases := []struct { + schema string + values []interface{} + preValues []interface{} + }{ + { + // test not prefix key + schema: `create table t1(a int, b varchar(20), unique key b(b))`, + values: []interface{}{10, "1234"}, + preValues: []interface{}{"1234"}, + }, + { + // test not string key + schema: `create table t1(a int, b text, unique key a(a))`, + values: []interface{}{10, "1234"}, + preValues: []interface{}{int64(10)}, + }, + { + // test keys + schema: `create table t1(a int, b text, unique key b(b(3)))`, + values: []interface{}{10, "1234"}, + preValues: []interface{}{"123"}, + }, + { + // test multi keys + schema: `create table t1(a int, b text, unique key c2(a, b(3)))`, + values: []interface{}{10, "1234"}, + preValues: []interface{}{int64(10), "123"}, + }, + } + + for i, tc := range testCases { + schemaStr := tc.schema + assert := func(obtained interface{}, checker Checker, args ...interface{}) { + c.Assert(obtained, checker, append(args, Commentf("test case schema: %s", schemaStr))...) + } + ti, err := createTableInfo(p, se, int64(i+1), tc.schema) + assert(err, IsNil) + dti := schema.GetDownStreamTi(ti, ti) + assert(dti, NotNil) + assert(dti.AvailableUKIndexList, NotNil) + cols := make([]*model.ColumnInfo, 0, len(dti.AvailableUKIndexList[0].Columns)) + values := make([]interface{}, 0, len(dti.AvailableUKIndexList[0].Columns)) + for _, column := range dti.AvailableUKIndexList[0].Columns { + cols = append(cols, ti.Columns[column.Offset]) + values = append(values, tc.values[column.Offset]) + } + realPreValue := truncateIndexValues(dti.AvailableUKIndexList[0], cols, values) + assert(realPreValue, DeepEquals, tc.preValues) + } +} diff --git a/dm/tests/lightning_load_task/conf/dm-task.yaml b/dm/tests/lightning_load_task/conf/dm-task.yaml new file mode 100644 index 00000000000..b29aa864d12 --- /dev/null +++ b/dm/tests/lightning_load_task/conf/dm-task.yaml @@ -0,0 +1,50 @@ +--- +name: load_task1 +task-mode: all +is-sharding: false +meta-schema: "dm_meta" +heartbeat-update-interval: 1 +heartbeat-report-interval: 1 + +target-database: + host: "127.0.0.1" + port: 4000 + user: "test" + password: "/Q7B9DizNLLTTfiZHv9WoEAKamfpIUs=" + +mysql-instances: + - source-id: "mysql-replica-01" + block-allow-list: "instance" + mydumper-config-name: "global" + loader-config-name: "global" + syncer-config-name: "global" + + - source-id: "mysql-replica-02" + block-allow-list: "instance" + mydumper-config-name: "global" + loader-config-name: "global" + syncer-config-name: "global" + +tidb: + backend: "tidb" + +block-allow-list: + instance: + do-dbs: ["load_task1"] + +mydumpers: + global: + threads: 4 + chunk-filesize: 64 + skip-tz-utc: true + extra-args: "" + +loaders: + global: + pool-size: 16 + dir: "./dumped_data" + +syncers: + global: + worker-count: 16 + batch: 100 diff --git a/dm/tests/lightning_load_task/conf/dm-task2.yaml b/dm/tests/lightning_load_task/conf/dm-task2.yaml new file mode 100644 index 00000000000..a751da96d85 --- /dev/null +++ b/dm/tests/lightning_load_task/conf/dm-task2.yaml @@ -0,0 +1,50 @@ +--- +name: load_task2 +task-mode: all +is-sharding: false +meta-schema: "dm_meta" +heartbeat-update-interval: 1 +heartbeat-report-interval: 1 + +target-database: + host: "127.0.0.1" + port: 4000 + user: "test" + password: "/Q7B9DizNLLTTfiZHv9WoEAKamfpIUs=" + +mysql-instances: + - source-id: "mysql-replica-01" + block-allow-list: "instance" + mydumper-config-name: "global" + loader-config-name: "global" + syncer-config-name: "global" + + - source-id: "mysql-replica-02" + block-allow-list: "instance" + mydumper-config-name: "global" + loader-config-name: "global" + syncer-config-name: "global" + +tidb: + backend: "tidb" + +block-allow-list: + instance: + do-dbs: ["load_task2"] + +mydumpers: + global: + threads: 4 + chunk-filesize: 64 + skip-tz-utc: true + extra-args: "" + +loaders: + global: + pool-size: 16 + dir: "./dumped_data" + +syncers: + global: + worker-count: 16 + batch: 100 diff --git a/dm/tests/lightning_load_task/conf/dm-task3.yaml b/dm/tests/lightning_load_task/conf/dm-task3.yaml new file mode 100644 index 00000000000..e6395139e8e --- /dev/null +++ b/dm/tests/lightning_load_task/conf/dm-task3.yaml @@ -0,0 +1,44 @@ +--- +name: load_task3 +task-mode: all +is-sharding: false +meta-schema: "dm_meta" +heartbeat-update-interval: 1 +heartbeat-report-interval: 1 + +target-database: + host: "127.0.0.1" + port: 4000 + user: "test" + password: "/Q7B9DizNLLTTfiZHv9WoEAKamfpIUs=" + +mysql-instances: + - source-id: "mysql-replica-02" + block-allow-list: "instance" + mydumper-config-name: "global" + loader-config-name: "global" + syncer-config-name: "global" + +block-allow-list: + instance: + do-dbs: ["load_task3"] + +tidb: + backend: "tidb" + +mydumpers: + global: + threads: 4 + chunk-filesize: 64 + skip-tz-utc: true + extra-args: "" + +loaders: + global: + pool-size: 16 + dir: "./dumped_data" + +syncers: + global: + worker-count: 16 + batch: 100 diff --git a/dm/tests/lightning_load_task/conf/dm-task4.yaml b/dm/tests/lightning_load_task/conf/dm-task4.yaml new file mode 100644 index 00000000000..d2abd5092f3 --- /dev/null +++ b/dm/tests/lightning_load_task/conf/dm-task4.yaml @@ -0,0 +1,44 @@ +--- +name: load_task4 +task-mode: all +is-sharding: false +meta-schema: "dm_meta" +heartbeat-update-interval: 1 +heartbeat-report-interval: 1 + +target-database: + host: "127.0.0.1" + port: 4000 + user: "test" + password: "/Q7B9DizNLLTTfiZHv9WoEAKamfpIUs=" + +mysql-instances: + - source-id: "mysql-replica-01" + block-allow-list: "instance" + mydumper-config-name: "global" + loader-config-name: "global" + syncer-config-name: "global" + +block-allow-list: + instance: + do-dbs: ["load_task4"] + +mydumpers: + global: + threads: 4 + chunk-filesize: 64 + skip-tz-utc: true + extra-args: "" + +tidb: + backend: "tidb" + +loaders: + global: + pool-size: 16 + dir: "./dumped_data" + +syncers: + global: + worker-count: 16 + batch: 100 diff --git a/dm/tests/lightning_load_task/run.sh b/dm/tests/lightning_load_task/run.sh new file mode 100755 index 00000000000..f9eb0a57c47 --- /dev/null +++ b/dm/tests/lightning_load_task/run.sh @@ -0,0 +1,243 @@ +#!/bin/bash + +set -eu + +cur=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +DATA_DIR=$cur/../load_task/data +CONF_DIR=$cur/../load_task/conf +source $cur/../_utils/test_prepare +WORK_DIR=$TEST_DIR/$TEST_NAME +API_VERSION="v1alpha1" +WORKER1="worker1" +WORKER2="worker2" +WORKER3="worker3" + +function test_worker_restart() { + echo "test worker restart" + # worker1 offline + ps aux | grep dm-worker1 | awk '{print $2}' | xargs kill || true + check_port_offline $WORKER1_PORT 20 + + # source1 bound to worker3 + run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "list-member -w -n worker3" \ + "\"stage\": \"bound\"" 1 \ + "\"source\": \"mysql-replica-01\"" 1 + run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "list-member -w -n worker1" \ + "\"stage\": \"offline\"" 1 + + run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "query-status load_task1" \ + "different worker in load stage, previous worker: $WORKER1, current worker: $WORKER3" 1 \ + "Please check if the previous worker is online." 1 + + # worker1 online + export GO_FAILPOINTS="github.com/pingcap/ticdc/dm/loader/LightningLoadDataSlowDownByTask=return(\"load_task1\")" + run_dm_worker $WORK_DIR/worker1 $WORKER1_PORT $CONF_DIR/dm-worker1.toml + check_rpc_alive $cur/../bin/check_worker_online 127.0.0.1:$WORKER1_PORT + + # transfer to worker1 + check_log_contain_with_retry 'transfer source and worker.*worker1.*worker3.*mysql-replica-01' $WORK_DIR/master/log/dm-master.log + run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "list-member -w -n worker3" \ + "\"stage\": \"free\"" 1 + run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "list-member -w -n worker1" \ + "\"stage\": \"bound\"" 1 \ + "\"source\": \"mysql-replica-01\"" 1 + + run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "query-status load_task1" \ + "\"unit\": \"Load\"" 1 \ + "\"unit\": \"Sync\"" 1 + run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "query-status load_task2" \ + "\"unit\": \"Load\"" 1 \ + "\"unit\": \"Sync\"" 1 +} + +# almost never happen since user hardly start a load task after another load task failed. +function test_transfer_two_sources() { + echo "test_transfer_two_sources" + # worker2 offline + ps aux | grep dm-worker2 | awk '{print $2}' | xargs kill || true + check_port_offline $WORKER2_PORT 20 + + # source2 bound to worker3 + run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "list-member -w -n worker3" \ + "\"stage\": \"bound\"" 1 \ + "\"source\": \"mysql-replica-02\"" 1 + run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "query-status load_task2" \ + "different worker in load stage, previous worker: $WORKER2, current worker: $WORKER3" 1 + + # start load task for worker3 + run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "start-task $cur/conf/dm-task3.yaml --remove-meta" \ + "\"result\": true" 2 + run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "query-status load_task3" \ + "\"unit\": \"Load\"" 1 + + # worker2 online + export GO_FAILPOINTS="github.com/pingcap/ticdc/dm/loader/LightningLoadDataSlowDown=sleep(15000)" + run_dm_worker $WORK_DIR/worker2 $WORKER2_PORT $CONF_DIR/dm-worker2.toml + check_rpc_alive $cur/../bin/check_worker_online 127.0.0.1:$WORKER2_PORT + + # worker2 free since (worker3, source2) has load task(load_task3) + run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "list-member -w -n worker2" \ + "\"stage\": \"free\"" 1 + + # worker1 offline + ps aux | grep dm-worker1 | awk '{print $2}' | xargs kill || true + check_port_offline $WORKER1_PORT 20 + + # source1 bound to worker2 + run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "list-member -w -n worker2" \ + "\"stage\": \"bound\"" 1 \ + "\"source\": \"mysql-replica-01\"" 1 + + # start load_task4 on worker2 + run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "start-task $cur/conf/dm-task4.yaml --remove-meta" \ + "\"result\": true" 2 + run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "query-status load_task4" \ + "\"unit\": \"Load\"" 1 + + # worker1 online + export GO_FAILPOINTS="" + run_dm_worker $WORK_DIR/worker1 $WORKER1_PORT $CONF_DIR/dm-worker1.toml + check_rpc_alive $cur/../bin/check_worker_online 127.0.0.1:$WORKER1_PORT + + # worker1 free since (worker2, source1) has load task(load_task4) + run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "list-member -w -n worker1" \ + "\"stage\": \"free\"" 1 + + # now, worker2 waiting worker3 finish load_task3, worker1 waiting worker2 finish load_task4 + # worker3 offline + ps aux | grep dm-worker3 | awk '{print $2}' | xargs kill || true + check_port_offline $WORKER3_PORT 20 + + # source2 bound to worker1 + run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "list-member -w -n worker1" \ + "\"stage\": \"bound\"" 1 \ + "\"source\": \"mysql-replica-02\"" 1 + + # (worker1, source2), (worker2, source1) + run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "query-status load_task1" \ + "different worker in load stage, previous worker: $WORKER1, current worker: $WORKER2" 1 + run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "query-status load_task2" \ + "different worker in load stage, previous worker: $WORKER2, current worker: $WORKER1" 1 + + # worker2 finish load_task4 + # master transfer (worker1, source2), (worker2, source1) to (worker1, source1), (worker2, source2) + run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "list-member -w -n worker1" \ + "\"stage\": \"bound\"" 1 \ + "\"source\": \"mysql-replica-01\"" 1 + run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "list-member -w -n worker2" \ + "\"stage\": \"bound\"" 1 \ + "\"source\": \"mysql-replica-02\"" 1 + + # task1, 2, 4 running, task3 fail + run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "query-status" \ + "\"taskStatus\": \"Running\"" 3 \ + "taskStatus.*Error" 1 + + # worker3 online + export GO_FAILPOINTS="" + run_dm_worker $WORK_DIR/worker3 $WORKER3_PORT $CONF_DIR/dm-worker3.toml + check_rpc_alive $cur/../bin/check_worker_online 127.0.0.1:$WORKER3_PORT + + # source2 bound to worker3 since load_task3 + run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "list-member -w -n worker2" \ + "\"stage\": \"bound\"" 1 \ + "\"source\": \"mysql-replica-02\"" 1 + + # all task running + run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "query-status" \ + "\"taskStatus\": \"Running\"" 4 +} + +function run() { + echo "import prepare data" + run_sql_file $DATA_DIR/db1.prepare.sql $MYSQL_HOST1 $MYSQL_PORT1 $MYSQL_PASSWORD1 + check_contains 'Query OK, 2 rows affected' + run_sql_file $DATA_DIR/db2.prepare.sql $MYSQL_HOST2 $MYSQL_PORT2 $MYSQL_PASSWORD2 + check_contains 'Query OK, 3 rows affected' + + echo "start DM master, workers and sources" + run_dm_master $WORK_DIR/master $MASTER_PORT1 $CONF_DIR/dm-master.toml + check_rpc_alive $cur/../bin/check_master_online 127.0.0.1:$MASTER_PORT1 + + # worker1 loading load_task1 + export GO_FAILPOINTS="github.com/pingcap/ticdc/dm/loader/LightningLoadDataSlowDownByTask=return(\"load_task1\")" + run_dm_worker $WORK_DIR/worker1 $WORKER1_PORT $CONF_DIR/dm-worker1.toml + check_rpc_alive $cur/../bin/check_worker_online 127.0.0.1:$WORKER1_PORT + cp $CONF_DIR/source1.yaml $WORK_DIR/source1.yaml + sed -i "/relay-binlog-name/i\relay-dir: $WORK_DIR/worker1/relay_log" $WORK_DIR/source1.yaml + dmctl_operate_source create $WORK_DIR/source1.yaml $SOURCE_ID1 + + # worker2 loading load_task2 + export GO_FAILPOINTS="github.com/pingcap/ticdc/dm/loader/LightningLoadDataSlowDownByTask=return(\"load_task2\")" + run_dm_worker $WORK_DIR/worker2 $WORKER2_PORT $CONF_DIR/dm-worker2.toml + check_rpc_alive $cur/../bin/check_worker_online 127.0.0.1:$WORKER2_PORT + cp $CONF_DIR/source2.yaml $WORK_DIR/source2.yaml + sed -i "/relay-binlog-name/i\relay-dir: $WORK_DIR/worker2/relay_log" $WORK_DIR/source2.yaml + dmctl_operate_source create $WORK_DIR/source2.yaml $SOURCE_ID2 + + # worker3 loading load_task3 + export GO_FAILPOINTS="github.com/pingcap/ticdc/dm/loader/LightningLoadDataSlowDownByTask=return(\"load_task3\")" + run_dm_worker $WORK_DIR/worker3 $WORKER3_PORT $CONF_DIR/dm-worker3.toml + check_rpc_alive $cur/../bin/check_worker_online 127.0.0.1:$WORKER3_PORT + + echo "start DM task" + dmctl_start_task "$cur/conf/dm-task.yaml" "--remove-meta" + dmctl_start_task "$cur/conf/dm-task2.yaml" "--remove-meta" + + check_log_contain_with_retry 'inject failpoint LightningLoadDataSlowDownByTask' $WORK_DIR/worker1/log/dm-worker.log + check_log_contain_with_retry 'inject failpoint LightningLoadDataSlowDownByTask' $WORK_DIR/worker2/log/dm-worker.log + run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "query-status load_task1" \ + "\"unit\": \"Load\"" 1 \ + "\"unit\": \"Sync\"" 1 + run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "query-status load_task2" \ + "\"unit\": \"Load\"" 1 \ + "\"unit\": \"Sync\"" 1 + + test_worker_restart + + test_transfer_two_sources + + run_sql_file $DATA_DIR/db1.increment.sql $MYSQL_HOST1 $MYSQL_PORT1 $MYSQL_PASSWORD1 + run_sql_file $DATA_DIR/db2.increment.sql $MYSQL_HOST2 $MYSQL_PORT2 $MYSQL_PASSWORD2 + check_sync_diff $WORK_DIR $CONF_DIR/diff_config1.toml + check_sync_diff $WORK_DIR $CONF_DIR/diff_config2.toml + check_sync_diff $WORK_DIR $CONF_DIR/diff_config3.toml + check_sync_diff $WORK_DIR $CONF_DIR/diff_config4.toml +} + +cleanup_data load_task1 +cleanup_data load_task2 +cleanup_data load_task3 +cleanup_data load_task4 +# also cleanup dm processes in case of last run failed +cleanup_process $* +run $* +cleanup_process $* + +echo "[$(date)] <<<<<< test case $TEST_NAME success! >>>>>>" diff --git a/dm/tests/load_task/conf/dm-task.yaml b/dm/tests/load_task/conf/dm-task.yaml index 62ff585d4a2..32cacf0379a 100644 --- a/dm/tests/load_task/conf/dm-task.yaml +++ b/dm/tests/load_task/conf/dm-task.yaml @@ -3,7 +3,6 @@ name: load_task1 task-mode: all is-sharding: false meta-schema: "dm_meta" -# enable-heartbeat: true heartbeat-update-interval: 1 heartbeat-report-interval: 1 diff --git a/dm/tests/load_task/conf/dm-task2.yaml b/dm/tests/load_task/conf/dm-task2.yaml index 6352d396494..14c8b07645d 100644 --- a/dm/tests/load_task/conf/dm-task2.yaml +++ b/dm/tests/load_task/conf/dm-task2.yaml @@ -3,7 +3,6 @@ name: load_task2 task-mode: all is-sharding: false meta-schema: "dm_meta" -# enable-heartbeat: true heartbeat-update-interval: 1 heartbeat-report-interval: 1 diff --git a/dm/tests/load_task/conf/dm-task3.yaml b/dm/tests/load_task/conf/dm-task3.yaml index 133d33b4a18..0a06a4eabfb 100644 --- a/dm/tests/load_task/conf/dm-task3.yaml +++ b/dm/tests/load_task/conf/dm-task3.yaml @@ -3,7 +3,6 @@ name: load_task3 task-mode: all is-sharding: false meta-schema: "dm_meta" -# enable-heartbeat: true heartbeat-update-interval: 1 heartbeat-report-interval: 1 diff --git a/dm/tests/load_task/conf/dm-task4.yaml b/dm/tests/load_task/conf/dm-task4.yaml index e112ea86b3c..7e153eb4c49 100644 --- a/dm/tests/load_task/conf/dm-task4.yaml +++ b/dm/tests/load_task/conf/dm-task4.yaml @@ -3,7 +3,6 @@ name: load_task4 task-mode: all is-sharding: false meta-schema: "dm_meta" -# enable-heartbeat: true heartbeat-update-interval: 1 heartbeat-report-interval: 1 diff --git a/dm/tests/new_relay/conf/source2.yaml b/dm/tests/new_relay/conf/source2.yaml new file mode 100644 index 00000000000..6c272e728c1 --- /dev/null +++ b/dm/tests/new_relay/conf/source2.yaml @@ -0,0 +1,14 @@ +source-id: mysql-replica-02 +server-id: 123456 +flavor: 'mysql' +enable-gtid: true +relay-binlog-name: '' +relay-binlog-gtid: '' +enable-relay: false +from: + host: 127.0.0.1 + user: root + password: /Q7B9DizNLLTTfiZHv9WoEAKamfpIUs= + port: 3307 +checker: + check-enable: false diff --git a/dm/tests/new_relay/run.sh b/dm/tests/new_relay/run.sh index b8ea39b6198..549b1f1f32a 100755 --- a/dm/tests/new_relay/run.sh +++ b/dm/tests/new_relay/run.sh @@ -83,6 +83,82 @@ function test_cant_dail_downstream() { cleanup_data $TEST_NAME } +function test_restart_relay_status() { + cleanup_data $TEST_NAME + cleanup_process + + run_dm_master $WORK_DIR/master $MASTER_PORT $cur/conf/dm-master.toml + check_rpc_alive $cur/../bin/check_master_online 127.0.0.1:$MASTER_PORT + run_dm_worker $WORK_DIR/worker1 $WORKER1_PORT $cur/conf/dm-worker1.toml + check_rpc_alive $cur/../bin/check_worker_online 127.0.0.1:$WORKER1_PORT + + dmctl_operate_source create $cur/conf/source1.yaml $SOURCE_ID1 + + run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "start-relay -s $SOURCE_ID1 worker1" + run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "query-status -s $SOURCE_ID1" \ + "\"result\": true" 2 \ + "\"worker\": \"worker1\"" 1 + + run_dm_worker $WORK_DIR/worker2 $WORKER2_PORT $cur/conf/dm-worker2.toml + check_rpc_alive $cur/../bin/check_worker_online 127.0.0.1:$WORKER2_PORT + + dmctl_operate_source create $cur/conf/source2.yaml $SOURCE_ID2 + + run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "start-relay -s $SOURCE_ID2 worker2" + run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "query-status -s $SOURCE_ID2" \ + "\"result\": true" 2 \ + "\"worker\": \"worker2\"" 1 + + run_dm_worker $WORK_DIR/worker3 $WORKER3_PORT $cur/conf/dm-worker3.toml + check_rpc_alive $cur/../bin/check_worker_online 127.0.0.1:$WORKER3_PORT + + run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "start-relay -s $SOURCE_ID2 worker3" + run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "query-status -s $SOURCE_ID2" \ + "\"result\": true" 3 \ + "\"worker\": \"worker2\"" 1 \ + "\"worker\": \"worker3\"" 1 + + run_dm_ctl $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "list-member -n worker3" \ + "relay" 1 + + kill_dm_worker + kill_dm_master + + run_dm_master $WORK_DIR/master $MASTER_PORT $cur/conf/dm-master.toml + check_rpc_alive $cur/../bin/check_master_online 127.0.0.1:$MASTER_PORT + + run_dm_worker $WORK_DIR/worker1 $WORKER1_PORT $cur/conf/dm-worker1.toml + run_dm_worker $WORK_DIR/worker3 $WORKER3_PORT $cur/conf/dm-worker3.toml + check_rpc_alive $cur/../bin/check_worker_online 127.0.0.1:$WORKER1_PORT + check_rpc_alive $cur/../bin/check_worker_online 127.0.0.1:$WORKER3_PORT + + run_dm_worker $WORK_DIR/worker2 $WORKER2_PORT $cur/conf/dm-worker2.toml + check_rpc_alive $cur/../bin/check_worker_online 127.0.0.1:$WORKER2_PORT + + run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "query-status -s $SOURCE_ID1" \ + "\"result\": true" 2 \ + "\"worker\": \"worker1\"" 1 + + run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "query-status -s $SOURCE_ID2" \ + "\"result\": true" 3 \ + "\"worker\": \"worker2\"" 1 \ + "\"worker\": \"worker3\"" 1 + + run_dm_ctl $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "list-member --worker" \ + "relay" 1 \ + "bound" 2 +} + function test_kill_dump_connection() { cleanup_data $TEST_NAME cleanup_process @@ -108,7 +184,7 @@ function test_kill_dump_connection() { "\"worker\": \"worker1\"" 1 run_sql_source1 "show processlist" - # kill dumop connection to test wheather relay will auto reconnect db + # kill dump connection to test whether relay will auto reconnect db dump_conn_id=$(cat $TEST_DIR/sql_res.$TEST_NAME.txt | grep Binlog -B 4 | grep Id | cut -d : -f2) run_sql_source1 "kill ${dump_conn_id}" @@ -123,6 +199,7 @@ function test_kill_dump_connection() { } function run() { + test_restart_relay_status test_cant_dail_downstream test_cant_dail_upstream diff --git a/dm/tests/shardddl1/run.sh b/dm/tests/shardddl1/run.sh index 4a7583d0976..0b8ed1a536a 100644 --- a/dm/tests/shardddl1/run.sh +++ b/dm/tests/shardddl1/run.sh @@ -242,7 +242,7 @@ function DM_RemoveLock_CASE() { run_sql_source1 "alter table ${shardddl1}.${tb1} add column c double;" run_sql_source2 "alter table ${shardddl1}.${tb1} add column c double;" run_sql_source2 "alter table ${shardddl1}.${tb2} add column c double;" - check_log_contain_with_retry "wait new ddl info putted into etcd" $WORK_DIR/master/log/dm-master.log + check_log_contain_with_retry "wait new ddl info putted into etcd in ${1}" $WORK_DIR/master/log/dm-master.log check_metric_not_contains $MASTER_PORT "dm_master_shard_ddl_error" 3 run_sql_source1 "alter table ${shardddl1}.${tb1} drop column b;" diff --git a/dm/tests/tls/conf/dm-task.yaml b/dm/tests/tls/conf/dm-task.yaml index dd8d2dedc69..932ddac58ec 100644 --- a/dm/tests/tls/conf/dm-task.yaml +++ b/dm/tests/tls/conf/dm-task.yaml @@ -42,5 +42,5 @@ syncers: worker-count: 16 batch: 100 -tidb: - backend: "tidb" +#tidb: +# backend: "tidb" diff --git a/errors.toml b/errors.toml index f45a1592e54..8aa39c02fc7 100755 --- a/errors.toml +++ b/errors.toml @@ -661,6 +661,11 @@ error = ''' prewrite not match, key: %s, start-ts: %d, commit-ts: %d, type: %s, optype: %s ''' +["CDC:ErrProcessorDuplicateOperations"] +error = ''' +table processor duplicate operation, table-id: %d +''' + ["CDC:ErrProcessorEtcdWatch"] error = ''' etcd watch returns error @@ -851,6 +856,16 @@ error = ''' table %d not found in schema snapshot ''' +["CDC:ErrSorterClosed"] +error = ''' +sorter is closed +''' + +["CDC:ErrStartAStoppedLevelDBSystem"] +error = ''' +start a stopped leveldb system +''' + ["CDC:ErrStartTsBeforeGC"] error = ''' fail to create changefeed because start-ts %d is earlier than GC safepoint at %d diff --git a/go.mod b/go.mod index 6a93ea945f7..10f62c66b6a 100644 --- a/go.mod +++ b/go.mod @@ -57,7 +57,7 @@ require ( github.com/pingcap/log v0.0.0-20210906054005-afc726e70354 github.com/pingcap/tidb v1.1.0-beta.0.20211115203106-b076e193b320 github.com/pingcap/tidb-tools v5.2.2-0.20211019062242-37a8bef2fa17+incompatible - github.com/pingcap/tidb/parser v0.0.0-20211115203106-b076e193b320 + github.com/pingcap/tidb/parser v0.0.0-20211117085347-276721877cf8 github.com/pkg/errors v0.9.1 github.com/prometheus/client_golang v1.7.1 github.com/prometheus/client_model v0.2.0 diff --git a/go.sum b/go.sum index fe58549c876..76322854dad 100644 --- a/go.sum +++ b/go.sum @@ -789,8 +789,8 @@ github.com/pingcap/tidb-tools v5.2.2-0.20211019062242-37a8bef2fa17+incompatible github.com/pingcap/tidb-tools v5.2.2-0.20211019062242-37a8bef2fa17+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-20211115203106-b076e193b320 h1:cRM8wXnqwYXM8yelPkJlFvBSRHyNIPiCET/1EbTj29Y= -github.com/pingcap/tidb/parser v0.0.0-20211115203106-b076e193b320/go.mod h1:MAa22tagoj7nv5b1NBcxPkc5CiUNhqj1wuSQnw4f9WE= +github.com/pingcap/tidb/parser v0.0.0-20211117085347-276721877cf8 h1:Rj28acmWKZuDi4WdFl6+S/lZssqafHzNFEV4hATMTDo= +github.com/pingcap/tidb/parser v0.0.0-20211117085347-276721877cf8/go.mod h1:MAa22tagoj7nv5b1NBcxPkc5CiUNhqj1wuSQnw4f9WE= github.com/pingcap/tipb v0.0.0-20211008080435-3fd327dfce0e/go.mod h1:A7mrd7WHBl1o63LE2bIBGEJMTNWXqhgmYiOvMLxozfs= github.com/pingcap/tipb v0.0.0-20211105090418-71142a4d40e3 h1:xnp/Qkk5gELlB8TaY6oro0JNXMBXTafNVxU/vbrNU8I= github.com/pingcap/tipb v0.0.0-20211105090418-71142a4d40e3/go.mod h1:A7mrd7WHBl1o63LE2bIBGEJMTNWXqhgmYiOvMLxozfs= diff --git a/pkg/context/context.go b/pkg/context/context.go index 9c2445de027..1145c1c23c5 100644 --- a/pkg/context/context.go +++ b/pkg/context/context.go @@ -20,6 +20,7 @@ import ( "github.com/pingcap/ticdc/cdc/kv" "github.com/pingcap/ticdc/cdc/model" + "github.com/pingcap/ticdc/cdc/processor/pipeline/system" "github.com/pingcap/ticdc/pkg/config" "github.com/pingcap/ticdc/pkg/etcd" "github.com/pingcap/ticdc/pkg/version" @@ -33,11 +34,12 @@ import ( // the lifecycle of vars in the GlobalVars should be aligned with the ticdc server process. // All field in Vars should be READ-ONLY and THREAD-SAFE type GlobalVars struct { - PDClient pd.Client - KVStorage tidbkv.Storage - CaptureInfo *model.CaptureInfo - EtcdClient *etcd.CDCEtcdClient - GrpcPool kv.GrpcPool + PDClient pd.Client + KVStorage tidbkv.Storage + CaptureInfo *model.CaptureInfo + EtcdClient *etcd.CDCEtcdClient + GrpcPool kv.GrpcPool + TableActorSystem *system.System } // ChangefeedVars contains some vars which can be used anywhere in a pipeline diff --git a/pkg/errors/errors.go b/pkg/errors/errors.go index 785f9cf8f1c..e38810c661a 100644 --- a/pkg/errors/errors.go +++ b/pkg/errors/errors.go @@ -226,6 +226,9 @@ var ( ErrActorStopped = errors.Normalize("actor stopped", errors.RFCCodeText("CDC:ErrActorStopped")) ErrMailboxFull = errors.Normalize("mailbox is full, please try again. Internal use only, report a bug if seen externally", errors.RFCCodeText("CDC:ErrMailboxFull")) + // leveldb sorter errors + ErrStartAStoppedLevelDBSystem = errors.Normalize("start a stopped leveldb system", errors.RFCCodeText("CDC:ErrStartAStoppedLevelDBSystem")) + // workerpool errors ErrWorkerPoolHandleCancelled = errors.Normalize("workerpool handle is cancelled", errors.RFCCodeText("CDC:ErrWorkerPoolHandleCancelled")) ErrAsyncPoolExited = errors.Normalize("asyncPool has exited. Report a bug if seen externally.", errors.RFCCodeText("CDC:ErrAsyncPoolExited")) @@ -245,9 +248,11 @@ var ( ErrConflictingFileLocks = errors.Normalize("file lock conflict: %s", errors.RFCCodeText("ErrConflictingFileLocks")) ErrSortDirLockError = errors.Normalize("error encountered when locking sort-dir", errors.RFCCodeText("ErrSortDirLockError")) ErrLevelDBSorterError = errors.Normalize("leveldb error: %s", errors.RFCCodeText("CDC:ErrLevelDBSorterError")) + ErrSorterClosed = errors.Normalize("sorter is closed", errors.RFCCodeText("CDC:ErrSorterClosed")) // processor errors - ErrTableProcessorStoppedSafely = errors.Normalize("table processor stopped safely", errors.RFCCodeText("CDC:ErrTableProcessorStoppedSafely")) + ErrTableProcessorStoppedSafely = errors.Normalize("table processor stopped safely", errors.RFCCodeText("CDC:ErrTableProcessorStoppedSafely")) + ErrProcessorDuplicateOperations = errors.Normalize("table processor duplicate operation, table-id: %d", errors.RFCCodeText("CDC:ErrProcessorDuplicateOperations")) // owner errors ErrOwnerChangedUnexpectedly = errors.Normalize("owner changed unexpectedly", errors.RFCCodeText("CDC:ErrOwnerChangedUnexpectedly"))