From 5c7d2388fef19f49cdc517a03acd303649fa4e5a Mon Sep 17 00:00:00 2001 From: GMHDBJD <35025882+GMHDBJD@users.noreply.github.com> Date: Mon, 7 Feb 2022 19:05:35 +0800 Subject: [PATCH 1/2] Optimistic: support start task with inconsistent upstream table schema (#3903) close pingcap/tiflow#3629 --- Makefile | 13 +- dm/_utils/terror_gen/errors_release.txt | 1 + dm/dm/common/common.go | 5 +- dm/dm/common/common_test.go | 23 - dm/dm/master/scheduler/scheduler.go | 13 + dm/dm/master/scheduler/scheduler_test.go | 16 + dm/dm/master/server.go | 16 +- dm/dm/master/server_test.go | 4 +- dm/dm/master/shardddl/optimist.go | 233 ++++------ dm/dm/master/shardddl/optimist_test.go | 86 +--- dm/errors.toml | 6 + dm/pkg/shardddl/optimism/info.go | 7 +- dm/pkg/shardddl/optimism/info_test.go | 2 +- dm/pkg/shardddl/optimism/keeper.go | 160 ++++--- dm/pkg/shardddl/optimism/keeper_test.go | 174 ++++--- dm/pkg/shardddl/optimism/lock.go | 160 ++++++- dm/pkg/shardddl/optimism/lock_test.go | 194 +++++++- dm/pkg/shardddl/optimism/ops.go | 34 +- dm/pkg/shardddl/optimism/ops_test.go | 18 +- dm/pkg/shardddl/optimism/schema.go | 151 ------- dm/pkg/shardddl/optimism/schema_test.go | 109 ----- dm/pkg/terror/error_list.go | 2 + dm/syncer/checkpoint.go | 88 ++-- dm/syncer/optimist.go | 15 - dm/syncer/schema.go | 2 +- dm/syncer/shardddl/optimist.go | 38 +- dm/syncer/shardddl/optimist_test.go | 47 +- dm/syncer/syncer.go | 39 +- dm/tests/_utils/shardddl_lib.sh | 36 ++ dm/tests/_utils/test_prepare | 2 +- dm/tests/downstream_more_column/run.sh | 2 +- dm/tests/others_integration_2.txt | 1 + dm/tests/sequence_sharding_optimistic/run.sh | 4 +- dm/tests/shardddl1/run.sh | 9 +- .../shardddl_optimistic/conf/diff_config.toml | 44 ++ .../shardddl_optimistic/conf/dm-master.toml | 10 + .../shardddl_optimistic/conf/dm-worker1.toml | 2 + .../shardddl_optimistic/conf/dm-worker2.toml | 2 + .../conf/double-source-optimistic-incr.yaml | 63 +++ .../conf/double-source-optimistic.yaml | 58 +++ .../shardddl_optimistic/conf/source1.yaml | 11 + .../shardddl_optimistic/conf/source2.yaml | 11 + dm/tests/shardddl_optimistic/run.sh | 426 ++++++++++++++++++ 43 files changed, 1528 insertions(+), 809 deletions(-) delete mode 100644 dm/pkg/shardddl/optimism/schema.go delete mode 100644 dm/pkg/shardddl/optimism/schema_test.go 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-incr.yaml 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/Makefile b/Makefile index 2b12c40d678..c338ba4dac1 100644 --- a/Makefile +++ b/Makefile @@ -277,13 +277,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/tiflow/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/_utils/terror_gen/errors_release.txt b/dm/_utils/terror_gen/errors_release.txt index 9e115d4ce3a..2697655c99c 100644 --- a/dm/_utils/terror_gen/errors_release.txt +++ b/dm/_utils/terror_gen/errors_release.txt @@ -395,6 +395,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 4464dbb5b2b..a447c8ee373 100644 --- a/dm/dm/common/common.go +++ b/dm/dm/common/common.go @@ -78,7 +78,8 @@ 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/") + // 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 // If we don't identify different upstream tables, we may report an error for tb2 in the following case. @@ -112,8 +113,6 @@ func keyAdapterKeysLen(s KeyAdapter) int { ShardDDLPessimismInfoKeyAdapter, ShardDDLPessimismOperationKeyAdapter, ShardDDLOptimismSourceTablesKeyAdapter, LoadTaskKeyAdapter, TaskCliArgsKeyAdapter: 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 b2669ca5cc9..0dbebb690d3 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,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, @@ -108,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,18 +110,6 @@ func (t *testCommon) TestEncodeAsPrefix(c *C) { _, err := ca.adapter.Decode(encKey) 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/scheduler/scheduler.go b/dm/dm/master/scheduler/scheduler.go index e5b899a4837..c0784c57392 100644 --- a/dm/dm/master/scheduler/scheduler.go +++ b/dm/dm/master/scheduler/scheduler.go @@ -1020,6 +1020,19 @@ func (s *Scheduler) getSubTaskCfgByTaskSource(task, source string) *config.SubTa return &clone } +// 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 { + 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/scheduler/scheduler_test.go b/dm/dm/master/scheduler/scheduler_test.go index f045456407b..e9eb1627e6d 100644 --- a/dm/dm/master/scheduler/scheduler_test.go +++ b/dm/dm/master/scheduler/scheduler_test.go @@ -254,12 +254,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) @@ -629,6 +632,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/dm/master/server.go b/dm/dm/master/server.go index 4e24e10cfb9..48691c15c8e 100644 --- a/dm/dm/master/server.go +++ b/dm/dm/master/server.go @@ -132,7 +132,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) @@ -590,6 +590,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 } @@ -1558,7 +1570,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/server_test.go b/dm/dm/master/server_test.go index 73d79b41275..1438246ef44 100644 --- a/dm/dm/master/server_test.go +++ b/dm/dm/master/server_test.go @@ -960,7 +960,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"} @@ -1045,7 +1045,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 bab8ef7e86d..58a899b4c05 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, getDownstreamMetaFunc 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(getDownstreamMetaFunc), tk: optimism.NewTableKeeper(), } } @@ -162,9 +161,11 @@ 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. -func (o *Optimist) RemoveMetaData(task string) error { +// 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() if o.closed { @@ -185,10 +186,43 @@ func (o *Optimist) RemoveMetaData(task string) error { o.lk.RemoveLock(op.ID) } + o.lk.RemoveDownstreamMeta(task) 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() + } + + 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 + 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) + } + } + + 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)) + // clear meta data in etcd + _, err := optimism.DeleteInfosOperationsTablesByTaskAndSource(o.cli, task, sources, dropColumns) return err } @@ -250,26 +284,22 @@ 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. // 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. o.logger.Error("fail to recover locks", log.ShortError(err)) } + o.lk.SetDropColumns(nil) return revSource, revInfo, revOperation, nil } @@ -295,96 +325,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 +340,18 @@ 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 + } // 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 +370,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 +545,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 @@ -649,18 +601,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) @@ -761,8 +701,7 @@ 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) + 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 d8226ee996c..b3e6c84e13e 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" @@ -43,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 { @@ -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) @@ -197,14 +196,14 @@ 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 ( 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 = 5 * time.Second logger = log.L() - o = NewOptimist(&logger) + o = NewOptimist(&logger, getDownstreamMeta) task = "task-test-optimist" source1 = "mysql-replica-1" downSchema = "foo" @@ -724,7 +723,7 @@ func (t *testOptimist) TestOptimistLockMultipleTarget(c *C) { waitTime = 100 * time.Millisecond watchTimeout = 5 * time.Second logger = log.L() - o = NewOptimist(&logger) + o = NewOptimist(&logger, getDownstreamMeta) task = "test-optimist-lock-multiple-target" source = "mysql-replica-1" upSchema = "foo" @@ -919,7 +918,7 @@ func (t *testOptimist) TestOptimistInitSchema(c *C) { waitTime = 100 * time.Millisecond watchTimeout = 5 * time.Second logger = log.L() - o = NewOptimist(&logger) + o = NewOptimist(&logger, getDownstreamMeta) task = "test-optimist-init-schema" source = "mysql-replica-1" upSchema = "foo" @@ -954,11 +953,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) @@ -967,11 +961,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) @@ -1011,11 +1000,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) @@ -1023,16 +1007,11 @@ 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) { defer func() { - c.Assert(optimism.ClearTestInfoOperationSchema(cli), IsNil) + c.Assert(optimism.ClearTestInfoOperationColumn(cli), IsNil) }() var ( @@ -1104,7 +1083,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" @@ -1146,19 +1125,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) { @@ -1166,7 +1132,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" @@ -1182,11 +1148,10 @@ 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}) ) ctx, cancel := context.WithCancel(context.Background()) @@ -1209,21 +1174,8 @@ 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) { + return nil, "" } diff --git a/dm/errors.toml b/dm/errors.toml index da1ebb27a3e..96bab495cf8 100644 --- a/dm/errors.toml +++ b/dm/errors.toml @@ -2380,6 +2380,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 eb0179a7a58..bdb43c067ce 100644 --- a/dm/pkg/shardddl/optimism/info.go +++ b/dm/pkg/shardddl/optimism/info.go @@ -298,15 +298,14 @@ func deleteInfoOp(info Info) clientv3.Op { info.Task, info.Source, info.UpSchema, info.UpTable)) } -// 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()) - 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/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 b88493e78f1..f1f0d3dc57c 100644 --- a/dm/pkg/shardddl/optimism/keeper.go +++ b/dm/pkg/shardddl/optimism/keeper.go @@ -20,79 +20,63 @@ import ( "github.com/pingcap/tidb-tools/pkg/schemacmp" "go.etcd.io/etcd/clientv3" + "github.com/pingcap/tiflow/dm/dm/config" + "github.com/pingcap/tiflow/dm/pkg/log" "github.com/pingcap/tiflow/dm/pkg/terror" "github.com/pingcap/tiflow/dm/pkg/utils" ) +// DownstreamMeta used to fetch table info from downstream. +type DownstreamMeta struct { + dbConfig *config.DBConfig + 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. +// But the drop columns should be persistent. type LockKeeper struct { mu sync.RWMutex locks map[string]*Lock // lockID -> Lock + + downstreamMetaMap map[string]*DownstreamMeta + getDownstreamMetaFunc func(string) (*config.DBConfig, string) + // lockID -> column name -> source -> upSchema -> upTable -> int + dropColumns map[string]map[string]map[string]map[string]map[string]DropColumnStage } // NewLockKeeper creates a new LockKeeper instance. -func NewLockKeeper() *LockKeeper { +func NewLockKeeper(getDownstreamMetaFunc func(string) (*config.DBConfig, string)) *LockKeeper { return &LockKeeper{ - locks: make(map[string]*Lock), + locks: make(map[string]*Lock), + downstreamMetaMap: make(map[string]*DownstreamMeta), + getDownstreamMetaFunc: getDownstreamMetaFunc, } } -// 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) { + lk.dropColumns = dropColumns +} + +// getDownstreamMeta gets and cached downstream meta. +func (lk *LockKeeper) getDownstreamMeta(task string) (*DownstreamMeta, error) { + if downstreamMeta, ok := lk.downstreamMetaMap[task]; ok { + return downstreamMeta, nil } - // 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 - } - } - } + dbConfig, meta := lk.getDownstreamMetaFunc(task) + if dbConfig == nil { + return nil, terror.ErrMasterOptimisticDownstreamMetaNotFound.Generate(task) } + 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) { + delete(lk.downstreamMetaMap, task) } // TrySync tries to sync the lock. @@ -111,8 +95,20 @@ 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, only when recover locks + if lk.dropColumns != nil { + if cols, ok := lk.dropColumns[lockID]; ok { + l.columns = cols + } + } } newDDLs, cols, err := l.TrySync(info, tts) @@ -143,6 +139,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)) @@ -166,6 +177,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. @@ -245,6 +257,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 { @@ -276,6 +312,20 @@ func (tk *TableKeeper) RemoveTableByTask(task string) bool { return true } +// 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 + } + + for _, source := range sources { + delete(tk.tables[task], source) + } +} + // 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/keeper_test.go b/dm/pkg/shardddl/optimism/keeper_test.go index 6b686872c66..5aefe4173c2 100644 --- a/dm/pkg/shardddl/optimism/keeper_test.go +++ b/dm/pkg/shardddl/optimism/keeper_test.go @@ -17,13 +17,14 @@ 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/tiflow/dm/pkg/utils" + "github.com/pingcap/tiflow/dm/dm/config" + "github.com/pingcap/tiflow/dm/pkg/conn" + "github.com/pingcap/tiflow/dm/pkg/terror" ) type testKeeper struct{} @@ -41,7 +42,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" @@ -81,6 +82,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) @@ -110,6 +118,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) @@ -134,7 +149,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" @@ -277,6 +292,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) @@ -288,6 +308,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) @@ -300,11 +325,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) @@ -316,6 +351,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) @@ -346,6 +386,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) { @@ -414,72 +467,65 @@ func (t *testKeeper) TestTargetTablesForTask(c *C) { }) } -func (t *testKeeper) TestRebuildLocksAndTables(c *C) { - defer clearTestInfoOperation(c) +func getDownstreamMeta(string) (*config.DBConfig, string) { + return nil, "" +} + +func (t *testKeeper) TestGetDownstreamMeta(c *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{}{}}}), + 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, "" } + } - lockID = utils.GenDDLLockID(task, downSchema, downTable) + conn.InitMockDB(c) + lk := NewLockKeeper(getDownstreamMetaFunc) + c.Assert(lk.downstreamMetaMap, HasLen, 0) - 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, - } - ) + downstreamMeta, err := lk.getDownstreamMeta(task3) + c.Assert(downstreamMeta, IsNil) + c.Assert(terror.ErrMasterOptimisticDownstreamMetaNotFound.Equal(err), IsTrue) - 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)) + 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(cmp, Equals, 0) - cmp, err = lock.tables[source1][upSchema][upTable].Compare(schemacmp.Encode(ti0)) + c.Assert(lk.downstreamMetaMap, HasLen, 1) + c.Assert(downstreamMeta, Equals, downstreamMeta2) + + downstreamMeta3, err := lk.getDownstreamMeta(task2) c.Assert(err, IsNil) - c.Assert(cmp, Equals, 0) - cmp, err = lock.tables[source2][upSchema][upTable].Compare(schemacmp.Encode(ti2)) + 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(cmp, Equals, 0) - c.Assert(lock.columns, DeepEquals, colm[lockID]) + 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) } diff --git a/dm/pkg/shardddl/optimism/lock.go b/dm/pkg/shardddl/optimism/lock.go index 7b9740ce126..d1b207a9f48 100644 --- a/dm/pkg/shardddl/optimism/lock.go +++ b/dm/pkg/shardddl/optimism/lock.go @@ -14,16 +14,23 @@ 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/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" ) @@ -75,30 +82,94 @@ 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) - + // pre join because tables may have different schema at the beginning + l.joinTable() 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 { + return nil, terror.ErrMasterOptimisticDownstreamMetaNotFound.Generate(task) + } + + 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() + + 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, terror.ErrDBExecuteFailed.Delegate(row.Err(), query) + } + var tiBytes []byte + if err := row.Scan(&tiBytes); err != nil { + return nil, terror.ErrDBExecuteFailed.Delegate(err, query) + } + 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 +} + +// 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. @@ -398,6 +469,56 @@ 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() + + // 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 + } + } + } + + 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() + 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, @@ -533,6 +654,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 { @@ -548,7 +670,15 @@ 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) + 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 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 1ed4ebc64a0..5960b139491 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" @@ -83,7 +89,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 +391,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 +475,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 +546,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 +599,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 +642,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 +755,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 +920,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 +1042,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 +1230,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 +1373,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: { @@ -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`" @@ -1459,7 +1562,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 +1670,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 +1712,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 +1760,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 +1956,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: { @@ -1909,3 +2012,62 @@ 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)) + ) + + // 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{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{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) + c.Assert(ti, IsNil) + + // 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) + c.Assert(mock.ExpectationsWereMet(), IsNil) + c.Assert(ti, DeepEquals, ti0) +} diff --git a/dm/pkg/shardddl/optimism/ops.go b/dm/pkg/shardddl/optimism/ops.go index 7cd13bace1d..28c92f3e495 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, schema InitSchema) (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,8 +61,7 @@ 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, deleteDroppedColumnsByLockOp(lockID)) resp, rev, err := etcdutil.DoOpsInOneCmpsTxnWithRetry(cli, cmps, opsDel, []clientv3.Op{}) if err != nil { return 0, false, err @@ -71,16 +69,36 @@ func DeleteInfosOperationsSchemaColumn(cli *clientv3.Client, infos []Info, ops [ return rev, resp.Succeeded, nil } -// DeleteInfosOperationsTablesSchemasByTask deletes the shard DDL infos and operations in etcd. -func DeleteInfosOperationsTablesSchemasByTask(cli *clientv3.Client, task string, lockIDSet map[string]struct{}) (int64, error) { +// 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())) 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())) } _, rev, err := etcdutil.DoOpsInOneTxnWithRetry(cli, opsDel...) return rev, err } + +// 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 sources. +func DeleteInfosOperationsTablesByTaskAndSource(cli *clientv3.Client, task string, sources []string, dropColumns map[string][]string) (int64, error) { + opsDel := make([]clientv3.Op, 0, 5) + 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/pkg/shardddl/optimism/ops_test.go b/dm/pkg/shardddl/optimism/ops_test.go index 55270f0d5e8..da140184268 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,15 +51,8 @@ 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}, is) + _, deleted, err := DeleteInfosOperationsColumns(etcdTestCli, []Info{info}, []Operation{op}, genDDLLockID(info)) c.Assert(err, IsNil) c.Assert(deleted, IsFalse) @@ -71,12 +63,9 @@ 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}, is) + _, deleted, err = DeleteInfosOperationsColumns(etcdTestCli, []Info{infoWithVer}, []Operation{op}, genDDLLockID(infoWithVer)) c.Assert(err, IsNil) c.Assert(deleted, IsTrue) @@ -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 c4f4167e4df..00000000000 --- a/dm/pkg/shardddl/optimism/schema.go +++ /dev/null @@ -1,151 +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" - "go.etcd.io/etcd/clientv3" - "go.etcd.io/etcd/clientv3/clientv3util" - - "github.com/pingcap/tiflow/dm/dm/common" - "github.com/pingcap/tiflow/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)) -} diff --git a/dm/pkg/shardddl/optimism/schema_test.go b/dm/pkg/shardddl/optimism/schema_test.go deleted file mode 100644 index f5f3ba582a4..00000000000 --- a/dm/pkg/shardddl/optimism/schema_test.go +++ /dev/null @@ -1,109 +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" - "github.com/pingcap/tidb/parser" - "github.com/pingcap/tidb/util/mock" -) - -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/pkg/terror/error_list.go b/dm/pkg/terror/error_list.go index 06c4c958ec2..6c95627d941 100644 --- a/dm/pkg/terror/error_list.go +++ b/dm/pkg/terror/error_list.go @@ -496,6 +496,7 @@ const ( codeMasterFailToImportFromV10x codeMasterInconsistentOptimistDDLsAndInfo codeMasterOptimisticTableInfobeforeNotExist + codeMasterOptimisticDownstreamMetaNotFound ) // DM-worker error code. @@ -1143,6 +1144,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 0873d607c27..833bde9f798 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/tiflow/dm/dm/config" "github.com/pingcap/tiflow/dm/pkg/binlog" "github.com/pingcap/tiflow/dm/pkg/conn" @@ -61,6 +62,7 @@ var ( globalCpSchema = "" // global checkpoint's cp_schema globalCpTable = "" // global checkpoint's cp_table maxCheckPointTimeout = "1m" + batchFlushPoints = 100 ) type tablePoint struct { @@ -245,8 +247,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 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 @@ -712,45 +714,63 @@ 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) + for i := 0; i < len(tables); i += batchFlushPoints { + end := i + batchFlushPoints + 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, batchFlushPoints) + args := make([][]interface{}, 0, batchFlushPoints) + points := make([]*binlogPoint, 0, batchFlushPoints) + for j := i; j < end; j++ { + table := tables[j] + ti := tis[j] + sourceSchema, sourceTable := table.Schema, table.Name + + 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(), ti) + point = cp.points[sourceSchema][sourceTable] + } + 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 _, point := range points { + point.flush() + } } - point.flush() - return nil } diff --git a/dm/syncer/optimist.go b/dm/syncer/optimist.go index 6fed273462c..575dcdb5473 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/schema.go b/dm/syncer/schema.go index 979d814783e..eca48fb41d0 100644 --- a/dm/syncer/schema.go +++ b/dm/syncer/schema.go @@ -138,7 +138,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 027626d3726..4b076e96008 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() @@ -161,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 fa616f51bae..7b2a76bfbed 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 { @@ -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) @@ -206,42 +212,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 763d90efefb..b1ac38f52da 100644 --- a/dm/syncer/syncer.go +++ b/dm/syncer/syncer.go @@ -735,14 +735,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) @@ -1492,7 +1484,7 @@ func (s *Syncer) Run(ctx context.Context) (err error) { if err != nil { return err } - if s.cfg.Mode == config.ModeAll && fresh { + if fresh && s.cfg.Mode == config.ModeAll { delLoadTask = true flushCheckpoint = true err = s.loadTableStructureFromDump(ctx) @@ -1500,7 +1492,12 @@ func (s *Syncer) Run(ctx context.Context) (err error) { tctx.L().Warn("error happened when load table structure from dump files", zap.Error(err)) cleanDumpFile = false } - } else { + if s.cfg.ShardMode == config.ShardOptimistic { + s.flushOptimisticTableInfos(tctx) + } + } + + if s.cfg.Mode == config.ModeIncrement || !fresh { cleanDumpFile = false } @@ -3172,7 +3169,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 { @@ -3764,3 +3761,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)) + } +} 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/_utils/test_prepare b/dm/tests/_utils/test_prepare index f268325da4e..b4ea29bf170 100644 --- a/dm/tests/_utils/test_prepare +++ b/dm/tests/_utils/test_prepare @@ -324,7 +324,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/others_integration_2.txt b/dm/tests/others_integration_2.txt index e6f69950689..676318c3ce2 100644 --- a/dm/tests/others_integration_2.txt +++ b/dm/tests/others_integration_2.txt @@ -10,4 +10,5 @@ openapi duplicate_event tracker_ignored_ddl extend_column +shardddl_optimistic gbk diff --git a/dm/tests/sequence_sharding_optimistic/run.sh b/dm/tests/sequence_sharding_optimistic/run.sh index 58ea6fd456c..20f5f5a0d38 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" \ diff --git a/dm/tests/shardddl1/run.sh b/dm/tests/shardddl1/run.sh index 224390e17dc..101b9d4e7f8 100644 --- a/dm/tests/shardddl1/run.sh +++ b/dm/tests/shardddl1/run.sh @@ -394,10 +394,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" @@ -406,7 +413,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" 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..458b3e124f0 --- /dev/null +++ b/dm/tests/shardddl_optimistic/conf/dm-master.toml @@ -0,0 +1,10 @@ +# 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" + +[experimental] +openapi = true 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-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/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..09acecfa9bb --- /dev/null +++ b/dm/tests/shardddl_optimistic/run.sh @@ -0,0 +1,426 @@ +#!/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 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_with_retry $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 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 + 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 +} + +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 + + 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 +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 b0f52af9c3d805b0d90d421c24ea8c42c49391a8 Mon Sep 17 00:00:00 2001 From: will & database <87208113+db-will@users.noreply.github.com> Date: Mon, 7 Feb 2022 06:57:34 -0500 Subject: [PATCH 2/2] syncer(dm): Fix async flush log message (#4500) ref pingcap/tiflow#4287 --- dm/syncer/syncer.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dm/syncer/syncer.go b/dm/syncer/syncer.go index b1ac38f52da..212a3fa9bb1 100644 --- a/dm/syncer/syncer.go +++ b/dm/syncer/syncer.go @@ -1218,7 +1218,7 @@ func (s *Syncer) afterFlushCheckpoint(task *checkpointFlushTask) error { s.tctx.L().Info("after async flushed checkpoint, gc stale causality keys", zap.Int64("flush job seq", task.asyncflushJob.flushSeq)) s.addJob(newGCJob(task.asyncflushJob.flushSeq)) } else { - s.tctx.L().Info("after async flushed checkpoint, gc all causality keys") + s.tctx.L().Info("after sync flushed checkpoint, gc all causality keys") s.addJob(newGCJob(math.MaxInt64)) }