From cf3881c637a178fa43d981ba48f123c43727cea0 Mon Sep 17 00:00:00 2001 From: gmhdbjd Date: Tue, 14 Dec 2021 23:05:46 -0500 Subject: [PATCH 01/31] init logic --- dm/_utils/terror_gen/errors_release.txt | 1 + dm/dm/common/common.go | 6 +- dm/dm/common/common_test.go | 43 ++-- dm/dm/master/scheduler/scheduler.go | 13 ++ dm/dm/master/server.go | 2 +- dm/dm/master/server_test.go | 4 +- dm/dm/master/shardddl/optimist.go | 203 ++++++------------ dm/dm/master/shardddl/optimist_test.go | 111 +++++----- dm/errors.toml | 6 + dm/pkg/shardddl/optimism/info.go | 4 +- dm/pkg/shardddl/optimism/keeper.go | 189 ++++++++++++----- dm/pkg/shardddl/optimism/keeper_test.go | 147 ++++++------- dm/pkg/shardddl/optimism/lock.go | 73 +++++-- dm/pkg/shardddl/optimism/lock_test.go | 32 +-- dm/pkg/shardddl/optimism/ops.go | 8 +- dm/pkg/shardddl/optimism/ops_test.go | 30 +-- dm/pkg/shardddl/optimism/schema.go | 268 ++++++++++++------------ dm/pkg/shardddl/optimism/schema_test.go | 182 ++++++++-------- dm/pkg/terror/error_list.go | 2 + dm/syncer/checkpoint.go | 85 +++++--- dm/syncer/optimist.go | 26 +-- dm/syncer/schema.go | 2 +- dm/syncer/shardddl/optimist.go | 52 +++-- dm/syncer/shardddl/optimist_test.go | 76 +++---- dm/syncer/syncer.go | 53 +++-- 25 files changed, 872 insertions(+), 746 deletions(-) diff --git a/dm/_utils/terror_gen/errors_release.txt b/dm/_utils/terror_gen/errors_release.txt index afd95ce815a..276cef64b63 100644 --- a/dm/_utils/terror_gen/errors_release.txt +++ b/dm/_utils/terror_gen/errors_release.txt @@ -393,6 +393,7 @@ ErrMasterBoundChanging,[code=38052:class=dm-master:scope=internal:level=low], "M ErrMasterFailToImportFromV10x,[code=38053:class=dm-master:scope=internal:level=high], "Message: fail to import DM cluster from v1.0.x, Workaround: Please confirm that you have not violated any restrictions in the upgrade documentation." ErrMasterInconsistentOptimisticDDLsAndInfo,[code=38054:class=dm-master:scope=internal:level=high], "Message: inconsistent count of optimistic ddls and table infos, ddls: %d, table info: %d" ErrMasterOptimisticTableInfoBeforeNotExist,[code=38055:class=dm-master:scope=internal:level=high], "Message: table-info-before not exist in optimistic ddls: %v" +ErrMasterOptimisticDownstreamMetaNotFound,[code=38056:class=dm-master:scope=internal:level=high], "Message: downstream database config and meta for task %s not found" ErrWorkerParseFlagSet,[code=40001:class=dm-worker:scope=internal:level=medium], "Message: parse dm-worker config flag set" ErrWorkerInvalidFlag,[code=40002:class=dm-worker:scope=internal:level=medium], "Message: '%s' is an invalid flag" ErrWorkerDecodeConfigFromFile,[code=40003:class=dm-worker:scope=internal:level=medium], "Message: toml decode file, Workaround: Please check the configuration file has correct TOML format." diff --git a/dm/dm/common/common.go b/dm/dm/common/common.go index b373d9e70d5..6c8d1a187ca 100644 --- a/dm/dm/common/common.go +++ b/dm/dm/common/common.go @@ -78,7 +78,7 @@ var ( ShardDDLOptimismOperationKeyAdapter KeyAdapter = keyHexEncoderDecoder("/dm-master/shardddl-optimism/operation/") // ShardDDLOptimismInitSchemaKeyAdapter is used to store the initial schema (before constructed the lock) of merged tables. // k/v: Encode(task-name, downstream-schema-name, downstream-table-name) -> table schema. - ShardDDLOptimismInitSchemaKeyAdapter KeyAdapter = keyHexEncoderDecoder("/dm-master/shardddl-optimism/init-schema/") + // ShardDDLOptimismInitSchemaKeyAdapter KeyAdapter = keyHexEncoderDecoder("/dm-master/shardddl-optimism/init-schema/") // ShardDDLOptimismDroppedColumnsKeyAdapter is used to store the columns that are not fully dropped // k/v: Encode(lock-id, column-name, source-id, upstream-schema-name, upstream-table-name) -> int // If we don't identify different upstream tables, we may report an error for tb2 in the following case. @@ -106,8 +106,8 @@ func keyAdapterKeysLen(s KeyAdapter) int { ShardDDLPessimismInfoKeyAdapter, ShardDDLPessimismOperationKeyAdapter, ShardDDLOptimismSourceTablesKeyAdapter, LoadTaskKeyAdapter: return 2 - case ShardDDLOptimismInitSchemaKeyAdapter: - return 3 + // case ShardDDLOptimismInitSchemaKeyAdapter: + // return 3 case ShardDDLOptimismInfoKeyAdapter, ShardDDLOptimismOperationKeyAdapter: return 4 case ShardDDLOptimismDroppedColumnsKeyAdapter: diff --git a/dm/dm/common/common_test.go b/dm/dm/common/common_test.go index bc0493e96ac..f037f8b13f1 100644 --- a/dm/dm/common/common_test.go +++ b/dm/dm/common/common_test.go @@ -16,7 +16,6 @@ package common import ( "net" "path" - "strings" "testing" . "github.com/pingcap/check" @@ -61,11 +60,11 @@ func (t *testCommon) TestKeyAdapter(c *C) { adapter: UpstreamSubTaskKeyAdapter, want: "/dm-master/upstream/subtask/6d7973716c31/74657374", }, - { - keys: []string{"test", "target_db", "target_table"}, - adapter: ShardDDLOptimismInitSchemaKeyAdapter, - want: "/dm-master/shardddl-optimism/init-schema/74657374/7461726765745f6462/7461726765745f7461626c65", - }, + // { + // keys: []string{"test", "target_db", "target_table"}, + // adapter: ShardDDLOptimismInitSchemaKeyAdapter, + // want: "/dm-master/shardddl-optimism/init-schema/74657374/7461726765745f6462/7461726765745f7461626c65", + // }, { keys: []string{"test", "mysql_replica_01", "target_db", "target_table"}, adapter: ShardDDLOptimismInfoKeyAdapter, @@ -108,11 +107,11 @@ func (t *testCommon) TestEncodeAsPrefix(c *C) { adapter: UpstreamSubTaskKeyAdapter, want: "/dm-master/upstream/subtask/6d7973716c31/", }, - { - keys: []string{"test", "target_db"}, - adapter: ShardDDLOptimismInitSchemaKeyAdapter, - want: "/dm-master/shardddl-optimism/init-schema/74657374/7461726765745f6462/", - }, + // { + // keys: []string{"test", "target_db"}, + // adapter: ShardDDLOptimismInitSchemaKeyAdapter, + // want: "/dm-master/shardddl-optimism/init-schema/74657374/7461726765745f6462/", + // }, } for _, ca := range testCases { @@ -122,17 +121,17 @@ func (t *testCommon) TestEncodeAsPrefix(c *C) { c.Assert(err, NotNil) } - keys := []string{"test", "target_db", "target_table"} - fullEncodedKey := ShardDDLOptimismInitSchemaKeyAdapter.Encode(keys...) - prefixEncodedKey := ShardDDLOptimismInitSchemaKeyAdapter.Encode(keys[:len(keys)-1]...) - c.Assert(strings.HasPrefix(fullEncodedKey, prefixEncodedKey), IsTrue) - - keys2 := []string{"test", "target_db_2", "target_table_2"} - fullEncodedKey2 := ShardDDLOptimismInitSchemaKeyAdapter.Encode(keys2...) - prefixEncodedKey2 := ShardDDLOptimismInitSchemaKeyAdapter.Encode(keys2[:len(keys2)-1]...) - - c.Assert(strings.HasPrefix(fullEncodedKey, prefixEncodedKey2), IsFalse) - c.Assert(strings.HasPrefix(fullEncodedKey2, prefixEncodedKey), IsFalse) + // keys := []string{"test", "target_db", "target_table"} + // fullEncodedKey := ShardDDLOptimismInitSchemaKeyAdapter.Encode(keys...) + // prefixEncodedKey := ShardDDLOptimismInitSchemaKeyAdapter.Encode(keys[:len(keys)-1]...) + // c.Assert(strings.HasPrefix(fullEncodedKey, prefixEncodedKey), IsTrue) + // + // keys2 := []string{"test", "target_db_2", "target_table_2"} + // fullEncodedKey2 := ShardDDLOptimismInitSchemaKeyAdapter.Encode(keys2...) + // prefixEncodedKey2 := ShardDDLOptimismInitSchemaKeyAdapter.Encode(keys2[:len(keys2)-1]...) + // + // c.Assert(strings.HasPrefix(fullEncodedKey, prefixEncodedKey2), IsFalse) + // c.Assert(strings.HasPrefix(fullEncodedKey2, prefixEncodedKey), IsFalse) } func (t *testCommon) TestIsErrNetClosing(c *C) { diff --git a/dm/dm/master/scheduler/scheduler.go b/dm/dm/master/scheduler/scheduler.go index 5542ff87dc7..6c634f9a252 100644 --- a/dm/dm/master/scheduler/scheduler.go +++ b/dm/dm/master/scheduler/scheduler.go @@ -894,6 +894,19 @@ func (s *Scheduler) getSubTaskCfgByTaskSource(task, source string) *config.SubTa return &clone } +// GetDownstreamMetaDBCfgByTask gets downstream db config and meta config by task name. +func (s *Scheduler) GetDownstreamMetaByTask(task string) (*config.DBConfig, string) { + v, ok := s.subTaskCfgs.Load(task) + if !ok { + return nil, "" + } + cfgM := v.(map[string]config.SubTaskConfig) + for _, cfg := range cfgM { + return cfg.To.Clone(), cfg.MetaSchema + } + return nil, "" +} + // GetSubTaskCfgsByTask gets subtask configs' map by task name. func (s *Scheduler) GetSubTaskCfgsByTask(task string) map[string]*config.SubTaskConfig { v, ok := s.subTaskCfgs.Load(task) diff --git a/dm/dm/master/server.go b/dm/dm/master/server.go index a6077756c85..f58902ebe55 100644 --- a/dm/dm/master/server.go +++ b/dm/dm/master/server.go @@ -131,7 +131,7 @@ func NewServer(cfg *Config) *Server { ap: NewAgentPool(&RateLimitConfig{rate: cfg.RPCRateLimit, burst: cfg.RPCRateBurst}), } server.pessimist = shardddl.NewPessimist(&logger, server.getTaskResources) - server.optimist = shardddl.NewOptimist(&logger) + server.optimist = shardddl.NewOptimist(&logger, server.scheduler.GetDownstreamMetaByTask) server.closed.Store(true) setUseTLS(&cfg.Security) diff --git a/dm/dm/master/server_test.go b/dm/dm/master/server_test.go index 2497843313a..a3ba03b77ed 100644 --- a/dm/dm/master/server_test.go +++ b/dm/dm/master/server_test.go @@ -958,7 +958,7 @@ func (t *testMaster) TestStartTaskWithRemoveMeta(c *check.C) { server.scheduler, _ = t.testMockScheduler(ctx, &wg, c, sources, workers, "", makeWorkerClientsForHandle(ctrl, taskName, sources, workers, req)) server.pessimist = shardddl.NewPessimist(&logger, func(task string) []string { return sources }) - server.optimist = shardddl.NewOptimist(&logger) + server.optimist = shardddl.NewOptimist(&logger, server.scheduler.GetDownstreamMetaByTask) var ( DDLs = []string{"ALTER TABLE bar ADD COLUMN c1 INT"} @@ -1042,7 +1042,7 @@ func (t *testMaster) TestStartTaskWithRemoveMeta(c *check.C) { server.scheduler, _ = t.testMockScheduler(ctx, &wg, c, sources, workers, "", makeWorkerClientsForHandle(ctrl, taskName, sources, workers, req)) server.pessimist = shardddl.NewPessimist(&logger, func(task string) []string { return sources }) - server.optimist = shardddl.NewOptimist(&logger) + server.optimist = shardddl.NewOptimist(&logger, server.scheduler.GetDownstreamMetaByTask) var ( p = parser.New() diff --git a/dm/dm/master/shardddl/optimist.go b/dm/dm/master/shardddl/optimist.go index eda57ea0332..2a82d202a74 100644 --- a/dm/dm/master/shardddl/optimist.go +++ b/dm/dm/master/shardddl/optimist.go @@ -22,7 +22,6 @@ import ( "github.com/pingcap/failpoint" "github.com/pingcap/tidb-tools/pkg/dbutil" - "github.com/pingcap/tidb-tools/pkg/schemacmp" "go.etcd.io/etcd/clientv3" "go.uber.org/zap" @@ -53,11 +52,11 @@ type Optimist struct { } // NewOptimist creates a new Optimist instance. -func NewOptimist(pLogger *log.Logger) *Optimist { +func NewOptimist(pLogger *log.Logger, downstreamMetaFunc func(string) (*config.DBConfig, string)) *Optimist { return &Optimist{ logger: pLogger.WithFields(zap.String("component", "shard DDL optimist")), closed: true, - lk: optimism.NewLockKeeper(), + lk: optimism.NewLockKeeper(downstreamMetaFunc), tk: optimism.NewTableKeeper(), } } @@ -250,11 +249,11 @@ func (o *Optimist) rebuildLocks() (revSource, revInfo, revOperation int64, err e } o.logger.Info("get history shard DDL lock operation", zap.Int64("revision", revOperation)) - initSchemas, revInitSchemas, err := optimism.GetAllInitSchemas(o.cli) - if err != nil { - return 0, 0, 0, err - } - o.logger.Info("get all init schemas", zap.Int64("revision", revInitSchemas)) + // initSchemas, revInitSchemas, err := optimism.GetAllInitSchemas(o.cli) + // if err != nil { + // return 0, 0, 0, err + // } + // o.logger.Info("get all init schemas", zap.Int64("revision", revInitSchemas)) colm, _, err := optimism.GetAllDroppedColumns(o.cli) if err != nil { @@ -262,9 +261,10 @@ func (o *Optimist) rebuildLocks() (revSource, revInfo, revOperation int64, err e // then these unexpected columns can be handled by the user. o.logger.Error("fail to recover colms", log.ShortError(err)) } + o.lk.SetDropColumns(colm) // recover the shard DDL lock based on history shard DDL info & lock operation. - err = o.recoverLocks(ifm, opm, colm, initSchemas) + err = o.recoverLocks(ifm, opm) if err != nil { // only log the error, and don't return it to forbid the startup of the DM-master leader. // then these unexpected locks can be handled by the user. @@ -295,96 +295,11 @@ func sortInfos(ifm map[string]map[string]map[string]map[string]optimism.Info) [] return infos } -// buildLockJoinedAndTTS build joined table and target table slice for lock by history infos. -func (o *Optimist) buildLockJoinedAndTTS( - ifm map[string]map[string]map[string]map[string]optimism.Info, - initSchemas map[string]map[string]map[string]optimism.InitSchema) ( - map[string]schemacmp.Table, map[string][]optimism.TargetTable, - map[string]map[string]map[string]map[string]schemacmp.Table) { - type infoKey struct { - lockID string - source string - upSchema string - upTable string - } - infos := make(map[infoKey]optimism.Info) - lockTTS := make(map[string][]optimism.TargetTable) - for _, taskInfos := range ifm { - for _, sourceInfos := range taskInfos { - for _, schemaInfos := range sourceInfos { - for _, info := range schemaInfos { - lockID := utils.GenDDLLockID(info.Task, info.DownSchema, info.DownTable) - if _, ok := lockTTS[lockID]; !ok { - lockTTS[lockID] = o.tk.FindTables(info.Task, info.DownSchema, info.DownTable) - } - infos[infoKey{lockID, info.Source, info.UpSchema, info.UpTable}] = info - } - } - } - } - - lockJoined := make(map[string]schemacmp.Table) - missTable := make(map[string]map[string]map[string]map[string]schemacmp.Table) - for lockID, tts := range lockTTS { - for _, tt := range tts { - for upSchema, tables := range tt.UpTables { - for upTable := range tables { - var table schemacmp.Table - if info, ok := infos[infoKey{lockID, tt.Source, upSchema, upTable}]; ok && info.TableInfoBefore != nil { - table = schemacmp.Encode(info.TableInfoBefore) - } else if initSchema, ok := initSchemas[tt.Task][tt.DownSchema][tt.DownTable]; ok { - // If there is no optimism.Info for a upstream table, it indicates the table structure - // hasn't been changed since last removeLock. So the init schema should be its table info. - table = schemacmp.Encode(initSchema.TableInfo) - if _, ok := missTable[lockID]; !ok { - missTable[lockID] = make(map[string]map[string]map[string]schemacmp.Table) - } - if _, ok := missTable[lockID][tt.Source]; !ok { - missTable[lockID][tt.Source] = make(map[string]map[string]schemacmp.Table) - } - if _, ok := missTable[lockID][tt.Source][upSchema]; !ok { - missTable[lockID][tt.Source][upSchema] = make(map[string]schemacmp.Table) - } - missTable[lockID][tt.Source][upSchema][upTable] = table - } else { - o.logger.Error( - "can not find table info for upstream table", - zap.String("source", tt.Source), - zap.String("up-schema", upSchema), - zap.String("up-table", upTable), - ) - continue - } - if joined, ok := lockJoined[lockID]; !ok { - lockJoined[lockID] = table - } else { - newJoined, err := joined.Join(table) - // ignore error, will report it in TrySync later - if err != nil { - o.logger.Error(fmt.Sprintf("fail to join table info %s with %s, lockID: %s in recover lock", joined, newJoined, lockID), log.ShortError(err)) - } else { - lockJoined[lockID] = newJoined - } - } - } - } - } - } - return lockJoined, lockTTS, missTable -} - // recoverLocks recovers shard DDL locks based on shard DDL info and shard DDL lock operation. func (o *Optimist) recoverLocks( ifm map[string]map[string]map[string]map[string]optimism.Info, opm map[string]map[string]map[string]map[string]optimism.Operation, - colm map[string]map[string]map[string]map[string]map[string]optimism.DropColumnStage, - initSchemas map[string]map[string]map[string]optimism.InitSchema) error { - // construct joined table based on the shard DDL info. - o.logger.Info("build lock joined and tts") - lockJoined, lockTTS, missTable := o.buildLockJoinedAndTTS(ifm, initSchemas) - // build lock and restore table info - o.logger.Info("rebuild locks and tables") - o.lk.RebuildLocksAndTables(o.cli, ifm, colm, lockJoined, lockTTS, missTable) +) error { // sort infos by revision infos := sortInfos(ifm) var firstErr error @@ -395,12 +310,17 @@ func (o *Optimist) recoverLocks( } for _, info := range infos { + if info.IsDeleted { + // TODO: handle drop table + } + if !o.tk.SourceTableExist(info.Task, info.Source, info.UpSchema, info.UpTable, info.DownSchema, info.DownTable) { + continue + } // never mark the lock operation from `done` to `not-done` when recovering. err := o.handleInfo(info, true) if err != nil { o.logger.Error("fail to handle info while recovering locks", zap.Error(err)) setFirstErr(err) - continue } } @@ -419,7 +339,6 @@ func (o *Optimist) recoverLocks( err := lock.DeleteColumnsByOp(op) if err != nil { o.logger.Error("fail to update lock columns", zap.Error(err)) - continue } } } @@ -595,43 +514,45 @@ func (o *Optimist) handleOperationPut(ctx context.Context, opCh <-chan optimism. // avoid new ddl added while previous ddl resolved and remove lock // change lock granularity if needed o.mu.Lock() - lock := o.lk.FindLock(op.ID) - if lock == nil { - o.logger.Warn("no lock for the shard DDL lock operation exist", zap.Stringer("operation", op)) - o.mu.Unlock() - continue - } - - err := lock.DeleteColumnsByOp(op) - if err != nil { - o.logger.Error("fail to update lock columns", zap.Error(err)) - } - // in optimistic mode, we always try to mark a table as done after received the `done` status of the DDLs operation. - // NOTE: even all tables have done their previous DDLs operations, the lock may still not resolved, - // because these tables may have different schemas. - done := lock.TryMarkDone(op.Source, op.UpSchema, op.UpTable) - o.logger.Info("mark operation for a table as done", zap.Bool("done", done), zap.Stringer("operation", op)) - if !lock.IsResolved() { - o.logger.Info("the lock is still not resolved", zap.Stringer("operation", op)) - o.mu.Unlock() - continue - } - - // the lock has done, remove the lock. - o.logger.Info("the lock for the shard DDL lock operation has been resolved", zap.Stringer("operation", op)) - deleted, err := o.removeLock(lock) - if err != nil { - o.logger.Error("fail to delete the shard DDL infos and lock operations", zap.String("lock", lock.ID), log.ShortError(err)) - metrics.ReportDDLError(op.Task, metrics.OpErrRemoveLock) - } - if deleted { - o.logger.Info("the shard DDL infos and lock operations have been cleared", zap.Stringer("operation", op)) - } + o.handleOperation(op) o.mu.Unlock() } } } +func (o *Optimist) handleOperation(op optimism.Operation) { + lock := o.lk.FindLock(op.ID) + if lock == nil { + o.logger.Warn("no lock for the shard DDL lock operation exist", zap.Stringer("operation", op)) + return + } + + err := lock.DeleteColumnsByOp(op) + if err != nil { + o.logger.Error("fail to update lock columns", zap.Error(err)) + } + // in optimistic mode, we always try to mark a table as done after received the `done` status of the DDLs operation. + // NOTE: even all tables have done their previous DDLs operations, the lock may still not resolved, + // because these tables may have different schemas. + done := lock.TryMarkDone(op.Source, op.UpSchema, op.UpTable) + o.logger.Info("mark operation for a table as done", zap.Bool("done", done), zap.Stringer("operation", op)) + if !lock.IsResolved() { + o.logger.Info("the lock is still not resolved", zap.Stringer("operation", op)) + return + } + + // the lock has done, remove the lock. + o.logger.Info("the lock for the shard DDL lock operation has been resolved", zap.Stringer("operation", op)) + deleted, err := o.removeLock(lock) + if err != nil { + o.logger.Error("fail to delete the shard DDL infos and lock operations", zap.String("lock", lock.ID), log.ShortError(err)) + metrics.ReportDDLError(op.Task, metrics.OpErrRemoveLock) + } + if deleted { + o.logger.Info("the shard DDL infos and lock operations have been cleared", zap.Stringer("operation", op)) + } +} + // handleLock handles a single shard DDL lock. func (o *Optimist) handleLock(info optimism.Info, tts []optimism.TargetTable, skipDone bool) error { cfStage := optimism.ConflictNone @@ -651,16 +572,16 @@ func (o *Optimist) handleLock(info optimism.Info, tts []optimism.TargetTable, sk zap.String("lock", lockID), zap.Strings("ddls", newDDLs), zap.Strings("cols", cols), zap.String("info", info.ShortString()), zap.Bool("is deleted", info.IsDeleted)) // try to record the init schema before applied the DDL to the downstream. - initSchema := optimism.NewInitSchema(info.Task, info.DownSchema, info.DownTable, info.TableInfoBefore) - rev, putted, err2 := optimism.PutInitSchemaIfNotExist(o.cli, initSchema) - switch { - case err2 != nil: - return err2 - case putted: - o.logger.Info("recorded the initial schema", zap.String("info", info.ShortString())) - default: - o.logger.Debug("skip to record the initial schema", zap.String("info", info.ShortString()), zap.Int64("revision", rev)) - } + // initSchema := optimism.NewInitSchema(info.Task, info.DownSchema, info.DownTable, info.TableInfoBefore) + // rev, putted, err2 := optimism.PutInitSchemaIfNotExist(o.cli, initSchema) + // switch { + // case err2 != nil: + // return err2 + // case putted: + // o.logger.Info("recorded the initial schema", zap.String("info", info.ShortString())) + // default: + // o.logger.Debug("skip to record the initial schema", zap.String("info", info.ShortString()), zap.Int64("revision", rev)) + // } } lock := o.lk.FindLock(lockID) @@ -761,8 +682,8 @@ func (o *Optimist) deleteInfosOps(lock *optimism.Lock) (bool, error) { } } // NOTE: we rely on only `task`, `downSchema`, and `downTable` used for deletion. - initSchema := optimism.NewInitSchema(lock.Task, lock.DownSchema, lock.DownTable, nil) - rev, deleted, err := optimism.DeleteInfosOperationsSchemaColumn(o.cli, infos, ops, initSchema) + // initSchema := optimism.NewInitSchema(lock.Task, lock.DownSchema, lock.DownTable, nil) + rev, deleted, err := optimism.DeleteInfosOperationsSchemaColumn(o.cli, infos, ops, lock.Task, lock.DownSchema, lock.DownTable) if err != nil { return deleted, err } diff --git a/dm/dm/master/shardddl/optimist_test.go b/dm/dm/master/shardddl/optimist_test.go index b0ba6739437..166f199027b 100644 --- a/dm/dm/master/shardddl/optimist_test.go +++ b/dm/dm/master/shardddl/optimist_test.go @@ -20,7 +20,6 @@ import ( . "github.com/pingcap/check" "github.com/pingcap/tidb-tools/pkg/dbutil" - "github.com/pingcap/tidb-tools/pkg/schemacmp" tiddl "github.com/pingcap/tidb/ddl" "github.com/pingcap/tidb/parser" "github.com/pingcap/tidb/parser/ast" @@ -104,7 +103,7 @@ func (t *testOptimist) TestOptimistSourceTables(c *C) { var ( logger = log.L() - o = NewOptimist(&logger) + o = NewOptimist(&logger, getDownstreamMeta) task = "task" source1 = "mysql-replica-1" source2 = "mysql-replica-2" @@ -164,7 +163,7 @@ func (t *testOptimist) TestOptimistSourceTables(c *C) { o.Close() // CASE 4: create (not re-start) a new optimist with previous source tables. - o = NewOptimist(&logger) + o = NewOptimist(&logger, getDownstreamMeta) c.Assert(o.Start(ctx, etcdTestCli), IsNil) tts = o.tk.FindTables(task, downSchema, downTable) c.Assert(tts, HasLen, 2) @@ -204,7 +203,7 @@ func (t *testOptimist) testOptimist(c *C, cli *clientv3.Client, restart int) { backOff = 30 waitTime = 100 * time.Millisecond logger = log.L() - o = NewOptimist(&logger) + o = NewOptimist(&logger, getDownstreamMeta) rebuildOptimist = func(ctx context.Context) { switch restart { @@ -213,7 +212,7 @@ func (t *testOptimist) testOptimist(c *C, cli *clientv3.Client, restart int) { c.Assert(o.Start(ctx, cli), IsNil) case restartNewInstance: o.Close() - o = NewOptimist(&logger) + o = NewOptimist(&logger, getDownstreamMeta) c.Assert(o.Start(ctx, cli), IsNil) } } @@ -615,7 +614,7 @@ func (t *testOptimist) TestOptimistLockConflict(c *C) { var ( watchTimeout = 2 * time.Second logger = log.L() - o = NewOptimist(&logger) + o = NewOptimist(&logger, getDownstreamMeta) task = "task-test-optimist" source1 = "mysql-replica-1" downSchema = "foo" @@ -709,7 +708,7 @@ func (t *testOptimist) TestOptimistLockMultipleTarget(c *C) { waitTime = 100 * time.Millisecond watchTimeout = 2 * time.Second logger = log.L() - o = NewOptimist(&logger) + o = NewOptimist(&logger, getDownstreamMeta) task = "test-optimist-lock-multiple-target" source = "mysql-replica-1" upSchema = "foo" @@ -894,7 +893,7 @@ func (t *testOptimist) TestOptimistInitSchema(c *C) { waitTime = 100 * time.Millisecond watchTimeout = 2 * time.Second logger = log.L() - o = NewOptimist(&logger) + o = NewOptimist(&logger, getDownstreamMeta) task = "test-optimist-init-schema" source = "mysql-replica-1" upSchema = "foo" @@ -930,9 +929,9 @@ func (t *testOptimist) TestOptimistInitSchema(c *C) { c.Assert(o.Locks(), HasLen, 0) // no init schema exist now. - is, _, err := optimism.GetInitSchema(etcdTestCli, task, downSchema, downTable) - c.Assert(err, IsNil) - c.Assert(is.IsEmpty(), IsTrue) + // is, _, err := optimism.GetInitSchema(etcdTestCli, task, downSchema, downTable) + // c.Assert(err, IsNil) + // c.Assert(is.IsEmpty(), IsTrue) // PUT i11, will creat a lock. _, err = optimism.PutInfo(etcdTestCli, i11) @@ -943,9 +942,9 @@ func (t *testOptimist) TestOptimistInitSchema(c *C) { time.Sleep(waitTime) // sleep one more time to wait for update of init schema. // the init schema exist now. - is, _, err = optimism.GetInitSchema(etcdTestCli, task, downSchema, downTable) - c.Assert(err, IsNil) - c.Assert(is.TableInfo, DeepEquals, ti0) // the init schema. + // is, _, err = optimism.GetInitSchema(etcdTestCli, task, downSchema, downTable) + // c.Assert(err, IsNil) + // c.Assert(is.TableInfo, DeepEquals, ti0) // the init schema. // PUT i12, the lock will be synced. rev1, err := optimism.PutInfo(etcdTestCli, i12) @@ -982,9 +981,9 @@ func (t *testOptimist) TestOptimistInitSchema(c *C) { }), IsTrue) // the init schema should also be deleted. - is, _, err = optimism.GetInitSchema(etcdTestCli, task, downSchema, downTable) - c.Assert(err, IsNil) - c.Assert(is.IsEmpty(), IsTrue) + // is, _, err = optimism.GetInitSchema(etcdTestCli, task, downSchema, downTable) + // c.Assert(err, IsNil) + // c.Assert(is.IsEmpty(), IsTrue) // PUT i21 to create the lock again. _, err = optimism.PutInfo(etcdTestCli, i21) @@ -995,9 +994,9 @@ func (t *testOptimist) TestOptimistInitSchema(c *C) { time.Sleep(waitTime) // sleep one more time to wait for update of init schema. // the init schema exist now. - is, _, err = optimism.GetInitSchema(etcdTestCli, task, downSchema, downTable) - c.Assert(err, IsNil) - c.Assert(is.TableInfo, DeepEquals, ti1) // the init schema is ti1 now. + // is, _, err = optimism.GetInitSchema(etcdTestCli, task, downSchema, downTable) + // c.Assert(err, IsNil) + // c.Assert(is.TableInfo, DeepEquals, ti1) // the init schema is ti1 now. } func (t *testOptimist) testSortInfos(c *C, cli *clientv3.Client) { @@ -1074,7 +1073,7 @@ func (t *testOptimist) TestBuildLockJoinedAndTable(c *C) { var ( logger = log.L() - o = NewOptimist(&logger) + o = NewOptimist(&logger, getDownstreamMeta) task = "task" source1 = "mysql-replica-1" source2 = "mysql-replica-2" @@ -1117,18 +1116,18 @@ func (t *testOptimist) TestBuildLockJoinedAndTable(c *C) { c.Assert(err, IsNil) o.tk.Init(stm) - ifm, _, err := optimism.GetAllInfo(etcdTestCli) - c.Assert(err, IsNil) - - lockJoined, lockTTS, missTable := o.buildLockJoinedAndTTS(ifm, nil) - c.Assert(len(lockJoined), Equals, 1) - c.Assert(len(lockTTS), Equals, 1) - c.Assert(len(missTable), Equals, 0) - joined, ok := lockJoined[utils.GenDDLLockID(task, downSchema, downTable)] - c.Assert(ok, IsTrue) - cmp, err := joined.Compare(schemacmp.Encode(ti2)) - c.Assert(err, IsNil) - c.Assert(cmp, Equals, 0) + // ifm, _, err := optimism.GetAllInfo(etcdTestCli) + // c.Assert(err, IsNil) + // + // lockJoined, lockTTS, missTable := o.buildLockJoinedAndTTS(ifm, nil) + // c.Assert(len(lockJoined), Equals, 1) + // c.Assert(len(lockTTS), Equals, 1) + // c.Assert(len(missTable), Equals, 0) + // joined, ok := lockJoined[utils.GenDDLLockID(task, downSchema, downTable)] + // c.Assert(ok, IsTrue) + // cmp, err := joined.Compare(schemacmp.Encode(ti2)) + // c.Assert(err, IsNil) + // c.Assert(cmp, Equals, 0) } func (t *testOptimist) TestBuildLockWithInitSchema(c *C) { @@ -1136,7 +1135,7 @@ func (t *testOptimist) TestBuildLockWithInitSchema(c *C) { var ( logger = log.L() - o = NewOptimist(&logger) + o = NewOptimist(&logger, getDownstreamMeta) task = "task" source1 = "mysql-replica-1" source2 = "mysql-replica-2" @@ -1152,11 +1151,11 @@ func (t *testOptimist) TestBuildLockWithInitSchema(c *C) { ti1 = createTableInfo(c, p, se, tblID, `CREATE TABLE bar (a INT PRIMARY KEY, b INT)`) ti2 = createTableInfo(c, p, se, tblID, `CREATE TABLE bar (a INT PRIMARY KEY)`) - ddlDropB = "ALTER TABLE bar DROP COLUMN b" - ddlDropC = "ALTER TABLE bar DROP COLUMN c" - infoDropB = optimism.NewInfo(task, source1, "foo", "bar-1", downSchema, downTable, []string{ddlDropC}, ti0, []*model.TableInfo{ti1}) - infoDropC = optimism.NewInfo(task, source1, "foo", "bar-1", downSchema, downTable, []string{ddlDropB}, ti1, []*model.TableInfo{ti2}) - initSchema = optimism.NewInitSchema(task, downSchema, downTable, ti0) + ddlDropB = "ALTER TABLE bar DROP COLUMN b" + ddlDropC = "ALTER TABLE bar DROP COLUMN c" + infoDropB = optimism.NewInfo(task, source1, "foo", "bar-1", downSchema, downTable, []string{ddlDropC}, ti0, []*model.TableInfo{ti1}) + infoDropC = optimism.NewInfo(task, source1, "foo", "bar-1", downSchema, downTable, []string{ddlDropB}, ti1, []*model.TableInfo{ti2}) + // initSchema = optimism.NewInitSchema(task, downSchema, downTable, ti0) ) ctx, cancel := context.WithCancel(context.Background()) @@ -1180,20 +1179,24 @@ func (t *testOptimist) TestBuildLockWithInitSchema(c *C) { c.Assert(err, IsNil) o.tk.Init(stm) - ifm, _, err := optimism.GetAllInfo(etcdTestCli) - c.Assert(err, IsNil) + // ifm, _, err := optimism.GetAllInfo(etcdTestCli) + // c.Assert(err, IsNil) + // + // initSchemas := map[string]map[string]map[string]optimism.InitSchema{task: {downSchema: {downTable: initSchema}}} + // lockJoined, lockTTS, missTable := o.buildLockJoinedAndTTS(ifm, initSchemas) + // c.Assert(len(lockJoined), Equals, 1) + // c.Assert(len(lockTTS), Equals, 1) + // c.Assert(len(missTable), Equals, 1) + // cmp, err := missTable[utils.GenDDLLockID(task, downSchema, downTable)][source2]["foo"]["bar-1"].Compare(schemacmp.Encode(initSchema.TableInfo)) + // c.Assert(err, IsNil) + // c.Assert(cmp, Equals, 0) + // joined, ok := lockJoined[utils.GenDDLLockID(task, downSchema, downTable)] + // c.Assert(ok, IsTrue) + // cmp, err = joined.Compare(schemacmp.Encode(ti0)) + // c.Assert(err, IsNil) + // c.Assert(cmp, Equals, 0) +} - initSchemas := map[string]map[string]map[string]optimism.InitSchema{task: {downSchema: {downTable: initSchema}}} - lockJoined, lockTTS, missTable := o.buildLockJoinedAndTTS(ifm, initSchemas) - c.Assert(len(lockJoined), Equals, 1) - c.Assert(len(lockTTS), Equals, 1) - c.Assert(len(missTable), Equals, 1) - cmp, err := missTable[utils.GenDDLLockID(task, downSchema, downTable)][source2]["foo"]["bar-1"].Compare(schemacmp.Encode(initSchema.TableInfo)) - c.Assert(err, IsNil) - c.Assert(cmp, Equals, 0) - joined, ok := lockJoined[utils.GenDDLLockID(task, downSchema, downTable)] - c.Assert(ok, IsTrue) - cmp, err = joined.Compare(schemacmp.Encode(ti0)) - c.Assert(err, IsNil) - c.Assert(cmp, Equals, 0) +func getDownstreamMeta(string) (*config.DBConfig, string) { + return nil, "" } diff --git a/dm/errors.toml b/dm/errors.toml index e964a55155b..7f8c7faac7d 100644 --- a/dm/errors.toml +++ b/dm/errors.toml @@ -2368,6 +2368,12 @@ description = "" workaround = "" tags = ["internal", "high"] +[error.DM-dm-master-38056] +message = "downstream database config and meta for task %s not found" +description = "" +workaround = "" +tags = ["internal", "high"] + [error.DM-dm-worker-40001] message = "parse dm-worker config flag set" description = "" diff --git a/dm/pkg/shardddl/optimism/info.go b/dm/pkg/shardddl/optimism/info.go index ae703290c14..0ca280a0245 100644 --- a/dm/pkg/shardddl/optimism/info.go +++ b/dm/pkg/shardddl/optimism/info.go @@ -304,9 +304,9 @@ func ClearTestInfoOperationSchema(cli *clientv3.Client) error { clearSource := clientv3.OpDelete(common.ShardDDLOptimismSourceTablesKeyAdapter.Path(), clientv3.WithPrefix()) clearInfo := clientv3.OpDelete(common.ShardDDLOptimismInfoKeyAdapter.Path(), clientv3.WithPrefix()) clearOp := clientv3.OpDelete(common.ShardDDLOptimismOperationKeyAdapter.Path(), clientv3.WithPrefix()) - clearISOp := clientv3.OpDelete(common.ShardDDLOptimismInitSchemaKeyAdapter.Path(), clientv3.WithPrefix()) + // clearISOp := clientv3.OpDelete(common.ShardDDLOptimismInitSchemaKeyAdapter.Path(), clientv3.WithPrefix()) clearColumns := clientv3.OpDelete(common.ShardDDLOptimismDroppedColumnsKeyAdapter.Path(), clientv3.WithPrefix()) - _, err := cli.Txn(context.Background()).Then(clearSource, clearInfo, clearOp, clearISOp, clearColumns).Commit() + _, err := cli.Txn(context.Background()).Then(clearSource, clearInfo, clearOp, clearColumns).Commit() return err } diff --git a/dm/pkg/shardddl/optimism/keeper.go b/dm/pkg/shardddl/optimism/keeper.go index 8fc57437784..81f2eeaba08 100644 --- a/dm/pkg/shardddl/optimism/keeper.go +++ b/dm/pkg/shardddl/optimism/keeper.go @@ -20,81 +20,123 @@ import ( "github.com/pingcap/tidb-tools/pkg/schemacmp" "go.etcd.io/etcd/clientv3" + "github.com/pingcap/ticdc/dm/dm/config" + "github.com/pingcap/ticdc/dm/pkg/conn" + "github.com/pingcap/ticdc/dm/pkg/log" "github.com/pingcap/ticdc/dm/pkg/terror" "github.com/pingcap/ticdc/dm/pkg/utils" ) +type DownstreamMeta struct { + db *conn.BaseDB + meta string +} + // LockKeeper used to keep and handle DDL lock conveniently. // The lock information do not need to be persistent, and can be re-constructed from the shard DDL info. type LockKeeper struct { mu sync.RWMutex locks map[string]*Lock // lockID -> Lock + + downstreamMetaMap map[string]*DownstreamMeta + downDBFunc func(string) (*config.DBConfig, string) + dropColumns map[string]map[string]map[string]map[string]map[string]DropColumnStage } // NewLockKeeper creates a new LockKeeper instance. -func NewLockKeeper() *LockKeeper { +func NewLockKeeper(downDBFunc func(string) (*config.DBConfig, string)) *LockKeeper { return &LockKeeper{ - locks: make(map[string]*Lock), + locks: make(map[string]*Lock), + downstreamMetaMap: make(map[string]*DownstreamMeta), + downDBFunc: downDBFunc, } } -// RebuildLocksAndTables rebuild the locks and tables. -func (lk *LockKeeper) RebuildLocksAndTables( - cli *clientv3.Client, - ifm map[string]map[string]map[string]map[string]Info, - colm map[string]map[string]map[string]map[string]map[string]DropColumnStage, - lockJoined map[string]schemacmp.Table, - lockTTS map[string][]TargetTable, - missTable map[string]map[string]map[string]map[string]schemacmp.Table, -) { - var ( - lock *Lock - ok bool - ) - for task, taskInfos := range ifm { - for source, sourceInfos := range taskInfos { - for schema, schemaInfos := range sourceInfos { - for table, info := range schemaInfos { - lockID := utils.GenDDLLockID(info.Task, info.DownSchema, info.DownTable) - if lock, ok = lk.locks[lockID]; !ok { - lock = NewLock(cli, lockID, info.Task, info.DownSchema, info.DownTable, lockJoined[lockID], lockTTS[lockID]) - } - // filter info which doesn't have SourceTable - // SourceTable will be changed after user update block-allow-list - // But old infos still remain in etcd. - // TODO: add a mechanism to remove all outdated infos in etcd. - if !lock.TableExist(info.Source, info.UpSchema, info.UpTable) { - delete(ifm[task][source][schema], table) - continue - } - lk.locks[lockID] = lock - lock.tables[info.Source][info.UpSchema][info.UpTable] = schemacmp.Encode(info.TableInfoBefore) - if columns, ok := colm[lockID]; ok { - lock.columns = columns - } - } - } - } +// SetDropColumns set drop columns for lock keeper +func (lk *LockKeeper) SetDropColumns(dropColumns map[string]map[string]map[string]map[string]map[string]DropColumnStage) { + if dropColumns != nil { + lk.dropColumns = dropColumns } +} - // update missTable's table info for locks - for lockID, lockTable := range missTable { - for source, sourceTable := range lockTable { - for schema, schemaTable := range sourceTable { - for table, tableinfo := range schemaTable { - if _, ok := lk.locks[lockID]; !ok { - continue - } - if !lk.locks[lockID].TableExist(source, schema, table) { - continue - } - lk.locks[lockID].tables[source][schema][table] = tableinfo - } - } - } +func (lk *LockKeeper) getDownstreamMeta(task string) (*DownstreamMeta, error) { + if downstreamMeta, ok := lk.downstreamMetaMap[task]; ok { + return downstreamMeta, nil + } + + dbConfig, meta := lk.downDBFunc(task) + if dbConfig == nil { + return nil, terror.ErrMasterOptimisticDownstreamMetaNotFound.Generate(task) } + db, err := conn.DefaultDBProvider.Apply(dbConfig) + if err != nil { + return nil, err + } + downstreamMeta := &DownstreamMeta{db: db, meta: meta} + return downstreamMeta, nil } +// // RebuildLocksAndTables rebuild the locks and tables. +// func (lk *LockKeeper) RebuildLocksAndTables( +// cli *clientv3.Client, +// ifm map[string]map[string]map[string]map[string]Info, +// colm map[string]map[string]map[string]map[string]map[string]DropColumnStage, +// lockJoined map[string]schemacmp.Table, +// lockTTS map[string][]TargetTable, +// missTable map[string]map[string]map[string]map[string]schemacmp.Table, +// ) { +// var ( +// lock *Lock +// ok bool +// ) +// for task, taskInfos := range ifm { +// for source, sourceInfos := range taskInfos { +// for schema, schemaInfos := range sourceInfos { +// for table, info := range schemaInfos { +// lockID := utils.GenDDLLockID(info.Task, info.DownSchema, info.DownTable) +// if lock, ok = lk.locks[lockID]; !ok { +// downstreamMeta, err := lk.getDownstreamMeta(task) +// if err != nil { +// log.L().Error("get downstream meta", log.ShortError(err)) +// } +// lock = NewLock(cli, lockID, info.Task, info.DownSchema, info.DownTable, lockJoined[lockID], lockTTS[lockID], downstreamMeta) +// } +// // filter info which doesn't have SourceTable +// // SourceTable will be changed after user update block-allow-list +// // But old infos still remain in etcd. +// // TODO: add a mechanism to remove all outdated infos in etcd. +// if !lock.TableExist(info.Source, info.UpSchema, info.UpTable) { +// delete(ifm[task][source][schema], table) +// continue +// } +// lk.locks[lockID] = lock +// lock.tables[info.Source][info.UpSchema][info.UpTable] = schemacmp.Encode(info.TableInfoBefore) +// if columns, ok := colm[lockID]; ok { +// lock.columns = columns +// } +// } +// } +// } +// } +// +// // update missTable's table info for locks +// for lockID, lockTable := range missTable { +// for source, sourceTable := range lockTable { +// for schema, schemaTable := range sourceTable { +// for table, tableinfo := range schemaTable { +// if _, ok := lk.locks[lockID]; !ok { +// continue +// } +// if !lk.locks[lockID].TableExist(source, schema, table) { +// continue +// } +// lk.locks[lockID].tables[source][schema][table] = tableinfo +// } +// } +// } +// } +// } + // TrySync tries to sync the lock. func (lk *LockKeeper) TrySync(cli *clientv3.Client, info Info, tts []TargetTable) (string, []string, []string, error) { var ( @@ -111,8 +153,21 @@ func (lk *LockKeeper) TrySync(cli *clientv3.Client, info Info, tts []TargetTable } if l, ok = lk.locks[lockID]; !ok { - lk.locks[lockID] = NewLock(cli, lockID, info.Task, info.DownSchema, info.DownTable, schemacmp.Encode(info.TableInfoBefore), tts) + downstreamMeta, err := lk.getDownstreamMeta(info.Task) + if err != nil { + log.L().Error("get downstream meta", log.ShortError(err)) + } + + lk.locks[lockID] = NewLock(cli, lockID, info.Task, info.DownSchema, info.DownTable, schemacmp.Encode(info.TableInfoBefore), tts, downstreamMeta) l = lk.locks[lockID] + + // set drop columns + if cols, ok := lk.dropColumns[lockID]; ok { + l.columns = cols + delete(lk.dropColumns, lockID) + } + } else { + log.L().Error("lock already exist") } newDDLs, cols, err := l.TrySync(info, tts) @@ -245,6 +300,30 @@ func (tk *TableKeeper) AddTable(task, source, upSchema, upTable, downSchema, dow return added } +// SourceTableExist check whether a source table exist +func (tk *TableKeeper) SourceTableExist(task, source, upSchema, upTable, downSchema, downTable string) bool { + tk.mu.Lock() + defer tk.mu.Unlock() + + if _, ok := tk.tables[task]; !ok { + return false + } + if _, ok := tk.tables[task][source]; !ok { + return false + } + st := tk.tables[task][source] + targetTable := st.TargetTable(downSchema, downTable) + + if targetTable.UpTables != nil { + if tables, ok := targetTable.UpTables[upSchema]; ok { + if _, ok2 := tables[upTable]; ok2 { + return true + } + } + } + return false +} + // RemoveTable removes a table from the source tables. // it returns whether removed (exit before). func (tk *TableKeeper) RemoveTable(task, source, upSchema, upTable, downSchema, downTable string) bool { diff --git a/dm/pkg/shardddl/optimism/keeper_test.go b/dm/pkg/shardddl/optimism/keeper_test.go index adbba568ce4..896b0033d40 100644 --- a/dm/pkg/shardddl/optimism/keeper_test.go +++ b/dm/pkg/shardddl/optimism/keeper_test.go @@ -17,13 +17,12 @@ import ( "testing" . "github.com/pingcap/check" - "github.com/pingcap/tidb-tools/pkg/schemacmp" "github.com/pingcap/tidb/parser" "github.com/pingcap/tidb/parser/model" "github.com/pingcap/tidb/util/mock" "go.etcd.io/etcd/integration" - "github.com/pingcap/ticdc/dm/pkg/utils" + "github.com/pingcap/ticdc/dm/dm/config" ) type testKeeper struct{} @@ -41,7 +40,7 @@ func TestKeeper(t *testing.T) { func (t *testKeeper) TestLockKeeper(c *C) { var ( - lk = NewLockKeeper() + lk = NewLockKeeper(getDownstreamMeta) upSchema = "foo_1" upTable = "bar_1" downSchema = "foo" @@ -134,7 +133,7 @@ func (t *testKeeper) TestLockKeeper(c *C) { func (t *testKeeper) TestLockKeeperMultipleTarget(c *C) { var ( - lk = NewLockKeeper() + lk = NewLockKeeper(getDownstreamMeta) task = "test-lock-keeper-multiple-target" source = "mysql-replica-1" upSchema = "foo" @@ -414,72 +413,76 @@ func (t *testKeeper) TestTargetTablesForTask(c *C) { }) } -func (t *testKeeper) TestRebuildLocksAndTables(c *C) { - defer clearTestInfoOperation(c) - var ( - lk = NewLockKeeper() - task = "task" - source1 = "mysql-replica-1" - source2 = "mysql-replica-2" - upSchema = "foo" - upTable = "bar" - downSchema = "db" - downTable = "tbl" - DDLs1 = []string{"ALTER TABLE bar ADD COLUMN c1 INT"} - DDLs2 = []string{"ALTER TABLE bar DROP COLUMN c1"} - p = parser.New() - se = mock.NewContext() - tblID int64 = 111 - ti0 = createTableInfo(c, p, se, tblID, `CREATE TABLE bar (id INT PRIMARY KEY)`) - ti1 = createTableInfo(c, p, se, tblID, `CREATE TABLE bar (id INT PRIMARY KEY, c1 INT)`) - ti2 = createTableInfo(c, p, se, tblID, `CREATE TABLE bar (id INT PRIMARY KEY, c1 INT, c2 INT)`) - ti3 = createTableInfo(c, p, se, tblID, `CREATE TABLE bar (id INT PRIMARY KEY, c2 INT)`) - - i11 = NewInfo(task, source1, upSchema, upTable, downSchema, downTable, DDLs1, ti0, []*model.TableInfo{ti1}) - i21 = NewInfo(task, source2, upSchema, upTable, downSchema, downTable, DDLs2, ti2, []*model.TableInfo{ti3}) - - tts = []TargetTable{ - newTargetTable(task, source1, downSchema, downTable, map[string]map[string]struct{}{upSchema: {upTable: struct{}{}}}), - newTargetTable(task, source2, downSchema, downTable, map[string]map[string]struct{}{upSchema: {upTable: struct{}{}}}), - } - - lockID = utils.GenDDLLockID(task, downSchema, downTable) - - ifm = map[string]map[string]map[string]map[string]Info{ - task: { - source1: {upSchema: {upTable: i11}}, - source2: {upSchema: {upTable: i21}}, - }, - } - colm = map[string]map[string]map[string]map[string]map[string]DropColumnStage{ - lockID: { - "c3": { - source1: {upSchema: {upTable: DropNotDone}}, - source2: {upSchema: {upTable: DropNotDone}}, - }, - }, - } - lockJoined = map[string]schemacmp.Table{ - lockID: schemacmp.Encode(ti2), - } - lockTTS = map[string][]TargetTable{ - lockID: tts, - } - ) - - lk.RebuildLocksAndTables(etcdTestCli, ifm, colm, lockJoined, lockTTS, nil) - locks := lk.Locks() - c.Assert(len(locks), Equals, 1) - lock, ok := locks[lockID] - c.Assert(ok, IsTrue) - cmp, err := lock.Joined().Compare(schemacmp.Encode(ti2)) - c.Assert(err, IsNil) - c.Assert(cmp, Equals, 0) - cmp, err = lock.tables[source1][upSchema][upTable].Compare(schemacmp.Encode(ti0)) - c.Assert(err, IsNil) - c.Assert(cmp, Equals, 0) - cmp, err = lock.tables[source2][upSchema][upTable].Compare(schemacmp.Encode(ti2)) - c.Assert(err, IsNil) - c.Assert(cmp, Equals, 0) - c.Assert(lock.columns, DeepEquals, colm[lockID]) +//func (t *testKeeper) TestRebuildLocksAndTables(c *C) { +// defer clearTestInfoOperation(c) +// var ( +// lk = NewLockKeeper(getDownstreamMeta) +// task = "task" +// source1 = "mysql-replica-1" +// source2 = "mysql-replica-2" +// upSchema = "foo" +// upTable = "bar" +// downSchema = "db" +// downTable = "tbl" +// DDLs1 = []string{"ALTER TABLE bar ADD COLUMN c1 INT"} +// DDLs2 = []string{"ALTER TABLE bar DROP COLUMN c1"} +// p = parser.New() +// se = mock.NewContext() +// tblID int64 = 111 +// ti0 = createTableInfo(c, p, se, tblID, `CREATE TABLE bar (id INT PRIMARY KEY)`) +// ti1 = createTableInfo(c, p, se, tblID, `CREATE TABLE bar (id INT PRIMARY KEY, c1 INT)`) +// ti2 = createTableInfo(c, p, se, tblID, `CREATE TABLE bar (id INT PRIMARY KEY, c1 INT, c2 INT)`) +// ti3 = createTableInfo(c, p, se, tblID, `CREATE TABLE bar (id INT PRIMARY KEY, c2 INT)`) +// +// i11 = NewInfo(task, source1, upSchema, upTable, downSchema, downTable, DDLs1, ti0, []*model.TableInfo{ti1}) +// i21 = NewInfo(task, source2, upSchema, upTable, downSchema, downTable, DDLs2, ti2, []*model.TableInfo{ti3}) +// +// tts = []TargetTable{ +// newTargetTable(task, source1, downSchema, downTable, map[string]map[string]struct{}{upSchema: {upTable: struct{}{}}}), +// newTargetTable(task, source2, downSchema, downTable, map[string]map[string]struct{}{upSchema: {upTable: struct{}{}}}), +// } +// +// lockID = utils.GenDDLLockID(task, downSchema, downTable) +// +// ifm = map[string]map[string]map[string]map[string]Info{ +// task: { +// source1: {upSchema: {upTable: i11}}, +// source2: {upSchema: {upTable: i21}}, +// }, +// } +// colm = map[string]map[string]map[string]map[string]map[string]DropColumnStage{ +// lockID: { +// "c3": { +// source1: {upSchema: {upTable: DropNotDone}}, +// source2: {upSchema: {upTable: DropNotDone}}, +// }, +// }, +// } +// lockJoined = map[string]schemacmp.Table{ +// lockID: schemacmp.Encode(ti2), +// } +// lockTTS = map[string][]TargetTable{ +// lockID: tts, +// } +// ) +// +// lk.RebuildLocksAndTables(etcdTestCli, ifm, colm, lockJoined, lockTTS, nil) +// locks := lk.Locks() +// c.Assert(len(locks), Equals, 1) +// lock, ok := locks[lockID] +// c.Assert(ok, IsTrue) +// cmp, err := lock.Joined().Compare(schemacmp.Encode(ti2)) +// c.Assert(err, IsNil) +// c.Assert(cmp, Equals, 0) +// cmp, err = lock.tables[source1][upSchema][upTable].Compare(schemacmp.Encode(ti0)) +// c.Assert(err, IsNil) +// c.Assert(cmp, Equals, 0) +// cmp, err = lock.tables[source2][upSchema][upTable].Compare(schemacmp.Encode(ti2)) +// c.Assert(err, IsNil) +// c.Assert(cmp, Equals, 0) +// c.Assert(lock.columns, DeepEquals, colm[lockID]) +//} + +func getDownstreamMeta(string) (*config.DBConfig, string) { + return nil, "" } diff --git a/dm/pkg/shardddl/optimism/lock.go b/dm/pkg/shardddl/optimism/lock.go index 55d65f4cf5b..b76df97c7f9 100644 --- a/dm/pkg/shardddl/optimism/lock.go +++ b/dm/pkg/shardddl/optimism/lock.go @@ -14,16 +14,22 @@ package optimism import ( + "bytes" + "encoding/json" "fmt" "sync" + "github.com/pingcap/tidb-tools/pkg/dbutil" "github.com/pingcap/tidb-tools/pkg/schemacmp" "github.com/pingcap/tidb/parser" "github.com/pingcap/tidb/parser/ast" + "github.com/pingcap/tidb/parser/model" "go.etcd.io/etcd/clientv3" "go.uber.org/zap" + "golang.org/x/net/context" "github.com/pingcap/ticdc/dm/dm/master/metrics" + "github.com/pingcap/ticdc/dm/pkg/cputil" "github.com/pingcap/ticdc/dm/pkg/log" "github.com/pingcap/ticdc/dm/pkg/terror" ) @@ -75,23 +81,26 @@ type Lock struct { // record the partially dropped columns // column name -> source -> upSchema -> upTable -> int columns map[string]map[string]map[string]map[string]DropColumnStage + + downstreamMeta *DownstreamMeta } // NewLock creates a new Lock instance. // NOTE: we MUST give the initial table info when creating the lock now. -func NewLock(cli *clientv3.Client, id, task, downSchema, downTable string, joined schemacmp.Table, tts []TargetTable) *Lock { +func NewLock(cli *clientv3.Client, id, task, downSchema, downTable string, joined schemacmp.Table, tts []TargetTable, downstreamMeta *DownstreamMeta) *Lock { l := &Lock{ - cli: cli, - ID: id, - Task: task, - DownSchema: downSchema, - DownTable: downTable, - joined: joined, - tables: make(map[string]map[string]map[string]schemacmp.Table), - done: make(map[string]map[string]map[string]bool), - synced: true, - versions: make(map[string]map[string]map[string]int64), - columns: make(map[string]map[string]map[string]map[string]DropColumnStage), + cli: cli, + ID: id, + Task: task, + DownSchema: downSchema, + DownTable: downTable, + joined: joined, + tables: make(map[string]map[string]map[string]schemacmp.Table), + done: make(map[string]map[string]map[string]bool), + synced: true, + versions: make(map[string]map[string]map[string]int64), + columns: make(map[string]map[string]map[string]map[string]DropColumnStage), + downstreamMeta: downstreamMeta, } l.addTables(tts) metrics.ReportDDLPending(task, metrics.DDLPendingNone, metrics.DDLPendingSynced) @@ -99,6 +108,37 @@ func NewLock(cli *clientv3.Client, id, task, downSchema, downTable string, joine return l } +// FetchTableInfos fetch all table infos for a lock +func (l *Lock) FetchTableInfos(task, source, schema, table string) (*model.TableInfo, error) { + if l.downstreamMeta == nil { + log.L().Warn("nil downstream meta") + return nil, terror.ErrMasterOptimisticDownstreamMetaNotFound.Generate(task) + } + + db := l.downstreamMeta.db + ctx, cancel := context.WithTimeout(context.Background(), dbutil.DefaultTimeout) + defer cancel() + + query := `SELECT table_info FROM ` + dbutil.TableName(l.downstreamMeta.meta, cputil.SyncerCheckpoint(task)) + ` WHERE id = ? AND cp_schema = ?` + row := db.DB.QueryRowContext(ctx, query, source, schema) + if row.Err() != nil { + return nil, row.Err() + } + var tiBytes []byte + if err := row.Scan(&tiBytes); err != nil { + return nil, err + } + var ti *model.TableInfo + if bytes.Equal(tiBytes, []byte("null")) { + log.L().Warn("null table info", zap.String("query", query), zap.String("source", source), zap.String("schema", schema), zap.String("table", table)) + return nil, terror.ErrMasterOptimisticDownstreamMetaNotFound.Generate(task) + } + if err := json.Unmarshal(tiBytes, &ti); err != nil { + return nil, err + } + return ti, nil +} + // TrySync tries to sync the lock, re-entrant. // new upstream sources may join when the DDL lock is in syncing, // so we need to merge these new sources. @@ -548,7 +588,14 @@ func (l *Lock) addTables(tts []TargetTable) { } for table := range tables { if _, ok := l.tables[tt.Source][schema][table]; !ok { - l.tables[tt.Source][schema][table] = l.joined + ti, err := l.FetchTableInfos(tt.Task, tt.Source, schema, table) + if err != nil { + log.L().Error("source table info not found, use joined table info instead", zap.String("task", tt.Task), zap.String("source", tt.Source), zap.String("schema", schema), zap.String("table", table), log.ShortError(err)) + l.tables[tt.Source][schema][table] = l.joined + } else { + t := schemacmp.Encode(ti) + l.tables[tt.Source][schema][table] = t + } l.done[tt.Source][schema][table] = false l.versions[tt.Source][schema][table] = 0 log.L().Info("table added to the lock", zap.String("lock", l.ID), diff --git a/dm/pkg/shardddl/optimism/lock_test.go b/dm/pkg/shardddl/optimism/lock_test.go index 7987e9d35d2..d8a8596089e 100644 --- a/dm/pkg/shardddl/optimism/lock_test.go +++ b/dm/pkg/shardddl/optimism/lock_test.go @@ -83,7 +83,7 @@ func (t *testLock) TestLockTrySyncNormal(c *C) { newTargetTable(task, sources[1], downSchema, downTable, tables), } - l = NewLock(etcdTestCli, ID, task, downSchema, downTable, schemacmp.Encode(ti0), tts) + l = NewLock(etcdTestCli, ID, task, downSchema, downTable, schemacmp.Encode(ti0), tts, nil) vers = map[string]map[string]map[string]int64{ sources[0]: { @@ -385,7 +385,7 @@ func (t *testLock) TestLockTrySyncIndex(c *C) { newTargetTable(task, source, downSchema, downTable, tables), } - l = NewLock(etcdTestCli, ID, task, downSchema, downTable, schemacmp.Encode(ti0), tts) + l = NewLock(etcdTestCli, ID, task, downSchema, downTable, schemacmp.Encode(ti0), tts, nil) vers = map[string]map[string]map[string]int64{ source: { @@ -469,7 +469,7 @@ func (t *testLock) TestLockTrySyncNullNotNull(c *C) { newTargetTable(task, source, downSchema, downTable, tables), } - l = NewLock(etcdTestCli, ID, task, downSchema, downTable, schemacmp.Encode(ti0), tts) + l = NewLock(etcdTestCli, ID, task, downSchema, downTable, schemacmp.Encode(ti0), tts, nil) vers = map[string]map[string]map[string]int64{ source: { @@ -540,7 +540,7 @@ func (t *testLock) TestLockTrySyncIntBigint(c *C) { newTargetTable(task, source, downSchema, downTable, tables), } - l = NewLock(etcdTestCli, ID, task, downSchema, downTable, schemacmp.Encode(ti0), tts) + l = NewLock(etcdTestCli, ID, task, downSchema, downTable, schemacmp.Encode(ti0), tts, nil) vers = map[string]map[string]map[string]int64{ source: { @@ -593,7 +593,7 @@ func (t *testLock) TestLockTrySyncNoDiff(c *C) { newTargetTable(task, source, downSchema, downTable, tables), } - l = NewLock(etcdTestCli, ID, task, downSchema, downTable, schemacmp.Encode(ti0), tts) + l = NewLock(etcdTestCli, ID, task, downSchema, downTable, schemacmp.Encode(ti0), tts, nil) vers = map[string]map[string]map[string]int64{ source: { @@ -636,7 +636,7 @@ func (t *testLock) TestLockTrySyncNewTable(c *C) { tables = map[string]map[string]struct{}{db1: {tbl1: struct{}{}}} tts = []TargetTable{newTargetTable(task, source1, downSchema, downTable, tables)} - l = NewLock(etcdTestCli, ID, task, downSchema, downTable, schemacmp.Encode(ti0), tts) + l = NewLock(etcdTestCli, ID, task, downSchema, downTable, schemacmp.Encode(ti0), tts, nil) vers = map[string]map[string]map[string]int64{ source1: { db1: {tbl1: 0}, @@ -749,7 +749,7 @@ func (t *testLock) TestLockTrySyncRevert(c *C) { tables = map[string]map[string]struct{}{db: {tbls[0]: struct{}{}, tbls[1]: struct{}{}}} tts = []TargetTable{newTargetTable(task, source, downSchema, downTable, tables)} - l = NewLock(etcdTestCli, ID, task, downSchema, downTable, schemacmp.Encode(ti0), tts) + l = NewLock(etcdTestCli, ID, task, downSchema, downTable, schemacmp.Encode(ti0), tts, nil) vers = map[string]map[string]map[string]int64{ source: { @@ -914,7 +914,7 @@ func (t *testLock) TestLockTrySyncConflictNonIntrusive(c *C) { tables = map[string]map[string]struct{}{db: {tbls[0]: struct{}{}, tbls[1]: struct{}{}}} tts = []TargetTable{newTargetTable(task, source, downSchema, downTable, tables)} - l = NewLock(etcdTestCli, ID, task, downSchema, downTable, schemacmp.Encode(ti0), tts) + l = NewLock(etcdTestCli, ID, task, downSchema, downTable, schemacmp.Encode(ti0), tts, nil) vers = map[string]map[string]map[string]int64{ source: { @@ -1036,7 +1036,7 @@ func (t *testLock) TestLockTrySyncConflictIntrusive(c *C) { tables = map[string]map[string]struct{}{db: {tbls[0]: struct{}{}, tbls[1]: struct{}{}}} tts = []TargetTable{newTargetTable(task, source, downSchema, downTable, tables)} - l = NewLock(etcdTestCli, ID, task, downSchema, downTable, schemacmp.Encode(ti0), tts) + l = NewLock(etcdTestCli, ID, task, downSchema, downTable, schemacmp.Encode(ti0), tts, nil) vers = map[string]map[string]map[string]int64{ source: { @@ -1224,7 +1224,7 @@ func (t *testLock) TestLockTrySyncMultipleChangeDDL(c *C) { newTargetTable(task, sources[1], downSchema, downTable, tables), } - l = NewLock(etcdTestCli, ID, task, downSchema, downTable, schemacmp.Encode(ti0), tts) + l = NewLock(etcdTestCli, ID, task, downSchema, downTable, schemacmp.Encode(ti0), tts, nil) vers = map[string]map[string]map[string]int64{ sources[0]: { @@ -1367,7 +1367,7 @@ func (t *testLock) TestTryRemoveTable(c *C) { tables = map[string]map[string]struct{}{db: {tbl1: struct{}{}, tbl2: struct{}{}}} tts = []TargetTable{newTargetTable(task, source, downSchema, downTable, tables)} - l = NewLock(etcdTestCli, ID, task, downSchema, downTable, schemacmp.Encode(ti0), tts) + l = NewLock(etcdTestCli, ID, task, downSchema, downTable, schemacmp.Encode(ti0), tts, nil) vers = map[string]map[string]map[string]int64{ source: { @@ -1459,7 +1459,7 @@ func (t *testLock) TestLockTryMarkDone(c *C) { tables = map[string]map[string]struct{}{db: {tbls[0]: struct{}{}, tbls[1]: struct{}{}}} tts = []TargetTable{newTargetTable(task, source, downSchema, downTable, tables)} - l = NewLock(etcdTestCli, ID, task, downSchema, downTable, schemacmp.Encode(ti0), tts) + l = NewLock(etcdTestCli, ID, task, downSchema, downTable, schemacmp.Encode(ti0), tts, nil) vers = map[string]map[string]map[string]int64{ source: { @@ -1567,7 +1567,7 @@ func (t *testLock) TestAddDifferentFieldLenColumns(c *C) { tables = map[string]map[string]struct{}{db: {tbls[0]: struct{}{}, tbls[1]: struct{}{}}} tts = []TargetTable{newTargetTable(task, source, downSchema, downTable, tables)} - l = NewLock(etcdTestCli, ID, task, downSchema, downTable, schemacmp.Encode(ti0), tts) + l = NewLock(etcdTestCli, ID, task, downSchema, downTable, schemacmp.Encode(ti0), tts, nil) vers = map[string]map[string]map[string]int64{ source: { @@ -1609,7 +1609,7 @@ func (t *testLock) TestAddDifferentFieldLenColumns(c *C) { c.Assert(l.versions, DeepEquals, vers) // case 2: add a column with a smaller field length - l = NewLock(etcdTestCli, ID, task, downSchema, downTable, schemacmp.Encode(ti0), tts) + l = NewLock(etcdTestCli, ID, task, downSchema, downTable, schemacmp.Encode(ti0), tts, nil) // TrySync for the first table, no table has done the DDLs operation. vers[source][db][tbls[0]]-- @@ -1657,7 +1657,7 @@ func (t *testLock) TestAddNotFullyDroppedColumns(c *C) { tables = map[string]map[string]struct{}{db: {tbls[0]: struct{}{}, tbls[1]: struct{}{}}} tts = []TargetTable{newTargetTable(task, source, downSchema, downTable, tables)} - l = NewLock(etcdTestCli, ID, task, downSchema, downTable, schemacmp.Encode(ti0), tts) + l = NewLock(etcdTestCli, ID, task, downSchema, downTable, schemacmp.Encode(ti0), tts, nil) vers = map[string]map[string]map[string]int64{ source: { @@ -1853,7 +1853,7 @@ func (t *testLock) TestLockTrySyncDifferentIndex(c *C) { newTargetTable(task, source, downSchema, downTable, tables), } - l = NewLock(etcdTestCli, ID, task, downSchema, downTable, schemacmp.Encode(ti0), tts) + l = NewLock(etcdTestCli, ID, task, downSchema, downTable, schemacmp.Encode(ti0), tts, nil) vers = map[string]map[string]map[string]int64{ source: { diff --git a/dm/pkg/shardddl/optimism/ops.go b/dm/pkg/shardddl/optimism/ops.go index dc727789445..1a05f84c622 100644 --- a/dm/pkg/shardddl/optimism/ops.go +++ b/dm/pkg/shardddl/optimism/ops.go @@ -51,7 +51,7 @@ func PutSourceTablesDeleteInfo(cli *clientv3.Client, st SourceTables, info Info) // DeleteInfosOperationsSchemaColumn deletes the shard DDL infos, operations, init schemas and dropped columns in etcd. // This function should often be called by DM-master when removing the lock. // Only delete when all info's version are greater or equal to etcd's version, otherwise it means new info was putted into etcd before. -func DeleteInfosOperationsSchemaColumn(cli *clientv3.Client, infos []Info, ops []Operation, schema InitSchema) (int64, bool, error) { +func DeleteInfosOperationsSchemaColumn(cli *clientv3.Client, infos []Info, ops []Operation, task, downSchema, downTable string) (int64, bool, error) { opsDel := make([]clientv3.Op, 0, len(infos)+len(ops)) cmps := make([]clientv3.Cmp, 0, len(infos)) for _, info := range infos { @@ -62,8 +62,8 @@ func DeleteInfosOperationsSchemaColumn(cli *clientv3.Client, infos []Info, ops [ for _, op := range ops { opsDel = append(opsDel, deleteOperationOp(op)) } - opsDel = append(opsDel, deleteInitSchemaOp(schema.Task, schema.DownSchema, schema.DownTable)) - opsDel = append(opsDel, deleteDroppedColumnsByLockOp(utils.GenDDLLockID(schema.Task, schema.DownSchema, schema.DownTable))) + // opsDel = append(opsDel, deleteInitSchemaOp(schema.Task, schema.DownSchema, schema.DownTable)) + opsDel = append(opsDel, deleteDroppedColumnsByLockOp(utils.GenDDLLockID(task, downSchema, downTable))) resp, rev, err := etcdutil.DoOpsInOneCmpsTxnWithRetry(cli, cmps, opsDel, []clientv3.Op{}) if err != nil { return 0, false, err @@ -77,7 +77,7 @@ func DeleteInfosOperationsTablesSchemasByTask(cli *clientv3.Client, task string, opsDel = append(opsDel, clientv3.OpDelete(common.ShardDDLOptimismInfoKeyAdapter.Encode(task), clientv3.WithPrefix())) opsDel = append(opsDel, clientv3.OpDelete(common.ShardDDLOptimismOperationKeyAdapter.Encode(task), clientv3.WithPrefix())) opsDel = append(opsDel, clientv3.OpDelete(common.ShardDDLOptimismSourceTablesKeyAdapter.Encode(task), clientv3.WithPrefix())) - opsDel = append(opsDel, clientv3.OpDelete(common.ShardDDLOptimismInitSchemaKeyAdapter.Encode(task), clientv3.WithPrefix())) + // opsDel = append(opsDel, clientv3.OpDelete(common.ShardDDLOptimismInitSchemaKeyAdapter.Encode(task), clientv3.WithPrefix())) for lockID := range lockIDSet { opsDel = append(opsDel, clientv3.OpDelete(common.ShardDDLOptimismDroppedColumnsKeyAdapter.Encode(lockID), clientv3.WithPrefix())) } diff --git a/dm/pkg/shardddl/optimism/ops_test.go b/dm/pkg/shardddl/optimism/ops_test.go index 55270f0d5e8..99397c3e8e3 100644 --- a/dm/pkg/shardddl/optimism/ops_test.go +++ b/dm/pkg/shardddl/optimism/ops_test.go @@ -30,7 +30,7 @@ func (t *testForEtcd) TestDeleteInfosOperationsSchema(c *C) { DDLs = []string{"ALTER TABLE bar ADD COLUMN c1 INT"} info = NewInfo(task, source, upSchema, upTable, downSchema, downTable, DDLs, nil, nil) op = NewOperation("test-ID", task, source, upSchema, upTable, DDLs, ConflictResolved, "", false, []string{}) - is = NewInitSchema(task, downSchema, downTable, nil) + // is = NewInitSchema(task, downSchema, downTable, nil) ) // put info. @@ -52,15 +52,15 @@ func (t *testForEtcd) TestDeleteInfosOperationsSchema(c *C) { c.Assert(opm, HasLen, 1) c.Assert(opm[task][source][upSchema][upTable], DeepEquals, op) - // put init schema. - _, _, err = PutInitSchemaIfNotExist(etcdTestCli, is) - c.Assert(err, IsNil) - isc, _, err := GetInitSchema(etcdTestCli, is.Task, is.DownSchema, is.DownTable) - c.Assert(err, IsNil) - c.Assert(isc, DeepEquals, is) + // // put init schema. + // _, _, err = PutInitSchemaIfNotExist(etcdTestCli, is) + // c.Assert(err, IsNil) + // isc, _, err := GetInitSchema(etcdTestCli, is.Task, is.DownSchema, is.DownTable) + // c.Assert(err, IsNil) + // c.Assert(isc, DeepEquals, is) // DELETE info and operation with version 0 - _, deleted, err := DeleteInfosOperationsSchemaColumn(etcdTestCli, []Info{info}, []Operation{op}, is) + _, deleted, err := DeleteInfosOperationsSchemaColumn(etcdTestCli, []Info{info}, []Operation{op}, task, downSchema, downTable) c.Assert(err, IsNil) c.Assert(deleted, IsFalse) @@ -71,12 +71,12 @@ func (t *testForEtcd) TestDeleteInfosOperationsSchema(c *C) { opm, _, err = GetAllOperations(etcdTestCli) c.Assert(err, IsNil) c.Assert(opm, HasLen, 1) - isc, _, err = GetInitSchema(etcdTestCli, is.Task, is.DownSchema, is.DownTable) - c.Assert(err, IsNil) - c.Assert(isc.IsEmpty(), IsFalse) + // isc, _, err = GetInitSchema(etcdTestCli, is.Task, is.DownSchema, is.DownTable) + // c.Assert(err, IsNil) + // c.Assert(isc.IsEmpty(), IsFalse) // DELETE info and operation with version 1 - _, deleted, err = DeleteInfosOperationsSchemaColumn(etcdTestCli, []Info{infoWithVer}, []Operation{op}, is) + _, deleted, err = DeleteInfosOperationsSchemaColumn(etcdTestCli, []Info{infoWithVer}, []Operation{op}, task, downSchema, downTable) c.Assert(err, IsNil) c.Assert(deleted, IsTrue) @@ -87,9 +87,9 @@ func (t *testForEtcd) TestDeleteInfosOperationsSchema(c *C) { opm, _, err = GetAllOperations(etcdTestCli) c.Assert(err, IsNil) c.Assert(opm, HasLen, 0) - isc, _, err = GetInitSchema(etcdTestCli, is.Task, is.DownSchema, is.DownTable) - c.Assert(err, IsNil) - c.Assert(isc.IsEmpty(), IsTrue) + // isc, _, err = GetInitSchema(etcdTestCli, is.Task, is.DownSchema, is.DownTable) + // c.Assert(err, IsNil) + // c.Assert(isc.IsEmpty(), IsTrue) } func (t *testForEtcd) TestSourceTablesInfo(c *C) { diff --git a/dm/pkg/shardddl/optimism/schema.go b/dm/pkg/shardddl/optimism/schema.go index 7f4008eef2c..7d4542e2300 100644 --- a/dm/pkg/shardddl/optimism/schema.go +++ b/dm/pkg/shardddl/optimism/schema.go @@ -13,139 +13,135 @@ package optimism -import ( - "encoding/json" - - "github.com/pingcap/tidb/parser/model" - "go.etcd.io/etcd/clientv3" - "go.etcd.io/etcd/clientv3/clientv3util" - - "github.com/pingcap/ticdc/dm/dm/common" - "github.com/pingcap/ticdc/dm/pkg/etcdutil" -) - -// InitSchema represents the initial schema (schema before the lock constructed) of a merged table. -// NOTE: `Task`, `DownSchema` and `DownTable` are redundant in the etcd key path for convenient. -type InitSchema struct { - Task string `json:"task"` // data migration task name - DownSchema string `json:"down-schema"` // downstream/target schema name - DownTable string `json:"down-table"` // downstream/target table name - TableInfo *model.TableInfo `json:"table-info"` // the initial table info (schema) -} - -// NewInitSchema creates a new InitSchema instance. -func NewInitSchema(task, downSchema, downTable string, tableInfo *model.TableInfo) InitSchema { - return InitSchema{ - Task: task, - DownSchema: downSchema, - DownTable: downTable, - TableInfo: tableInfo, - } -} - -// String implements Stringer interface. -func (is InitSchema) String() string { - s, _ := is.toJSON() - return s -} - -// toJSON returns the string of JSON represent. -func (is InitSchema) toJSON() (string, error) { - data, err := json.Marshal(is) - if err != nil { - return "", err - } - return string(data), nil -} - -// IsEmpty returns true when this InitSchema has no value. -func (is InitSchema) IsEmpty() bool { - var emptyIS InitSchema - return is == emptyIS -} - -// infoFromJSON constructs InitSchema from its JSON represent. -func initSchemaFromJSON(s string) (is InitSchema, err error) { - err = json.Unmarshal([]byte(s), &is) - return -} - -// GetInitSchema gets the InitSchema for the specified downstream table. -func GetInitSchema(cli *clientv3.Client, task, downSchema, downTable string) (InitSchema, int64, error) { - var is InitSchema - op := clientv3.OpGet(common.ShardDDLOptimismInitSchemaKeyAdapter.Encode(task, downSchema, downTable)) - respTxn, rev, err := etcdutil.DoOpsInOneTxnWithRetry(cli, op) - if err != nil { - return is, 0, err - } - resp := respTxn.Responses[0].GetResponseRange() - - if resp.Count > 0 { - is, err = initSchemaFromJSON(string(resp.Kvs[0].Value)) - if err != nil { - return is, 0, err - } - } - return is, rev, nil -} - -// GetAllInitSchemas gets all init schemas from etcd. -// This function should often be called by DM-master. -// k/k/k/v: task-name -> downstream-schema-name -> downstream-table-name -> InitSchema. -func GetAllInitSchemas(cli *clientv3.Client) (map[string]map[string]map[string]InitSchema, int64, error) { - initSchemas := make(map[string]map[string]map[string]InitSchema) - op := clientv3.OpGet(common.ShardDDLOptimismInitSchemaKeyAdapter.Path(), clientv3.WithPrefix()) - respTxn, rev, err := etcdutil.DoOpsInOneTxnWithRetry(cli, op) - if err != nil { - return nil, 0, err - } - resp := respTxn.Responses[0].GetResponseRange() - - for _, kv := range resp.Kvs { - schema, err := initSchemaFromJSON(string(kv.Value)) - if err != nil { - return nil, 0, err - } - if _, ok := initSchemas[schema.Task]; !ok { - initSchemas[schema.Task] = make(map[string]map[string]InitSchema) - } - if _, ok := initSchemas[schema.Task][schema.DownSchema]; !ok { - initSchemas[schema.Task][schema.DownSchema] = make(map[string]InitSchema) - } - initSchemas[schema.Task][schema.DownSchema][schema.DownTable] = schema - } - return initSchemas, rev, nil -} - -// PutInitSchemaIfNotExist puts the InitSchema into ectd if no previous one exists. -func PutInitSchemaIfNotExist(cli *clientv3.Client, is InitSchema) (rev int64, putted bool, err error) { - value, err := is.toJSON() - if err != nil { - return 0, false, err - } - key := common.ShardDDLOptimismInitSchemaKeyAdapter.Encode(is.Task, is.DownSchema, is.DownTable) - - cmp := clientv3util.KeyMissing(key) - op := clientv3.OpPut(key, value) - - resp, rev, err := etcdutil.DoOpsInOneCmpsTxnWithRetry(cli, []clientv3.Cmp{cmp}, []clientv3.Op{op}, []clientv3.Op{}) - if err != nil { - return 0, false, err - } - return rev, resp.Succeeded, nil -} - -// DeleteInitSchema tries to delete the InitSchema for the specified downstream table. -func DeleteInitSchema(cli *clientv3.Client, task, downSchema, downTable string) (rev int64, deleted bool, err error) { - op := deleteInitSchemaOp(task, downSchema, downTable) - resp, rev, err := etcdutil.DoOpsInOneTxnWithRetry(cli, op) - if err != nil { - return 0, false, err - } - return rev, resp.Succeeded, nil -} - -// deleteInitSchemaOp returns a DELETE etcd operation for init schema. -func deleteInitSchemaOp(task, downSchema, downTable string) clientv3.Op { - return clientv3.OpDelete(common.ShardDDLOptimismInitSchemaKeyAdapter.Encode(task, downSchema, downTable)) -} +// +// import ( +// "encoding/json" +// +// "github.com/pingcap/tidb/parser/model" +// ) +// +// // InitSchema represents the initial schema (schema before the lock constructed) of a merged table. +// // NOTE: `Task`, `DownSchema` and `DownTable` are redundant in the etcd key path for convenient. +// type InitSchema struct { +// Task string `json:"task"` // data migration task name +// DownSchema string `json:"down-schema"` // downstream/target schema name +// DownTable string `json:"down-table"` // downstream/target table name +// TableInfo *model.TableInfo `json:"table-info"` // the initial table info (schema) +// } +// +// // NewInitSchema creates a new InitSchema instance. +// func NewInitSchema(task, downSchema, downTable string, tableInfo *model.TableInfo) InitSchema { +// return InitSchema{ +// Task: task, +// DownSchema: downSchema, +// DownTable: downTable, +// TableInfo: tableInfo, +// } +// } +// +// // String implements Stringer interface. +// func (is InitSchema) String() string { +// s, _ := is.toJSON() +// return s +// } +// +// // toJSON returns the string of JSON represent. +// func (is InitSchema) toJSON() (string, error) { +// data, err := json.Marshal(is) +// if err != nil { +// return "", err +// } +// return string(data), nil +// } +// +// // IsEmpty returns true when this InitSchema has no value. +// func (is InitSchema) IsEmpty() bool { +// var emptyIS InitSchema +// return is == emptyIS +// } +// +// // infoFromJSON constructs InitSchema from its JSON represent. +// func initSchemaFromJSON(s string) (is InitSchema, err error) { +// err = json.Unmarshal([]byte(s), &is) +// return +// } +// +// // GetInitSchema gets the InitSchema for the specified downstream table. +// func GetInitSchema(cli *clientv3.Client, task, downSchema, downTable string) (InitSchema, int64, error) { +// var is InitSchema +// op := clientv3.OpGet(common.ShardDDLOptimismInitSchemaKeyAdapter.Encode(task, downSchema, downTable)) +// respTxn, rev, err := etcdutil.DoOpsInOneTxnWithRetry(cli, op) +// if err != nil { +// return is, 0, err +// } +// resp := respTxn.Responses[0].GetResponseRange() +// +// if resp.Count > 0 { +// is, err = initSchemaFromJSON(string(resp.Kvs[0].Value)) +// if err != nil { +// return is, 0, err +// } +// } +// return is, rev, nil +// } +// +// // GetAllInitSchemas gets all init schemas from etcd. +// // This function should often be called by DM-master. +// // k/k/k/v: task-name -> downstream-schema-name -> downstream-table-name -> InitSchema. +// func GetAllInitSchemas(cli *clientv3.Client) (map[string]map[string]map[string]InitSchema, int64, error) { +// initSchemas := make(map[string]map[string]map[string]InitSchema) +// op := clientv3.OpGet(common.ShardDDLOptimismInitSchemaKeyAdapter.Path(), clientv3.WithPrefix()) +// respTxn, rev, err := etcdutil.DoOpsInOneTxnWithRetry(cli, op) +// if err != nil { +// return nil, 0, err +// } +// resp := respTxn.Responses[0].GetResponseRange() +// +// for _, kv := range resp.Kvs { +// schema, err := initSchemaFromJSON(string(kv.Value)) +// if err != nil { +// return nil, 0, err +// } +// if _, ok := initSchemas[schema.Task]; !ok { +// initSchemas[schema.Task] = make(map[string]map[string]InitSchema) +// } +// if _, ok := initSchemas[schema.Task][schema.DownSchema]; !ok { +// initSchemas[schema.Task][schema.DownSchema] = make(map[string]InitSchema) +// } +// initSchemas[schema.Task][schema.DownSchema][schema.DownTable] = schema +// } +// return initSchemas, rev, nil +// } +// +// // PutInitSchemaIfNotExist puts the InitSchema into ectd if no previous one exists. +// func PutInitSchemaIfNotExist(cli *clientv3.Client, is InitSchema) (rev int64, putted bool, err error) { +// value, err := is.toJSON() +// if err != nil { +// return 0, false, err +// } +// key := common.ShardDDLOptimismInitSchemaKeyAdapter.Encode(is.Task, is.DownSchema, is.DownTable) +// +// cmp := clientv3util.KeyMissing(key) +// op := clientv3.OpPut(key, value) +// +// resp, rev, err := etcdutil.DoOpsInOneCmpsTxnWithRetry(cli, []clientv3.Cmp{cmp}, []clientv3.Op{op}, []clientv3.Op{}) +// if err != nil { +// return 0, false, err +// } +// return rev, resp.Succeeded, nil +// } + +// // DeleteInitSchema tries to delete the InitSchema for the specified downstream table. +// func DeleteInitSchema(cli *clientv3.Client, task, downSchema, downTable string) (rev int64, deleted bool, err error) { +// op := deleteInitSchemaOp(task, downSchema, downTable) +// resp, rev, err := etcdutil.DoOpsInOneTxnWithRetry(cli, op) +// if err != nil { +// return 0, false, err +// } +// return rev, resp.Succeeded, nil +// } +// +// // deleteInitSchemaOp returns a DELETE etcd operation for init schema. +// func deleteInitSchemaOp(task, downSchema, downTable string) clientv3.Op { +// return clientv3.OpDelete(common.ShardDDLOptimismInitSchemaKeyAdapter.Encode(task, downSchema, downTable)) +// } diff --git a/dm/pkg/shardddl/optimism/schema_test.go b/dm/pkg/shardddl/optimism/schema_test.go index f5f3ba582a4..db23bfce59c 100644 --- a/dm/pkg/shardddl/optimism/schema_test.go +++ b/dm/pkg/shardddl/optimism/schema_test.go @@ -13,97 +13,95 @@ package optimism -import ( - . "github.com/pingcap/check" - "github.com/pingcap/tidb/parser" - "github.com/pingcap/tidb/util/mock" -) +//import ( +// . "github.com/pingcap/check" +//) -func (t *testForEtcd) TestInitSchemaJSON(c *C) { - is1 := NewInitSchema("test", "foo", "bar", nil) - j, err := is1.toJSON() - c.Assert(err, IsNil) - c.Assert(j, Equals, `{"task":"test","down-schema":"foo","down-table":"bar","table-info":null}`) - c.Assert(j, Equals, is1.String()) - - is2, err := initSchemaFromJSON(j) - c.Assert(err, IsNil) - c.Assert(is2, DeepEquals, is1) -} - -func (t *testForEtcd) TestInitSchemaEtcd(c *C) { - defer clearTestInfoOperation(c) - - var ( - task = "test-init-schema-etcd" - downSchema = "foo" - downTable = "bar" - downTable2 = "bar2" - p = parser.New() - se = mock.NewContext() - tblID int64 = 111 - tblI1 = createTableInfo(c, p, se, tblID, "CREATE TABLE bar (id INT PRIMARY KEY)") - tblI2 = createTableInfo(c, p, se, tblID, "CREATE TABLE bar (id INT PRIMARY KEY, c1 INT)") - tblI3 = createTableInfo(c, p, se, tblID, "CREATE TABLE bar2 (id INT PRIMARY KEY, c INT)") - is1 = NewInitSchema(task, downSchema, downTable, tblI1) - is2 = NewInitSchema(task, downSchema, downTable, tblI2) - is3 = NewInitSchema(task, downSchema, downTable2, tblI3) - ) - - // try to get, but no one exists. - isc, rev0, err := GetInitSchema(etcdTestCli, task, downSchema, downTable) - c.Assert(err, IsNil) - c.Assert(rev0, Greater, int64(0)) - c.Assert(isc.IsEmpty(), IsTrue) - - // put the init schema. - rev1, putted, err := PutInitSchemaIfNotExist(etcdTestCli, is1) - c.Assert(err, IsNil) - c.Assert(rev1, Greater, rev0) - c.Assert(putted, IsTrue) - - // get it back. - isc, rev2, err := GetInitSchema(etcdTestCli, task, downSchema, downTable) - c.Assert(err, IsNil) - c.Assert(rev2, Equals, rev1) - c.Assert(isc, DeepEquals, is1) - - // can't put again if a previous one exist. - rev3, putted, err := PutInitSchemaIfNotExist(etcdTestCli, is1) - c.Assert(err, IsNil) - c.Assert(rev3, Equals, rev1) - c.Assert(putted, IsFalse) - rev3, putted, err = PutInitSchemaIfNotExist(etcdTestCli, is2) - c.Assert(err, IsNil) - c.Assert(rev3, Equals, rev1) - c.Assert(putted, IsFalse) - - // put new init schema with different downstream info. - rev4, putted, err := PutInitSchemaIfNotExist(etcdTestCli, is3) - c.Assert(err, IsNil) - c.Assert(rev4, Greater, rev3) - c.Assert(putted, IsTrue) - - // get all init schemas. - initSchemas, rev5, err := GetAllInitSchemas(etcdTestCli) - c.Assert(err, IsNil) - c.Assert(rev5, Equals, rev4) - c.Assert(initSchemas[is1.Task][is1.DownSchema][is1.DownTable], DeepEquals, is1) - c.Assert(initSchemas[is3.Task][is3.DownSchema][is3.DownTable], DeepEquals, is3) - - // delete the schema. - rev6, deleted, err := DeleteInitSchema(etcdTestCli, is1.Task, is1.DownSchema, is1.DownTable) - c.Assert(err, IsNil) - c.Assert(rev6, Greater, rev5) - c.Assert(deleted, IsTrue) - rev7, deleted, err := DeleteInitSchema(etcdTestCli, is3.Task, is3.DownSchema, is3.DownTable) - c.Assert(err, IsNil) - c.Assert(rev7, Greater, rev6) - c.Assert(deleted, IsTrue) +//func (t *testForEtcd) TestInitSchemaJSON(c *C) { +// is1 := NewInitSchema("test", "foo", "bar", nil) +// j, err := is1.toJSON() +// c.Assert(err, IsNil) +// c.Assert(j, Equals, `{"task":"test","down-schema":"foo","down-table":"bar","table-info":null}`) +// c.Assert(j, Equals, is1.String()) +// +// is2, err := initSchemaFromJSON(j) +// c.Assert(err, IsNil) +// c.Assert(is2, DeepEquals, is1) +//} - // not exist now. - initSchemas, rev8, err := GetAllInitSchemas(etcdTestCli) - c.Assert(err, IsNil) - c.Assert(rev8, Equals, rev7) - c.Assert(initSchemas, HasLen, 0) -} +// func (t *testForEtcd) TestInitSchemaEtcd(c *C) { +// defer clearTestInfoOperation(c) +// +// var ( +// task = "test-init-schema-etcd" +// downSchema = "foo" +// downTable = "bar" +// downTable2 = "bar2" +// p = parser.New() +// se = mock.NewContext() +// tblID int64 = 111 +// tblI1 = createTableInfo(c, p, se, tblID, "CREATE TABLE bar (id INT PRIMARY KEY)") +// tblI2 = createTableInfo(c, p, se, tblID, "CREATE TABLE bar (id INT PRIMARY KEY, c1 INT)") +// tblI3 = createTableInfo(c, p, se, tblID, "CREATE TABLE bar2 (id INT PRIMARY KEY, c INT)") +// is1 = NewInitSchema(task, downSchema, downTable, tblI1) +// is2 = NewInitSchema(task, downSchema, downTable, tblI2) +// is3 = NewInitSchema(task, downSchema, downTable2, tblI3) +// ) +// +// // try to get, but no one exists. +// isc, rev0, err := GetInitSchema(etcdTestCli, task, downSchema, downTable) +// c.Assert(err, IsNil) +// c.Assert(rev0, Greater, int64(0)) +// c.Assert(isc.IsEmpty(), IsTrue) +// +// // put the init schema. +// rev1, putted, err := PutInitSchemaIfNotExist(etcdTestCli, is1) +// c.Assert(err, IsNil) +// c.Assert(rev1, Greater, rev0) +// c.Assert(putted, IsTrue) +// +// // get it back. +// isc, rev2, err := GetInitSchema(etcdTestCli, task, downSchema, downTable) +// c.Assert(err, IsNil) +// c.Assert(rev2, Equals, rev1) +// c.Assert(isc, DeepEquals, is1) +// +// // can't put again if a previous one exist. +// rev3, putted, err := PutInitSchemaIfNotExist(etcdTestCli, is1) +// c.Assert(err, IsNil) +// c.Assert(rev3, Equals, rev1) +// c.Assert(putted, IsFalse) +// rev3, putted, err = PutInitSchemaIfNotExist(etcdTestCli, is2) +// c.Assert(err, IsNil) +// c.Assert(rev3, Equals, rev1) +// c.Assert(putted, IsFalse) +// +// // put new init schema with different downstream info. +// rev4, putted, err := PutInitSchemaIfNotExist(etcdTestCli, is3) +// c.Assert(err, IsNil) +// c.Assert(rev4, Greater, rev3) +// c.Assert(putted, IsTrue) +// +// // get all init schemas. +// initSchemas, rev5, err := GetAllInitSchemas(etcdTestCli) +// c.Assert(err, IsNil) +// c.Assert(rev5, Equals, rev4) +// c.Assert(initSchemas[is1.Task][is1.DownSchema][is1.DownTable], DeepEquals, is1) +// c.Assert(initSchemas[is3.Task][is3.DownSchema][is3.DownTable], DeepEquals, is3) +// +// // delete the schema. +// rev6, deleted, err := DeleteInitSchema(etcdTestCli, is1.Task, is1.DownSchema, is1.DownTable) +// c.Assert(err, IsNil) +// c.Assert(rev6, Greater, rev5) +// c.Assert(deleted, IsTrue) +// rev7, deleted, err := DeleteInitSchema(etcdTestCli, is3.Task, is3.DownSchema, is3.DownTable) +// c.Assert(err, IsNil) +// c.Assert(rev7, Greater, rev6) +// c.Assert(deleted, IsTrue) +// +// // not exist now. +// initSchemas, rev8, err := GetAllInitSchemas(etcdTestCli) +// c.Assert(err, IsNil) +// c.Assert(rev8, Equals, rev7) +// c.Assert(initSchemas, HasLen, 0) +// } diff --git a/dm/pkg/terror/error_list.go b/dm/pkg/terror/error_list.go index 40ec97749ea..d0ef47c4f88 100644 --- a/dm/pkg/terror/error_list.go +++ b/dm/pkg/terror/error_list.go @@ -494,6 +494,7 @@ const ( codeMasterFailToImportFromV10x codeMasterInconsistentOptimistDDLsAndInfo codeMasterOptimisticTableInfobeforeNotExist + codeMasterOptimisticDownstreamMetaNotFound ) // DM-worker error code. @@ -1137,6 +1138,7 @@ var ( ErrMasterInconsistentOptimisticDDLsAndInfo = New(codeMasterInconsistentOptimistDDLsAndInfo, ClassDMMaster, ScopeInternal, LevelHigh, "inconsistent count of optimistic ddls and table infos, ddls: %d, table info: %d", "") ErrMasterOptimisticTableInfoBeforeNotExist = New(codeMasterOptimisticTableInfobeforeNotExist, ClassDMMaster, ScopeInternal, LevelHigh, "table-info-before not exist in optimistic ddls: %v", "") + ErrMasterOptimisticDownstreamMetaNotFound = New(codeMasterOptimisticDownstreamMetaNotFound, ClassDMMaster, ScopeInternal, LevelHigh, "downstream database config and meta for task %s not found", "") // DM-worker error. ErrWorkerParseFlagSet = New(codeWorkerParseFlagSet, ClassDMWorker, ScopeInternal, LevelMedium, "parse dm-worker config flag set", "") diff --git a/dm/syncer/checkpoint.go b/dm/syncer/checkpoint.go index c7c0d474cfb..259fa8d10d2 100644 --- a/dm/syncer/checkpoint.go +++ b/dm/syncer/checkpoint.go @@ -24,6 +24,7 @@ import ( "sync" "time" + "github.com/pingcap/errors" "github.com/pingcap/ticdc/dm/dm/config" "github.com/pingcap/ticdc/dm/pkg/binlog" "github.com/pingcap/ticdc/dm/pkg/conn" @@ -244,8 +245,8 @@ type CheckPoint interface { // corresponding to Meta.Flush FlushPointsExcept(tctx *tcontext.Context, snapshotID int, exceptTables []*filter.Table, extraSQLs []string, extraArgs [][]interface{}) error - // FlushPointWithTableInfo flushed the table point with given table info - FlushPointWithTableInfo(tctx *tcontext.Context, table *filter.Table, ti *model.TableInfo) error + // FlushPointsWithTableInfos flushed the table point with given table info + FlushPointsWithTableInfos(tctx *tcontext.Context, table []*filter.Table, ti []*model.TableInfo) error // FlushSafeModeExitPoint flushed the global checkpoint's with given table info FlushSafeModeExitPoint(tctx *tcontext.Context) error @@ -698,45 +699,61 @@ func (cp *RemoteCheckPoint) FlushPointsExcept( return nil } -// FlushPointWithTableInfo implements CheckPoint.FlushPointWithTableInfo. -func (cp *RemoteCheckPoint) FlushPointWithTableInfo(tctx *tcontext.Context, table *filter.Table, ti *model.TableInfo) error { +// FlushPointsWithTableInfos implements CheckPoint.FlushPointsWithTableInfos. +func (cp *RemoteCheckPoint) FlushPointsWithTableInfos(tctx *tcontext.Context, tables []*filter.Table, tis []*model.TableInfo) error { cp.Lock() defer cp.Unlock() - sourceSchema, sourceTable := table.Schema, table.Name - sqls := make([]string, 0, 1) - args := make([][]interface{}, 0, 10) - point := newBinlogPoint(binlog.NewLocation(cp.cfg.Flavor), binlog.NewLocation(cp.cfg.Flavor), nil, nil, cp.cfg.EnableGTID) - - if tablePoints, ok := cp.points[sourceSchema]; ok { - if p, ok2 := tablePoints[sourceTable]; ok2 { - point = p - } - } - - tiBytes, err := json.Marshal(ti) - if err != nil { - return terror.ErrSchemaTrackerCannotSerialize.Delegate(err, sourceSchema, sourceTable) + // should not happened + if len(tables) != len(tis) { + return errors.Errorf("the length of the tables is not equal to the length of the table infos, left: %d, right: %d", len(tables), len(tis)) } - location := point.MySQLLocation() - sql2, arg := cp.genUpdateSQL(sourceSchema, sourceTable, location, nil, tiBytes, false) - sqls = append(sqls, sql2) - args = append(args, arg) + batch := 100 + for i := 0; i < len(tables); i += batch { + end := i + batch + if end > len(tables) { + end = len(tables) + } - // use a new context apart from syncer, to make sure when syncer call `cancel` checkpoint could update - tctx2, cancel := tctx.WithContext(context.Background()).WithTimeout(utils.DefaultDBTimeout) - defer cancel() - _, err = cp.dbConn.ExecuteSQL(tctx2, sqls, args...) - if err != nil { - return err - } + sqls := make([]string, 0, batch) + args := make([][]interface{}, 0, batch) + points := make([]*binlogPoint, 0, batch) + for j := i; j < end; j++ { + table := tables[j] + ti := tis[j] + sourceSchema, sourceTable := table.Schema, table.Name + point := newBinlogPoint(binlog.NewLocation(cp.cfg.Flavor), binlog.NewLocation(cp.cfg.Flavor), nil, nil, cp.cfg.EnableGTID) + if tablePoints, ok := cp.points[sourceSchema]; ok { + if p, ok2 := tablePoints[sourceTable]; ok2 { + point = p + } + } + tiBytes, err := json.Marshal(ti) + if err != nil { + return terror.ErrSchemaTrackerCannotSerialize.Delegate(err, sourceSchema, sourceTable) + } + location := point.MySQLLocation() + sql, arg := cp.genUpdateSQL(sourceSchema, sourceTable, location, nil, tiBytes, false) + sqls = append(sqls, sql) + args = append(args, arg) + points = append(points, point) + } + // use a new context apart from syncer, to make sure when syncer call `cancel` checkpoint could update + tctx2, cancel := tctx.WithContext(context.Background()).WithTimeout(utils.DefaultDBTimeout) + defer cancel() + _, err := cp.dbConn.ExecuteSQL(tctx2, sqls, args...) + if err != nil { + return err + } - err = point.save(point.savedPoint.location, ti) - if err != nil { - return err + for idx, point := range points { + err = point.save(point.savedPoint.location, tis[i+idx]) + if err != nil { + return err + } + point.flush() + } } - point.flush() - return nil } diff --git a/dm/syncer/optimist.go b/dm/syncer/optimist.go index ae383a728c6..8bbbe0479a0 100644 --- a/dm/syncer/optimist.go +++ b/dm/syncer/optimist.go @@ -220,16 +220,16 @@ func (s *Syncer) handleQueryEventOptimistic(qec *queryEventContext) error { } // trackInitTableInfoOptimistic tries to get the initial table info (before modified by other tables) and track it in optimistic shard mode. -func (s *Syncer) trackInitTableInfoOptimistic(sourceTable, targetTable *filter.Table) (*model.TableInfo, error) { - ti, err := s.optimist.GetTableInfo(targetTable.Schema, targetTable.Name) - if err != nil { - return nil, terror.ErrSchemaTrackerCannotGetTable.Delegate(err, sourceTable) - } - if ti != nil { - err = s.schemaTracker.CreateTableIfNotExists(sourceTable, ti) - if err != nil { - return nil, terror.ErrSchemaTrackerCannotCreateTable.Delegate(err, sourceTable) - } - } - return ti, nil -} +// func (s *Syncer) trackInitTableInfoOptimistic(sourceTable, targetTable *filter.Table) (*model.TableInfo, error) { +// ti, err := s.optimist.GetTableInfo(targetTable.Schema, targetTable.Name) +// if err != nil { +// return nil, terror.ErrSchemaTrackerCannotGetTable.Delegate(err, sourceTable) +// } +// if ti != nil { +// err = s.schemaTracker.CreateTableIfNotExists(sourceTable, ti) +// if err != nil { +// return nil, terror.ErrSchemaTrackerCannotCreateTable.Delegate(err, sourceTable) +// } +// } +// return ti, nil +// } diff --git a/dm/syncer/schema.go b/dm/syncer/schema.go index f23b6fa2154..1a626415b19 100644 --- a/dm/syncer/schema.go +++ b/dm/syncer/schema.go @@ -119,7 +119,7 @@ func (s *Syncer) OperateSchema(ctx context.Context, req *pb.OperateWorkerSchemaR if req.Flush { log.L().Info("flush table info", zap.String("table info", newSQL)) - err = s.checkpoint.FlushPointWithTableInfo(tcontext.NewContext(ctx, log.L()), sourceTable, ti) + err = s.checkpoint.FlushPointsWithTableInfos(tcontext.NewContext(ctx, log.L()), []*filter.Table{sourceTable}, []*model.TableInfo{ti}) if err != nil { return "", err } diff --git a/dm/syncer/shardddl/optimist.go b/dm/syncer/shardddl/optimist.go index 6cfd690d5ea..8548cd37464 100644 --- a/dm/syncer/shardddl/optimist.go +++ b/dm/syncer/shardddl/optimist.go @@ -17,6 +17,7 @@ import ( "context" "sync" + filter "github.com/pingcap/tidb-tools/pkg/table-filter" "github.com/pingcap/tidb/parser/model" "go.etcd.io/etcd/clientv3" "go.uber.org/zap" @@ -70,6 +71,25 @@ func (o *Optimist) Init(sourceTables map[string]map[string]map[string]map[string return err } +// Tables clone and return tables +// first one is sourceTable, second one is targetTable +func (o *Optimist) Tables() [][]filter.Table { + o.mu.Lock() + defer o.mu.Unlock() + + tbls := make([][]filter.Table, 0) + for downSchema, downTables := range o.tables.Tables { + for downTable, upSchemas := range downTables { + for upSchema, upTables := range upSchemas { + for upTable := range upTables { + tbls = append(tbls, []filter.Table{{Schema: upSchema, Name: upTable}, {Schema: downSchema, Name: downTable}}) + } + } + } + } + return tbls +} + // Reset resets the internal state of the optimist. func (o *Optimist) Reset() { o.mu.Lock() @@ -162,22 +182,22 @@ func (o *Optimist) DoneOperation(op optimism.Operation) error { } // GetTableInfo tries to get the init schema of the downstream table. -func (o *Optimist) GetTableInfo(downSchema, downTable string) (*model.TableInfo, error) { - if downTable == "" { - return nil, nil - } - - is, rev, err := optimism.GetInitSchema(o.cli, o.task, downSchema, downTable) - if err != nil { - return nil, err - } - if is.IsEmpty() { - o.logger.Info("no init schema exists", zap.String("schema", downSchema), zap.String("table", downTable), zap.Int64("revision", rev)) - } else { - o.logger.Info("got init schema", zap.Stringer("init schema", is)) - } - return is.TableInfo, nil -} +// func (o *Optimist) GetTableInfo(downSchema, downTable string) (*model.TableInfo, error) { +// if downTable == "" { +// return nil, nil +// } +// +// is, rev, err := optimism.GetInitSchema(o.cli, o.task, downSchema, downTable) +// if err != nil { +// return nil, err +// } +// if is.IsEmpty() { +// o.logger.Info("no init schema exists", zap.String("schema", downSchema), zap.String("table", downTable), zap.Int64("revision", rev)) +// } else { +// o.logger.Info("got init schema", zap.Stringer("init schema", is)) +// } +// return is.TableInfo, nil +// } // PendingInfo returns the shard DDL info which is pending to handle. func (o *Optimist) PendingInfo() *optimism.Info { diff --git a/dm/syncer/shardddl/optimist_test.go b/dm/syncer/shardddl/optimist_test.go index 2e4cbd20c5d..a4e7f856537 100644 --- a/dm/syncer/shardddl/optimist_test.go +++ b/dm/syncer/shardddl/optimist_test.go @@ -207,41 +207,41 @@ func (t *testOptimist) TestOptimist(c *C) { c.Assert(o.PendingOperation(), IsNil) } -func (t *testOptimist) TestGetTableInfo(c *C) { - defer clearOptimistTestSourceInfoOperation(c) - - var ( - task = "test-get-table-info" - source = "mysql-replica-1" - logger = log.L() - o = NewOptimist(&logger, etcdTestCli, task, source) - - downSchema = "foo" - downTable = "bar" - p = parser.New() - se = mock.NewContext() - tblID int64 = 111 - is = optimism.NewInitSchema(task, downSchema, downTable, - createTableInfo(c, p, se, tblID, `CREATE TABLE bar (id INT PRIMARY KEY)`)) - ) - - // no table info exist now - ti, err := o.GetTableInfo(downSchema, downTable) - c.Assert(err, IsNil) - c.Assert(ti, IsNil) - - // put the table schema. - _, putted, err := optimism.PutInitSchemaIfNotExist(etcdTestCli, is) - c.Assert(err, IsNil) - c.Assert(putted, IsTrue) - - // can get the table info now. - ti, err = o.GetTableInfo(downSchema, downTable) - c.Assert(err, IsNil) - c.Assert(ti, DeepEquals, is.TableInfo) - - // no table info for database. - ti, err = o.GetTableInfo(downSchema, "") - c.Assert(err, IsNil) - c.Assert(ti, IsNil) -} +// func (t *testOptimist) TestGetTableInfo(c *C) { +// defer clearOptimistTestSourceInfoOperation(c) +// +// var ( +// task = "test-get-table-info" +// source = "mysql-replica-1" +// logger = log.L() +// o = NewOptimist(&logger, etcdTestCli, task, source) +// +// downSchema = "foo" +// downTable = "bar" +// // p = parser.New() +// // se = mock.NewContext() +// // tblID int64 = 111 +// // is = optimism.NewInitSchema(task, downSchema, downTable, +// // createTableInfo(c, p, se, tblID, `CREATE TABLE bar (id INT PRIMARY KEY)`)) +// ) +// +// // no table info exist now +// ti, err := o.GetTableInfo(downSchema, downTable) +// c.Assert(err, IsNil) +// c.Assert(ti, IsNil) +// +// // // put the table schema. +// // _, putted, err := optimism.PutInitSchemaIfNotExist(etcdTestCli, is) +// // c.Assert(err, IsNil) +// // c.Assert(putted, IsTrue) +// +// // // can get the table info now. +// // ti, err = o.GetTableInfo(downSchema, downTable) +// // c.Assert(err, IsNil) +// // c.Assert(ti, DeepEquals, is.TableInfo) +// +// // no table info for database. +// ti, err = o.GetTableInfo(downSchema, "") +// c.Assert(err, IsNil) +// c.Assert(ti, IsNil) +// } diff --git a/dm/syncer/syncer.go b/dm/syncer/syncer.go index 1979478d296..2394f2fb52d 100644 --- a/dm/syncer/syncer.go +++ b/dm/syncer/syncer.go @@ -727,14 +727,14 @@ func (s *Syncer) getTableInfo(tctx *tcontext.Context, sourceTable, targetTable * return ti, nil } - // in optimistic shard mode, we should try to get the init schema (the one before modified by other tables) first. - if s.cfg.ShardMode == config.ShardOptimistic { - ti, err = s.trackInitTableInfoOptimistic(sourceTable, targetTable) - if err != nil { - return nil, err - } - } - + // // in optimistic shard mode, we should try to get the init schema (the one before modified by other tables) first. + // if s.cfg.ShardMode == config.ShardOptimistic { + // ti, err = s.trackInitTableInfoOptimistic(sourceTable, targetTable) + // if err != nil { + // return nil, err + // } + // } + // // if the table does not exist (IsTableNotExists(err)), continue to fetch the table from downstream and create it. if ti == nil { err = s.trackTableInfoFromDownstream(tctx, sourceTable, targetTable) @@ -1451,15 +1451,36 @@ func (s *Syncer) Run(ctx context.Context) (err error) { if err != nil { return err } - if s.cfg.Mode == config.ModeAll && fresh { - delLoadTask = true - flushCheckpoint = true - err = s.loadTableStructureFromDump(ctx) + if fresh { + if s.cfg.Mode == config.ModeAll { + delLoadTask = true + flushCheckpoint = true + err = s.loadTableStructureFromDump(ctx) + if err != nil { + tctx.L().Warn("error happened when load table structure from dump files", zap.Error(err)) + cleanDumpFile = false + } + } + tbls := s.optimist.Tables() + sourceTables := make([]*filter.Table, 0, len(tbls)) + tableInfos := make([]*model.TableInfo, 0, len(tbls)) + for _, tbl := range tbls { + sourceTable := tbl[0] + targetTable := tbl[1] + tableInfo, err := s.getTableInfo(tctx, &sourceTable, &targetTable) + if err != nil { + return err + } + sourceTables = append(sourceTables, &sourceTable) + tableInfos = append(tableInfos, tableInfo) + } + err := s.checkpoint.FlushPointsWithTableInfos(tctx, sourceTables, tableInfos) if err != nil { - tctx.L().Warn("error happened when load table structure from dump files", zap.Error(err)) - cleanDumpFile = false + tctx.L().Error("failed to flush table points with table infos", log.ShortError(err)) } - } else { + } + + if s.cfg.Mode == config.ModeIncrement || !fresh { cleanDumpFile = false } @@ -2984,7 +3005,7 @@ func (s *Syncer) loadTableStructureFromDump(ctx context.Context) error { continue } } - logger.Info("fetch table structure form dump files", + logger.Info("fetch table structure from dump files", zap.Strings("database", dbs), zap.Any("tables", tables)) for _, db := range dbs { From 3022cc5c8912e2e73e9d18757dde57f249e92a43 Mon Sep 17 00:00:00 2001 From: gmhdbjd Date: Thu, 16 Dec 2021 01:14:34 -0500 Subject: [PATCH 02/31] remove old code --- dm/dm/common/common.go | 3 +- dm/dm/common/common_test.go | 21 ---- dm/dm/master/shardddl/optimist.go | 19 --- dm/pkg/shardddl/optimism/info.go | 1 - dm/pkg/shardddl/optimism/keeper.go | 61 ---------- dm/pkg/shardddl/optimism/keeper_test.go | 70 ----------- dm/pkg/shardddl/optimism/ops.go | 2 - dm/pkg/shardddl/optimism/ops_test.go | 14 --- dm/pkg/shardddl/optimism/schema.go | 147 ------------------------ dm/pkg/shardddl/optimism/schema_test.go | 107 ----------------- dm/syncer/optimist.go | 15 --- dm/syncer/shardddl/optimist.go | 18 --- dm/syncer/shardddl/optimist_test.go | 39 ------- dm/syncer/syncer.go | 8 -- 14 files changed, 1 insertion(+), 524 deletions(-) delete mode 100644 dm/pkg/shardddl/optimism/schema.go delete mode 100644 dm/pkg/shardddl/optimism/schema_test.go diff --git a/dm/dm/common/common.go b/dm/dm/common/common.go index 6c8d1a187ca..ce879746ebe 100644 --- a/dm/dm/common/common.go +++ b/dm/dm/common/common.go @@ -78,6 +78,7 @@ var ( ShardDDLOptimismOperationKeyAdapter KeyAdapter = keyHexEncoderDecoder("/dm-master/shardddl-optimism/operation/") // ShardDDLOptimismInitSchemaKeyAdapter is used to store the initial schema (before constructed the lock) of merged tables. // k/v: Encode(task-name, downstream-schema-name, downstream-table-name) -> table schema. + // TODO: prune in etcd when upgrade // ShardDDLOptimismInitSchemaKeyAdapter KeyAdapter = keyHexEncoderDecoder("/dm-master/shardddl-optimism/init-schema/") // ShardDDLOptimismDroppedColumnsKeyAdapter is used to store the columns that are not fully dropped // k/v: Encode(lock-id, column-name, source-id, upstream-schema-name, upstream-table-name) -> int @@ -106,8 +107,6 @@ func keyAdapterKeysLen(s KeyAdapter) int { ShardDDLPessimismInfoKeyAdapter, ShardDDLPessimismOperationKeyAdapter, ShardDDLOptimismSourceTablesKeyAdapter, LoadTaskKeyAdapter: return 2 - // case ShardDDLOptimismInitSchemaKeyAdapter: - // return 3 case ShardDDLOptimismInfoKeyAdapter, ShardDDLOptimismOperationKeyAdapter: return 4 case ShardDDLOptimismDroppedColumnsKeyAdapter: diff --git a/dm/dm/common/common_test.go b/dm/dm/common/common_test.go index f037f8b13f1..672eb3101f1 100644 --- a/dm/dm/common/common_test.go +++ b/dm/dm/common/common_test.go @@ -60,11 +60,6 @@ func (t *testCommon) TestKeyAdapter(c *C) { adapter: UpstreamSubTaskKeyAdapter, want: "/dm-master/upstream/subtask/6d7973716c31/74657374", }, - // { - // keys: []string{"test", "target_db", "target_table"}, - // adapter: ShardDDLOptimismInitSchemaKeyAdapter, - // want: "/dm-master/shardddl-optimism/init-schema/74657374/7461726765745f6462/7461726765745f7461626c65", - // }, { keys: []string{"test", "mysql_replica_01", "target_db", "target_table"}, adapter: ShardDDLOptimismInfoKeyAdapter, @@ -107,11 +102,6 @@ func (t *testCommon) TestEncodeAsPrefix(c *C) { adapter: UpstreamSubTaskKeyAdapter, want: "/dm-master/upstream/subtask/6d7973716c31/", }, - // { - // keys: []string{"test", "target_db"}, - // adapter: ShardDDLOptimismInitSchemaKeyAdapter, - // want: "/dm-master/shardddl-optimism/init-schema/74657374/7461726765745f6462/", - // }, } for _, ca := range testCases { @@ -121,17 +111,6 @@ func (t *testCommon) TestEncodeAsPrefix(c *C) { c.Assert(err, NotNil) } - // keys := []string{"test", "target_db", "target_table"} - // fullEncodedKey := ShardDDLOptimismInitSchemaKeyAdapter.Encode(keys...) - // prefixEncodedKey := ShardDDLOptimismInitSchemaKeyAdapter.Encode(keys[:len(keys)-1]...) - // c.Assert(strings.HasPrefix(fullEncodedKey, prefixEncodedKey), IsTrue) - // - // keys2 := []string{"test", "target_db_2", "target_table_2"} - // fullEncodedKey2 := ShardDDLOptimismInitSchemaKeyAdapter.Encode(keys2...) - // prefixEncodedKey2 := ShardDDLOptimismInitSchemaKeyAdapter.Encode(keys2[:len(keys2)-1]...) - // - // c.Assert(strings.HasPrefix(fullEncodedKey, prefixEncodedKey2), IsFalse) - // c.Assert(strings.HasPrefix(fullEncodedKey2, prefixEncodedKey), IsFalse) } func (t *testCommon) TestIsErrNetClosing(c *C) { diff --git a/dm/dm/master/shardddl/optimist.go b/dm/dm/master/shardddl/optimist.go index 2a82d202a74..e3f854fc3df 100644 --- a/dm/dm/master/shardddl/optimist.go +++ b/dm/dm/master/shardddl/optimist.go @@ -249,12 +249,6 @@ func (o *Optimist) rebuildLocks() (revSource, revInfo, revOperation int64, err e } o.logger.Info("get history shard DDL lock operation", zap.Int64("revision", revOperation)) - // initSchemas, revInitSchemas, err := optimism.GetAllInitSchemas(o.cli) - // if err != nil { - // return 0, 0, 0, err - // } - // o.logger.Info("get all init schemas", zap.Int64("revision", revInitSchemas)) - colm, _, err := optimism.GetAllDroppedColumns(o.cli) if err != nil { // only log the error, and don't return it to forbid the startup of the DM-master leader. @@ -570,18 +564,6 @@ func (o *Optimist) handleLock(info optimism.Info, tts []optimism.TargetTable, sk default: o.logger.Info("the shard DDL lock returned some DDLs", zap.String("lock", lockID), zap.Strings("ddls", newDDLs), zap.Strings("cols", cols), zap.String("info", info.ShortString()), zap.Bool("is deleted", info.IsDeleted)) - - // try to record the init schema before applied the DDL to the downstream. - // initSchema := optimism.NewInitSchema(info.Task, info.DownSchema, info.DownTable, info.TableInfoBefore) - // rev, putted, err2 := optimism.PutInitSchemaIfNotExist(o.cli, initSchema) - // switch { - // case err2 != nil: - // return err2 - // case putted: - // o.logger.Info("recorded the initial schema", zap.String("info", info.ShortString())) - // default: - // o.logger.Debug("skip to record the initial schema", zap.String("info", info.ShortString()), zap.Int64("revision", rev)) - // } } lock := o.lk.FindLock(lockID) @@ -682,7 +664,6 @@ func (o *Optimist) deleteInfosOps(lock *optimism.Lock) (bool, error) { } } // NOTE: we rely on only `task`, `downSchema`, and `downTable` used for deletion. - // initSchema := optimism.NewInitSchema(lock.Task, lock.DownSchema, lock.DownTable, nil) rev, deleted, err := optimism.DeleteInfosOperationsSchemaColumn(o.cli, infos, ops, lock.Task, lock.DownSchema, lock.DownTable) if err != nil { return deleted, err diff --git a/dm/pkg/shardddl/optimism/info.go b/dm/pkg/shardddl/optimism/info.go index 0ca280a0245..f5f063ee81e 100644 --- a/dm/pkg/shardddl/optimism/info.go +++ b/dm/pkg/shardddl/optimism/info.go @@ -304,7 +304,6 @@ func ClearTestInfoOperationSchema(cli *clientv3.Client) error { clearSource := clientv3.OpDelete(common.ShardDDLOptimismSourceTablesKeyAdapter.Path(), clientv3.WithPrefix()) clearInfo := clientv3.OpDelete(common.ShardDDLOptimismInfoKeyAdapter.Path(), clientv3.WithPrefix()) clearOp := clientv3.OpDelete(common.ShardDDLOptimismOperationKeyAdapter.Path(), clientv3.WithPrefix()) - // clearISOp := clientv3.OpDelete(common.ShardDDLOptimismInitSchemaKeyAdapter.Path(), clientv3.WithPrefix()) clearColumns := clientv3.OpDelete(common.ShardDDLOptimismDroppedColumnsKeyAdapter.Path(), clientv3.WithPrefix()) _, err := cli.Txn(context.Background()).Then(clearSource, clearInfo, clearOp, clearColumns).Commit() return err diff --git a/dm/pkg/shardddl/optimism/keeper.go b/dm/pkg/shardddl/optimism/keeper.go index 81f2eeaba08..a65e317972b 100644 --- a/dm/pkg/shardddl/optimism/keeper.go +++ b/dm/pkg/shardddl/optimism/keeper.go @@ -76,67 +76,6 @@ func (lk *LockKeeper) getDownstreamMeta(task string) (*DownstreamMeta, error) { return downstreamMeta, nil } -// // RebuildLocksAndTables rebuild the locks and tables. -// func (lk *LockKeeper) RebuildLocksAndTables( -// cli *clientv3.Client, -// ifm map[string]map[string]map[string]map[string]Info, -// colm map[string]map[string]map[string]map[string]map[string]DropColumnStage, -// lockJoined map[string]schemacmp.Table, -// lockTTS map[string][]TargetTable, -// missTable map[string]map[string]map[string]map[string]schemacmp.Table, -// ) { -// var ( -// lock *Lock -// ok bool -// ) -// for task, taskInfos := range ifm { -// for source, sourceInfos := range taskInfos { -// for schema, schemaInfos := range sourceInfos { -// for table, info := range schemaInfos { -// lockID := utils.GenDDLLockID(info.Task, info.DownSchema, info.DownTable) -// if lock, ok = lk.locks[lockID]; !ok { -// downstreamMeta, err := lk.getDownstreamMeta(task) -// if err != nil { -// log.L().Error("get downstream meta", log.ShortError(err)) -// } -// lock = NewLock(cli, lockID, info.Task, info.DownSchema, info.DownTable, lockJoined[lockID], lockTTS[lockID], downstreamMeta) -// } -// // filter info which doesn't have SourceTable -// // SourceTable will be changed after user update block-allow-list -// // But old infos still remain in etcd. -// // TODO: add a mechanism to remove all outdated infos in etcd. -// if !lock.TableExist(info.Source, info.UpSchema, info.UpTable) { -// delete(ifm[task][source][schema], table) -// continue -// } -// lk.locks[lockID] = lock -// lock.tables[info.Source][info.UpSchema][info.UpTable] = schemacmp.Encode(info.TableInfoBefore) -// if columns, ok := colm[lockID]; ok { -// lock.columns = columns -// } -// } -// } -// } -// } -// -// // update missTable's table info for locks -// for lockID, lockTable := range missTable { -// for source, sourceTable := range lockTable { -// for schema, schemaTable := range sourceTable { -// for table, tableinfo := range schemaTable { -// if _, ok := lk.locks[lockID]; !ok { -// continue -// } -// if !lk.locks[lockID].TableExist(source, schema, table) { -// continue -// } -// lk.locks[lockID].tables[source][schema][table] = tableinfo -// } -// } -// } -// } -// } - // TrySync tries to sync the lock. func (lk *LockKeeper) TrySync(cli *clientv3.Client, info Info, tts []TargetTable) (string, []string, []string, error) { var ( diff --git a/dm/pkg/shardddl/optimism/keeper_test.go b/dm/pkg/shardddl/optimism/keeper_test.go index 896b0033d40..c3f785950a9 100644 --- a/dm/pkg/shardddl/optimism/keeper_test.go +++ b/dm/pkg/shardddl/optimism/keeper_test.go @@ -413,76 +413,6 @@ func (t *testKeeper) TestTargetTablesForTask(c *C) { }) } -//func (t *testKeeper) TestRebuildLocksAndTables(c *C) { -// defer clearTestInfoOperation(c) -// var ( -// lk = NewLockKeeper(getDownstreamMeta) -// task = "task" -// source1 = "mysql-replica-1" -// source2 = "mysql-replica-2" -// upSchema = "foo" -// upTable = "bar" -// downSchema = "db" -// downTable = "tbl" -// DDLs1 = []string{"ALTER TABLE bar ADD COLUMN c1 INT"} -// DDLs2 = []string{"ALTER TABLE bar DROP COLUMN c1"} -// p = parser.New() -// se = mock.NewContext() -// tblID int64 = 111 -// ti0 = createTableInfo(c, p, se, tblID, `CREATE TABLE bar (id INT PRIMARY KEY)`) -// ti1 = createTableInfo(c, p, se, tblID, `CREATE TABLE bar (id INT PRIMARY KEY, c1 INT)`) -// ti2 = createTableInfo(c, p, se, tblID, `CREATE TABLE bar (id INT PRIMARY KEY, c1 INT, c2 INT)`) -// ti3 = createTableInfo(c, p, se, tblID, `CREATE TABLE bar (id INT PRIMARY KEY, c2 INT)`) -// -// i11 = NewInfo(task, source1, upSchema, upTable, downSchema, downTable, DDLs1, ti0, []*model.TableInfo{ti1}) -// i21 = NewInfo(task, source2, upSchema, upTable, downSchema, downTable, DDLs2, ti2, []*model.TableInfo{ti3}) -// -// tts = []TargetTable{ -// newTargetTable(task, source1, downSchema, downTable, map[string]map[string]struct{}{upSchema: {upTable: struct{}{}}}), -// newTargetTable(task, source2, downSchema, downTable, map[string]map[string]struct{}{upSchema: {upTable: struct{}{}}}), -// } -// -// lockID = utils.GenDDLLockID(task, downSchema, downTable) -// -// ifm = map[string]map[string]map[string]map[string]Info{ -// task: { -// source1: {upSchema: {upTable: i11}}, -// source2: {upSchema: {upTable: i21}}, -// }, -// } -// colm = map[string]map[string]map[string]map[string]map[string]DropColumnStage{ -// lockID: { -// "c3": { -// source1: {upSchema: {upTable: DropNotDone}}, -// source2: {upSchema: {upTable: DropNotDone}}, -// }, -// }, -// } -// lockJoined = map[string]schemacmp.Table{ -// lockID: schemacmp.Encode(ti2), -// } -// lockTTS = map[string][]TargetTable{ -// lockID: tts, -// } -// ) -// -// lk.RebuildLocksAndTables(etcdTestCli, ifm, colm, lockJoined, lockTTS, nil) -// locks := lk.Locks() -// c.Assert(len(locks), Equals, 1) -// lock, ok := locks[lockID] -// c.Assert(ok, IsTrue) -// cmp, err := lock.Joined().Compare(schemacmp.Encode(ti2)) -// c.Assert(err, IsNil) -// c.Assert(cmp, Equals, 0) -// cmp, err = lock.tables[source1][upSchema][upTable].Compare(schemacmp.Encode(ti0)) -// c.Assert(err, IsNil) -// c.Assert(cmp, Equals, 0) -// cmp, err = lock.tables[source2][upSchema][upTable].Compare(schemacmp.Encode(ti2)) -// c.Assert(err, IsNil) -// c.Assert(cmp, Equals, 0) -// c.Assert(lock.columns, DeepEquals, colm[lockID]) -//} - func getDownstreamMeta(string) (*config.DBConfig, string) { return nil, "" } diff --git a/dm/pkg/shardddl/optimism/ops.go b/dm/pkg/shardddl/optimism/ops.go index 1a05f84c622..b7ede23c2fd 100644 --- a/dm/pkg/shardddl/optimism/ops.go +++ b/dm/pkg/shardddl/optimism/ops.go @@ -62,7 +62,6 @@ func DeleteInfosOperationsSchemaColumn(cli *clientv3.Client, infos []Info, ops [ for _, op := range ops { opsDel = append(opsDel, deleteOperationOp(op)) } - // opsDel = append(opsDel, deleteInitSchemaOp(schema.Task, schema.DownSchema, schema.DownTable)) opsDel = append(opsDel, deleteDroppedColumnsByLockOp(utils.GenDDLLockID(task, downSchema, downTable))) resp, rev, err := etcdutil.DoOpsInOneCmpsTxnWithRetry(cli, cmps, opsDel, []clientv3.Op{}) if err != nil { @@ -77,7 +76,6 @@ func DeleteInfosOperationsTablesSchemasByTask(cli *clientv3.Client, task string, opsDel = append(opsDel, clientv3.OpDelete(common.ShardDDLOptimismInfoKeyAdapter.Encode(task), clientv3.WithPrefix())) opsDel = append(opsDel, clientv3.OpDelete(common.ShardDDLOptimismOperationKeyAdapter.Encode(task), clientv3.WithPrefix())) opsDel = append(opsDel, clientv3.OpDelete(common.ShardDDLOptimismSourceTablesKeyAdapter.Encode(task), clientv3.WithPrefix())) - // opsDel = append(opsDel, clientv3.OpDelete(common.ShardDDLOptimismInitSchemaKeyAdapter.Encode(task), clientv3.WithPrefix())) for lockID := range lockIDSet { opsDel = append(opsDel, clientv3.OpDelete(common.ShardDDLOptimismDroppedColumnsKeyAdapter.Encode(lockID), clientv3.WithPrefix())) } diff --git a/dm/pkg/shardddl/optimism/ops_test.go b/dm/pkg/shardddl/optimism/ops_test.go index 99397c3e8e3..1353a363673 100644 --- a/dm/pkg/shardddl/optimism/ops_test.go +++ b/dm/pkg/shardddl/optimism/ops_test.go @@ -30,7 +30,6 @@ func (t *testForEtcd) TestDeleteInfosOperationsSchema(c *C) { DDLs = []string{"ALTER TABLE bar ADD COLUMN c1 INT"} info = NewInfo(task, source, upSchema, upTable, downSchema, downTable, DDLs, nil, nil) op = NewOperation("test-ID", task, source, upSchema, upTable, DDLs, ConflictResolved, "", false, []string{}) - // is = NewInitSchema(task, downSchema, downTable, nil) ) // put info. @@ -52,13 +51,6 @@ func (t *testForEtcd) TestDeleteInfosOperationsSchema(c *C) { c.Assert(opm, HasLen, 1) c.Assert(opm[task][source][upSchema][upTable], DeepEquals, op) - // // put init schema. - // _, _, err = PutInitSchemaIfNotExist(etcdTestCli, is) - // c.Assert(err, IsNil) - // isc, _, err := GetInitSchema(etcdTestCli, is.Task, is.DownSchema, is.DownTable) - // c.Assert(err, IsNil) - // c.Assert(isc, DeepEquals, is) - // DELETE info and operation with version 0 _, deleted, err := DeleteInfosOperationsSchemaColumn(etcdTestCli, []Info{info}, []Operation{op}, task, downSchema, downTable) c.Assert(err, IsNil) @@ -71,9 +63,6 @@ func (t *testForEtcd) TestDeleteInfosOperationsSchema(c *C) { opm, _, err = GetAllOperations(etcdTestCli) c.Assert(err, IsNil) c.Assert(opm, HasLen, 1) - // isc, _, err = GetInitSchema(etcdTestCli, is.Task, is.DownSchema, is.DownTable) - // c.Assert(err, IsNil) - // c.Assert(isc.IsEmpty(), IsFalse) // DELETE info and operation with version 1 _, deleted, err = DeleteInfosOperationsSchemaColumn(etcdTestCli, []Info{infoWithVer}, []Operation{op}, task, downSchema, downTable) @@ -87,9 +76,6 @@ func (t *testForEtcd) TestDeleteInfosOperationsSchema(c *C) { opm, _, err = GetAllOperations(etcdTestCli) c.Assert(err, IsNil) c.Assert(opm, HasLen, 0) - // isc, _, err = GetInitSchema(etcdTestCli, is.Task, is.DownSchema, is.DownTable) - // c.Assert(err, IsNil) - // c.Assert(isc.IsEmpty(), IsTrue) } func (t *testForEtcd) TestSourceTablesInfo(c *C) { diff --git a/dm/pkg/shardddl/optimism/schema.go b/dm/pkg/shardddl/optimism/schema.go deleted file mode 100644 index 7d4542e2300..00000000000 --- a/dm/pkg/shardddl/optimism/schema.go +++ /dev/null @@ -1,147 +0,0 @@ -// Copyright 2020 PingCAP, Inc. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// See the License for the specific language governing permissions and -// limitations under the License. - -package optimism - -// -// import ( -// "encoding/json" -// -// "github.com/pingcap/tidb/parser/model" -// ) -// -// // InitSchema represents the initial schema (schema before the lock constructed) of a merged table. -// // NOTE: `Task`, `DownSchema` and `DownTable` are redundant in the etcd key path for convenient. -// type InitSchema struct { -// Task string `json:"task"` // data migration task name -// DownSchema string `json:"down-schema"` // downstream/target schema name -// DownTable string `json:"down-table"` // downstream/target table name -// TableInfo *model.TableInfo `json:"table-info"` // the initial table info (schema) -// } -// -// // NewInitSchema creates a new InitSchema instance. -// func NewInitSchema(task, downSchema, downTable string, tableInfo *model.TableInfo) InitSchema { -// return InitSchema{ -// Task: task, -// DownSchema: downSchema, -// DownTable: downTable, -// TableInfo: tableInfo, -// } -// } -// -// // String implements Stringer interface. -// func (is InitSchema) String() string { -// s, _ := is.toJSON() -// return s -// } -// -// // toJSON returns the string of JSON represent. -// func (is InitSchema) toJSON() (string, error) { -// data, err := json.Marshal(is) -// if err != nil { -// return "", err -// } -// return string(data), nil -// } -// -// // IsEmpty returns true when this InitSchema has no value. -// func (is InitSchema) IsEmpty() bool { -// var emptyIS InitSchema -// return is == emptyIS -// } -// -// // infoFromJSON constructs InitSchema from its JSON represent. -// func initSchemaFromJSON(s string) (is InitSchema, err error) { -// err = json.Unmarshal([]byte(s), &is) -// return -// } -// -// // GetInitSchema gets the InitSchema for the specified downstream table. -// func GetInitSchema(cli *clientv3.Client, task, downSchema, downTable string) (InitSchema, int64, error) { -// var is InitSchema -// op := clientv3.OpGet(common.ShardDDLOptimismInitSchemaKeyAdapter.Encode(task, downSchema, downTable)) -// respTxn, rev, err := etcdutil.DoOpsInOneTxnWithRetry(cli, op) -// if err != nil { -// return is, 0, err -// } -// resp := respTxn.Responses[0].GetResponseRange() -// -// if resp.Count > 0 { -// is, err = initSchemaFromJSON(string(resp.Kvs[0].Value)) -// if err != nil { -// return is, 0, err -// } -// } -// return is, rev, nil -// } -// -// // GetAllInitSchemas gets all init schemas from etcd. -// // This function should often be called by DM-master. -// // k/k/k/v: task-name -> downstream-schema-name -> downstream-table-name -> InitSchema. -// func GetAllInitSchemas(cli *clientv3.Client) (map[string]map[string]map[string]InitSchema, int64, error) { -// initSchemas := make(map[string]map[string]map[string]InitSchema) -// op := clientv3.OpGet(common.ShardDDLOptimismInitSchemaKeyAdapter.Path(), clientv3.WithPrefix()) -// respTxn, rev, err := etcdutil.DoOpsInOneTxnWithRetry(cli, op) -// if err != nil { -// return nil, 0, err -// } -// resp := respTxn.Responses[0].GetResponseRange() -// -// for _, kv := range resp.Kvs { -// schema, err := initSchemaFromJSON(string(kv.Value)) -// if err != nil { -// return nil, 0, err -// } -// if _, ok := initSchemas[schema.Task]; !ok { -// initSchemas[schema.Task] = make(map[string]map[string]InitSchema) -// } -// if _, ok := initSchemas[schema.Task][schema.DownSchema]; !ok { -// initSchemas[schema.Task][schema.DownSchema] = make(map[string]InitSchema) -// } -// initSchemas[schema.Task][schema.DownSchema][schema.DownTable] = schema -// } -// return initSchemas, rev, nil -// } -// -// // PutInitSchemaIfNotExist puts the InitSchema into ectd if no previous one exists. -// func PutInitSchemaIfNotExist(cli *clientv3.Client, is InitSchema) (rev int64, putted bool, err error) { -// value, err := is.toJSON() -// if err != nil { -// return 0, false, err -// } -// key := common.ShardDDLOptimismInitSchemaKeyAdapter.Encode(is.Task, is.DownSchema, is.DownTable) -// -// cmp := clientv3util.KeyMissing(key) -// op := clientv3.OpPut(key, value) -// -// resp, rev, err := etcdutil.DoOpsInOneCmpsTxnWithRetry(cli, []clientv3.Cmp{cmp}, []clientv3.Op{op}, []clientv3.Op{}) -// if err != nil { -// return 0, false, err -// } -// return rev, resp.Succeeded, nil -// } - -// // DeleteInitSchema tries to delete the InitSchema for the specified downstream table. -// func DeleteInitSchema(cli *clientv3.Client, task, downSchema, downTable string) (rev int64, deleted bool, err error) { -// op := deleteInitSchemaOp(task, downSchema, downTable) -// resp, rev, err := etcdutil.DoOpsInOneTxnWithRetry(cli, op) -// if err != nil { -// return 0, false, err -// } -// return rev, resp.Succeeded, nil -// } -// -// // deleteInitSchemaOp returns a DELETE etcd operation for init schema. -// func deleteInitSchemaOp(task, downSchema, downTable string) clientv3.Op { -// return clientv3.OpDelete(common.ShardDDLOptimismInitSchemaKeyAdapter.Encode(task, downSchema, downTable)) -// } diff --git a/dm/pkg/shardddl/optimism/schema_test.go b/dm/pkg/shardddl/optimism/schema_test.go deleted file mode 100644 index db23bfce59c..00000000000 --- a/dm/pkg/shardddl/optimism/schema_test.go +++ /dev/null @@ -1,107 +0,0 @@ -// Copyright 2020 PingCAP, Inc. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// See the License for the specific language governing permissions and -// limitations under the License. - -package optimism - -//import ( -// . "github.com/pingcap/check" -//) - -//func (t *testForEtcd) TestInitSchemaJSON(c *C) { -// is1 := NewInitSchema("test", "foo", "bar", nil) -// j, err := is1.toJSON() -// c.Assert(err, IsNil) -// c.Assert(j, Equals, `{"task":"test","down-schema":"foo","down-table":"bar","table-info":null}`) -// c.Assert(j, Equals, is1.String()) -// -// is2, err := initSchemaFromJSON(j) -// c.Assert(err, IsNil) -// c.Assert(is2, DeepEquals, is1) -//} - -// func (t *testForEtcd) TestInitSchemaEtcd(c *C) { -// defer clearTestInfoOperation(c) -// -// var ( -// task = "test-init-schema-etcd" -// downSchema = "foo" -// downTable = "bar" -// downTable2 = "bar2" -// p = parser.New() -// se = mock.NewContext() -// tblID int64 = 111 -// tblI1 = createTableInfo(c, p, se, tblID, "CREATE TABLE bar (id INT PRIMARY KEY)") -// tblI2 = createTableInfo(c, p, se, tblID, "CREATE TABLE bar (id INT PRIMARY KEY, c1 INT)") -// tblI3 = createTableInfo(c, p, se, tblID, "CREATE TABLE bar2 (id INT PRIMARY KEY, c INT)") -// is1 = NewInitSchema(task, downSchema, downTable, tblI1) -// is2 = NewInitSchema(task, downSchema, downTable, tblI2) -// is3 = NewInitSchema(task, downSchema, downTable2, tblI3) -// ) -// -// // try to get, but no one exists. -// isc, rev0, err := GetInitSchema(etcdTestCli, task, downSchema, downTable) -// c.Assert(err, IsNil) -// c.Assert(rev0, Greater, int64(0)) -// c.Assert(isc.IsEmpty(), IsTrue) -// -// // put the init schema. -// rev1, putted, err := PutInitSchemaIfNotExist(etcdTestCli, is1) -// c.Assert(err, IsNil) -// c.Assert(rev1, Greater, rev0) -// c.Assert(putted, IsTrue) -// -// // get it back. -// isc, rev2, err := GetInitSchema(etcdTestCli, task, downSchema, downTable) -// c.Assert(err, IsNil) -// c.Assert(rev2, Equals, rev1) -// c.Assert(isc, DeepEquals, is1) -// -// // can't put again if a previous one exist. -// rev3, putted, err := PutInitSchemaIfNotExist(etcdTestCli, is1) -// c.Assert(err, IsNil) -// c.Assert(rev3, Equals, rev1) -// c.Assert(putted, IsFalse) -// rev3, putted, err = PutInitSchemaIfNotExist(etcdTestCli, is2) -// c.Assert(err, IsNil) -// c.Assert(rev3, Equals, rev1) -// c.Assert(putted, IsFalse) -// -// // put new init schema with different downstream info. -// rev4, putted, err := PutInitSchemaIfNotExist(etcdTestCli, is3) -// c.Assert(err, IsNil) -// c.Assert(rev4, Greater, rev3) -// c.Assert(putted, IsTrue) -// -// // get all init schemas. -// initSchemas, rev5, err := GetAllInitSchemas(etcdTestCli) -// c.Assert(err, IsNil) -// c.Assert(rev5, Equals, rev4) -// c.Assert(initSchemas[is1.Task][is1.DownSchema][is1.DownTable], DeepEquals, is1) -// c.Assert(initSchemas[is3.Task][is3.DownSchema][is3.DownTable], DeepEquals, is3) -// -// // delete the schema. -// rev6, deleted, err := DeleteInitSchema(etcdTestCli, is1.Task, is1.DownSchema, is1.DownTable) -// c.Assert(err, IsNil) -// c.Assert(rev6, Greater, rev5) -// c.Assert(deleted, IsTrue) -// rev7, deleted, err := DeleteInitSchema(etcdTestCli, is3.Task, is3.DownSchema, is3.DownTable) -// c.Assert(err, IsNil) -// c.Assert(rev7, Greater, rev6) -// c.Assert(deleted, IsTrue) -// -// // not exist now. -// initSchemas, rev8, err := GetAllInitSchemas(etcdTestCli) -// c.Assert(err, IsNil) -// c.Assert(rev8, Equals, rev7) -// c.Assert(initSchemas, HasLen, 0) -// } diff --git a/dm/syncer/optimist.go b/dm/syncer/optimist.go index 8bbbe0479a0..8b22c89997b 100644 --- a/dm/syncer/optimist.go +++ b/dm/syncer/optimist.go @@ -218,18 +218,3 @@ func (s *Syncer) handleQueryEventOptimistic(qec *queryEventContext) error { s.tctx.L().Info("finish to handle ddls in optimistic shard mode", zap.String("event", "query"), zap.Stringer("queryEventContext", qec)) return nil } - -// trackInitTableInfoOptimistic tries to get the initial table info (before modified by other tables) and track it in optimistic shard mode. -// func (s *Syncer) trackInitTableInfoOptimistic(sourceTable, targetTable *filter.Table) (*model.TableInfo, error) { -// ti, err := s.optimist.GetTableInfo(targetTable.Schema, targetTable.Name) -// if err != nil { -// return nil, terror.ErrSchemaTrackerCannotGetTable.Delegate(err, sourceTable) -// } -// if ti != nil { -// err = s.schemaTracker.CreateTableIfNotExists(sourceTable, ti) -// if err != nil { -// return nil, terror.ErrSchemaTrackerCannotCreateTable.Delegate(err, sourceTable) -// } -// } -// return ti, nil -// } diff --git a/dm/syncer/shardddl/optimist.go b/dm/syncer/shardddl/optimist.go index 8548cd37464..0a056740efd 100644 --- a/dm/syncer/shardddl/optimist.go +++ b/dm/syncer/shardddl/optimist.go @@ -181,24 +181,6 @@ func (o *Optimist) DoneOperation(op optimism.Operation) error { return nil } -// GetTableInfo tries to get the init schema of the downstream table. -// func (o *Optimist) GetTableInfo(downSchema, downTable string) (*model.TableInfo, error) { -// if downTable == "" { -// return nil, nil -// } -// -// is, rev, err := optimism.GetInitSchema(o.cli, o.task, downSchema, downTable) -// if err != nil { -// return nil, err -// } -// if is.IsEmpty() { -// o.logger.Info("no init schema exists", zap.String("schema", downSchema), zap.String("table", downTable), zap.Int64("revision", rev)) -// } else { -// o.logger.Info("got init schema", zap.Stringer("init schema", is)) -// } -// return is.TableInfo, nil -// } - // PendingInfo returns the shard DDL info which is pending to handle. func (o *Optimist) PendingInfo() *optimism.Info { o.mu.RLock() diff --git a/dm/syncer/shardddl/optimist_test.go b/dm/syncer/shardddl/optimist_test.go index a4e7f856537..a79dfd6538e 100644 --- a/dm/syncer/shardddl/optimist_test.go +++ b/dm/syncer/shardddl/optimist_test.go @@ -206,42 +206,3 @@ func (t *testOptimist) TestOptimist(c *C) { c.Assert(o.PendingInfo(), IsNil) c.Assert(o.PendingOperation(), IsNil) } - -// func (t *testOptimist) TestGetTableInfo(c *C) { -// defer clearOptimistTestSourceInfoOperation(c) -// -// var ( -// task = "test-get-table-info" -// source = "mysql-replica-1" -// logger = log.L() -// o = NewOptimist(&logger, etcdTestCli, task, source) -// -// downSchema = "foo" -// downTable = "bar" -// // p = parser.New() -// // se = mock.NewContext() -// // tblID int64 = 111 -// // is = optimism.NewInitSchema(task, downSchema, downTable, -// // createTableInfo(c, p, se, tblID, `CREATE TABLE bar (id INT PRIMARY KEY)`)) -// ) -// -// // no table info exist now -// ti, err := o.GetTableInfo(downSchema, downTable) -// c.Assert(err, IsNil) -// c.Assert(ti, IsNil) -// -// // // put the table schema. -// // _, putted, err := optimism.PutInitSchemaIfNotExist(etcdTestCli, is) -// // c.Assert(err, IsNil) -// // c.Assert(putted, IsTrue) -// -// // // can get the table info now. -// // ti, err = o.GetTableInfo(downSchema, downTable) -// // c.Assert(err, IsNil) -// // c.Assert(ti, DeepEquals, is.TableInfo) -// -// // no table info for database. -// ti, err = o.GetTableInfo(downSchema, "") -// c.Assert(err, IsNil) -// c.Assert(ti, IsNil) -// } diff --git a/dm/syncer/syncer.go b/dm/syncer/syncer.go index 2394f2fb52d..93f8ffa13fa 100644 --- a/dm/syncer/syncer.go +++ b/dm/syncer/syncer.go @@ -727,14 +727,6 @@ func (s *Syncer) getTableInfo(tctx *tcontext.Context, sourceTable, targetTable * return ti, nil } - // // in optimistic shard mode, we should try to get the init schema (the one before modified by other tables) first. - // if s.cfg.ShardMode == config.ShardOptimistic { - // ti, err = s.trackInitTableInfoOptimistic(sourceTable, targetTable) - // if err != nil { - // return nil, err - // } - // } - // // if the table does not exist (IsTableNotExists(err)), continue to fetch the table from downstream and create it. if ti == nil { err = s.trackTableInfoFromDownstream(tctx, sourceTable, targetTable) From 25c1d591537693d29f04fe407f9f20901b526a2b Mon Sep 17 00:00:00 2001 From: gmhdbjd Date: Fri, 17 Dec 2021 07:12:08 -0500 Subject: [PATCH 03/31] remove meta when stop task --- dm/dm/common/common_test.go | 1 - dm/dm/master/server.go | 13 +++++++++-- dm/dm/master/shardddl/optimist.go | 39 +++++++++++++++++++++++++++++-- dm/pkg/shardddl/optimism/info.go | 18 ++++++++++++++ dm/pkg/shardddl/optimism/lock.go | 15 ++++++++++++ dm/pkg/shardddl/optimism/ops.go | 12 +++++++++- dm/syncer/checkpoint.go | 9 ++++++- 7 files changed, 100 insertions(+), 7 deletions(-) diff --git a/dm/dm/common/common_test.go b/dm/dm/common/common_test.go index 672eb3101f1..27372d9d4c4 100644 --- a/dm/dm/common/common_test.go +++ b/dm/dm/common/common_test.go @@ -110,7 +110,6 @@ func (t *testCommon) TestEncodeAsPrefix(c *C) { _, err := ca.adapter.Decode(encKey) c.Assert(err, NotNil) } - } func (t *testCommon) TestIsErrNetClosing(c *C) { diff --git a/dm/dm/master/server.go b/dm/dm/master/server.go index f58902ebe55..089a061d870 100644 --- a/dm/dm/master/server.go +++ b/dm/dm/master/server.go @@ -551,7 +551,16 @@ func (s *Server) OperateTask(ctx context.Context, req *pb.OperateTaskRequest) (* } var err error if expect == pb.Stage_Stopped { - err = s.scheduler.RemoveSubTasks(req.Name, sources...) + if err = s.scheduler.RemoveSubTasks(req.Name, sources...); err != nil { + resp.Msg = err.Error() + // nolint:nilerr + return resp, nil + } + if len(req.Sources) == 0 { + err = s.optimist.RemoveMetaDataWithTask(req.Name) + } else { + err = s.optimist.RemoveMetaDataWithTaskAndSources(req.Name, sources...) + } } else { err = s.scheduler.UpdateExpectSubTaskStage(expect, req.Name, sources...) } @@ -1502,7 +1511,7 @@ func (s *Server) removeMetaData(ctx context.Context, taskName, metaSchema string if err != nil { return err } - err = s.optimist.RemoveMetaData(taskName) + err = s.optimist.RemoveMetaDataWithTask(taskName) if err != nil { return err } diff --git a/dm/dm/master/shardddl/optimist.go b/dm/dm/master/shardddl/optimist.go index e3f854fc3df..b49290986bd 100644 --- a/dm/dm/master/shardddl/optimist.go +++ b/dm/dm/master/shardddl/optimist.go @@ -163,7 +163,7 @@ func (o *Optimist) ShowLocks(task string, sources []string) []*pb.DDLLock { // RemoveMetaData removes meta data for a specified task // NOTE: this function can only be used when the specified task is not running. -func (o *Optimist) RemoveMetaData(task string) error { +func (o *Optimist) RemoveMetaDataWithTask(task string) error { o.mu.Lock() defer o.mu.Unlock() if o.closed { @@ -187,10 +187,45 @@ func (o *Optimist) RemoveMetaData(task string) error { o.tk.RemoveTableByTask(task) // clear meta data in etcd - _, err = optimism.DeleteInfosOperationsTablesSchemasByTask(o.cli, task, lockIDSet) + _, err = optimism.DeleteInfosOperationsTablesByTask(o.cli, task, lockIDSet) return err } +// RemoveMetaDataWithTaskAndSources removes meta data for a specified task and sources +// NOTE: this function can only be used when the specified task for source is not running. +func (o *Optimist) RemoveMetaDataWithTaskAndSources(task string, sources ...string) error { + o.mu.Lock() + defer o.mu.Unlock() + if o.closed { + return terror.ErrMasterOptimistNotStarted.Generate() + } + + for _, source := range sources { + infos, _, err := optimism.GetInfosByTaskAndSource(o.cli, task, source) + if err != nil { + return err + } + for _, info := range infos { + lock := o.lk.FindLockByInfo(info) + if lock != nil { + removed := lock.TryRemoveTable(info.Source, info.UpSchema, info.UpTable) + o.logger.Debug("the table name remove from the table keeper", zap.Bool("removed", removed), zap.String("info", info.ShortString())) + removed = o.tk.RemoveTable(info.Task, info.Source, info.UpSchema, info.UpTable, info.DownSchema, info.DownTable) + o.logger.Debug("a table removed for info from the lock", zap.Bool("removed", removed), zap.String("info", info.ShortString())) + } + if !lock.HasTables() { + o.lk.RemoveLock(lock.ID) + } + } + // clear meta data in etcd + _, err = optimism.DeleteInfosOperationsTablesByTaskAndSource(o.cli, task, source) + if err != nil { + return err + } + } + return nil +} + // run runs jobs in the background. func (o *Optimist) run(ctx context.Context, revSource, revInfo, revOperation int64) error { for { diff --git a/dm/pkg/shardddl/optimism/info.go b/dm/pkg/shardddl/optimism/info.go index f5f063ee81e..611c744168e 100644 --- a/dm/pkg/shardddl/optimism/info.go +++ b/dm/pkg/shardddl/optimism/info.go @@ -298,6 +298,24 @@ func deleteInfoOp(info Info) clientv3.Op { info.Task, info.Source, info.UpSchema, info.UpTable)) } +// GetInfosByTaskAndSource gets all shard DDL info and operation by task and source in etcd currently. +func GetInfosByTaskAndSource(cli *clientv3.Client, task string, source string) ([]Info, int64, error) { + respTxn, _, err := etcdutil.DoOpsInOneTxnWithRetry(cli, clientv3.OpGet(common.ShardDDLOptimismInfoKeyAdapter.Encode(task, source), clientv3.WithPrefix())) + if err != nil { + return nil, 0, err + } + infoResp := respTxn.Responses[0].GetResponseRange() + infos := make([]Info, 0, len(infoResp.Kvs)) + for _, kv := range infoResp.Kvs { + info, err2 := infoFromJSON(string(kv.Value)) + if err2 != nil { + return nil, 0, err2 + } + infos = append(infos, info) + } + return infos, respTxn.Header.Revision, nil +} + // ClearTestInfoOperationSchema is used to clear all shard DDL information in optimism mode. // it only used for testing now. func ClearTestInfoOperationSchema(cli *clientv3.Client) error { diff --git a/dm/pkg/shardddl/optimism/lock.go b/dm/pkg/shardddl/optimism/lock.go index b76df97c7f9..91379c0b74b 100644 --- a/dm/pkg/shardddl/optimism/lock.go +++ b/dm/pkg/shardddl/optimism/lock.go @@ -438,6 +438,21 @@ func (l *Lock) TryRemoveTable(source, schema, table string) bool { return true } +// HasTables check whether a lock has tables +func (l *Lock) HasTables() bool { + l.mu.Lock() + defer l.mu.Unlock() + + for _, schemas := range l.tables { + for _, tables := range schemas { + for range tables { + return true + } + } + } + return false +} + // IsSynced returns whether the lock has synced. // In the optimistic mode, we call it `synced` if table info of all tables are the same, // and we define `remain` as the table count which have different table info with the joined one, diff --git a/dm/pkg/shardddl/optimism/ops.go b/dm/pkg/shardddl/optimism/ops.go index b7ede23c2fd..0bc8f94e21b 100644 --- a/dm/pkg/shardddl/optimism/ops.go +++ b/dm/pkg/shardddl/optimism/ops.go @@ -71,7 +71,7 @@ func DeleteInfosOperationsSchemaColumn(cli *clientv3.Client, infos []Info, ops [ } // DeleteInfosOperationsTablesSchemasByTask deletes the shard DDL infos and operations in etcd. -func DeleteInfosOperationsTablesSchemasByTask(cli *clientv3.Client, task string, lockIDSet map[string]struct{}) (int64, error) { +func DeleteInfosOperationsTablesByTask(cli *clientv3.Client, task string, lockIDSet map[string]struct{}) (int64, error) { opsDel := make([]clientv3.Op, 0, 5) opsDel = append(opsDel, clientv3.OpDelete(common.ShardDDLOptimismInfoKeyAdapter.Encode(task), clientv3.WithPrefix())) opsDel = append(opsDel, clientv3.OpDelete(common.ShardDDLOptimismOperationKeyAdapter.Encode(task), clientv3.WithPrefix())) @@ -82,3 +82,13 @@ func DeleteInfosOperationsTablesSchemasByTask(cli *clientv3.Client, task string, _, rev, err := etcdutil.DoOpsInOneTxnWithRetry(cli, opsDel...) return rev, err } + +// DeleteInfosOperationsTablesByTaskAndSource deletes the shard DDL infos and operations in etcd by task and source. +func DeleteInfosOperationsTablesByTaskAndSource(cli *clientv3.Client, task string, source string) (int64, error) { + opsDel := make([]clientv3.Op, 0, 5) + opsDel = append(opsDel, clientv3.OpDelete(common.ShardDDLOptimismInfoKeyAdapter.Encode(task), clientv3.WithPrefix())) + opsDel = append(opsDel, clientv3.OpDelete(common.ShardDDLOptimismOperationKeyAdapter.Encode(task), clientv3.WithPrefix())) + opsDel = append(opsDel, clientv3.OpDelete(common.ShardDDLOptimismSourceTablesKeyAdapter.Encode(task), clientv3.WithPrefix())) + _, rev, err := etcdutil.DoOpsInOneTxnWithRetry(cli, opsDel...) + return rev, err +} diff --git a/dm/syncer/checkpoint.go b/dm/syncer/checkpoint.go index 259fa8d10d2..58aee3942e8 100644 --- a/dm/syncer/checkpoint.go +++ b/dm/syncer/checkpoint.go @@ -722,12 +722,19 @@ func (cp *RemoteCheckPoint) FlushPointsWithTableInfos(tctx *tcontext.Context, ta table := tables[j] ti := tis[j] sourceSchema, sourceTable := table.Schema, table.Name - point := newBinlogPoint(binlog.NewLocation(cp.cfg.Flavor), binlog.NewLocation(cp.cfg.Flavor), nil, nil, cp.cfg.EnableGTID) + + var point *binlogPoint + // if point already in memory, use it if tablePoints, ok := cp.points[sourceSchema]; ok { if p, ok2 := tablePoints[sourceTable]; ok2 { point = p } } + // create new point + if point == nil { + cp.saveTablePoint(table, cp.globalPoint.MySQLLocation(), nil) + point = cp.points[sourceSchema][sourceTable] + } tiBytes, err := json.Marshal(ti) if err != nil { return terror.ErrSchemaTrackerCannotSerialize.Delegate(err, sourceSchema, sourceTable) From 296fca964c62497b5b4c72c74a701b17807673ff Mon Sep 17 00:00:00 2001 From: gmhdbjd Date: Sun, 19 Dec 2021 08:50:53 -0500 Subject: [PATCH 04/31] fix sequence_sharding_optimistic --- dm/tests/sequence_sharding_optimistic/run.sh | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/dm/tests/sequence_sharding_optimistic/run.sh b/dm/tests/sequence_sharding_optimistic/run.sh index 1cba19ec415..1fe2a38333a 100755 --- a/dm/tests/sequence_sharding_optimistic/run.sh +++ b/dm/tests/sequence_sharding_optimistic/run.sh @@ -64,9 +64,9 @@ run() { "query-status $task_name" \ "\"stage\": \"Paused\"" 2 - # try to get schema for the table, but can't get because no DDL/DML replicated yet. + # try to get schema for the table, table exists for optimistic. curl -X PUT ${API_URL} -d '{"op":1, "task":"sequence_sharding_optimistic", "sources": ["mysql-replica-01"], "database":"sharding_seq_opt", "table":"t1"}' >${WORK_DIR}/get_schema.log - check_log_contains ${WORK_DIR}/get_schema.log "Table 'sharding_seq_opt.t1' doesn't exist" 1 + check_log_contains ${WORK_DIR}/get_schema.log 'CREATE TABLE `t1` ( `id` bigint(20) NOT NULL, `c1` varchar(20) DEFAULT NULL, `c2` varchar(20) DEFAULT NULL, PRIMARY KEY (`id`) .*) ENGINE=InnoDB DEFAULT CHARSET=latin1 COLLATE=latin1_bin' 1 # resume task manually. run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" \ From f9a691f17a169b0334ed7848b74d425a56a0d45f Mon Sep 17 00:00:00 2001 From: gmhdbjd Date: Sun, 19 Dec 2021 08:58:40 -0500 Subject: [PATCH 05/31] remove old code --- dm/dm/master/shardddl/optimist_test.go | 51 -------------------------- 1 file changed, 51 deletions(-) diff --git a/dm/dm/master/shardddl/optimist_test.go b/dm/dm/master/shardddl/optimist_test.go index 743af833f74..103e856bb11 100644 --- a/dm/dm/master/shardddl/optimist_test.go +++ b/dm/dm/master/shardddl/optimist_test.go @@ -928,11 +928,6 @@ func (t *testOptimist) TestOptimistInitSchema(c *C) { c.Assert(o.Start(ctx, etcdTestCli), IsNil) c.Assert(o.Locks(), HasLen, 0) - // no init schema exist now. - // is, _, err := optimism.GetInitSchema(etcdTestCli, task, downSchema, downTable) - // c.Assert(err, IsNil) - // c.Assert(is.IsEmpty(), IsTrue) - // PUT i11, will creat a lock. _, err = optimism.PutInfo(etcdTestCli, i11) c.Assert(err, IsNil) @@ -941,11 +936,6 @@ func (t *testOptimist) TestOptimistInitSchema(c *C) { }), IsTrue) time.Sleep(waitTime) // sleep one more time to wait for update of init schema. - // the init schema exist now. - // is, _, err = optimism.GetInitSchema(etcdTestCli, task, downSchema, downTable) - // c.Assert(err, IsNil) - // c.Assert(is.TableInfo, DeepEquals, ti0) // the init schema. - // PUT i12, the lock will be synced. rev1, err := optimism.PutInfo(etcdTestCli, i12) c.Assert(err, IsNil) @@ -980,11 +970,6 @@ func (t *testOptimist) TestOptimistInitSchema(c *C) { return len(o.Locks()) == 0 }), IsTrue) - // the init schema should also be deleted. - // is, _, err = optimism.GetInitSchema(etcdTestCli, task, downSchema, downTable) - // c.Assert(err, IsNil) - // c.Assert(is.IsEmpty(), IsTrue) - // PUT i21 to create the lock again. _, err = optimism.PutInfo(etcdTestCli, i21) c.Assert(err, IsNil) @@ -992,11 +977,6 @@ func (t *testOptimist) TestOptimistInitSchema(c *C) { return len(o.Locks()) == 1 }), IsTrue) time.Sleep(waitTime) // sleep one more time to wait for update of init schema. - - // the init schema exist now. - // is, _, err = optimism.GetInitSchema(etcdTestCli, task, downSchema, downTable) - // c.Assert(err, IsNil) - // c.Assert(is.TableInfo, DeepEquals, ti1) // the init schema is ti1 now. } func (t *testOptimist) testSortInfos(c *C, cli *clientv3.Client) { @@ -1115,19 +1095,6 @@ func (t *testOptimist) TestBuildLockJoinedAndTable(c *C) { stm, _, err := optimism.GetAllSourceTables(etcdTestCli) c.Assert(err, IsNil) o.tk.Init(stm) - - // ifm, _, err := optimism.GetAllInfo(etcdTestCli) - // c.Assert(err, IsNil) - // - // lockJoined, lockTTS, missTable := o.buildLockJoinedAndTTS(ifm, nil) - // c.Assert(len(lockJoined), Equals, 1) - // c.Assert(len(lockTTS), Equals, 1) - // c.Assert(len(missTable), Equals, 0) - // joined, ok := lockJoined[utils.GenDDLLockID(task, downSchema, downTable)] - // c.Assert(ok, IsTrue) - // cmp, err := joined.Compare(schemacmp.Encode(ti2)) - // c.Assert(err, IsNil) - // c.Assert(cmp, Equals, 0) } func (t *testOptimist) TestBuildLockWithInitSchema(c *C) { @@ -1155,7 +1122,6 @@ func (t *testOptimist) TestBuildLockWithInitSchema(c *C) { ddlDropC = "ALTER TABLE bar DROP COLUMN c" infoDropB = optimism.NewInfo(task, source1, "foo", "bar-1", downSchema, downTable, []string{ddlDropC}, ti0, []*model.TableInfo{ti1}) infoDropC = optimism.NewInfo(task, source1, "foo", "bar-1", downSchema, downTable, []string{ddlDropB}, ti1, []*model.TableInfo{ti2}) - // initSchema = optimism.NewInitSchema(task, downSchema, downTable, ti0) ) ctx, cancel := context.WithCancel(context.Background()) @@ -1178,23 +1144,6 @@ func (t *testOptimist) TestBuildLockWithInitSchema(c *C) { stm, _, err := optimism.GetAllSourceTables(etcdTestCli) c.Assert(err, IsNil) o.tk.Init(stm) - - // ifm, _, err := optimism.GetAllInfo(etcdTestCli) - // c.Assert(err, IsNil) - // - // initSchemas := map[string]map[string]map[string]optimism.InitSchema{task: {downSchema: {downTable: initSchema}}} - // lockJoined, lockTTS, missTable := o.buildLockJoinedAndTTS(ifm, initSchemas) - // c.Assert(len(lockJoined), Equals, 1) - // c.Assert(len(lockTTS), Equals, 1) - // c.Assert(len(missTable), Equals, 1) - // cmp, err := missTable[utils.GenDDLLockID(task, downSchema, downTable)][source2]["foo"]["bar-1"].Compare(schemacmp.Encode(initSchema.TableInfo)) - // c.Assert(err, IsNil) - // c.Assert(cmp, Equals, 0) - // joined, ok := lockJoined[utils.GenDDLLockID(task, downSchema, downTable)] - // c.Assert(ok, IsTrue) - // cmp, err = joined.Compare(schemacmp.Encode(ti0)) - // c.Assert(err, IsNil) - // c.Assert(cmp, Equals, 0) } func getDownstreamMeta(string) (*config.DBConfig, string) { From 48b58b729dd900ec3a763fdf82158eb174399f8f Mon Sep 17 00:00:00 2001 From: gmhdbjd Date: Sun, 19 Dec 2021 10:16:06 -0500 Subject: [PATCH 06/31] fix lint --- dm/pkg/shardddl/optimism/keeper.go | 4 ++-- dm/pkg/shardddl/optimism/lock.go | 4 ++-- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/dm/pkg/shardddl/optimism/keeper.go b/dm/pkg/shardddl/optimism/keeper.go index 9278a148e0d..7b5391e8ab3 100644 --- a/dm/pkg/shardddl/optimism/keeper.go +++ b/dm/pkg/shardddl/optimism/keeper.go @@ -52,7 +52,7 @@ func NewLockKeeper(downDBFunc func(string) (*config.DBConfig, string)) *LockKeep } } -// SetDropColumns set drop columns for lock keeper +// SetDropColumns set drop columns for lock keeper. func (lk *LockKeeper) SetDropColumns(dropColumns map[string]map[string]map[string]map[string]map[string]DropColumnStage) { if dropColumns != nil { lk.dropColumns = dropColumns @@ -239,7 +239,7 @@ func (tk *TableKeeper) AddTable(task, source, upSchema, upTable, downSchema, dow return added } -// SourceTableExist check whether a source table exist +// SourceTableExist check whether a source table exist. func (tk *TableKeeper) SourceTableExist(task, source, upSchema, upTable, downSchema, downTable string) bool { tk.mu.Lock() defer tk.mu.Unlock() diff --git a/dm/pkg/shardddl/optimism/lock.go b/dm/pkg/shardddl/optimism/lock.go index 1a9eae1bf74..15f299844c5 100644 --- a/dm/pkg/shardddl/optimism/lock.go +++ b/dm/pkg/shardddl/optimism/lock.go @@ -108,7 +108,7 @@ func NewLock(cli *clientv3.Client, id, task, downSchema, downTable string, joine return l } -// FetchTableInfos fetch all table infos for a lock +// FetchTableInfos fetch all table infos for a lock. func (l *Lock) FetchTableInfos(task, source, schema, table string) (*model.TableInfo, error) { if l.downstreamMeta == nil { log.L().Warn("nil downstream meta") @@ -438,7 +438,7 @@ func (l *Lock) TryRemoveTable(source, schema, table string) bool { return true } -// HasTables check whether a lock has tables +// HasTables check whether a lock has tables. func (l *Lock) HasTables() bool { l.mu.Lock() defer l.mu.Unlock() From 462cc78175a95dcfca7e18df495566158449ca3f Mon Sep 17 00:00:00 2001 From: gmhdbjd Date: Sun, 19 Dec 2021 22:11:36 -0500 Subject: [PATCH 07/31] fix update ba rule test --- dm/tests/shardddl1/run.sh | 9 ++++++++- 1 file changed, 8 insertions(+), 1 deletion(-) diff --git a/dm/tests/shardddl1/run.sh b/dm/tests/shardddl1/run.sh index 2e32d5638e1..a13a83acd7b 100644 --- a/dm/tests/shardddl1/run.sh +++ b/dm/tests/shardddl1/run.sh @@ -398,10 +398,17 @@ function DM_UpdateBARule_CASE() { sed -i 's/do-dbs: \["shardddl1","shardddl2"\]/do-dbs: \["shardddl1"\]/g' $WORK_DIR/task.yaml echo 'ignore-checking-items: ["schema_of_shard_tables"]' >>$WORK_DIR/task.yaml + # source1: db1.tb1(id,new_col1,new_col3) + # source2: db1.tb1(id) run_dm_ctl $WORK_DIR "127.0.0.1:$MASTER_PORT" \ "start-task $WORK_DIR/task.yaml" \ "\"result\": true" 3 + # no lock exist when task begin + run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "show-ddl-locks" \ + "no DDL lock exists" 1 + run_sql_source1 "insert into ${shardddl1}.${tb1} values(13,13,13);" run_sql_source2 "insert into ${shardddl1}.${tb1} values(14);" run_sql_tidb_with_retry "select count(1) from ${shardddl}.${tb};" "count(1): 14" @@ -410,7 +417,7 @@ function DM_UpdateBARule_CASE() { run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" \ "show-ddl-locks" \ - "\"ID\": \"test-\`shardddl\`.\`tb\`\"" 1 + "no DDL lock exists" 1 run_sql_source1 "alter table ${shardddl1}.${tb1} drop column new_col1" run_sql_source2 "alter table ${shardddl1}.${tb1} add column new_col3 int" From d61d3fe87d2116a77c38177495ddaa6e5fa6d50c Mon Sep 17 00:00:00 2001 From: gmhdbjd Date: Mon, 20 Dec 2021 00:55:25 -0500 Subject: [PATCH 08/31] fix ut --- dm/dm/master/server.go | 9 +++++++-- dm/dm/master/shardddl/optimist.go | 1 + dm/pkg/shardddl/optimism/lock.go | 1 - 3 files changed, 8 insertions(+), 3 deletions(-) diff --git a/dm/dm/master/server.go b/dm/dm/master/server.go index 6fee47eb5b0..a742087625f 100644 --- a/dm/dm/master/server.go +++ b/dm/dm/master/server.go @@ -556,10 +556,15 @@ func (s *Server) OperateTask(ctx context.Context, req *pb.OperateTaskRequest) (* // nolint:nilerr return resp, nil } + + // delete meta data for optimist if len(req.Sources) == 0 { - err = s.optimist.RemoveMetaDataWithTask(req.Name) + err2 = s.optimist.RemoveMetaDataWithTask(req.Name) } else { - err = s.optimist.RemoveMetaDataWithTaskAndSources(req.Name, sources...) + err2 = s.optimist.RemoveMetaDataWithTaskAndSources(req.Name, sources...) + } + if err2 != nil { + log.L().Error("failed to delete metadata for task", zap.String("task name", req.Name), log.ShortError(err2)) } } else { err = s.scheduler.UpdateExpectSubTaskStage(expect, req.Name, sources...) diff --git a/dm/dm/master/shardddl/optimist.go b/dm/dm/master/shardddl/optimist.go index eb798cd0428..7e3976aad94 100644 --- a/dm/dm/master/shardddl/optimist.go +++ b/dm/dm/master/shardddl/optimist.go @@ -341,6 +341,7 @@ func (o *Optimist) recoverLocks( for _, info := range infos { if info.IsDeleted { // TODO: handle drop table + continue } if !o.tk.SourceTableExist(info.Task, info.Source, info.UpSchema, info.UpTable, info.DownSchema, info.DownTable) { continue diff --git a/dm/pkg/shardddl/optimism/lock.go b/dm/pkg/shardddl/optimism/lock.go index 15f299844c5..7a3e199db4d 100644 --- a/dm/pkg/shardddl/optimism/lock.go +++ b/dm/pkg/shardddl/optimism/lock.go @@ -111,7 +111,6 @@ func NewLock(cli *clientv3.Client, id, task, downSchema, downTable string, joine // FetchTableInfos fetch all table infos for a lock. func (l *Lock) FetchTableInfos(task, source, schema, table string) (*model.TableInfo, error) { if l.downstreamMeta == nil { - log.L().Warn("nil downstream meta") return nil, terror.ErrMasterOptimisticDownstreamMetaNotFound.Generate(task) } From fdfe741a48d5936bd42732fcb6de0062a85a5b65 Mon Sep 17 00:00:00 2001 From: gmhdbjd Date: Mon, 20 Dec 2021 02:09:48 -0500 Subject: [PATCH 09/31] code review --- dm/dm/master/shardddl/optimist.go | 13 +++++---- dm/dm/master/shardddl/optimist_test.go | 6 ++--- dm/pkg/shardddl/optimism/info.go | 4 +-- dm/pkg/shardddl/optimism/info_test.go | 2 +- dm/pkg/shardddl/optimism/keeper.go | 37 +++++++++++++++++++------- dm/pkg/shardddl/optimism/lock.go | 2 +- dm/pkg/shardddl/optimism/ops.go | 17 ++++++------ dm/pkg/shardddl/optimism/ops_test.go | 4 +-- dm/syncer/checkpoint.go | 4 +-- dm/syncer/shardddl/optimist.go | 2 +- dm/syncer/shardddl/optimist_test.go | 2 +- 11 files changed, 57 insertions(+), 36 deletions(-) diff --git a/dm/dm/master/shardddl/optimist.go b/dm/dm/master/shardddl/optimist.go index 7e3976aad94..52506990c74 100644 --- a/dm/dm/master/shardddl/optimist.go +++ b/dm/dm/master/shardddl/optimist.go @@ -52,11 +52,11 @@ type Optimist struct { } // NewOptimist creates a new Optimist instance. -func NewOptimist(pLogger *log.Logger, downstreamMetaFunc func(string) (*config.DBConfig, string)) *Optimist { +func NewOptimist(pLogger *log.Logger, getDownstreamMetaFunc func(string) (*config.DBConfig, string)) *Optimist { return &Optimist{ logger: pLogger.WithFields(zap.String("component", "shard DDL optimist")), closed: true, - lk: optimism.NewLockKeeper(downstreamMetaFunc), + lk: optimism.NewLockKeeper(getDownstreamMetaFunc), tk: optimism.NewTableKeeper(), } } @@ -201,22 +201,25 @@ func (o *Optimist) RemoveMetaDataWithTaskAndSources(task string, sources ...stri } for _, source := range sources { + // gets all infos for this task and source. infos, _, err := optimism.GetInfosByTaskAndSource(o.cli, task, source) if err != nil { return err } for _, info := range infos { lock := o.lk.FindLockByInfo(info) + // remove table for related lock if lock != nil { removed := lock.TryRemoveTable(info.Source, info.UpSchema, info.UpTable) o.logger.Debug("the table name remove from the table keeper", zap.Bool("removed", removed), zap.String("info", info.ShortString())) - removed = o.tk.RemoveTable(info.Task, info.Source, info.UpSchema, info.UpTable, info.DownSchema, info.DownTable) - o.logger.Debug("a table removed for info from the lock", zap.Bool("removed", removed), zap.String("info", info.ShortString())) } if !lock.HasTables() { o.lk.RemoveLock(lock.ID) } } + // remove source table in table keeper + removed := o.tk.RemoveTableByTaskAndSource(task, source) + o.logger.Debug("a table removed for info from the lock", zap.Bool("removed", removed), zap.String("task", task), zap.String("source", source)) // clear meta data in etcd _, err = optimism.DeleteInfosOperationsTablesByTaskAndSource(o.cli, task, source) if err != nil { @@ -700,7 +703,7 @@ func (o *Optimist) deleteInfosOps(lock *optimism.Lock) (bool, error) { } } // NOTE: we rely on only `task`, `downSchema`, and `downTable` used for deletion. - rev, deleted, err := optimism.DeleteInfosOperationsSchemaColumn(o.cli, infos, ops, lock.Task, lock.DownSchema, lock.DownTable) + rev, deleted, err := optimism.DeleteInfosOperationsColumns(o.cli, infos, ops, lock.ID) if err != nil { return deleted, err } diff --git a/dm/dm/master/shardddl/optimist_test.go b/dm/dm/master/shardddl/optimist_test.go index 103e856bb11..14296f4c998 100644 --- a/dm/dm/master/shardddl/optimist_test.go +++ b/dm/dm/master/shardddl/optimist_test.go @@ -42,7 +42,7 @@ var _ = SerialSuites(&testOptimist{}) // clear keys in etcd test cluster. func clearOptimistTestSourceInfoOperation(c *C) { - c.Assert(optimism.ClearTestInfoOperationSchema(etcdTestCli), IsNil) + c.Assert(optimism.ClearTestInfoOperationColumn(etcdTestCli), IsNil) } func createTableInfo(c *C, p *parser.Parser, se sessionctx.Context, tableID int64, sql string) *model.TableInfo { @@ -196,7 +196,7 @@ func (t *testOptimist) TestOptimist(c *C) { func (t *testOptimist) testOptimist(c *C, cli *clientv3.Client, restart int) { defer func() { - c.Assert(optimism.ClearTestInfoOperationSchema(cli), IsNil) + c.Assert(optimism.ClearTestInfoOperationColumn(cli), IsNil) }() var ( @@ -981,7 +981,7 @@ func (t *testOptimist) TestOptimistInitSchema(c *C) { func (t *testOptimist) testSortInfos(c *C, cli *clientv3.Client) { defer func() { - c.Assert(optimism.ClearTestInfoOperationSchema(cli), IsNil) + c.Assert(optimism.ClearTestInfoOperationColumn(cli), IsNil) }() var ( diff --git a/dm/pkg/shardddl/optimism/info.go b/dm/pkg/shardddl/optimism/info.go index b0655c4c637..7d7cb4675de 100644 --- a/dm/pkg/shardddl/optimism/info.go +++ b/dm/pkg/shardddl/optimism/info.go @@ -316,9 +316,9 @@ func GetInfosByTaskAndSource(cli *clientv3.Client, task string, source string) ( return infos, respTxn.Header.Revision, nil } -// ClearTestInfoOperationSchema is used to clear all shard DDL information in optimism mode. +// ClearTestInfoOperationColumns is used to clear all shard DDL information in optimism mode. // it only used for testing now. -func ClearTestInfoOperationSchema(cli *clientv3.Client) error { +func ClearTestInfoOperationColumn(cli *clientv3.Client) error { clearSource := clientv3.OpDelete(common.ShardDDLOptimismSourceTablesKeyAdapter.Path(), clientv3.WithPrefix()) clearInfo := clientv3.OpDelete(common.ShardDDLOptimismInfoKeyAdapter.Path(), clientv3.WithPrefix()) clearOp := clientv3.OpDelete(common.ShardDDLOptimismOperationKeyAdapter.Path(), clientv3.WithPrefix()) diff --git a/dm/pkg/shardddl/optimism/info_test.go b/dm/pkg/shardddl/optimism/info_test.go index 63df2a422a1..44a35c1c41d 100644 --- a/dm/pkg/shardddl/optimism/info_test.go +++ b/dm/pkg/shardddl/optimism/info_test.go @@ -47,7 +47,7 @@ func TestInfo(t *testing.T) { // clear keys in etcd test cluster. func clearTestInfoOperation(c *C) { - c.Assert(ClearTestInfoOperationSchema(etcdTestCli), IsNil) + c.Assert(ClearTestInfoOperationColumn(etcdTestCli), IsNil) } func createTableInfo(c *C, p *parser.Parser, se sessionctx.Context, tableID int64, sql string) *model.TableInfo { diff --git a/dm/pkg/shardddl/optimism/keeper.go b/dm/pkg/shardddl/optimism/keeper.go index 7b5391e8ab3..3af452029ea 100644 --- a/dm/pkg/shardddl/optimism/keeper.go +++ b/dm/pkg/shardddl/optimism/keeper.go @@ -27,6 +27,7 @@ import ( "github.com/pingcap/tiflow/dm/pkg/utils" ) +// DownstreamMeta used to fetch table info from downstream. type DownstreamMeta struct { db *conn.BaseDB meta string @@ -34,21 +35,22 @@ type DownstreamMeta struct { // LockKeeper used to keep and handle DDL lock conveniently. // The lock information do not need to be persistent, and can be re-constructed from the shard DDL info. +// But the drop columns should be persistent. type LockKeeper struct { mu sync.RWMutex locks map[string]*Lock // lockID -> Lock - downstreamMetaMap map[string]*DownstreamMeta - downDBFunc func(string) (*config.DBConfig, string) - dropColumns map[string]map[string]map[string]map[string]map[string]DropColumnStage + downstreamMetaMap map[string]*DownstreamMeta + getDownstreamMetaFunc func(string) (*config.DBConfig, string) + dropColumns map[string]map[string]map[string]map[string]map[string]DropColumnStage } // NewLockKeeper creates a new LockKeeper instance. -func NewLockKeeper(downDBFunc func(string) (*config.DBConfig, string)) *LockKeeper { +func NewLockKeeper(getDownstreamMetaFunc func(string) (*config.DBConfig, string)) *LockKeeper { return &LockKeeper{ - locks: make(map[string]*Lock), - downstreamMetaMap: make(map[string]*DownstreamMeta), - downDBFunc: downDBFunc, + locks: make(map[string]*Lock), + downstreamMetaMap: make(map[string]*DownstreamMeta), + getDownstreamMetaFunc: getDownstreamMetaFunc, } } @@ -59,12 +61,13 @@ func (lk *LockKeeper) SetDropColumns(dropColumns map[string]map[string]map[strin } } +// getDownstreamMeta gets and cached downstream meta. func (lk *LockKeeper) getDownstreamMeta(task string) (*DownstreamMeta, error) { if downstreamMeta, ok := lk.downstreamMetaMap[task]; ok { return downstreamMeta, nil } - dbConfig, meta := lk.downDBFunc(task) + dbConfig, meta := lk.getDownstreamMetaFunc(task) if dbConfig == nil { return nil, terror.ErrMasterOptimisticDownstreamMetaNotFound.Generate(task) } @@ -105,8 +108,6 @@ func (lk *LockKeeper) TrySync(cli *clientv3.Client, info Info, tts []TargetTable l.columns = cols delete(lk.dropColumns, lockID) } - } else { - log.L().Error("lock already exist") } newDDLs, cols, err := l.TrySync(info, tts) @@ -294,6 +295,22 @@ func (tk *TableKeeper) RemoveTableByTask(task string) bool { return true } +// RemoveTableByTaskAndSource removes tables from the source tables through task name and source. +// it returns whether removed (exit before). +func (tk *TableKeeper) RemoveTableByTaskAndSource(task, source string) bool { + tk.mu.Lock() + defer tk.mu.Unlock() + + if _, ok := tk.tables[task]; !ok { + return false + } + if _, ok := tk.tables[task][source]; !ok { + return false + } + delete(tk.tables[task], source) + return true +} + // FindTables finds source tables by task name and downstream table name. func (tk *TableKeeper) FindTables(task, downSchema, downTable string) []TargetTable { tk.mu.RLock() diff --git a/dm/pkg/shardddl/optimism/lock.go b/dm/pkg/shardddl/optimism/lock.go index 7a3e199db4d..a505661f338 100644 --- a/dm/pkg/shardddl/optimism/lock.go +++ b/dm/pkg/shardddl/optimism/lock.go @@ -86,7 +86,6 @@ type Lock struct { } // NewLock creates a new Lock instance. -// NOTE: we MUST give the initial table info when creating the lock now. func NewLock(cli *clientv3.Client, id, task, downSchema, downTable string, joined schemacmp.Table, tts []TargetTable, downstreamMeta *DownstreamMeta) *Lock { l := &Lock{ cli: cli, @@ -587,6 +586,7 @@ func (l *Lock) tryRevertDone(source, schema, table string) { } // addTables adds any not-existing tables into the lock. +// For a new table, try to fetch table info from downstream. func (l *Lock) addTables(tts []TargetTable) { for _, tt := range tts { if _, ok := l.tables[tt.Source]; !ok { diff --git a/dm/pkg/shardddl/optimism/ops.go b/dm/pkg/shardddl/optimism/ops.go index ebfe8c11b83..266c139dfe8 100644 --- a/dm/pkg/shardddl/optimism/ops.go +++ b/dm/pkg/shardddl/optimism/ops.go @@ -18,7 +18,6 @@ import ( "github.com/pingcap/tiflow/dm/dm/common" "github.com/pingcap/tiflow/dm/pkg/etcdutil" - "github.com/pingcap/tiflow/dm/pkg/utils" ) // PutSourceTablesInfo puts source tables and a shard DDL info. @@ -48,10 +47,10 @@ func PutSourceTablesDeleteInfo(cli *clientv3.Client, st SourceTables, info Info) return rev, err } -// DeleteInfosOperationsSchemaColumn deletes the shard DDL infos, operations, init schemas and dropped columns in etcd. +// DeleteInfosOperationsColumns deletes the shard DDL infos, operations, and dropped columns in etcd. // This function should often be called by DM-master when removing the lock. // Only delete when all info's version are greater or equal to etcd's version, otherwise it means new info was putted into etcd before. -func DeleteInfosOperationsSchemaColumn(cli *clientv3.Client, infos []Info, ops []Operation, task, downSchema, downTable string) (int64, bool, error) { +func DeleteInfosOperationsColumns(cli *clientv3.Client, infos []Info, ops []Operation, lockID string) (int64, bool, error) { opsDel := make([]clientv3.Op, 0, len(infos)+len(ops)) cmps := make([]clientv3.Cmp, 0, len(infos)) for _, info := range infos { @@ -62,7 +61,7 @@ func DeleteInfosOperationsSchemaColumn(cli *clientv3.Client, infos []Info, ops [ for _, op := range ops { opsDel = append(opsDel, deleteOperationOp(op)) } - opsDel = append(opsDel, deleteDroppedColumnsByLockOp(utils.GenDDLLockID(task, downSchema, downTable))) + opsDel = append(opsDel, deleteDroppedColumnsByLockOp(lockID)) resp, rev, err := etcdutil.DoOpsInOneCmpsTxnWithRetry(cli, cmps, opsDel, []clientv3.Op{}) if err != nil { return 0, false, err @@ -70,7 +69,8 @@ func DeleteInfosOperationsSchemaColumn(cli *clientv3.Client, infos []Info, ops [ return rev, resp.Succeeded, nil } -// DeleteInfosOperationsTablesSchemasByTask deletes the shard DDL infos and operations in etcd. +// DeleteInfosOperationsTablesByTask deletes the shard DDL infos and operations in etcd. +// This function should often be called by DM-master when stop a task for all sources. func DeleteInfosOperationsTablesByTask(cli *clientv3.Client, task string, lockIDSet map[string]struct{}) (int64, error) { opsDel := make([]clientv3.Op, 0, 5) opsDel = append(opsDel, clientv3.OpDelete(common.ShardDDLOptimismInfoKeyAdapter.Encode(task), clientv3.WithPrefix())) @@ -84,11 +84,12 @@ func DeleteInfosOperationsTablesByTask(cli *clientv3.Client, task string, lockID } // DeleteInfosOperationsTablesByTaskAndSource deletes the shard DDL infos and operations in etcd by task and source. +// This function should often be called by DM-master when stop a task for a certain source. func DeleteInfosOperationsTablesByTaskAndSource(cli *clientv3.Client, task string, source string) (int64, error) { opsDel := make([]clientv3.Op, 0, 5) - opsDel = append(opsDel, clientv3.OpDelete(common.ShardDDLOptimismInfoKeyAdapter.Encode(task), clientv3.WithPrefix())) - opsDel = append(opsDel, clientv3.OpDelete(common.ShardDDLOptimismOperationKeyAdapter.Encode(task), clientv3.WithPrefix())) - opsDel = append(opsDel, clientv3.OpDelete(common.ShardDDLOptimismSourceTablesKeyAdapter.Encode(task), clientv3.WithPrefix())) + opsDel = append(opsDel, clientv3.OpDelete(common.ShardDDLOptimismInfoKeyAdapter.Encode(task, source), clientv3.WithPrefix())) + opsDel = append(opsDel, clientv3.OpDelete(common.ShardDDLOptimismOperationKeyAdapter.Encode(task, source), clientv3.WithPrefix())) + opsDel = append(opsDel, clientv3.OpDelete(common.ShardDDLOptimismSourceTablesKeyAdapter.Encode(task, source), clientv3.WithPrefix())) _, rev, err := etcdutil.DoOpsInOneTxnWithRetry(cli, opsDel...) return rev, err } diff --git a/dm/pkg/shardddl/optimism/ops_test.go b/dm/pkg/shardddl/optimism/ops_test.go index 1353a363673..da140184268 100644 --- a/dm/pkg/shardddl/optimism/ops_test.go +++ b/dm/pkg/shardddl/optimism/ops_test.go @@ -52,7 +52,7 @@ func (t *testForEtcd) TestDeleteInfosOperationsSchema(c *C) { c.Assert(opm[task][source][upSchema][upTable], DeepEquals, op) // DELETE info and operation with version 0 - _, deleted, err := DeleteInfosOperationsSchemaColumn(etcdTestCli, []Info{info}, []Operation{op}, task, downSchema, downTable) + _, deleted, err := DeleteInfosOperationsColumns(etcdTestCli, []Info{info}, []Operation{op}, genDDLLockID(info)) c.Assert(err, IsNil) c.Assert(deleted, IsFalse) @@ -65,7 +65,7 @@ func (t *testForEtcd) TestDeleteInfosOperationsSchema(c *C) { c.Assert(opm, HasLen, 1) // DELETE info and operation with version 1 - _, deleted, err = DeleteInfosOperationsSchemaColumn(etcdTestCli, []Info{infoWithVer}, []Operation{op}, task, downSchema, downTable) + _, deleted, err = DeleteInfosOperationsColumns(etcdTestCli, []Info{infoWithVer}, []Operation{op}, genDDLLockID(infoWithVer)) c.Assert(err, IsNil) c.Assert(deleted, IsTrue) diff --git a/dm/syncer/checkpoint.go b/dm/syncer/checkpoint.go index 1abcb987c83..f0e3136bb7d 100644 --- a/dm/syncer/checkpoint.go +++ b/dm/syncer/checkpoint.go @@ -245,8 +245,8 @@ type CheckPoint interface { // corresponding to Meta.Flush FlushPointsExcept(tctx *tcontext.Context, snapshotID int, exceptTables []*filter.Table, extraSQLs []string, extraArgs [][]interface{}) error - // FlushPointsWithTableInfos flushed the table point with given table info - FlushPointsWithTableInfos(tctx *tcontext.Context, table []*filter.Table, ti []*model.TableInfo) error + // FlushPointsWithTableInfos flushed the table points with given table infos + FlushPointsWithTableInfos(tctx *tcontext.Context, tables []*filter.Table, tis []*model.TableInfo) error // FlushSafeModeExitPoint flushed the global checkpoint's with given table info FlushSafeModeExitPoint(tctx *tcontext.Context) error diff --git a/dm/syncer/shardddl/optimist.go b/dm/syncer/shardddl/optimist.go index 606f2867cc5..4b076e96008 100644 --- a/dm/syncer/shardddl/optimist.go +++ b/dm/syncer/shardddl/optimist.go @@ -72,7 +72,7 @@ func (o *Optimist) Init(sourceTables map[string]map[string]map[string]map[string } // Tables clone and return tables -// first one is sourceTable, second one is targetTable +// first one is sourceTable, second one is targetTable. func (o *Optimist) Tables() [][]filter.Table { o.mu.Lock() defer o.mu.Unlock() diff --git a/dm/syncer/shardddl/optimist_test.go b/dm/syncer/shardddl/optimist_test.go index 1ac4926656b..40dbeb67ae2 100644 --- a/dm/syncer/shardddl/optimist_test.go +++ b/dm/syncer/shardddl/optimist_test.go @@ -36,7 +36,7 @@ var _ = Suite(&testOptimist{}) // clear keys in etcd test cluster. func clearOptimistTestSourceInfoOperation(c *C) { - c.Assert(optimism.ClearTestInfoOperationSchema(etcdTestCli), IsNil) + c.Assert(optimism.ClearTestInfoOperationColumn(etcdTestCli), IsNil) } func createTableInfo(c *C, p *parser.Parser, se sessionctx.Context, tableID int64, sql string) *model.TableInfo { From 6327cbc68f5ce04f4fac2e600ada3d63687f450f Mon Sep 17 00:00:00 2001 From: gmhdbjd Date: Mon, 20 Dec 2021 02:33:37 -0500 Subject: [PATCH 10/31] fix lint --- dm/syncer/syncer.go | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/dm/syncer/syncer.go b/dm/syncer/syncer.go index ba58f454d74..6518acdd3f6 100644 --- a/dm/syncer/syncer.go +++ b/dm/syncer/syncer.go @@ -1461,14 +1461,14 @@ func (s *Syncer) Run(ctx context.Context) (err error) { for _, tbl := range tbls { sourceTable := tbl[0] targetTable := tbl[1] - tableInfo, err := s.getTableInfo(tctx, &sourceTable, &targetTable) - if err != nil { - return err + tableInfo, err2 := s.getTableInfo(tctx, &sourceTable, &targetTable) + if err2 != nil { + return err2 } sourceTables = append(sourceTables, &sourceTable) tableInfos = append(tableInfos, tableInfo) } - err := s.checkpoint.FlushPointsWithTableInfos(tctx, sourceTables, tableInfos) + err = s.checkpoint.FlushPointsWithTableInfos(tctx, sourceTables, tableInfos) if err != nil { tctx.L().Error("failed to flush table points with table infos", log.ShortError(err)) } From b2e89e3465daa7e595788df92f9e67d04e263b41 Mon Sep 17 00:00:00 2001 From: gmhdbjd Date: Mon, 20 Dec 2021 04:18:56 -0500 Subject: [PATCH 11/31] add DIFFERENT_SCHEMA_FULL_CASE --- dm/pkg/shardddl/optimism/lock.go | 5 +- .../shardddl_optimistic/conf/diff_config.toml | 44 ++++++++++ .../shardddl_optimistic/conf/dm-master.toml | 7 ++ .../shardddl_optimistic/conf/dm-worker1.toml | 2 + .../shardddl_optimistic/conf/dm-worker2.toml | 2 + .../conf/double-source-optimistic.yaml | 58 ++++++++++++++ .../shardddl_optimistic/conf/source1.yaml | 11 +++ .../shardddl_optimistic/conf/source2.yaml | 11 +++ dm/tests/shardddl_optimistic/run.sh | 80 +++++++++++++++++++ 9 files changed, 218 insertions(+), 2 deletions(-) create mode 100644 dm/tests/shardddl_optimistic/conf/diff_config.toml create mode 100644 dm/tests/shardddl_optimistic/conf/dm-master.toml create mode 100644 dm/tests/shardddl_optimistic/conf/dm-worker1.toml create mode 100644 dm/tests/shardddl_optimistic/conf/dm-worker2.toml create mode 100644 dm/tests/shardddl_optimistic/conf/double-source-optimistic.yaml create mode 100644 dm/tests/shardddl_optimistic/conf/source1.yaml create mode 100644 dm/tests/shardddl_optimistic/conf/source2.yaml create mode 100644 dm/tests/shardddl_optimistic/run.sh diff --git a/dm/pkg/shardddl/optimism/lock.go b/dm/pkg/shardddl/optimism/lock.go index a505661f338..000c7c34c2e 100644 --- a/dm/pkg/shardddl/optimism/lock.go +++ b/dm/pkg/shardddl/optimism/lock.go @@ -117,8 +117,8 @@ func (l *Lock) FetchTableInfos(task, source, schema, table string) (*model.Table ctx, cancel := context.WithTimeout(context.Background(), dbutil.DefaultTimeout) defer cancel() - query := `SELECT table_info FROM ` + dbutil.TableName(l.downstreamMeta.meta, cputil.SyncerCheckpoint(task)) + ` WHERE id = ? AND cp_schema = ?` - row := db.DB.QueryRowContext(ctx, query, source, schema) + query := `SELECT table_info FROM ` + dbutil.TableName(l.downstreamMeta.meta, cputil.SyncerCheckpoint(task)) + ` WHERE id = ? AND cp_schema = ? AND cp_table = ?` + row := db.DB.QueryRowContext(ctx, query, source, schema, table) if row.Err() != nil { return nil, row.Err() } @@ -608,6 +608,7 @@ func (l *Lock) addTables(tts []TargetTable) { l.tables[tt.Source][schema][table] = l.joined } else { t := schemacmp.Encode(ti) + log.L().Debug("get source table info", zap.String("task", tt.Task), zap.String("source", tt.Source), zap.String("schema", schema), zap.String("table", table), zap.Stringer("info", t)) l.tables[tt.Source][schema][table] = t } l.done[tt.Source][schema][table] = false diff --git a/dm/tests/shardddl_optimistic/conf/diff_config.toml b/dm/tests/shardddl_optimistic/conf/diff_config.toml new file mode 100644 index 00000000000..cf3abc02891 --- /dev/null +++ b/dm/tests/shardddl_optimistic/conf/diff_config.toml @@ -0,0 +1,44 @@ +# diff Configuration. + +check-thread-count = 4 + +export-fix-sql = true + +check-struct-only = false + +[task] + output-dir = "/tmp/ticdc_dm_test/output" + + source-instances = ["mysql1", "mysql2"] + + target-instance = "tidb0" + + target-check-tables = ["shardddl.tb"] + +[routes.rule1] +schema-pattern = "shardddl[1-2]" +table-pattern = "tb*" +target-schema = "shardddl" +target-table = "tb" + + +[data-sources] +[data-sources.mysql1] +host = "127.0.0.1" +port = 3306 +user = "root" +password = "123456" +route-rules = ["rule1"] + +[data-sources.mysql2] +host = "127.0.0.1" +port = 3307 +user = "root" +password = "123456" +route-rules = ["rule1"] + +[data-sources.tidb0] +host = "127.0.0.1" +port = 4000 +user = "test" +password = "123456" diff --git a/dm/tests/shardddl_optimistic/conf/dm-master.toml b/dm/tests/shardddl_optimistic/conf/dm-master.toml new file mode 100644 index 00000000000..e700eeffa5c --- /dev/null +++ b/dm/tests/shardddl_optimistic/conf/dm-master.toml @@ -0,0 +1,7 @@ +# Master Configuration. +name = "master1" +master-addr = ":8261" +advertise-addr = "127.0.0.1:8261" +peer-urls = "127.0.0.1:8291" +initial-cluster = "master1=http://127.0.0.1:8291" +auto-compaction-retention = "3s" diff --git a/dm/tests/shardddl_optimistic/conf/dm-worker1.toml b/dm/tests/shardddl_optimistic/conf/dm-worker1.toml new file mode 100644 index 00000000000..6f1d1b5344f --- /dev/null +++ b/dm/tests/shardddl_optimistic/conf/dm-worker1.toml @@ -0,0 +1,2 @@ +name = "worker1" +join = "127.0.0.1:8261" \ No newline at end of file diff --git a/dm/tests/shardddl_optimistic/conf/dm-worker2.toml b/dm/tests/shardddl_optimistic/conf/dm-worker2.toml new file mode 100644 index 00000000000..83949162686 --- /dev/null +++ b/dm/tests/shardddl_optimistic/conf/dm-worker2.toml @@ -0,0 +1,2 @@ +name = "worker2" +join = "127.0.0.1:8261" \ No newline at end of file diff --git a/dm/tests/shardddl_optimistic/conf/double-source-optimistic.yaml b/dm/tests/shardddl_optimistic/conf/double-source-optimistic.yaml new file mode 100644 index 00000000000..8e60eab721d --- /dev/null +++ b/dm/tests/shardddl_optimistic/conf/double-source-optimistic.yaml @@ -0,0 +1,58 @@ +--- +name: test +task-mode: all +is-sharding: true +shard-mode: "optimistic" +meta-schema: "dm_meta" +ignore-checking-items: ["all"] + +target-database: + host: "127.0.0.1" + port: 4000 + user: "test" + password: "/Q7B9DizNLLTTfiZHv9WoEAKamfpIUs=" + +mysql-instances: + - source-id: "mysql-replica-01" + block-allow-list: "instance" + route-rules: ["sharding-table-rules","sharding-schema-rules"] + mydumper-config-name: "global" + loader-config-name: "global" + syncer-config-name: "global" + - source-id: "mysql-replica-02" + block-allow-list: "instance" + route-rules: ["sharding-table-rules","sharding-schema-rules"] + mydumper-config-name: "global" + loader-config-name: "global" + syncer-config-name: "global" + +block-allow-list: + instance: + do-dbs: ["shardddl1","shardddl2"] + +routes: + sharding-table-rules: + schema-pattern: "shardddl*" + target-schema: "shardddl" + table-pattern: "tb*" + target-table: "tb" + sharding-schema-rules: + schema-pattern: "shardddl*" + target-schema: "shardddl" + +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/shardddl_optimistic/conf/source1.yaml b/dm/tests/shardddl_optimistic/conf/source1.yaml new file mode 100644 index 00000000000..175e07df7a5 --- /dev/null +++ b/dm/tests/shardddl_optimistic/conf/source1.yaml @@ -0,0 +1,11 @@ +source-id: mysql-replica-01 +flavor: '' +enable-gtid: false +enable-relay: false +from: + host: 127.0.0.1 + user: root + password: /Q7B9DizNLLTTfiZHv9WoEAKamfpIUs= + port: 3306 +checker: + check-enable: false diff --git a/dm/tests/shardddl_optimistic/conf/source2.yaml b/dm/tests/shardddl_optimistic/conf/source2.yaml new file mode 100644 index 00000000000..e6508fd9d4f --- /dev/null +++ b/dm/tests/shardddl_optimistic/conf/source2.yaml @@ -0,0 +1,11 @@ +source-id: mysql-replica-02 +flavor: '' +enable-gtid: true +enable-relay: true +from: + host: 127.0.0.1 + user: root + password: /Q7B9DizNLLTTfiZHv9WoEAKamfpIUs= + port: 3307 +checker: + check-enable: false diff --git a/dm/tests/shardddl_optimistic/run.sh b/dm/tests/shardddl_optimistic/run.sh new file mode 100644 index 00000000000..69d14d2bdaa --- /dev/null +++ b/dm/tests/shardddl_optimistic/run.sh @@ -0,0 +1,80 @@ +#!/bin/bash + +set -eu + +cur=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +source $cur/../_utils/test_prepare +WORK_DIR=$TEST_DIR/$TEST_NAME +source $cur/../_utils/shardddl_lib.sh + +function DM_DIFFERENT_SCHEMA_FULL_CASE() { + run_sql_tidb_with_retry "select count(1) from ${shardddl}.${tb}" "count(1): 4" + run_sql_source1 "insert into ${shardddl1}.${tb1} values(5);" + run_sql_source1 "insert into ${shardddl1}.${tb2} values(6,'6');" + run_sql_source2 "insert into ${shardddl1}.${tb1} values(7,'77');" + run_sql_source2 "insert into ${shardddl1}.${tb2} values(8,'8','88');" + + run_sql_source1 "alter table ${shardddl1}.${tb1} add column c text;" + # source1.tb1(a,c); source1.tb2(a,b); source2.tb1(a,c); source2.tb2(a,b,c) + run_sql_source1 "insert into ${shardddl1}.${tb1} values(9,'999');" + run_sql_source1 "insert into ${shardddl1}.${tb2} values(10,'1010');" + run_sql_source2 "insert into ${shardddl1}.${tb1} values(11,'111111');" + run_sql_source2 "insert into ${shardddl1}.${tb2} values(12,'1212','121212');" + + run_sql_source2 "alter table ${shardddl1}.${tb2} drop column b;" + # source1.tb1(a,c); source1.tb2(a,b); source2.tb1(a,c); source2.tb2(a,c) + run_sql_source1 "insert into ${shardddl1}.${tb1} values(13,'131313');" + run_sql_source1 "insert into ${shardddl1}.${tb2} values(14,'1414');" + run_sql_source2 "insert into ${shardddl1}.${tb1} values(15,'151515');" + run_sql_source2 "insert into ${shardddl1}.${tb2} values(16,'161616');" + + run_sql_source1 "alter table ${shardddl1}.${tb2} drop column b;" + # source1.tb1(a,c); source1.tb2(a); source2.tb1(a,c); source2.tb2(a,c) + run_sql_source1 "insert into ${shardddl1}.${tb1} values(17,'171717');" + run_sql_source1 "insert into ${shardddl1}.${tb2} values(18);" + run_sql_source2 "insert into ${shardddl1}.${tb1} values(19,'191919');" + run_sql_source2 "insert into ${shardddl1}.${tb2} values(20,'202020');" + + run_sql_source1 "alter table ${shardddl1}.${tb2} add column c text;" + # source1.tb1(a,c); source1.tb2(a,c); source2.tb1(a,c); source2.tb2(a,c) + run_sql_source1 "insert into ${shardddl1}.${tb1} values(21,'212121');" + run_sql_source1 "insert into ${shardddl1}.${tb2} values(22,'222222');" + run_sql_source2 "insert into ${shardddl1}.${tb1} values(23,'232323');" + run_sql_source2 "insert into ${shardddl1}.${tb2} values(24,'242424');" + + run_sql_tidb_with_retry "select count(1) from ${shardddl}.${tb}" "count(1): 24" + check_sync_diff $WORK_DIR $cur/conf/diff_config.toml +} + +function DM_DIFFERENT_SCHEMA_FULL() { + # create table with different schema, init data, and create table in downstream manually + run_case DIFFERENT_SCHEMA_FULL "double-source-optimistic" \ + "run_sql_source1 \"create table ${shardddl1}.${tb1} (a int primary key);\"; \ + run_sql_source1 \"create table ${shardddl1}.${tb2} (a int primary key, b varchar(10));\"; \ + run_sql_source2 \"create table ${shardddl1}.${tb1} (a int primary key, c text);\"; \ + run_sql_source2 \"create table ${shardddl1}.${tb2} (a int primary key, b varchar(10), c text);\"; \ + run_sql_source1 \"insert into ${shardddl1}.${tb1} values(1);\"; \ + run_sql_source1 \"insert into ${shardddl1}.${tb2} values(2,'22');\"; \ + run_sql_source2 \"insert into ${shardddl1}.${tb1} values(3,'333');\"; \ + run_sql_source2 \"insert into ${shardddl1}.${tb2} values(4,'44','444');\"; \ + run_sql_tidb \"create database if not exists ${shardddl};\"; \ + run_sql_tidb \"create table ${shardddl}.${tb} (a int primary key, b varchar(10), c text);\"" \ + "clean_table" "optimistic" +} + +function run() { + init_cluster + init_database + + DM_DIFFERENT_SCHEMA_FULL +} + +cleanup_data $shardddl +cleanup_data $shardddl1 +cleanup_data $shardddl2 +# also cleanup dm processes in case of last run failed +cleanup_process $* +run $* +cleanup_process $* + +echo "[$(date)] <<<<<< test case $TEST_NAME success! >>>>>>" From c31815d2abb7f5548f98212c4b17ad46cba5b3ea Mon Sep 17 00:00:00 2001 From: gmhdbjd Date: Mon, 20 Dec 2021 04:52:40 -0500 Subject: [PATCH 12/31] fix lint --- dm/tests/shardddl_optimistic/run.sh | 48 ++++++++++++++--------------- 1 file changed, 24 insertions(+), 24 deletions(-) diff --git a/dm/tests/shardddl_optimistic/run.sh b/dm/tests/shardddl_optimistic/run.sh index 69d14d2bdaa..dcc75019095 100644 --- a/dm/tests/shardddl_optimistic/run.sh +++ b/dm/tests/shardddl_optimistic/run.sh @@ -9,38 +9,38 @@ source $cur/../_utils/shardddl_lib.sh function DM_DIFFERENT_SCHEMA_FULL_CASE() { run_sql_tidb_with_retry "select count(1) from ${shardddl}.${tb}" "count(1): 4" - run_sql_source1 "insert into ${shardddl1}.${tb1} values(5);" - run_sql_source1 "insert into ${shardddl1}.${tb2} values(6,'6');" - run_sql_source2 "insert into ${shardddl1}.${tb1} values(7,'77');" - run_sql_source2 "insert into ${shardddl1}.${tb2} values(8,'8','88');" + run_sql_source1 "insert into ${shardddl1}.${tb1} values(5);" + run_sql_source1 "insert into ${shardddl1}.${tb2} values(6,'6');" + run_sql_source2 "insert into ${shardddl1}.${tb1} values(7,'77');" + run_sql_source2 "insert into ${shardddl1}.${tb2} values(8,'8','88');" - run_sql_source1 "alter table ${shardddl1}.${tb1} add column c text;" + run_sql_source1 "alter table ${shardddl1}.${tb1} add column c text;" # source1.tb1(a,c); source1.tb2(a,b); source2.tb1(a,c); source2.tb2(a,b,c) - run_sql_source1 "insert into ${shardddl1}.${tb1} values(9,'999');" - run_sql_source1 "insert into ${shardddl1}.${tb2} values(10,'1010');" - run_sql_source2 "insert into ${shardddl1}.${tb1} values(11,'111111');" - run_sql_source2 "insert into ${shardddl1}.${tb2} values(12,'1212','121212');" + run_sql_source1 "insert into ${shardddl1}.${tb1} values(9,'999');" + run_sql_source1 "insert into ${shardddl1}.${tb2} values(10,'1010');" + run_sql_source2 "insert into ${shardddl1}.${tb1} values(11,'111111');" + run_sql_source2 "insert into ${shardddl1}.${tb2} values(12,'1212','121212');" - run_sql_source2 "alter table ${shardddl1}.${tb2} drop column b;" + run_sql_source2 "alter table ${shardddl1}.${tb2} drop column b;" # source1.tb1(a,c); source1.tb2(a,b); source2.tb1(a,c); source2.tb2(a,c) - run_sql_source1 "insert into ${shardddl1}.${tb1} values(13,'131313');" - run_sql_source1 "insert into ${shardddl1}.${tb2} values(14,'1414');" - run_sql_source2 "insert into ${shardddl1}.${tb1} values(15,'151515');" - run_sql_source2 "insert into ${shardddl1}.${tb2} values(16,'161616');" + run_sql_source1 "insert into ${shardddl1}.${tb1} values(13,'131313');" + run_sql_source1 "insert into ${shardddl1}.${tb2} values(14,'1414');" + run_sql_source2 "insert into ${shardddl1}.${tb1} values(15,'151515');" + run_sql_source2 "insert into ${shardddl1}.${tb2} values(16,'161616');" - run_sql_source1 "alter table ${shardddl1}.${tb2} drop column b;" + run_sql_source1 "alter table ${shardddl1}.${tb2} drop column b;" # source1.tb1(a,c); source1.tb2(a); source2.tb1(a,c); source2.tb2(a,c) - run_sql_source1 "insert into ${shardddl1}.${tb1} values(17,'171717');" - run_sql_source1 "insert into ${shardddl1}.${tb2} values(18);" - run_sql_source2 "insert into ${shardddl1}.${tb1} values(19,'191919');" - run_sql_source2 "insert into ${shardddl1}.${tb2} values(20,'202020');" + run_sql_source1 "insert into ${shardddl1}.${tb1} values(17,'171717');" + run_sql_source1 "insert into ${shardddl1}.${tb2} values(18);" + run_sql_source2 "insert into ${shardddl1}.${tb1} values(19,'191919');" + run_sql_source2 "insert into ${shardddl1}.${tb2} values(20,'202020');" - run_sql_source1 "alter table ${shardddl1}.${tb2} add column c text;" + run_sql_source1 "alter table ${shardddl1}.${tb2} add column c text;" # source1.tb1(a,c); source1.tb2(a,c); source2.tb1(a,c); source2.tb2(a,c) - run_sql_source1 "insert into ${shardddl1}.${tb1} values(21,'212121');" - run_sql_source1 "insert into ${shardddl1}.${tb2} values(22,'222222');" - run_sql_source2 "insert into ${shardddl1}.${tb1} values(23,'232323');" - run_sql_source2 "insert into ${shardddl1}.${tb2} values(24,'242424');" + run_sql_source1 "insert into ${shardddl1}.${tb1} values(21,'212121');" + run_sql_source1 "insert into ${shardddl1}.${tb2} values(22,'222222');" + run_sql_source2 "insert into ${shardddl1}.${tb1} values(23,'232323');" + run_sql_source2 "insert into ${shardddl1}.${tb2} values(24,'242424');" run_sql_tidb_with_retry "select count(1) from ${shardddl}.${tb}" "count(1): 24" check_sync_diff $WORK_DIR $cur/conf/diff_config.toml From 11a6908b3b4ff6d366539cba2a4a769b3481ea4f Mon Sep 17 00:00:00 2001 From: gmhdbjd Date: Mon, 20 Dec 2021 04:59:50 -0500 Subject: [PATCH 13/31] add test --- dm/tests/others_integration_2.txt | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/dm/tests/others_integration_2.txt b/dm/tests/others_integration_2.txt index 32bdb9e2371..824ecff6bc9 100644 --- a/dm/tests/others_integration_2.txt +++ b/dm/tests/others_integration_2.txt @@ -9,4 +9,5 @@ http_proxies openapi duplicate_event tracker_ignored_ddl -extend_column \ No newline at end of file +extend_column +shardddl_optimistic \ No newline at end of file From d6988fdd64183a1a76b374588821430296a3f25f Mon Sep 17 00:00:00 2001 From: gmhdbjd Date: Mon, 20 Dec 2021 06:45:24 -0500 Subject: [PATCH 14/31] add DIFFERENT_SCHEMA_INCREMENTAL_CASE --- dm/tests/_utils/test_prepare | 2 +- dm/tests/downstream_more_column/run.sh | 2 +- .../shardddl_optimistic/conf/dm-master.toml | 3 + .../conf/double-source-optimistic-incr.yaml | 63 +++++++++++ dm/tests/shardddl_optimistic/run.sh | 104 ++++++++++++++++++ 5 files changed, 172 insertions(+), 2 deletions(-) create mode 100644 dm/tests/shardddl_optimistic/conf/double-source-optimistic-incr.yaml diff --git a/dm/tests/_utils/test_prepare b/dm/tests/_utils/test_prepare index b507fdaaa25..690d91e1554 100644 --- a/dm/tests/_utils/test_prepare +++ b/dm/tests/_utils/test_prepare @@ -315,7 +315,7 @@ function init_cluster(){ } function get_master_status() { - arr=$(echo "show master status;" | MYSQL_PWD=123456 mysql -uroot -h127.0.0.1 -P3306 | awk 'NR==2') + arr=$(echo "show master status;" | MYSQL_PWD=123456 mysql -uroot -h$1 -P$2 | awk 'NR==2') echo $arr } diff --git a/dm/tests/downstream_more_column/run.sh b/dm/tests/downstream_more_column/run.sh index eb8eca00da0..edfef1dabe1 100755 --- a/dm/tests/downstream_more_column/run.sh +++ b/dm/tests/downstream_more_column/run.sh @@ -43,7 +43,7 @@ function run() { # start DM task in incremental mode # schemaTracker create table from downstream - master_status=($(get_master_status)) + master_status=($(get_master_status $MYSQL_HOST1 $MYSQL_PORT1)) cp $cur/conf/dm-task-incremental.yaml $WORK_DIR/dm-task-incremental.yaml sed -i "s/binlog-gtid-placeholder/${master_status[2]}/g" $WORK_DIR/dm-task-incremental.yaml run_dm_ctl $WORK_DIR "127.0.0.1:$MASTER_PORT" \ diff --git a/dm/tests/shardddl_optimistic/conf/dm-master.toml b/dm/tests/shardddl_optimistic/conf/dm-master.toml index e700eeffa5c..458b3e124f0 100644 --- a/dm/tests/shardddl_optimistic/conf/dm-master.toml +++ b/dm/tests/shardddl_optimistic/conf/dm-master.toml @@ -5,3 +5,6 @@ advertise-addr = "127.0.0.1:8261" peer-urls = "127.0.0.1:8291" initial-cluster = "master1=http://127.0.0.1:8291" auto-compaction-retention = "3s" + +[experimental] +openapi = true diff --git a/dm/tests/shardddl_optimistic/conf/double-source-optimistic-incr.yaml b/dm/tests/shardddl_optimistic/conf/double-source-optimistic-incr.yaml new file mode 100644 index 00000000000..541c103ddbc --- /dev/null +++ b/dm/tests/shardddl_optimistic/conf/double-source-optimistic-incr.yaml @@ -0,0 +1,63 @@ +--- +name: test +task-mode: incremental +is-sharding: true +shard-mode: "optimistic" +meta-schema: "dm_meta" +ignore-checking-items: ["all"] + +target-database: + host: "127.0.0.1" + port: 4000 + user: "test" + password: "/Q7B9DizNLLTTfiZHv9WoEAKamfpIUs=" + +mysql-instances: + - source-id: "mysql-replica-01" + block-allow-list: "instance" + route-rules: ["sharding-table-rules","sharding-schema-rules"] + mydumper-config-name: "global" + loader-config-name: "global" + syncer-config-name: "global" + meta: + binlog-pos: pos-holder + binlog-name: name-holder + - source-id: "mysql-replica-02" + block-allow-list: "instance" + route-rules: ["sharding-table-rules","sharding-schema-rules"] + mydumper-config-name: "global" + loader-config-name: "global" + syncer-config-name: "global" + meta: + binlog-gtid: gtid-holder + +block-allow-list: + instance: + do-dbs: ["shardddl1","shardddl2"] + +routes: + sharding-table-rules: + schema-pattern: "shardddl*" + target-schema: "shardddl" + table-pattern: "tb*" + target-table: "tb" + sharding-schema-rules: + schema-pattern: "shardddl*" + target-schema: "shardddl" + +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/shardddl_optimistic/run.sh b/dm/tests/shardddl_optimistic/run.sh index dcc75019095..fc5863b16dc 100644 --- a/dm/tests/shardddl_optimistic/run.sh +++ b/dm/tests/shardddl_optimistic/run.sh @@ -62,11 +62,115 @@ function DM_DIFFERENT_SCHEMA_FULL() { "clean_table" "optimistic" } +function DM_DIFFERENT_SCHEMA_INCREMENTAL_CASE() { + run_sql_tidb_with_retry "select count(1) from ${shardddl}.${tb}" "count(1): 4" + + # get checkpoint + source1_status=($(get_master_status $MYSQL_HOST1 $MYSQL_PORT1)) + source2_status=($(get_master_status $MYSQL_HOST2 $MYSQL_PORT2)) + + run_dm_ctl $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "pause-task test" \ + "\"result\": true" 3 + + # save schema + curl -X GET http://127.0.0.1:8261/api/v1/tasks/test/sources/mysql-replica-01/schemas/${shardddl1}/${tb1} | jq -r .schema_create_sql > $WORK_DIR/schema11.sql + curl -X GET http://127.0.0.1:8261/api/v1/tasks/test/sources/mysql-replica-01/schemas/${shardddl1}/${tb2} | jq -r .schema_create_sql > $WORK_DIR/schema12.sql + curl -X GET http://127.0.0.1:8261/api/v1/tasks/test/sources/mysql-replica-02/schemas/${shardddl1}/${tb1} | jq -r .schema_create_sql > $WORK_DIR/schema21.sql + curl -X GET http://127.0.0.1:8261/api/v1/tasks/test/sources/mysql-replica-02/schemas/${shardddl1}/${tb2} | jq -r .schema_create_sql > $WORK_DIR/schema22.sql + + run_dm_ctl $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "stop-task test" \ + "\"result\": true" 3 + + # incremental data + run_sql_source1 "insert into ${shardddl1}.${tb1} values(5);" + run_sql_source1 "insert into ${shardddl1}.${tb2} values(6,'6');" + run_sql_source2 "insert into ${shardddl1}.${tb1} values(7,'77');" + run_sql_source2 "insert into ${shardddl1}.${tb2} values(8,'8','88');" + + run_sql_source1 "alter table ${shardddl1}.${tb1} add column c text;" + # source1.tb1(a,c); source1.tb2(a,b); source2.tb1(a,c); source2.tb2(a,b,c) + run_sql_source1 "insert into ${shardddl1}.${tb1} values(9,'999');" + run_sql_source1 "insert into ${shardddl1}.${tb2} values(10,'1010');" + run_sql_source2 "insert into ${shardddl1}.${tb1} values(11,'111111');" + run_sql_source2 "insert into ${shardddl1}.${tb2} values(12,'1212','121212');" + + run_sql_source2 "alter table ${shardddl1}.${tb2} drop column b;" + # source1.tb1(a,c); source1.tb2(a,b); source2.tb1(a,c); source2.tb2(a,c) + run_sql_source1 "insert into ${shardddl1}.${tb1} values(13,'131313');" + run_sql_source1 "insert into ${shardddl1}.${tb2} values(14,'1414');" + run_sql_source2 "insert into ${shardddl1}.${tb1} values(15,'151515');" + run_sql_source2 "insert into ${shardddl1}.${tb2} values(16,'161616');" + + run_sql_source1 "alter table ${shardddl1}.${tb2} drop column b;" + # source1.tb1(a,c); source1.tb2(a); source2.tb1(a,c); source2.tb2(a,c) + run_sql_source1 "insert into ${shardddl1}.${tb1} values(17,'171717');" + run_sql_source1 "insert into ${shardddl1}.${tb2} values(18);" + run_sql_source2 "insert into ${shardddl1}.${tb1} values(19,'191919');" + run_sql_source2 "insert into ${shardddl1}.${tb2} values(20,'202020');" + + run_sql_source1 "alter table ${shardddl1}.${tb2} add column c text;" + # source1.tb1(a,c); source1.tb2(a,c); source2.tb1(a,c); source2.tb2(a,c) + run_sql_source1 "insert into ${shardddl1}.${tb1} values(21,'212121');" + run_sql_source1 "insert into ${shardddl1}.${tb2} values(22,'222222');" + run_sql_source2 "insert into ${shardddl1}.${tb1} values(23,'232323');" + run_sql_source2 "insert into ${shardddl1}.${tb2} values(24,'242424');" + + # start task with current checkpoint + sed "s/pos-holder/${source1_status[1]}/g" $cur/conf/double-source-optimistic-incr.yaml >$WORK_DIR/task.yaml + sed -i "s/name-holder/${source1_status[0]}/g" $WORK_DIR/task.yaml + sed -i "s/gtid-holder/${source2_status[2]}/g" $WORK_DIR/task.yaml + run_dm_ctl $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "start-task $WORK_DIR/task.yaml --remove-meta" + + run_dm_ctl $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "query-status test" \ + "Column count doesn't match" 2 + + run_dm_ctl $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "operate-schema set -s mysql-replica-01 test -d ${shardddl1} -t ${tb1} $WORK_DIR/schema11.sql" \ + "\"result\": true" 2 + run_dm_ctl $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "operate-schema set -s mysql-replica-01 test -d ${shardddl1} -t ${tb2} $WORK_DIR/schema12.sql" \ + "\"result\": true" 2 + run_dm_ctl $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "operate-schema set -s mysql-replica-02 test -d ${shardddl1} -t ${tb1} $WORK_DIR/schema21.sql" \ + "\"result\": true" 2 + run_dm_ctl $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "operate-schema set -s mysql-replica-02 test -d ${shardddl1} -t ${tb2} $WORK_DIR/schema22.sql" \ + "\"result\": true" 2 + + run_dm_ctl $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "resume-task test" \ + "\"result\": true" 3 + + run_sql_tidb_with_retry "select count(1) from ${shardddl}.${tb}" "count(1): 24" + check_sync_diff $WORK_DIR $cur/conf/diff_config.toml +} + +function DM_DIFFERENT_SCHEMA_INCREMENTAL() { + # create table with different schema, init data, and create table in downstream manually + run_case DIFFERENT_SCHEMA_INCREMENTAL "double-source-optimistic" \ + "run_sql_source1 \"create table ${shardddl1}.${tb1} (a int primary key);\"; \ + run_sql_source1 \"create table ${shardddl1}.${tb2} (a int primary key, b varchar(10));\"; \ + run_sql_source2 \"create table ${shardddl1}.${tb1} (a int primary key, c text);\"; \ + run_sql_source2 \"create table ${shardddl1}.${tb2} (a int primary key, b varchar(10), c text);\"; \ + run_sql_source1 \"insert into ${shardddl1}.${tb1} values(1);\"; \ + run_sql_source1 \"insert into ${shardddl1}.${tb2} values(2,'22');\"; \ + run_sql_source2 \"insert into ${shardddl1}.${tb1} values(3,'333');\"; \ + run_sql_source2 \"insert into ${shardddl1}.${tb2} values(4,'44','444');\"; \ + run_sql_tidb \"create database if not exists ${shardddl};\"; \ + run_sql_tidb \"create table ${shardddl}.${tb} (a int primary key, b varchar(10), c text);\"" \ + "clean_table" "optimistic" +} + function run() { init_cluster init_database DM_DIFFERENT_SCHEMA_FULL + DM_DIFFERENT_SCHEMA_INCREMENTAL } cleanup_data $shardddl From 9d1921b09b2afa47c09fc6859f724983d9454bc5 Mon Sep 17 00:00:00 2001 From: gmhdbjd Date: Wed, 22 Dec 2021 01:05:24 -0500 Subject: [PATCH 15/31] add more integration test --- dm/dm/master/shardddl/optimist.go | 43 ++-- dm/pkg/shardddl/optimism/info.go | 18 -- dm/pkg/shardddl/optimism/keeper.go | 42 ++-- dm/pkg/shardddl/optimism/lock.go | 25 +++ dm/pkg/shardddl/optimism/ops.go | 19 +- dm/tests/_utils/shardddl_lib.sh | 36 ++++ dm/tests/shardddl_optimistic/run.sh | 305 +++++++++++++++++++++++++++- 7 files changed, 420 insertions(+), 68 deletions(-) diff --git a/dm/dm/master/shardddl/optimist.go b/dm/dm/master/shardddl/optimist.go index 52506990c74..a631e41d3e7 100644 --- a/dm/dm/master/shardddl/optimist.go +++ b/dm/dm/master/shardddl/optimist.go @@ -200,33 +200,27 @@ func (o *Optimist) RemoveMetaDataWithTaskAndSources(task string, sources ...stri return terror.ErrMasterOptimistNotStarted.Generate() } - for _, source := range sources { - // gets all infos for this task and source. - infos, _, err := optimism.GetInfosByTaskAndSource(o.cli, task, source) - if err != nil { - return err - } - for _, info := range infos { - lock := o.lk.FindLockByInfo(info) - // remove table for related lock - if lock != nil { - removed := lock.TryRemoveTable(info.Source, info.UpSchema, info.UpTable) - o.logger.Debug("the table name remove from the table keeper", zap.Bool("removed", removed), zap.String("info", info.ShortString())) - } - if !lock.HasTables() { - o.lk.RemoveLock(lock.ID) - } + dropColumns := make(map[string][]string) + + // gets all locks for this task + locks := o.lk.FindLocksByTask(task) + for _, lock := range locks { + // remove table by sources for related lock + if lock != nil { + cols := lock.TryRemoveTableBySources(sources) + dropColumns[lock.ID] = cols + o.logger.Debug("the tables removed from the lock", zap.String("task", task), zap.Strings("sources", sources)) } - // remove source table in table keeper - removed := o.tk.RemoveTableByTaskAndSource(task, source) - o.logger.Debug("a table removed for info from the lock", zap.Bool("removed", removed), zap.String("task", task), zap.String("source", source)) - // clear meta data in etcd - _, err = optimism.DeleteInfosOperationsTablesByTaskAndSource(o.cli, task, source) - if err != nil { - return err + if !lock.HasTables() { + o.lk.RemoveLock(lock.ID) } } - return nil + // remove source table in table keeper + o.tk.RemoveTableByTaskAndSources(task, sources) + o.logger.Debug("the tables removed from the table keeper", zap.String("task", task), zap.Strings("source", sources)) + // clear meta data in etcd + _, err := optimism.DeleteInfosOperationsTablesByTaskAndSource(o.cli, task, sources, dropColumns) + return err } // run runs jobs in the background. @@ -302,6 +296,7 @@ func (o *Optimist) rebuildLocks() (revSource, revInfo, revOperation int64, err e // then these unexpected locks can be handled by the user. o.logger.Error("fail to recover locks", log.ShortError(err)) } + o.lk.SetDropColumns(nil) return revSource, revInfo, revOperation, nil } diff --git a/dm/pkg/shardddl/optimism/info.go b/dm/pkg/shardddl/optimism/info.go index 7d7cb4675de..bdb43c067ce 100644 --- a/dm/pkg/shardddl/optimism/info.go +++ b/dm/pkg/shardddl/optimism/info.go @@ -298,24 +298,6 @@ func deleteInfoOp(info Info) clientv3.Op { info.Task, info.Source, info.UpSchema, info.UpTable)) } -// GetInfosByTaskAndSource gets all shard DDL info and operation by task and source in etcd currently. -func GetInfosByTaskAndSource(cli *clientv3.Client, task string, source string) ([]Info, int64, error) { - respTxn, _, err := etcdutil.DoOpsInOneTxnWithRetry(cli, clientv3.OpGet(common.ShardDDLOptimismInfoKeyAdapter.Encode(task, source), clientv3.WithPrefix())) - if err != nil { - return nil, 0, err - } - infoResp := respTxn.Responses[0].GetResponseRange() - infos := make([]Info, 0, len(infoResp.Kvs)) - for _, kv := range infoResp.Kvs { - info, err2 := infoFromJSON(string(kv.Value)) - if err2 != nil { - return nil, 0, err2 - } - infos = append(infos, info) - } - return infos, respTxn.Header.Revision, nil -} - // ClearTestInfoOperationColumns is used to clear all shard DDL information in optimism mode. // it only used for testing now. func ClearTestInfoOperationColumn(cli *clientv3.Client) error { diff --git a/dm/pkg/shardddl/optimism/keeper.go b/dm/pkg/shardddl/optimism/keeper.go index 3af452029ea..7a8e0d3fd45 100644 --- a/dm/pkg/shardddl/optimism/keeper.go +++ b/dm/pkg/shardddl/optimism/keeper.go @@ -56,9 +56,7 @@ func NewLockKeeper(getDownstreamMetaFunc func(string) (*config.DBConfig, string) // SetDropColumns set drop columns for lock keeper. func (lk *LockKeeper) SetDropColumns(dropColumns map[string]map[string]map[string]map[string]map[string]DropColumnStage) { - if dropColumns != nil { - lk.dropColumns = dropColumns - } + lk.dropColumns = dropColumns } // getDownstreamMeta gets and cached downstream meta. @@ -103,10 +101,11 @@ func (lk *LockKeeper) TrySync(cli *clientv3.Client, info Info, tts []TargetTable lk.locks[lockID] = NewLock(cli, lockID, info.Task, info.DownSchema, info.DownTable, schemacmp.Encode(info.TableInfoBefore), tts, downstreamMeta) l = lk.locks[lockID] - // set drop columns - if cols, ok := lk.dropColumns[lockID]; ok { - l.columns = cols - delete(lk.dropColumns, lockID) + // set drop columns, only when recover locks + if lk.dropColumns != nil { + if cols, ok := lk.dropColumns[lockID]; ok { + l.columns = cols + } } } @@ -138,6 +137,21 @@ func (lk *LockKeeper) FindLock(lockID string) *Lock { return lk.locks[lockID] } +// FindLocksByTask finds locks by task. +func (lk *LockKeeper) FindLocksByTask(task string) []*Lock { + lk.mu.RLock() + defer lk.mu.RUnlock() + + locks := make([]*Lock, 0) + for _, lock := range lk.locks { + if lock.Task == task { + locks = append(locks, lock) + } + } + + return locks +} + // FindLockByInfo finds a lock with a shard DDL info. func (lk *LockKeeper) FindLockByInfo(info Info) *Lock { return lk.FindLock(genDDLLockID(info)) @@ -295,20 +309,18 @@ func (tk *TableKeeper) RemoveTableByTask(task string) bool { return true } -// RemoveTableByTaskAndSource removes tables from the source tables through task name and source. -// it returns whether removed (exit before). -func (tk *TableKeeper) RemoveTableByTaskAndSource(task, source string) bool { +// RemoveTableByTaskAndSource removes tables from the source tables through task name and sources. +func (tk *TableKeeper) RemoveTableByTaskAndSources(task string, sources []string) { tk.mu.Lock() defer tk.mu.Unlock() if _, ok := tk.tables[task]; !ok { - return false + return } - if _, ok := tk.tables[task][source]; !ok { - return false + + for _, source := range sources { + delete(tk.tables[task], source) } - delete(tk.tables[task], source) - return true } // FindTables finds source tables by task name and downstream table name. diff --git a/dm/pkg/shardddl/optimism/lock.go b/dm/pkg/shardddl/optimism/lock.go index 000c7c34c2e..15b76fc20a9 100644 --- a/dm/pkg/shardddl/optimism/lock.go +++ b/dm/pkg/shardddl/optimism/lock.go @@ -436,6 +436,31 @@ func (l *Lock) TryRemoveTable(source, schema, table string) bool { return true } +// TryRemoveTable tries to remove tables in the lock by sources. +// return drop columns for later use. +func (l *Lock) TryRemoveTableBySources(sources []string) []string { + l.mu.Lock() + defer l.mu.Unlock() + + dropColumns := make([]string, 0) + for _, source := range sources { + if _, ok := l.tables[source]; !ok { + continue + } + + delete(l.tables, source) + _, remain := l.syncStatus() + l.synced = remain == 0 + delete(l.done, source) + delete(l.versions, source) + for _, sourceColumns := range l.columns { + delete(sourceColumns, source) + } + log.L().Info("tables removed from the lock", zap.String("lock", l.ID), zap.String("source", source)) + } + return dropColumns +} + // HasTables check whether a lock has tables. func (l *Lock) HasTables() bool { l.mu.Lock() diff --git a/dm/pkg/shardddl/optimism/ops.go b/dm/pkg/shardddl/optimism/ops.go index 266c139dfe8..28c92f3e495 100644 --- a/dm/pkg/shardddl/optimism/ops.go +++ b/dm/pkg/shardddl/optimism/ops.go @@ -84,12 +84,21 @@ func DeleteInfosOperationsTablesByTask(cli *clientv3.Client, task string, lockID } // DeleteInfosOperationsTablesByTaskAndSource deletes the shard DDL infos and operations in etcd by task and source. -// This function should often be called by DM-master when stop a task for a certain source. -func DeleteInfosOperationsTablesByTaskAndSource(cli *clientv3.Client, task string, source string) (int64, error) { +// This function should often be called by DM-master when stop a task for sources. +func DeleteInfosOperationsTablesByTaskAndSource(cli *clientv3.Client, task string, sources []string, dropColumns map[string][]string) (int64, error) { opsDel := make([]clientv3.Op, 0, 5) - opsDel = append(opsDel, clientv3.OpDelete(common.ShardDDLOptimismInfoKeyAdapter.Encode(task, source), clientv3.WithPrefix())) - opsDel = append(opsDel, clientv3.OpDelete(common.ShardDDLOptimismOperationKeyAdapter.Encode(task, source), clientv3.WithPrefix())) - opsDel = append(opsDel, clientv3.OpDelete(common.ShardDDLOptimismSourceTablesKeyAdapter.Encode(task, source), clientv3.WithPrefix())) + for _, source := range sources { + opsDel = append(opsDel, clientv3.OpDelete(common.ShardDDLOptimismInfoKeyAdapter.Encode(task, source), clientv3.WithPrefix())) + opsDel = append(opsDel, clientv3.OpDelete(common.ShardDDLOptimismOperationKeyAdapter.Encode(task, source), clientv3.WithPrefix())) + opsDel = append(opsDel, clientv3.OpDelete(common.ShardDDLOptimismSourceTablesKeyAdapter.Encode(task, source), clientv3.WithPrefix())) + for lockID, cols := range dropColumns { + for _, col := range cols { + for _, source := range sources { + opsDel = append(opsDel, clientv3.OpDelete(common.ShardDDLOptimismDroppedColumnsKeyAdapter.Encode(lockID, col, source), clientv3.WithPrefix())) + } + } + } + } _, rev, err := etcdutil.DoOpsInOneTxnWithRetry(cli, opsDel...) return rev, err } diff --git a/dm/tests/_utils/shardddl_lib.sh b/dm/tests/_utils/shardddl_lib.sh index 3860a971204..b5f1ffc9c30 100644 --- a/dm/tests/_utils/shardddl_lib.sh +++ b/dm/tests/_utils/shardddl_lib.sh @@ -54,3 +54,39 @@ function restart_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 } + +function restart_worker1() { + echo "restart dm-worker1" + ps aux | grep worker1 | awk '{print $2}' | xargs kill || true + check_port_offline $WORKER1_PORT 20 + 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 +} + +function restart_worker2() { + echo "restart dm-worker2" + ps aux | grep worker2 | awk '{print $2}' | xargs kill || true + check_port_offline $WORKER2_PORT 20 + 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 +} + +function restart_task() { + echo "restart task" + + run_dm_ctl $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "stop-task test" + + run_dm_ctl $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "start-task $1" + + if [[ "$task_conf" == *"single"* ]]; then + run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "query-status test" \ + "\"unit\": \"Sync\"" 1 + elif [[ "$task_conf" == *"double"* ]]; then + run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "query-status test" \ + "\"unit\": \"Sync\"" 2 + fi +} diff --git a/dm/tests/shardddl_optimistic/run.sh b/dm/tests/shardddl_optimistic/run.sh index fc5863b16dc..e6e7cb49a04 100644 --- a/dm/tests/shardddl_optimistic/run.sh +++ b/dm/tests/shardddl_optimistic/run.sh @@ -74,10 +74,10 @@ function DM_DIFFERENT_SCHEMA_INCREMENTAL_CASE() { "\"result\": true" 3 # save schema - curl -X GET http://127.0.0.1:8261/api/v1/tasks/test/sources/mysql-replica-01/schemas/${shardddl1}/${tb1} | jq -r .schema_create_sql > $WORK_DIR/schema11.sql - curl -X GET http://127.0.0.1:8261/api/v1/tasks/test/sources/mysql-replica-01/schemas/${shardddl1}/${tb2} | jq -r .schema_create_sql > $WORK_DIR/schema12.sql - curl -X GET http://127.0.0.1:8261/api/v1/tasks/test/sources/mysql-replica-02/schemas/${shardddl1}/${tb1} | jq -r .schema_create_sql > $WORK_DIR/schema21.sql - curl -X GET http://127.0.0.1:8261/api/v1/tasks/test/sources/mysql-replica-02/schemas/${shardddl1}/${tb2} | jq -r .schema_create_sql > $WORK_DIR/schema22.sql + curl -X GET http://127.0.0.1:8261/api/v1/tasks/test/sources/mysql-replica-01/schemas/${shardddl1}/${tb1} | jq -r .schema_create_sql >$WORK_DIR/schema11.sql + curl -X GET http://127.0.0.1:8261/api/v1/tasks/test/sources/mysql-replica-01/schemas/${shardddl1}/${tb2} | jq -r .schema_create_sql >$WORK_DIR/schema12.sql + curl -X GET http://127.0.0.1:8261/api/v1/tasks/test/sources/mysql-replica-02/schemas/${shardddl1}/${tb1} | jq -r .schema_create_sql >$WORK_DIR/schema21.sql + curl -X GET http://127.0.0.1:8261/api/v1/tasks/test/sources/mysql-replica-02/schemas/${shardddl1}/${tb2} | jq -r .schema_create_sql >$WORK_DIR/schema22.sql run_dm_ctl $WORK_DIR "127.0.0.1:$MASTER_PORT" \ "stop-task test" \ @@ -117,7 +117,7 @@ function DM_DIFFERENT_SCHEMA_INCREMENTAL_CASE() { run_sql_source2 "insert into ${shardddl1}.${tb1} values(23,'232323');" run_sql_source2 "insert into ${shardddl1}.${tb2} values(24,'242424');" - # start task with current checkpoint + # start task with current checkpoint sed "s/pos-holder/${source1_status[1]}/g" $cur/conf/double-source-optimistic-incr.yaml >$WORK_DIR/task.yaml sed -i "s/name-holder/${source1_status[0]}/g" $WORK_DIR/task.yaml sed -i "s/gtid-holder/${source2_status[2]}/g" $WORK_DIR/task.yaml @@ -140,7 +140,7 @@ function DM_DIFFERENT_SCHEMA_INCREMENTAL_CASE() { run_dm_ctl $WORK_DIR "127.0.0.1:$MASTER_PORT" \ "operate-schema set -s mysql-replica-02 test -d ${shardddl1} -t ${tb2} $WORK_DIR/schema22.sql" \ "\"result\": true" 2 - + run_dm_ctl $WORK_DIR "127.0.0.1:$MASTER_PORT" \ "resume-task test" \ "\"result\": true" 3 @@ -165,12 +165,305 @@ function DM_DIFFERENT_SCHEMA_INCREMENTAL() { "clean_table" "optimistic" } +function DM_RESTART_TASK_MASTER_WORKER_CASE() { + run_sql_tidb_with_retry "select count(1) from ${shardddl}.${tb}" "count(1): 4" + run_sql_source1 "insert into ${shardddl1}.${tb1} values(5);" + run_sql_source1 "insert into ${shardddl1}.${tb2} values(6,'6');" + run_sql_source2 "insert into ${shardddl1}.${tb1} values(7,'77');" + run_sql_source2 "insert into ${shardddl1}.${tb2} values(8,'8','88');" + + run_sql_source1 "alter table ${shardddl1}.${tb1} add column c text;" + random_restart + + # source1.tb1(a,c); source1.tb2(a,b); source2.tb1(a,c); source2.tb2(a,b,c) + run_sql_source1 "insert into ${shardddl1}.${tb1} values(9,'999');" + run_sql_source1 "insert into ${shardddl1}.${tb2} values(10,'1010');" + run_sql_source2 "insert into ${shardddl1}.${tb1} values(11,'111111');" + run_sql_source2 "insert into ${shardddl1}.${tb2} values(12,'1212','121212');" + + run_sql_source2 "alter table ${shardddl1}.${tb2} drop column b;" + random_restart + + # source1.tb1(a,c); source1.tb2(a,b); source2.tb1(a,c); source2.tb2(a,c) + run_sql_source1 "insert into ${shardddl1}.${tb1} values(13,'131313');" + run_sql_source1 "insert into ${shardddl1}.${tb2} values(14,'1414');" + run_sql_source2 "insert into ${shardddl1}.${tb1} values(15,'151515');" + run_sql_source2 "insert into ${shardddl1}.${tb2} values(16,'161616');" + + run_sql_source1 "alter table ${shardddl1}.${tb2} drop column b;" + random_restart + + # source1.tb1(a,c); source1.tb2(a); source2.tb1(a,c); source2.tb2(a,c) + run_sql_source1 "insert into ${shardddl1}.${tb1} values(17,'171717');" + run_sql_source1 "insert into ${shardddl1}.${tb2} values(18);" + run_sql_source2 "insert into ${shardddl1}.${tb1} values(19,'191919');" + run_sql_source2 "insert into ${shardddl1}.${tb2} values(20,'202020');" + + run_sql_source1 "alter table ${shardddl1}.${tb2} add column c text;" + random_restart + + # source1.tb1(a,c); source1.tb2(a,c); source2.tb1(a,c); source2.tb2(a,c) + run_sql_source1 "insert into ${shardddl1}.${tb1} values(21,'212121');" + run_sql_source1 "insert into ${shardddl1}.${tb2} values(22,'222222');" + run_sql_source2 "insert into ${shardddl1}.${tb1} values(23,'232323');" + run_sql_source2 "insert into ${shardddl1}.${tb2} values(24,'242424');" + + run_sql_tidb_with_retry "select count(1) from ${shardddl}.${tb}" "count(1): 24" + check_sync_diff $WORK_DIR $cur/conf/diff_config.toml +} + +function DM_RESTART_TASK_MASTER_WORKER() { + # create table with different schema, init data, and create table in downstream manually + run_case RESTART_TASK_MASTER_WORKER "double-source-optimistic" \ + "run_sql_source1 \"create table ${shardddl1}.${tb1} (a int primary key);\"; \ + run_sql_source1 \"create table ${shardddl1}.${tb2} (a int primary key, b varchar(10));\"; \ + run_sql_source2 \"create table ${shardddl1}.${tb1} (a int primary key, c text);\"; \ + run_sql_source2 \"create table ${shardddl1}.${tb2} (a int primary key, b varchar(10), c text);\"; \ + run_sql_source1 \"insert into ${shardddl1}.${tb1} values(1);\"; \ + run_sql_source1 \"insert into ${shardddl1}.${tb2} values(2,'22');\"; \ + run_sql_source2 \"insert into ${shardddl1}.${tb1} values(3,'333');\"; \ + run_sql_source2 \"insert into ${shardddl1}.${tb2} values(4,'44','444');\"; \ + run_sql_tidb \"create database if not exists ${shardddl};\"; \ + run_sql_tidb \"create table ${shardddl}.${tb} (a int primary key, b varchar(10), c text);\"" \ + "clean_table" "optimistic" +} + +function random_restart() { + mod=$RANDOM%4 + if [[ "$mod" == "0" ]]; then + restart_master + elif [[ "$mod" == "1" ]]; then + restart_worker1 + elif [[ "$mod" == "2" ]]; then + restart_worker2 + else + restart_task $cur/conf/double-source-optimistic.yaml + fi +} + +function DM_STOP_TASK_FOR_A_SOURCE_CASE() { + run_sql_source1 "insert into ${shardddl1}.${tb1} values(1);" + run_sql_source1 "insert into ${shardddl1}.${tb2} values(2);" + run_sql_source2 "insert into ${shardddl1}.${tb1} values(3);" + run_sql_source2 "insert into ${shardddl1}.${tb2} values(4);" + + run_sql_source1 "alter table ${shardddl1}.${tb1} add column b varchar(10);" + run_sql_source1 "alter table ${shardddl1}.${tb2} add column b varchar(10);" + run_sql_source2 "alter table ${shardddl1}.${tb1} add column b varchar(10);" + run_sql_source2 "alter table ${shardddl1}.${tb2} add column b varchar(10);" + + run_sql_source1 "insert into ${shardddl1}.${tb1} values(5,'aaa');" + run_sql_source1 "insert into ${shardddl1}.${tb2} values(6,'bbb');" + run_sql_source2 "insert into ${shardddl1}.${tb1} values(7,'ccc');" + run_sql_source2 "insert into ${shardddl1}.${tb2} values(8,'ddd');" + + run_sql_source1 "alter table ${shardddl1}.${tb1} add column c text;" + run_sql_source1 "insert into ${shardddl1}.${tb1} values(9,'eee','eee');" + run_sql_source1 "alter table ${shardddl1}.${tb2} drop column b;" + run_sql_source1 "insert into ${shardddl1}.${tb2} values(10);" + run_sql_source2 "alter table ${shardddl1}.${tb1} add column c text;" + run_sql_source2 "insert into ${shardddl1}.${tb1} values(11,'fff','fff');" + run_sql_source2 "alter table ${shardddl1}.${tb2} drop column b;" + run_sql_source2 "insert into ${shardddl1}.${tb2} values(12);" + + run_sql_tidb_with_retry "select count(1) from ${shardddl}.${tb}" "count(1): 12" + + run_dm_ctl $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "stop-task test -s mysql-replica-02" \ + "\"result\": true" 2 + + run_sql_source1 "alter table ${shardddl1}.${tb1} drop column b;" + run_sql_source1 "insert into ${shardddl1}.${tb1} values(13,'ggg');" + run_sql_source1 "alter table ${shardddl1}.${tb2} add column c text;" + run_sql_source1 "insert into ${shardddl1}.${tb2} values(14,'hhh');" + + run_sql_tidb_with_retry "select count(1) from ${shardddl}.${tb}" "count(1): 14" + run_sql_tidb_with_retry "select count(1) from INFORMATION_SCHEMA.COLUMNS where TABLE_SCHEMA='${shardddl}' AND TABLE_NAME='${tb}';" \ + "count(1): 2" + + run_dm_ctl $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "start-task $cur/conf/double-source-optimistic.yaml -s mysql-replica-02" \ + "\"result\": true" 2 + + run_sql_source1 "insert into ${shardddl1}.${tb1} values(15,'iii');" + run_sql_source1 "insert into ${shardddl1}.${tb1} values(16,'jjj');" + run_sql_source2 "alter table ${shardddl1}.${tb1} drop column b;" + run_sql_source2 "insert into ${shardddl1}.${tb1} values(17,'kkk');" + run_sql_source2 "alter table ${shardddl1}.${tb2} add column c text;" + run_sql_source2 "insert into ${shardddl1}.${tb2} values(18,'lll');" + + check_sync_diff $WORK_DIR $cur/conf/diff_config.toml +} + +function DM_STOP_TASK_FOR_A_SOURCE_CASE() { + run_sql_source1 "insert into ${shardddl1}.${tb1} values(1);" + run_sql_source1 "insert into ${shardddl1}.${tb2} values(2);" + run_sql_source2 "insert into ${shardddl1}.${tb1} values(3);" + run_sql_source2 "insert into ${shardddl1}.${tb2} values(4);" + + run_sql_source1 "alter table ${shardddl1}.${tb1} add column b varchar(10);" + run_sql_source1 "alter table ${shardddl1}.${tb2} add column b varchar(10);" + run_sql_source2 "alter table ${shardddl1}.${tb1} add column b varchar(10);" + run_sql_source2 "alter table ${shardddl1}.${tb2} add column b varchar(10);" + + run_sql_source1 "insert into ${shardddl1}.${tb1} values(5,'aaa');" + run_sql_source1 "insert into ${shardddl1}.${tb2} values(6,'bbb');" + run_sql_source2 "insert into ${shardddl1}.${tb1} values(7,'ccc');" + run_sql_source2 "insert into ${shardddl1}.${tb2} values(8,'ddd');" + + run_sql_source1 "alter table ${shardddl1}.${tb1} add column c text;" + run_sql_source1 "insert into ${shardddl1}.${tb1} values(9,'eee','eee');" + run_sql_source1 "alter table ${shardddl1}.${tb2} drop column b;" + run_sql_source1 "insert into ${shardddl1}.${tb2} values(10);" + run_sql_source2 "alter table ${shardddl1}.${tb1} add column c text;" + run_sql_source2 "insert into ${shardddl1}.${tb1} values(11,'fff','fff');" + run_sql_source2 "alter table ${shardddl1}.${tb2} drop column b;" + run_sql_source2 "insert into ${shardddl1}.${tb2} values(12);" + + run_sql_tidb_with_retry "select count(1) from ${shardddl}.${tb}" "count(1): 12" + + run_dm_ctl $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "stop-task test -s mysql-replica-02" \ + "\"result\": true" 2 + + run_sql_source1 "alter table ${shardddl1}.${tb1} drop column b;" + run_sql_source1 "insert into ${shardddl1}.${tb1} values(13,'ggg');" + run_sql_source1 "alter table ${shardddl1}.${tb2} add column c text;" + run_sql_source1 "insert into ${shardddl1}.${tb2} values(14,'hhh');" + + run_sql_tidb_with_retry "select count(1) from ${shardddl}.${tb}" "count(1): 14" + run_sql_tidb_with_retry "select count(1) from INFORMATION_SCHEMA.COLUMNS where TABLE_SCHEMA='${shardddl}' AND TABLE_NAME='${tb}';" \ + "count(1): 2" + + run_dm_ctl $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "start-task $cur/conf/double-source-optimistic.yaml -s mysql-replica-02" \ + "\"result\": true" 2 + + run_sql_source1 "insert into ${shardddl1}.${tb1} values(15,'iii');" + run_sql_source1 "insert into ${shardddl1}.${tb1} values(16,'jjj');" + run_sql_source2 "alter table ${shardddl1}.${tb1} drop column b;" + run_sql_source2 "insert into ${shardddl1}.${tb1} values(17,'kkk');" + run_sql_source2 "alter table ${shardddl1}.${tb2} add column c text;" + run_sql_source2 "insert into ${shardddl1}.${tb2} values(18,'lll');" + + check_sync_diff $WORK_DIR $cur/conf/diff_config.toml +} + +function DM_STOP_TASK_FOR_A_SOURCE() { + run_case STOP_TASK_FOR_A_SOURCE "double-source-optimistic" "init_table 111 112 211 212" "clean_table" "optimistic" +} + +function DM_UPDATE_BA_ROUTE_CASE() { + run_sql_source1 "insert into ${shardddl1}.${tb1} values(1);" + run_sql_source1 "insert into ${shardddl2}.${tb1} values(2);" + run_sql_source2 "insert into ${shardddl1}.${tb1} values(3);" + run_sql_source2 "insert into ${shardddl2}.${tb1} values(4);" + + run_sql_source1 "alter table ${shardddl1}.${tb1} add column new_col1 int" + run_sql_source1 "alter table ${shardddl2}.${tb1} add column new_col1 int" + run_sql_source2 "alter table ${shardddl1}.${tb1} add column new_col1 int" + run_sql_source2 "alter table ${shardddl2}.${tb1} add column new_col1 int" + + run_sql_source1 "insert into ${shardddl1}.${tb1} values(5,5);" + run_sql_source1 "insert into ${shardddl2}.${tb1} values(6,6);" + run_sql_source2 "insert into ${shardddl1}.${tb1} values(7,7);" + run_sql_source2 "insert into ${shardddl2}.${tb1} values(8,8);" + + # source1 db2.tb1 add column and then drop column + run_sql_source1 "alter table ${shardddl2}.${tb1} add column new_col2 int" + run_sql_source1 "insert into ${shardddl2}.${tb1} values(9,9,9);" + run_sql_source1 "alter table ${shardddl2}.${tb1} drop column new_col2" + run_sql_source1 "insert into ${shardddl2}.${tb1} values(10,10);" + + # source1.db1.tb1, source2.db2.tb1 add column + run_sql_source1 "alter table ${shardddl1}.${tb1} add column new_col3 int" + run_sql_source1 "insert into ${shardddl1}.${tb1} values(11,11,11);" + run_sql_source2 "alter table ${shardddl2}.${tb1} add column new_col3 int" + run_sql_source2 "insert into ${shardddl2}.${tb1} values(12,12,12);" + + # source2 db1.tb1 drop column + run_sql_source2 "alter table ${shardddl1}.${tb1} drop column new_col1" + run_sql_source2 "insert into ${shardddl1}.${tb1} values(13);" + + run_sql_tidb_with_retry "select count(1) from ${shardddl}.${tb}" "count(1): 13" + + run_dm_ctl $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "stop-task test" \ + "\"result\": true" 3 + + cp $cur/conf/double-source-optimistic.yaml $WORK_DIR/task.yaml + sed 's/do-dbs: \["shardddl1","shardddl2"\]/do-dbs: \["shardddl1"\]/g' $cur/conf/double-source-optimistic.yaml >$WORK_DIR/task.yaml + + # source1: db1.tb1(id,new_col1,new_col3) + # source2: db1.tb1(id) + run_dm_ctl $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "start-task $WORK_DIR/task.yaml" \ + "\"result\": true" 3 + + # no lock exist when task begin + run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "show-ddl-locks" \ + "no DDL lock exists" 1 + + run_sql_source1 "insert into ${shardddl1}.${tb1} values(14,14,14);" + run_sql_source2 "insert into ${shardddl1}.${tb1} values(15);" + run_sql_tidb_with_retry "select count(1) from ${shardddl}.${tb};" "count(1): 15" + + run_sql_source1 "alter table ${shardddl1}.${tb1} drop column new_col1" + run_sql_source2 "alter table ${shardddl1}.${tb1} add column new_col3 int" + run_sql_source1 "insert into ${shardddl1}.${tb1} values(16,16);" + run_sql_source2 "insert into ${shardddl1}.${tb1} values(17,17);" + run_sql_tidb_with_retry "select count(1) from ${shardddl}.${tb};" "count(1): 17" + + run_dm_ctl $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "stop-task test" \ + "\"result\": true" 3 + run_dm_ctl $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "start-task $cur/conf/double-source-optimistic.yaml" \ + "\"result\": true" 3 + + # source1: db1.tb1(id,new_col3), db2.tb1(id,new_col1) + # source2: db1.tb1(id,new_col3), db2.tb1(id,new_col1,new_col3) + run_sql_source1 "insert into ${shardddl1}.${tb1} values(18,18);" + run_sql_source1 "insert into ${shardddl2}.${tb1} values(19,19);" + run_sql_source2 "insert into ${shardddl1}.${tb1} values(20,20);" + run_sql_source2 "insert into ${shardddl2}.${tb1} values(21,21,21);" + + run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "query-status test" \ + "Unknown column 'new_col1' in 'field list'" 2 + + run_sql_tidb "alter table ${shardddl}.${tb} add column new_col1 int" + + run_dm_ctl $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "resume-task test" \ + "\"result\": true" 3 + + run_sql_source1 "alter table ${shardddl2}.${tb1} drop column new_col1" + run_sql_source2 "alter table ${shardddl2}.${tb1} drop column new_col1" + run_sql_source1 "alter table ${shardddl2}.${tb1} add column new_col3 int" + + run_sql_source1 "insert into ${shardddl1}.${tb1} values(22,22);" + run_sql_source1 "insert into ${shardddl2}.${tb1} values(23,23);" + run_sql_source2 "insert into ${shardddl1}.${tb1} values(24,24);" + run_sql_source2 "insert into ${shardddl2}.${tb1} values(25,25);" + + check_sync_diff $WORK_DIR $cur/conf/diff_config.toml +} + +function DM_UPDATE_BA_ROUTE() { + run_case UPDATE_BA_ROUTE "double-source-optimistic" "init_table 111 121 211 221" "clean_table" "optimistic" +} + function run() { init_cluster init_database DM_DIFFERENT_SCHEMA_FULL DM_DIFFERENT_SCHEMA_INCREMENTAL + DM_RESTART_TASK_MASTER_WORKER + DM_STOP_TASK_FOR_A_SOURCE + DM_UPDATE_BA_ROUTE } cleanup_data $shardddl From 0af4f1882bddce5a36d0d393a9971b5e21de8963 Mon Sep 17 00:00:00 2001 From: gmhdbjd Date: Wed, 22 Dec 2021 07:01:29 -0500 Subject: [PATCH 16/31] add unit test --- Makefile | 13 +- dm/dm/master/scheduler/scheduler_test.go | 16 +++ dm/pkg/shardddl/optimism/keeper_test.go | 52 ++++++++ dm/pkg/shardddl/optimism/lock.go | 12 +- dm/pkg/shardddl/optimism/lock_test.go | 160 +++++++++++++++++++++++ dm/syncer/shardddl/optimist_test.go | 6 + 6 files changed, 255 insertions(+), 4 deletions(-) diff --git a/Makefile b/Makefile index f28c7b995da..7753066bb9b 100644 --- a/Makefile +++ b/Makefile @@ -275,13 +275,22 @@ dm_generate_openapi: tools/bin/oapi-codegen cd dm && ../tools/bin/oapi-codegen --config=openapi/spec/types-gen-cfg.yaml openapi/spec/dm.yaml cd dm && ../tools/bin/oapi-codegen --config=openapi/spec/client-gen-cfg.yaml openapi/spec/dm.yaml -dm_unit_test: check_failpoint_ctl +define run_dm_unit_test + @echo "running unit test for packages:" $(1) mkdir -p $(DM_TEST_DIR) $(FAILPOINT_ENABLE) @export log_level=error; \ - $(GOTEST) -timeout 5m -covermode=atomic -coverprofile="$(DM_TEST_DIR)/cov.unit_test.out" $(DM_PACKAGES) \ + $(GOTEST) -timeout 5m -covermode=atomic -coverprofile="$(DM_TEST_DIR)/cov.unit_test.out" $(1) \ || { $(FAILPOINT_DISABLE); exit 1; } $(FAILPOINT_DISABLE) +endef + +dm_unit_test: check_failpoint_ctl + $(call run_dm_unit_test,$(DM_PACKAGES)) + +# run unit test for the specified pkg only, like `make dm_unit_test_pkg PKG=github.com/pingcap/dm/dm/master` +dm_unit_test_pkg: check_failpoint_ctl + $(call run_dm_unit_test,$(PKG)) dm_unit_test_in_verify_ci: check_failpoint_ctl tools/bin/gotestsum tools/bin/gocov tools/bin/gocov-xml mkdir -p $(DM_TEST_DIR) diff --git a/dm/dm/master/scheduler/scheduler_test.go b/dm/dm/master/scheduler/scheduler_test.go index c50100ff555..b197d892d4a 100644 --- a/dm/dm/master/scheduler/scheduler_test.go +++ b/dm/dm/master/scheduler/scheduler_test.go @@ -253,12 +253,15 @@ func (t *testScheduler) testSchedulerProgress(c *C, restart int) { c.Assert(s.AddSubTasks(false), IsNil) // can call without configs, return without error, but take no effect. t.subTaskCfgNotExist(c, s, taskName1, sourceID1) t.subTaskStageMatch(c, s, taskName1, sourceID1, pb.Stage_InvalidStage) + t.downstreamMetaNotExist(c, s, taskName1) // start the task. c.Assert(s.AddSubTasks(false, subtaskCfg1), IsNil) c.Assert(terror.ErrSchedulerSubTaskExist.Equal(s.AddSubTasks(false, subtaskCfg1)), IsTrue) // add again. // subtask config and stage exist. t.subTaskCfgExist(c, s, subtaskCfg1) t.subTaskStageMatch(c, s, taskName1, sourceID1, pb.Stage_Running) + t.downstreamMetaExist(c, s, taskName1, subtaskCfg1.To, subtaskCfg1.MetaSchema) + t.downstreamMetaNotExist(c, s, taskName2) // update source config when task already started will failed c.Assert(terror.ErrSchedulerSourceOpTaskExist.Equal(s.UpdateSourceCfg(sourceCfg1)), IsTrue) @@ -596,6 +599,19 @@ func (t *testScheduler) subTaskCfgExist(c *C, s *Scheduler, expectCfg config.Sub c.Assert(cfgM[expectCfg.Name], DeepEquals, expectCfg) } +func (t *testScheduler) downstreamMetaNotExist(c *C, s *Scheduler, task string) { + dbConfig, metaConfig := s.GetDownstreamMetaByTask(task) + c.Assert(dbConfig, IsNil) + c.Assert(metaConfig, Equals, "") +} + +func (t *testScheduler) downstreamMetaExist(c *C, s *Scheduler, task string, expectDBCfg config.DBConfig, expectMetaConfig string) { + dbConfig, metaConfig := s.GetDownstreamMetaByTask(task) + c.Assert(dbConfig, NotNil) + c.Assert(dbConfig, DeepEquals, &expectDBCfg) + c.Assert(metaConfig, Equals, expectMetaConfig) +} + func (t *testScheduler) workerNotExist(c *C, s *Scheduler, worker string) { c.Assert(s.GetWorkerByName(worker), IsNil) wm, _, err := ha.GetAllWorkerInfo(etcdTestCli) diff --git a/dm/pkg/shardddl/optimism/keeper_test.go b/dm/pkg/shardddl/optimism/keeper_test.go index 4a416a2ffe9..1ed49459b94 100644 --- a/dm/pkg/shardddl/optimism/keeper_test.go +++ b/dm/pkg/shardddl/optimism/keeper_test.go @@ -80,6 +80,13 @@ func (t *testKeeper) TestLockKeeper(c *C) { c.Assert(lock1, NotNil) c.Assert(lock1.ID, Equals, lockID1) c.Assert(lk.FindLockByInfo(i11).ID, Equals, lockID1) + + lks := lk.FindLocksByTask("hahaha") + c.Assert(len(lks), Equals, 0) + lks = lk.FindLocksByTask(task1) + c.Assert(len(lks), Equals, 1) + c.Assert(lks[0].ID, Equals, lockID1) + synced, remain := lock1.IsSynced() c.Assert(synced, IsFalse) c.Assert(remain, Equals, 1) @@ -109,6 +116,13 @@ func (t *testKeeper) TestLockKeeper(c *C) { c.Assert(synced, IsTrue) c.Assert(remain, Equals, 0) + lks = lk.FindLocksByTask(task1) + c.Assert(len(lks), Equals, 1) + c.Assert(lks[0].ID, Equals, lockID1) + lks = lk.FindLocksByTask(task2) + c.Assert(len(lks), Equals, 1) + c.Assert(lks[0].ID, Equals, lockID2) + // try to find not-exists lock. lockIDNotExists := "lock-not-exists" c.Assert(lk.FindLock(lockIDNotExists), IsNil) @@ -276,6 +290,11 @@ func (t *testKeeper) TestTableKeeper(c *C) { // no tables exist before Init/Update. c.Assert(tk.FindTables(task1, downSchema, downTable), IsNil) + for schema, tables := range tt11.UpTables { + for table := range tables { + c.Assert(tk.SourceTableExist(tt11.Task, tt11.Source, schema, table, downSchema, downTable), IsFalse) + } + } // Init with `nil` is fine. tk.Init(nil) @@ -287,6 +306,11 @@ func (t *testKeeper) TestTableKeeper(c *C) { c.Assert(tts, HasLen, 2) c.Assert(tts[0], DeepEquals, tt11) c.Assert(tts[1], DeepEquals, tt12) + for schema, tables := range tt11.UpTables { + for table := range tables { + c.Assert(tk.SourceTableExist(tt11.Task, tt11.Source, schema, table, downSchema, downTable), IsTrue) + } + } // adds new tables. c.Assert(tk.Update(st21), IsTrue) @@ -299,11 +323,21 @@ func (t *testKeeper) TestTableKeeper(c *C) { tts = tk.FindTables(task2, downSchema, downTable) c.Assert(tts, HasLen, 1) c.Assert(tts[0], DeepEquals, tt22) + for schema, tables := range tt22.UpTables { + for table := range tables { + c.Assert(tk.SourceTableExist(tt22.Task, tt22.Source, schema, table, downSchema, downTable), IsTrue) + } + } // deletes tables. st22.IsDeleted = true c.Assert(tk.Update(st22), IsTrue) c.Assert(tk.FindTables(task2, downSchema, downTable), IsNil) + for schema, tables := range tt22.UpTables { + for table := range tables { + c.Assert(tk.SourceTableExist(tt22.Task, tt22.Source, schema, table, downSchema, downTable), IsFalse) + } + } // try to delete, but not exist. c.Assert(tk.Update(st22), IsFalse) @@ -315,6 +349,11 @@ func (t *testKeeper) TestTableKeeper(c *C) { c.Assert(tts, HasLen, 2) c.Assert(tts[0], DeepEquals, tt11) c.Assert(tts[1], DeepEquals, tt12) + for schema, tables := range tt11.UpTables { + for table := range tables { + c.Assert(tk.SourceTableExist(tt11.Task, tt11.Source, schema, table, downSchema, downTable), IsTrue) + } + } // add a table for st11. c.Assert(tk.AddTable(task1, st11.Source, "db-2", "tbl-3", downSchema, downTable), IsTrue) @@ -345,6 +384,19 @@ func (t *testKeeper) TestTableKeeper(c *C) { c.Assert(tk.RemoveTable(task1, "not-exit", "db", "tbl-1", downSchema, downTable), IsFalse) tts = tk.FindTables(task1, downSchema, downTable) c.Assert(tts[1], DeepEquals, tt12) + + c.Assert(tk.RemoveTableByTask("hahaha"), IsFalse) + tk.RemoveTableByTaskAndSources("hahaha", nil) + tts = tk.FindTables(task1, downSchema, downTable) + c.Assert(tts, HasLen, 3) + tk.RemoveTableByTaskAndSources(task1, []string{"hahaha"}) + tts = tk.FindTables(task1, downSchema, downTable) + c.Assert(tts, HasLen, 3) + tk.RemoveTableByTaskAndSources(task1, []string{source1, source2}) + tts = tk.FindTables(task1, downSchema, downTable) + c.Assert(tts, HasLen, 1) + c.Assert(tts[0].Source, Equals, "new-source") + c.Assert(tts[0].UpTables["db-2"], HasKey, "tbl-3") } func (t *testKeeper) TestTargetTablesForTask(c *C) { diff --git a/dm/pkg/shardddl/optimism/lock.go b/dm/pkg/shardddl/optimism/lock.go index 15b76fc20a9..fe31a1a7194 100644 --- a/dm/pkg/shardddl/optimism/lock.go +++ b/dm/pkg/shardddl/optimism/lock.go @@ -120,11 +120,11 @@ func (l *Lock) FetchTableInfos(task, source, schema, table string) (*model.Table query := `SELECT table_info FROM ` + dbutil.TableName(l.downstreamMeta.meta, cputil.SyncerCheckpoint(task)) + ` WHERE id = ? AND cp_schema = ? AND cp_table = ?` row := db.DB.QueryRowContext(ctx, query, source, schema, table) if row.Err() != nil { - return nil, row.Err() + return nil, terror.ErrDBExecuteFailed.Delegate(row.Err(), query) } var tiBytes []byte if err := row.Scan(&tiBytes); err != nil { - return nil, err + return nil, terror.ErrDBExecuteFailed.Delegate(err, query) } var ti *model.TableInfo if bytes.Equal(tiBytes, []byte("null")) { @@ -443,6 +443,14 @@ func (l *Lock) TryRemoveTableBySources(sources []string) []string { defer l.mu.Unlock() dropColumns := make([]string, 0) + for col, sourceColumns := range l.columns { + for _, source := range sources { + if _, ok := sourceColumns[source]; ok { + dropColumns = append(dropColumns, col) + } + } + } + for _, source := range sources { if _, ok := l.tables[source]; !ok { continue diff --git a/dm/pkg/shardddl/optimism/lock_test.go b/dm/pkg/shardddl/optimism/lock_test.go index ed76be7ad56..4eead3f18bf 100644 --- a/dm/pkg/shardddl/optimism/lock_test.go +++ b/dm/pkg/shardddl/optimism/lock_test.go @@ -14,8 +14,11 @@ package optimism import ( + "encoding/json" + "fmt" "testing" + "github.com/DATA-DOG/go-sqlmock" . "github.com/pingcap/check" "github.com/pingcap/tidb-tools/pkg/schemacmp" "github.com/pingcap/tidb/parser" @@ -24,6 +27,9 @@ import ( "github.com/pingcap/tidb/util/mock" "go.etcd.io/etcd/integration" + "github.com/pingcap/tiflow/dm/dm/config" + "github.com/pingcap/tiflow/dm/pkg/conn" + "github.com/pingcap/tiflow/dm/pkg/cputil" "github.com/pingcap/tiflow/dm/pkg/log" "github.com/pingcap/tiflow/dm/pkg/terror" "github.com/pingcap/tiflow/dm/pkg/utils" @@ -1437,6 +1443,103 @@ func (t *testLock) TestTryRemoveTable(c *C) { c.Assert(l.TryRemoveTable("not-exist", db, tbl1), IsFalse) } +func (t *testLock) TestTryRemoveTableWithSources(c *C) { + var ( + ID = "test_lock_try_remove_table-`foo`.`bar`" + task = "test_lock_try_remove_table" + source1 = "mysql-replica-1" + source2 = "mysql-replica-2" + downSchema = "foo" + downTable = "bar" + db = "foo" + tbl1 = "bar1" + tbl2 = "bar2" + p = parser.New() + se = mock.NewContext() + tblID int64 = 111 + DDLs1 = []string{"ALTER TABLE bar DROP COLUMN c1"} + ti0 = createTableInfo(c, p, se, tblID, `CREATE TABLE bar (id INT PRIMARY KEY, c1 INT)`) + ti1 = createTableInfo(c, p, se, tblID, `CREATE TABLE bar (id INT PRIMARY KEY)`) + + tables = map[string]map[string]struct{}{db: {tbl1: struct{}{}, tbl2: struct{}{}}} + tts = []TargetTable{newTargetTable(task, source1, downSchema, downTable, tables), newTargetTable(task, source2, downSchema, downTable, tables)} + l = NewLock(etcdTestCli, ID, task, downSchema, downTable, schemacmp.Encode(ti0), tts, nil) + + vers = map[string]map[string]map[string]int64{ + source1: { + db: {tbl1: 0, tbl2: 0}, + }, + source2: { + db: {tbl1: 0, tbl2: 0}, + }, + } + ) + + // only one table exists before TrySync. + t.checkLockSynced(c, l) + t.checkLockNoDone(c, l) + + // TrySync for the first table. + info := newInfoWithVersion(task, source1, db, tbl1, downSchema, downTable, DDLs1, ti0, []*model.TableInfo{ti1}, vers) + DDLs, cols, err := l.TrySync(info, tts) + c.Assert(err, IsNil) + c.Assert(DDLs, DeepEquals, []string{}) + c.Assert(cols, DeepEquals, []string{"c1"}) + c.Assert(l.versions, DeepEquals, vers) + ready := l.Ready() + c.Assert(ready, HasLen, 2) + c.Assert(ready[source1], HasLen, 1) + c.Assert(ready[source1][db], HasLen, 2) + c.Assert(ready[source1][db][tbl1], IsFalse) + c.Assert(ready[source1][db][tbl2], IsTrue) + c.Assert(ready[source2], HasLen, 1) + c.Assert(ready[source2][db], HasLen, 2) + c.Assert(ready[source2][db][tbl1], IsTrue) + c.Assert(ready[source2][db][tbl2], IsTrue) + + // TryRemoveTableBySources with nil + c.Assert(len(l.TryRemoveTableBySources(nil)), Equals, 0) + ready = l.Ready() + c.Assert(ready, HasLen, 2) + + // TryRemoveTableBySources with wrong source + tts = tts[:1] + c.Assert(len(l.TryRemoveTableBySources([]string{"hahaha"})), Equals, 0) + ready = l.Ready() + c.Assert(ready, HasLen, 2) + + // TryRemoveTableBySources with source2 + c.Assert(len(l.TryRemoveTableBySources([]string{source2})), Equals, 0) + ready = l.Ready() + c.Assert(ready, HasLen, 1) + c.Assert(ready[source1], HasLen, 1) + c.Assert(ready[source1][db], HasLen, 2) + c.Assert(ready[source1][db][tbl1], IsFalse) + c.Assert(ready[source1][db][tbl2], IsTrue) + delete(vers, source2) + c.Assert(l.versions, DeepEquals, vers) + c.Assert(l.HasTables(), IsTrue) + + // TrySync with second table + info = newInfoWithVersion(task, source1, db, tbl2, downSchema, downTable, DDLs1, ti0, []*model.TableInfo{ti1}, vers) + DDLs, cols, err = l.TrySync(info, tts) + c.Assert(err, IsNil) + c.Assert(DDLs, DeepEquals, DDLs1) + c.Assert(cols, DeepEquals, []string{"c1"}) + c.Assert(l.versions, DeepEquals, vers) + ready = l.Ready() + c.Assert(ready, HasLen, 1) + c.Assert(ready[source1], HasLen, 1) + c.Assert(ready[source1][db], HasLen, 2) + c.Assert(ready[source1][db][tbl1], IsTrue) + c.Assert(ready[source1][db][tbl2], IsTrue) + + // TryRemoveTableBySources with source1,source2 + cols = l.TryRemoveTableBySources([]string{source1}) + c.Assert(cols, DeepEquals, []string{"c1"}) + c.Assert(l.HasTables(), IsFalse) +} + func (t *testLock) TestLockTryMarkDone(c *C) { var ( ID = "test_lock_try_mark_done-`foo`.`bar`" @@ -1909,3 +2012,60 @@ func (t *testLock) TestLockTrySyncDifferentIndex(c *C) { c.Assert(l.versions, DeepEquals, vers) t.checkLockSynced(c, l) } + +func (t *testLock) TestFetchTableInfo(c *C) { + var ( + meta = "meta" + ID = "test_lock_try_sync_index-`foo`.`bar`" + task = "test_lock_try_sync_index" + source = "mysql-replica-1" + downSchema = "db" + downTable = "bar" + schema = "db" + tbls = []string{"bar1", "bar2"} + p = parser.New() + se = mock.NewContext() + tblID int64 = 111 + ti0 = createTableInfo(c, p, se, tblID, `CREATE TABLE bar (id INT PRIMARY KEY, c1 INT, UNIQUE INDEX idx_c1(c1))`) + tables = map[string]map[string]struct{}{ + schema: {tbls[0]: struct{}{}, tbls[1]: struct{}{}}, + } + tts = []TargetTable{ + newTargetTable(task, source, downSchema, downTable, tables), + } + query = fmt.Sprintf("SELECT table_info FROM `%s`.`%s` WHERE id = \\? AND cp_schema = \\? AND cp_table = \\?", meta, cputil.SyncerCheckpoint(task)) + ) + + mock := conn.InitMockDB(c) + baseDB, err := conn.DefaultDBProvider.Apply(&config.DBConfig{}) + c.Assert(err, IsNil) + + // nil downstream meta + l := NewLock(etcdTestCli, ID, task, downSchema, downTable, schemacmp.Encode(ti0), tts, nil) + ti, err := l.FetchTableInfos(task, source, schema, tbls[0]) + c.Assert(terror.ErrMasterOptimisticDownstreamMetaNotFound.Equal(err), IsTrue) + c.Assert(ti, IsNil) + + // table info not exist + l = NewLock(etcdTestCli, ID, task, downSchema, downTable, schemacmp.Encode(ti0), tts, &DownstreamMeta{db: baseDB, meta: meta}) + mock.ExpectQuery(query).WithArgs(source, schema, tbls[0]).WillReturnRows(sqlmock.NewRows([]string{"table_info"})) + ti, err = l.FetchTableInfos(task, source, schema, tbls[0]) + c.Assert(terror.ErrDBExecuteFailed.Equal(err), IsTrue) + c.Assert(ti, IsNil) + + // null table info + l = NewLock(etcdTestCli, ID, task, downSchema, downTable, schemacmp.Encode(ti0), tts, &DownstreamMeta{db: baseDB, meta: meta}) + mock.ExpectQuery(query).WithArgs(source, schema, tbls[0]).WillReturnRows(sqlmock.NewRows([]string{"table_info"}).AddRow("null")) + ti, err = l.FetchTableInfos(task, source, schema, tbls[0]) + c.Assert(terror.ErrMasterOptimisticDownstreamMetaNotFound.Equal(err), IsTrue) + c.Assert(ti, IsNil) + + // succeed + tiBytes, err := json.Marshal(ti0) + c.Assert(err, IsNil) + mock.ExpectQuery(query).WithArgs(source, schema, tbls[0]).WillReturnRows(sqlmock.NewRows([]string{"table_info"}).AddRow(tiBytes)) + ti, err = l.FetchTableInfos(task, source, schema, tbls[0]) + c.Assert(err, IsNil) + c.Assert(mock.ExpectationsWereMet(), IsNil) + c.Assert(ti, DeepEquals, ti0) +} diff --git a/dm/syncer/shardddl/optimist_test.go b/dm/syncer/shardddl/optimist_test.go index 40dbeb67ae2..7b2a76bfbed 100644 --- a/dm/syncer/shardddl/optimist_test.go +++ b/dm/syncer/shardddl/optimist_test.go @@ -95,6 +95,9 @@ func (t *testOptimist) TestOptimist(c *C) { ctx, cancel := context.WithCancel(context.Background()) defer cancel() + tables := o.Tables() + c.Assert(len(tables), Equals, 0) + // init with some source tables. err := o.Init(sourceTables) c.Assert(err, IsNil) @@ -104,6 +107,9 @@ func (t *testOptimist) TestOptimist(c *C) { c.Assert(stm[task], HasLen, 1) c.Assert(stm[task][source], DeepEquals, o.tables) + tables = o.Tables() + c.Assert(len(tables), Equals, 4) + // no info and operation in pending. c.Assert(o.PendingInfo(), IsNil) c.Assert(o.PendingOperation(), IsNil) From 7c449aca0bf6da16d347b8d53829b0e8184089c7 Mon Sep 17 00:00:00 2001 From: gmhdbjd Date: Wed, 22 Dec 2021 07:06:03 -0500 Subject: [PATCH 17/31] update fmt --- dm/tests/shardddl_optimistic/run.sh | 54 ++++++++++++++--------------- 1 file changed, 27 insertions(+), 27 deletions(-) diff --git a/dm/tests/shardddl_optimistic/run.sh b/dm/tests/shardddl_optimistic/run.sh index e6e7cb49a04..526ae3169de 100644 --- a/dm/tests/shardddl_optimistic/run.sh +++ b/dm/tests/shardddl_optimistic/run.sh @@ -50,15 +50,15 @@ function DM_DIFFERENT_SCHEMA_FULL() { # create table with different schema, init data, and create table in downstream manually run_case DIFFERENT_SCHEMA_FULL "double-source-optimistic" \ "run_sql_source1 \"create table ${shardddl1}.${tb1} (a int primary key);\"; \ - run_sql_source1 \"create table ${shardddl1}.${tb2} (a int primary key, b varchar(10));\"; \ - run_sql_source2 \"create table ${shardddl1}.${tb1} (a int primary key, c text);\"; \ - run_sql_source2 \"create table ${shardddl1}.${tb2} (a int primary key, b varchar(10), c text);\"; \ - run_sql_source1 \"insert into ${shardddl1}.${tb1} values(1);\"; \ - run_sql_source1 \"insert into ${shardddl1}.${tb2} values(2,'22');\"; \ - run_sql_source2 \"insert into ${shardddl1}.${tb1} values(3,'333');\"; \ - run_sql_source2 \"insert into ${shardddl1}.${tb2} values(4,'44','444');\"; \ - run_sql_tidb \"create database if not exists ${shardddl};\"; \ - run_sql_tidb \"create table ${shardddl}.${tb} (a int primary key, b varchar(10), c text);\"" \ + run_sql_source1 \"create table ${shardddl1}.${tb2} (a int primary key, b varchar(10));\"; \ + run_sql_source2 \"create table ${shardddl1}.${tb1} (a int primary key, c text);\"; \ + run_sql_source2 \"create table ${shardddl1}.${tb2} (a int primary key, b varchar(10), c text);\"; \ + run_sql_source1 \"insert into ${shardddl1}.${tb1} values(1);\"; \ + run_sql_source1 \"insert into ${shardddl1}.${tb2} values(2,'22');\"; \ + run_sql_source2 \"insert into ${shardddl1}.${tb1} values(3,'333');\"; \ + run_sql_source2 \"insert into ${shardddl1}.${tb2} values(4,'44','444');\"; \ + run_sql_tidb \"create database if not exists ${shardddl};\"; \ + run_sql_tidb \"create table ${shardddl}.${tb} (a int primary key, b varchar(10), c text);\"" \ "clean_table" "optimistic" } @@ -153,15 +153,15 @@ function DM_DIFFERENT_SCHEMA_INCREMENTAL() { # create table with different schema, init data, and create table in downstream manually run_case DIFFERENT_SCHEMA_INCREMENTAL "double-source-optimistic" \ "run_sql_source1 \"create table ${shardddl1}.${tb1} (a int primary key);\"; \ - run_sql_source1 \"create table ${shardddl1}.${tb2} (a int primary key, b varchar(10));\"; \ - run_sql_source2 \"create table ${shardddl1}.${tb1} (a int primary key, c text);\"; \ - run_sql_source2 \"create table ${shardddl1}.${tb2} (a int primary key, b varchar(10), c text);\"; \ - run_sql_source1 \"insert into ${shardddl1}.${tb1} values(1);\"; \ - run_sql_source1 \"insert into ${shardddl1}.${tb2} values(2,'22');\"; \ - run_sql_source2 \"insert into ${shardddl1}.${tb1} values(3,'333');\"; \ - run_sql_source2 \"insert into ${shardddl1}.${tb2} values(4,'44','444');\"; \ - run_sql_tidb \"create database if not exists ${shardddl};\"; \ - run_sql_tidb \"create table ${shardddl}.${tb} (a int primary key, b varchar(10), c text);\"" \ + run_sql_source1 \"create table ${shardddl1}.${tb2} (a int primary key, b varchar(10));\"; \ + run_sql_source2 \"create table ${shardddl1}.${tb1} (a int primary key, c text);\"; \ + run_sql_source2 \"create table ${shardddl1}.${tb2} (a int primary key, b varchar(10), c text);\"; \ + run_sql_source1 \"insert into ${shardddl1}.${tb1} values(1);\"; \ + run_sql_source1 \"insert into ${shardddl1}.${tb2} values(2,'22');\"; \ + run_sql_source2 \"insert into ${shardddl1}.${tb1} values(3,'333');\"; \ + run_sql_source2 \"insert into ${shardddl1}.${tb2} values(4,'44','444');\"; \ + run_sql_tidb \"create database if not exists ${shardddl};\"; \ + run_sql_tidb \"create table ${shardddl}.${tb} (a int primary key, b varchar(10), c text);\"" \ "clean_table" "optimistic" } @@ -216,15 +216,15 @@ function DM_RESTART_TASK_MASTER_WORKER() { # create table with different schema, init data, and create table in downstream manually run_case RESTART_TASK_MASTER_WORKER "double-source-optimistic" \ "run_sql_source1 \"create table ${shardddl1}.${tb1} (a int primary key);\"; \ - run_sql_source1 \"create table ${shardddl1}.${tb2} (a int primary key, b varchar(10));\"; \ - run_sql_source2 \"create table ${shardddl1}.${tb1} (a int primary key, c text);\"; \ - run_sql_source2 \"create table ${shardddl1}.${tb2} (a int primary key, b varchar(10), c text);\"; \ - run_sql_source1 \"insert into ${shardddl1}.${tb1} values(1);\"; \ - run_sql_source1 \"insert into ${shardddl1}.${tb2} values(2,'22');\"; \ - run_sql_source2 \"insert into ${shardddl1}.${tb1} values(3,'333');\"; \ - run_sql_source2 \"insert into ${shardddl1}.${tb2} values(4,'44','444');\"; \ - run_sql_tidb \"create database if not exists ${shardddl};\"; \ - run_sql_tidb \"create table ${shardddl}.${tb} (a int primary key, b varchar(10), c text);\"" \ + run_sql_source1 \"create table ${shardddl1}.${tb2} (a int primary key, b varchar(10));\"; \ + run_sql_source2 \"create table ${shardddl1}.${tb1} (a int primary key, c text);\"; \ + run_sql_source2 \"create table ${shardddl1}.${tb2} (a int primary key, b varchar(10), c text);\"; \ + run_sql_source1 \"insert into ${shardddl1}.${tb1} values(1);\"; \ + run_sql_source1 \"insert into ${shardddl1}.${tb2} values(2,'22');\"; \ + run_sql_source2 \"insert into ${shardddl1}.${tb1} values(3,'333');\"; \ + run_sql_source2 \"insert into ${shardddl1}.${tb2} values(4,'44','444');\"; \ + run_sql_tidb \"create database if not exists ${shardddl};\"; \ + run_sql_tidb \"create table ${shardddl}.${tb} (a int primary key, b varchar(10), c text);\"" \ "clean_table" "optimistic" } From 84d9aeba16b0349a78077fb4ee8a81358fb1c671 Mon Sep 17 00:00:00 2001 From: gmhdbjd Date: Wed, 22 Dec 2021 22:27:13 -0500 Subject: [PATCH 18/31] code review --- dm/dm/master/shardddl/optimist.go | 2 ++ dm/pkg/shardddl/optimism/keeper.go | 1 + dm/pkg/shardddl/optimism/lock.go | 2 ++ 3 files changed, 5 insertions(+) diff --git a/dm/dm/master/shardddl/optimist.go b/dm/dm/master/shardddl/optimist.go index a631e41d3e7..d88199f55a8 100644 --- a/dm/dm/master/shardddl/optimist.go +++ b/dm/dm/master/shardddl/optimist.go @@ -163,6 +163,8 @@ func (o *Optimist) ShowLocks(task string, sources []string) []*pb.DDLLock { // RemoveMetaData removes meta data for a specified task // NOTE: this function can only be used when the specified task is not running. +// This function only be used when --remove-meta or stop-task +// NOTE: For stop-task, we still delete drop columns in etcd though user may restart the task again later. func (o *Optimist) RemoveMetaDataWithTask(task string) error { o.mu.Lock() defer o.mu.Unlock() diff --git a/dm/pkg/shardddl/optimism/keeper.go b/dm/pkg/shardddl/optimism/keeper.go index 7a8e0d3fd45..6ea72a5c6a4 100644 --- a/dm/pkg/shardddl/optimism/keeper.go +++ b/dm/pkg/shardddl/optimism/keeper.go @@ -74,6 +74,7 @@ func (lk *LockKeeper) getDownstreamMeta(task string) (*DownstreamMeta, error) { return nil, err } downstreamMeta := &DownstreamMeta{db: db, meta: meta} + lk.downstreamMetaMap[task] = downstreamMeta return downstreamMeta, nil } diff --git a/dm/pkg/shardddl/optimism/lock.go b/dm/pkg/shardddl/optimism/lock.go index fe31a1a7194..2b91f1d9199 100644 --- a/dm/pkg/shardddl/optimism/lock.go +++ b/dm/pkg/shardddl/optimism/lock.go @@ -442,11 +442,13 @@ func (l *Lock) TryRemoveTableBySources(sources []string) []string { l.mu.Lock() defer l.mu.Unlock() + // record drop columns for sources dropColumns := make([]string, 0) for col, sourceColumns := range l.columns { for _, source := range sources { if _, ok := sourceColumns[source]; ok { dropColumns = append(dropColumns, col) + break } } } From 9fb1d3a502ed5fec1305e8a609ea0bdc38e772a6 Mon Sep 17 00:00:00 2001 From: gmhdbjd Date: Thu, 23 Dec 2021 00:58:16 -0500 Subject: [PATCH 19/31] remove downstream when stop task --- dm/dm/master/shardddl/optimist.go | 3 ++ dm/pkg/shardddl/optimism/keeper.go | 6 +++ dm/pkg/shardddl/optimism/keeper_test.go | 61 +++++++++++++++++++++++++ 3 files changed, 70 insertions(+) diff --git a/dm/dm/master/shardddl/optimist.go b/dm/dm/master/shardddl/optimist.go index d88199f55a8..fb9a1d81e4d 100644 --- a/dm/dm/master/shardddl/optimist.go +++ b/dm/dm/master/shardddl/optimist.go @@ -186,6 +186,7 @@ func (o *Optimist) RemoveMetaDataWithTask(task string) error { o.lk.RemoveLock(op.ID) } + o.lk.RemoveDownstreamMeta(task) o.tk.RemoveTableByTask(task) // clear meta data in etcd @@ -217,6 +218,8 @@ func (o *Optimist) RemoveMetaDataWithTaskAndSources(task string, sources ...stri o.lk.RemoveLock(lock.ID) } } + + o.lk.RemoveDownstreamMeta(task) // remove source table in table keeper o.tk.RemoveTableByTaskAndSources(task, sources) o.logger.Debug("the tables removed from the table keeper", zap.String("task", task), zap.Strings("source", sources)) diff --git a/dm/pkg/shardddl/optimism/keeper.go b/dm/pkg/shardddl/optimism/keeper.go index 6ea72a5c6a4..cb0e9f999cf 100644 --- a/dm/pkg/shardddl/optimism/keeper.go +++ b/dm/pkg/shardddl/optimism/keeper.go @@ -78,6 +78,11 @@ func (lk *LockKeeper) getDownstreamMeta(task string) (*DownstreamMeta, error) { return downstreamMeta, nil } +// RemoveDownstreamMeta removes downstream mate by task. +func (lk *LockKeeper) RemoveDownstreamMeta(task string) { + delete(lk.downstreamMetaMap, task) +} + // TrySync tries to sync the lock. func (lk *LockKeeper) TrySync(cli *clientv3.Client, info Info, tts []TargetTable) (string, []string, []string, error) { var ( @@ -176,6 +181,7 @@ func (lk *LockKeeper) Clear() { defer lk.mu.Unlock() lk.locks = make(map[string]*Lock) + lk.downstreamMetaMap = make(map[string]*DownstreamMeta) } // genDDLLockID generates DDL lock ID from its info. diff --git a/dm/pkg/shardddl/optimism/keeper_test.go b/dm/pkg/shardddl/optimism/keeper_test.go index 1ed49459b94..5aefe4173c2 100644 --- a/dm/pkg/shardddl/optimism/keeper_test.go +++ b/dm/pkg/shardddl/optimism/keeper_test.go @@ -23,6 +23,8 @@ import ( "go.etcd.io/etcd/integration" "github.com/pingcap/tiflow/dm/dm/config" + "github.com/pingcap/tiflow/dm/pkg/conn" + "github.com/pingcap/tiflow/dm/pkg/terror" ) type testKeeper struct{} @@ -468,3 +470,62 @@ func (t *testKeeper) TestTargetTablesForTask(c *C) { func getDownstreamMeta(string) (*config.DBConfig, string) { return nil, "" } + +func (t *testKeeper) TestGetDownstreamMeta(c *C) { + var ( + task1 = "hahaha" + task2 = "hihihi" + task3 = "hehehe" + ) + getDownstreamMetaFunc := func(task string) (*config.DBConfig, string) { + switch task { + case task1, task2: + return &config.DBConfig{}, "meta" + default: + return nil, "" + } + } + + conn.InitMockDB(c) + lk := NewLockKeeper(getDownstreamMetaFunc) + c.Assert(lk.downstreamMetaMap, HasLen, 0) + + downstreamMeta, err := lk.getDownstreamMeta(task3) + c.Assert(downstreamMeta, IsNil) + c.Assert(terror.ErrMasterOptimisticDownstreamMetaNotFound.Equal(err), IsTrue) + + downstreamMeta, err = lk.getDownstreamMeta(task1) + c.Assert(err, IsNil) + c.Assert(lk.downstreamMetaMap, HasLen, 1) + c.Assert(downstreamMeta, Equals, lk.downstreamMetaMap[task1]) + downstreamMeta2, err := lk.getDownstreamMeta(task1) + c.Assert(err, IsNil) + c.Assert(lk.downstreamMetaMap, HasLen, 1) + c.Assert(downstreamMeta, Equals, downstreamMeta2) + + downstreamMeta3, err := lk.getDownstreamMeta(task2) + c.Assert(err, IsNil) + c.Assert(lk.downstreamMetaMap, HasLen, 2) + c.Assert(lk.downstreamMetaMap, HasKey, task1) + c.Assert(lk.downstreamMetaMap, HasKey, task2) + c.Assert(downstreamMeta3, Equals, lk.downstreamMetaMap[task2]) + + lk.RemoveDownstreamMeta(task3) + c.Assert(lk.downstreamMetaMap, HasLen, 2) + c.Assert(lk.downstreamMetaMap, HasKey, task1) + c.Assert(lk.downstreamMetaMap, HasKey, task2) + + lk.RemoveDownstreamMeta(task1) + c.Assert(lk.downstreamMetaMap, HasLen, 1) + c.Assert(lk.downstreamMetaMap, HasKey, task2) + c.Assert(downstreamMeta3, Equals, lk.downstreamMetaMap[task2]) + + downstreamMeta, err = lk.getDownstreamMeta(task1) + c.Assert(err, IsNil) + c.Assert(lk.downstreamMetaMap, HasLen, 2) + c.Assert(downstreamMeta, Equals, lk.downstreamMetaMap[task1]) + c.Assert(downstreamMeta3, Equals, lk.downstreamMetaMap[task2]) + + lk.Clear() + c.Assert(lk.downstreamMetaMap, HasLen, 0) +} From 00171da324f9a2a6d6c07efaf574e1ef241b3c9c Mon Sep 17 00:00:00 2001 From: gmhdbjd Date: Thu, 23 Dec 2021 02:01:30 -0500 Subject: [PATCH 20/31] remove metadata after stop task succeed --- dm/dm/master/server.go | 28 +++++++++++++--------------- dm/tests/shardddl_optimistic/run.sh | 2 +- 2 files changed, 14 insertions(+), 16 deletions(-) diff --git a/dm/dm/master/server.go b/dm/dm/master/server.go index 98a5db7012b..bf0794829a4 100644 --- a/dm/dm/master/server.go +++ b/dm/dm/master/server.go @@ -551,21 +551,7 @@ func (s *Server) OperateTask(ctx context.Context, req *pb.OperateTaskRequest) (* } var err error if expect == pb.Stage_Stopped { - if err = s.scheduler.RemoveSubTasks(req.Name, sources...); err != nil { - resp.Msg = err.Error() - // nolint:nilerr - return resp, nil - } - - // delete meta data for optimist - if len(req.Sources) == 0 { - err2 = s.optimist.RemoveMetaDataWithTask(req.Name) - } else { - err2 = s.optimist.RemoveMetaDataWithTaskAndSources(req.Name, sources...) - } - if err2 != nil { - log.L().Error("failed to delete metadata for task", zap.String("task name", req.Name), log.ShortError(err2)) - } + err = s.scheduler.RemoveSubTasks(req.Name, sources...) } else { err = s.scheduler.UpdateExpectSubTaskStage(expect, req.Name, sources...) } @@ -577,6 +563,18 @@ func (s *Server) OperateTask(ctx context.Context, req *pb.OperateTaskRequest) (* resp.Result = true resp.Sources = s.getSourceRespsAfterOperation(ctx, req.Name, sources, []string{}, req) + + if expect == pb.Stage_Stopped { + // delete meta data for optimist + if len(req.Sources) == 0 { + err2 = s.optimist.RemoveMetaDataWithTask(req.Name) + } else { + err2 = s.optimist.RemoveMetaDataWithTaskAndSources(req.Name, sources...) + } + if err2 != nil { + log.L().Error("failed to delete metadata for task", zap.String("task name", req.Name), log.ShortError(err2)) + } + } return resp, nil } diff --git a/dm/tests/shardddl_optimistic/run.sh b/dm/tests/shardddl_optimistic/run.sh index 526ae3169de..7372d7a0428 100644 --- a/dm/tests/shardddl_optimistic/run.sh +++ b/dm/tests/shardddl_optimistic/run.sh @@ -229,7 +229,7 @@ function DM_RESTART_TASK_MASTER_WORKER() { } function random_restart() { - mod=$RANDOM%4 + mod=$(($RANDOM%4)) if [[ "$mod" == "0" ]]; then restart_master elif [[ "$mod" == "1" ]]; then From fd2ab4f456c34e4ad2df5f6b186c6ef559bc9f0e Mon Sep 17 00:00:00 2001 From: gmhdbjd Date: Thu, 23 Dec 2021 03:56:30 -0500 Subject: [PATCH 21/31] fix fmt --- dm/tests/shardddl_optimistic/run.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dm/tests/shardddl_optimistic/run.sh b/dm/tests/shardddl_optimistic/run.sh index 7372d7a0428..ea18348d1df 100644 --- a/dm/tests/shardddl_optimistic/run.sh +++ b/dm/tests/shardddl_optimistic/run.sh @@ -229,7 +229,7 @@ function DM_RESTART_TASK_MASTER_WORKER() { } function random_restart() { - mod=$(($RANDOM%4)) + mod=$(($RANDOM % 4)) if [[ "$mod" == "0" ]]; then restart_master elif [[ "$mod" == "1" ]]; then From d179ffb62637e726b533f2dc06f351c70e1931b8 Mon Sep 17 00:00:00 2001 From: gmhdbjd Date: Thu, 23 Dec 2021 21:29:45 -0500 Subject: [PATCH 22/31] fix test --- dm/tests/shardddl_optimistic/run.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dm/tests/shardddl_optimistic/run.sh b/dm/tests/shardddl_optimistic/run.sh index ea18348d1df..e097956f148 100644 --- a/dm/tests/shardddl_optimistic/run.sh +++ b/dm/tests/shardddl_optimistic/run.sh @@ -124,7 +124,7 @@ function DM_DIFFERENT_SCHEMA_INCREMENTAL_CASE() { run_dm_ctl $WORK_DIR "127.0.0.1:$MASTER_PORT" \ "start-task $WORK_DIR/task.yaml --remove-meta" - run_dm_ctl $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" \ "query-status test" \ "Column count doesn't match" 2 From 54279fc169c116b65c0c83a63523322ce8a55474 Mon Sep 17 00:00:00 2001 From: gmhdbjd Date: Fri, 24 Dec 2021 02:41:44 -0500 Subject: [PATCH 23/31] pre join table for new lock --- dm/pkg/shardddl/optimism/lock.go | 29 ++++++++++++++++++++++++++++- 1 file changed, 28 insertions(+), 1 deletion(-) diff --git a/dm/pkg/shardddl/optimism/lock.go b/dm/pkg/shardddl/optimism/lock.go index 2b91f1d9199..5d98fe32512 100644 --- a/dm/pkg/shardddl/optimism/lock.go +++ b/dm/pkg/shardddl/optimism/lock.go @@ -103,7 +103,8 @@ func NewLock(cli *clientv3.Client, id, task, downSchema, downTable string, joine } l.addTables(tts) metrics.ReportDDLPending(task, metrics.DDLPendingNone, metrics.DDLPendingSynced) - + // pre join because tables may have different schema at the beginning + l.joinTable() return l } @@ -137,6 +138,32 @@ func (l *Lock) FetchTableInfos(task, source, schema, table string) (*model.Table return ti, nil } +// joinTable join tables for a lock and update l.joined. +func (l *Lock) joinTable() { + var ( + joined = l.joined + firstTable = true + ) + for _, schemaTables := range l.tables { + for _, tables := range schemaTables { + for _, ti := range tables { + if firstTable { + joined = ti + firstTable = false + } else { + newJoined, err := joined.Join(ti) + if err != nil { + log.L().Error(fmt.Sprintf("fail to join table info %s with %s", joined, ti), zap.String("lockID", l.ID), log.ShortError(err)) + return + } + joined = newJoined + } + } + } + } + l.joined = joined +} + // TrySync tries to sync the lock, re-entrant. // new upstream sources may join when the DDL lock is in syncing, // so we need to merge these new sources. From 642cf8fae1ec452234563766f72bb86f0aa6345c Mon Sep 17 00:00:00 2001 From: gmhdbjd Date: Fri, 24 Dec 2021 02:42:53 -0500 Subject: [PATCH 24/31] address comment --- Makefile | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/Makefile b/Makefile index 0207eeb9378..2a10fe02198 100644 --- a/Makefile +++ b/Makefile @@ -288,7 +288,7 @@ endef dm_unit_test: check_failpoint_ctl $(call run_dm_unit_test,$(DM_PACKAGES)) -# run unit test for the specified pkg only, like `make dm_unit_test_pkg PKG=github.com/pingcap/dm/dm/master` +# run unit test for the specified pkg only, like `make dm_unit_test_pkg PKG=github.com/pingcap/tiflow/dm/dm/master` dm_unit_test_pkg: check_failpoint_ctl $(call run_dm_unit_test,$(PKG)) From 8e2520e7666981a35b2924f8fe4f25bd8e6c6004 Mon Sep 17 00:00:00 2001 From: gmhdbjd Date: Tue, 18 Jan 2022 14:57:36 +0800 Subject: [PATCH 25/31] address comment --- dm/dm/master/shardddl/optimist.go | 8 +++---- dm/pkg/shardddl/optimism/keeper.go | 5 +++- dm/syncer/syncer.go | 38 +++++++++++++++++------------- 3 files changed, 29 insertions(+), 22 deletions(-) diff --git a/dm/dm/master/shardddl/optimist.go b/dm/dm/master/shardddl/optimist.go index fb9a1d81e4d..cfb571047a8 100644 --- a/dm/dm/master/shardddl/optimist.go +++ b/dm/dm/master/shardddl/optimist.go @@ -209,11 +209,9 @@ func (o *Optimist) RemoveMetaDataWithTaskAndSources(task string, sources ...stri locks := o.lk.FindLocksByTask(task) for _, lock := range locks { // remove table by sources for related lock - if lock != nil { - cols := lock.TryRemoveTableBySources(sources) - dropColumns[lock.ID] = cols - o.logger.Debug("the tables removed from the lock", zap.String("task", task), zap.Strings("sources", sources)) - } + cols := lock.TryRemoveTableBySources(sources) + dropColumns[lock.ID] = cols + o.logger.Debug("the tables removed from the lock", zap.String("task", task), zap.Strings("sources", sources)) if !lock.HasTables() { o.lk.RemoveLock(lock.ID) } diff --git a/dm/pkg/shardddl/optimism/keeper.go b/dm/pkg/shardddl/optimism/keeper.go index cb0e9f999cf..1d3833040a2 100644 --- a/dm/pkg/shardddl/optimism/keeper.go +++ b/dm/pkg/shardddl/optimism/keeper.go @@ -80,7 +80,10 @@ func (lk *LockKeeper) getDownstreamMeta(task string) (*DownstreamMeta, error) { // RemoveDownstreamMeta removes downstream mate by task. func (lk *LockKeeper) RemoveDownstreamMeta(task string) { - delete(lk.downstreamMetaMap, task) + if downstreamMeta, ok := lk.downstreamMetaMap[task]; ok { + downstreamMeta.db.Close() + delete(lk.downstreamMetaMap, task) + } } // TrySync tries to sync the lock. diff --git a/dm/syncer/syncer.go b/dm/syncer/syncer.go index c5403cd3938..362ff483b27 100644 --- a/dm/syncer/syncer.go +++ b/dm/syncer/syncer.go @@ -1490,22 +1490,8 @@ func (s *Syncer) Run(ctx context.Context) (err error) { cleanDumpFile = false } } - tbls := s.optimist.Tables() - sourceTables := make([]*filter.Table, 0, len(tbls)) - tableInfos := make([]*model.TableInfo, 0, len(tbls)) - for _, tbl := range tbls { - sourceTable := tbl[0] - targetTable := tbl[1] - tableInfo, err2 := s.getTableInfo(tctx, &sourceTable, &targetTable) - if err2 != nil { - return err2 - } - sourceTables = append(sourceTables, &sourceTable) - tableInfos = append(tableInfos, tableInfo) - } - err = s.checkpoint.FlushPointsWithTableInfos(tctx, sourceTables, tableInfos) - if err != nil { - tctx.L().Error("failed to flush table points with table infos", log.ShortError(err)) + if s.cfg.ShardMode == config.ShardOptimistic { + s.flushOptimisticTableInfos(tctx) } } @@ -3771,3 +3757,23 @@ func calculateChanSize(queueSize, workerCount int, compact bool) int { } return chanSize } + +func (s *Syncer) flushOptimisticTableInfos(tctx *tcontext.Context) { + tbls := s.optimist.Tables() + sourceTables := make([]*filter.Table, 0, len(tbls)) + tableInfos := make([]*model.TableInfo, 0, len(tbls)) + for _, tbl := range tbls { + sourceTable := tbl[0] + targetTable := tbl[1] + tableInfo, err := s.getTableInfo(tctx, &sourceTable, &targetTable) + if err != nil { + tctx.L().Error("failed to get table infos", log.ShortError(err)) + continue + } + sourceTables = append(sourceTables, &sourceTable) + tableInfos = append(tableInfos, tableInfo) + } + if err := s.checkpoint.FlushPointsWithTableInfos(tctx, sourceTables, tableInfos); err != nil { + tctx.L().Error("failed to flush table points with table infos", log.ShortError(err)) + } +} From ece233f88b082725f85b96371f093942e9d06e2d Mon Sep 17 00:00:00 2001 From: gmhdbjd Date: Wed, 19 Jan 2022 16:13:59 +0800 Subject: [PATCH 26/31] address comment --- dm/pkg/shardddl/optimism/keeper.go | 16 ++++------------ dm/pkg/shardddl/optimism/lock.go | 8 +++++++- dm/pkg/shardddl/optimism/lock_test.go | 14 ++++++++------ 3 files changed, 19 insertions(+), 19 deletions(-) diff --git a/dm/pkg/shardddl/optimism/keeper.go b/dm/pkg/shardddl/optimism/keeper.go index 1d3833040a2..d380d43e130 100644 --- a/dm/pkg/shardddl/optimism/keeper.go +++ b/dm/pkg/shardddl/optimism/keeper.go @@ -21,7 +21,6 @@ import ( "go.etcd.io/etcd/clientv3" "github.com/pingcap/tiflow/dm/dm/config" - "github.com/pingcap/tiflow/dm/pkg/conn" "github.com/pingcap/tiflow/dm/pkg/log" "github.com/pingcap/tiflow/dm/pkg/terror" "github.com/pingcap/tiflow/dm/pkg/utils" @@ -29,8 +28,8 @@ import ( // DownstreamMeta used to fetch table info from downstream. type DownstreamMeta struct { - db *conn.BaseDB - meta string + dbConfig *config.DBConfig + meta string } // LockKeeper used to keep and handle DDL lock conveniently. @@ -69,21 +68,14 @@ func (lk *LockKeeper) getDownstreamMeta(task string) (*DownstreamMeta, error) { if dbConfig == nil { return nil, terror.ErrMasterOptimisticDownstreamMetaNotFound.Generate(task) } - db, err := conn.DefaultDBProvider.Apply(dbConfig) - if err != nil { - return nil, err - } - downstreamMeta := &DownstreamMeta{db: db, meta: meta} + downstreamMeta := &DownstreamMeta{dbConfig: dbConfig, meta: meta} lk.downstreamMetaMap[task] = downstreamMeta return downstreamMeta, nil } // RemoveDownstreamMeta removes downstream mate by task. func (lk *LockKeeper) RemoveDownstreamMeta(task string) { - if downstreamMeta, ok := lk.downstreamMetaMap[task]; ok { - downstreamMeta.db.Close() - delete(lk.downstreamMetaMap, task) - } + delete(lk.downstreamMetaMap, task) } // TrySync tries to sync the lock. diff --git a/dm/pkg/shardddl/optimism/lock.go b/dm/pkg/shardddl/optimism/lock.go index 5d98fe32512..d1b207a9f48 100644 --- a/dm/pkg/shardddl/optimism/lock.go +++ b/dm/pkg/shardddl/optimism/lock.go @@ -29,6 +29,7 @@ import ( "golang.org/x/net/context" "github.com/pingcap/tiflow/dm/dm/master/metrics" + "github.com/pingcap/tiflow/dm/pkg/conn" "github.com/pingcap/tiflow/dm/pkg/cputil" "github.com/pingcap/tiflow/dm/pkg/log" "github.com/pingcap/tiflow/dm/pkg/terror" @@ -114,7 +115,12 @@ func (l *Lock) FetchTableInfos(task, source, schema, table string) (*model.Table return nil, terror.ErrMasterOptimisticDownstreamMetaNotFound.Generate(task) } - db := l.downstreamMeta.db + db, err := conn.DefaultDBProvider.Apply(l.downstreamMeta.dbConfig) + if err != nil { + return nil, err + } + defer db.Close() + ctx, cancel := context.WithTimeout(context.Background(), dbutil.DefaultTimeout) defer cancel() diff --git a/dm/pkg/shardddl/optimism/lock_test.go b/dm/pkg/shardddl/optimism/lock_test.go index 4eead3f18bf..5960b139491 100644 --- a/dm/pkg/shardddl/optimism/lock_test.go +++ b/dm/pkg/shardddl/optimism/lock_test.go @@ -2036,10 +2036,6 @@ func (t *testLock) TestFetchTableInfo(c *C) { query = fmt.Sprintf("SELECT table_info FROM `%s`.`%s` WHERE id = \\? AND cp_schema = \\? AND cp_table = \\?", meta, cputil.SyncerCheckpoint(task)) ) - mock := conn.InitMockDB(c) - baseDB, err := conn.DefaultDBProvider.Apply(&config.DBConfig{}) - c.Assert(err, IsNil) - // nil downstream meta l := NewLock(etcdTestCli, ID, task, downSchema, downTable, schemacmp.Encode(ti0), tts, nil) ti, err := l.FetchTableInfos(task, source, schema, tbls[0]) @@ -2047,14 +2043,18 @@ func (t *testLock) TestFetchTableInfo(c *C) { c.Assert(ti, IsNil) // table info not exist - l = NewLock(etcdTestCli, ID, task, downSchema, downTable, schemacmp.Encode(ti0), tts, &DownstreamMeta{db: baseDB, meta: meta}) + l = NewLock(etcdTestCli, ID, task, downSchema, downTable, schemacmp.Encode(ti0), tts, &DownstreamMeta{dbConfig: &config.DBConfig{}, meta: meta}) + conn.DefaultDBProvider = &conn.DefaultDBProviderImpl{} + mock := conn.InitMockDB(c) mock.ExpectQuery(query).WithArgs(source, schema, tbls[0]).WillReturnRows(sqlmock.NewRows([]string{"table_info"})) ti, err = l.FetchTableInfos(task, source, schema, tbls[0]) c.Assert(terror.ErrDBExecuteFailed.Equal(err), IsTrue) c.Assert(ti, IsNil) // null table info - l = NewLock(etcdTestCli, ID, task, downSchema, downTable, schemacmp.Encode(ti0), tts, &DownstreamMeta{db: baseDB, meta: meta}) + l = NewLock(etcdTestCli, ID, task, downSchema, downTable, schemacmp.Encode(ti0), tts, &DownstreamMeta{dbConfig: &config.DBConfig{}, meta: meta}) + conn.DefaultDBProvider = &conn.DefaultDBProviderImpl{} + mock = conn.InitMockDB(c) mock.ExpectQuery(query).WithArgs(source, schema, tbls[0]).WillReturnRows(sqlmock.NewRows([]string{"table_info"}).AddRow("null")) ti, err = l.FetchTableInfos(task, source, schema, tbls[0]) c.Assert(terror.ErrMasterOptimisticDownstreamMetaNotFound.Equal(err), IsTrue) @@ -2063,6 +2063,8 @@ func (t *testLock) TestFetchTableInfo(c *C) { // succeed tiBytes, err := json.Marshal(ti0) c.Assert(err, IsNil) + conn.DefaultDBProvider = &conn.DefaultDBProviderImpl{} + mock = conn.InitMockDB(c) mock.ExpectQuery(query).WithArgs(source, schema, tbls[0]).WillReturnRows(sqlmock.NewRows([]string{"table_info"}).AddRow(tiBytes)) ti, err = l.FetchTableInfos(task, source, schema, tbls[0]) c.Assert(err, IsNil) From 58f25f83076bbdb9f0d0ed48f55af8068e984a84 Mon Sep 17 00:00:00 2001 From: gmhdbjd Date: Thu, 20 Jan 2022 16:45:54 +0800 Subject: [PATCH 27/31] address comment --- dm/syncer/checkpoint.go | 8 ++------ 1 file changed, 2 insertions(+), 6 deletions(-) diff --git a/dm/syncer/checkpoint.go b/dm/syncer/checkpoint.go index f0e3136bb7d..d8580426019 100644 --- a/dm/syncer/checkpoint.go +++ b/dm/syncer/checkpoint.go @@ -732,7 +732,7 @@ func (cp *RemoteCheckPoint) FlushPointsWithTableInfos(tctx *tcontext.Context, ta } // create new point if point == nil { - cp.saveTablePoint(table, cp.globalPoint.MySQLLocation(), nil) + cp.saveTablePoint(table, cp.globalPoint.MySQLLocation(), ti) point = cp.points[sourceSchema][sourceTable] } tiBytes, err := json.Marshal(ti) @@ -753,11 +753,7 @@ func (cp *RemoteCheckPoint) FlushPointsWithTableInfos(tctx *tcontext.Context, ta return err } - for idx, point := range points { - err = point.save(point.savedPoint.location, tis[i+idx]) - if err != nil { - return err - } + for _, point := range points { point.flush() } } From 1a8d47235327b18f0c0c66af5a422a26fc40938d Mon Sep 17 00:00:00 2001 From: gmhdbjd Date: Tue, 25 Jan 2022 13:51:23 +0800 Subject: [PATCH 28/31] address comment --- dm/dm/master/scheduler/scheduler.go | 2 +- dm/dm/master/shardddl/optimist.go | 2 +- dm/pkg/shardddl/optimism/keeper.go | 3 ++- dm/syncer/syncer.go | 16 +++++++--------- dm/tests/shardddl_optimistic/run.sh | 1 - 5 files changed, 11 insertions(+), 13 deletions(-) diff --git a/dm/dm/master/scheduler/scheduler.go b/dm/dm/master/scheduler/scheduler.go index 9b6e284f7a9..c870887272e 100644 --- a/dm/dm/master/scheduler/scheduler.go +++ b/dm/dm/master/scheduler/scheduler.go @@ -985,7 +985,7 @@ func (s *Scheduler) getSubTaskCfgByTaskSource(task, source string) *config.SubTa return &clone } -// GetDownstreamMetaDBCfgByTask gets downstream db config and meta config by task name. +// GetDownstreamMetaByTask gets downstream db config and meta config by task name. func (s *Scheduler) GetDownstreamMetaByTask(task string) (*config.DBConfig, string) { v, ok := s.subTaskCfgs.Load(task) if !ok { diff --git a/dm/dm/master/shardddl/optimist.go b/dm/dm/master/shardddl/optimist.go index cfb571047a8..58a899b4c05 100644 --- a/dm/dm/master/shardddl/optimist.go +++ b/dm/dm/master/shardddl/optimist.go @@ -161,7 +161,7 @@ func (o *Optimist) ShowLocks(task string, sources []string) []*pb.DDLLock { return ret } -// RemoveMetaData removes meta data for a specified task +// RemoveMetaDataWithTask removes meta data for a specified task // NOTE: this function can only be used when the specified task is not running. // This function only be used when --remove-meta or stop-task // NOTE: For stop-task, we still delete drop columns in etcd though user may restart the task again later. diff --git a/dm/pkg/shardddl/optimism/keeper.go b/dm/pkg/shardddl/optimism/keeper.go index d380d43e130..b18ba831faa 100644 --- a/dm/pkg/shardddl/optimism/keeper.go +++ b/dm/pkg/shardddl/optimism/keeper.go @@ -41,7 +41,8 @@ type LockKeeper struct { downstreamMetaMap map[string]*DownstreamMeta getDownstreamMetaFunc func(string) (*config.DBConfig, string) - dropColumns map[string]map[string]map[string]map[string]map[string]DropColumnStage + // column name -> source -> upSchema -> upTable -> int + dropColumns map[string]map[string]map[string]map[string]map[string]DropColumnStage } // NewLockKeeper creates a new LockKeeper instance. diff --git a/dm/syncer/syncer.go b/dm/syncer/syncer.go index 362ff483b27..4d30bdf1784 100644 --- a/dm/syncer/syncer.go +++ b/dm/syncer/syncer.go @@ -1480,15 +1480,13 @@ func (s *Syncer) Run(ctx context.Context) (err error) { if err != nil { return err } - if fresh { - if s.cfg.Mode == config.ModeAll { - delLoadTask = true - flushCheckpoint = true - err = s.loadTableStructureFromDump(ctx) - if err != nil { - tctx.L().Warn("error happened when load table structure from dump files", zap.Error(err)) - cleanDumpFile = false - } + if fresh && s.cfg.Mode == config.ModeAll { + delLoadTask = true + flushCheckpoint = true + err = s.loadTableStructureFromDump(ctx) + if err != nil { + tctx.L().Warn("error happened when load table structure from dump files", zap.Error(err)) + cleanDumpFile = false } if s.cfg.ShardMode == config.ShardOptimistic { s.flushOptimisticTableInfos(tctx) diff --git a/dm/tests/shardddl_optimistic/run.sh b/dm/tests/shardddl_optimistic/run.sh index e097956f148..44b98a188bc 100644 --- a/dm/tests/shardddl_optimistic/run.sh +++ b/dm/tests/shardddl_optimistic/run.sh @@ -391,7 +391,6 @@ function DM_UPDATE_BA_ROUTE_CASE() { "stop-task test" \ "\"result\": true" 3 - cp $cur/conf/double-source-optimistic.yaml $WORK_DIR/task.yaml sed 's/do-dbs: \["shardddl1","shardddl2"\]/do-dbs: \["shardddl1"\]/g' $cur/conf/double-source-optimistic.yaml >$WORK_DIR/task.yaml # source1: db1.tb1(id,new_col1,new_col3) From c9a821c2c23f45e5182d3e230e8fbc22ba345fde Mon Sep 17 00:00:00 2001 From: GMHDBJD <35025882+GMHDBJD@users.noreply.github.com> Date: Wed, 26 Jan 2022 14:17:50 +0800 Subject: [PATCH 29/31] Update dm/pkg/shardddl/optimism/keeper.go Co-authored-by: lance6716 --- dm/pkg/shardddl/optimism/keeper.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dm/pkg/shardddl/optimism/keeper.go b/dm/pkg/shardddl/optimism/keeper.go index b18ba831faa..f1f0d3dc57c 100644 --- a/dm/pkg/shardddl/optimism/keeper.go +++ b/dm/pkg/shardddl/optimism/keeper.go @@ -41,7 +41,7 @@ type LockKeeper struct { downstreamMetaMap map[string]*DownstreamMeta getDownstreamMetaFunc func(string) (*config.DBConfig, string) - // column name -> source -> upSchema -> upTable -> int + // lockID -> column name -> source -> upSchema -> upTable -> int dropColumns map[string]map[string]map[string]map[string]map[string]DropColumnStage } From 5afac751bf77e9a2216b79fb9aa8ae73572e66ea Mon Sep 17 00:00:00 2001 From: gmhdbjd Date: Wed, 26 Jan 2022 20:44:02 +0800 Subject: [PATCH 30/31] address comment --- dm/syncer/checkpoint.go | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/dm/syncer/checkpoint.go b/dm/syncer/checkpoint.go index 15459b40ae7..e1e2497a32f 100644 --- a/dm/syncer/checkpoint.go +++ b/dm/syncer/checkpoint.go @@ -62,6 +62,7 @@ var ( globalCpSchema = "" // global checkpoint's cp_schema globalCpTable = "" // global checkpoint's cp_table maxCheckPointTimeout = "1m" + batchFlushPoints = 100 ) type tablePoint struct { @@ -722,16 +723,15 @@ func (cp *RemoteCheckPoint) FlushPointsWithTableInfos(tctx *tcontext.Context, ta return errors.Errorf("the length of the tables is not equal to the length of the table infos, left: %d, right: %d", len(tables), len(tis)) } - batch := 100 - for i := 0; i < len(tables); i += batch { - end := i + batch + for i := 0; i < len(tables); i += batchFlushPoints { + end := i + batchFlushPoints if end > len(tables) { end = len(tables) } - sqls := make([]string, 0, batch) - args := make([][]interface{}, 0, batch) - points := make([]*binlogPoint, 0, batch) + sqls := make([]string, 0, batchFlushPoints) + args := make([][]interface{}, 0, batchFlushPoints) + points := make([]*binlogPoint, 0, batchFlushPoints) for j := i; j < end; j++ { table := tables[j] ti := tis[j] From 41cf696a23d1041a75b4cfc70d05f1a4e8cd0c41 Mon Sep 17 00:00:00 2001 From: gmhdbjd Date: Mon, 7 Feb 2022 18:05:40 +0800 Subject: [PATCH 31/31] address comment --- dm/tests/shardddl_optimistic/run.sh | 58 ++--------------------------- 1 file changed, 4 insertions(+), 54 deletions(-) diff --git a/dm/tests/shardddl_optimistic/run.sh b/dm/tests/shardddl_optimistic/run.sh index 44b98a188bc..09acecfa9bb 100644 --- a/dm/tests/shardddl_optimistic/run.sh +++ b/dm/tests/shardddl_optimistic/run.sh @@ -231,12 +231,16 @@ function DM_RESTART_TASK_MASTER_WORKER() { function random_restart() { mod=$(($RANDOM % 4)) if [[ "$mod" == "0" ]]; then + echo "restart master" restart_master elif [[ "$mod" == "1" ]]; then + echo "restart worker1" restart_worker1 elif [[ "$mod" == "2" ]]; then + echo "restart worker2" restart_worker2 else + echo "restart task" restart_task $cur/conf/double-source-optimistic.yaml fi } @@ -295,60 +299,6 @@ function DM_STOP_TASK_FOR_A_SOURCE_CASE() { check_sync_diff $WORK_DIR $cur/conf/diff_config.toml } -function DM_STOP_TASK_FOR_A_SOURCE_CASE() { - run_sql_source1 "insert into ${shardddl1}.${tb1} values(1);" - run_sql_source1 "insert into ${shardddl1}.${tb2} values(2);" - run_sql_source2 "insert into ${shardddl1}.${tb1} values(3);" - run_sql_source2 "insert into ${shardddl1}.${tb2} values(4);" - - run_sql_source1 "alter table ${shardddl1}.${tb1} add column b varchar(10);" - run_sql_source1 "alter table ${shardddl1}.${tb2} add column b varchar(10);" - run_sql_source2 "alter table ${shardddl1}.${tb1} add column b varchar(10);" - run_sql_source2 "alter table ${shardddl1}.${tb2} add column b varchar(10);" - - run_sql_source1 "insert into ${shardddl1}.${tb1} values(5,'aaa');" - run_sql_source1 "insert into ${shardddl1}.${tb2} values(6,'bbb');" - run_sql_source2 "insert into ${shardddl1}.${tb1} values(7,'ccc');" - run_sql_source2 "insert into ${shardddl1}.${tb2} values(8,'ddd');" - - run_sql_source1 "alter table ${shardddl1}.${tb1} add column c text;" - run_sql_source1 "insert into ${shardddl1}.${tb1} values(9,'eee','eee');" - run_sql_source1 "alter table ${shardddl1}.${tb2} drop column b;" - run_sql_source1 "insert into ${shardddl1}.${tb2} values(10);" - run_sql_source2 "alter table ${shardddl1}.${tb1} add column c text;" - run_sql_source2 "insert into ${shardddl1}.${tb1} values(11,'fff','fff');" - run_sql_source2 "alter table ${shardddl1}.${tb2} drop column b;" - run_sql_source2 "insert into ${shardddl1}.${tb2} values(12);" - - run_sql_tidb_with_retry "select count(1) from ${shardddl}.${tb}" "count(1): 12" - - run_dm_ctl $WORK_DIR "127.0.0.1:$MASTER_PORT" \ - "stop-task test -s mysql-replica-02" \ - "\"result\": true" 2 - - run_sql_source1 "alter table ${shardddl1}.${tb1} drop column b;" - run_sql_source1 "insert into ${shardddl1}.${tb1} values(13,'ggg');" - run_sql_source1 "alter table ${shardddl1}.${tb2} add column c text;" - run_sql_source1 "insert into ${shardddl1}.${tb2} values(14,'hhh');" - - run_sql_tidb_with_retry "select count(1) from ${shardddl}.${tb}" "count(1): 14" - run_sql_tidb_with_retry "select count(1) from INFORMATION_SCHEMA.COLUMNS where TABLE_SCHEMA='${shardddl}' AND TABLE_NAME='${tb}';" \ - "count(1): 2" - - run_dm_ctl $WORK_DIR "127.0.0.1:$MASTER_PORT" \ - "start-task $cur/conf/double-source-optimistic.yaml -s mysql-replica-02" \ - "\"result\": true" 2 - - run_sql_source1 "insert into ${shardddl1}.${tb1} values(15,'iii');" - run_sql_source1 "insert into ${shardddl1}.${tb1} values(16,'jjj');" - run_sql_source2 "alter table ${shardddl1}.${tb1} drop column b;" - run_sql_source2 "insert into ${shardddl1}.${tb1} values(17,'kkk');" - run_sql_source2 "alter table ${shardddl1}.${tb2} add column c text;" - run_sql_source2 "insert into ${shardddl1}.${tb2} values(18,'lll');" - - check_sync_diff $WORK_DIR $cur/conf/diff_config.toml -} - function DM_STOP_TASK_FOR_A_SOURCE() { run_case STOP_TASK_FOR_A_SOURCE "double-source-optimistic" "init_table 111 112 211 212" "clean_table" "optimistic" }