diff --git a/.gitignore b/.gitignore index b6679237de..6b79b237e4 100644 --- a/.gitignore +++ b/.gitignore @@ -16,3 +16,4 @@ vendor tidb-slow.log /monitoring/dashboards/dm.json /monitoring/rules/dm_worker.rules.yml +mysql.*.log \ No newline at end of file diff --git a/Dockerfile b/Dockerfile index 39f66d49dc..e957383d2c 100644 --- a/Dockerfile +++ b/Dockerfile @@ -19,6 +19,8 @@ RUN GO111MODULE=on go mod download COPY . . +RUN apk update && apk add bash + RUN make dm-worker dm-master dmctl FROM alpine:3.10 diff --git a/_utils/terror_gen/errors_release.txt b/_utils/terror_gen/errors_release.txt index 1c8f20f246..d3d4535873 100644 --- a/_utils/terror_gen/errors_release.txt +++ b/_utils/terror_gen/errors_release.txt @@ -312,12 +312,13 @@ ErrSyncerUnitUpdateConfigInSharding,[code=36058:class=sync-unit:scope=internal:l ErrSyncerUnitExecWithNoBlockingDDL,[code=36059:class=sync-unit:scope=internal:level=high], "Message: process unit not waiting for sharding DDL to sync" ErrSyncerUnitGenBAList,[code=36060:class=sync-unit:scope=internal:level=high], "Message: generate block allow list, Workaround: Please check the `block-allow-list` config in task configuration file." ErrSyncerUnitHandleDDLFailed,[code=36061:class=sync-unit:scope=internal:level=high], "Message: fail to handle ddl job for %s" -ErrSyncerShardDDLConflict,[code=36062:class=sync-unit:scope=internal:level=high], "Message: fail to handle shard ddl %v in optimistic mode, because schema conflict detected, Workaround: Please use show-ddl-locks command for more details." +ErrSyncerShardDDLConflict,[code=36062:class=sync-unit:scope=internal:level=high], "Message: fail to handle shard ddl %v in optimistic mode, because schema conflict detected, conflict error: %s, Workaround: Please use show-ddl-locks command for more details." ErrSyncerFailpoint,[code=36063:class=sync-unit:scope=internal:level=low], "Message: failpoint specified error" ErrSyncerReplaceEvent,[code=36064:class=sync-unit:scope=internal:level=high] ErrSyncerOperatorNotExist,[code=36065:class=sync-unit:scope=internal:level=low], "Message: error operator not exist, position: %s" ErrSyncerReplaceEventNotExist,[code=36066:class=sync-unit:scope=internal:level=high], "Message: replace event not exist, location: %s" ErrSyncerParseDDL,[code=36067:class=sync-unit:scope=internal:level=high], "Message: parse DDL: %s, Workaround: Please confirm your DDL statement is correct and needed. For TiDB compatible DDL, see https://docs.pingcap.com/tidb/stable/mysql-compatibility#ddl. You can use `handle-error` command to skip or replace the DDL or add a binlog filter rule to ignore it if the DDL is not needed." +ErrSyncerUnsupportedStmt,[code=36068:class=sync-unit:scope=internal:level=high], "Message: `%s` statement not supported in %s mode" ErrMasterSQLOpNilRequest,[code=38001:class=dm-master:scope=internal:level=medium], "Message: nil request not valid" ErrMasterSQLOpNotSupport,[code=38002:class=dm-master:scope=internal:level=medium], "Message: op %s not supported" ErrMasterSQLOpWithoutSharding,[code=38003:class=dm-master:scope=internal:level=medium], "Message: operate request without --sharding specified not valid" @@ -372,6 +373,7 @@ ErrMasterTLSConfigNotValid,[code=38051:class=dm-master:scope=internal:level=high ErrMasterBoundChanging,[code=38052:class=dm-master:scope=internal:level=low], "Message: source bound is changed too frequently, last old bound %s:, new bound %s, Workaround: Please try again later" 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" 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/common/common.go b/dm/common/common.go index 8e00d2618d..78212a31b9 100644 --- a/dm/common/common.go +++ b/dm/common/common.go @@ -79,6 +79,15 @@ var ( // 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/") + // ShardDDLOptimismDroppedColumnsKeyAdapter is used to store the columns that are not fully dropped + // k/v: Encode(task-name, downstream-schema-name, downstream-table-name, column-name, source-id, upstream-schema-name, upstream-table-name) -> empty + // If we don't identify different upstream tables, we may report an error for tb2 in the following case. + // Time series: (+a/-a means add/drop column a) + // older ----------------> newer + // tb1: +a +b +c -c + // tb2: +a +b +c + // tb3: +a +b +c + ShardDDLOptimismDroppedColumnsKeyAdapter KeyAdapter = keyHexEncoderDecoder("/dm-master/shardddl-optimism/dropped-columns/") ) func keyAdapterKeysLen(s KeyAdapter) int { @@ -95,7 +104,8 @@ func keyAdapterKeysLen(s KeyAdapter) int { return 3 case ShardDDLOptimismInfoKeyAdapter, ShardDDLOptimismOperationKeyAdapter: return 4 - + case ShardDDLOptimismDroppedColumnsKeyAdapter: + return 7 } return -1 } diff --git a/dm/master/config.go b/dm/master/config.go index b4f122d3b9..10f7f71c88 100644 --- a/dm/master/config.go +++ b/dm/master/config.go @@ -37,11 +37,15 @@ import ( ) const ( - defaultRPCTimeout = "30s" - defaultNamePrefix = "dm-master" - defaultDataDirPrefix = "default" - defaultPeerUrls = "http://127.0.0.1:8291" - defaultInitialClusterState = embed.ClusterStateFlagNew + defaultRPCTimeout = "30s" + defaultNamePrefix = "dm-master" + defaultDataDirPrefix = "default" + defaultPeerUrls = "http://127.0.0.1:8291" + defaultInitialClusterState = embed.ClusterStateFlagNew + defaultAutoCompactionMode = "periodic" + defaultAutoCompactionRetention = "1h" + defaultQuotaBackendBytes = 2 * 1024 * 1024 * 1024 // 2GB + quotaBackendBytesLowerBound = 500 * 1024 * 1024 // 500MB ) var ( @@ -75,6 +79,9 @@ func NewConfig() *Config { fs.StringVar(&cfg.PeerUrls, "peer-urls", defaultPeerUrls, "URLs for peer traffic") fs.StringVar(&cfg.AdvertisePeerUrls, "advertise-peer-urls", "", `advertise URLs for peer traffic (default "${peer-urls}")`) fs.StringVar(&cfg.Join, "join", "", `join to an existing cluster (usage: cluster's "${master-addr}" list, e.g. "127.0.0.1:8261,127.0.0.1:18261"`) + fs.StringVar(&cfg.AutoCompactionMode, "auto-compaction-mode", defaultAutoCompactionMode, `etcd's auto-compaction-mode, either 'periodic' or 'revision'`) + fs.StringVar(&cfg.AutoCompactionRetention, "auto-compaction-retention", defaultAutoCompactionRetention, `etcd's auto-compaction-retention, accept values like '5h' or '5' (5 hours in 'periodic' mode or 5 revisions in 'revision')`) + fs.Int64Var(&cfg.QuotaBackendBytes, "quota-backend-bytes", defaultQuotaBackendBytes, `etcd's storage quota in bytes`) fs.StringVar(&cfg.SSLCA, "ssl-ca", "", "path of file that contains list of trusted SSL CAs for connection") fs.StringVar(&cfg.SSLCert, "ssl-cert", "", "path of file that contains X509 certificate in PEM format for connection") @@ -108,13 +115,16 @@ type Config struct { // etcd relative config items // NOTE: we use `MasterAddr` to generate `ClientUrls` and `AdvertiseClientUrls` // NOTE: more items will be add when adding leader election - Name string `toml:"name" json:"name"` - DataDir string `toml:"data-dir" json:"data-dir"` - PeerUrls string `toml:"peer-urls" json:"peer-urls"` - AdvertisePeerUrls string `toml:"advertise-peer-urls" json:"advertise-peer-urls"` - InitialCluster string `toml:"initial-cluster" json:"initial-cluster"` - InitialClusterState string `toml:"initial-cluster-state" json:"initial-cluster-state"` - Join string `toml:"join" json:"join"` // cluster's client address (endpoints), not peer address + Name string `toml:"name" json:"name"` + DataDir string `toml:"data-dir" json:"data-dir"` + PeerUrls string `toml:"peer-urls" json:"peer-urls"` + AdvertisePeerUrls string `toml:"advertise-peer-urls" json:"advertise-peer-urls"` + InitialCluster string `toml:"initial-cluster" json:"initial-cluster"` + InitialClusterState string `toml:"initial-cluster-state" json:"initial-cluster-state"` + Join string `toml:"join" json:"join"` // cluster's client address (endpoints), not peer address + AutoCompactionMode string `toml:"auto-compaction-mode" json:"auto-compaction-mode"` + AutoCompactionRetention string `toml:"auto-compaction-retention" json:"auto-compaction-retention"` + QuotaBackendBytes int64 `toml:"quota-backend-bytes" json:"quota-backend-bytes"` // directory path used to store source config files when upgrading from v1.0.x. // if this path set, DM-master leader will try to upgrade from v1.0.x to the current version. @@ -301,6 +311,13 @@ func (c *Config) adjust() error { c.Join = utils.WrapSchemes(c.Join, c.SSLCA != "") } + if c.QuotaBackendBytes < quotaBackendBytesLowerBound { + log.L().Warn("quota-backend-bytes is too low, will adjust it", + zap.Int64("from", c.QuotaBackendBytes), + zap.Int64("to", quotaBackendBytesLowerBound)) + c.QuotaBackendBytes = quotaBackendBytesLowerBound + } + return err } @@ -345,6 +362,9 @@ func (c *Config) genEmbedEtcdConfig(cfg *embed.Config) (*embed.Config, error) { cfg.InitialCluster = c.InitialCluster cfg.ClusterState = c.InitialClusterState + cfg.AutoCompactionMode = c.AutoCompactionMode + cfg.AutoCompactionRetention = c.AutoCompactionRetention + cfg.QuotaBackendBytes = c.QuotaBackendBytes err = cfg.Validate() // verify & trigger the builder if err != nil { diff --git a/dm/master/config_test.go b/dm/master/config_test.go index 0743536c0e..7c9748de9a 100644 --- a/dm/master/config_test.go +++ b/dm/master/config_test.go @@ -191,6 +191,9 @@ func (t *testConfigSuite) TestGenEmbedEtcdConfig(c *check.C) { c.Assert(etcdCfg.APUrls, check.DeepEquals, []url.URL{{Scheme: "http", Host: "127.0.0.1:8291"}}) c.Assert(etcdCfg.InitialCluster, check.DeepEquals, fmt.Sprintf("dm-master-%s=http://127.0.0.1:8291", hostname)) c.Assert(etcdCfg.ClusterState, check.Equals, embed.ClusterStateFlagExisting) + c.Assert(etcdCfg.AutoCompactionMode, check.Equals, "periodic") + c.Assert(etcdCfg.AutoCompactionRetention, check.Equals, "1h") + c.Assert(etcdCfg.QuotaBackendBytes, check.Equals, int64(2*1024*1024*1024)) cfg2 := *cfg1 cfg2.MasterAddr = "127.0.0.1\n:8261" diff --git a/dm/master/scheduler/scheduler.go b/dm/master/scheduler/scheduler.go index 96f981ceab..f5baf55420 100644 --- a/dm/master/scheduler/scheduler.go +++ b/dm/master/scheduler/scheduler.go @@ -118,7 +118,7 @@ type Scheduler struct { // expectant relay stages for sources, source ID -> stage. // add: - // - bound the source to a worker (at first time). + // - bound the source to a worker (at first time). // TODO: change this to add a relay-enabled source // - recover from etcd (calling `recoverSources`). // update: // - update stage by user request (calling `UpdateExpectRelayStage`). @@ -386,7 +386,7 @@ func (s *Scheduler) TransferSource(source, worker string) error { s.logger.Warn("in transfer source, found a free worker and not bound source, which should not happened", zap.String("source", source), zap.String("worker", worker)) - err := s.boundSourceToWorker(source, w) + err := s.boundSourceToWorker(source, w, s.sourceCfgs[source].EnableRelay) if err == nil { delete(s.unbounds, source) } @@ -412,7 +412,8 @@ func (s *Scheduler) TransferSource(source, worker string) error { failpoint.Inject("failToReplaceSourceBound", func(_ failpoint.Value) { failpoint.Return(errors.New("failToPutSourceBound")) }) - _, err := ha.ReplaceSourceBound(s.etcdCli, source, oldWorker.BaseInfo().Name, worker) + enableRelay := s.sourceCfgs[source].EnableRelay + _, err := ha.ReplaceSourceBound(s.etcdCli, source, oldWorker.BaseInfo().Name, worker, enableRelay) if err != nil { return err } @@ -779,10 +780,16 @@ func (s *Scheduler) UpdateExpectRelayStage(newStage pb.Stage, sources ...string) stages = make([]ha.Stage, 0, len(sources)) ) for _, source := range sources { - if currStage, ok := s.expectRelayStages[source]; !ok { + if _, ok := s.sourceCfgs[source]; !ok { notExistSourcesM[source] = struct{}{} - } else { + continue + } + + if currStage, ok := s.expectRelayStages[source]; ok { currStagesM[currStage.Expect.String()] = struct{}{} + } else { + s.logger.Warn("will write relay stage for a source that doesn't have previous stage", + zap.String("source", source)) } stages = append(stages, ha.NewRelayStage(newStage, source)) } @@ -1036,6 +1043,13 @@ func (s *Scheduler) recoverWorkersBounds(cli *clientv3.Client) (int64, error) { // 6. put trigger source bounds info to etcd to order dm-workers to start source if len(boundsToTrigger) > 0 { + for _, bound := range boundsToTrigger { + if s.sourceCfgs[bound.Source].EnableRelay { + if _, err2 := ha.PutRelayConfig(cli, bound.Source, bound.Worker); err2 != nil { + return 0, err2 + } + } + } _, err = ha.PutSourceBound(cli, boundsToTrigger...) if err != nil { return 0, nil @@ -1180,6 +1194,11 @@ func (s *Scheduler) handleWorkerOnline(ev ha.WorkerEvent, toLock bool) error { // 2. check whether is bound. if w.Stage() == WorkerBound { + if s.sourceCfgs[w.Bound().Source].EnableRelay { + if _, err := ha.PutRelayConfig(s.etcdCli, w.Bound().Source, w.Bound().Worker); err != nil { + return err + } + } // TODO: When dm-worker keepalive is broken, it will turn off its own running source // After keepalive is restored, this dm-worker should continue to run the previously bound source // So we PutSourceBound here to trigger dm-worker to get this event and start source again. @@ -1282,7 +1301,7 @@ func (s *Scheduler) tryBoundForWorker(w *Worker) (bounded bool, err error) { }() // 3. try to bound them. - err = s.boundSourceToWorker(source, w) + err = s.boundSourceToWorker(source, w, s.sourceCfgs[source].EnableRelay) if err != nil { return false, err } @@ -1323,7 +1342,7 @@ func (s *Scheduler) tryBoundForSource(source string) (bool, error) { } // 2. try to bound them. - err := s.boundSourceToWorker(source, worker) + err := s.boundSourceToWorker(source, worker, s.sourceCfgs[source].EnableRelay) if err != nil { return false, err } @@ -1332,23 +1351,31 @@ func (s *Scheduler) tryBoundForSource(source string) (bool, error) { // boundSourceToWorker bounds the source and worker together. // we should check the bound relationship of the source and the stage of the worker in the caller. -func (s *Scheduler) boundSourceToWorker(source string, w *Worker) error { +func (s *Scheduler) boundSourceToWorker(source string, w *Worker, enableRelay bool) error { // 1. put the bound relationship into etcd. var err error bound := ha.NewSourceBound(source, w.BaseInfo().Name) if _, ok := s.expectRelayStages[source]; ok { // the relay stage exists before, only put the bound relationship. + // TODO: we also put relay config for that worker temporary + _, err = ha.PutRelayConfig(s.etcdCli, bound.Source, bound.Worker) + if err != nil { + return err + } _, err = ha.PutSourceBound(s.etcdCli, bound) - } else { - // no relay stage exists before, create a `Runnng` stage and put it with the bound relationship. + } else if enableRelay { + // dont enable relay for it + // no relay stage exists before, create a `Running` stage and put it with the bound relationship. stage := ha.NewRelayStage(pb.Stage_Running, source) - _, err = ha.PutRelayStageSourceBound(s.etcdCli, stage, bound) + _, err = ha.PutRelayStageRelayConfigSourceBound(s.etcdCli, stage, bound) defer func() { if err == nil { // 1.1 if no error exist when returning, record the stage. s.expectRelayStages[source] = stage } }() + } else { + _, err = ha.PutSourceBound(s.etcdCli, bound) } if err != nil { return err diff --git a/dm/master/scheduler/scheduler_test.go b/dm/master/scheduler/scheduler_test.go index 992cdcda09..3c7362df2d 100644 --- a/dm/master/scheduler/scheduler_test.go +++ b/dm/master/scheduler/scheduler_test.go @@ -118,6 +118,7 @@ func (t *testScheduler) testSchedulerProgress(c *C, restart int) { ) c.Assert(sourceCfg1.LoadFromFile(sourceSampleFile), IsNil) sourceCfg1.SourceID = sourceID1 + sourceCfg1.EnableRelay = true sourceCfg2 := sourceCfg1 sourceCfg2.SourceID = sourceID2 @@ -1052,9 +1053,9 @@ func (t *testScheduler) TestTransferSource(c *C) { s.sourceCfgs[sourceID2] = config.SourceConfig{} worker1.ToFree() - c.Assert(s.boundSourceToWorker(sourceID1, worker1), IsNil) + c.Assert(s.boundSourceToWorker(sourceID1, worker1, false), IsNil) worker2.ToFree() - c.Assert(s.boundSourceToWorker(sourceID2, worker2), IsNil) + c.Assert(s.boundSourceToWorker(sourceID2, worker2, false), IsNil) c.Assert(s.bounds[sourceID1], DeepEquals, worker1) c.Assert(s.bounds[sourceID2], DeepEquals, worker2) diff --git a/dm/master/server_test.go b/dm/master/server_test.go index b56f1d9bf0..9cb0e1f87f 100644 --- a/dm/master/server_test.go +++ b/dm/master/server_test.go @@ -634,14 +634,15 @@ func (t *testMaster) TestStartTaskWithRemoveMeta(c *check.C) { tiBefore = createTableInfo(c, p, se, tblID, `CREATE TABLE bar (id INT PRIMARY KEY)`) tiAfter1 = createTableInfo(c, p, se, tblID, `CREATE TABLE bar (id INT PRIMARY KEY, c1 TEXT)`) info1 = optimism.NewInfo(taskName, sources[0], "foo-1", "bar-1", schema, table, DDLs1, tiBefore, []*model.TableInfo{tiAfter1}) - op1 = optimism.NewOperation(ID, taskName, sources[0], info1.UpSchema, info1.UpTable, DDLs1, optimism.ConflictNone, false) + op1 = optimism.NewOperation(ID, taskName, sources[0], info1.UpSchema, info1.UpTable, DDLs1, optimism.ConflictNone, "", false) ) + st1.AddTable("foo-1", "bar-1", schema, table) _, err = optimism.PutSourceTables(etcdTestCli, st1) c.Assert(err, check.IsNil) _, err = optimism.PutInfo(etcdTestCli, info1) c.Assert(err, check.IsNil) - _, succ, err = optimism.PutOperation(etcdTestCli, false, op1) + _, succ, err = optimism.PutOperation(etcdTestCli, false, op1, 0) c.Assert(succ, check.IsTrue) c.Assert(err, check.IsNil) @@ -1470,7 +1471,7 @@ func (t *testMaster) TestOfflineMember(c *check.C) { c.Assert(err, check.IsNil) c.Assert(listResp.Members, check.HasLen, 3) - // make sure s3 is not the leader, otherwise it will take some time to campain a new leader after close s3, and it may cause timeout + // make sure s3 is not the leader, otherwise it will take some time to campaign a new leader after close s3, and it may cause timeout c.Assert(utils.WaitSomething(20, 500*time.Millisecond, func() bool { _, leaderID, _, err = s1.election.LeaderInfo(ctx) if err != nil { @@ -1770,3 +1771,43 @@ func createTableInfo(c *check.C, p *parser.Parser, se sessionctx.Context, tableI } return info } + +type testEtcd struct { +} + +var _ = check.Suite(&testEtcd{}) + +func (t *testEtcd) TestEtcdAutoCompaction(c *check.C) { + cfg := NewConfig() + c.Assert(cfg.Parse([]string{"-config=./dm-master.toml"}), check.IsNil) + + cfg.DataDir = c.MkDir() + cfg.MasterAddr = tempurl.Alloc()[len("http://"):] + cfg.AutoCompactionRetention = "1s" + + ctx, cancel := context.WithCancel(context.Background()) + s := NewServer(cfg) + c.Assert(s.Start(ctx), check.IsNil) + + etcdCli, err := clientv3.New(clientv3.Config{ + Endpoints: []string{cfg.MasterAddr}, + }) + c.Assert(err, check.IsNil) + + for i := 0; i < 100; i++ { + _, err = etcdCli.Put(ctx, "key", fmt.Sprintf("%03d", i)) + c.Assert(err, check.IsNil) + } + time.Sleep(3 * time.Second) + resp, err := etcdCli.Get(ctx, "key") + c.Assert(err, check.IsNil) + + utils.WaitSomething(10, time.Second, func() bool { + _, err = etcdCli.Get(ctx, "key", clientv3.WithRev(resp.Header.Revision-1)) + return err != nil + }) + c.Assert(err, check.ErrorMatches, ".*required revision has been compacted.*") + + cancel() + s.Close() +} diff --git a/dm/master/shardddl/optimist.go b/dm/master/shardddl/optimist.go index 70067ed91e..9fabc1494e 100644 --- a/dm/master/shardddl/optimist.go +++ b/dm/master/shardddl/optimist.go @@ -22,6 +22,7 @@ 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" @@ -33,6 +34,7 @@ import ( "github.com/pingcap/dm/pkg/log" "github.com/pingcap/dm/pkg/shardddl/optimism" "github.com/pingcap/dm/pkg/terror" + "github.com/pingcap/dm/pkg/utils" ) // Optimist is used to coordinate the shard DDL migration in optimism mode. @@ -245,39 +247,105 @@ func (o *Optimist) rebuildLocks() (revSource, revInfo, revOperation int64, err e } o.logger.Info("get history shard DDL lock operation", zap.Int64("revision", revOperation)) + 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)) + } + // recover the shard DDL lock based on history shard DDL info & lock operation. - err = o.recoverLocks(ifm, opm) + err = o.recoverLocks(ifm, opm, colm) 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)) } + return revSource, revInfo, revOperation, nil } -// 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) error { - // construct locks based on the shard DDL info. - for task, ifTask := range ifm { +// sortInfos sort all infos by revision +func sortInfos(ifm map[string]map[string]map[string]map[string]optimism.Info) []optimism.Info { + infos := make([]optimism.Info, 0, len(ifm)) + + for _, ifTask := range ifm { for _, ifSource := range ifTask { for _, ifSchema := range ifSource { for _, info := range ifSchema { - tts := o.tk.FindTables(task, info.DownSchema, info.DownTable) - _, _, err := o.lk.TrySync(info, tts) - if err != nil { - return err + infos = append(infos, info) + } + } + } + } + + // sort according to the Revision + sort.Slice(infos, func(i, j int) bool { + return infos[i].Revision < infos[j].Revision + }) + 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) (map[string]schemacmp.Table, map[string][]optimism.TargetTable) { + lockJoined := make(map[string]schemacmp.Table) + 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 joined, ok := lockJoined[lockID]; !ok { + lockJoined[lockID] = schemacmp.Encode(info.TableInfoBefore) + } else { + newJoined, err := joined.Join(schemacmp.Encode(info.TableInfoBefore)) + // 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 + } } - // never mark the lock operation from `done` to `not-done` when recovering. - err = o.handleLock(info, tts, true) - if err != nil { - return err + if _, ok := lockTTS[lockID]; !ok { + lockTTS[lockID] = o.tk.FindTables(info.Task, info.DownSchema, info.DownTable) } } } } } + return lockJoined, lockTTS +} + +// 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]struct{}) error { + // construct joined table based on the shard DDL info. + o.logger.Info("build lock joined and tts") + lockJoined, lockTTS := o.buildLockJoinedAndTTS(ifm) + // build lock and restore table info + o.logger.Info("rebuild locks and tables") + o.lk.RebuildLocksAndTables(o.cli, ifm, colm, lockJoined, lockTTS) + // sort infos by revision + infos := sortInfos(ifm) + var firstErr error + setFirstErr := func(err error) { + if firstErr == nil && err != nil { + firstErr = err + } + } + + for _, info := range infos { + // 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 + } + } // update the done status of the lock. for _, opTask := range opm { @@ -291,12 +359,17 @@ func (o *Optimist) recoverLocks( } if op.Done { lock.TryMarkDone(op.Source, op.UpSchema, op.UpTable) + err := lock.DeleteColumnsByDDLs(op.DDLs) + if err != nil { + o.logger.Error("fail to update lock columns", zap.Error(err)) + continue + } } } } } } - return nil + return firstErr } // watchSourceInfoOperation watches the etcd operation for source tables, shard DDL infos and shard DDL operations. @@ -338,7 +411,7 @@ func (o *Optimist) watchSourceInfoOperation( }() go func() { defer wg.Done() - o.handleInfo(ctx, infoCh) + o.handleInfoPut(ctx, infoCh) }() // watch for the shard DDL lock operation and handle them. @@ -381,8 +454,8 @@ func (o *Optimist) handleSourceTables(ctx context.Context, sourceCh <-chan optim } } -// handleInfo handles PUT and DELETE for the shard DDL info. -func (o *Optimist) handleInfo(ctx context.Context, infoCh <-chan optimism.Info) { +// handleInfoPut handles PUT and DELETE for the shard DDL info. +func (o *Optimist) handleInfoPut(ctx context.Context, infoCh <-chan optimism.Info) { for { select { case <-ctx.Done(): @@ -414,35 +487,38 @@ func (o *Optimist) handleInfo(ctx context.Context, infoCh <-chan optimism.Info) continue } - added := o.tk.AddTable(info.Task, info.Source, info.UpSchema, info.UpTable, info.DownSchema, info.DownTable) - o.logger.Debug("a table added for info", zap.Bool("added", added), zap.Stringer("info", info)) - - tts := o.tk.FindTables(info.Task, info.DownSchema, info.DownTable) - if tts == nil { - // WATCH for SourceTables may fall behind WATCH for Info although PUT earlier, - // so we try to get SourceTables again. - // NOTE: check SourceTables for `info.Source` if needed later. - stm, _, err := optimism.GetAllSourceTables(o.cli) - if err != nil { - o.logger.Error("fail to get source tables", log.ShortError(err)) - } else if tts2 := optimism.TargetTablesForTask(info.Task, info.DownSchema, info.DownTable, stm); tts2 != nil { - tts = tts2 - } - } // put operation for the table. we don't set `skipDone=true` now, // because in optimism mode, one table may execute/done multiple DDLs but other tables may do nothing. - err := o.handleLock(info, tts, false) - if err != nil { - o.logger.Error("fail to handle the shard DDL lock", zap.Stringer("info", info), log.ShortError(err)) - metrics.ReportDDLError(info.Task, metrics.InfoErrHandleLock) - o.mu.Unlock() - continue - } + _ = o.handleInfo(info, false) o.mu.Unlock() } } } +func (o *Optimist) handleInfo(info optimism.Info, skipDone bool) error { + added := o.tk.AddTable(info.Task, info.Source, info.UpSchema, info.UpTable, info.DownSchema, info.DownTable) + o.logger.Debug("a table added for info", zap.Bool("added", added), zap.Stringer("info", info)) + + tts := o.tk.FindTables(info.Task, info.DownSchema, info.DownTable) + if tts == nil { + // WATCH for SourceTables may fall behind WATCH for Info although PUT earlier, + // so we try to get SourceTables again. + // NOTE: check SourceTables for `info.Source` if needed later. + stm, _, err := optimism.GetAllSourceTables(o.cli) + if err != nil { + o.logger.Error("fail to get source tables", log.ShortError(err)) + } else if tts2 := optimism.TargetTablesForTask(info.Task, info.DownSchema, info.DownTable, stm); tts2 != nil { + tts = tts2 + } + } + err := o.handleLock(info, tts, skipDone) + if err != nil { + o.logger.Error("fail to handle the shard DDL lock", zap.Stringer("info", info), log.ShortError(err)) + metrics.ReportDDLError(info.Task, metrics.InfoErrHandleLock) + } + return err +} + // handleOperationPut handles PUT for the shard DDL lock operations. func (o *Optimist) handleOperationPut(ctx context.Context, opCh <-chan optimism.Operation) { for { @@ -469,6 +545,10 @@ func (o *Optimist) handleOperationPut(ctx context.Context, opCh <-chan optimism. continue } + err := lock.DeleteColumnsByDDLs(op.DDLs) + 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. @@ -497,13 +577,15 @@ func (o *Optimist) handleOperationPut(ctx context.Context, opCh <-chan optimism. // handleLock handles a single shard DDL lock. func (o *Optimist) handleLock(info optimism.Info, tts []optimism.TargetTable, skipDone bool) error { - lockID, newDDLs, err := o.lk.TrySync(info, tts) + lockID, newDDLs, err := o.lk.TrySync(o.cli, info, tts) var cfStage = optimism.ConflictNone + var cfMsg = "" if info.IgnoreConflict { o.logger.Warn("error occur when trying to sync for shard DDL info, this often means shard DDL conflict detected", zap.String("lock", lockID), zap.Stringer("info", info), zap.Bool("is deleted", info.IsDeleted), log.ShortError(err)) } else if err != nil { cfStage = optimism.ConflictDetected // we treat any errors returned from `TrySync` as conflict detected now. + cfMsg = err.Error() o.logger.Warn("error occur when trying to sync for shard DDL info, this often means shard DDL conflict detected", zap.String("lock", lockID), zap.Stringer("info", info), zap.Bool("is deleted", info.IsDeleted), log.ShortError(err)) } else { @@ -543,8 +625,8 @@ func (o *Optimist) handleLock(info optimism.Info, tts []optimism.TargetTable, sk return nil } - op := optimism.NewOperation(lockID, lock.Task, info.Source, info.UpSchema, info.UpTable, newDDLs, cfStage, false) - rev, succ, err := optimism.PutOperation(o.cli, skipDone, op) + op := optimism.NewOperation(lockID, lock.Task, info.Source, info.UpSchema, info.UpTable, newDDLs, cfStage, cfMsg, false) + rev, succ, err := optimism.PutOperation(o.cli, skipDone, op, info.Revision) if err != nil { return err } @@ -615,13 +697,13 @@ func (o *Optimist) deleteInfosOps(lock *optimism.Lock) (bool, error) { info := optimism.NewInfo(lock.Task, source, schema, table, lock.DownSchema, lock.DownTable, nil, nil, nil) info.Version = lock.GetVersion(source, schema, table) infos = append(infos, info) - ops = append(ops, optimism.NewOperation(lock.ID, lock.Task, source, schema, table, nil, optimism.ConflictNone, false)) + ops = append(ops, optimism.NewOperation(lock.ID, lock.Task, source, schema, table, nil, optimism.ConflictNone, "", false)) } } } // 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.DeleteInfosOperationsSchema(o.cli, infos, ops, initSchema) + rev, deleted, err := optimism.DeleteInfosOperationsSchemaColumn(o.cli, infos, ops, initSchema) if err != nil { return deleted, err } diff --git a/dm/master/shardddl/optimist_test.go b/dm/master/shardddl/optimist_test.go index a8c73ed3bb..ea1b6360d1 100644 --- a/dm/master/shardddl/optimist_test.go +++ b/dm/master/shardddl/optimist_test.go @@ -23,9 +23,12 @@ import ( "github.com/pingcap/parser/ast" "github.com/pingcap/parser/model" "github.com/pingcap/tidb-tools/pkg/dbutil" + "github.com/pingcap/tidb-tools/pkg/schemacmp" tiddl "github.com/pingcap/tidb/ddl" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/util/mock" + "go.etcd.io/etcd/clientv3" + "go.etcd.io/etcd/integration" "github.com/pingcap/dm/dm/config" "github.com/pingcap/dm/dm/pb" @@ -36,7 +39,7 @@ import ( type testOptimist struct{} -var _ = Suite(&testOptimist{}) +var _ = SerialSuites(&testOptimist{}) // clear keys in etcd test cluster. func clearOptimistTestSourceInfoOperation(c *C) { @@ -145,13 +148,20 @@ func (t *testOptimist) TestOptimistSourceTables(c *C) { } func (t *testOptimist) TestOptimist(c *C) { - t.testOptimist(c, noRestart) - t.testOptimist(c, restartOnly) - t.testOptimist(c, restartNewInstance) + cluster := integration.NewClusterV3(tt, &integration.ClusterConfig{Size: 1}) + defer cluster.Terminate(tt) + + cli := cluster.RandClient() + t.testOptimist(c, cli, noRestart) + t.testOptimist(c, cli, restartOnly) + t.testOptimist(c, cli, restartNewInstance) + t.testSortInfos(c, cli) } -func (t *testOptimist) testOptimist(c *C, restart int) { - defer clearOptimistTestSourceInfoOperation(c) +func (t *testOptimist) testOptimist(c *C, cli *clientv3.Client, restart int) { + defer func() { + c.Assert(optimism.ClearTestInfoOperationSchema(cli), IsNil) + }() var ( backOff = 30 @@ -164,11 +174,11 @@ func (t *testOptimist) testOptimist(c *C, restart int) { switch restart { case restartOnly: o.Close() - c.Assert(o.Start(ctx, etcdTestCli), IsNil) + c.Assert(o.Start(ctx, cli), IsNil) case restartNewInstance: o.Close() o = NewOptimist(&logger) - c.Assert(o.Start(ctx, etcdTestCli), IsNil) + c.Assert(o.Start(ctx, cli), IsNil) } } @@ -206,24 +216,24 @@ func (t *testOptimist) testOptimist(c *C, restart int) { st32.AddTable("foo-2", "bar-3", downSchema, downTable) // put source tables first. - _, err := optimism.PutSourceTables(etcdTestCli, st1) + _, err := optimism.PutSourceTables(cli, st1) c.Assert(err, IsNil) ctx, cancel := context.WithCancel(context.Background()) defer cancel() // CASE 1: start without any previous shard DDL info. - c.Assert(o.Start(ctx, etcdTestCli), IsNil) + c.Assert(o.Start(ctx, cli), IsNil) c.Assert(o.Locks(), HasLen, 0) o.Close() o.Close() // close multiple times. // CASE 2: start again without any previous shard DDL info. - c.Assert(o.Start(ctx, etcdTestCli), IsNil) + c.Assert(o.Start(ctx, cli), IsNil) c.Assert(o.Locks(), HasLen, 0) // PUT i11, will create a lock but not synced. - rev1, err := optimism.PutInfo(etcdTestCli, i11) + rev1, err := optimism.PutInfo(cli, i11) c.Assert(err, IsNil) c.Assert(utils.WaitSomething(backOff, waitTime, func() bool { return len(o.Locks()) == 1 @@ -255,7 +265,7 @@ func (t *testOptimist) testOptimist(c *C, restart int) { opCh := make(chan optimism.Operation, 10) errCh := make(chan error, 10) ctx2, cancel2 := context.WithCancel(ctx) - go optimism.WatchOperationPut(ctx2, etcdTestCli, i11.Task, i11.Source, i11.UpSchema, i11.UpTable, rev1, opCh, errCh) + go optimism.WatchOperationPut(ctx2, cli, i11.Task, i11.Source, i11.UpSchema, i11.UpTable, rev1, opCh, errCh) utils.WaitSomething(10, watchTimeout, func() bool { return len(opCh) != 0 }) @@ -272,7 +282,7 @@ func (t *testOptimist) testOptimist(c *C, restart int) { // mark op11 as done. op11c := op11 op11c.Done = true - _, putted, err := optimism.PutOperation(etcdTestCli, false, op11c) + _, putted, err := optimism.PutOperation(cli, false, op11c, 0) c.Assert(err, IsNil) c.Assert(putted, IsTrue) c.Assert(utils.WaitSomething(backOff, waitTime, func() bool { @@ -286,7 +296,7 @@ func (t *testOptimist) testOptimist(c *C, restart int) { c.Assert(o.ShowLocks("", []string{}), DeepEquals, expectedLock) // PUT i12, the lock will be synced. - rev2, err := optimism.PutInfo(etcdTestCli, i12) + rev2, err := optimism.PutInfo(cli, i12) c.Assert(err, IsNil) c.Assert(utils.WaitSomething(backOff, waitTime, func() bool { synced, _ = o.Locks()[lockID].IsSynced() @@ -313,7 +323,7 @@ func (t *testOptimist) testOptimist(c *C, restart int) { opCh = make(chan optimism.Operation, 10) errCh = make(chan error, 10) ctx2, cancel2 = context.WithCancel(ctx) - go optimism.WatchOperationPut(ctx2, etcdTestCli, i12.Task, i12.Source, i12.UpSchema, i12.UpTable, rev2, opCh, errCh) + go optimism.WatchOperationPut(ctx2, cli, i12.Task, i12.Source, i12.UpSchema, i12.UpTable, rev2, opCh, errCh) utils.WaitSomething(10, watchTimeout, func() bool { return len(opCh) != 0 }) @@ -330,7 +340,7 @@ func (t *testOptimist) testOptimist(c *C, restart int) { // mark op12 as done, the lock should be resolved. op12c := op12 op12c.Done = true - _, putted, err = optimism.PutOperation(etcdTestCli, false, op12c) + _, putted, err = optimism.PutOperation(cli, false, op12c, 0) c.Assert(err, IsNil) c.Assert(putted, IsTrue) c.Assert(utils.WaitSomething(backOff, waitTime, func() bool { @@ -341,15 +351,15 @@ func (t *testOptimist) testOptimist(c *C, restart int) { c.Assert(o.ShowLocks("", nil), HasLen, 0) // no shard DDL info or lock operation exists. - ifm, _, err := optimism.GetAllInfo(etcdTestCli) + ifm, _, err := optimism.GetAllInfo(cli) c.Assert(err, IsNil) c.Assert(ifm, HasLen, 0) - opm, _, err := optimism.GetAllOperations(etcdTestCli) + opm, _, err := optimism.GetAllOperations(cli) c.Assert(err, IsNil) c.Assert(opm, HasLen, 0) // put another table info. - rev1, err = optimism.PutInfo(etcdTestCli, i21) + rev1, err = optimism.PutInfo(cli, i21) c.Assert(err, IsNil) c.Assert(utils.WaitSomething(backOff, waitTime, func() bool { return len(o.Locks()) == 1 @@ -363,7 +373,7 @@ func (t *testOptimist) testOptimist(c *C, restart int) { opCh = make(chan optimism.Operation, 10) errCh = make(chan error, 10) ctx2, cancel2 = context.WithCancel(ctx) - go optimism.WatchOperationPut(ctx2, etcdTestCli, i21.Task, i21.Source, i21.UpSchema, i21.UpTable, rev1, opCh, errCh) + go optimism.WatchOperationPut(ctx2, cli, i21.Task, i21.Source, i21.UpSchema, i21.UpTable, rev1, opCh, errCh) utils.WaitSomething(10, watchTimeout, func() bool { return len(opCh) != 0 }) @@ -385,7 +395,7 @@ func (t *testOptimist) testOptimist(c *C, restart int) { c.Assert(remain, Equals, 1) // put table info for a new table (to simulate `CREATE TABLE`). - rev3, err := optimism.PutSourceTablesInfo(etcdTestCli, st32, i23) + rev3, err := optimism.PutSourceTablesInfo(cli, st32, i23) c.Assert(err, IsNil) c.Assert(utils.WaitSomething(backOff, waitTime, func() bool { ready := o.Locks()[lockID].Ready() @@ -430,7 +440,7 @@ func (t *testOptimist) testOptimist(c *C, restart int) { opCh = make(chan optimism.Operation, 10) errCh = make(chan error, 10) ctx2, cancel2 = context.WithCancel(ctx) - go optimism.WatchOperationPut(ctx2, etcdTestCli, i23.Task, i23.Source, i23.UpSchema, i23.UpTable, rev3, opCh, errCh) + go optimism.WatchOperationPut(ctx2, cli, i23.Task, i23.Source, i23.UpSchema, i23.UpTable, rev3, opCh, errCh) utils.WaitSomething(10, watchTimeout, func() bool { return len(opCh) != 0 }) @@ -444,13 +454,13 @@ func (t *testOptimist) testOptimist(c *C, restart int) { c.Assert(len(errCh), Equals, 0) // delete i12 for a table (to simulate `DROP TABLE`), the lock should become synced again. - rev2, err = optimism.PutInfo(etcdTestCli, i12) // put i12 first to trigger DELETE for i12. + rev2, err = optimism.PutInfo(cli, i12) // put i12 first to trigger DELETE for i12. c.Assert(err, IsNil) // wait until operation for i12 ready. opCh = make(chan optimism.Operation, 10) errCh = make(chan error, 10) ctx2, cancel2 = context.WithCancel(ctx) - go optimism.WatchOperationPut(ctx2, etcdTestCli, i12.Task, i12.Source, i12.UpSchema, i12.UpTable, rev2, opCh, errCh) + go optimism.WatchOperationPut(ctx2, cli, i12.Task, i12.Source, i12.UpSchema, i12.UpTable, rev2, opCh, errCh) utils.WaitSomething(10, watchTimeout, func() bool { return len(opCh) != 0 }) @@ -460,7 +470,7 @@ func (t *testOptimist) testOptimist(c *C, restart int) { c.Assert(len(opCh), Equals, 1) c.Assert(len(errCh), Equals, 0) - _, err = optimism.PutSourceTablesDeleteInfo(etcdTestCli, st31, i12) + _, err = optimism.PutSourceTablesDeleteInfo(cli, st31, i12) c.Assert(err, IsNil) c.Assert(utils.WaitSomething(backOff, waitTime, func() bool { synced, _ = o.Locks()[lockID].IsSynced() @@ -491,7 +501,7 @@ func (t *testOptimist) testOptimist(c *C, restart int) { // mark op21 as done. op21c := op21 op21c.Done = true - _, putted, err = optimism.PutOperation(etcdTestCli, false, op21c) + _, putted, err = optimism.PutOperation(cli, false, op21c, 0) c.Assert(err, IsNil) c.Assert(putted, IsTrue) c.Assert(utils.WaitSomething(backOff, waitTime, func() bool { @@ -511,7 +521,7 @@ func (t *testOptimist) testOptimist(c *C, restart int) { // mark op23 as done. op23c := op23 op23c.Done = true - _, putted, err = optimism.PutOperation(etcdTestCli, false, op23c) + _, putted, err = optimism.PutOperation(cli, false, op23c, 0) c.Assert(err, IsNil) c.Assert(putted, IsTrue) c.Assert(utils.WaitSomething(backOff, waitTime, func() bool { @@ -521,7 +531,7 @@ func (t *testOptimist) testOptimist(c *C, restart int) { c.Assert(o.Locks(), HasLen, 0) // PUT i31, will create a lock but not synced (to test `DROP COLUMN`) - rev1, err = optimism.PutInfo(etcdTestCli, i31) + rev1, err = optimism.PutInfo(cli, i31) c.Assert(err, IsNil) c.Assert(utils.WaitSomething(backOff, waitTime, func() bool { return len(o.Locks()) == 1 @@ -553,7 +563,7 @@ func (t *testOptimist) testOptimist(c *C, restart int) { opCh = make(chan optimism.Operation, 10) errCh = make(chan error, 10) ctx2, cancel2 = context.WithCancel(ctx) - go optimism.WatchOperationPut(ctx2, etcdTestCli, i31.Task, i31.Source, i31.UpSchema, i31.UpTable, rev1, opCh, errCh) + go optimism.WatchOperationPut(ctx2, cli, i31.Task, i31.Source, i31.UpSchema, i31.UpTable, rev1, opCh, errCh) utils.WaitSomething(10, watchTimeout, func() bool { return len(opCh) != 0 }) @@ -570,7 +580,7 @@ func (t *testOptimist) testOptimist(c *C, restart int) { // mark op31 as done. op31c := op31 op31c.Done = true - _, putted, err = optimism.PutOperation(etcdTestCli, false, op31c) + _, putted, err = optimism.PutOperation(cli, false, op31c, 0) c.Assert(err, IsNil) c.Assert(putted, IsTrue) c.Assert(utils.WaitSomething(backOff, waitTime, func() bool { @@ -579,7 +589,7 @@ func (t *testOptimist) testOptimist(c *C, restart int) { c.Assert(o.ShowLocks("", []string{}), DeepEquals, expectedLock) // PUT i33, the lock will be synced. - rev3, err = optimism.PutInfo(etcdTestCli, i33) + rev3, err = optimism.PutInfo(cli, i33) c.Assert(err, IsNil) c.Assert(utils.WaitSomething(backOff, waitTime, func() bool { synced, _ = o.Locks()[lockID].IsSynced() @@ -606,7 +616,7 @@ func (t *testOptimist) testOptimist(c *C, restart int) { opCh = make(chan optimism.Operation, 10) errCh = make(chan error, 10) ctx2, cancel2 = context.WithCancel(ctx) - go optimism.WatchOperationPut(ctx2, etcdTestCli, i33.Task, i33.Source, i33.UpSchema, i33.UpTable, rev3, opCh, errCh) + go optimism.WatchOperationPut(ctx2, cli, i33.Task, i33.Source, i33.UpSchema, i33.UpTable, rev3, opCh, errCh) utils.WaitSomething(10, watchTimeout, func() bool { return len(opCh) != 0 }) @@ -623,7 +633,7 @@ func (t *testOptimist) testOptimist(c *C, restart int) { // mark op33 as done, the lock should be resolved. op33c := op33 op33c.Done = true - _, putted, err = optimism.PutOperation(etcdTestCli, false, op33c) + _, putted, err = optimism.PutOperation(cli, false, op33c, 0) c.Assert(err, IsNil) c.Assert(putted, IsTrue) c.Assert(utils.WaitSomething(backOff, waitTime, func() bool { @@ -656,14 +666,13 @@ func (t *testOptimist) TestOptimistLockConflict(c *C) { tblID int64 = 111 DDLs1 = []string{"ALTER TABLE bar ADD COLUMN c1 TEXT"} DDLs2 = []string{"ALTER TABLE bar ADD COLUMN c1 DATETIME"} - DDLs3 = []string{"ALTER TABLE bar DROP COLUMN c1"} 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 TEXT)`) ti2 = createTableInfo(c, p, se, tblID, `CREATE TABLE bar (id INT PRIMARY KEY, c1 DATETIME)`) ti3 = ti0 i1 = optimism.NewInfo(task, source1, "foo", "bar-1", downSchema, downTable, DDLs1, ti0, []*model.TableInfo{ti1}) i2 = optimism.NewInfo(task, source1, "foo", "bar-2", downSchema, downTable, DDLs2, ti0, []*model.TableInfo{ti2}) - i3 = optimism.NewInfo(task, source1, "foo", "bar-2", downSchema, downTable, DDLs3, ti2, []*model.TableInfo{ti3}) + i3 = optimism.NewInfo(task, source1, "foo", "bar-2", downSchema, downTable, DDLs1, ti0, []*model.TableInfo{ti3}) ) st1.AddTable("foo", "bar-1", downSchema, downTable) @@ -714,6 +723,7 @@ func (t *testOptimist) TestOptimistLockConflict(c *C) { c.Assert(len(errCh), Equals, 0) // PUT i3, no conflict now. + // case for handle-error replace rev3, err := optimism.PutInfo(etcdTestCli, i3) c.Assert(err, IsNil) // wait operation for i3 become available. @@ -866,12 +876,12 @@ func (t *testOptimist) TestOptimistLockMultipleTarget(c *C) { op11c := op12 op11c.Done = true op11c.UpTable = i11.UpTable // overwrite `UpTable`. - _, putted, err := optimism.PutOperation(etcdTestCli, false, op11c) + _, putted, err := optimism.PutOperation(etcdTestCli, false, op11c, 0) c.Assert(err, IsNil) c.Assert(putted, IsTrue) op12c := op12 op12c.Done = true - _, putted, err = optimism.PutOperation(etcdTestCli, false, op12c) + _, putted, err = optimism.PutOperation(etcdTestCli, false, op12c, 0) c.Assert(err, IsNil) c.Assert(putted, IsTrue) c.Assert(utils.WaitSomething(backOff, waitTime, func() bool { @@ -900,12 +910,12 @@ func (t *testOptimist) TestOptimistLockMultipleTarget(c *C) { op21c := op22 op21c.Done = true op21c.UpTable = i21.UpTable // overwrite `UpTable`. - _, putted, err = optimism.PutOperation(etcdTestCli, false, op21c) + _, putted, err = optimism.PutOperation(etcdTestCli, false, op21c, 0) c.Assert(err, IsNil) c.Assert(putted, IsTrue) op22c := op22 op22c.Done = true - _, putted, err = optimism.PutOperation(etcdTestCli, false, op22c) + _, putted, err = optimism.PutOperation(etcdTestCli, false, op22c, 0) c.Assert(err, IsNil) c.Assert(putted, IsTrue) c.Assert(utils.WaitSomething(backOff, waitTime, func() bool { @@ -999,12 +1009,12 @@ func (t *testOptimist) TestOptimistInitSchema(c *C) { op11c := op12 op11c.Done = true op11c.UpTable = i11.UpTable // overwrite `UpTable`. - _, putted, err := optimism.PutOperation(etcdTestCli, false, op11c) + _, putted, err := optimism.PutOperation(etcdTestCli, false, op11c, 0) c.Assert(err, IsNil) c.Assert(putted, IsTrue) op12c := op12 op12c.Done = true - _, putted, err = optimism.PutOperation(etcdTestCli, false, op12c) + _, putted, err = optimism.PutOperation(etcdTestCli, false, op12c, 0) c.Assert(err, IsNil) c.Assert(putted, IsTrue) c.Assert(utils.WaitSomething(backOff, waitTime, func() bool { @@ -1029,3 +1039,129 @@ func (t *testOptimist) TestOptimistInitSchema(c *C) { 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) + }() + + var ( + task = "test-optimist-init-schema" + sources = []string{"mysql-replica-1", "mysql-replica-2"} + upSchema = "foo" + upTables = []string{"bar-1", "bar-2"} + downSchema = "foo" + downTable = "bar" + + p = parser.New() + se = mock.NewContext() + tblID int64 = 111 + DDLs1 = []string{"ALTER TABLE bar ADD COLUMN c1 TEXT"} + DDLs2 = []string{"ALTER TABLE bar ADD COLUMN c2 INT"} + 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 TEXT)`) + ti2 = createTableInfo(c, p, se, tblID, `CREATE TABLE bar (id INT PRIMARY KEY, c1 TEXT, c2 INT)`) + i11 = optimism.NewInfo(task, sources[0], upSchema, upTables[0], downSchema, downTable, DDLs1, ti0, []*model.TableInfo{ti1}) + i12 = optimism.NewInfo(task, sources[0], upSchema, upTables[1], downSchema, downTable, DDLs1, ti0, []*model.TableInfo{ti1}) + i21 = optimism.NewInfo(task, sources[1], upSchema, upTables[1], downSchema, downTable, DDLs2, ti1, []*model.TableInfo{ti2}) + ) + + rev1, err := optimism.PutInfo(cli, i11) + c.Assert(err, IsNil) + ifm, _, err := optimism.GetAllInfo(cli) + c.Assert(err, IsNil) + infos := sortInfos(ifm) + c.Assert(len(infos), Equals, 1) + i11.Version = 1 + i11.Revision = rev1 + c.Assert(infos[0], DeepEquals, i11) + + rev2, err := optimism.PutInfo(cli, i12) + c.Assert(err, IsNil) + ifm, _, err = optimism.GetAllInfo(cli) + c.Assert(err, IsNil) + infos = sortInfos(ifm) + c.Assert(len(infos), Equals, 2) + i11.Version = 1 + i11.Revision = rev1 + i12.Version = 1 + i12.Revision = rev2 + c.Assert(infos[0], DeepEquals, i11) + c.Assert(infos[1], DeepEquals, i12) + + rev3, err := optimism.PutInfo(cli, i21) + c.Assert(err, IsNil) + rev4, err := optimism.PutInfo(cli, i11) + c.Assert(err, IsNil) + ifm, _, err = optimism.GetAllInfo(cli) + c.Assert(err, IsNil) + infos = sortInfos(ifm) + c.Assert(len(infos), Equals, 3) + + i11.Version = 2 + i11.Revision = rev4 + i12.Version = 1 + i12.Revision = rev2 + i21.Version = 1 + i21.Revision = rev3 + c.Assert(infos[0], DeepEquals, i12) + c.Assert(infos[1], DeepEquals, i21) + c.Assert(infos[2], DeepEquals, i11) +} + +func (t *testOptimist) TestBuildLockJoinedAndTable(c *C) { + defer clearOptimistTestSourceInfoOperation(c) + + var ( + logger = log.L() + o = NewOptimist(&logger) + task = "task" + source1 = "mysql-replica-1" + source2 = "mysql-replica-2" + downSchema = "db" + downTable = "tbl" + st1 = optimism.NewSourceTables(task, source1) + st2 = optimism.NewSourceTables(task, source2) + 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 = optimism.NewInfo(task, source1, "foo", "bar-1", downSchema, downTable, DDLs1, ti0, []*model.TableInfo{ti1}) + i21 = optimism.NewInfo(task, source2, "foo", "bar-1", downSchema, downTable, DDLs2, ti2, []*model.TableInfo{ti3}) + ) + + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + + st1.AddTable("db", "tbl-1", downSchema, downTable) + st2.AddTable("db", "tbl-1", downSchema, downTable) + + c.Assert(o.Start(ctx, etcdTestCli), IsNil) + _, err := optimism.PutSourceTables(etcdTestCli, st1) + c.Assert(err, IsNil) + _, err = optimism.PutSourceTables(etcdTestCli, st2) + c.Assert(err, IsNil) + + _, err = optimism.PutInfo(etcdTestCli, i21) + c.Assert(err, IsNil) + _, err = optimism.PutInfo(etcdTestCli, i11) + c.Assert(err, IsNil) + + ifm, _, err := optimism.GetAllInfo(etcdTestCli) + c.Assert(err, IsNil) + + lockJoined, lockTTS := o.buildLockJoinedAndTTS(ifm) + c.Assert(len(lockJoined), Equals, 1) + c.Assert(len(lockTTS), Equals, 1) + 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) +} diff --git a/dm/master/shardddl/pessimist_test.go b/dm/master/shardddl/pessimist_test.go index 2498a9536a..8e796c0e3e 100644 --- a/dm/master/shardddl/pessimist_test.go +++ b/dm/master/shardddl/pessimist_test.go @@ -37,6 +37,7 @@ import ( var ( etcdTestCli *clientv3.Client etcdErrCompacted = v3rpc.ErrCompacted + tt *testing.T ) const ( @@ -54,6 +55,7 @@ func TestShardDDL(t *testing.T) { if err != nil { t.Fatal(err) } + tt = t mockCluster := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1}) defer mockCluster.Terminate(t) diff --git a/dm/worker/join.go b/dm/worker/join.go index b6d366df20..b0cd59f9ea 100644 --- a/dm/worker/join.go +++ b/dm/worker/join.go @@ -16,7 +16,6 @@ package worker import ( "context" "strings" - "sync" "time" "github.com/pingcap/failpoint" @@ -105,7 +104,7 @@ func (s *Server) KeepAlive() { failpoint.Label("bypass") // TODO: report the error. - err := s.stopWorker("") + err := s.stopWorker("", true) if err != nil { log.L().Error("fail to stop worker", zap.Error(err)) return // return if failed to stop the worker. @@ -120,18 +119,8 @@ func (s *Server) KeepAlive() { } } -// TODO: a channel is enough to avoid data race, check TTL not changed at receiving end of channel -var keepAliveLock sync.Mutex - // UpdateKeepAliveTTL updates keepalive key with new lease TTL in place, to avoid watcher observe a DELETE event func (s *Server) UpdateKeepAliveTTL(newTTL int64) { - keepAliveLock.Lock() - defer keepAliveLock.Unlock() - if ha.CurrentKeepAliveTTL == newTTL { - log.L().Info("not changing keepalive TTL, skip", zap.Int64("ttl", newTTL)) - return - } - ha.CurrentKeepAliveTTL = newTTL ha.KeepAliveUpdateCh <- newTTL log.L().Debug("received update keepalive TTL request, should be updated soon", zap.Int64("new ttl", newTTL)) } diff --git a/dm/worker/metrics.go b/dm/worker/metrics.go index 7c44569555..a2c816a402 100644 --- a/dm/worker/metrics.go +++ b/dm/worker/metrics.go @@ -37,6 +37,7 @@ import ( const ( opErrTypeBeforeOp = "BeforeAnyOp" opErrTypeSourceBound = "SourceBound" + opErrTypeRelaySource = "RelaySource" ) var ( diff --git a/dm/worker/server.go b/dm/worker/server.go index b79ec8062a..702ff8975e 100644 --- a/dm/worker/server.go +++ b/dm/worker/server.go @@ -48,6 +48,7 @@ var ( keepaliveTimeout = 3 * time.Second keepaliveTime = 3 * time.Second retryGetSourceBoundConfig = 5 + retryGetRelayConfig = 5 retryConnectSleepTime = time.Second syncMasterEndpointsTime = 3 * time.Second getMinLocForSubTaskFunc = getMinLocForSubTask @@ -130,25 +131,46 @@ func (s *Server) Start() error { s.startKeepAlive() + relaySource, revRelay, err := ha.GetRelayConfig(s.etcdClient, s.cfg.Name) + if err != nil { + return err + } + if relaySource != nil { + log.L().Warn("worker has been assigned relay before keepalive", zap.String("relay source", relaySource.SourceID)) + w, err2 := s.getOrStartWorker(relaySource, true) + s.setSourceStatus(relaySource.SourceID, err2, true) + if err2 != nil { + // if DM-worker can't handle pre-assigned source before keepalive, it simply exits with the error, + // because no re-assigned mechanism exists for keepalived DM-worker yet. + return err2 + } + s.UpdateKeepAliveTTL(s.cfg.RelayKeepAliveTTL) + if err2 = w.EnableRelay(); err2 != nil { + return err2 + } + } + + s.wg.Add(1) + go func(ctx context.Context) { + defer s.wg.Done() + // TODO: handle fatal error from observeRelayConfig + //nolint:errcheck + s.observeRelayConfig(ctx, revRelay) + }(s.ctx) + bound, sourceCfg, revBound, err := ha.GetSourceBoundConfig(s.etcdClient, s.cfg.Name) if err != nil { return err } if !bound.IsEmpty() { log.L().Warn("worker has been assigned source before keepalive", zap.Stringer("bound", bound), zap.Bool("is deleted", bound.IsDeleted)) - w, err2 := s.getOrStartWorker(&sourceCfg) + w, err2 := s.getOrStartWorker(&sourceCfg, true) s.setSourceStatus(bound.Source, err2, true) if err2 != nil { // if DM-worker can't handle pre-assigned source before keepalive, it simply exits with the error, // because no re-assigned mechanism exists for keepalived DM-worker yet. return err2 } - if sourceCfg.EnableRelay { - s.UpdateKeepAliveTTL(s.cfg.RelayKeepAliveTTL) - if err2 = w.EnableRelay(); err2 != nil { - return err2 - } - } if err2 = w.EnableHandleSubtasks(); err2 != nil { return err2 } @@ -159,7 +181,6 @@ func (s *Server) Start() error { go func(ctx context.Context) { defer s.wg.Done() for { - // TODO: ObserveRelayConfig? err1 := s.observeSourceBound(ctx, revBound) if err1 == nil { return @@ -264,6 +285,98 @@ func (s *Server) syncMasterEndpoints(ctx context.Context) { } } +func (s *Server) observeRelayConfig(ctx context.Context, rev int64) error { + var wg sync.WaitGroup + for { + relayCh := make(chan ha.RelaySource, 10) + relayErrCh := make(chan error, 10) + wg.Add(1) + // use ctx1, cancel1 to make sure old watcher has been released + ctx1, cancel1 := context.WithCancel(ctx) + go func() { + defer func() { + close(relayCh) + close(relayErrCh) + wg.Done() + }() + ha.WatchRelayConfig(ctx1, s.etcdClient, s.cfg.Name, rev+1, relayCh, relayErrCh) + }() + err := s.handleRelayConfig(ctx1, relayCh, relayErrCh) + cancel1() + wg.Wait() + + if etcdutil.IsRetryableError(err) { + rev = 0 + retryNum := 1 + for rev == 0 { + select { + case <-ctx.Done(): + return nil + case <-time.After(500 * time.Millisecond): + relaySource, rev1, err1 := ha.GetRelayConfig(s.etcdClient, s.cfg.Name) + if err1 != nil { + log.L().Error("get relay config from etcd failed, will retry later", zap.Error(err1), zap.Int("retryNum", retryNum)) + retryNum++ + if retryNum > retryGetRelayConfig && etcdutil.IsLimitedRetryableError(err1) { + return err1 + } + break + } + rev = rev1 + if relaySource == nil { + err = s.disableRelay("") + if err != nil { + log.L().Error("fail to disableRelay after etcd retryable error", zap.Error(err)) + return err // return if failed to stop the worker. + } + } else { + err2 := func() error { + s.Lock() + defer s.Unlock() + + if w := s.getWorker(false); w != nil && w.cfg.SourceID == relaySource.SourceID { + // we may face both relay config and subtask bound changed in a compaction error, so here + // we check if observeSourceBound has started a worker + // TODO: add a test for this situation + if !w.relayEnabled.Get() { + if err2 := w.EnableRelay(); err2 != nil { + return err2 + } + } + return nil + } + err = s.stopWorker("", false) + if err != nil { + log.L().Error("fail to stop worker", zap.Error(err)) + return err // return if failed to stop the worker. + } + w, err2 := s.getOrStartWorker(relaySource, false) + s.setSourceStatus(relaySource.SourceID, err2, false) + if err2 != nil { + w.l.Error("fail to recover observeRelayConfig", + zap.String("relay source", relaySource.SourceID), + zap.Error(err2)) + } + s.UpdateKeepAliveTTL(s.cfg.RelayKeepAliveTTL) + return w.EnableRelay() + }() + if err2 != nil { + return err2 + } + } + } + } + } else { + if err != nil { + log.L().Error("observeRelayConfig is failed and will quit now", zap.Error(err)) + } else { + log.L().Info("observeRelayConfig will quit now") + } + return err + } + } +} + func (s *Server) observeSourceBound(ctx context.Context, rev int64) error { var wg sync.WaitGroup for { @@ -303,33 +416,44 @@ func (s *Server) observeSourceBound(ctx context.Context, rev int64) error { } rev = rev1 if bound.IsEmpty() { - err = s.stopWorker("") + err = s.disableHandleSubtasks("") if err != nil { - log.L().Error("fail to stop worker", zap.Error(err)) + log.L().Error("fail to disableHandleSubtasks after etcd retryable error", zap.Error(err)) return err // return if failed to stop the worker. } } else { - if w := s.getWorker(true); w != nil && w.cfg.SourceID == bound.Source { - continue - } - err = s.stopWorker("") - if err != nil { - log.L().Error("fail to stop worker", zap.Error(err)) - return err // return if failed to stop the worker. - } - w, err2 := s.getOrStartWorker(&cfg) - if err2 == nil { - if cfg.EnableRelay { - s.UpdateKeepAliveTTL(s.cfg.RelayKeepAliveTTL) - if err2 = w.EnableRelay(); err2 != nil { - return err2 + err2 := func() error { + s.Lock() + defer s.Unlock() + + if w := s.getWorker(false); w != nil && w.cfg.SourceID == bound.Source { + // we may face both relay config and subtask bound changed in a compaction error, so here + // we check if observeRelayConfig has started a worker + // TODO: add a test for this situation + if !w.subTaskEnabled.Get() { + if err2 := w.EnableHandleSubtasks(); err2 != nil { + return err2 + } } + return nil } - err2 = w.EnableHandleSubtasks() - } - s.setSourceStatus(bound.Source, err2, true) + err = s.stopWorker("", false) + if err != nil { + log.L().Error("fail to stop worker", zap.Error(err)) + return err // return if failed to stop the worker. + } + w, err2 := s.getOrStartWorker(&cfg, false) + if err2 == nil { + err2 = w.EnableHandleSubtasks() + } + s.setSourceStatus(bound.Source, err2, false) + if err2 != nil { + w.l.Error("fail to operate sourceBound on worker", zap.Stringer("bound", bound), zap.Bool("is deleted", bound.IsDeleted), zap.Error(err2)) + } + return nil + }() if err2 != nil { - w.l.Error("fail to operate sourceBound on worker", zap.Stringer("bound", bound), zap.Bool("is deleted", bound.IsDeleted), zap.Error(err2)) + return err2 } } } @@ -409,6 +533,10 @@ func (s *Server) setSourceStatus(source string, err error, needLock bool) { s.Lock() defer s.Unlock() } + // now setSourceStatus will be concurrently called. skip setting a source status if worker has been closed + if s.getWorker(false) == nil && source != "" { + return + } s.sourceStatus = pb.SourceStatus{ Source: source, Worker: s.cfg.Name, @@ -424,23 +552,22 @@ func (s *Server) setSourceStatus(source string, err error, needLock bool) { // if sourceID is set to "", worker will be closed directly // if sourceID is not "", we will check sourceID with w.cfg.SourceID -func (s *Server) stopWorker(sourceID string) error { - s.Lock() +func (s *Server) stopWorker(sourceID string, needLock bool) error { + if needLock { + s.Lock() + defer s.Unlock() + } w := s.getWorker(false) if w == nil { - s.Unlock() log.L().Warn("worker has not been started, no need to stop", zap.String("source", sourceID)) return nil // no need to stop because not started yet } if sourceID != "" && w.cfg.SourceID != sourceID { - s.Unlock() return terror.ErrWorkerSourceNotMatch } - // TODO: and when disable relay s.UpdateKeepAliveTTL(s.cfg.KeepAliveTTL) s.setWorker(nil, false) s.setSourceStatus("", nil, false) - s.Unlock() w.Close() return nil } @@ -480,9 +607,48 @@ OUTER: return nil } +func (s *Server) handleRelayConfig(ctx context.Context, relayCh chan ha.RelaySource, errCh chan error) error { +OUTER: + for { + select { + case <-ctx.Done(): + break OUTER + case relaySource, ok := <-relayCh: + if !ok { + break OUTER + } + log.L().Info("receive relay source", zap.String("relay source", relaySource.Source), zap.Bool("is deleted", relaySource.IsDeleted)) + err := s.operateRelaySource(relaySource) + s.setSourceStatus(relaySource.Source, err, true) + if err != nil { + opErrCounter.WithLabelValues(s.cfg.Name, opErrTypeRelaySource).Inc() + log.L().Error("fail to operate relay source on worker", + zap.String("relay source", relaySource.Source), + zap.Bool("is deleted", relaySource.IsDeleted), + zap.Error(err)) + if etcdutil.IsRetryableError(err) { + return err + } + } + case err, ok := <-errCh: + // currently no value is sent to errCh + if !ok { + break OUTER + } + // TODO: Deal with err + log.L().Error("WatchRelayConfig received an error", zap.Error(err)) + if etcdutil.IsRetryableError(err) { + return err + } + } + } + log.L().Info("worker server is closed, handleRelayConfig will quit now") + return nil +} + func (s *Server) operateSourceBound(bound ha.SourceBound) error { if bound.IsDeleted { - return s.stopWorker(bound.Source) + return s.disableHandleSubtasks(bound.Source) } scm, _, err := ha.GetSourceCfg(s.etcdClient, bound.Source, bound.Revision) if err != nil { @@ -493,19 +659,71 @@ func (s *Server) operateSourceBound(bound ha.SourceBound) error { if !ok { return terror.ErrWorkerFailToGetSourceConfigFromEtcd.Generate(bound.Source) } - w, err := s.getOrStartWorker(&sourceCfg) + w, err := s.getOrStartWorker(&sourceCfg, true) + s.setSourceStatus(bound.Source, err, true) if err != nil { return err } - if sourceCfg.EnableRelay { - s.UpdateKeepAliveTTL(s.cfg.RelayKeepAliveTTL) - if err = w.EnableRelay(); err != nil { - return err - } - } return w.EnableHandleSubtasks() } +func (s *Server) disableHandleSubtasks(source string) error { + s.Lock() + defer s.Unlock() + w := s.getWorker(false) + if w == nil { + log.L().Warn("worker has already stopped before DisableHandleSubtasks", zap.String("source", source)) + return nil + } + w.DisableHandleSubtasks() + var err error + if !w.relayEnabled.Get() { + log.L().Info("relay is not enabled after disabling subtask, so stop worker") + err = s.stopWorker(source, false) + } + return err +} + +func (s *Server) operateRelaySource(relaySource ha.RelaySource) error { + if relaySource.IsDeleted { + return s.disableRelay(relaySource.Source) + } + scm, _, err := ha.GetSourceCfg(s.etcdClient, relaySource.Source, relaySource.Revision) + if err != nil { + // TODO: need retry + return err + } + sourceCfg, ok := scm[relaySource.Source] + if !ok { + return terror.ErrWorkerFailToGetSourceConfigFromEtcd.Generate(relaySource.Source) + } + w, err := s.getOrStartWorker(&sourceCfg, true) + s.setSourceStatus(relaySource.Source, err, true) + if err != nil { + return err + } + s.UpdateKeepAliveTTL(s.cfg.RelayKeepAliveTTL) + return w.EnableRelay() +} + +func (s *Server) disableRelay(source string) error { + s.Lock() + defer s.Unlock() + w := s.getWorker(false) + if w == nil { + log.L().Warn("worker has already stopped before DisableRelay", zap.Any("relaySource", source)) + return nil + } + s.UpdateKeepAliveTTL(s.cfg.KeepAliveTTL) + w.DisableRelay() + var err error + if !w.subTaskEnabled.Get() { + log.L().Info("subtask is not enabled after disabling relay, so stop worker") + err = s.stopWorker(source, false) + } + return err +} + // QueryStatus implements WorkerServer.QueryStatus func (s *Server) QueryStatus(ctx context.Context, req *pb.QueryStatusRequest) (*pb.QueryStatusResponse, error) { log.L().Info("", zap.String("request", "QueryStatus"), zap.Stringer("payload", req)) @@ -575,9 +793,12 @@ func (s *Server) OperateSchema(ctx context.Context, req *pb.OperateWorkerSchemaR }, nil } -func (s *Server) getOrStartWorker(cfg *config.SourceConfig) (*Worker, error) { - s.Lock() - defer s.Unlock() +func (s *Server) getOrStartWorker(cfg *config.SourceConfig, needLock bool) (*Worker, error) { + if needLock { + s.Lock() + defer s.Unlock() + } + if w := s.getWorker(false); w != nil { if w.cfg.SourceID == cfg.SourceID { log.L().Info("mysql source is being handled", zap.String("sourceID", s.worker.cfg.SourceID)) diff --git a/dm/worker/server_test.go b/dm/worker/server_test.go index 787da5782b..37d90c0998 100644 --- a/dm/worker/server_test.go +++ b/dm/worker/server_test.go @@ -99,7 +99,7 @@ func createMockETCD(dir string, host string) (*embed.Etcd, error) { func (t *testServer) TestServer(c *C) { var ( - masterAddr = "127.0.0.1:8261" + masterAddr = tempurl.Alloc()[len("http://"):] workerAddr1 = "127.0.0.1:8262" keepAliveTTL = int64(1) ) @@ -109,6 +109,7 @@ func (t *testServer) TestServer(c *C) { defer ETCD.Close() cfg := NewConfig() c.Assert(cfg.Parse([]string{"-config=./dm-worker.toml"}), IsNil) + cfg.Join = masterAddr cfg.KeepAliveTTL = keepAliveTTL cfg.RelayKeepAliveTTL = keepAliveTTL @@ -227,7 +228,7 @@ func (t *testServer) TestServer(c *C) { func (t *testServer) TestHandleSourceBoundAfterError(c *C) { var ( - masterAddr = "127.0.0.1:8261" + masterAddr = tempurl.Alloc()[len("http://"):] keepAliveTTL = int64(1) ) // start etcd server @@ -237,6 +238,7 @@ func (t *testServer) TestHandleSourceBoundAfterError(c *C) { defer ETCD.Close() cfg := NewConfig() c.Assert(cfg.Parse([]string{"-config=./dm-worker.toml"}), IsNil) + cfg.Join = masterAddr cfg.KeepAliveTTL = keepAliveTTL // new etcd client @@ -333,23 +335,19 @@ func (t *testServer) TestHandleSourceBoundAfterError(c *C) { _, err = ha.PutSourceCfg(etcdCli, sourceCfg) c.Assert(err, IsNil) c.Assert(utils.WaitSomething(30, 100*time.Millisecond, func() bool { - s.Mutex.Lock() - defer s.Mutex.Unlock() - return s.worker != nil + return s.getWorker(true) != nil }), IsTrue) _, err = ha.DeleteSourceBound(etcdCli, s.cfg.Name) c.Assert(err, IsNil) c.Assert(utils.WaitSomething(30, 100*time.Millisecond, func() bool { - s.Mutex.Lock() - defer s.Mutex.Unlock() - return s.worker == nil + return s.getWorker(true) == nil }), IsTrue) } func (t *testServer) TestWatchSourceBoundEtcdCompact(c *C) { var ( - masterAddr = "127.0.0.1:8261" + masterAddr = tempurl.Alloc()[len("http://"):] keepAliveTTL = int64(1) startRev = int64(1) ) @@ -359,6 +357,7 @@ func (t *testServer) TestWatchSourceBoundEtcdCompact(c *C) { defer ETCD.Close() cfg := NewConfig() c.Assert(cfg.Parse([]string{"-config=./dm-worker.toml"}), IsNil) + cfg.Join = masterAddr cfg.KeepAliveTTL = keepAliveTTL cfg.RelayKeepAliveTTL = keepAliveTTL @@ -387,7 +386,7 @@ func (t *testServer) TestWatchSourceBoundEtcdCompact(c *C) { rev, err := ha.DeleteSourceBound(etcdCli, cfg.Name) c.Assert(err, IsNil) // step 2: start source at this worker - w, err := s.getOrStartWorker(&sourceCfg) + w, err := s.getOrStartWorker(&sourceCfg, true) c.Assert(err, IsNil) c.Assert(w.EnableHandleSubtasks(), IsNil) // step 3: trigger etcd compaction and check whether we can receive it through watcher @@ -423,7 +422,7 @@ func (t *testServer) TestWatchSourceBoundEtcdCompact(c *C) { c.Assert(*cfg2, DeepEquals, sourceCfg) cancel1() wg.Wait() - c.Assert(s.stopWorker(sourceCfg.SourceID), IsNil) + c.Assert(s.stopWorker(sourceCfg.SourceID, true), IsNil) // step 5: start observeSourceBound from compacted revision again, should start worker ctx2, cancel2 := context.WithCancel(ctx) wg.Add(1) @@ -472,7 +471,7 @@ func (t *testServer) testOperateWorker(c *C, s *Server, dir string, start bool) // put mysql config into relative etcd key adapter to trigger operation event _, err := ha.PutSourceCfg(s.etcdClient, sourceCfg) c.Assert(err, IsNil) - _, err = ha.PutRelayStageSourceBound(s.etcdClient, ha.NewRelayStage(pb.Stage_Running, sourceCfg.SourceID), + _, err = ha.PutRelayStageRelayConfigSourceBound(s.etcdClient, ha.NewRelayStage(pb.Stage_Running, sourceCfg.SourceID), ha.NewSourceBound(sourceCfg.SourceID, s.cfg.Name)) c.Assert(err, IsNil) // worker should be started and without error @@ -488,7 +487,7 @@ func (t *testServer) testOperateWorker(c *C, s *Server, dir string, start bool) c.Assert(w.closed.Get(), IsFalse) _, err := ha.DeleteSourceCfgRelayStageSourceBound(s.etcdClient, sourceCfg.SourceID, s.cfg.Name) c.Assert(err, IsNil) - // worker should be started and without error + // worker should be closed and without error c.Assert(utils.WaitSomething(30, 100*time.Millisecond, func() bool { currentWorker := s.getWorker(true) return currentWorker == nil && w.closed.Get() diff --git a/dm/worker/subtask.go b/dm/worker/subtask.go index e798591f5f..233c10ccfb 100644 --- a/dm/worker/subtask.go +++ b/dm/worker/subtask.go @@ -215,6 +215,7 @@ func (st *SubTask) run() { st.fail(err) return } else if ctx.Err() != nil { + st.l.Error("exit SubTask.run", log.ShortError(ctx.Err())) return } diff --git a/dm/worker/subtask_holder.go b/dm/worker/subtask_holder.go index e8754ca6fa..7627db23af 100644 --- a/dm/worker/subtask_holder.go +++ b/dm/worker/subtask_holder.go @@ -14,6 +14,7 @@ package worker import ( + "context" "sync" ) @@ -53,6 +54,8 @@ func (h *subTaskHolder) resetAllSubTasks(useRelay bool) { for _, st := range h.subTasks { stage := st.Stage() st.Close() + // TODO: make a st.reset + st.ctx, st.cancel = context.WithCancel(context.Background()) st.cfg.UseRelay = useRelay st.Run(stage) } diff --git a/dm/worker/task_checker_test.go b/dm/worker/task_checker_test.go index bb2d95f5a0..b7bc9eb058 100644 --- a/dm/worker/task_checker_test.go +++ b/dm/worker/task_checker_test.go @@ -304,7 +304,7 @@ func (s *testTaskCheckerSuite) TestIsResumableError(c *check.C) { {terror.ErrParserParseRelayLog.New("parse relay log file bin.000018 from offset 500 in dir /home/tidb/deploy/relay_log/d2e831df-b4ec-11e9-9237-0242ac110008.000004: parse relay log file bin.000018 from offset 0 in dir /home/tidb/deploy/relay_log/d2e831df-b4ec-11e9-9237-0242ac110008.000004: parse relay log file /home/tidb/deploy/relay_log/d2e831df-b4ec-11e9-9237-0242ac110008.000004/bin.000018: get event err EOF, need 1567488104 but got 316323"), false}, // unresumable terror codes {terror.ErrSyncUnitDDLWrongSequence.Generate("wrong sequence", "right sequence"), false}, - {terror.ErrSyncerShardDDLConflict.Generate("conflict DDL"), false}, + {terror.ErrSyncerShardDDLConflict.Generate("conflict DDL", "conflict"), false}, // others {nil, true}, {errors.New("unknown error"), true}, diff --git a/dm/worker/worker_test.go b/dm/worker/worker_test.go index 1b0bc723ef..3a6e59e422 100644 --- a/dm/worker/worker_test.go +++ b/dm/worker/worker_test.go @@ -172,7 +172,7 @@ func (t *testServer2) TestTaskAutoResume(c *C) { if s.closed.Get() { return false } - w, err2 := s.getOrStartWorker(&sourceConfig) + w, err2 := s.getOrStartWorker(&sourceConfig, true) c.Assert(err2, IsNil) // we set sourceConfig.EnableRelay = true above c.Assert(w.EnableRelay(), IsNil) @@ -356,7 +356,7 @@ func (t *testWorkerFunctionalities) testEnableRelay(c *C, w *Worker, etcdCli *cl _, err := ha.PutSourceCfg(etcdCli, sourceCfg) c.Assert(err, IsNil) - _, err = ha.PutRelayStageSourceBound(etcdCli, ha.NewRelayStage(pb.Stage_Running, sourceCfg.SourceID), + _, err = ha.PutRelayStageRelayConfigSourceBound(etcdCli, ha.NewRelayStage(pb.Stage_Running, sourceCfg.SourceID), ha.NewSourceBound(sourceCfg.SourceID, cfg.Name)) c.Assert(err, IsNil) c.Assert(utils.WaitSomething(30, 100*time.Millisecond, func() bool { @@ -576,7 +576,7 @@ func (t *testWorkerEtcdCompact) TestWatchRelayStageEtcdCompact(c *C) { c.Assert(w.relayHolder, NotNil) _, err = ha.PutSourceCfg(etcdCli, sourceCfg) c.Assert(err, IsNil) - rev, err := ha.PutRelayStageSourceBound(etcdCli, ha.NewRelayStage(pb.Stage_Running, sourceCfg.SourceID), + rev, err := ha.PutRelayStageRelayConfigSourceBound(etcdCli, ha.NewRelayStage(pb.Stage_Running, sourceCfg.SourceID), ha.NewSourceBound(sourceCfg.SourceID, cfg.Name)) c.Assert(err, IsNil) // check relay stage, should be running diff --git a/errors.toml b/errors.toml index 4a77f15f7e..dd45513a87 100644 --- a/errors.toml +++ b/errors.toml @@ -1883,7 +1883,7 @@ workaround = "" tags = ["internal", "high"] [error.DM-sync-unit-36062] -message = "fail to handle shard ddl %v in optimistic mode, because schema conflict detected" +message = "fail to handle shard ddl %v in optimistic mode, because schema conflict detected, conflict error: %s" description = "" workaround = "Please use show-ddl-locks command for more details." tags = ["internal", "high"] @@ -1918,6 +1918,12 @@ description = "" workaround = "Please confirm your DDL statement is correct and needed. For TiDB compatible DDL, see https://docs.pingcap.com/tidb/stable/mysql-compatibility#ddl. You can use `handle-error` command to skip or replace the DDL or add a binlog filter rule to ignore it if the DDL is not needed." tags = ["internal", "high"] +[error.DM-sync-unit-36068] +message = "`%s` statement not supported in %s mode" +description = "" +workaround = "" +tags = ["internal", "high"] + [error.DM-dm-master-38001] message = "nil request not valid" description = "" @@ -2242,6 +2248,12 @@ description = "" workaround = "" tags = ["internal", "high"] +[error.DM-dm-master-38055] +message = "table-info-before not exist in optimistic ddls: %v" +description = "" +workaround = "" +tags = ["internal", "high"] + [error.DM-dm-worker-40001] message = "parse dm-worker config flag set" description = "" diff --git a/go.sum b/go.sum index 5c07f02eaa..3585514f6d 100644 --- a/go.sum +++ b/go.sum @@ -178,6 +178,7 @@ github.com/coocood/rtutil v0.0.0-20190304133409-c84515f646f2 h1:NnLfQ77q0G4k2Of2 github.com/coocood/rtutil v0.0.0-20190304133409-c84515f646f2/go.mod h1:7qG7YFnOALvsx6tKTNmQot8d7cGFXM9TidzvRFLWYwM= github.com/coreos/bbolt v1.3.2/go.mod h1:iRUV2dpdMOn7Bo10OQBFzIJO9kkE559Wcmn+qkEiiKk= github.com/coreos/etcd v3.3.10+incompatible/go.mod h1:uF7uidLiAD3TWHmW31ZFd/JWoc32PjwdhPthX9715RE= +github.com/coreos/etcd v3.3.12+incompatible h1:pAWNwdf7QiT1zfaWyqCtNZQWCLByQyA3JrSQyuYAqnQ= github.com/coreos/etcd v3.3.12+incompatible/go.mod h1:uF7uidLiAD3TWHmW31ZFd/JWoc32PjwdhPthX9715RE= github.com/coreos/go-etcd v2.0.0+incompatible/go.mod h1:Jez6KQU2B/sWsbdaef3ED8NzMklzPG4d5KIOhIy30Tk= github.com/coreos/go-semver v0.2.0/go.mod h1:nnelYz7RCh+5ahJtPPxZlU+153eP4D4r3EedlOD2RNk= @@ -940,7 +941,6 @@ github.com/shurcooL/sanitized_anchor_name v1.0.0/go.mod h1:1NzhyTcUVG4SuEtjjoZeV github.com/shurcooL/vfsgen v0.0.0-20181020040650-a97a25d856ca/go.mod h1:TrYk7fJVaAttu97ZZKrO9UbRa8izdowaMIZcxYMbVaw= github.com/shurcooL/vfsgen v0.0.0-20181202132449-6a9ea43bcacd h1:ug7PpSOB5RBPK1Kg6qskGBoP3Vnj/aNYFTznWvlkGo0= github.com/shurcooL/vfsgen v0.0.0-20181202132449-6a9ea43bcacd/go.mod h1:TrYk7fJVaAttu97ZZKrO9UbRa8izdowaMIZcxYMbVaw= -github.com/siddontang/go-log v0.0.0-20180807004314-8d05993dda07 h1:oI+RNwuC9jF2g2lP0u0cVEEZrc/AYBCuFdvwrLWM/6Q= github.com/siddontang/go-log v0.0.0-20180807004314-8d05993dda07/go.mod h1:yFdBgwXP24JziuRl2NMUahT7nGLNOKi1SIiFxMttVD4= github.com/siddontang/go-log v0.0.0-20190221022429-1e957dd83bed h1:KMgQoLJGCq1IoZpLZE3AIffh9veYWoVlsvA4ib55TMM= github.com/siddontang/go-log v0.0.0-20190221022429-1e957dd83bed/go.mod h1:yFdBgwXP24JziuRl2NMUahT7nGLNOKi1SIiFxMttVD4= diff --git a/loader/loader.go b/loader/loader.go index 50a2891835..8be524421d 100644 --- a/loader/loader.go +++ b/loader/loader.go @@ -18,7 +18,6 @@ import ( "bytes" "context" "encoding/hex" - "fmt" "io" "io/ioutil" "os" @@ -38,6 +37,7 @@ import ( "github.com/pingcap/dm/pkg/log" "github.com/pingcap/dm/pkg/terror" "github.com/pingcap/dm/pkg/utils" + "golang.org/x/sync/errgroup" "github.com/pingcap/errors" "github.com/pingcap/failpoint" @@ -164,7 +164,7 @@ func (w *Worker) run(ctx context.Context, fileJobQueue chan *fileJob, runFatalCh } sqls := make([]string, 0, 3) - sqls = append(sqls, fmt.Sprintf("USE `%s`;", unescapePercent(job.schema, w.logger))) + sqls = append(sqls, "USE `"+unescapePercent(job.schema, w.logger)+"`;") sqls = append(sqls, job.sql) offsetSQL := w.checkPoint.GenSQL(job.file, job.offset) @@ -1137,7 +1137,7 @@ func (l *Loader) restoreStructure(ctx context.Context, conn *DBConn, sqlFile str dstSchema, dstTable := fetchMatchedLiteral(tctx, l.tableRouter, schema, table) // for table if table != "" { - sqls = append(sqls, fmt.Sprintf("USE `%s`;", unescapePercent(dstSchema, l.logger))) + sqls = append(sqls, "USE `"+unescapePercent(dstSchema, l.logger)+"`;") query = renameShardingTable(query, table, dstTable, ansiquote) } else { query = renameShardingSchema(query, schema, dstSchema, ansiquote) @@ -1188,82 +1188,223 @@ func fetchMatchedLiteral(ctx *tcontext.Context, router *router.Table, schema, ta return targetSchema, targetTable } -func (l *Loader) restoreData(ctx context.Context) error { - begin := time.Now() +// `restore Schema Job` present a data structure of schema restoring job +type restoreSchemaJob struct { + loader *Loader + session *DBConn + database string // database name + table string // table name, empty if it's a schema of database + filepath string // file path of dumpped schema file +} - baseConn, err := l.toDB.GetBaseConn(ctx) - if err != nil { - return err +// `jobQueue` of schema restoring which (only) support consumptions concurrently +type jobQueue struct { + ctx context.Context + msgq chan *restoreSchemaJob // job message queue channel + consumerCount int // count of consumers + eg *errgroup.Group // err wait group of consumer's go-routines +} + +// `newJobQueue` consturct a jobQueue +func newJobQueue(ctx context.Context, consumerCount, length int) *jobQueue { + eg, selfCtx := errgroup.WithContext(ctx) + return &jobQueue{ + ctx: selfCtx, + msgq: make(chan *restoreSchemaJob, length), + consumerCount: consumerCount, + eg: eg, } - defer func() { - err2 := l.toDB.CloseBaseConn(baseConn) - if err2 != nil { - l.logger.Warn("fail to close connection", zap.Error(err2)) - } - }() +} - dbConn := &DBConn{ - cfg: l.cfg, - baseConn: baseConn, - resetBaseConnFn: func(*tcontext.Context, *conn.BaseConn) (*conn.BaseConn, error) { - return nil, terror.ErrDBBadConn.Generate("bad connection error restoreData") - }, +// `push` will append a job to the queue +func (q *jobQueue) push(job *restoreSchemaJob) error { + var err error + select { + case <-q.ctx.Done(): + err = q.ctx.Err() + case q.msgq <- job: } + return terror.WithScope(err, terror.ScopeInternal) +} - dispatchMap := make(map[string]*fileJob) +// `close` wait jobs done and close queue forever +func (q *jobQueue) close() error { + // queue is closing + close(q.msgq) + // wait until go-routines of consumption was exited + return q.eg.Wait() +} - // restore db in sort +// `startConsumers` run multiple go-routines of job consumption with user defined handler +func (q *jobQueue) startConsumers(handler func(ctx context.Context, job *restoreSchemaJob) error) { + for i := 0; i < q.consumerCount; i++ { + q.eg.Go(func() error { + var err error + var session *DBConn + consumeLoop: + for { + select { + case <-q.ctx.Done(): + err = q.ctx.Err() + break consumeLoop + case job, active := <-q.msgq: + if !active { + break consumeLoop + } + // test condition for `job.session` means db session still could be controlled outside, + // it's used in unit test for now. + if session == nil && job.session == nil { + baseConn, err := job.loader.toDB.GetBaseConn(q.ctx) + if err != nil { + return err + } + defer func(baseConn *conn.BaseConn) { + err := job.loader.toDB.CloseBaseConn(baseConn) + if err != nil { + job.loader.logger.Warn("fail to close connection", zap.Error(err)) + } + }(baseConn) + session = &DBConn{ + cfg: job.loader.cfg, + baseConn: baseConn, + resetBaseConnFn: func(*tcontext.Context, *conn.BaseConn) (*conn.BaseConn, error) { + return nil, terror.ErrDBBadConn.Generate("bad connection error restoreData") + }, + } + } + if job.session == nil { + job.session = session + } + err = handler(q.ctx, job) + if err != nil { + break consumeLoop + } + } + } + return err + }) + } +} + +func (l *Loader) restoreData(ctx context.Context) error { + begin := time.Now() + dispatchMap := make(map[string]*fileJob) + concurrency := l.cfg.PoolSize + // `for v := range map` would present random order + // `dbs` array keep same order for restore schema job generating + var err error dbs := make([]string, 0, len(l.db2Tables)) for db := range l.db2Tables { dbs = append(dbs, db) } - tctx := tcontext.NewContext(ctx, l.logger) + // run consumers of restore database schema queue + dbRestoreQueue := newJobQueue(ctx, concurrency, concurrency /** length of queue */) + dbRestoreQueue.startConsumers(func(ctx context.Context, job *restoreSchemaJob) error { + // restore database schema + job.loader.logger.Info("start to create schema", zap.String("schema file", job.filepath)) + err := job.loader.restoreSchema(ctx, job.session, job.filepath, job.database) + if err != nil { + return err + } + job.loader.logger.Info("finish to create schema", zap.String("schema file", job.filepath)) + return nil + }) + + // push database schema restoring jobs to the queue for _, db := range dbs { - tables := l.db2Tables[db] + schemaFile := l.cfg.Dir + "/" + db + "-schema-create.sql" // cache friendly + err = dbRestoreQueue.push(&restoreSchemaJob{ + loader: l, + database: db, + table: "", + filepath: schemaFile, + }) + if err != nil { + break + } + } + + // check producing error + if err != nil { + runtimeErr := dbRestoreQueue.close() + if errors.ErrorEqual(err, context.Canceled) { + err = runtimeErr + } + return err + } + // wait whole task done & close queue + err = dbRestoreQueue.close() + if err != nil { + return err + } - // create db - dbFile := fmt.Sprintf("%s/%s-schema-create.sql", l.cfg.Dir, db) - l.logger.Info("start to create schema", zap.String("schema file", dbFile)) - err = l.restoreSchema(ctx, dbConn, dbFile, db) + // run consumers of restore table schema queue + tblRestoreQueue := newJobQueue(ctx, concurrency, concurrency /** length of queue */) + tblRestoreQueue.startConsumers(func(ctx context.Context, job *restoreSchemaJob) error { + job.loader.logger.Info("start to create table", zap.String("table file", job.filepath)) + err := job.loader.restoreTable(ctx, job.session, job.filepath, job.database, job.table) if err != nil { return err } - l.logger.Info("finish to create schema", zap.String("schema file", dbFile)) + job.loader.logger.Info("finish to create table", zap.String("table file", job.filepath)) + return nil + }) - tnames := make([]string, 0, len(tables)) - for t := range tables { - tnames = append(tnames, t) - } - for _, table := range tnames { - dataFiles := tables[table] - tableFile := fmt.Sprintf("%s/%s.%s-schema.sql", l.cfg.Dir, db, table) + // push table schema restoring jobs to the queue +tblSchemaLoop: + for _, db := range dbs { + for table := range l.db2Tables[db] { + schemaFile := l.cfg.Dir + "/" + db + "." + table + "-schema.sql" // cache friendly if _, ok := l.tableInfos[tableName(db, table)]; !ok { - l.tableInfos[tableName(db, table)], err = parseTable(tctx, l.tableRouter, db, table, tableFile, l.cfg.LoaderConfig.SQLMode) + l.tableInfos[tableName(db, table)], err = parseTable(tctx, l.tableRouter, db, table, schemaFile, l.cfg.LoaderConfig.SQLMode) if err != nil { - return terror.Annotatef(err, "parse table %s/%s", db, table) + err = terror.Annotatef(err, "parse table %s/%s", db, table) + break tblSchemaLoop } } - if l.checkPoint.IsTableFinished(db, table) { l.logger.Info("table has finished, skip it.", zap.String("schema", db), zap.String("table", table)) continue } - - // create table - l.logger.Info("start to create table", zap.String("table file", tableFile)) - err := l.restoreTable(ctx, dbConn, tableFile, db, table) + err = tblRestoreQueue.push(&restoreSchemaJob{ + loader: l, + database: db, + table: table, + filepath: schemaFile, + }) if err != nil { - return err + break tblSchemaLoop } - l.logger.Info("finish to create table", zap.String("table file", tableFile)) + } + } + + // check producing error + if err != nil { + runtimeErr := tblRestoreQueue.close() + if errors.ErrorEqual(err, context.Canceled) { + err = runtimeErr + } + return err + } + // wait whole task done & close queue + err = tblRestoreQueue.close() + if err != nil { + return err + } + + // all schemas was restored + l.logger.Info("finish to create tables", zap.Duration("cost time", time.Since(begin))) + // generate restore table data file job + for _, db := range dbs { + table2DataFileMap := l.db2Tables[db] + for table := range table2DataFileMap { restoringFiles := l.checkPoint.GetRestoringFileInfo(db, table) l.logger.Debug("restoring table data", zap.String("schema", db), zap.String("table", table), zap.Reflect("data files", restoringFiles)) - info := l.tableInfos[tableName(db, table)] - for _, file := range dataFiles { + for _, file := range table2DataFileMap[table] { select { case <-ctx.Done(): l.logger.Warn("stop generate data file job", log.ShortError(ctx.Err())) @@ -1271,7 +1412,6 @@ func (l *Loader) restoreData(ctx context.Context) error { default: // do nothing } - l.logger.Debug("dispatch data file", zap.String("schema", db), zap.String("table", table), zap.String("data file", file)) offset := int64(uninitializedOffset) @@ -1279,19 +1419,16 @@ func (l *Loader) restoreData(ctx context.Context) error { if ok { offset = posSet[0] } - - j := &fileJob{ + dispatchMap[db+"_"+table+"_"+file] = &fileJob{ schema: db, table: table, dataFile: file, offset: offset, - info: info, + info: l.tableInfos[tableName(db, table)], } - dispatchMap[fmt.Sprintf("%s_%s_%s", db, table, file)] = j } } } - l.logger.Info("finish to create tables", zap.Duration("cost time", time.Since(begin))) // a simple and naive approach to dispatch files randomly based on the feature of golang map(range by random) for _, j := range dispatchMap { diff --git a/loader/loader_test.go b/loader/loader_test.go new file mode 100644 index 0000000000..e124a30398 --- /dev/null +++ b/loader/loader_test.go @@ -0,0 +1,82 @@ +// Copyright 2021 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package loader + +import ( + "context" + + . "github.com/pingcap/check" + "github.com/pingcap/errors" +) + +var _ = Suite(&testLoaderSuite{}) + +type testLoaderSuite struct{} + +func (*testLoaderSuite) TestJobQueue(c *C) { + procedure := func(ctx context.Context, jobsCount int, handler func(ctx context.Context, job *restoreSchemaJob) error) error { + jobQueue := newJobQueue(ctx, 16, 16) + jobQueue.startConsumers(handler) + for i := 0; i < jobsCount; i++ { + job := &restoreSchemaJob{ + session: &DBConn{}, // just for testing + } + if i == jobsCount/2 { + job.database = "error" + } + err := jobQueue.push(job) + if err != nil { + runtimeErr := jobQueue.close() + if errors.ErrorEqual(err, context.Canceled) { + err = runtimeErr + } + return err + } + } + return jobQueue.close() + } + + injectErr := errors.New("random injected error") + cases := []struct { + ctx context.Context + jobsCount int + handler func(ctx context.Context, job *restoreSchemaJob) error + exceptedErr error + }{ + { + ctx: context.Background(), + jobsCount: 128, + handler: func(ctx context.Context, job *restoreSchemaJob) error { + if job.database == "error" { + return injectErr + } + return nil + }, + exceptedErr: injectErr, + }, + { + ctx: context.Background(), + jobsCount: 128, + handler: func(ctx context.Context, job *restoreSchemaJob) error { + return nil + }, + exceptedErr: nil, + }, + } + + for _, testcase := range cases { + err := procedure(testcase.ctx, testcase.jobsCount, testcase.handler) + c.Assert(err, Equals, testcase.exceptedErr) + } +} diff --git a/loader/util.go b/loader/util.go index 5069778618..f1a8855fea 100644 --- a/loader/util.go +++ b/loader/util.go @@ -123,7 +123,7 @@ func getDBAndTableFromFilename(filename string) (string, string, error) { } fields := strings.Split(filename[:idx], ".") if len(fields) != 2 && len(fields) != 3 { - return "", "", fmt.Errorf("%s doesn't have correct `.` seperator", filename) + return "", "", fmt.Errorf("%s doesn't have correct `.` separator", filename) } return fields[0], fields[1], nil } diff --git a/loader/util_test.go b/loader/util_test.go index 1d399ae65c..cc5bc4d0b5 100644 --- a/loader/util_test.go +++ b/loader/util_test.go @@ -110,7 +110,7 @@ func (t *testUtilSuite) TestGetDBAndTableFromFilename(c *C) { {"sqldb.tbl.0.sql", "sqldb", "tbl", ""}, {"db.tbl.sql0.sql", "db", "tbl", ""}, {"db.tbl.0", "", "", ".*doesn't have a `.sql` suffix.*"}, - {"db.sql", "", "", ".*doesn't have correct `.` seperator.*"}, + {"db.sql", "", "", ".*doesn't have correct `.` separator.*"}, {"db.0.sql", "db", "0", ""}, // treat `0` as the table name. } diff --git a/monitoring/dashboards/dashboard.go b/monitoring/dashboards/dashboard.go index a71db5c8a3..ac6f0421b4 100644 --- a/monitoring/dashboards/dashboard.go +++ b/monitoring/dashboards/dashboard.go @@ -84,7 +84,7 @@ func filterDashboard(str string, dashboard string, title string) string { str = strings.ReplaceAll(str, fmt.Sprintf("${%s}", datasource), datasourceName) } - // delete input defination + // delete input definition if gjson.Get(str, "__inputs").Exists() { str, err = sjson.Delete(str, "__inputs") checkErr(err, "delete path failed") diff --git a/pkg/etcdutil/etcdutil.go b/pkg/etcdutil/etcdutil.go index 48bc1eff7d..6d8c9b9b51 100644 --- a/pkg/etcdutil/etcdutil.go +++ b/pkg/etcdutil/etcdutil.go @@ -108,7 +108,7 @@ func DoOpsInOneTxnWithRetry(cli *clientv3.Client, ops ...clientv3.Op) (*clientv3 ret, _, err := etcdDefaultTxnStrategy.Apply(tctx, etcdDefaultTxnRetryParam, func(t *tcontext.Context) (ret interface{}, err error) { resp, err := cli.Txn(ctx).Then(ops...).Commit() if err != nil { - return nil, err + return nil, errors.Trace(err) } return resp, nil }) diff --git a/pkg/ha/bound.go b/pkg/ha/bound.go index 8b8e32dd28..6bd74487c2 100644 --- a/pkg/ha/bound.go +++ b/pkg/ha/bound.go @@ -47,7 +47,7 @@ type SourceBound struct { // only used to report to the caller of the watcher, do not marsh it. // if it's true, it means the bound has been deleted in etcd. IsDeleted bool `json:"-"` - // record the etcd ModRevision of this bound + // record the etcd Revision of this bound Revision int64 `json:"-"` } @@ -105,7 +105,7 @@ func PutSourceBound(cli *clientv3.Client, bounds ...SourceBound) (int64, error) func DeleteSourceBound(cli *clientv3.Client, workers ...string) (int64, error) { ops := make([]clientv3.Op, 0, len(workers)) for _, worker := range workers { - ops = append(ops, deleteSourceBoundOp(worker)) + ops = append(ops, deleteSourceBoundOp(worker)...) } _, rev, err := etcdutil.DoOpsInOneTxnWithRetry(cli, ops...) return rev, err @@ -113,14 +113,19 @@ func DeleteSourceBound(cli *clientv3.Client, workers ...string) (int64, error) { // ReplaceSourceBound deletes an old bound and puts a new bound in one transaction, so a bound source will not become // unbound because of failing halfway -func ReplaceSourceBound(cli *clientv3.Client, source, oldWorker, newWorker string) (int64, error) { - ops := make([]clientv3.Op, 0, 3) - ops = append(ops, deleteSourceBoundOp(oldWorker)) - op, err := putSourceBoundOp(NewSourceBound(source, newWorker)) +// TODO: remove replace relay parameter because we didn't plan it in future +func ReplaceSourceBound(cli *clientv3.Client, source, oldWorker, newWorker string, replaceRelay bool) (int64, error) { + deleteOps := deleteSourceBoundOp(oldWorker) + putOps, err := putSourceBoundOp(NewSourceBound(source, newWorker)) if err != nil { return 0, err } - ops = append(ops, op...) + ops := make([]clientv3.Op, 0, len(deleteOps)+len(putOps)) + ops = append(ops, deleteOps...) + ops = append(ops, putOps...) + if replaceRelay { + ops = append(ops, putRelayConfigOp(newWorker, source)) + } _, rev, err := etcdutil.DoOpsInOneTxnWithRetry(cli, ops...) return rev, err } @@ -351,8 +356,12 @@ func sourceBoundFromResp(worker string, resp *clientv3.GetResponse) (map[string] } // deleteSourceBoundOp returns a DELETE etcd operation for the bound relationship of the specified DM-worker. -func deleteSourceBoundOp(worker string) clientv3.Op { - return clientv3.OpDelete(common.UpstreamBoundWorkerKeyAdapter.Encode(worker)) +func deleteSourceBoundOp(worker string) []clientv3.Op { + // TODO: move this to stop-relay, and wait until worker has disabled relay + return []clientv3.Op{ + clientv3.OpDelete(common.UpstreamBoundWorkerKeyAdapter.Encode(worker)), + clientv3.OpDelete(common.UpstreamRelayWorkerKeyAdapter.Encode(worker)), + } } // deleteLastSourceBoundOp returns a DELETE etcd operation for the last bound relationship of the specified DM-worker. diff --git a/pkg/ha/keepalive.go b/pkg/ha/keepalive.go index f2c0fdd3d0..be4f2e7035 100644 --- a/pkg/ha/keepalive.go +++ b/pkg/ha/keepalive.go @@ -29,8 +29,8 @@ import ( ) var ( - // CurrentKeepAliveTTL may be assigned to KeepAliveTTL or RelayKeepAliveTTL - CurrentKeepAliveTTL int64 + // currentKeepAliveTTL may be assigned to KeepAliveTTL or RelayKeepAliveTTL + currentKeepAliveTTL int64 // KeepAliveUpdateCh is used to notify keepalive TTL changing, in order to let watcher not see a DELETE of old key KeepAliveUpdateCh = make(chan int64, 10) ) @@ -84,8 +84,8 @@ func KeepAlive(ctx context.Context, cli *clientv3.Client, workerName string, kee for len(KeepAliveUpdateCh) > 0 { keepAliveTTL = <-KeepAliveUpdateCh } - // though in regular routine there's no concurrent KeepAlive, we need to handle tests - atomic.StoreInt64(&CurrentKeepAliveTTL, keepAliveTTL) + // a test concurrently call KeepAlive though in normal running we don't do that + atomic.StoreInt64(¤tKeepAliveTTL, keepAliveTTL) k := common.WorkerKeepAliveKeyAdapter.Encode(workerName) workerEventJSON, err := WorkerEvent{ @@ -145,6 +145,11 @@ func KeepAlive(ctx context.Context, cli *clientv3.Client, workerName string, kee keepAliveCancel() // make go vet happy return nil case newTTL := <-KeepAliveUpdateCh: + if newTTL == currentKeepAliveTTL { + log.L().Info("ignore same keepalive TTL change", zap.Int64("TTL", newTTL)) + continue + } + // create a new lease with new TTL, and overwrite original KV oldLeaseID := leaseID leaseID, err = grantAndPutKV(k, workerEventJSON, newTTL) @@ -161,6 +166,7 @@ func KeepAlive(ctx context.Context, cli *clientv3.Client, workerName string, kee keepAliveCancel() // make go vet happy return err } + currentKeepAliveTTL = newTTL log.L().Info("dynamically changed keepalive TTL to", zap.Int64("ttl in seconds", newTTL)) // after new keepalive is succeed, we cancel the old keepalive diff --git a/pkg/ha/ops.go b/pkg/ha/ops.go index 03cced27b8..3570df74a9 100644 --- a/pkg/ha/ops.go +++ b/pkg/ha/ops.go @@ -21,10 +21,11 @@ import ( "github.com/pingcap/dm/pkg/etcdutil" ) -// PutRelayStageSourceBound puts the following data in one txn. +// PutRelayStageRelayConfigSourceBound puts the following data in one txn. // - relay stage. +// - relay config for a worker // - source bound relationship. -func PutRelayStageSourceBound(cli *clientv3.Client, stage Stage, bound SourceBound) (int64, error) { +func PutRelayStageRelayConfigSourceBound(cli *clientv3.Client, stage Stage, bound SourceBound) (int64, error) { ops1, err := putRelayStageOp(stage) if err != nil { return 0, err @@ -33,9 +34,11 @@ func PutRelayStageSourceBound(cli *clientv3.Client, stage Stage, bound SourceBou if err != nil { return 0, err } - ops := make([]clientv3.Op, 0, 3) + op3 := putRelayConfigOp(bound.Worker, bound.Source) + ops := make([]clientv3.Op, 0, len(ops1)+len(op2)+1) ops = append(ops, ops1...) ops = append(ops, op2...) + ops = append(ops, op3) _, rev, err := etcdutil.DoOpsInOneTxnWithRetry(cli, ops...) return rev, err } @@ -49,7 +52,13 @@ func DeleteSourceCfgRelayStageSourceBound(cli *clientv3.Client, source, worker s relayStageOp := deleteRelayStageOp(source) sourceBoundOp := deleteSourceBoundOp(worker) lastBoundOp := deleteLastSourceBoundOp(worker) - _, rev, err := etcdutil.DoOpsInOneTxnWithRetry(cli, sourceCfgOp, relayStageOp, sourceBoundOp, lastBoundOp) + ops := make([]clientv3.Op, 0, 3+len(sourceBoundOp)) + ops = append(ops, sourceCfgOp) + ops = append(ops, relayStageOp) + ops = append(ops, sourceBoundOp...) + ops = append(ops, lastBoundOp) + + _, rev, err := etcdutil.DoOpsInOneTxnWithRetry(cli, ops...) return rev, err } diff --git a/pkg/ha/ops_test.go b/pkg/ha/ops_test.go index af85bee5a9..3ad51caaff 100644 --- a/pkg/ha/ops_test.go +++ b/pkg/ha/ops_test.go @@ -49,7 +49,7 @@ func (t *testForEtcd) TestOpsEtcd(c *C) { c.Assert(subtaskCfg2.Adjust(true), IsNil) // put relay stage and source bound. - rev1, err := PutRelayStageSourceBound(etcdTestCli, relayStage, bound) + rev1, err := PutRelayStageRelayConfigSourceBound(etcdTestCli, relayStage, bound) c.Assert(err, IsNil) c.Assert(rev1, Greater, int64(0)) // put source config. diff --git a/pkg/ha/relay.go b/pkg/ha/relay.go index 9103ef88ea..2baaf8f985 100644 --- a/pkg/ha/relay.go +++ b/pkg/ha/relay.go @@ -18,6 +18,7 @@ import ( "time" "go.etcd.io/etcd/clientv3" + "go.etcd.io/etcd/mvcc/mvccpb" "go.uber.org/zap" "github.com/pingcap/dm/dm/common" @@ -27,8 +28,19 @@ import ( "github.com/pingcap/dm/pkg/terror" ) +// RelaySource represents the bound relationship between the DM-worker instance and its upstream relay source. +type RelaySource struct { + Source string + // only used to report to the caller of the watcher, do not marsh it. + // if it's true, it means the bound has been deleted in etcd. + IsDeleted bool + // record the etcd ModRevision of this bound + Revision int64 +} + // PutRelayConfig puts the relay config for given workers. // k/v: worker-name -> source-id. +// TODO: let caller wait until worker has enabled relay func PutRelayConfig(cli *clientv3.Client, source string, workers ...string) (int64, error) { ops := make([]clientv3.Op, 0, len(workers)) for _, worker := range workers { @@ -140,3 +152,54 @@ func putRelayConfigOp(worker, source string) clientv3.Op { func deleteRelayConfigOp(worker string) clientv3.Op { return clientv3.OpDelete(common.UpstreamRelayWorkerKeyAdapter.Encode(worker)) } + +// WatchRelayConfig watches PUT & DELETE operations for the relay relationship of the specified DM-worker. +// For the DELETE operations, it returns an nil source config. +func WatchRelayConfig(ctx context.Context, cli *clientv3.Client, + worker string, revision int64, outCh chan<- RelaySource, errCh chan<- error) { + ch := cli.Watch(ctx, common.UpstreamRelayWorkerKeyAdapter.Encode(worker), clientv3.WithRev(revision)) + + for { + select { + case <-ctx.Done(): + return + case resp, ok := <-ch: + if !ok { + return + } + if resp.Canceled { + // TODO(csuzhangxc): do retry here. + if resp.Err() != nil { + select { + case errCh <- resp.Err(): + case <-ctx.Done(): + } + } + return + } + + for _, ev := range resp.Events { + var bound RelaySource + switch ev.Type { + case mvccpb.PUT: + bound.Source = string(ev.Kv.Value) + bound.IsDeleted = false + case mvccpb.DELETE: + bound.IsDeleted = true + default: + // this should not happen. + log.L().Error("unsupported etcd event type", zap.Reflect("kv", ev.Kv), zap.Reflect("type", ev.Type)) + continue + } + bound.Revision = ev.Kv.ModRevision + + select { + case outCh <- bound: + case <-ctx.Done(): + return + } + } + } + } + +} diff --git a/pkg/ha/stage.go b/pkg/ha/stage.go index 584e61fb57..9cf39cea69 100644 --- a/pkg/ha/stage.go +++ b/pkg/ha/stage.go @@ -38,7 +38,7 @@ type Stage struct { // only used to report to the caller of the watcher, do not marsh it. // if it's true, it means the stage has been deleted in etcd. IsDeleted bool `json:"-"` - // record the etcd ModRevision of this Stage + // record the etcd Revision of this Stage Revision int64 `json:"-"` } diff --git a/pkg/shardddl/optimism/column.go b/pkg/shardddl/optimism/column.go new file mode 100644 index 0000000000..bd0f4e7f5f --- /dev/null +++ b/pkg/shardddl/optimism/column.go @@ -0,0 +1,106 @@ +// Copyright 2021 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package optimism + +import ( + "go.etcd.io/etcd/clientv3" + + "github.com/pingcap/dm/dm/common" + "github.com/pingcap/dm/pkg/etcdutil" +) + +// GetAllDroppedColumns gets the all partially dropped columns. +// return lockID -> column-name -> source-id -> upstream-schema-name -> upstream-table-name +func GetAllDroppedColumns(cli *clientv3.Client) (map[string]map[string]map[string]map[string]map[string]struct{}, int64, error) { + colm := make(map[string]map[string]map[string]map[string]map[string]struct{}) + op := clientv3.OpGet(common.ShardDDLOptimismDroppedColumnsKeyAdapter.Path(), clientv3.WithPrefix()) + respTxn, rev, err := etcdutil.DoOpsInOneTxnWithRetry(cli, op) + if err != nil { + return colm, 0, err + } + resp := respTxn.Responses[0].GetResponseRange() + + if resp.Count > 0 { + for _, kv := range resp.Kvs { + keys, err := common.ShardDDLOptimismDroppedColumnsKeyAdapter.Decode(string(kv.Key)) + if err != nil { + return colm, 0, err + } + task := keys[0] + downSchema := keys[1] + downTable := keys[2] + column := keys[3] + source := keys[4] + upSchema := keys[5] + upTable := keys[6] + info := NewInfo(task, source, upSchema, upTable, downSchema, downTable, nil, nil, nil) + lockID := genDDLLockID(info) + if _, ok := colm[lockID]; !ok { + colm[lockID] = make(map[string]map[string]map[string]map[string]struct{}) + } + if _, ok := colm[lockID][column]; !ok { + colm[lockID][column] = make(map[string]map[string]map[string]struct{}) + } + if _, ok := colm[lockID][column][source]; !ok { + colm[lockID][column][source] = make(map[string]map[string]struct{}) + } + if _, ok := colm[lockID][column][source][upSchema]; !ok { + colm[lockID][column][source][upSchema] = make(map[string]struct{}) + } + colm[lockID][column][source][upSchema][upTable] = struct{}{} + } + } + return colm, rev, nil +} + +// PutDroppedColumn puts the partially dropped column name into ectd. +// When we drop a column, we save this column's name in etcd. +func PutDroppedColumn(cli *clientv3.Client, info Info, column string) (rev int64, putted bool, err error) { + key := common.ShardDDLOptimismDroppedColumnsKeyAdapter.Encode( + info.Task, info.DownSchema, info.DownTable, column, info.Source, info.UpSchema, info.UpTable) + + op := clientv3.OpPut(key, "") + + resp, rev, err := etcdutil.DoOpsInOneTxnWithRetry(cli, op) + if err != nil { + return 0, false, err + } + return rev, resp.Succeeded, nil +} + +// DeleteDroppedColumns tries to delete the partially dropped columns for the specified lock ID. +// Only when this column is fully dropped in downstream database, +// in other words, **we receive a `Done` operation from dm-worker**, +// we can delete this column's name from the etcd. +func DeleteDroppedColumns(cli *clientv3.Client, task, downSchema, downTable string, columns ...string) (rev int64, deleted bool, err error) { + ops := make([]clientv3.Op, 0, len(columns)) + for _, col := range columns { + ops = append(ops, deleteDroppedColumnByColumnOp(task, downSchema, downTable, col)) + } + resp, rev, err := etcdutil.DoOpsInOneTxnWithRetry(cli, ops...) + if err != nil { + return 0, false, err + } + return rev, resp.Succeeded, nil +} + +// deleteDroppedColumnOp returns a DELETE etcd operation for the specified task and column name. +func deleteDroppedColumnByColumnOp(task, downSchema, downTable, column string) clientv3.Op { + return clientv3.OpDelete(common.ShardDDLOptimismDroppedColumnsKeyAdapter.Encode(task, downSchema, downTable, column), clientv3.WithPrefix()) +} + +// deleteDroppedColumnsByLockOp returns a DELETE etcd operation for the specified lock. +func deleteDroppedColumnsByLockOp(task, downSchema, downTable string) clientv3.Op { + return clientv3.OpDelete(common.ShardDDLOptimismDroppedColumnsKeyAdapter.Encode(task, downSchema, downTable), clientv3.WithPrefix()) +} diff --git a/pkg/shardddl/optimism/column_test.go b/pkg/shardddl/optimism/column_test.go new file mode 100644 index 0000000000..6f05e53606 --- /dev/null +++ b/pkg/shardddl/optimism/column_test.go @@ -0,0 +1,79 @@ +// Copyright 2021 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package optimism + +import . "github.com/pingcap/check" + +type testColumn struct{} + +var _ = Suite(&testColumn{}) + +func (t *testColumn) TestColumnETCD(c *C) { + defer clearTestInfoOperation(c) + + var ( + task = "test" + downSchema = "shardddl" + downTable = "tb" + source1 = "mysql-replica-1" + source2 = "mysql-replica-2" + upSchema1 = "shardddl1" + upTable1 = "tb1" + upSchema2 = "shardddl2" + upTable2 = "tb2" + info1 = NewInfo(task, source1, upSchema1, upTable1, downSchema, downTable, nil, nil, nil) + info2 = NewInfo(task, source1, upSchema2, upTable2, downSchema, downTable, nil, nil, nil) + info3 = NewInfo(task, source2, upSchema1, upTable1, downSchema, downTable, nil, nil, nil) + lockID = genDDLLockID(info1) + ) + rev1, putted, err := PutDroppedColumn(etcdTestCli, info1, "a") + c.Assert(err, IsNil) + c.Assert(putted, IsTrue) + rev2, putted, err := PutDroppedColumn(etcdTestCli, info1, "b") + c.Assert(err, IsNil) + c.Assert(putted, IsTrue) + c.Assert(rev2, Greater, rev1) + rev3, putted, err := PutDroppedColumn(etcdTestCli, info2, "b") + c.Assert(err, IsNil) + c.Assert(putted, IsTrue) + c.Assert(rev3, Greater, rev2) + rev4, putted, err := PutDroppedColumn(etcdTestCli, info3, "b") + c.Assert(err, IsNil) + c.Assert(putted, IsTrue) + c.Assert(rev4, Greater, rev3) + + expectedColm := map[string]map[string]map[string]map[string]map[string]struct{}{ + lockID: { + "a": {source1: {upSchema1: {upTable1: struct{}{}}}}, + "b": {source1: {upSchema1: {upTable1: struct{}{}}, + upSchema2: {upTable2: struct{}{}}}, + source2: {upSchema1: {upTable1: struct{}{}}}}, + }, + } + colm, rev5, err := GetAllDroppedColumns(etcdTestCli) + c.Assert(err, IsNil) + c.Assert(colm, DeepEquals, expectedColm) + c.Assert(rev5, Equals, rev4) + + rev6, deleted, err := DeleteDroppedColumns(etcdTestCli, task, downSchema, downTable, "b") + c.Assert(err, IsNil) + c.Assert(deleted, IsTrue) + c.Assert(rev6, Greater, rev5) + + delete(expectedColm[lockID], "b") + colm, rev7, err := GetAllDroppedColumns(etcdTestCli) + c.Assert(err, IsNil) + c.Assert(colm, DeepEquals, expectedColm) + c.Assert(rev7, Equals, rev6) +} diff --git a/pkg/shardddl/optimism/info.go b/pkg/shardddl/optimism/info.go index 38c11918d6..9ea6adc16d 100644 --- a/pkg/shardddl/optimism/info.go +++ b/pkg/shardddl/optimism/info.go @@ -54,6 +54,10 @@ type Info struct { // only set it when get/watch from etcd Version int64 `json:"-"` + // only set it when get from etcd + // use for sort infos in recoverlock + Revision int64 `json:"-"` + // use to resolve conflict IgnoreConflict bool `json:"ignore-conflict"` } @@ -132,6 +136,7 @@ func GetAllInfo(cli *clientv3.Client) (map[string]map[string]map[string]map[stri return nil, 0, err2 } info.Version = kv.Version + info.Revision = kv.ModRevision if _, ok := ifm[info.Task]; !ok { ifm[info.Task] = make(map[string]map[string]map[string]Info) @@ -182,6 +187,7 @@ func WatchInfo(ctx context.Context, cli *clientv3.Client, revision int64, case mvccpb.PUT: info, err = infoFromJSON(string(ev.Kv.Value)) info.Version = ev.Kv.Version + info.Revision = ev.Kv.ModRevision case mvccpb.DELETE: info, err = infoFromJSON(string(ev.PrevKv.Value)) info.IsDeleted = true @@ -232,6 +238,7 @@ func ClearTestInfoOperationSchema(cli *clientv3.Client) error { clearInfo := clientv3.OpDelete(common.ShardDDLOptimismInfoKeyAdapter.Path(), clientv3.WithPrefix()) clearOp := clientv3.OpDelete(common.ShardDDLOptimismOperationKeyAdapter.Path(), clientv3.WithPrefix()) clearISOp := clientv3.OpDelete(common.ShardDDLOptimismInitSchemaKeyAdapter.Path(), clientv3.WithPrefix()) - _, err := cli.Txn(context.Background()).Then(clearSource, clearInfo, clearOp, clearISOp).Commit() + clearColumns := clientv3.OpDelete(common.ShardDDLOptimismDroppedColumnsKeyAdapter.Path(), clientv3.WithPrefix()) + _, err := cli.Txn(context.Background()).Then(clearSource, clearInfo, clearOp, clearISOp, clearColumns).Commit() return err } diff --git a/pkg/shardddl/optimism/info_test.go b/pkg/shardddl/optimism/info_test.go index 12b824c5bb..7746e130e9 100644 --- a/pkg/shardddl/optimism/info_test.go +++ b/pkg/shardddl/optimism/info_test.go @@ -128,6 +128,7 @@ func (t *testForEtcd) TestInfoEtcd(c *C) { c.Assert(ifm[task1][source1][upSchema], HasLen, 1) i11WithVer := i11 i11WithVer.Version = 2 + i11WithVer.Revision = rev2 c.Assert(ifm[task1][source1][upSchema][upTable], DeepEquals, i11WithVer) // put another key and get again with 2 info. @@ -141,6 +142,7 @@ func (t *testForEtcd) TestInfoEtcd(c *C) { c.Assert(ifm[task1][source1][upSchema][upTable], DeepEquals, i11WithVer) i12WithVer := i12 i12WithVer.Version = 1 + i12WithVer.Revision = rev4 c.Assert(ifm[task1][source2][upSchema][upTable], DeepEquals, i12WithVer) // start the watcher. @@ -157,20 +159,22 @@ func (t *testForEtcd) TestInfoEtcd(c *C) { // put another key for a different task. // version start from 1 - _, err = PutInfo(etcdTestCli, i21) + rev5, err := PutInfo(etcdTestCli, i21) c.Assert(err, IsNil) infoWithVer := <-wch i21WithVer := i21 i21WithVer.Version = 1 + i21WithVer.Revision = rev5 c.Assert(infoWithVer, DeepEquals, i21WithVer) c.Assert(len(ech), Equals, 0) // put again // version increase - _, err = PutInfo(etcdTestCli, i21) + rev6, err := PutInfo(etcdTestCli, i21) c.Assert(err, IsNil) infoWithVer = <-wch i21WithVer.Version++ + i21WithVer.Revision = rev6 c.Assert(infoWithVer, DeepEquals, i21WithVer) c.Assert(len(ech), Equals, 0) @@ -187,10 +191,11 @@ func (t *testForEtcd) TestInfoEtcd(c *C) { // put again // version reset to 1 - _, err = PutInfo(etcdTestCli, i21) + rev7, err := PutInfo(etcdTestCli, i21) c.Assert(err, IsNil) infoWithVer = <-wch i21WithVer.Version = 1 + i21WithVer.Revision = rev7 c.Assert(infoWithVer, DeepEquals, i21WithVer) c.Assert(len(ech), Equals, 0) @@ -210,8 +215,10 @@ func (t *testForEtcd) TestInfoEtcd(c *C) { c.Assert(ifm, HasKey, task1) c.Assert(ifm, HasKey, task2) c.Assert(ifm[task1], HasLen, 1) + i11WithVer.Revision = ifm[task1][source1][upSchema][upTable].Revision c.Assert(ifm[task1][source1][upSchema][upTable], DeepEquals, i11WithVer) c.Assert(ifm[task2], HasLen, 1) + i21WithVer.Revision = ifm[task2][source1][upSchema][upTable].Revision c.Assert(ifm[task2][source1][upSchema][upTable], DeepEquals, i21WithVer) // watch the deletion for i12. diff --git a/pkg/shardddl/optimism/keeper.go b/pkg/shardddl/optimism/keeper.go index 5c6a9ec67e..72cf2b40d3 100644 --- a/pkg/shardddl/optimism/keeper.go +++ b/pkg/shardddl/optimism/keeper.go @@ -17,6 +17,10 @@ import ( "sort" "sync" + "github.com/pingcap/tidb-tools/pkg/schemacmp" + "go.etcd.io/etcd/clientv3" + + "github.com/pingcap/dm/pkg/terror" "github.com/pingcap/dm/pkg/utils" ) @@ -34,8 +38,38 @@ func NewLockKeeper() *LockKeeper { } } +// 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]struct{}, + lockJoined map[string]schemacmp.Table, + lockTTS map[string][]TargetTable) { + var ( + lock *Lock + ok bool + ) + 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 lock, ok = lk.locks[lockID]; !ok { + lk.locks[lockID] = NewLock(cli, lockID, info.Task, info.DownSchema, info.DownTable, lockJoined[lockID], lockTTS[lockID]) + lock = lk.locks[lockID] + } + lock.tables[info.Source][info.UpSchema][info.UpTable] = schemacmp.Encode(info.TableInfoBefore) + if columns, ok := colm[lockID]; ok { + lock.columns = columns + } + } + } + } + } +} + // TrySync tries to sync the lock. -func (lk *LockKeeper) TrySync(info Info, tts []TargetTable) (string, []string, error) { +func (lk *LockKeeper) TrySync(cli *clientv3.Client, info Info, tts []TargetTable) (string, []string, error) { var ( lockID = genDDLLockID(info) l *Lock @@ -45,8 +79,12 @@ func (lk *LockKeeper) TrySync(info Info, tts []TargetTable) (string, []string, e lk.mu.Lock() defer lk.mu.Unlock() + if info.TableInfoBefore == nil { + return "", nil, terror.ErrMasterOptimisticTableInfoBeforeNotExist.Generate(info.DDLs) + } + if l, ok = lk.locks[lockID]; !ok { - lk.locks[lockID] = NewLock(lockID, info.Task, info.DownSchema, info.DownTable, info.TableInfoBefore, tts) + lk.locks[lockID] = NewLock(cli, lockID, info.Task, info.DownSchema, info.DownTable, schemacmp.Encode(info.TableInfoBefore), tts) l = lk.locks[lockID] } diff --git a/pkg/shardddl/optimism/keeper_test.go b/pkg/shardddl/optimism/keeper_test.go index e1d62423c3..c049197f3e 100644 --- a/pkg/shardddl/optimism/keeper_test.go +++ b/pkg/shardddl/optimism/keeper_test.go @@ -15,8 +15,10 @@ package optimism import ( . "github.com/pingcap/check" + "github.com/pingcap/dm/pkg/utils" "github.com/pingcap/parser" "github.com/pingcap/parser/model" + "github.com/pingcap/tidb-tools/pkg/schemacmp" "github.com/pingcap/tidb/util/mock" ) @@ -57,7 +59,7 @@ func (t *testKeeper) TestLockKeeper(c *C) { ) // lock with 2 sources. - lockID1, newDDLs, err := lk.TrySync(i11, tts1) + lockID1, newDDLs, err := lk.TrySync(etcdTestCli, i11, tts1) c.Assert(err, IsNil) c.Assert(lockID1, Equals, "task1-`foo`.`bar`") c.Assert(newDDLs, DeepEquals, DDLs) @@ -69,7 +71,7 @@ func (t *testKeeper) TestLockKeeper(c *C) { c.Assert(synced, IsFalse) c.Assert(remain, Equals, 1) - lockID1, newDDLs, err = lk.TrySync(i12, tts1) + lockID1, newDDLs, err = lk.TrySync(etcdTestCli, i12, tts1) c.Assert(err, IsNil) c.Assert(lockID1, Equals, "task1-`foo`.`bar`") c.Assert(newDDLs, DeepEquals, DDLs) @@ -81,7 +83,7 @@ func (t *testKeeper) TestLockKeeper(c *C) { c.Assert(remain, Equals, 0) // lock with only 1 source. - lockID2, newDDLs, err := lk.TrySync(i21, tts2) + lockID2, newDDLs, err := lk.TrySync(etcdTestCli, i21, tts2) c.Assert(err, IsNil) c.Assert(lockID2, Equals, "task2-`foo`.`bar`") c.Assert(newDDLs, DeepEquals, DDLs) @@ -150,13 +152,13 @@ func (t *testKeeper) TestLockKeeperMultipleTarget(c *C) { ) // lock for target1. - lockID1, newDDLs, err := lk.TrySync(i11, tts1) + lockID1, newDDLs, err := lk.TrySync(etcdTestCli, i11, tts1) c.Assert(err, IsNil) c.Assert(lockID1, DeepEquals, "test-lock-keeper-multiple-target-`foo`.`bar`") c.Assert(newDDLs, DeepEquals, DDLs) // lock for target2. - lockID2, newDDLs, err := lk.TrySync(i21, tts2) + lockID2, newDDLs, err := lk.TrySync(etcdTestCli, i21, tts2) c.Assert(err, IsNil) c.Assert(lockID2, DeepEquals, "test-lock-keeper-multiple-target-`foo`.`rab`") c.Assert(newDDLs, DeepEquals, DDLs) @@ -178,11 +180,11 @@ func (t *testKeeper) TestLockKeeperMultipleTarget(c *C) { c.Assert(remain, Equals, 1) // sync for two locks. - lockID1, newDDLs, err = lk.TrySync(i12, tts1) + lockID1, newDDLs, err = lk.TrySync(etcdTestCli, i12, tts1) c.Assert(err, IsNil) c.Assert(lockID1, DeepEquals, "test-lock-keeper-multiple-target-`foo`.`bar`") c.Assert(newDDLs, DeepEquals, DDLs) - lockID2, newDDLs, err = lk.TrySync(i22, tts2) + lockID2, newDDLs, err = lk.TrySync(etcdTestCli, i22, tts2) c.Assert(err, IsNil) c.Assert(lockID2, DeepEquals, "test-lock-keeper-multiple-target-`foo`.`rab`") c.Assert(newDDLs, DeepEquals, DDLs) @@ -391,3 +393,73 @@ func (t *testKeeper) TestTargetTablesForTask(c *C) { }), }) } + +func (t *testKeeper) TestRebuildLocksAndTables(c *C) { + defer clearTestInfoOperation(c) + var ( + lk = NewLockKeeper() + task = "task" + source1 = "mysql-replica-1" + source2 = "mysql-replica-2" + upSchema = "foo" + upTable = "bar" + downSchema = "db" + downTable = "tbl" + DDLs1 = []string{"ALTER TABLE bar ADD COLUMN c1 INT"} + DDLs2 = []string{"ALTER TABLE bar DROP COLUMN c1"} + p = parser.New() + se = mock.NewContext() + tblID int64 = 111 + ti0 = createTableInfo(c, p, se, tblID, `CREATE TABLE bar (id INT PRIMARY KEY)`) + ti1 = createTableInfo(c, p, se, tblID, `CREATE TABLE bar (id INT PRIMARY KEY, c1 INT)`) + ti2 = createTableInfo(c, p, se, tblID, `CREATE TABLE bar (id INT PRIMARY KEY, c1 INT, c2 INT)`) + ti3 = createTableInfo(c, p, se, tblID, `CREATE TABLE bar (id INT PRIMARY KEY, c2 INT)`) + + i11 = NewInfo(task, source1, upSchema, upTable, downSchema, downTable, DDLs1, ti0, []*model.TableInfo{ti1}) + i21 = NewInfo(task, source2, upSchema, upTable, downSchema, downTable, DDLs2, ti2, []*model.TableInfo{ti3}) + + tts = []TargetTable{ + newTargetTable(task, source1, downSchema, downTable, map[string]map[string]struct{}{upSchema: {upTable: struct{}{}}}), + newTargetTable(task, source2, downSchema, downTable, map[string]map[string]struct{}{upSchema: {upTable: struct{}{}}}), + } + + lockID = utils.GenDDLLockID(task, downSchema, downTable) + + ifm = map[string]map[string]map[string]map[string]Info{ + task: { + source1: {upSchema: {upTable: i11}}, + source2: {upSchema: {upTable: i21}}, + }, + } + colm = map[string]map[string]map[string]map[string]map[string]struct{}{ + lockID: { + "c3": { + source1: {upSchema: {upTable: {}}}, + source2: {upSchema: {upTable: {}}}, + }, + }, + } + lockJoined = map[string]schemacmp.Table{ + lockID: schemacmp.Encode(ti2), + } + lockTTS = map[string][]TargetTable{ + lockID: tts, + } + ) + + lk.RebuildLocksAndTables(etcdTestCli, ifm, colm, lockJoined, lockTTS) + locks := lk.Locks() + c.Assert(len(locks), Equals, 1) + lock, ok := locks[lockID] + c.Assert(ok, IsTrue) + cmp, err := lock.Joined().Compare(schemacmp.Encode(ti2)) + c.Assert(err, IsNil) + c.Assert(cmp, Equals, 0) + cmp, err = lock.tables[source1][upSchema][upTable].Compare(schemacmp.Encode(ti0)) + c.Assert(err, IsNil) + c.Assert(cmp, Equals, 0) + cmp, err = lock.tables[source2][upSchema][upTable].Compare(schemacmp.Encode(ti2)) + c.Assert(err, IsNil) + c.Assert(cmp, Equals, 0) + c.Assert(lock.columns, DeepEquals, colm[lockID]) +} diff --git a/pkg/shardddl/optimism/lock.go b/pkg/shardddl/optimism/lock.go index 51f6946b3e..9c0062f5d6 100644 --- a/pkg/shardddl/optimism/lock.go +++ b/pkg/shardddl/optimism/lock.go @@ -19,8 +19,8 @@ import ( "github.com/pingcap/parser" "github.com/pingcap/parser/ast" - "github.com/pingcap/parser/model" "github.com/pingcap/tidb-tools/pkg/schemacmp" + "go.etcd.io/etcd/clientv3" "go.uber.org/zap" "github.com/pingcap/dm/dm/master/metrics" @@ -33,6 +33,8 @@ import ( type Lock struct { mu sync.RWMutex + cli *clientv3.Client + ID string // lock's ID Task string // lock's corresponding task name @@ -45,6 +47,7 @@ type Lock struct { // upstream source ID -> upstream schema name -> upstream table name -> table info. // if all of them are the same, then we call the lock `synced`. tables map[string]map[string]map[string]schemacmp.Table + synced bool // whether DDLs operations have done (execute the shard DDL) to the downstream. @@ -56,21 +59,27 @@ type Lock struct { // upstream source ID -> upstream schema name -> upstream table name -> info version. versions map[string]map[string]map[string]int64 + + // record the partially dropped columns + // column name -> source -> upSchema -> upTable -> struct{} + columns map[string]map[string]map[string]map[string]struct{} } // NewLock creates a new Lock instance. // NOTE: we MUST give the initial table info when creating the lock now. -func NewLock(ID, task, downSchema, downTable string, ti *model.TableInfo, tts []TargetTable) *Lock { +func NewLock(cli *clientv3.Client, ID, task, downSchema, downTable string, joined schemacmp.Table, tts []TargetTable) *Lock { l := &Lock{ + cli: cli, ID: ID, Task: task, DownSchema: downSchema, DownTable: downTable, - joined: schemacmp.Encode(ti), + 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]struct{}), } l.addTables(tts) metrics.ReportDDLPending(task, metrics.DDLPendingNone, metrics.DDLPendingSynced) @@ -100,6 +109,8 @@ func (l *Lock) TrySync(info Info, tts []TargetTable) (newDDLs []string, err erro newTIs = info.TableInfosAfter infoVersion = info.Version ignoreConflict = info.IgnoreConflict + oldSynced = l.synced + emptyDDLs = []string{} ) l.mu.Lock() defer func() { @@ -111,7 +122,6 @@ func (l *Lock) TrySync(info Info, tts []TargetTable) (newDDLs []string, err erro l.mu.Unlock() }() - oldSynced := l.synced defer func() { _, remain := l.syncStatus() l.synced = remain == 0 @@ -149,6 +159,10 @@ func (l *Lock) TrySync(info Info, tts []TargetTable) (newDDLs []string, err erro return ddls, terror.ErrMasterInconsistentOptimisticDDLsAndInfo.Generate(len(ddls), len(newTIs)) } + // should not happen + if info.TableInfoBefore == nil { + return ddls, terror.ErrMasterOptimisticTableInfoBeforeNotExist.Generate(ddls) + } // handle the case where // is not in old source tables and current new source tables. // duplicate append is not a problem. @@ -160,12 +174,7 @@ func (l *Lock) TrySync(info Info, tts []TargetTable) (newDDLs []string, err erro l.versions[callerSource][callerSchema][callerTable] = infoVersion } - var emptyDDLs = []string{} - prevTable := l.tables[callerSource][callerSchema][callerTable] - oldJoined := l.joined - lastTableInfo := schemacmp.Encode(newTIs[len(newTIs)-1]) - defer func() { // only update table info if no error or ignore conflict if ignoreConflict || err == nil { @@ -175,6 +184,19 @@ func (l *Lock) TrySync(info Info, tts []TargetTable) (newDDLs []string, err erro } }() + prevTable := schemacmp.Encode(info.TableInfoBefore) + // if preTable not equal table in master, we always use preTable + // this often happens when an info TrySync twice, e.g. worker restart/resume task + if cmp, err := prevTable.Compare(l.tables[callerSource][callerSchema][callerTable]); err != nil || cmp != 0 { + l.tables[callerSource][callerSchema][callerTable] = prevTable + prevJoined, err := joinTable(prevTable) + if err != nil { + return emptyDDLs, err + } + l.joined = prevJoined + } + oldJoined := l.joined + lastJoined, err := joinTable(lastTableInfo) if err != nil { return emptyDDLs, err @@ -205,8 +227,15 @@ func (l *Lock) TrySync(info Info, tts []TargetTable) (newDDLs []string, err erro // special case: if the DDL does not affect the schema at all, assume it is // idempotent and just execute the DDL directly. // if any real conflicts after joined exist, they will be detected by the following steps. + // this often happens when executing `CREATE TABLE` statement var cmp int if cmp, err = nextTable.Compare(oldJoined); err == nil && cmp == 0 { + if col, err := GetColumnName(l.ID, ddls[idx], ast.AlterTableAddColumns); err != nil { + return newDDLs, err + } else if len(col) > 0 && l.IsDroppedColumn(info, col) { + return newDDLs, terror.ErrShardDDLOptimismTrySyncFail.Generate( + l.ID, fmt.Sprintf("add column %s that wasn't fully dropped in downstream. ddl: %s", col, ddls[idx])) + } newDDLs = append(newDDLs, ddls[idx]) continue } @@ -234,9 +263,9 @@ func (l *Lock) TrySync(info Info, tts []TargetTable) (newDDLs []string, err erro // for these two cases, we should execute the DDLs to the downstream to update the schema. log.L().Info("joined table info changed", zap.String("lock", l.ID), zap.Int("cmp", cmp), zap.Stringer("from", oldJoined), zap.Stringer("to", newJoined), zap.String("source", callerSource), zap.String("schema", callerSchema), zap.String("table", callerTable), zap.Strings("ddls", ddls)) - // check for add column with different field lengths + // check for add column with a larger field len if cmp < 0 { - err = AddDifferentFieldLenColumns(l.ID, ddls[idx], oldJoined, newJoined) + _, err = AddDifferentFieldLenColumns(l.ID, ddls[idx], oldJoined, newJoined) if err != nil { return ddls, err } @@ -264,15 +293,25 @@ func (l *Lock) TrySync(info Info, tts []TargetTable) (newDDLs []string, err erro cmp, _ = prevTable.Compare(nextTable) // we have checked `err` returned above. if cmp < 0 { - // check for add column with different field lengths - err = AddDifferentFieldLenColumns(l.ID, ddls[idx], nextTable, newJoined) - if err != nil { + // check for add column with a smaller field len + if col, err := AddDifferentFieldLenColumns(l.ID, ddls[idx], nextTable, newJoined); err != nil { return ddls, err + } else if len(col) > 0 && l.IsDroppedColumn(info, col) { + return ddls, terror.ErrShardDDLOptimismTrySyncFail.Generate( + l.ID, fmt.Sprintf("add column %s that wasn't fully dropped in downstream. ddl: %s", col, ddls[idx])) } // let every table to replicate the DDL. newDDLs = append(newDDLs, ddls[idx]) continue } else if cmp > 0 { + if col, err := GetColumnName(l.ID, ddls[idx], ast.AlterTableDropColumn); err != nil { + return ddls, err + } else if len(col) > 0 { + err = l.AddDroppedColumn(info, col) + if err != nil { + log.L().Error("fail to add dropped column info in etcd", zap.Error(err)) + } + } // last shard table won't go here continue } @@ -480,7 +519,6 @@ func (l *Lock) addTables(tts []TargetTable) { } for table := range tables { if _, ok := l.tables[tt.Source][schema][table]; !ok { - // NOTE: the newly added table uses the current table info. l.tables[tt.Source][schema][table] = l.joined l.done[tt.Source][schema][table] = false l.versions[tt.Source][schema][table] = 0 @@ -501,27 +539,123 @@ func (l *Lock) GetVersion(source string, schema string, table string) int64 { return l.versions[source][schema][table] } +// IsDroppedColumn checks whether this column is a partially dropped column for this lock +func (l *Lock) IsDroppedColumn(info Info, col string) bool { + if _, ok := l.columns[col]; !ok { + return false + } + source, upSchema, upTable := info.Source, info.UpSchema, info.UpTable + if _, ok := l.columns[col][source]; !ok { + return false + } + if _, ok := l.columns[col][source][upSchema]; !ok { + return false + } + if _, ok := l.columns[col][source][upSchema][upTable]; !ok { + return false + } + return true +} + +// AddDroppedColumn adds a dropped column name in both etcd and lock's column map +func (l *Lock) AddDroppedColumn(info Info, col string) error { + if l.IsDroppedColumn(info, col) { + return nil + } + log.L().Debug("add partially dropped columns", zap.String("column", col), zap.Stringer("info", info)) + + source, upSchema, upTable := info.Source, info.UpSchema, info.UpTable + _, _, err := PutDroppedColumn(l.cli, info, col) + if err != nil { + return err + } + + if _, ok := l.columns[col]; !ok { + l.columns[col] = make(map[string]map[string]map[string]struct{}) + } + if _, ok := l.columns[col][source]; !ok { + l.columns[col][source] = make(map[string]map[string]struct{}) + } + if _, ok := l.columns[col][source][upSchema]; !ok { + l.columns[col][source][upSchema] = make(map[string]struct{}) + } + l.columns[col][source][upSchema][upTable] = struct{}{} + return nil +} + +// DeleteColumnsByDDLs deletes the partially dropped columns that extracted from DDLs. +// We can not remove columns from the partially dropped columns map unless this column is dropped in the downstream database, +// that is to say, op.Done is true and ddls contains drop column DDL. +func (l *Lock) DeleteColumnsByDDLs(ddls []string) error { + l.mu.Lock() + defer l.mu.Unlock() + + colsToDelete := make([]string, 0, len(ddls)) + for _, ddl := range ddls { + col, err := GetColumnName(l.ID, ddl, ast.AlterTableDropColumn) + if err != nil { + return err + } + if len(col) > 0 { + colsToDelete = append(colsToDelete, col) + } + } + if len(colsToDelete) > 0 { + log.L().Debug("delete partially dropped columns", + zap.String("lockID", l.ID), zap.Strings("columns", colsToDelete)) + + _, _, err := DeleteDroppedColumns(l.cli, l.Task, l.DownSchema, l.DownTable, colsToDelete...) + if err != nil { + return err + } + + for _, col := range colsToDelete { + delete(l.columns, col) + } + } + + return nil +} + // AddDifferentFieldLenColumns checks whether dm adds columns with different field lengths -func AddDifferentFieldLenColumns(lockID, ddl string, oldJoined, newJoined schemacmp.Table) error { +func AddDifferentFieldLenColumns(lockID, ddl string, oldJoined, newJoined schemacmp.Table) (string, error) { + col, err := GetColumnName(lockID, ddl, ast.AlterTableAddColumns) + if err != nil { + return col, err + } + if len(col) > 0 { + oldJoinedCols := schemacmp.DecodeColumnFieldTypes(oldJoined) + newJoinedCols := schemacmp.DecodeColumnFieldTypes(newJoined) + oldCol, ok1 := oldJoinedCols[col] + newCol, ok2 := newJoinedCols[col] + if ok1 && ok2 && newCol.Flen != oldCol.Flen { + return col, terror.ErrShardDDLOptimismTrySyncFail.Generate( + lockID, fmt.Sprintf("add columns with different field lengths."+ + "ddl: %s, origLen: %d, newLen: %d", ddl, oldCol.Flen, newCol.Flen)) + } + } + return col, nil +} + +// GetColumnName checks whether dm adds/drops a column, and return this column's name +func GetColumnName(lockID, ddl string, tp ast.AlterTableType) (string, error) { if stmt, err := parser.New().ParseOneStmt(ddl, "", ""); err != nil { - return terror.ErrShardDDLOptimismTrySyncFail.Delegate( + return "", terror.ErrShardDDLOptimismTrySyncFail.Delegate( err, lockID, fmt.Sprintf("fail to parse ddl %s", ddl)) } else if v, ok := stmt.(*ast.AlterTableStmt); ok && len(v.Specs) > 0 { spec := v.Specs[0] - if spec.Tp == ast.AlterTableAddColumns && len(spec.NewColumns) > 0 { - col := spec.NewColumns[0].Name.Name.O - oldJoinedCols := schemacmp.DecodeColumnFieldTypes(oldJoined) - newJoinedCols := schemacmp.DecodeColumnFieldTypes(newJoined) - oldCol, ok1 := oldJoinedCols[col] - newCol, ok2 := newJoinedCols[col] - if ok1 && ok2 { - if newCol.Flen != oldCol.Flen { - return terror.ErrShardDDLOptimismTrySyncFail.Generate( - lockID, fmt.Sprintf("add columns with different field lengths."+ - "ddl: %s, origLen: %d, newLen: %d", ddl, oldCol.Flen, newCol.Flen)) + if spec.Tp == tp { + switch spec.Tp { + case ast.AlterTableAddColumns: + if len(spec.NewColumns) > 0 { + return spec.NewColumns[0].Name.Name.O, nil + } + case ast.AlterTableDropColumn: + if spec.OldColumnName != nil { + return spec.OldColumnName.Name.O, nil } } } } - return nil + return "", nil } diff --git a/pkg/shardddl/optimism/lock_test.go b/pkg/shardddl/optimism/lock_test.go index 44b39f315b..0d1b0c2ab3 100644 --- a/pkg/shardddl/optimism/lock_test.go +++ b/pkg/shardddl/optimism/lock_test.go @@ -14,11 +14,15 @@ package optimism import ( + "testing" + . "github.com/pingcap/check" "github.com/pingcap/parser" + "github.com/pingcap/parser/ast" "github.com/pingcap/parser/model" "github.com/pingcap/tidb-tools/pkg/schemacmp" "github.com/pingcap/tidb/util/mock" + "go.etcd.io/etcd/integration" "github.com/pingcap/dm/pkg/log" "github.com/pingcap/dm/pkg/terror" @@ -28,10 +32,23 @@ type testLock struct{} var _ = Suite(&testLock{}) +func TestLock(t *testing.T) { + mockCluster := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1}) + defer mockCluster.Terminate(t) + + etcdTestCli = mockCluster.RandClient() + + TestingT(t) +} + func (t *testLock) SetUpSuite(c *C) { c.Assert(log.InitLogger(&log.Config{}), IsNil) } +func (t *testLock) TearDownSuite(c *C) { + clearTestInfoOperation(c) +} + func (t *testLock) TestLockTrySyncNormal(c *C) { var ( ID = "test_lock_try_sync_normal-`foo`.`bar`" @@ -65,7 +82,7 @@ func (t *testLock) TestLockTrySyncNormal(c *C) { newTargetTable(task, sources[1], downSchema, downTable, tables), } - l = NewLock(ID, task, downSchema, downTable, ti0, tts) + l = NewLock(etcdTestCli, ID, task, downSchema, downTable, schemacmp.Encode(ti0), tts) vers = map[string]map[string]map[string]int64{ sources[0]: { @@ -100,7 +117,7 @@ func (t *testLock) TestLockTrySyncNormal(c *C) { for _, db := range dbs { for _, tbl := range tbls { - info := newInfoWithVersion(task, source, db, tbl, downSchema, downTable, DDLs1, nil, []*model.TableInfo{ti1}, vers) + info := newInfoWithVersion(task, source, db, tbl, downSchema, downTable, DDLs1, ti0, []*model.TableInfo{ti1}, vers) DDLs, err := l.TrySync(info, tts) c.Assert(err, IsNil) c.Assert(DDLs, DeepEquals, DDLs1) @@ -119,7 +136,7 @@ func (t *testLock) TestLockTrySyncNormal(c *C) { t.checkLockNoDone(c, l) // CASE: TrySync again after synced is idempotent. - info := newInfoWithVersion(task, sources[0], dbs[0], tbls[0], downSchema, downTable, DDLs1, nil, []*model.TableInfo{ti1}, vers) + info := newInfoWithVersion(task, sources[0], dbs[0], tbls[0], downSchema, downTable, DDLs1, ti0, []*model.TableInfo{ti1}, vers) DDLs, err := l.TrySync(info, tts) c.Assert(err, IsNil) c.Assert(DDLs, DeepEquals, DDLs1) @@ -129,7 +146,7 @@ func (t *testLock) TestLockTrySyncNormal(c *C) { // CASE: need to add more than one DDL to reach the desired schema (schema become larger). // add two columns for one table. - info = newInfoWithVersion(task, sources[0], dbs[0], tbls[0], downSchema, downTable, DDLs2, nil, []*model.TableInfo{ti2_1, ti2}, vers) + info = newInfoWithVersion(task, sources[0], dbs[0], tbls[0], downSchema, downTable, DDLs2, ti1, []*model.TableInfo{ti2_1, ti2}, vers) DDLs, err = l.TrySync(info, tts) c.Assert(err, IsNil) c.Assert(DDLs, DeepEquals, DDLs2) @@ -139,7 +156,7 @@ func (t *testLock) TestLockTrySyncNormal(c *C) { c.Assert(ready[sources[0]][dbs[0]][tbls[1]], IsFalse) // TrySync again is idempotent (more than one DDL). - info = newInfoWithVersion(task, sources[0], dbs[0], tbls[0], downSchema, downTable, DDLs2, nil, []*model.TableInfo{ti2_1, ti2}, vers) + info = newInfoWithVersion(task, sources[0], dbs[0], tbls[0], downSchema, downTable, DDLs2, ti1, []*model.TableInfo{ti2_1, ti2}, vers) DDLs, err = l.TrySync(info, tts) c.Assert(err, IsNil) c.Assert(DDLs, DeepEquals, DDLs2) @@ -149,7 +166,7 @@ func (t *testLock) TestLockTrySyncNormal(c *C) { c.Assert(ready[sources[0]][dbs[0]][tbls[1]], IsFalse) // add only the first column for another table. - info = newInfoWithVersion(task, sources[0], dbs[0], tbls[1], downSchema, downTable, DDLs2[0:1], nil, []*model.TableInfo{ti2_1}, vers) + info = newInfoWithVersion(task, sources[0], dbs[0], tbls[1], downSchema, downTable, DDLs2[0:1], ti1, []*model.TableInfo{ti2_1}, vers) DDLs, err = l.TrySync(info, tts) c.Assert(err, IsNil) c.Assert(DDLs, DeepEquals, DDLs2[0:1]) @@ -165,17 +182,18 @@ func (t *testLock) TestLockTrySyncNormal(c *C) { c.Assert(err, IsNil) c.Assert(cmp, Equals, 1) - // TrySync again (only the first DDL). - info = newInfoWithVersion(task, sources[0], dbs[0], tbls[1], downSchema, downTable, DDLs2[0:1], nil, []*model.TableInfo{ti2_1}, vers) + // TrySync again is idempotent + info = newInfoWithVersion(task, sources[0], dbs[0], tbls[1], downSchema, downTable, DDLs2[0:1], ti1, []*model.TableInfo{ti2_1}, vers) DDLs, err = l.TrySync(info, tts) c.Assert(err, IsNil) - c.Assert(DDLs, DeepEquals, []string{}) // NOTE: special case, joined has larger schema. + c.Assert(DDLs, DeepEquals, DDLs2[0:1]) c.Assert(l.versions, DeepEquals, vers) ready = l.Ready() + c.Assert(ready[sources[0]][dbs[0]][tbls[0]], IsTrue) c.Assert(ready[sources[0]][dbs[0]][tbls[1]], IsFalse) // add the second column for another table. - info = newInfoWithVersion(task, sources[0], dbs[0], tbls[1], downSchema, downTable, DDLs2[1:2], nil, []*model.TableInfo{ti2}, vers) + info = newInfoWithVersion(task, sources[0], dbs[0], tbls[1], downSchema, downTable, DDLs2[1:2], ti2_1, []*model.TableInfo{ti2}, vers) DDLs, err = l.TrySync(info, tts) c.Assert(err, IsNil) c.Assert(DDLs, DeepEquals, DDLs2[1:2]) @@ -191,24 +209,13 @@ func (t *testLock) TestLockTrySyncNormal(c *C) { c.Assert(cmp, Equals, 0) // Try again (for the second DDL). - info = newInfoWithVersion(task, sources[0], dbs[0], tbls[1], downSchema, downTable, DDLs2[1:2], nil, []*model.TableInfo{ti2}, vers) + info = newInfoWithVersion(task, sources[0], dbs[0], tbls[1], downSchema, downTable, DDLs2[1:2], ti2_1, []*model.TableInfo{ti2}, vers) DDLs, err = l.TrySync(info, tts) c.Assert(err, IsNil) c.Assert(DDLs, DeepEquals, DDLs2[1:2]) c.Assert(l.versions, DeepEquals, vers) - // try add columns for all tables to reach the same schema. - resultDDLs := map[string]map[string]map[string][]string{ - sources[0]: { - dbs[0]: {tbls[0]: DDLs2[1:], tbls[1]: DDLs2[1:]}, - dbs[1]: {tbls[0]: DDLs2, tbls[1]: DDLs2}, - }, - sources[1]: { - dbs[0]: {tbls[0]: DDLs2, tbls[1]: DDLs2}, - dbs[1]: {tbls[0]: DDLs2, tbls[1]: DDLs2}, - }, - } - t.trySyncForAllTablesLarger(c, l, DDLs2, []*model.TableInfo{ti2_1, ti2}, tts, vers, resultDDLs) + t.trySyncForAllTablesLarger(c, l, DDLs2, ti1, []*model.TableInfo{ti2_1, ti2}, tts, vers) t.checkLockSynced(c, l) t.checkLockNoDone(c, l) @@ -230,7 +237,7 @@ func (t *testLock) TestLockTrySyncNormal(c *C) { for _, db := range dbs { for _, tbl := range tbls { syncedCount++ - info = newInfoWithVersion(task, source, db, tbl, downSchema, downTable, DDLs3, nil, []*model.TableInfo{ti3}, vers) + info = newInfoWithVersion(task, source, db, tbl, downSchema, downTable, DDLs3, ti2, []*model.TableInfo{ti3}, vers) DDLs, err = l.TrySync(info, tts) c.Assert(err, IsNil) c.Assert(l.versions, DeepEquals, vers) @@ -253,7 +260,7 @@ func (t *testLock) TestLockTrySyncNormal(c *C) { // CASE: need to drop more than one DDL to reach the desired schema (schema become smaller). // drop two columns for one table. - info = newInfoWithVersion(task, sources[0], dbs[0], tbls[0], downSchema, downTable, DDLs4, nil, []*model.TableInfo{ti4_1, ti4}, vers) + info = newInfoWithVersion(task, sources[0], dbs[0], tbls[0], downSchema, downTable, DDLs4, ti3, []*model.TableInfo{ti4_1, ti4}, vers) DDLs, err = l.TrySync(info, tts) c.Assert(err, IsNil) c.Assert(DDLs, DeepEquals, []string{}) @@ -263,17 +270,17 @@ func (t *testLock) TestLockTrySyncNormal(c *C) { c.Assert(ready[sources[0]][dbs[0]][tbls[1]], IsTrue) // TrySync again is idempotent. - info = newInfoWithVersion(task, sources[0], dbs[0], tbls[0], downSchema, downTable, DDLs4, nil, []*model.TableInfo{ti4_1, ti4}, vers) + info = newInfoWithVersion(task, sources[0], dbs[0], tbls[0], downSchema, downTable, DDLs4, ti3, []*model.TableInfo{ti4_1, ti4}, vers) DDLs, err = l.TrySync(info, tts) c.Assert(err, IsNil) - c.Assert(DDLs, DeepEquals, DDLs4[:1]) + c.Assert(DDLs, DeepEquals, []string{}) c.Assert(l.versions, DeepEquals, vers) ready = l.Ready() c.Assert(ready[sources[0]][dbs[0]][tbls[0]], IsFalse) c.Assert(ready[sources[0]][dbs[0]][tbls[1]], IsTrue) // drop only the first column for another table. - info = newInfoWithVersion(task, sources[0], dbs[0], tbls[1], downSchema, downTable, DDLs4[0:1], nil, []*model.TableInfo{ti4_1}, vers) + info = newInfoWithVersion(task, sources[0], dbs[0], tbls[1], downSchema, downTable, DDLs4[0:1], ti3, []*model.TableInfo{ti4_1}, vers) DDLs, err = l.TrySync(info, tts) c.Assert(err, IsNil) c.Assert(DDLs, DeepEquals, []string{}) @@ -286,14 +293,14 @@ func (t *testLock) TestLockTrySyncNormal(c *C) { c.Assert(cmp, Equals, -1) // TrySync again (only the first DDL). - info = newInfoWithVersion(task, sources[0], dbs[0], tbls[1], downSchema, downTable, DDLs4[0:1], nil, []*model.TableInfo{ti4_1}, vers) + info = newInfoWithVersion(task, sources[0], dbs[0], tbls[1], downSchema, downTable, DDLs4[0:1], ti3, []*model.TableInfo{ti4_1}, vers) DDLs, err = l.TrySync(info, tts) c.Assert(err, IsNil) c.Assert(DDLs, DeepEquals, []string{}) c.Assert(l.versions, DeepEquals, vers) // drop the second column for another table. - info = newInfoWithVersion(task, sources[0], dbs[0], tbls[1], downSchema, downTable, DDLs4[1:2], nil, []*model.TableInfo{ti4}, vers) + info = newInfoWithVersion(task, sources[0], dbs[0], tbls[1], downSchema, downTable, DDLs4[1:2], ti4_1, []*model.TableInfo{ti4}, vers) DDLs, err = l.TrySync(info, tts) c.Assert(err, IsNil) c.Assert(DDLs, DeepEquals, []string{}) @@ -306,7 +313,7 @@ func (t *testLock) TestLockTrySyncNormal(c *C) { c.Assert(cmp, Equals, 0) // TrySync again (for the second DDL). - info = newInfoWithVersion(task, sources[0], dbs[0], tbls[1], downSchema, downTable, DDLs4[1:2], nil, []*model.TableInfo{ti4}, vers) + info = newInfoWithVersion(task, sources[0], dbs[0], tbls[1], downSchema, downTable, DDLs4[1:2], ti4_1, []*model.TableInfo{ti4}, vers) DDLs, err = l.TrySync(info, tts) c.Assert(err, IsNil) c.Assert(DDLs, DeepEquals, []string{}) @@ -318,7 +325,7 @@ func (t *testLock) TestLockTrySyncNormal(c *C) { for schema, tables := range schemaTables { for table, synced2 := range tables { if synced2 { // do not `TrySync` again for previous two (un-synced now). - info = newInfoWithVersion(task, source, schema, table, downSchema, downTable, DDLs4, nil, []*model.TableInfo{ti4_1, ti4}, vers) + info = newInfoWithVersion(task, source, schema, table, downSchema, downTable, DDLs4, ti3, []*model.TableInfo{ti4_1, ti4}, vers) DDLs, err = l.TrySync(info, tts) c.Assert(err, IsNil) c.Assert(l.versions, DeepEquals, vers) @@ -360,7 +367,7 @@ func (t *testLock) TestLockTrySyncIndex(c *C) { newTargetTable(task, source, downSchema, downTable, tables), } - l = NewLock(ID, task, downSchema, downTable, ti0, tts) + l = NewLock(etcdTestCli, ID, task, downSchema, downTable, schemacmp.Encode(ti0), tts) vers = map[string]map[string]map[string]int64{ source: { @@ -375,7 +382,7 @@ func (t *testLock) TestLockTrySyncIndex(c *C) { // try sync for one table, `DROP INDEX` returned directly (to make schema become more compatible). // `DROP INDEX` is handled like `ADD COLUMN`. - info := newInfoWithVersion(task, source, db, tbls[0], downSchema, downTable, DDLs1, nil, []*model.TableInfo{ti1}, vers) + info := newInfoWithVersion(task, source, db, tbls[0], downSchema, downTable, DDLs1, ti0, []*model.TableInfo{ti1}, vers) DDLs, err := l.TrySync(info, tts) c.Assert(err, IsNil) c.Assert(DDLs, DeepEquals, DDLs1) @@ -386,7 +393,7 @@ func (t *testLock) TestLockTrySyncIndex(c *C) { c.Assert(remain, Equals, 1) // try sync for another table, also got `DROP INDEX` now. - info = newInfoWithVersion(task, source, db, tbls[1], downSchema, downTable, DDLs1, nil, []*model.TableInfo{ti1}, vers) + info = newInfoWithVersion(task, source, db, tbls[1], downSchema, downTable, DDLs1, ti0, []*model.TableInfo{ti1}, vers) DDLs, err = l.TrySync(info, tts) c.Assert(err, IsNil) c.Assert(DDLs, DeepEquals, DDLs1) @@ -395,7 +402,7 @@ func (t *testLock) TestLockTrySyncIndex(c *C) { // try sync for one table, `ADD INDEX` not returned directly (to keep the schema more compatible). // `ADD INDEX` is handled like `DROP COLUMN`. - info = newInfoWithVersion(task, source, db, tbls[0], downSchema, downTable, DDLs2, nil, []*model.TableInfo{ti2}, vers) + info = newInfoWithVersion(task, source, db, tbls[0], downSchema, downTable, DDLs2, ti1, []*model.TableInfo{ti2}, vers) DDLs, err = l.TrySync(info, tts) c.Assert(err, IsNil) c.Assert(DDLs, DeepEquals, []string{}) // no DDLs returned @@ -406,7 +413,7 @@ func (t *testLock) TestLockTrySyncIndex(c *C) { c.Assert(remain, Equals, 1) // try sync for another table, got `ADD INDEX` now. - info = newInfoWithVersion(task, source, db, tbls[1], downSchema, downTable, DDLs2, nil, []*model.TableInfo{ti2}, vers) + info = newInfoWithVersion(task, source, db, tbls[1], downSchema, downTable, DDLs2, ti1, []*model.TableInfo{ti2}, vers) DDLs, err = l.TrySync(info, tts) c.Assert(err, IsNil) c.Assert(DDLs, DeepEquals, DDLs2) @@ -439,7 +446,7 @@ func (t *testLock) TestLockTrySyncNullNotNull(c *C) { newTargetTable(task, source, downSchema, downTable, tables), } - l = NewLock(ID, task, downSchema, downTable, ti0, tts) + l = NewLock(etcdTestCli, ID, task, downSchema, downTable, schemacmp.Encode(ti0), tts) vers = map[string]map[string]map[string]int64{ source: { @@ -454,28 +461,28 @@ func (t *testLock) TestLockTrySyncNullNotNull(c *C) { for i := 0; i < 2; i++ { // two round // try sync for one table, from `NULL` to `NOT NULL`, no DDLs returned. - info := newInfoWithVersion(task, source, db, tbls[0], downSchema, downTable, DDLs1, nil, []*model.TableInfo{ti1}, vers) + info := newInfoWithVersion(task, source, db, tbls[0], downSchema, downTable, DDLs1, ti0, []*model.TableInfo{ti1}, vers) DDLs, err := l.TrySync(info, tts) c.Assert(err, IsNil) c.Assert(DDLs, DeepEquals, []string{}) c.Assert(l.versions, DeepEquals, vers) // try sync for another table, DDLs returned. - info = newInfoWithVersion(task, source, db, tbls[1], downSchema, downTable, DDLs1, nil, []*model.TableInfo{ti1}, vers) + info = newInfoWithVersion(task, source, db, tbls[1], downSchema, downTable, DDLs1, ti0, []*model.TableInfo{ti1}, vers) DDLs, err = l.TrySync(info, tts) c.Assert(err, IsNil) c.Assert(DDLs, DeepEquals, DDLs1) c.Assert(l.versions, DeepEquals, vers) // try sync for one table, from `NOT NULL` to `NULL`, DDLs returned. - info = newInfoWithVersion(task, source, db, tbls[0], downSchema, downTable, DDLs2, nil, []*model.TableInfo{ti2}, vers) + info = newInfoWithVersion(task, source, db, tbls[0], downSchema, downTable, DDLs2, ti1, []*model.TableInfo{ti2}, vers) DDLs, err = l.TrySync(info, tts) c.Assert(err, IsNil) c.Assert(DDLs, DeepEquals, DDLs2) c.Assert(l.versions, DeepEquals, vers) // try sync for another table, from `NOT NULL` to `NULL`, DDLs, returned. - info = newInfoWithVersion(task, source, db, tbls[1], downSchema, downTable, DDLs2, nil, []*model.TableInfo{ti2}, vers) + info = newInfoWithVersion(task, source, db, tbls[1], downSchema, downTable, DDLs2, ti1, []*model.TableInfo{ti2}, vers) DDLs, err = l.TrySync(info, tts) c.Assert(err, IsNil) c.Assert(DDLs, DeepEquals, DDLs2) @@ -506,7 +513,7 @@ func (t *testLock) TestLockTrySyncIntBigint(c *C) { newTargetTable(task, source, downSchema, downTable, tables), } - l = NewLock(ID, task, downSchema, downTable, ti0, tts) + l = NewLock(etcdTestCli, ID, task, downSchema, downTable, schemacmp.Encode(ti0), tts) vers = map[string]map[string]map[string]int64{ source: { @@ -520,14 +527,14 @@ func (t *testLock) TestLockTrySyncIntBigint(c *C) { t.checkLockNoDone(c, l) // try sync for one table, from `INT` to `BIGINT`, DDLs returned. - info := newInfoWithVersion(task, source, db, tbls[0], downSchema, downTable, DDLs1, nil, []*model.TableInfo{ti1}, vers) + info := newInfoWithVersion(task, source, db, tbls[0], downSchema, downTable, DDLs1, ti0, []*model.TableInfo{ti1}, vers) DDLs, err := l.TrySync(info, tts) c.Assert(err, IsNil) c.Assert(DDLs, DeepEquals, DDLs1) c.Assert(l.versions, DeepEquals, vers) // try sync for another table, DDLs returned. - info = newInfoWithVersion(task, source, db, tbls[1], downSchema, downTable, DDLs1, nil, []*model.TableInfo{ti1}, vers) + info = newInfoWithVersion(task, source, db, tbls[1], downSchema, downTable, DDLs1, ti0, []*model.TableInfo{ti1}, vers) DDLs, err = l.TrySync(info, tts) c.Assert(err, IsNil) c.Assert(DDLs, DeepEquals, DDLs1) @@ -557,7 +564,7 @@ func (t *testLock) TestLockTrySyncNoDiff(c *C) { newTargetTable(task, source, downSchema, downTable, tables), } - l = NewLock(ID, task, downSchema, downTable, ti0, tts) + l = NewLock(etcdTestCli, ID, task, downSchema, downTable, schemacmp.Encode(ti0), tts) vers = map[string]map[string]map[string]int64{ source: { @@ -571,7 +578,7 @@ func (t *testLock) TestLockTrySyncNoDiff(c *C) { t.checkLockNoDone(c, l) // try sync for one table. - info := newInfoWithVersion(task, source, db, tbls[0], downSchema, downTable, DDLs1, nil, []*model.TableInfo{ti1}, vers) + info := newInfoWithVersion(task, source, db, tbls[0], downSchema, downTable, DDLs1, ti0, []*model.TableInfo{ti1}, vers) DDLs, err := l.TrySync(info, tts) c.Assert(terror.ErrShardDDLOptimismTrySyncFail.Equal(err), IsTrue) c.Assert(DDLs, DeepEquals, []string{}) @@ -599,7 +606,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(ID, task, downSchema, downTable, ti0, tts) + l = NewLock(etcdTestCli, ID, task, downSchema, downTable, schemacmp.Encode(ti0), tts) vers = map[string]map[string]map[string]int64{ source1: { db1: {tbl1: 0}, @@ -615,7 +622,7 @@ func (t *testLock) TestLockTrySyncNewTable(c *C) { t.checkLockNoDone(c, l) // TrySync for a new table as the caller. - info := newInfoWithVersion(task, source2, db2, tbl2, downSchema, downTable, DDLs1, nil, []*model.TableInfo{ti1}, vers) + info := newInfoWithVersion(task, source2, db2, tbl2, downSchema, downTable, DDLs1, ti0, []*model.TableInfo{ti1}, vers) DDLs, err := l.TrySync(info, tts) c.Assert(err, IsNil) c.Assert(DDLs, DeepEquals, DDLs1) @@ -631,7 +638,7 @@ func (t *testLock) TestLockTrySyncNewTable(c *C) { c.Assert(ready[source2][db2][tbl2], IsTrue) // TrySync for two new tables as extra sources. - // we treat all newly added sources as synced. + // newly added work table use tableInfoBefore as table info tts = append(tts, newTargetTable(task, source1, downSchema, downTable, map[string]map[string]struct{}{db1: {tbl2: struct{}{}}}), newTargetTable(task, source2, downTable, downTable, map[string]map[string]struct{}{db2: {tbl1: struct{}{}}}), @@ -639,7 +646,24 @@ func (t *testLock) TestLockTrySyncNewTable(c *C) { vers[source1][db1][tbl2] = 0 vers[source2][db2][tbl1] = 0 - info = newInfoWithVersion(task, source1, db1, tbl1, downSchema, downTable, DDLs1, nil, []*model.TableInfo{ti1}, vers) + info = newInfoWithVersion(task, source1, db1, tbl1, downSchema, downTable, DDLs1, ti0, []*model.TableInfo{ti1}, vers) + DDLs, err = l.TrySync(info, tts) + c.Assert(err, IsNil) + c.Assert(DDLs, DeepEquals, DDLs1) + c.Assert(l.versions, DeepEquals, vers) + + ready = l.Ready() + c.Assert(ready, HasLen, 2) + c.Assert(ready[source1], HasLen, 1) + c.Assert(ready[source1][db1], HasLen, 2) + c.Assert(ready[source1][db1][tbl1], IsTrue) + c.Assert(ready[source1][db1][tbl2], IsTrue) + c.Assert(ready[source2], HasLen, 1) + c.Assert(ready[source2][db2], HasLen, 2) + c.Assert(ready[source2][db2][tbl1], IsTrue) + c.Assert(ready[source2][db2][tbl2], IsTrue) + + info = newInfoWithVersion(task, source1, db1, tbl2, downSchema, downTable, DDLs1, ti0, []*model.TableInfo{ti1}, vers) DDLs, err = l.TrySync(info, tts) c.Assert(err, IsNil) c.Assert(DDLs, DeepEquals, DDLs1) @@ -692,7 +716,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(ID, task, downSchema, downTable, ti0, tts) + l = NewLock(etcdTestCli, ID, task, downSchema, downTable, schemacmp.Encode(ti0), tts) vers = map[string]map[string]map[string]int64{ source: { @@ -707,7 +731,7 @@ func (t *testLock) TestLockTrySyncRevert(c *C) { // CASE: revert for single DDL. // TrySync for one table. - info := newInfoWithVersion(task, source, db, tbls[0], downSchema, downTable, DDLs1, nil, []*model.TableInfo{ti1}, vers) + info := newInfoWithVersion(task, source, db, tbls[0], downSchema, downTable, DDLs1, ti0, []*model.TableInfo{ti1}, vers) DDLs, err := l.TrySync(info, tts) c.Assert(err, IsNil) c.Assert(DDLs, DeepEquals, DDLs1) @@ -723,7 +747,7 @@ func (t *testLock) TestLockTrySyncRevert(c *C) { c.Assert(cmp, Equals, -1) // revert for the table, become synced again. - info = newInfoWithVersion(task, source, db, tbls[0], downSchema, downTable, DDLs2, nil, []*model.TableInfo{ti2}, vers) + info = newInfoWithVersion(task, source, db, tbls[0], downSchema, downTable, DDLs2, ti1, []*model.TableInfo{ti2}, vers) DDLs, err = l.TrySync(info, tts) c.Assert(err, IsNil) c.Assert(DDLs, DeepEquals, DDLs2) @@ -733,7 +757,7 @@ func (t *testLock) TestLockTrySyncRevert(c *C) { // CASE: revert for multiple DDLs. // TrySync for one table. - info = newInfoWithVersion(task, source, db, tbls[0], downSchema, downTable, DDLs3, nil, []*model.TableInfo{ti4, ti3}, vers) + info = newInfoWithVersion(task, source, db, tbls[0], downSchema, downTable, DDLs3, ti0, []*model.TableInfo{ti4, ti3}, vers) DDLs, err = l.TrySync(info, tts) c.Assert(err, IsNil) c.Assert(DDLs, DeepEquals, DDLs3) @@ -749,7 +773,7 @@ func (t *testLock) TestLockTrySyncRevert(c *C) { c.Assert(cmp, Equals, -1) // revert part of the DDLs. - info = newInfoWithVersion(task, source, db, tbls[0], downSchema, downTable, DDLs4, nil, []*model.TableInfo{ti4}, vers) + info = newInfoWithVersion(task, source, db, tbls[0], downSchema, downTable, DDLs4, ti3, []*model.TableInfo{ti4}, vers) DDLs, err = l.TrySync(info, tts) c.Assert(err, IsNil) c.Assert(DDLs, DeepEquals, DDLs4) @@ -764,7 +788,7 @@ func (t *testLock) TestLockTrySyncRevert(c *C) { c.Assert(cmp, Equals, -1) // revert the reset part of the DDLs. - info = newInfoWithVersion(task, source, db, tbls[0], downSchema, downTable, DDLs5, nil, []*model.TableInfo{ti5}, vers) + info = newInfoWithVersion(task, source, db, tbls[0], downSchema, downTable, DDLs5, ti4, []*model.TableInfo{ti5}, vers) DDLs, err = l.TrySync(info, tts) c.Assert(err, IsNil) c.Assert(DDLs, DeepEquals, DDLs5) @@ -774,7 +798,7 @@ func (t *testLock) TestLockTrySyncRevert(c *C) { // CASE: revert part of multiple DDLs. // TrySync for one table. - info = newInfoWithVersion(task, source, db, tbls[0], downSchema, downTable, DDLs6, nil, []*model.TableInfo{ti7, ti6}, vers) + info = newInfoWithVersion(task, source, db, tbls[0], downSchema, downTable, DDLs6, ti0, []*model.TableInfo{ti7, ti6}, vers) DDLs, err = l.TrySync(info, tts) c.Assert(err, IsNil) c.Assert(DDLs, DeepEquals, DDLs6) @@ -789,7 +813,7 @@ func (t *testLock) TestLockTrySyncRevert(c *C) { c.Assert(cmp, Equals, -1) // revert part of the DDLs. - info = newInfoWithVersion(task, source, db, tbls[0], downSchema, downTable, DDLs7, nil, []*model.TableInfo{ti7}, vers) + info = newInfoWithVersion(task, source, db, tbls[0], downSchema, downTable, DDLs7, ti3, []*model.TableInfo{ti7}, vers) DDLs, err = l.TrySync(info, tts) c.Assert(err, IsNil) c.Assert(DDLs, DeepEquals, DDLs7) @@ -804,7 +828,7 @@ func (t *testLock) TestLockTrySyncRevert(c *C) { c.Assert(cmp, Equals, -1) // TrySync for another table. - info = newInfoWithVersion(task, source, db, tbls[1], downSchema, downTable, DDLs8, nil, []*model.TableInfo{ti8}, vers) + info = newInfoWithVersion(task, source, db, tbls[1], downSchema, downTable, DDLs8, ti0, []*model.TableInfo{ti8}, vers) DDLs, err = l.TrySync(info, tts) c.Assert(err, IsNil) c.Assert(DDLs, DeepEquals, DDLs8) @@ -839,7 +863,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(ID, task, downSchema, downTable, ti0, tts) + l = NewLock(etcdTestCli, ID, task, downSchema, downTable, schemacmp.Encode(ti0), tts) vers = map[string]map[string]map[string]int64{ source: { @@ -853,7 +877,7 @@ func (t *testLock) TestLockTrySyncConflictNonIntrusive(c *C) { t.checkLockNoDone(c, l) // TrySync for the first table, construct the joined schema. - info := newInfoWithVersion(task, source, db, tbls[0], downSchema, downTable, DDLs1, nil, []*model.TableInfo{ti1}, vers) + info := newInfoWithVersion(task, source, db, tbls[0], downSchema, downTable, DDLs1, ti0, []*model.TableInfo{ti1}, vers) DDLs, err := l.TrySync(info, tts) c.Assert(err, IsNil) c.Assert(DDLs, DeepEquals, DDLs1) @@ -869,7 +893,7 @@ func (t *testLock) TestLockTrySyncConflictNonIntrusive(c *C) { c.Assert(cmp, Equals, -1) // TrySync for the second table with another schema (add two columns, one of them will cause conflict). - info = newInfoWithVersion(task, source, db, tbls[1], downSchema, downTable, DDLs2, nil, []*model.TableInfo{ti2_1, ti2}, vers) + info = newInfoWithVersion(task, source, db, tbls[1], downSchema, downTable, DDLs2, ti0, []*model.TableInfo{ti2_1, ti2}, vers) DDLs, err = l.TrySync(info, tts) c.Assert(terror.ErrShardDDLOptimismTrySyncFail.Equal(err), IsTrue) c.Assert(DDLs, DeepEquals, []string{}) @@ -882,7 +906,7 @@ func (t *testLock) TestLockTrySyncConflictNonIntrusive(c *C) { c.Assert(ready[source][db][tbls[1]], IsFalse) // TrySync for the first table to resolve the conflict. - info = newInfoWithVersion(task, source, db, tbls[0], downSchema, downTable, DDLs3, nil, []*model.TableInfo{ti3}, vers) + info = newInfoWithVersion(task, source, db, tbls[0], downSchema, downTable, DDLs3, ti1, []*model.TableInfo{ti3}, vers) DDLs, err = l.TrySync(info, tts) c.Assert(err, IsNil) c.Assert(DDLs, DeepEquals, DDLs3) @@ -898,7 +922,7 @@ func (t *testLock) TestLockTrySyncConflictNonIntrusive(c *C) { c.Assert(cmp, Equals, 0) // TrySync for the second table, succeed now - info = newInfoWithVersion(task, source, db, tbls[1], downSchema, downTable, DDLs2, nil, []*model.TableInfo{ti2_1, ti2}, vers) + info = newInfoWithVersion(task, source, db, tbls[1], downSchema, downTable, DDLs2, ti0, []*model.TableInfo{ti2_1, ti2}, vers) DDLs, err = l.TrySync(info, tts) c.Assert(err, IsNil) c.Assert(DDLs, DeepEquals, DDLs2) @@ -910,7 +934,7 @@ func (t *testLock) TestLockTrySyncConflictNonIntrusive(c *C) { c.Assert(ready[source][db][tbls[1]], IsTrue) // TrySync for the first table. - info = newInfoWithVersion(task, source, db, tbls[0], downSchema, downTable, DDLs4, nil, []*model.TableInfo{ti4_1, ti4}, vers) + info = newInfoWithVersion(task, source, db, tbls[0], downSchema, downTable, DDLs4, ti0, []*model.TableInfo{ti4_1, ti4}, vers) DDLs, err = l.TrySync(info, tts) c.Assert(err, IsNil) c.Assert(DDLs, DeepEquals, DDLs4) @@ -933,19 +957,17 @@ func (t *testLock) TestLockTrySyncConflictIntrusive(c *C) { tblID int64 = 111 DDLs1 = []string{"ALTER TABLE bar ADD COLUMN c1 TEXT"} DDLs2 = []string{"ALTER TABLE bar ADD COLUMN c1 DATETIME", "ALTER TABLE bar ADD COLUMN c2 INT"} - DDLs3 = []string{"ALTER TABLE bar DROP COLUMN c2"} - DDLs4 = []string{"ALTER TABLE bar DROP COLUMN c1"} + DDLs3 = []string{"ALTER TABLE bar ADD COLUMN c1 DATETIME"} 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 TEXT)`) ti2 = createTableInfo(c, p, se, tblID, `CREATE TABLE bar (id INT PRIMARY KEY, c1 DATETIME, c2 INT)`) ti3 = createTableInfo(c, p, se, tblID, `CREATE TABLE bar (id INT PRIMARY KEY, c1 DATETIME)`) - ti4 = ti0 DDLs5 = []string{"ALTER TABLE bar ADD COLUMN c2 TEXT"} DDLs6 = []string{"ALTER TABLE bar ADD COLUMN c2 DATETIME", "ALTER TABLE bar ADD COLUMN c3 INT"} - DDLs7 = []string{"ALTER TABLE bar DROP COLUMN c2"} - DDLs8_1 = []string{"ALTER TABLE bar ADD COLUMN c3 INT"} - DDLs8_2 = []string{"ALTER TABLE bar ADD COLUMN c2 TEXT"} + DDLs7 = []string{"ALTER TABLE bar ADD COLUMN c3 INT"} + DDLs8_1 = DDLs7 + DDLs8_2 = DDLs5 ti5 = createTableInfo(c, p, se, tblID, `CREATE TABLE bar (id INT PRIMARY KEY, c1 TEXT, c2 TEXT)`) ti6 = createTableInfo(c, p, se, tblID, `CREATE TABLE bar (id INT PRIMARY KEY, c1 TEXT, c2 DATETIME, c3 INT)`) ti6_1 = createTableInfo(c, p, se, tblID, `CREATE TABLE bar (id INT PRIMARY KEY, c1 TEXT, c2 DATETIME)`) @@ -954,7 +976,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(ID, task, downSchema, downTable, ti0, tts) + l = NewLock(etcdTestCli, ID, task, downSchema, downTable, schemacmp.Encode(ti0), tts) vers = map[string]map[string]map[string]int64{ source: { @@ -969,7 +991,7 @@ func (t *testLock) TestLockTrySyncConflictIntrusive(c *C) { // CASE: conflict happen, revert all changes to resolve the conflict. // TrySync for the first table, construct the joined schema. - info := newInfoWithVersion(task, source, db, tbls[0], downSchema, downTable, DDLs1, nil, []*model.TableInfo{ti1}, vers) + info := newInfoWithVersion(task, source, db, tbls[0], downSchema, downTable, DDLs1, ti0, []*model.TableInfo{ti1}, vers) DDLs, err := l.TrySync(info, tts) c.Assert(err, IsNil) c.Assert(DDLs, DeepEquals, DDLs1) @@ -985,7 +1007,7 @@ func (t *testLock) TestLockTrySyncConflictIntrusive(c *C) { c.Assert(cmp, Equals, -1) // TrySync for the second table with another schema (add two columns, one of them will cause conflict). - info = newInfoWithVersion(task, source, db, tbls[1], downSchema, downTable, DDLs2, nil, []*model.TableInfo{ti3, ti2}, vers) + info = newInfoWithVersion(task, source, db, tbls[1], downSchema, downTable, DDLs2, ti0, []*model.TableInfo{ti3, ti2}, vers) DDLs, err = l.TrySync(info, tts) c.Assert(terror.ErrShardDDLOptimismTrySyncFail.Equal(err), IsTrue) c.Assert(DDLs, DeepEquals, []string{}) @@ -998,7 +1020,7 @@ func (t *testLock) TestLockTrySyncConflictIntrusive(c *C) { c.Assert(ready[source][db][tbls[1]], IsFalse) // TrySync again. - info = newInfoWithVersion(task, source, db, tbls[1], downSchema, downTable, DDLs2, nil, []*model.TableInfo{ti3, ti2}, vers) + info = newInfoWithVersion(task, source, db, tbls[1], downSchema, downTable, DDLs2, ti0, []*model.TableInfo{ti3, ti2}, vers) DDLs, err = l.TrySync(info, tts) c.Assert(terror.ErrShardDDLOptimismTrySyncFail.Equal(err), IsTrue) c.Assert(DDLs, DeepEquals, []string{}) @@ -1007,8 +1029,8 @@ func (t *testLock) TestLockTrySyncConflictIntrusive(c *C) { c.Assert(err, IsNil) c.Assert(cmp, Equals, -1) - // TrySync for the second table to drop the non-conflict column, the conflict should still exist. - info = newInfoWithVersion(task, source, db, tbls[1], downSchema, downTable, DDLs3, nil, []*model.TableInfo{ti3}, vers) + // TrySync for the second table to replace a new ddl without non-conflict column, the conflict should still exist. + info = newInfoWithVersion(task, source, db, tbls[1], downSchema, downTable, DDLs3, ti0, []*model.TableInfo{ti3}, vers) DDLs, err = l.TrySync(info, tts) c.Assert(terror.ErrShardDDLOptimismTrySyncFail.Equal(err), IsTrue) c.Assert(DDLs, DeepEquals, []string{}) @@ -1019,20 +1041,8 @@ func (t *testLock) TestLockTrySyncConflictIntrusive(c *C) { ready = l.Ready() c.Assert(ready[source][db][tbls[1]], IsFalse) - // TrySync for the second table to drop the conflict column, the conflict should be resolved. - info = newInfoWithVersion(task, source, db, tbls[1], downSchema, downTable, DDLs4, nil, []*model.TableInfo{ti4}, vers) - DDLs, err = l.TrySync(info, tts) - c.Assert(err, IsNil) - c.Assert(DDLs, DeepEquals, []string{}) - c.Assert(l.versions, DeepEquals, vers) - cmp, err = l.tables[source][db][tbls[1]].Compare(l.Joined()) - c.Assert(err, IsNil) - c.Assert(cmp, Equals, -1) - ready = l.Ready() - c.Assert(ready[source][db][tbls[1]], IsFalse) - // TrySync for the second table as we did for the first table, the lock should be synced. - info = newInfoWithVersion(task, source, db, tbls[1], downSchema, downTable, DDLs1, nil, []*model.TableInfo{ti1}, vers) + info = newInfoWithVersion(task, source, db, tbls[1], downSchema, downTable, DDLs1, ti0, []*model.TableInfo{ti1}, vers) DDLs, err = l.TrySync(info, tts) c.Assert(err, IsNil) c.Assert(DDLs, DeepEquals, DDLs) @@ -1045,7 +1055,7 @@ func (t *testLock) TestLockTrySyncConflictIntrusive(c *C) { // CASE: conflict happen, revert part of changes to resolve the conflict. // TrySync for the first table, construct the joined schema. - info = newInfoWithVersion(task, source, db, tbls[0], downSchema, downTable, DDLs5, nil, []*model.TableInfo{ti5}, vers) + info = newInfoWithVersion(task, source, db, tbls[0], downSchema, downTable, DDLs5, ti1, []*model.TableInfo{ti5}, vers) DDLs, err = l.TrySync(info, tts) c.Assert(err, IsNil) c.Assert(DDLs, DeepEquals, DDLs5) @@ -1061,7 +1071,7 @@ func (t *testLock) TestLockTrySyncConflictIntrusive(c *C) { c.Assert(cmp, Equals, -1) // TrySync for the second table with another schema (add two columns, one of them will cause conflict). - info = newInfoWithVersion(task, source, db, tbls[1], downSchema, downTable, DDLs6, nil, []*model.TableInfo{ti6_1, ti6}, vers) + info = newInfoWithVersion(task, source, db, tbls[1], downSchema, downTable, DDLs6, ti1, []*model.TableInfo{ti6_1, ti6}, vers) DDLs, err = l.TrySync(info, tts) c.Assert(terror.ErrShardDDLOptimismTrySyncFail.Equal(err), IsTrue) c.Assert(DDLs, DeepEquals, []string{}) @@ -1072,12 +1082,12 @@ func (t *testLock) TestLockTrySyncConflictIntrusive(c *C) { ready = l.Ready() c.Assert(ready[source][db][tbls[1]], IsFalse) - // TrySync for the second table to drop the conflict column, the conflict should be resolved. + // TrySync for the second table to replace a new ddl without conflict column, the conflict should be resolved. // but both of tables are not synced now. - info = newInfoWithVersion(task, source, db, tbls[1], downSchema, downTable, DDLs7, nil, []*model.TableInfo{ti7}, vers) + info = newInfoWithVersion(task, source, db, tbls[1], downSchema, downTable, DDLs7, ti1, []*model.TableInfo{ti7}, vers) DDLs, err = l.TrySync(info, tts) c.Assert(err, IsNil) - c.Assert(DDLs, DeepEquals, DDLs7) // special case: these DDLs should not be replicated to the downstream. + c.Assert(DDLs, DeepEquals, DDLs7) c.Assert(l.versions, DeepEquals, vers) ready = l.Ready() c.Assert(ready[source][db][tbls[0]], IsFalse) @@ -1090,7 +1100,7 @@ func (t *testLock) TestLockTrySyncConflictIntrusive(c *C) { c.Assert(cmp, Equals, -1) // TrySync for the first table to become synced. - info = newInfoWithVersion(task, source, db, tbls[0], downSchema, downTable, DDLs8_1, nil, []*model.TableInfo{ti8}, vers) + info = newInfoWithVersion(task, source, db, tbls[0], downSchema, downTable, DDLs8_1, ti5, []*model.TableInfo{ti8}, vers) DDLs, err = l.TrySync(info, tts) c.Assert(err, IsNil) c.Assert(DDLs, DeepEquals, DDLs8_1) @@ -1098,7 +1108,7 @@ func (t *testLock) TestLockTrySyncConflictIntrusive(c *C) { c.Assert(ready[source][db][tbls[0]], IsTrue) // TrySync for the second table to become synced. - info = newInfoWithVersion(task, source, db, tbls[1], downSchema, downTable, DDLs8_2, nil, []*model.TableInfo{ti8}, vers) + info = newInfoWithVersion(task, source, db, tbls[1], downSchema, downTable, DDLs8_2, ti7, []*model.TableInfo{ti8}, vers) DDLs, err = l.TrySync(info, tts) c.Assert(err, IsNil) c.Assert(DDLs, DeepEquals, DDLs8_2) @@ -1123,7 +1133,7 @@ func (t *testLock) TestLockTrySyncMultipleChangeDDL(c *C) { p = parser.New() se = mock.NewContext() tblID int64 = 111 - DDLs1 = []string{"ALTER TABLE bar ADD COLUMN c2 INT", "ALTER TABLE DROP COLUMN c1"} + DDLs1 = []string{"ALTER TABLE bar ADD COLUMN c2 INT", "ALTER TABLE bar DROP COLUMN c1"} DDLs2 = []string{"ALTER TABLE bar DROP COLUMN c2", "ALTER TABLE bar ADD COLUMN c3 TEXT"} // DDLs3 = []string{"ALTER TABLE bar DROP COLUMN c3"} // DDLs4 = []string{"ALTER TABLE bar DROP COLUMN c2", "ALTER TABLE bar DROP COLUMN c1"} @@ -1144,7 +1154,7 @@ func (t *testLock) TestLockTrySyncMultipleChangeDDL(c *C) { newTargetTable(task, sources[1], downSchema, downTable, tables), } - l = NewLock(ID, task, downSchema, downTable, ti0, tts) + l = NewLock(etcdTestCli, ID, task, downSchema, downTable, schemacmp.Encode(ti0), tts) vers = map[string]map[string]map[string]int64{ sources[0]: { @@ -1163,12 +1173,12 @@ func (t *testLock) TestLockTrySyncMultipleChangeDDL(c *C) { t.checkLockNoDone(c, l) // inconsistent ddls and table infos - info := newInfoWithVersion(task, sources[0], dbs[0], tbls[0], downSchema, downTable, DDLs1[:1], nil, []*model.TableInfo{ti1_1, ti1}, vers) + info := newInfoWithVersion(task, sources[0], dbs[0], tbls[0], downSchema, downTable, DDLs1[:1], ti0, []*model.TableInfo{ti1_1, ti1}, vers) DDLs, err := l.TrySync(info, tts) c.Assert(DDLs, DeepEquals, DDLs1[:1]) c.Assert(terror.ErrMasterInconsistentOptimisticDDLsAndInfo.Equal(err), IsTrue) - info = newInfoWithVersion(task, sources[0], dbs[0], tbls[0], downSchema, downTable, DDLs1, nil, []*model.TableInfo{ti1}, vers) + info = newInfoWithVersion(task, sources[0], dbs[0], tbls[0], downSchema, downTable, DDLs1, ti0, []*model.TableInfo{ti1}, vers) DDLs, err = l.TrySync(info, tts) c.Assert(DDLs, DeepEquals, DDLs1) c.Assert(terror.ErrMasterInconsistentOptimisticDDLsAndInfo.Equal(err), IsTrue) @@ -1191,7 +1201,7 @@ func (t *testLock) TestLockTrySyncMultipleChangeDDL(c *C) { for _, source := range sources { for _, db := range dbs { for _, tbl := range tbls { - info = newInfoWithVersion(task, source, db, tbl, downSchema, downTable, DDLs1, nil, []*model.TableInfo{ti1_1, ti1}, vers) + info = newInfoWithVersion(task, source, db, tbl, downSchema, downTable, DDLs1, ti0, []*model.TableInfo{ti1_1, ti1}, vers) DDLs, err = l.TrySync(info, tts) c.Assert(err, IsNil) c.Assert(DDLs, DeepEquals, resultDDLs1[source][db][tbl]) @@ -1210,7 +1220,7 @@ func (t *testLock) TestLockTrySyncMultipleChangeDDL(c *C) { // CASE: TrySync again after synced is idempotent. // both ddl will sync again - info = newInfoWithVersion(task, sources[0], dbs[0], tbls[0], downSchema, downTable, DDLs1, nil, []*model.TableInfo{ti1_1, ti1}, vers) + info = newInfoWithVersion(task, sources[0], dbs[0], tbls[0], downSchema, downTable, DDLs1, ti0, []*model.TableInfo{ti1_1, ti1}, vers) DDLs, err = l.TrySync(info, tts) c.Assert(err, IsNil) c.Assert(DDLs, DeepEquals, DDLs1) @@ -1233,7 +1243,7 @@ func (t *testLock) TestLockTrySyncMultipleChangeDDL(c *C) { for _, source := range sources { for _, db := range dbs { for _, tbl := range tbls { - info = newInfoWithVersion(task, source, db, tbl, downSchema, downTable, DDLs2, nil, []*model.TableInfo{ti2_1, ti2}, vers) + info = newInfoWithVersion(task, source, db, tbl, downSchema, downTable, DDLs2, ti1, []*model.TableInfo{ti2_1, ti2}, vers) DDLs, err = l.TrySync(info, tts) c.Assert(err, IsNil) c.Assert(DDLs, DeepEquals, resultDDLs2[source][db][tbl]) @@ -1251,11 +1261,10 @@ func (t *testLock) TestLockTrySyncMultipleChangeDDL(c *C) { t.checkLockNoDone(c, l) // CASE: TrySync again after synced is idempotent. - // only the second ddl(ADD COLUMN) will sync, the first one(DROP COLUMN) will not sync since oldJoined==newJoined - info = newInfoWithVersion(task, sources[0], dbs[0], tbls[0], downSchema, downTable, DDLs2, nil, []*model.TableInfo{ti2_1, ti2}, vers) + info = newInfoWithVersion(task, sources[0], dbs[0], tbls[0], downSchema, downTable, DDLs2, ti1, []*model.TableInfo{ti2_1, ti2}, vers) DDLs, err = l.TrySync(info, tts) c.Assert(err, IsNil) - c.Assert(DDLs, DeepEquals, DDLs2[1:]) + c.Assert(DDLs, DeepEquals, DDLs2) c.Assert(l.versions, DeepEquals, vers) t.checkLockSynced(c, l) t.checkLockNoDone(c, l) @@ -1282,7 +1291,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(ID, task, downSchema, downTable, ti0, tts) + l = NewLock(etcdTestCli, ID, task, downSchema, downTable, schemacmp.Encode(ti0), tts) vers = map[string]map[string]map[string]int64{ source: { @@ -1297,7 +1306,7 @@ func (t *testLock) TestTryRemoveTable(c *C) { // CASE: remove a table as normal. // TrySync for the first table. - info := newInfoWithVersion(task, source, db, tbl1, downSchema, downTable, DDLs1, nil, []*model.TableInfo{ti1}, vers) + info := newInfoWithVersion(task, source, db, tbl1, downSchema, downTable, DDLs1, ti0, []*model.TableInfo{ti1}, vers) DDLs, err := l.TrySync(info, tts) c.Assert(err, IsNil) c.Assert(DDLs, DeepEquals, DDLs1) @@ -1322,7 +1331,7 @@ func (t *testLock) TestTryRemoveTable(c *C) { // CASE: remove a table will not rebuild joined schema now. // TrySync to add the second back. vers[source][db][tbl2] = 0 - info = newInfoWithVersion(task, source, db, tbl2, downSchema, downTable, DDLs2, nil, []*model.TableInfo{ti2}, vers) + info = newInfoWithVersion(task, source, db, tbl2, downSchema, downTable, DDLs2, ti1, []*model.TableInfo{ti2}, vers) DDLs, err = l.TrySync(info, tts) c.Assert(err, IsNil) c.Assert(DDLs, DeepEquals, DDLs2) @@ -1372,7 +1381,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(ID, task, downSchema, downTable, ti0, tts) + l = NewLock(etcdTestCli, ID, task, downSchema, downTable, schemacmp.Encode(ti0), tts) vers = map[string]map[string]map[string]int64{ source: { @@ -1387,7 +1396,7 @@ func (t *testLock) TestLockTryMarkDone(c *C) { c.Assert(l.IsResolved(), IsFalse) // TrySync for the first table, no table has done the DDLs operation. - info := newInfoWithVersion(task, source, db, tbls[0], downSchema, downTable, DDLs1, nil, []*model.TableInfo{ti1}, vers) + info := newInfoWithVersion(task, source, db, tbls[0], downSchema, downTable, DDLs1, ti0, []*model.TableInfo{ti1}, vers) DDLs, err := l.TrySync(info, tts) c.Assert(err, IsNil) c.Assert(DDLs, DeepEquals, DDLs1) @@ -1402,7 +1411,7 @@ func (t *testLock) TestLockTryMarkDone(c *C) { c.Assert(l.IsResolved(), IsFalse) // TrySync for the second table, the joined schema become larger. - info = newInfoWithVersion(task, source, db, tbls[1], downSchema, downTable, DDLs2, nil, []*model.TableInfo{ti1, ti2}, vers) + info = newInfoWithVersion(task, source, db, tbls[1], downSchema, downTable, DDLs2, ti0, []*model.TableInfo{ti1, ti2}, vers) DDLs, err = l.TrySync(info, tts) c.Assert(err, IsNil) c.Assert(DDLs, DeepEquals, DDLs2) @@ -1421,7 +1430,7 @@ func (t *testLock) TestLockTryMarkDone(c *C) { c.Assert(l.IsResolved(), IsFalse) // TrySync for the first table, all tables become synced. - info = newInfoWithVersion(task, source, db, tbls[0], downSchema, downTable, DDLs3, nil, []*model.TableInfo{ti3}, vers) + info = newInfoWithVersion(task, source, db, tbls[0], downSchema, downTable, DDLs3, ti1, []*model.TableInfo{ti3}, vers) DDLs, err = l.TrySync(info, tts) c.Assert(err, IsNil) c.Assert(DDLs, DeepEquals, DDLs3) @@ -1453,8 +1462,8 @@ func (t *testLock) TestLockTryMarkDone(c *C) { func (t *testLock) TestAddDifferentFieldLenColumns(c *C) { var ( - ID = "test_lock_try_mark_done-`foo`.`bar`" - task = "test_lock_try_mark_done" + ID = "test_lock_add_diff_flen_cols-`foo`.`bar`" + task = "test_lock_add_diff_flen_cols" source = "mysql-replica-1" downSchema = "foo" downTable = "bar" @@ -1477,7 +1486,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(ID, task, downSchema, downTable, ti0, tts) + l = NewLock(etcdTestCli, ID, task, downSchema, downTable, schemacmp.Encode(ti0), tts) vers = map[string]map[string]map[string]int64{ source: { @@ -1485,9 +1494,15 @@ func (t *testLock) TestAddDifferentFieldLenColumns(c *C) { }, } ) - c.Assert(AddDifferentFieldLenColumns(ID, DDLs1[0], table1, table2), IsNil) - c.Assert(AddDifferentFieldLenColumns(ID, DDLs2[0], table2, table3), ErrorMatches, ".*add columns with different field lengths.*") - c.Assert(AddDifferentFieldLenColumns(ID, DDLs1[0], table3, table2), ErrorMatches, ".*add columns with different field lengths.*") + col, err := AddDifferentFieldLenColumns(ID, DDLs1[0], table1, table2) + c.Assert(col, Equals, "c1") + c.Assert(err, IsNil) + col, err = AddDifferentFieldLenColumns(ID, DDLs2[0], table2, table3) + c.Assert(col, Equals, "c1") + c.Assert(err, ErrorMatches, ".*add columns with different field lengths.*") + col, err = AddDifferentFieldLenColumns(ID, DDLs1[0], table3, table2) + c.Assert(col, Equals, "c1") + c.Assert(err, ErrorMatches, ".*add columns with different field lengths.*") // the initial status is synced but not resolved. t.checkLockSynced(c, l) @@ -1495,7 +1510,7 @@ func (t *testLock) TestAddDifferentFieldLenColumns(c *C) { c.Assert(l.IsResolved(), IsFalse) // TrySync for the first table, no table has done the DDLs operation. - info := newInfoWithVersion(task, source, db, tbls[0], downSchema, downTable, DDLs1, nil, []*model.TableInfo{ti1}, vers) + info := newInfoWithVersion(task, source, db, tbls[0], downSchema, downTable, DDLs1, ti0, []*model.TableInfo{ti1}, vers) DDLs, err := l.TrySync(info, tts) c.Assert(err, IsNil) c.Assert(DDLs, DeepEquals, DDLs1) @@ -1504,18 +1519,18 @@ func (t *testLock) TestAddDifferentFieldLenColumns(c *C) { c.Assert(l.IsResolved(), IsFalse) // TrySync for the second table, add a table with a larger field length - info = newInfoWithVersion(task, source, db, tbls[1], downSchema, downTable, DDLs2, nil, []*model.TableInfo{ti2}, vers) + info = newInfoWithVersion(task, source, db, tbls[1], downSchema, downTable, DDLs2, ti0, []*model.TableInfo{ti2}, vers) DDLs, err = l.TrySync(info, tts) c.Assert(err, ErrorMatches, ".*add columns with different field lengths.*") c.Assert(DDLs, DeepEquals, DDLs2) c.Assert(l.versions, DeepEquals, vers) // case 2: add a column with a smaller field length - l = NewLock(ID, task, downSchema, downTable, ti0, tts) + l = NewLock(etcdTestCli, ID, task, downSchema, downTable, schemacmp.Encode(ti0), tts) // TrySync for the first table, no table has done the DDLs operation. vers[source][db][tbls[0]]-- - info = NewInfo(task, source, db, tbls[1], downSchema, downTable, DDLs2, nil, []*model.TableInfo{ti2}) + info = NewInfo(task, source, db, tbls[1], downSchema, downTable, DDLs2, ti0, []*model.TableInfo{ti2}) info.Version = vers[source][db][tbls[1]] DDLs, err = l.TrySync(info, tts) c.Assert(err, IsNil) @@ -1525,22 +1540,139 @@ func (t *testLock) TestAddDifferentFieldLenColumns(c *C) { c.Assert(l.IsResolved(), IsFalse) // TrySync for the second table, add a table with a smaller field length - info = newInfoWithVersion(task, source, db, tbls[0], downSchema, downTable, DDLs1, nil, []*model.TableInfo{ti1}, vers) + info = newInfoWithVersion(task, source, db, tbls[0], downSchema, downTable, DDLs1, ti0, []*model.TableInfo{ti1}, vers) DDLs, err = l.TrySync(info, tts) c.Assert(err, ErrorMatches, ".*add columns with different field lengths.*") c.Assert(DDLs, DeepEquals, DDLs1) c.Assert(l.versions, DeepEquals, vers) } +func (t *testLock) TestAddNotFullyDroppedColumns(c *C) { + var ( + ID = "test_lock_add_not_fully_dropped_cols-`foo`.`bar`" + task = "test_lock_add_not_fully_dropped_cols" + source = "mysql-replica-1" + downSchema = "foo" + downTable = "bar" + db = "foo" + 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, b int, c int)`) + ti1 = createTableInfo(c, p, se, tblID, `CREATE TABLE bar (id INT PRIMARY KEY, b int)`) + ti2 = createTableInfo(c, p, se, tblID, `CREATE TABLE bar (id INT PRIMARY KEY)`) + ti3 = createTableInfo(c, p, se, tblID, `CREATE TABLE bar (id INT PRIMARY KEY, c int)`) + + DDLs1 = []string{"ALTER TABLE bar DROP COLUMN c"} + DDLs2 = []string{"ALTER TABLE bar DROP COLUMN b"} + DDLs3 = []string{"ALTER TABLE bar ADD COLUMN b INT"} + DDLs4 = []string{"ALTER TABLE bar ADD COLUMN c INT"} + + 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) + + vers = map[string]map[string]map[string]int64{ + source: { + db: {tbls[0]: 0, tbls[1]: 0}, + }, + } + + colm1 = map[string]map[string]map[string]map[string]map[string]struct{}{ + ID: { + "b": {source: {db: {tbls[0]: struct{}{}}}}, + "c": {source: {db: {tbls[0]: struct{}{}}}}, + }, + } + colm2 = map[string]map[string]map[string]map[string]map[string]struct{}{ + ID: { + "c": {source: {db: {tbls[0]: struct{}{}}}}, + }, + } + ) + col, err := GetColumnName(ID, DDLs1[0], ast.AlterTableDropColumn) + c.Assert(col, Equals, "c") + c.Assert(err, IsNil) + col, err = GetColumnName(ID, DDLs2[0], ast.AlterTableDropColumn) + c.Assert(col, Equals, "b") + c.Assert(err, IsNil) + + // the initial status is synced but not resolved. + t.checkLockSynced(c, l) + t.checkLockNoDone(c, l) + c.Assert(l.IsResolved(), IsFalse) + + // TrySync for the first table, drop column c + DDLs, err := l.TrySync(newInfoWithVersion(task, source, db, tbls[0], downSchema, downTable, DDLs1, ti0, []*model.TableInfo{ti1}, vers), tts) + c.Assert(err, IsNil) + c.Assert(DDLs, DeepEquals, []string{}) + c.Assert(l.versions, DeepEquals, vers) + c.Assert(l.IsResolved(), IsFalse) + + // TrySync for the first table, drop column b + DDLs, err = l.TrySync(newInfoWithVersion(task, source, db, tbls[0], downSchema, downTable, DDLs2, ti1, []*model.TableInfo{ti2}, vers), tts) + c.Assert(err, IsNil) + c.Assert(DDLs, DeepEquals, []string{}) + c.Assert(l.versions, DeepEquals, vers) + c.Assert(l.IsResolved(), IsFalse) + + colm, _, err := GetAllDroppedColumns(etcdTestCli) + c.Assert(err, IsNil) + c.Assert(colm, DeepEquals, colm1) + + // TrySync for the second table, drop column b, this column should be fully dropped + DDLs, err = l.TrySync(newInfoWithVersion(task, source, db, tbls[1], downSchema, downTable, DDLs2, ti0, []*model.TableInfo{ti3}, vers), tts) + c.Assert(err, IsNil) + c.Assert(DDLs, DeepEquals, DDLs2) + c.Assert(l.versions, DeepEquals, vers) + c.Assert(l.IsResolved(), IsFalse) + // Simulate watch done operation from dm-worker + c.Assert(l.DeleteColumnsByDDLs(DDLs), IsNil) + + colm, _, err = GetAllDroppedColumns(etcdTestCli) + c.Assert(err, IsNil) + c.Assert(colm, DeepEquals, colm2) + + // TrySync for the first table, add column b, should succeed, because this column is fully dropped in the downstream + DDLs, err = l.TrySync(newInfoWithVersion(task, source, db, tbls[0], downSchema, downTable, DDLs3, ti2, []*model.TableInfo{ti1}, vers), tts) + c.Assert(err, IsNil) + c.Assert(DDLs, DeepEquals, DDLs3) + c.Assert(l.versions, DeepEquals, vers) + c.Assert(l.IsResolved(), IsFalse) + + // TrySync for the first table, add column c, should fail, because this column isn't fully dropped in the downstream + _, err = l.TrySync(newInfoWithVersion(task, source, db, tbls[0], downSchema, downTable, DDLs4, ti1, []*model.TableInfo{ti0}, vers), tts) + c.Assert(err, ErrorMatches, ".*add column c that wasn't fully dropped in downstream.*") + c.Assert(l.IsResolved(), IsFalse) + + // TrySync for the second table, drop column c, this column should be fully dropped + DDLs, err = l.TrySync(newInfoWithVersion(task, source, db, tbls[1], downSchema, downTable, DDLs1, ti3, []*model.TableInfo{ti2}, vers), tts) + c.Assert(err, IsNil) + c.Assert(DDLs, DeepEquals, DDLs1) + c.Assert(l.versions, DeepEquals, vers) + c.Assert(l.IsResolved(), IsFalse) + // Simulate watch done operation from dm-worker + c.Assert(l.DeleteColumnsByDDLs(DDLs), IsNil) + + // TrySync for the first table, add column c, should succeed, because this column is fully dropped in the downstream + DDLs, err = l.TrySync(newInfoWithVersion(task, source, db, tbls[0], downSchema, downTable, DDLs4, ti1, []*model.TableInfo{ti0}, vers), tts) + c.Assert(err, IsNil) + c.Assert(DDLs, DeepEquals, DDLs4) + c.Assert(l.versions, DeepEquals, vers) + c.Assert(l.IsResolved(), IsFalse) +} + func (t *testLock) trySyncForAllTablesLarger(c *C, l *Lock, - DDLs []string, tis []*model.TableInfo, tts []TargetTable, vers map[string]map[string]map[string]int64, resultDDLs map[string]map[string]map[string][]string) { + DDLs []string, tableInfoBefore *model.TableInfo, tis []*model.TableInfo, tts []TargetTable, vers map[string]map[string]map[string]int64) { for source, schemaTables := range l.Ready() { for schema, tables := range schemaTables { for table := range tables { - info := newInfoWithVersion(l.Task, source, schema, table, l.DownSchema, l.DownTable, DDLs, nil, tis, vers) + info := newInfoWithVersion(l.Task, source, schema, table, l.DownSchema, l.DownTable, DDLs, tableInfoBefore, tis, vers) DDLs2, err := l.TrySync(info, tts) c.Assert(err, IsNil) - c.Assert(DDLs2, DeepEquals, resultDDLs[source][schema][table]) + c.Assert(DDLs2, DeepEquals, DDLs) } } } @@ -1580,3 +1712,81 @@ func newInfoWithVersion(task, source, upSchema, upTable, downSchema, downTable s info.Version = vers[source][upSchema][upTable] return info } + +func (t *testLock) TestLockTrySyncDifferentIndex(c *C) { + var ( + ID = "test_lock_try_sync_index-`foo`.`bar`" + task = "test_lock_try_sync_index" + source = "mysql-replica-1" + downSchema = "db" + downTable = "bar" + db = "db" + tbls = []string{"bar1", "bar2"} + p = parser.New() + se = mock.NewContext() + tblID int64 = 111 + DDLs1 = []string{"ALTER TABLE bar DROP INDEX idx_c1"} + DDLs2 = []string{"ALTER TABLE bar ADD INDEX new_idx(c1)"} + ti0 = createTableInfo(c, p, se, tblID, `CREATE TABLE bar (id INT PRIMARY KEY, c1 INT, UNIQUE INDEX idx_c1(c1))`) + 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, INDEX new_idx(c1))`) + 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) + + vers = map[string]map[string]map[string]int64{ + source: { + db: {tbls[0]: 0, tbls[1]: 0}, + }, + } + ) + + // the initial status is synced. + t.checkLockSynced(c, l) + t.checkLockNoDone(c, l) + + // try sync for one table, `DROP INDEX` returned directly (to make schema become more compatible). + // `DROP INDEX` is handled like `ADD COLUMN`. + info := newInfoWithVersion(task, source, db, tbls[0], downSchema, downTable, DDLs1, ti0, []*model.TableInfo{ti1}, vers) + DDLs, err := l.TrySync(info, tts) + c.Assert(err, IsNil) + c.Assert(DDLs, DeepEquals, DDLs1) + c.Assert(l.versions, DeepEquals, vers) + synced, remain := l.IsSynced() + c.Assert(synced, Equals, l.synced) + c.Assert(synced, IsFalse) + c.Assert(remain, Equals, 1) + + cmp, err := l.tables[source][db][tbls[1]].Compare(schemacmp.Encode(ti0)) + c.Assert(err, IsNil) + c.Assert(cmp, Equals, 0) + + // try sync ADD another INDEX for another table + // `ADD INDEX` is handled like `DROP COLUMN`. + info = newInfoWithVersion(task, source, db, tbls[1], downSchema, downTable, DDLs2, ti1, []*model.TableInfo{ti2}, vers) + DDLs, err = l.TrySync(info, tts) + c.Assert(err, IsNil) + c.Assert(DDLs, DeepEquals, []string{}) // no DDLs returned + c.Assert(l.versions, DeepEquals, vers) + synced, remain = l.IsSynced() + c.Assert(synced, Equals, l.synced) + c.Assert(synced, IsFalse) + c.Assert(remain, Equals, 1) + + cmp, err = l.tables[source][db][tbls[0]].Compare(l.joined) + c.Assert(err, IsNil) + c.Assert(cmp, Equals, 0) + + // try sync ADD INDEX for first table + info = newInfoWithVersion(task, source, db, tbls[0], downSchema, downTable, DDLs2, ti1, []*model.TableInfo{ti2}, vers) + DDLs, err = l.TrySync(info, tts) + c.Assert(err, IsNil) + c.Assert(DDLs, DeepEquals, DDLs2) + c.Assert(l.versions, DeepEquals, vers) + t.checkLockSynced(c, l) +} diff --git a/pkg/shardddl/optimism/operation.go b/pkg/shardddl/optimism/operation.go index 645cd2783e..ac8b8526fe 100644 --- a/pkg/shardddl/optimism/operation.go +++ b/pkg/shardddl/optimism/operation.go @@ -49,19 +49,20 @@ const ( // and is deleted when removing the lock by DM-master. // because we need the newest stage in Operation to recover the lock when restarting DM-master. type Operation struct { - ID string `json:"id"` // the corresponding DDL lock ID - Task string `json:"task"` // data migration task name - Source string `json:"source"` // upstream source ID - UpSchema string `json:"up-schema"` // upstream/source schema name, different sources can have the same schema name - UpTable string `json:"up-table"` // upstream/source table name, different sources can have the same table name - DDLs []string `json:"ddls"` // DDL statements need to apply to the downstream. - ConflictStage ConflictStage `json:"conflict-stage"` // current conflict stage. - Done bool `json:"done"` // whether the operation has done + ID string `json:"id"` // the corresponding DDL lock ID + Task string `json:"task"` // data migration task name + Source string `json:"source"` // upstream source ID + UpSchema string `json:"up-schema"` // upstream/source schema name, different sources can have the same schema name + UpTable string `json:"up-table"` // upstream/source table name, different sources can have the same table name + DDLs []string `json:"ddls"` // DDL statements need to apply to the downstream. + ConflictStage ConflictStage `json:"conflict-stage"` // current conflict stage. + ConflictMsg string `json:"conflict-message"` // current conflict message + Done bool `json:"done"` // whether the operation has done } // NewOperation creates a new Operation instance. func NewOperation(ID, task, source, upSchema, upTable string, - DDLs []string, conflictStage ConflictStage, done bool) Operation { + DDLs []string, conflictStage ConflictStage, conflictMsg string, done bool) Operation { return Operation{ ID: ID, Task: task, @@ -70,6 +71,7 @@ func NewOperation(ID, task, source, upSchema, upTable string, UpTable: upTable, DDLs: DDLs, ConflictStage: conflictStage, + ConflictMsg: conflictMsg, Done: done, } } @@ -96,7 +98,7 @@ func operationFromJSON(s string) (o Operation, err error) { } // PutOperation puts the shard DDL operation into etcd. -func PutOperation(cli *clientv3.Client, skipDone bool, op Operation) (rev int64, putted bool, err error) { +func PutOperation(cli *clientv3.Client, skipDone bool, op Operation, infoModRev int64) (rev int64, putted bool, err error) { value, err := op.toJSON() if err != nil { return 0, false, err @@ -106,6 +108,7 @@ func PutOperation(cli *clientv3.Client, skipDone bool, op Operation) (rev int64, cmpsNotExist := make([]clientv3.Cmp, 0, 1) cmpsNotDone := make([]clientv3.Cmp, 0, 1) + cmpsLessRev := make([]clientv3.Cmp, 0, 1) if skipDone { opDone := op opDone.Done = true // set `done` to `true`. @@ -115,6 +118,7 @@ func PutOperation(cli *clientv3.Client, skipDone bool, op Operation) (rev int64, } cmpsNotExist = append(cmpsNotExist, clientv3util.KeyMissing(key)) cmpsNotDone = append(cmpsNotDone, clientv3.Compare(clientv3.Value(key), "!=", valueDone)) + cmpsLessRev = append(cmpsLessRev, clientv3.Compare(clientv3.ModRevision(key), "<", infoModRev)) } ctx, cancel := context.WithTimeout(cli.Ctx(), etcdutil.DefaultRequestTimeout) @@ -130,6 +134,20 @@ func PutOperation(cli *clientv3.Client, skipDone bool, op Operation) (rev int64, // txn 2: try to PUT if the key "the `done`" field is not `true`. resp, err = cli.Txn(ctx).If(cmpsNotDone...).Then(opPut).Commit() + if err != nil { + return 0, false, err + } else if resp.Succeeded { + return resp.Header.Revision, resp.Succeeded, nil + } + + // txn 3: try to PUT if the key has less mod revision than info's mod revision, which means this operation is an old one + // without this, failed case time series: + // 1. dm-master received an old done DDL operation from dm-worker + // 2. dm-worker putted a new DDL info into dm-master + // 3. dm-master quited before dm-master putted the DDL operation to dm-worker + // 4. dm-master restarted and tried to put DDL operation, but found a done one and failed to put + // 5. dm-worker didn't receive a DDL operation, will get blocked forever + resp, err = cli.Txn(ctx).If(cmpsLessRev...).Then(opPut).Commit() if err != nil { return 0, false, err } diff --git a/pkg/shardddl/optimism/operation_test.go b/pkg/shardddl/optimism/operation_test.go index 116b12a4af..e6832c3899 100644 --- a/pkg/shardddl/optimism/operation_test.go +++ b/pkg/shardddl/optimism/operation_test.go @@ -23,11 +23,11 @@ import ( func (t *testForEtcd) TestOperationJSON(c *C) { o1 := NewOperation("test-ID", "test", "mysql-replica-1", "db-1", "tbl-1", []string{ "ALTER TABLE tbl ADD COLUMN c1 INT", - }, ConflictDetected, true) + }, ConflictDetected, "conflict", true) j, err := o1.toJSON() c.Assert(err, IsNil) - c.Assert(j, Equals, `{"id":"test-ID","task":"test","source":"mysql-replica-1","up-schema":"db-1","up-table":"tbl-1","ddls":["ALTER TABLE tbl ADD COLUMN c1 INT"],"conflict-stage":"detected","done":true}`) + c.Assert(j, Equals, `{"id":"test-ID","task":"test","source":"mysql-replica-1","up-schema":"db-1","up-table":"tbl-1","ddls":["ALTER TABLE tbl ADD COLUMN c1 INT"],"conflict-stage":"detected","conflict-message":"conflict","done":true}`) c.Assert(j, Equals, o1.String()) o2, err := operationFromJSON(j) @@ -48,15 +48,15 @@ func (t *testForEtcd) TestOperationEtcd(c *C) { ID2 = "test2-`foo`.`bar`" source1 = "mysql-replica-1" DDLs = []string{"ALTER TABLE bar ADD COLUMN c1 INT"} - op11 = NewOperation(ID1, task1, source1, upSchema, upTable, DDLs, ConflictNone, false) - op21 = NewOperation(ID2, task2, source1, upSchema, upTable, DDLs, ConflictResolved, true) + op11 = NewOperation(ID1, task1, source1, upSchema, upTable, DDLs, ConflictNone, "", false) + op21 = NewOperation(ID2, task2, source1, upSchema, upTable, DDLs, ConflictResolved, "", true) ) // put the same keys twice. - rev1, succ, err := PutOperation(etcdTestCli, false, op11) + rev1, succ, err := PutOperation(etcdTestCli, false, op11, 0) c.Assert(err, IsNil) c.Assert(succ, IsTrue) - rev2, succ, err := PutOperation(etcdTestCli, false, op11) + rev2, succ, err := PutOperation(etcdTestCli, false, op11, 0) c.Assert(err, IsNil) c.Assert(succ, IsTrue) c.Assert(rev2, Greater, rev1) @@ -76,7 +76,7 @@ func (t *testForEtcd) TestOperationEtcd(c *C) { c.Assert(<-wch, DeepEquals, op11) // put for another task. - rev3, succ, err := PutOperation(etcdTestCli, false, op21) + rev3, succ, err := PutOperation(etcdTestCli, false, op21, 0) c.Assert(err, IsNil) c.Assert(succ, IsTrue) @@ -109,7 +109,7 @@ func (t *testForEtcd) TestOperationEtcd(c *C) { // put for `skipDone` with `done` in etcd, the operations should not be skipped. // case: kv's "the `done` field is not `true`". - rev5, succ, err := PutOperation(etcdTestCli, true, op11) + rev5, succ, err := PutOperation(etcdTestCli, true, op11, 0) c.Assert(err, IsNil) c.Assert(succ, IsTrue) c.Assert(rev5, Greater, rev4) @@ -126,7 +126,7 @@ func (t *testForEtcd) TestOperationEtcd(c *C) { // put for `skipDone` with `done` in etcd, the operations should not be skipped. // case: kv "not exist". - rev6, succ, err := PutOperation(etcdTestCli, true, op11) + rev6, succ, err := PutOperation(etcdTestCli, true, op11, 0) c.Assert(err, IsNil) c.Assert(succ, IsTrue) @@ -139,15 +139,22 @@ func (t *testForEtcd) TestOperationEtcd(c *C) { // update op11 to `done`. op11c := op11 op11c.Done = true - rev7, succ, err := PutOperation(etcdTestCli, true, op11c) + rev7, succ, err := PutOperation(etcdTestCli, true, op11c, 0) c.Assert(err, IsNil) c.Assert(succ, IsTrue) c.Assert(rev7, Greater, rev6) + // put for `skipDone` with `done` in etcd, the operations should not be skipped. + // case: operation modRevision < info's modRevision + rev8, succ, err := PutOperation(etcdTestCli, true, op11c, rev7+10) + c.Assert(err, IsNil) + c.Assert(succ, IsTrue) + c.Assert(rev8, Greater, rev7) + // put for `skipDone` with `done` in etcd, the operations should be skipped. // case: kv's ("exist" and "the `done` field is `true`"). - rev8, succ, err := PutOperation(etcdTestCli, true, op11) + rev9, succ, err := PutOperation(etcdTestCli, true, op11, rev6) c.Assert(err, IsNil) c.Assert(succ, IsFalse) - c.Assert(rev8, Equals, rev7) + c.Assert(rev9, Equals, rev8) } diff --git a/pkg/shardddl/optimism/ops.go b/pkg/shardddl/optimism/ops.go index b855656b31..b5424f47af 100644 --- a/pkg/shardddl/optimism/ops.go +++ b/pkg/shardddl/optimism/ops.go @@ -47,10 +47,10 @@ func PutSourceTablesDeleteInfo(cli *clientv3.Client, st SourceTables, info Info) return rev, err } -// DeleteInfosOperationsSchema deletes the shard DDL infos, operations and init schemas in etcd. +// DeleteInfosOperationsSchemaColumn deletes the shard DDL infos, operations, init schemas and dropped columns in etcd. // This function should often be called by DM-master when removing the lock. // Only delete when all info's version are greater or equal to etcd's version, otherwise it means new info was putted into etcd before. -func DeleteInfosOperationsSchema(cli *clientv3.Client, infos []Info, ops []Operation, schema InitSchema) (int64, bool, error) { +func DeleteInfosOperationsSchemaColumn(cli *clientv3.Client, infos []Info, ops []Operation, schema InitSchema) (int64, bool, error) { opsDel := make([]clientv3.Op, 0, len(infos)+len(ops)) cmps := make([]clientv3.Cmp, 0, len(infos)) for _, info := range infos { @@ -62,6 +62,7 @@ func DeleteInfosOperationsSchema(cli *clientv3.Client, infos []Info, ops []Opera opsDel = append(opsDel, deleteOperationOp(op)) } opsDel = append(opsDel, deleteInitSchemaOp(schema.Task, schema.DownSchema, schema.DownTable)) + opsDel = append(opsDel, deleteDroppedColumnsByLockOp(schema.Task, schema.DownSchema, schema.DownTable)) resp, rev, err := etcdutil.DoOpsInOneCmpsTxnWithRetry(cli, cmps, opsDel, []clientv3.Op{}) if err != nil { return 0, false, err @@ -71,11 +72,12 @@ func DeleteInfosOperationsSchema(cli *clientv3.Client, infos []Info, ops []Opera // DeleteInfosOperationsTablesSchemasByTask deletes the shard DDL infos and operations in etcd. func DeleteInfosOperationsTablesSchemasByTask(cli *clientv3.Client, task string) (int64, error) { - opsDel := make([]clientv3.Op, 0, 3) + 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())) + opsDel = append(opsDel, clientv3.OpDelete(common.ShardDDLOptimismDroppedColumnsKeyAdapter.Encode(task), clientv3.WithPrefix())) _, rev, err := etcdutil.DoOpsInOneTxnWithRetry(cli, opsDel...) return rev, err } diff --git a/pkg/shardddl/optimism/ops_test.go b/pkg/shardddl/optimism/ops_test.go index f6f4794e81..1258a71100 100644 --- a/pkg/shardddl/optimism/ops_test.go +++ b/pkg/shardddl/optimism/ops_test.go @@ -29,22 +29,23 @@ func (t *testForEtcd) TestDeleteInfosOperationsSchema(c *C) { downTable = "bar" 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) + op = NewOperation("test-ID", task, source, upSchema, upTable, DDLs, ConflictResolved, "", false) is = NewInitSchema(task, downSchema, downTable, nil) ) // put info. - _, err := PutInfo(etcdTestCli, info) + rev, err := PutInfo(etcdTestCli, info) c.Assert(err, IsNil) ifm, _, err := GetAllInfo(etcdTestCli) c.Assert(err, IsNil) c.Assert(ifm, HasLen, 1) infoWithVer := info infoWithVer.Version = 1 + infoWithVer.Revision = rev c.Assert(ifm[task][source][upSchema][upTable], DeepEquals, infoWithVer) // put operation. - _, _, err = PutOperation(etcdTestCli, false, op) + _, _, err = PutOperation(etcdTestCli, false, op, 0) c.Assert(err, IsNil) opm, _, err := GetAllOperations(etcdTestCli) c.Assert(err, IsNil) @@ -59,7 +60,7 @@ func (t *testForEtcd) TestDeleteInfosOperationsSchema(c *C) { c.Assert(isc, DeepEquals, is) // DELETE info and operation with version 0 - _, deleted, err := DeleteInfosOperationsSchema(etcdTestCli, []Info{info}, []Operation{op}, is) + _, deleted, err := DeleteInfosOperationsSchemaColumn(etcdTestCli, []Info{info}, []Operation{op}, is) c.Assert(err, IsNil) c.Assert(deleted, IsFalse) @@ -75,7 +76,7 @@ func (t *testForEtcd) TestDeleteInfosOperationsSchema(c *C) { c.Assert(isc.IsEmpty(), IsFalse) // DELETE info and operation with version 1 - _, deleted, err = DeleteInfosOperationsSchema(etcdTestCli, []Info{infoWithVer}, []Operation{op}, is) + _, deleted, err = DeleteInfosOperationsSchemaColumn(etcdTestCli, []Info{infoWithVer}, []Operation{op}, is) c.Assert(err, IsNil) c.Assert(deleted, IsTrue) @@ -133,6 +134,7 @@ func (t *testForEtcd) TestSourceTablesInfo(c *C) { c.Assert(ifm[task][source][upSchema], HasLen, 1) i11WithVer := i11 i11WithVer.Version = 1 + i11WithVer.Revision = rev3 c.Assert(ifm[task][source][upSchema][upTable], DeepEquals, i11WithVer) // put/update source tables and delete info. diff --git a/pkg/streamer/reader.go b/pkg/streamer/reader.go index 2d27d4659f..3fce85cc1d 100644 --- a/pkg/streamer/reader.go +++ b/pkg/streamer/reader.go @@ -75,6 +75,7 @@ type BinlogReader struct { tctx *tcontext.Context + usingGTID bool prevGset, currGset mysql.GTIDSet } @@ -251,6 +252,7 @@ func (r *BinlogReader) StartSyncByPos(pos mysql.Position) (Streamer, error) { // StartSyncByGTID start sync by gtid func (r *BinlogReader) StartSyncByGTID(gset mysql.GTIDSet) (Streamer, error) { r.tctx.L().Info("begin to sync binlog", zap.Stringer("GTID Set", gset)) + r.usingGTID = true if r.running { return nil, terror.ErrReaderAlreadyRunning.Generate() @@ -400,6 +402,7 @@ func (r *BinlogReader) parseFileAsPossible(ctx context.Context, s *LocalStreamer needReParse bool ) latestPos = offset + replaceWithHeartbeat := false r.tctx.L().Debug("start to parse relay log file", zap.String("file", relayLogFile), zap.Int64("position", latestPos), zap.String("directory", relayLogDir)) for { @@ -408,7 +411,7 @@ func (r *BinlogReader) parseFileAsPossible(ctx context.Context, s *LocalStreamer return false, 0, "", "", ctx.Err() default: } - needSwitch, needReParse, latestPos, nextUUID, nextBinlogName, err = r.parseFile(ctx, s, relayLogFile, latestPos, relayLogDir, firstParse, currentUUID, possibleLast) + needSwitch, needReParse, latestPos, nextUUID, nextBinlogName, replaceWithHeartbeat, err = r.parseFile(ctx, s, relayLogFile, latestPos, relayLogDir, firstParse, currentUUID, possibleLast, replaceWithHeartbeat) firstParse = false // set to false to handle the `continue` below if err != nil { return false, 0, "", "", terror.Annotatef(err, "parse relay log file %s from offset %d in dir %s", relayLogFile, latestPos, relayLogDir) @@ -422,18 +425,25 @@ func (r *BinlogReader) parseFileAsPossible(ctx context.Context, s *LocalStreamer } // parseFile parses single relay log file from specified offset +// TODO: move all stateful variables into a class, such as r.fileParser func (r *BinlogReader) parseFile( - ctx context.Context, s *LocalStreamer, relayLogFile string, offset int64, - relayLogDir string, firstParse bool, currentUUID string, possibleLast bool) ( - needSwitch, needReParse bool, latestPos int64, nextUUID string, nextBinlogName string, err error) { + ctx context.Context, + s *LocalStreamer, + relayLogFile string, + offset int64, + relayLogDir string, + firstParse bool, + currentUUID string, + possibleLast bool, + replaceWithHeartbeat bool, +) (needSwitch, needReParse bool, latestPos int64, nextUUID, nextBinlogName string, currentReplaceFlag bool, err error) { _, suffixInt, err := utils.ParseSuffixForUUID(currentUUID) if err != nil { - return false, false, 0, "", "", err + return false, false, 0, "", "", false, err } uuidSuffix := utils.SuffixIntToStr(suffixInt) // current UUID's suffix, which will be added to binlog name latestPos = offset // set to argument passed in - replaceWithHeartbeat := false onEventFunc := func(e *replication.BinlogEvent) error { r.tctx.L().Debug("read event", zap.Reflect("header", e.Header)) @@ -527,11 +537,11 @@ func (r *BinlogReader) parseFile( // ref: https://github.com/mysql/mysql-server/blob/4f1d7cf5fcb11a3f84cff27e37100d7295e7d5ca/sql/rpl_binlog_sender.cc#L248 e, err2 := utils.GenFakeRotateEvent(relayLogFile, uint64(offset), r.latestServerID) if err2 != nil { - return false, false, 0, "", "", terror.Annotatef(err2, "generate fake RotateEvent for (%s: %d)", relayLogFile, offset) + return false, false, 0, "", "", false, terror.Annotatef(err2, "generate fake RotateEvent for (%s: %d)", relayLogFile, offset) } err2 = onEventFunc(e) if err2 != nil { - return false, false, 0, "", "", terror.Annotatef(err2, "send event %+v", e.Header) + return false, false, 0, "", "", false, terror.Annotatef(err2, "send event %+v", e.Header) } r.tctx.L().Info("start parse relay log file", zap.String("file", fullPath), zap.Int64("offset", offset)) } else { @@ -545,7 +555,7 @@ func (r *BinlogReader) parseFile( r.tctx.L().Warn("fail to parse relay log file, meet some ignorable error", zap.String("file", fullPath), zap.Int64("offset", offset), zap.Error(err)) } else { r.tctx.L().Error("parse relay log file", zap.String("file", fullPath), zap.Int64("offset", offset), zap.Error(err)) - return false, false, 0, "", "", terror.ErrParserParseRelayLog.Delegate(err, fullPath) + return false, false, 0, "", "", false, terror.ErrParserParseRelayLog.Delegate(err, fullPath) } } r.tctx.L().Debug("parse relay log file", zap.String("file", fullPath), zap.Int64("offset", latestPos)) @@ -553,7 +563,7 @@ func (r *BinlogReader) parseFile( if !possibleLast { // there are more relay log files in current sub directory, continue to re-collect them r.tctx.L().Info("more relay log files need to parse", zap.String("directory", relayLogDir)) - return false, false, latestPos, "", "", nil + return false, false, latestPos, "", "", false, nil } switchCh := make(chan SwitchPath, 1) @@ -581,30 +591,30 @@ func (r *BinlogReader) parseFile( select { case <-ctx.Done(): - return false, false, 0, "", "", nil + return false, false, 0, "", "", false, nil case switchResp := <-switchCh: // wait to ensure old file not updated pathUpdated := utils.WaitSomething(3, watcherInterval, func() bool { return len(updatePathCh) > 0 }) if pathUpdated { // re-parse it - return false, true, latestPos, "", "", nil + return false, true, latestPos, "", "", replaceWithHeartbeat, nil } // update new uuid if err = r.updateUUIDs(); err != nil { - return false, false, 0, "", "", nil + return false, false, 0, "", "", false, nil } - return true, false, 0, switchResp.nextUUID, switchResp.nextBinlogName, nil + return true, false, 0, switchResp.nextUUID, switchResp.nextBinlogName, false, nil case updatePath := <-updatePathCh: if strings.HasSuffix(updatePath, relayLogFile) { // current relay log file updated, need to re-parse it - return false, true, latestPos, "", "", nil + return false, true, latestPos, "", "", replaceWithHeartbeat, nil } // need parse next relay log file or re-collect files - return false, false, latestPos, "", "", nil + return false, false, latestPos, "", "", false, nil case err := <-switchErrCh: - return false, false, 0, "", "", err + return false, false, 0, "", "", false, err case err := <-updateErrCh: - return false, false, 0, "", "", err + return false, false, 0, "", "", false, err } } diff --git a/pkg/streamer/reader_test.go b/pkg/streamer/reader_test.go index 8b4ffd417d..5000b60cf9 100644 --- a/pkg/streamer/reader_test.go +++ b/pkg/streamer/reader_test.go @@ -89,14 +89,15 @@ func (t *testReaderSuite) TestParseFileBase(c *C) { relayDir := filepath.Join(baseDir, currentUUID) cfg := &BinlogReaderConfig{RelayDir: baseDir, Flavor: mysql.MySQLFlavor} r := NewBinlogReader(log.L(), cfg) - needSwitch, needReParse, latestPos, nextUUID, nextBinlogName, err := r.parseFile( - ctx, s, filename, offset, relayDir, firstParse, currentUUID, possibleLast) + needSwitch, needReParse, latestPos, nextUUID, nextBinlogName, replaceWithHeartbeat, err := r.parseFile( + ctx, s, filename, offset, relayDir, firstParse, currentUUID, possibleLast, false) c.Assert(err, ErrorMatches, ".*invalid-current-uuid.*") c.Assert(needSwitch, IsFalse) c.Assert(needReParse, IsFalse) c.Assert(latestPos, Equals, int64(0)) c.Assert(nextUUID, Equals, "") c.Assert(nextBinlogName, Equals, "") + c.Assert(replaceWithHeartbeat, Equals, false) // change to valid currentUUID currentUUID = "b60868af-5a6f-11e9-9ea3-0242ac160006.000001" @@ -106,14 +107,15 @@ func (t *testReaderSuite) TestParseFileBase(c *C) { r = NewBinlogReader(log.L(), cfg) // relay log file not exists, failed - needSwitch, needReParse, latestPos, nextUUID, nextBinlogName, err = r.parseFile( - ctx, s, filename, offset, relayDir, firstParse, currentUUID, possibleLast) + needSwitch, needReParse, latestPos, nextUUID, nextBinlogName, replaceWithHeartbeat, err = r.parseFile( + ctx, s, filename, offset, relayDir, firstParse, currentUUID, possibleLast, false) c.Assert(err, ErrorMatches, ".*(no such file or directory|The system cannot find the path specified).*") c.Assert(needSwitch, IsFalse) c.Assert(needReParse, IsFalse) c.Assert(latestPos, Equals, int64(0)) c.Assert(nextUUID, Equals, "") c.Assert(nextBinlogName, Equals, "") + c.Assert(replaceWithHeartbeat, Equals, false) // empty relay log file, failed, got EOF err = os.MkdirAll(relayDir, 0700) @@ -121,14 +123,15 @@ func (t *testReaderSuite) TestParseFileBase(c *C) { f, err := os.OpenFile(fullPath, os.O_CREATE|os.O_WRONLY, 0600) c.Assert(err, IsNil) defer f.Close() - needSwitch, needReParse, latestPos, nextUUID, nextBinlogName, err = r.parseFile( - ctx, s, filename, offset, relayDir, firstParse, currentUUID, possibleLast) + needSwitch, needReParse, latestPos, nextUUID, nextBinlogName, replaceWithHeartbeat, err = r.parseFile( + ctx, s, filename, offset, relayDir, firstParse, currentUUID, possibleLast, false) c.Assert(errors.Cause(err), Equals, io.EOF) c.Assert(needSwitch, IsFalse) c.Assert(needReParse, IsFalse) c.Assert(latestPos, Equals, int64(0)) c.Assert(nextUUID, Equals, "") c.Assert(nextBinlogName, Equals, "") + c.Assert(replaceWithHeartbeat, Equals, false) // write some events to binlog file _, err = f.Write(replication.BinLogFileHeader) @@ -141,14 +144,15 @@ func (t *testReaderSuite) TestParseFileBase(c *C) { t.purgeStreamer(c, s) // base test with only one valid binlog file - needSwitch, needReParse, latestPos, nextUUID, nextBinlogName, err = r.parseFile( - ctx, s, filename, offset, relayDir, firstParse, currentUUID, possibleLast) + needSwitch, needReParse, latestPos, nextUUID, nextBinlogName, replaceWithHeartbeat, err = r.parseFile( + ctx, s, filename, offset, relayDir, firstParse, currentUUID, possibleLast, false) c.Assert(err, IsNil) c.Assert(needSwitch, IsFalse) c.Assert(needReParse, IsFalse) c.Assert(latestPos, Equals, int64(baseEvents[len(baseEvents)-1].Header.LogPos)) c.Assert(nextUUID, Equals, "") c.Assert(nextBinlogName, Equals, "") + c.Assert(replaceWithHeartbeat, Equals, false) // try get events back, firstParse should have fake RotateEvent var fakeRotateEventCount int @@ -173,14 +177,15 @@ func (t *testReaderSuite) TestParseFileBase(c *C) { // try get events back, not firstParse should have no fake RotateEvent firstParse = false - needSwitch, needReParse, latestPos, nextUUID, nextBinlogName, err = r.parseFile( - ctx, s, filename, offset, relayDir, firstParse, currentUUID, possibleLast) + needSwitch, needReParse, latestPos, nextUUID, nextBinlogName, replaceWithHeartbeat, err = r.parseFile( + ctx, s, filename, offset, relayDir, firstParse, currentUUID, possibleLast, false) c.Assert(err, IsNil) c.Assert(needSwitch, IsFalse) c.Assert(needReParse, IsFalse) c.Assert(latestPos, Equals, int64(baseEvents[len(baseEvents)-1].Header.LogPos)) c.Assert(nextUUID, Equals, "") c.Assert(nextBinlogName, Equals, "") + c.Assert(replaceWithHeartbeat, Equals, false) fakeRotateEventCount = 0 i = 0 for { @@ -208,26 +213,28 @@ func (t *testReaderSuite) TestParseFileBase(c *C) { c.Assert(err, IsNil) // latest is still the end_log_pos of the last event, not the next relay file log file's position - needSwitch, needReParse, latestPos, nextUUID, nextBinlogName, err = r.parseFile( - ctx, s, filename, offset, relayDir, firstParse, currentUUID, possibleLast) + needSwitch, needReParse, latestPos, nextUUID, nextBinlogName, replaceWithHeartbeat, err = r.parseFile( + ctx, s, filename, offset, relayDir, firstParse, currentUUID, possibleLast, false) c.Assert(err, IsNil) c.Assert(needSwitch, IsFalse) c.Assert(needReParse, IsFalse) c.Assert(latestPos, Equals, int64(rotateEv.Header.LogPos)) c.Assert(nextUUID, Equals, "") c.Assert(nextBinlogName, Equals, "") + c.Assert(replaceWithHeartbeat, Equals, false) t.purgeStreamer(c, s) // parse from a non-zero offset offset = int64(rotateEv.Header.LogPos - rotateEv.Header.EventSize) - needSwitch, needReParse, latestPos, nextUUID, nextBinlogName, err = r.parseFile( - ctx, s, filename, offset, relayDir, firstParse, currentUUID, possibleLast) + needSwitch, needReParse, latestPos, nextUUID, nextBinlogName, replaceWithHeartbeat, err = r.parseFile( + ctx, s, filename, offset, relayDir, firstParse, currentUUID, possibleLast, false) c.Assert(err, IsNil) c.Assert(needSwitch, IsFalse) c.Assert(needReParse, IsFalse) c.Assert(latestPos, Equals, int64(rotateEv.Header.LogPos)) c.Assert(nextUUID, Equals, "") c.Assert(nextBinlogName, Equals, "") + c.Assert(replaceWithHeartbeat, Equals, false) // should only got a RotateEvent and a FormatDescriptionEven i = 0 @@ -284,14 +291,15 @@ func (t *testReaderSuite) TestParseFileRelaySubDirUpdated(c *C) { // no valid update for relay sub dir, timeout, no error ctx1, cancel1 := context.WithTimeout(context.Background(), 100*time.Millisecond) defer cancel1() - needSwitch, needReParse, latestPos, nextUUID, nextBinlogName, err := r.parseFile( - ctx1, s, filename, offset, relayDir, firstParse, currentUUID, possibleLast) + needSwitch, needReParse, latestPos, nextUUID, nextBinlogName, replaceWithHeartbeat, err := r.parseFile( + ctx1, s, filename, offset, relayDir, firstParse, currentUUID, possibleLast, false) c.Assert(err, IsNil) c.Assert(needSwitch, IsFalse) c.Assert(needReParse, IsFalse) c.Assert(latestPos, Equals, int64(0)) c.Assert(nextUUID, Equals, "") c.Assert(nextBinlogName, Equals, "") + c.Assert(replaceWithHeartbeat, Equals, false) t.purgeStreamer(c, s) // current relay log file updated, need to re-parse it @@ -306,14 +314,15 @@ func (t *testReaderSuite) TestParseFileRelaySubDirUpdated(c *C) { }() ctx2, cancel2 := context.WithTimeout(context.Background(), parseFileTimeout) defer cancel2() - needSwitch, needReParse, latestPos, nextUUID, nextBinlogName, err = r.parseFile( - ctx2, s, filename, offset, relayDir, firstParse, currentUUID, possibleLast) + needSwitch, needReParse, latestPos, nextUUID, nextBinlogName, replaceWithHeartbeat, err = r.parseFile( + ctx2, s, filename, offset, relayDir, firstParse, currentUUID, possibleLast, false) c.Assert(err, IsNil) c.Assert(needSwitch, IsFalse) c.Assert(needReParse, IsTrue) c.Assert(latestPos, Equals, int64(baseEvents[len(baseEvents)-1].Header.LogPos)) c.Assert(nextUUID, Equals, "") c.Assert(nextBinlogName, Equals, "") + c.Assert(replaceWithHeartbeat, Equals, false) wg.Wait() t.purgeStreamer(c, s) @@ -327,14 +336,15 @@ func (t *testReaderSuite) TestParseFileRelaySubDirUpdated(c *C) { }() ctx3, cancel3 := context.WithTimeout(context.Background(), parseFileTimeout) defer cancel3() - needSwitch, needReParse, latestPos, nextUUID, nextBinlogName, err = r.parseFile( - ctx3, s, filename, offset, relayDir, firstParse, currentUUID, possibleLast) + needSwitch, needReParse, latestPos, nextUUID, nextBinlogName, replaceWithHeartbeat, err = r.parseFile( + ctx3, s, filename, offset, relayDir, firstParse, currentUUID, possibleLast, false) c.Assert(err, IsNil) c.Assert(needSwitch, IsFalse) c.Assert(needReParse, IsFalse) c.Assert(latestPos, Equals, int64(extraEvents[0].Header.LogPos)) c.Assert(nextUUID, Equals, "") c.Assert(nextBinlogName, Equals, "") + c.Assert(replaceWithHeartbeat, Equals, false) wg.Wait() t.purgeStreamer(c, s) } @@ -377,14 +387,15 @@ func (t *testReaderSuite) TestParseFileRelayNeedSwitchSubDir(c *C) { t.writeUUIDs(c, baseDir, r.uuids) ctx1, cancel1 := context.WithTimeout(context.Background(), parseFileTimeout) defer cancel1() - needSwitch, needReParse, latestPos, nextUUID, nextBinlogName, err := r.parseFile( - ctx1, s, filename, offset, relayDir, firstParse, currentUUID, possibleLast) + needSwitch, needReParse, latestPos, nextUUID, nextBinlogName, replaceWithHeartbeat, err := r.parseFile( + ctx1, s, filename, offset, relayDir, firstParse, currentUUID, possibleLast, false) c.Assert(err, ErrorMatches, ".*not valid.*") c.Assert(needSwitch, IsFalse) c.Assert(needReParse, IsFalse) c.Assert(latestPos, Equals, int64(0)) c.Assert(nextUUID, Equals, "") c.Assert(nextBinlogName, Equals, "") + c.Assert(replaceWithHeartbeat, Equals, false) t.purgeStreamer(c, s) // next sub dir exits, need to switch @@ -398,14 +409,15 @@ func (t *testReaderSuite) TestParseFileRelayNeedSwitchSubDir(c *C) { // has relay log file in next sub directory, need to switch ctx2, cancel2 := context.WithTimeout(context.Background(), parseFileTimeout) defer cancel2() - needSwitch, needReParse, latestPos, nextUUID, nextBinlogName, err = r.parseFile( - ctx2, s, filename, offset, relayDir, firstParse, currentUUID, possibleLast) + needSwitch, needReParse, latestPos, nextUUID, nextBinlogName, replaceWithHeartbeat, err = r.parseFile( + ctx2, s, filename, offset, relayDir, firstParse, currentUUID, possibleLast, false) c.Assert(err, IsNil) c.Assert(needSwitch, IsTrue) c.Assert(needReParse, IsFalse) c.Assert(latestPos, Equals, int64(0)) c.Assert(nextUUID, Equals, switchedUUID) c.Assert(nextBinlogName, Equals, nextFilename) + c.Assert(replaceWithHeartbeat, Equals, false) t.purgeStreamer(c, s) // NOTE: if we want to test the returned `needReParse` of `needSwitchSubDir`, @@ -438,14 +450,15 @@ func (t *testReaderSuite) TestParseFileRelayWithIgnorableError(c *C) { // file has no data, meet io.EOF error (when reading file header) and ignore it. ctx1, cancel1 := context.WithTimeout(context.Background(), parseFileTimeout) defer cancel1() - needSwitch, needReParse, latestPos, nextUUID, nextBinlogName, err := r.parseFile( - ctx1, s, filename, offset, relayDir, firstParse, currentUUID, possibleLast) + needSwitch, needReParse, latestPos, nextUUID, nextBinlogName, replaceWithHeartbeat, err := r.parseFile( + ctx1, s, filename, offset, relayDir, firstParse, currentUUID, possibleLast, false) c.Assert(err, IsNil) c.Assert(needSwitch, IsFalse) c.Assert(needReParse, IsFalse) c.Assert(latestPos, Equals, int64(0)) c.Assert(nextUUID, Equals, "") c.Assert(nextBinlogName, Equals, "") + c.Assert(replaceWithHeartbeat, Equals, false) _, err = f.Write(replication.BinLogFileHeader) c.Assert(err, IsNil) @@ -459,14 +472,15 @@ func (t *testReaderSuite) TestParseFileRelayWithIgnorableError(c *C) { // meet `err EOF` error (when parsing binlog event) ignored ctx2, cancel2 := context.WithTimeout(context.Background(), parseFileTimeout) defer cancel2() - needSwitch, needReParse, latestPos, nextUUID, nextBinlogName, err = r.parseFile( - ctx2, s, filename, offset, relayDir, firstParse, currentUUID, possibleLast) + needSwitch, needReParse, latestPos, nextUUID, nextBinlogName, replaceWithHeartbeat, err = r.parseFile( + ctx2, s, filename, offset, relayDir, firstParse, currentUUID, possibleLast, false) c.Assert(err, IsNil) c.Assert(needSwitch, IsFalse) c.Assert(needReParse, IsTrue) c.Assert(latestPos, Equals, int64(baseEvents[len(baseEvents)-1].Header.LogPos)) c.Assert(nextUUID, Equals, "") c.Assert(nextBinlogName, Equals, "") + c.Assert(replaceWithHeartbeat, Equals, false) } func (t *testReaderSuite) TestUpdateUUIDs(c *C) { @@ -1026,6 +1040,96 @@ func (t *testReaderSuite) TestAdvanceCurrentGTIDSet(c *C) { c.Assert(r.currGset.String(), Equals, "0-1-6") } +func (t *testReaderSuite) TestReParseUsingGTID(c *C) { + var ( + baseDir = c.MkDir() + cfg = &BinlogReaderConfig{RelayDir: baseDir, Flavor: mysql.MySQLFlavor} + r = NewBinlogReader(log.L(), cfg) + uuid = "ba8f633f-1f15-11eb-b1c7-0242ac110002.000001" + gtidStr = "ba8f633f-1f15-11eb-b1c7-0242ac110002:1" + file = "mysql.000001" + latestPos uint32 + ) + + startGTID, err := gtid.ParserGTID(mysql.MySQLFlavor, "") + c.Assert(err, IsNil) + lastGTID, err := gtid.ParserGTID(mysql.MySQLFlavor, gtidStr) + c.Assert(err, IsNil) + + // prepare a minimal relay log file + c.Assert(ioutil.WriteFile(r.indexPath, []byte(uuid), 0600), IsNil) + + uuidDir := path.Join(baseDir, uuid) + c.Assert(os.MkdirAll(uuidDir, 0700), IsNil) + f, err := os.OpenFile(path.Join(uuidDir, file), os.O_CREATE|os.O_WRONLY, 0600) + c.Assert(err, IsNil) + _, err = f.Write(replication.BinLogFileHeader) + c.Assert(err, IsNil) + + meta := Meta{BinLogName: file, BinLogPos: latestPos, BinlogGTID: startGTID.String()} + metaFile, err := os.Create(path.Join(uuidDir, utils.MetaFilename)) + c.Assert(err, IsNil) + c.Assert(toml.NewEncoder(metaFile).Encode(meta), IsNil) + c.Assert(metaFile.Close(), IsNil) + + // prepare some regular events, + // FORMAT_DESC + PREVIOUS_GTIDS, some events generated from a DDL, some events generated from a DML + genType := []replication.EventType{ + replication.PREVIOUS_GTIDS_EVENT, + replication.QUERY_EVENT, + replication.XID_EVENT} + events, _, _, latestGTIDSet := t.genEvents(c, genType, 4, lastGTID, startGTID) + c.Assert(events, HasLen, 1+1+2+5) + + // write FORMAT_DESC + PREVIOUS_GTIDS + _, err = f.Write(events[0].RawData) + c.Assert(err, IsNil) + _, err = f.Write(events[1].RawData) + c.Assert(err, IsNil) + + // we use latestGTIDSet to start sync, which means we already received all binlog events, so expect no DML/DDL + s, err := r.StartSyncByGTID(latestGTIDSet.Origin()) + c.Assert(err, IsNil) + var wg sync.WaitGroup + wg.Add(1) + + go func() { + expected := map[uint32]replication.EventType{} + for _, e := range events { + switch e.Event.(type) { + // keeps same + case *replication.FormatDescriptionEvent, *replication.PreviousGTIDsEvent: + expected[e.Header.LogPos] = e.Header.EventType + default: + expected[e.Header.LogPos] = replication.HEARTBEAT_EVENT + } + } + // fake rotate + expected[0] = replication.ROTATE_EVENT + lastLogPos := events[len(events)-1].Header.LogPos + + ctx, cancel := context.WithCancel(context.Background()) + for { + ev, err2 := s.GetEvent(ctx) + c.Assert(err2, IsNil) + c.Assert(ev.Header.EventType, Equals, expected[ev.Header.LogPos]) + if ev.Header.LogPos == lastLogPos { + break + } + } + cancel() + wg.Done() + }() + + for i := 2; i < len(events); i++ { + // hope a second is enough to trigger needReParse + time.Sleep(time.Second) + _, err = f.Write(events[i].RawData) + c.Assert(err, IsNil) + } + wg.Wait() +} + func (t *testReaderSuite) genBinlogEvents(c *C, latestPos uint32, latestGTID gtid.Set) ([]*replication.BinlogEvent, uint32, gtid.Set) { var ( header = &replication.EventHeader{ diff --git a/pkg/terror/error_list.go b/pkg/terror/error_list.go index 7e1b0a5122..57f3f15797 100644 --- a/pkg/terror/error_list.go +++ b/pkg/terror/error_list.go @@ -411,6 +411,7 @@ const ( codeSyncerOperatorNotExist codeSyncerReplaceEventNotExist codeSyncerParseDDL + codeSyncerUnsupportedStmt ) // DM-master error code @@ -469,6 +470,7 @@ const ( codeMasterBoundChanging codeMasterFailToImportFromV10x codeMasterInconsistentOptimistDDLsAndInfo + codeMasterOptimisticTableInfobeforeNotExist ) // DM-worker error code @@ -992,12 +994,13 @@ var ( ErrSyncerUnitExecWithNoBlockingDDL = New(codeSyncerUnitExecWithNoBlockingDDL, ClassSyncUnit, ScopeInternal, LevelHigh, "process unit not waiting for sharding DDL to sync", "") ErrSyncerUnitGenBAList = New(codeSyncerUnitGenBAList, ClassSyncUnit, ScopeInternal, LevelHigh, "generate block allow list", "Please check the `block-allow-list` config in task configuration file.") ErrSyncerUnitHandleDDLFailed = New(codeSyncerUnitHandleDDLFailed, ClassSyncUnit, ScopeInternal, LevelHigh, "fail to handle ddl job for %s", "") - ErrSyncerShardDDLConflict = New(codeSyncerShardDDLConflict, ClassSyncUnit, ScopeInternal, LevelHigh, "fail to handle shard ddl %v in optimistic mode, because schema conflict detected", "Please use show-ddl-locks command for more details.") + ErrSyncerShardDDLConflict = New(codeSyncerShardDDLConflict, ClassSyncUnit, ScopeInternal, LevelHigh, "fail to handle shard ddl %v in optimistic mode, because schema conflict detected, conflict error: %s", "Please use show-ddl-locks command for more details.") ErrSyncerFailpoint = New(codeSyncerFailpoint, ClassSyncUnit, ScopeInternal, LevelLow, "failpoint specified error", "") ErrSyncerReplaceEvent = New(codeSyncerReplaceEvent, ClassSyncUnit, ScopeInternal, LevelHigh, "", "") ErrSyncerOperatorNotExist = New(codeSyncerOperatorNotExist, ClassSyncUnit, ScopeInternal, LevelLow, "error operator not exist, position: %s", "") ErrSyncerReplaceEventNotExist = New(codeSyncerReplaceEventNotExist, ClassSyncUnit, ScopeInternal, LevelHigh, "replace event not exist, location: %s", "") ErrSyncerParseDDL = New(codeSyncerParseDDL, ClassSyncUnit, ScopeInternal, LevelHigh, "parse DDL: %s", "Please confirm your DDL statement is correct and needed. For TiDB compatible DDL, see https://docs.pingcap.com/tidb/stable/mysql-compatibility#ddl. You can use `handle-error` command to skip or replace the DDL or add a binlog filter rule to ignore it if the DDL is not needed.") + ErrSyncerUnsupportedStmt = New(codeSyncerUnsupportedStmt, ClassSyncUnit, ScopeInternal, LevelHigh, "`%s` statement not supported in %s mode", "") // DM-master error ErrMasterSQLOpNilRequest = New(codeMasterSQLOpNilRequest, ClassDMMaster, ScopeInternal, LevelMedium, "nil request not valid", "") @@ -1059,6 +1062,7 @@ var ( ErrMasterFailToImportFromV10x = New(codeMasterFailToImportFromV10x, ClassDMMaster, ScopeInternal, LevelHigh, "fail to import DM cluster from v1.0.x", "Please confirm that you have not violated any restrictions in the upgrade documentation.") 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", "") // DM-worker error ErrWorkerParseFlagSet = New(codeWorkerParseFlagSet, ClassDMWorker, ScopeInternal, LevelMedium, "parse dm-worker config flag set", "") diff --git a/pkg/utils/db_test.go b/pkg/utils/db_test.go index b68c0d9771..8c9c02370f 100644 --- a/pkg/utils/db_test.go +++ b/pkg/utils/db_test.go @@ -50,7 +50,7 @@ func (t *testDBSuite) TestGetFlavor(c *C) { c.Assert(mock.ExpectationsWereMet(), IsNil) // others - mock.ExpectQuery(`SHOW GLOBAL VARIABLES LIKE 'version';`).WillReturnRows(sqlmock.NewRows([]string{"Variable_name", "Value"}).AddRow("version", "unkown")) + mock.ExpectQuery(`SHOW GLOBAL VARIABLES LIKE 'version';`).WillReturnRows(sqlmock.NewRows([]string{"Variable_name", "Value"}).AddRow("version", "unknown")) flavor, err = GetFlavor(context.Background(), db) c.Assert(err, IsNil) c.Assert(flavor, Equals, "mysql") // as MySQL diff --git a/syncer/err-operator/operator.go b/syncer/err-operator/operator.go index 0e2d89b4ac..33aa1bafc8 100644 --- a/syncer/err-operator/operator.go +++ b/syncer/err-operator/operator.go @@ -179,7 +179,7 @@ func (h *Holder) RemoveOutdated(flushLocation binlog.Location) error { return err } if binlog.ComparePosition(position, flushLocation.Position) == -1 { - h.logger.Info("remove a outdated operator", zap.Stringer("position", position), zap.Stringer("flush postion", flushLocation.Position), zap.Stringer("operator", h.operators[pos])) + h.logger.Info("remove a outdated operator", zap.Stringer("position", position), zap.Stringer("flush position", flushLocation.Position), zap.Stringer("operator", h.operators[pos])) delete(h.operators, pos) } } diff --git a/syncer/optimist.go b/syncer/optimist.go index 5566036c19..955442104d 100644 --- a/syncer/optimist.go +++ b/syncer/optimist.go @@ -177,7 +177,7 @@ func (s *Syncer) handleQueryEventOptimistic( } if op.ConflictStage == optimism.ConflictDetected { - return terror.ErrSyncerShardDDLConflict.Generate(needHandleDDLs) + return terror.ErrSyncerShardDDLConflict.Generate(needHandleDDLs, op.ConflictMsg) } // updated needHandleDDLs to DDLs received from DM-master. diff --git a/syncer/schema.go b/syncer/schema.go index 6d0d3526fd..3cc8a5b88e 100644 --- a/syncer/schema.go +++ b/syncer/schema.go @@ -102,7 +102,8 @@ func (s *Syncer) OperateSchema(ctx context.Context, req *pb.OperateWorkerSchemaR break } downSchema, downTable := s.renameShardingSchema(req.Database, req.Table) - info := s.optimist.ConstructInfo(req.Database, req.Table, downSchema, downTable, []string{""}, nil, []*model.TableInfo{ti}) + // use new table info as tableInfoBefore, we can also use the origin table from schemaTracker + info := s.optimist.ConstructInfo(req.Database, req.Table, downSchema, downTable, []string{""}, ti, []*model.TableInfo{ti}) info.IgnoreConflict = true log.L().Info("sync info with operate-schema", zap.Stringer("info", info)) _, err = s.optimist.PutInfo(info) diff --git a/syncer/shardddl/optimist.go b/syncer/shardddl/optimist.go index afb7bd9a56..9a8bba0f39 100644 --- a/syncer/shardddl/optimist.go +++ b/syncer/shardddl/optimist.go @@ -148,7 +148,7 @@ func (o *Optimist) GetOperation(ctx context.Context, info optimism.Info, rev int // DoneOperation marks the shard DDL lock operation as done. func (o *Optimist) DoneOperation(op optimism.Operation) error { op.Done = true - _, _, err := optimism.PutOperation(o.cli, false, op) + _, _, err := optimism.PutOperation(o.cli, false, op, 0) if err != nil { return err } diff --git a/syncer/shardddl/optimist_test.go b/syncer/shardddl/optimist_test.go index 689f2d884f..e727e2b914 100644 --- a/syncer/shardddl/optimist_test.go +++ b/syncer/shardddl/optimist_test.go @@ -27,6 +27,7 @@ import ( "github.com/pingcap/dm/pkg/log" "github.com/pingcap/dm/pkg/shardddl/optimism" + "github.com/pingcap/dm/pkg/terror" ) type testOptimist struct { @@ -82,9 +83,9 @@ func (t *testOptimist) TestOptimist(c *C) { tiAfter1 = createTableInfo(c, p, se, tblID, `CREATE TABLE bar (id INT PRIMARY KEY, c1 TEXT)`) tiAfter2 = createTableInfo(c, p, se, tblID, `CREATE TABLE bar (id INT PRIMARY KEY, c1 DATETIME)`) info1 = o.ConstructInfo("foo-1", "bar-1", downSchema, downTable, DDLs1, tiBefore, []*model.TableInfo{tiAfter1}) - op1 = optimism.NewOperation(ID, task, source, info1.UpSchema, info1.UpTable, DDLs1, optimism.ConflictNone, false) + op1 = optimism.NewOperation(ID, task, source, info1.UpSchema, info1.UpTable, DDLs1, optimism.ConflictNone, "", false) info2 = o.ConstructInfo("foo-1", "bar-2", downSchema, downTable, DDLs2, tiBefore, []*model.TableInfo{tiAfter2}) - op2 = optimism.NewOperation(ID, task, source, info2.UpSchema, info2.UpTable, DDLs2, optimism.ConflictDetected, false) + op2 = optimism.NewOperation(ID, task, source, info2.UpSchema, info2.UpTable, DDLs2, optimism.ConflictDetected, terror.ErrShardDDLOptimismTrySyncFail.Generate(ID, "conflict").Error(), false) infoCreate = o.ConstructInfo("foo-new", "bar-new", downSchema, downTable, []string{`CREATE TABLE bar (id INT PRIMARY KEY)`}, tiBefore, []*model.TableInfo{tiBefore}) // same table info. @@ -119,7 +120,7 @@ func (t *testOptimist) TestOptimist(c *C) { c.Assert(*info1c, DeepEquals, info1) // put the lock operation. - rev2, putted, err := optimism.PutOperation(etcdTestCli, false, op1) + rev2, putted, err := optimism.PutOperation(etcdTestCli, false, op1, rev1) c.Assert(err, IsNil) c.Assert(rev2, Greater, rev1) c.Assert(putted, IsTrue) @@ -146,6 +147,7 @@ func (t *testOptimist) TestOptimist(c *C) { c.Assert(ifm[task][source][info1.UpSchema], HasLen, 1) info1WithVer := info1 info1WithVer.Version = 1 + info1WithVer.Revision = rev1 c.Assert(ifm[task][source][info1.UpSchema][info1.UpTable], DeepEquals, info1WithVer) opc := op1c opc.Done = true @@ -169,6 +171,7 @@ func (t *testOptimist) TestOptimist(c *C) { c.Assert(err, IsNil) infoCreateWithVer := infoCreate infoCreateWithVer.Version = 1 + infoCreateWithVer.Revision = rev3 c.Assert(ifm[task][source][infoCreate.UpSchema][infoCreate.UpTable], DeepEquals, infoCreateWithVer) c.Assert(o.tables.Tables[infoCreate.DownSchema][infoCreate.DownTable][infoCreate.UpSchema], HasKey, infoCreate.UpTable) @@ -189,7 +192,7 @@ func (t *testOptimist) TestOptimist(c *C) { c.Assert(o.PendingOperation(), IsNil) // put another lock operation. - rev6, putted, err := optimism.PutOperation(etcdTestCli, false, op2) + rev6, putted, err := optimism.PutOperation(etcdTestCli, false, op2, rev5) c.Assert(err, IsNil) c.Assert(rev6, Greater, rev5) c.Assert(putted, IsTrue) diff --git a/syncer/syncer.go b/syncer/syncer.go index 35ab156bde..f2be039714 100644 --- a/syncer/syncer.go +++ b/syncer/syncer.go @@ -863,8 +863,15 @@ func (s *Syncer) resetShardingGroup(schema, table string) { // // we may need to refactor the concurrency model to make the work-flow more clearer later func (s *Syncer) flushCheckPoints() error { - if s.execError.Get() != nil { - s.tctx.L().Warn("error detected when executing SQL job, skip flush checkpoint", zap.Stringer("checkpoint", s.checkpoint)) + err := s.execError.Get() + // TODO: for now, if any error occurred (including user canceled), checkpoint won't be updated. But if we have put + // optimistic shard info, DM-master may resolved the optimistic lock and let other worker execute DDL. So after this + // worker resume, it can not execute the DML/DDL in old binlog because of downstream table structure mismatching. + // We should find a way to (compensating) implement a transaction containing interaction with both etcd and SQL. + if err != nil { + s.tctx.L().Warn("error detected when executing SQL job, skip flush checkpoint", + zap.Stringer("checkpoint", s.checkpoint), + zap.Error(err)) return nil } @@ -885,7 +892,7 @@ func (s *Syncer) flushCheckPoints() error { s.tctx.L().Info("prepare flush sqls", zap.Strings("shard meta sqls", shardMetaSQLs), zap.Reflect("shard meta arguments", shardMetaArgs)) } - err := s.checkpoint.FlushPointsExcept(s.tctx, exceptTables, shardMetaSQLs, shardMetaArgs) + err = s.checkpoint.FlushPointsExcept(s.tctx, exceptTables, shardMetaSQLs, shardMetaArgs) if err != nil { return terror.Annotatef(err, "flush checkpoint %s", s.checkpoint) } @@ -1856,6 +1863,8 @@ func (s *Syncer) handleQueryEvent(ev *replication.QueryEvent, ec eventContext, o case *ast.TruncateTableStmt: ec.tctx.L().Info("ignore truncate table statement in shard group", zap.String("event", "query"), zap.String("statement", sqlDDL)) continue + case *ast.RenameTableStmt: + return terror.ErrSyncerUnsupportedStmt.Generate("RENAME TABLE", config.ShardOptimistic) } } diff --git a/tests/README.md b/tests/README.md index 0adce0d518..b0d1bc503f 100644 --- a/tests/README.md +++ b/tests/README.md @@ -13,6 +13,8 @@ * `bin/dm-worker.test.previous` # generated from `bin/dm-worker.test` by `make compatibility_test` * [gh-ost](https://github.com/github/gh-ost) # must be added to path, or you can `export GHOST_BINARY=/path/to/gh-ost-binary` * [pt-online-schema-change](https://www.percona.com/doc/percona-toolkit/LATEST/pt-online-schema-change.html) # must be added to path, or you can `export PTOSC_BINARY=/path/to/pt-osc-binary` + * [GNU sed](https://www.gnu.org/software/sed/) # must be added to path, BSD sed user should be attention + * [GNU awk](https://www.gnu.org/software/gawk/) # must be added to path 2. The following programs must be installed: diff --git a/tests/_utils/env_variables b/tests/_utils/env_variables index 6ff200db53..efcef55b25 100755 --- a/tests/_utils/env_variables +++ b/tests/_utils/env_variables @@ -6,7 +6,7 @@ MYSQL_PASSWORD1=${MYSQL_PASSWORD1:-123456} MYSQL_PASSWORD2=${MYSQL_PASSWORD2:-123456} TIDB_PASSWORD=${TIDB_PASSWORD:-123456} -TIDB_PORT=4000 +TIDB_PORT=${TIDB_PORT:-4000} MASTER_PORT=8261 MASTER_PORT1=8261 MASTER_PORT2=8361 @@ -21,4 +21,6 @@ WORKER4_PORT=18262 WORKER5_PORT=18263 SOURCE_ID1="mysql-replica-01" SOURCE_ID2="mysql-replica-02" -RESET_MASTER=${RESET_MASTER:-true} \ No newline at end of file +RESET_MASTER=${RESET_MASTER:-true} + +VERBOSE=${VERBOSE:-false} diff --git a/tests/_utils/test_prepare b/tests/_utils/test_prepare index 579a04d9ec..34a9163e94 100644 --- a/tests/_utils/test_prepare +++ b/tests/_utils/test_prepare @@ -136,7 +136,12 @@ function run_case() { "\"unit\": \"Sync\"" 2 fi - DM_${case}_CASE $5 + args="" + for((i=5;i<=$#;i++)); do + j=${!i} + args="${args} $j " + done + DM_${case}_CASE $args run_dm_ctl $WORK_DIR "127.0.0.1:$MASTER_PORT" \ "stop-task test" diff --git a/tests/all_mode/conf/dm-master.toml b/tests/all_mode/conf/dm-master.toml index 9b360834d3..53a294e7d0 100644 --- a/tests/all_mode/conf/dm-master.toml +++ b/tests/all_mode/conf/dm-master.toml @@ -3,3 +3,4 @@ master-addr = ":8261" advertise-addr = "127.0.0.1:8261" rpc-timeout = "30s" +auto-compaction-retention = "3s" diff --git a/tests/dm_syncer/conf/dm-master.toml b/tests/dm_syncer/conf/dm-master.toml index bdeb27b605..7cecf59ad8 100644 --- a/tests/dm_syncer/conf/dm-master.toml +++ b/tests/dm_syncer/conf/dm-master.toml @@ -1,4 +1,4 @@ # Master Configuration. master-addr = ":8261" advertise-addr = "127.0.0.1:8261" - +auto-compaction-retention = "3s" diff --git a/tests/dmctl_advance/conf/dm-master.toml b/tests/dmctl_advance/conf/dm-master.toml index 14ac020290..c014ffb07c 100644 --- a/tests/dmctl_advance/conf/dm-master.toml +++ b/tests/dmctl_advance/conf/dm-master.toml @@ -2,3 +2,4 @@ master-addr = ":8261" advertise-addr = "127.0.0.1:8261" +auto-compaction-retention = "3s" diff --git a/tests/dmctl_basic/conf/get_master1.toml b/tests/dmctl_basic/conf/get_master1.toml index bc1589d944..8df893c4d1 100644 --- a/tests/dmctl_basic/conf/get_master1.toml +++ b/tests/dmctl_basic/conf/get_master1.toml @@ -15,6 +15,9 @@ advertise-peer-urls = "http://127.0.0.1:8291" initial-cluster = "master1=http://127.0.0.1:8291" initial-cluster-state = "new" join = "" +auto-compaction-mode = "periodic" +auto-compaction-retention = "1h" +quota-backend-bytes = 2147483648 v1-sources-path = "" ssl-ca = "" ssl-cert = "" diff --git a/tests/dmctl_command/conf/dm-master.toml b/tests/dmctl_command/conf/dm-master.toml index 9a36bcbc84..7cecf59ad8 100644 --- a/tests/dmctl_command/conf/dm-master.toml +++ b/tests/dmctl_command/conf/dm-master.toml @@ -1,3 +1,4 @@ # Master Configuration. master-addr = ":8261" advertise-addr = "127.0.0.1:8261" +auto-compaction-retention = "3s" diff --git a/tests/drop_column_with_index/conf/dm-master.toml b/tests/drop_column_with_index/conf/dm-master.toml index 9a36bcbc84..7cecf59ad8 100644 --- a/tests/drop_column_with_index/conf/dm-master.toml +++ b/tests/drop_column_with_index/conf/dm-master.toml @@ -1,3 +1,4 @@ # Master Configuration. master-addr = ":8261" advertise-addr = "127.0.0.1:8261" +auto-compaction-retention = "3s" diff --git a/tests/full_mode/conf/dm-master.toml b/tests/full_mode/conf/dm-master.toml index 9a36bcbc84..7cecf59ad8 100644 --- a/tests/full_mode/conf/dm-master.toml +++ b/tests/full_mode/conf/dm-master.toml @@ -1,3 +1,4 @@ # Master Configuration. master-addr = ":8261" advertise-addr = "127.0.0.1:8261" +auto-compaction-retention = "3s" diff --git a/tests/gtid/conf/dm-master.toml b/tests/gtid/conf/dm-master.toml index 9b360834d3..53a294e7d0 100644 --- a/tests/gtid/conf/dm-master.toml +++ b/tests/gtid/conf/dm-master.toml @@ -3,3 +3,4 @@ master-addr = ":8261" advertise-addr = "127.0.0.1:8261" rpc-timeout = "30s" +auto-compaction-retention = "3s" diff --git a/tests/ha/conf/dm-master1.toml b/tests/ha/conf/dm-master1.toml index 13eb92f3cc..78fd5845ae 100644 --- a/tests/ha/conf/dm-master1.toml +++ b/tests/ha/conf/dm-master1.toml @@ -4,3 +4,4 @@ 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,master2=http://127.0.0.1:8292" +auto-compaction-retention = "3s" diff --git a/tests/ha/conf/dm-master2.toml b/tests/ha/conf/dm-master2.toml index 587092d387..41344e9049 100644 --- a/tests/ha/conf/dm-master2.toml +++ b/tests/ha/conf/dm-master2.toml @@ -3,4 +3,5 @@ name = "master2" master-addr = ":8361" advertise-addr = "127.0.0.1:8361" peer-urls = "127.0.0.1:8292" -initial-cluster = "master1=http://127.0.0.1:8291,master2=http://127.0.0.1:8292" \ No newline at end of file +initial-cluster = "master1=http://127.0.0.1:8291,master2=http://127.0.0.1:8292" +auto-compaction-retention = "3s" diff --git a/tests/ha/conf/dm-master3.toml b/tests/ha/conf/dm-master3.toml index 68132a9e7b..0e2d24d983 100644 --- a/tests/ha/conf/dm-master3.toml +++ b/tests/ha/conf/dm-master3.toml @@ -3,4 +3,5 @@ name = "master3" master-addr = ":8461" advertise-addr = "127.0.0.1:8461" peer-urls = "http://127.0.0.1:8293" -join = "127.0.0.1:8261,127.0.0.1:8361" \ No newline at end of file +join = "127.0.0.1:8261,127.0.0.1:8361" +auto-compaction-retention = "3s" diff --git a/tests/ha/conf/dm-master4.toml b/tests/ha/conf/dm-master4.toml index 00ea67f196..44fc0689b5 100644 --- a/tests/ha/conf/dm-master4.toml +++ b/tests/ha/conf/dm-master4.toml @@ -4,3 +4,4 @@ master-addr = ":8561" advertise-addr = "127.0.0.1:8561" peer-urls = "http://127.0.0.1:8294" join = "127.0.0.1:8261,127.0.0.1:8361,127.0.0.1:8461" +auto-compaction-retention = "3s" diff --git a/tests/ha/conf/dm-master5.toml b/tests/ha/conf/dm-master5.toml index 9c06c40544..15e4f3c96e 100644 --- a/tests/ha/conf/dm-master5.toml +++ b/tests/ha/conf/dm-master5.toml @@ -4,3 +4,4 @@ master-addr = ":8661" advertise-addr = "127.0.0.1:8661" peer-urls = "http://127.0.0.1:8295" join = "127.0.0.1:8261,127.0.0.1:8361,127.0.0.1:8461,127.0.0.1:8561" +auto-compaction-retention = "3s" diff --git a/tests/ha/conf/dm-master6.toml b/tests/ha/conf/dm-master6.toml index 4fb0059c43..e8494af359 100644 --- a/tests/ha/conf/dm-master6.toml +++ b/tests/ha/conf/dm-master6.toml @@ -4,3 +4,4 @@ master-addr = ":8761" advertise-addr = "127.0.0.1:8761" peer-urls = "http://127.0.0.1:8296" join = "127.0.0.1:8261,127.0.0.1:8361,127.0.0.1:8461,127.0.0.1:8561,127.0.0.1:8661" +auto-compaction-retention = "3s" diff --git a/tests/ha_cases/conf/dm-master-join1.toml b/tests/ha_cases/conf/dm-master-join1.toml index aa24d7d7d3..e700eeffa5 100644 --- a/tests/ha_cases/conf/dm-master-join1.toml +++ b/tests/ha_cases/conf/dm-master-join1.toml @@ -4,3 +4,4 @@ master-addr = ":8261" advertise-addr = "127.0.0.1:8261" peer-urls = "127.0.0.1:8291" initial-cluster = "master1=http://127.0.0.1:8291" +auto-compaction-retention = "3s" diff --git a/tests/ha_cases/conf/dm-master-join2.toml b/tests/ha_cases/conf/dm-master-join2.toml index c0fef373e5..ffc2aa6563 100644 --- a/tests/ha_cases/conf/dm-master-join2.toml +++ b/tests/ha_cases/conf/dm-master-join2.toml @@ -3,4 +3,5 @@ name = "master2" master-addr = ":8361" advertise-addr = "127.0.0.1:8361" peer-urls = "http://127.0.0.1:8292" -join = "127.0.0.1:8261" \ No newline at end of file +join = "127.0.0.1:8261" +auto-compaction-retention = "3s" diff --git a/tests/ha_cases/conf/dm-master-join3.toml b/tests/ha_cases/conf/dm-master-join3.toml index d03e40f8c8..d187440f25 100644 --- a/tests/ha_cases/conf/dm-master-join3.toml +++ b/tests/ha_cases/conf/dm-master-join3.toml @@ -3,4 +3,5 @@ name = "master3" master-addr = ":8461" advertise-addr = "127.0.0.1:8461" peer-urls = "http://127.0.0.1:8293" -join = "127.0.0.1:8261,127.0.0.1:8361" \ No newline at end of file +join = "127.0.0.1:8261,127.0.0.1:8361" +auto-compaction-retention = "3s" diff --git a/tests/ha_cases/conf/dm-master-join4.toml b/tests/ha_cases/conf/dm-master-join4.toml index d458a9f2b2..1067b5541a 100644 --- a/tests/ha_cases/conf/dm-master-join4.toml +++ b/tests/ha_cases/conf/dm-master-join4.toml @@ -3,4 +3,5 @@ name = "master4" master-addr = ":8561" advertise-addr = "127.0.0.1:8561" peer-urls = "http://127.0.0.1:8294" -join = "127.0.0.1:8261,127.0.0.1:8361,127.0.0.1:8461" \ No newline at end of file +join = "127.0.0.1:8261,127.0.0.1:8361,127.0.0.1:8461" +auto-compaction-retention = "3s" diff --git a/tests/ha_cases/conf/dm-master-join5.toml b/tests/ha_cases/conf/dm-master-join5.toml index 82c09289a9..fbe749f31d 100644 --- a/tests/ha_cases/conf/dm-master-join5.toml +++ b/tests/ha_cases/conf/dm-master-join5.toml @@ -3,4 +3,5 @@ name = "master5" master-addr = ":8661" advertise-addr = "127.0.0.1:8661" peer-urls = "http://127.0.0.1:8295" -join = "127.0.0.1:8261,127.0.0.1:8361,127.0.0.1:8461,127.0.0.1:8561" \ No newline at end of file +join = "127.0.0.1:8261,127.0.0.1:8361,127.0.0.1:8461,127.0.0.1:8561" +auto-compaction-retention = "3s" diff --git a/tests/ha_cases/conf/dm-master1.toml b/tests/ha_cases/conf/dm-master1.toml index 491b5487df..3c6aa40c1f 100644 --- a/tests/ha_cases/conf/dm-master1.toml +++ b/tests/ha_cases/conf/dm-master1.toml @@ -4,3 +4,4 @@ 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,master2=http://127.0.0.1:8292,master3=http://127.0.0.1:8293" +auto-compaction-retention = "3s" diff --git a/tests/ha_cases/conf/dm-master2.toml b/tests/ha_cases/conf/dm-master2.toml index 47c241dbc6..4edd2de4bb 100644 --- a/tests/ha_cases/conf/dm-master2.toml +++ b/tests/ha_cases/conf/dm-master2.toml @@ -3,4 +3,5 @@ name = "master2" master-addr = ":8361" advertise-addr = "127.0.0.1:8361" peer-urls = "127.0.0.1:8292" -initial-cluster = "master1=http://127.0.0.1:8291,master2=http://127.0.0.1:8292,master3=http://127.0.0.1:8293" \ No newline at end of file +initial-cluster = "master1=http://127.0.0.1:8291,master2=http://127.0.0.1:8292,master3=http://127.0.0.1:8293" +auto-compaction-retention = "3s" diff --git a/tests/ha_cases/conf/dm-master3.toml b/tests/ha_cases/conf/dm-master3.toml index 47caf87fed..40972db391 100644 --- a/tests/ha_cases/conf/dm-master3.toml +++ b/tests/ha_cases/conf/dm-master3.toml @@ -3,4 +3,5 @@ name = "master3" master-addr = ":8461" advertise-addr = "127.0.0.1:8461" peer-urls = "127.0.0.1:8293" -initial-cluster = "master1=http://127.0.0.1:8291,master2=http://127.0.0.1:8292,master3=http://127.0.0.1:8293" \ No newline at end of file +initial-cluster = "master1=http://127.0.0.1:8291,master2=http://127.0.0.1:8292,master3=http://127.0.0.1:8293" +auto-compaction-retention = "3s" diff --git a/tests/ha_cases/run.sh b/tests/ha_cases/run.sh index bab90f49dc..e23dabd7d1 100755 --- a/tests/ha_cases/run.sh +++ b/tests/ha_cases/run.sh @@ -727,7 +727,8 @@ function test_last_bound() { run_sql_file_withdb $cur/data/db1.increment2.sql $MYSQL_HOST1 $MYSQL_PORT1 $MYSQL_PASSWORD1 $ha_test run_sql "flush logs;" $MYSQL_PORT2 $MYSQL_PASSWORD2 run_sql_file_withdb $cur/data/db2.increment2.sql $MYSQL_HOST2 $MYSQL_PORT2 $MYSQL_PASSWORD2 $ha_test - sleep 1 + # wait the checkpoint updated + check_sync_diff $WORK_DIR $cur/conf/diff_config.toml kill_2_worker_ensure_unbound 3 4 # start 1 then 2 @@ -738,7 +739,7 @@ function test_last_bound() { # other workers has forwarded the sync progress, if moved to a new binlog file, original relay log could be removed num1=`grep "will try purge whole relay dir for new relay log" $WORK_DIR/worker1/log/dm-worker.log | wc -l` num2=`grep "will try purge whole relay dir for new relay log" $WORK_DIR/worker2/log/dm-worker.log | wc -l` - echo "num1$num1 num2$num2" + echo "num1 $num1 num2 $num2" [[ $num1+$num2 -eq 3 ]] echo "[$(date)] <<<<<< finish test_last_bound >>>>>>" diff --git a/tests/ha_master/conf/dm-master-join1.toml b/tests/ha_master/conf/dm-master-join1.toml index 986ccff55a..2e19311c12 100644 --- a/tests/ha_master/conf/dm-master-join1.toml +++ b/tests/ha_master/conf/dm-master-join1.toml @@ -4,3 +4,4 @@ master-addr = ":8261" advertise-addr = "localhost:8261" peer-urls = "http://localhost:8291" join = "localhost:8461,localhost:8561,localhost:8661" +auto-compaction-retention = "3s" diff --git a/tests/ha_master/conf/dm-master1.toml b/tests/ha_master/conf/dm-master1.toml index e7277abe51..b5554c2d3b 100644 --- a/tests/ha_master/conf/dm-master1.toml +++ b/tests/ha_master/conf/dm-master1.toml @@ -3,4 +3,5 @@ name = "master1" master-addr = ":8261" advertise-addr = "localhost:8261" peer-urls = "localhost:8291" -initial-cluster = "master1=http://localhost:8291,master2=http://localhost:8292,master3=http://localhost:8293,master4=http://localhost:8294,master5=http://localhost:8295" \ No newline at end of file +initial-cluster = "master1=http://localhost:8291,master2=http://localhost:8292,master3=http://localhost:8293,master4=http://localhost:8294,master5=http://localhost:8295" +auto-compaction-retention = "3s" diff --git a/tests/ha_master/conf/dm-master2.toml b/tests/ha_master/conf/dm-master2.toml index 1c48cfb521..bae72e78c7 100644 --- a/tests/ha_master/conf/dm-master2.toml +++ b/tests/ha_master/conf/dm-master2.toml @@ -4,3 +4,4 @@ master-addr = ":8361" advertise-addr = "localhost:8361" peer-urls = "localhost:8292" initial-cluster = "master1=http://localhost:8291,master2=http://localhost:8292,master3=http://localhost:8293,master4=http://localhost:8294,master5=http://localhost:8295" +auto-compaction-retention = "3s" diff --git a/tests/ha_master/conf/dm-master3.toml b/tests/ha_master/conf/dm-master3.toml index 5f746ab210..9a82bd6376 100644 --- a/tests/ha_master/conf/dm-master3.toml +++ b/tests/ha_master/conf/dm-master3.toml @@ -4,3 +4,4 @@ master-addr = ":8461" advertise-addr = "localhost:8461" peer-urls = "localhost:8293" initial-cluster = "master1=http://localhost:8291,master2=http://localhost:8292,master3=http://localhost:8293,master4=http://localhost:8294,master5=http://localhost:8295" +auto-compaction-retention = "3s" diff --git a/tests/ha_master/conf/dm-master4.toml b/tests/ha_master/conf/dm-master4.toml index 5d2b093778..84c1143c30 100644 --- a/tests/ha_master/conf/dm-master4.toml +++ b/tests/ha_master/conf/dm-master4.toml @@ -4,3 +4,4 @@ master-addr = ":8561" advertise-addr = "localhost:8561" peer-urls = "localhost:8294" initial-cluster = "master1=http://localhost:8291,master2=http://localhost:8292,master3=http://localhost:8293,master4=http://localhost:8294,master5=http://localhost:8295" +auto-compaction-retention = "3s" diff --git a/tests/ha_master/conf/dm-master5.toml b/tests/ha_master/conf/dm-master5.toml index fd28f23ac8..461bb1b5aa 100644 --- a/tests/ha_master/conf/dm-master5.toml +++ b/tests/ha_master/conf/dm-master5.toml @@ -4,3 +4,4 @@ master-addr = ":8661" advertise-addr = "localhost:8661" peer-urls = "localhost:8295" initial-cluster = "master1=http://localhost:8291,master2=http://localhost:8292,master3=http://localhost:8293,master4=http://localhost:8294,master5=http://localhost:8295" +auto-compaction-retention = "3s" diff --git a/tests/handle_error/conf/dm-master.toml b/tests/handle_error/conf/dm-master.toml index aa24d7d7d3..e700eeffa5 100644 --- a/tests/handle_error/conf/dm-master.toml +++ b/tests/handle_error/conf/dm-master.toml @@ -4,3 +4,4 @@ master-addr = ":8261" advertise-addr = "127.0.0.1:8261" peer-urls = "127.0.0.1:8291" initial-cluster = "master1=http://127.0.0.1:8291" +auto-compaction-retention = "3s" diff --git a/tests/http_apis/conf/dm-master.toml b/tests/http_apis/conf/dm-master.toml index 2811f72c6e..c9c8ec3fde 100644 --- a/tests/http_apis/conf/dm-master.toml +++ b/tests/http_apis/conf/dm-master.toml @@ -2,3 +2,4 @@ name = "master1" master-addr = ":8261" advertise-addr = "127.0.0.1:8261" +auto-compaction-retention = "3s" diff --git a/tests/import_goroutine_leak/conf/dm-master.toml b/tests/import_goroutine_leak/conf/dm-master.toml index 9a36bcbc84..7cecf59ad8 100644 --- a/tests/import_goroutine_leak/conf/dm-master.toml +++ b/tests/import_goroutine_leak/conf/dm-master.toml @@ -1,3 +1,4 @@ # Master Configuration. master-addr = ":8261" advertise-addr = "127.0.0.1:8261" +auto-compaction-retention = "3s" diff --git a/tests/import_goroutine_leak/conf/source1.yaml b/tests/import_goroutine_leak/conf/source1.yaml index 7d67feb8cf..6d0d3ba09c 100644 --- a/tests/import_goroutine_leak/conf/source1.yaml +++ b/tests/import_goroutine_leak/conf/source1.yaml @@ -1,7 +1,7 @@ source-id: mysql-replica-01 flavor: '' enable-gtid: false -enable-relay: true +enable-relay: false # in this case enable-relay will trigger a subtask reset, which fails the test relay-binlog-name: '' relay-binlog-gtid: '' from: diff --git a/tests/import_goroutine_leak/run.sh b/tests/import_goroutine_leak/run.sh index a38aaf3dca..a192007b96 100644 --- a/tests/import_goroutine_leak/run.sh +++ b/tests/import_goroutine_leak/run.sh @@ -111,7 +111,7 @@ function run() { # wait until the task running run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" \ "query-status test" \ - '"stage": "Running"' 2 + '"stage": "Running"' 1 sleep 2 # wait to be blocked # check to be blocked diff --git a/tests/import_v10x/conf/dm-master.toml b/tests/import_v10x/conf/dm-master.toml index 3c534e9f28..f091bf702a 100644 --- a/tests/import_v10x/conf/dm-master.toml +++ b/tests/import_v10x/conf/dm-master.toml @@ -2,3 +2,4 @@ master-addr = ":8261" advertise-addr = "127.0.0.1:8261" v1-sources-path = "./v1-sources" +auto-compaction-retention = "3s" diff --git a/tests/incremental_mode/conf/dm-master.toml b/tests/incremental_mode/conf/dm-master.toml index 9a36bcbc84..7cecf59ad8 100644 --- a/tests/incremental_mode/conf/dm-master.toml +++ b/tests/incremental_mode/conf/dm-master.toml @@ -1,3 +1,4 @@ # Master Configuration. master-addr = ":8261" advertise-addr = "127.0.0.1:8261" +auto-compaction-retention = "3s" diff --git a/tests/initial_unit/conf/dm-master.toml b/tests/initial_unit/conf/dm-master.toml index 9a36bcbc84..7cecf59ad8 100644 --- a/tests/initial_unit/conf/dm-master.toml +++ b/tests/initial_unit/conf/dm-master.toml @@ -1,3 +1,4 @@ # Master Configuration. master-addr = ":8261" advertise-addr = "127.0.0.1:8261" +auto-compaction-retention = "3s" diff --git a/tests/load_interrupt/conf/dm-master.toml b/tests/load_interrupt/conf/dm-master.toml index 9a36bcbc84..7cecf59ad8 100644 --- a/tests/load_interrupt/conf/dm-master.toml +++ b/tests/load_interrupt/conf/dm-master.toml @@ -1,3 +1,4 @@ # Master Configuration. master-addr = ":8261" advertise-addr = "127.0.0.1:8261" +auto-compaction-retention = "3s" diff --git a/tests/many_tables/conf/dm-master.toml b/tests/many_tables/conf/dm-master.toml index 9a36bcbc84..7cecf59ad8 100644 --- a/tests/many_tables/conf/dm-master.toml +++ b/tests/many_tables/conf/dm-master.toml @@ -1,3 +1,4 @@ # Master Configuration. master-addr = ":8261" advertise-addr = "127.0.0.1:8261" +auto-compaction-retention = "3s" diff --git a/tests/online_ddl/conf/dm-master.toml b/tests/online_ddl/conf/dm-master.toml index 9a36bcbc84..7cecf59ad8 100644 --- a/tests/online_ddl/conf/dm-master.toml +++ b/tests/online_ddl/conf/dm-master.toml @@ -1,3 +1,4 @@ # Master Configuration. master-addr = ":8261" advertise-addr = "127.0.0.1:8261" +auto-compaction-retention = "3s" diff --git a/tests/only_dml/conf/dm-master.toml b/tests/only_dml/conf/dm-master.toml index 9a36bcbc84..7cecf59ad8 100644 --- a/tests/only_dml/conf/dm-master.toml +++ b/tests/only_dml/conf/dm-master.toml @@ -1,3 +1,4 @@ # Master Configuration. master-addr = ":8261" advertise-addr = "127.0.0.1:8261" +auto-compaction-retention = "3s" diff --git a/tests/print_status/conf/dm-master.toml b/tests/print_status/conf/dm-master.toml index 9a36bcbc84..7cecf59ad8 100644 --- a/tests/print_status/conf/dm-master.toml +++ b/tests/print_status/conf/dm-master.toml @@ -1,3 +1,4 @@ # Master Configuration. master-addr = ":8261" advertise-addr = "127.0.0.1:8261" +auto-compaction-retention = "3s" diff --git a/tests/relay_interrupt/conf/dm-master.toml b/tests/relay_interrupt/conf/dm-master.toml index 9a36bcbc84..7cecf59ad8 100644 --- a/tests/relay_interrupt/conf/dm-master.toml +++ b/tests/relay_interrupt/conf/dm-master.toml @@ -1,3 +1,4 @@ # Master Configuration. master-addr = ":8261" advertise-addr = "127.0.0.1:8261" +auto-compaction-retention = "3s" diff --git a/tests/retry_cancel/conf/dm-master.toml b/tests/retry_cancel/conf/dm-master.toml index 9a36bcbc84..7cecf59ad8 100644 --- a/tests/retry_cancel/conf/dm-master.toml +++ b/tests/retry_cancel/conf/dm-master.toml @@ -1,3 +1,4 @@ # Master Configuration. master-addr = ":8261" advertise-addr = "127.0.0.1:8261" +auto-compaction-retention = "3s" diff --git a/tests/run.sh b/tests/run.sh index 91c07987af..9793558675 100755 --- a/tests/run.sh +++ b/tests/run.sh @@ -76,10 +76,19 @@ start_services function run() { script=$1 echo "Running test $script..." - TEST_DIR="$TEST_DIR" \ - PATH="tests/_utils:$PATH" \ - TEST_NAME="$(basename "$(dirname "$script")")" \ - bash +x "$script" + # run in verbose mode? + echo "Verbose mode = $VERBOSE" + if $VERBOSE ; then + TEST_DIR="$TEST_DIR" \ + PATH="tests/_utils:$PATH" \ + TEST_NAME="$(basename "$(dirname "$script")")" \ + bash -x "$script" + else + TEST_DIR="$TEST_DIR" \ + PATH="tests/_utils:$PATH" \ + TEST_NAME="$(basename "$(dirname "$script")")" \ + bash +x "$script" + fi } if [ "$test_case" == "*" ]; then diff --git a/tests/safe_mode/conf/dm-master.toml b/tests/safe_mode/conf/dm-master.toml index 9a36bcbc84..7cecf59ad8 100644 --- a/tests/safe_mode/conf/dm-master.toml +++ b/tests/safe_mode/conf/dm-master.toml @@ -1,3 +1,4 @@ # Master Configuration. master-addr = ":8261" advertise-addr = "127.0.0.1:8261" +auto-compaction-retention = "3s" diff --git a/tests/sequence_safe_mode/conf/dm-master.toml b/tests/sequence_safe_mode/conf/dm-master.toml index 9a36bcbc84..7cecf59ad8 100644 --- a/tests/sequence_safe_mode/conf/dm-master.toml +++ b/tests/sequence_safe_mode/conf/dm-master.toml @@ -1,3 +1,4 @@ # Master Configuration. master-addr = ":8261" advertise-addr = "127.0.0.1:8261" +auto-compaction-retention = "3s" diff --git a/tests/sequence_sharding/conf/dm-master.toml b/tests/sequence_sharding/conf/dm-master.toml index 9a36bcbc84..7cecf59ad8 100644 --- a/tests/sequence_sharding/conf/dm-master.toml +++ b/tests/sequence_sharding/conf/dm-master.toml @@ -1,3 +1,4 @@ # Master Configuration. master-addr = ":8261" advertise-addr = "127.0.0.1:8261" +auto-compaction-retention = "3s" diff --git a/tests/sequence_sharding_optimistic/conf/dm-master.toml b/tests/sequence_sharding_optimistic/conf/dm-master.toml index 9a36bcbc84..7cecf59ad8 100644 --- a/tests/sequence_sharding_optimistic/conf/dm-master.toml +++ b/tests/sequence_sharding_optimistic/conf/dm-master.toml @@ -1,3 +1,4 @@ # Master Configuration. master-addr = ":8261" advertise-addr = "127.0.0.1:8261" +auto-compaction-retention = "3s" diff --git a/tests/sequence_sharding_optimistic/data/db1.increment.sql b/tests/sequence_sharding_optimistic/data/db1.increment.sql index 4476682c10..dfea7e602b 100644 --- a/tests/sequence_sharding_optimistic/data/db1.increment.sql +++ b/tests/sequence_sharding_optimistic/data/db1.increment.sql @@ -27,24 +27,9 @@ alter table t2 drop column c1; insert into t1 (id, c2) values (100006, '120006'); insert into t2 (id, c2, c3) values (200006, '220006', 230006); -alter table t2 add column c1 varchar(20); --- at this point: --- t1(id, c2) --- t2(id, c1, c2, c3) -insert into t2 (id, c1, c2, c3) values (200007, '210007', '220007', 230007); -insert into t1 (id, c2) values (100007, '120007'); - -alter table t2 drop column c1; --- at this point: --- t1(id, c2) --- t2(id, c2, c3) -insert into t1 (id, c2) values (100008, '120008'); -insert into t2 (id, c2, c3) values (200008, '220008', 230008); - alter table t1 add column c3 int; -- at this point: -- t1(id, c2, c3) -- t2(id, c2, c3) -insert into t2 (id, c2, c3) values (200009, '220009', 230009); -insert into t1 (id, c2, c3) values (100009, '120009', 130009); - +insert into t2 (id, c2, c3) values (200007, '220007', 230007); +insert into t1 (id, c2, c3) values (100007, '120007', 130007); diff --git a/tests/sequence_sharding_optimistic/data/db1.increment2.sql b/tests/sequence_sharding_optimistic/data/db1.increment2.sql index f4d881f3c9..f289b3185d 100644 --- a/tests/sequence_sharding_optimistic/data/db1.increment2.sql +++ b/tests/sequence_sharding_optimistic/data/db1.increment2.sql @@ -1,8 +1,16 @@ use `sharding_seq_opt`; +alter table t2 add column c1 varchar(20); +-- at this point: +-- t1(id, c2, c3) +-- t2(id, c1, c2, c3) +insert into t2 (id, c1, c2, c3) values (200008, '210008', '220008', 230008); +insert into t1 (id, c2, c3) values (100008, '120008', 130008); + +alter table t2 drop column c1; -- at this point: -- t1(id, c2, c3) -- t2(id, c2, c3) -insert into t1 (id, c2, c3) values (100010, '120010', 130010); -insert into t2 (id, c2, c3) values (200010, '220010', 230010); +insert into t1 (id, c2, c3) values (100009, '120009', 130009); +insert into t2 (id, c2, c3) values (200009, '220009', 230009); diff --git a/tests/sequence_sharding_optimistic/run.sh b/tests/sequence_sharding_optimistic/run.sh index d0b9ab7127..d03edd54b0 100755 --- a/tests/sequence_sharding_optimistic/run.sh +++ b/tests/sequence_sharding_optimistic/run.sh @@ -157,7 +157,7 @@ run() { # try to get schema for the table, the latest schema got. 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 # this is NON-CLUSTERED index - check_log_contains ${WORK_DIR}/get_schema.log 'CREATE TABLE `t1` ( `id` bigint(20) NOT NULL, `c2` varchar(20) DEFAULT NULL, `c3` int(11) DEFAULT NULL, PRIMARY KEY (`id`) /\*T!\[clustered_index\] NONCLUSTERED \*/) ENGINE=InnoDB DEFAULT CHARSET=latin1 COLLATE=latin1_bin' 1 + check_log_contains ${WORK_DIR}/get_schema.log 'CREATE TABLE `t1` ( `id` bigint(20) NOT NULL, `c2` varchar(20) DEFAULT NULL, `c3` int(11) DEFAULT NULL, PRIMARY KEY (`id`) .*) ENGINE=InnoDB DEFAULT CHARSET=latin1 COLLATE=latin1_bin' 1 # drop the schema. curl -X PUT ${API_URL} -d '{"op":3, "task":"sequence_sharding_optimistic", "sources": ["mysql-replica-01"], "database":"sharding_seq_opt", "table":"t1"}' > ${WORK_DIR}/remove_schema.log @@ -188,7 +188,7 @@ run() { 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 cat ${WORK_DIR}/get_schema.log # schema tracker enables alter-primary-key, so this is NONCLUSTERED index - check_log_contains ${WORK_DIR}/get_schema.log 'CREATE TABLE `t1` ( `id` bigint(20) NOT NULL, `c2` varchar(20) DEFAULT NULL, `c3` bigint(11) DEFAULT NULL, PRIMARY KEY (`id`) /\*T!\[clustered_index\] NONCLUSTERED \*/) ENGINE=InnoDB DEFAULT CHARSET=latin1 COLLATE=latin1_bin' 1 + check_log_contains ${WORK_DIR}/get_schema.log 'CREATE TABLE `t1` ( `id` bigint(20) NOT NULL, `c2` varchar(20) DEFAULT NULL, `c3` bigint(11) DEFAULT NULL, PRIMARY KEY (`id`) .*) ENGINE=InnoDB DEFAULT CHARSET=latin1 COLLATE=latin1_bin' 1 # more data run_sql_file $cur/data/db1.increment2.sql $MYSQL_HOST1 $MYSQL_PORT1 $MYSQL_PASSWORD1 diff --git a/tests/sequence_sharding_removemeta/conf/dm-master.toml b/tests/sequence_sharding_removemeta/conf/dm-master.toml index 9a36bcbc84..7cecf59ad8 100644 --- a/tests/sequence_sharding_removemeta/conf/dm-master.toml +++ b/tests/sequence_sharding_removemeta/conf/dm-master.toml @@ -1,3 +1,4 @@ # Master Configuration. master-addr = ":8261" advertise-addr = "127.0.0.1:8261" +auto-compaction-retention = "3s" diff --git a/tests/shardddl1/conf/dm-master.toml b/tests/shardddl1/conf/dm-master.toml index aa24d7d7d3..e700eeffa5 100644 --- a/tests/shardddl1/conf/dm-master.toml +++ b/tests/shardddl1/conf/dm-master.toml @@ -4,3 +4,4 @@ master-addr = ":8261" advertise-addr = "127.0.0.1:8261" peer-urls = "127.0.0.1:8291" initial-cluster = "master1=http://127.0.0.1:8291" +auto-compaction-retention = "3s" diff --git a/tests/shardddl1/run.sh b/tests/shardddl1/run.sh index d2597ae75b..a2f7af1dc5 100644 --- a/tests/shardddl1/run.sh +++ b/tests/shardddl1/run.sh @@ -465,6 +465,48 @@ function DM_035() { run_case 035 "double-source-optimistic" "init_table 111 211 212" "clean_table" "optimistic" } +function DM_RENAME_TABLE_CASE() { + run_sql_source1 "insert into ${shardddl1}.${tb1} values(1);" + run_sql_source2 "insert into ${shardddl1}.${tb1} values(2);" + run_sql_source2 "insert into ${shardddl1}.${tb2} values(3);" + + run_sql_source1 "alter table ${shardddl1}.${tb1} add column a int;" + run_sql_source2 "alter table ${shardddl1}.${tb1} add column a int;" + run_sql_source2 "alter table ${shardddl1}.${tb2} add column a int;" + + run_sql_source1 "insert into ${shardddl1}.${tb1} values(4,4);" + run_sql_source2 "insert into ${shardddl1}.${tb1} values(5,5);" + run_sql_source2 "insert into ${shardddl1}.${tb2} values(6,6);" + + run_sql_source1 "rename table ${shardddl1}.${tb1} to ${shardddl1}.${tb3};" + run_sql_source2 "rename table ${shardddl1}.${tb1} to ${shardddl1}.${tb3};" + run_sql_source2 "rename table ${shardddl1}.${tb2} to ${shardddl1}.${tb4};" + + run_sql_source1 "insert into ${shardddl1}.${tb3} values(7,7)" + run_sql_source2 "insert into ${shardddl1}.${tb3} values(8,8);" + run_sql_source2 "insert into ${shardddl1}.${tb4} values(9,9);" + + run_sql_source1 "alter table ${shardddl1}.${tb3} add column b int;" + run_sql_source2 "alter table ${shardddl1}.${tb3} add column b int;" + run_sql_source2 "alter table ${shardddl1}.${tb4} add column b int;" + + run_sql_source1 "insert into ${shardddl1}.${tb3} values(10,10,10)" + run_sql_source2 "insert into ${shardddl1}.${tb3} values(11,11,11);" + run_sql_source2 "insert into ${shardddl1}.${tb4} values(12,12,12);" + + if [[ "$1" = "pessimistic" ]]; then + check_sync_diff $WORK_DIR $cur/conf/diff_config.toml + else + run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "query-status test" \ + "\`RENAME TABLE\` statement not supported in $1 mode" 2 + fi +} + +function DM_RENAME_TABLE() { + run_case RENAME_TABLE "double-source-pessimistic" "init_table 111 211 212" "clean_table" "pessimistic" + run_case RENAME_TABLE "double-source-optimistic" "init_table 111 211 212" "clean_table" "optimistic" +} function DM_RENAME_COLUMN_OPTIMISTIC_CASE() { run_sql_source1 "insert into ${shardddl1}.${tb1} values(1,'aaa');" @@ -535,8 +577,7 @@ function DM_RENAME_COLUMN_OPTIMISTIC_CASE() { "\"result\": true" 2 run_dm_ctl $WORK_DIR "127.0.0.1:$MASTER_PORT" \ - "resume-task test -s mysql-replica-02" \ - "\"result\": true" 2 + "resume-task test -s mysql-replica-02" # source2.table2's ddl fails # Unknown column 'a' in 'tb2' @@ -589,6 +630,70 @@ function DM_RENAME_COLUMN_OPTIMISTIC() { "clean_table" "optimistic" } +function DM_RECOVER_LOCK_CASE() { + # tb1(a,b) tb2(a,b) + run_sql_source1 "insert into ${shardddl1}.${tb1} values(1,1);" + run_sql_source2 "insert into ${shardddl1}.${tb2} values(2,2);" + + # tb1(a,b,c); tb2(a,b) + run_sql_source1 "alter table ${shardddl1}.${tb1} add column c varchar(10);" + run_sql_source1 "insert into ${shardddl1}.${tb1} values(3,3,'aaa');" + check_log_contain_with_retry "putted a shard DDL.*tb1.*ALTER TABLE .* ADD COLUMN" $WORK_DIR/worker1/log/dm-worker.log $WORK_DIR/worker2/log/dm-worker.log + + # tb1(a,b,c); tb2(a) + run_sql_source2 "alter table ${shardddl1}.${tb2} drop column b;" + check_log_contain_with_retry "putted a shard DDL.*tb2.*ALTER TABLE .* DROP COLUMN" $WORK_DIR/worker1/log/dm-worker.log $WORK_DIR/worker2/log/dm-worker.log + + echo "restart dm-master" + ps aux | grep dm-master |awk '{print $2}'|xargs kill || true + check_port_offline $MASTER_PORT 20 + sleep 2 + run_dm_master $WORK_DIR/master $MASTER_PORT $cur/conf/dm-master.toml + check_rpc_alive $cur/../bin/check_master_online 127.0.0.1:$MASTER_PORT + + run_sql_source1 "alter table ${shardddl1}.${tb1} drop column b;" + run_sql_source1 "insert into ${shardddl1}.${tb1} values(4,'bbb');" + run_sql_source2 "insert into ${shardddl1}.${tb2} values(5);" + check_log_contain_with_retry "putted a shard DDL.*tb1.*ALTER TABLE .* DROP COLUMN" $WORK_DIR/worker1/log/dm-worker.log $WORK_DIR/worker2/log/dm-worker.log + + # tb1(a,c); tb2(a,b) + run_sql_source2 "alter table ${shardddl1}.${tb2} add column b int;" + run_sql_source1 "insert into ${shardddl1}.${tb1} values(6,'ccc');" + run_sql_source2 "insert into ${shardddl1}.${tb2} values(7,7);" + check_log_contain_with_retry "putted a shard DDL.*tb2.*ALTER TABLE .* ADD COLUMN" $WORK_DIR/worker1/log/dm-worker.log $WORK_DIR/worker2/log/dm-worker.log + + # recover lock, tb1's info: (a,b,c)->(a,c); tb2's info: (a)->(a,b) + # joined(a,b,c); tb1(a,b,c); tb2(a) + # TrySync tb1: joined(a,b,c); tb1(a,c); tb2(a) + # TrySync tb2: joined(a,c); tb1(a,c); tb2(a,b) + echo "restart dm-master" + ps aux | grep dm-master |awk '{print $2}'|xargs kill || true + check_port_offline $MASTER_PORT 20 + sleep 2 + run_dm_master $WORK_DIR/master $MASTER_PORT $cur/conf/dm-master.toml + check_rpc_alive $cur/../bin/check_master_online 127.0.0.1:$MASTER_PORT + + run_sql_source1 "insert into ${shardddl1}.${tb1} values(8,'eee');" + run_sql_source2 "insert into ${shardddl1}.${tb2} values(9,9);" + + run_sql_source1 "alter table ${shardddl1}.${tb1} add column b int;" + run_sql_source2 "alter table ${shardddl1}.${tb2} add column c varchar(10) after a;" + run_sql_source1 "insert into ${shardddl1}.${tb1} values(10,'fff',10);" + run_sql_source2 "insert into ${shardddl1}.${tb2} values(11,'ggg',11);" + + check_sync_diff $WORK_DIR $cur/conf/diff_config.toml + run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "show-ddl-locks" \ + "no DDL lock exists" 1 +} + +function DM_RECOVER_LOCK() { + run_case RECOVER_LOCK "double-source-optimistic" \ + "run_sql_source1 \"create table ${shardddl1}.${tb1} (a int primary key, b int) DEFAULT CHARSET=latin1;\"; \ + run_sql_source2 \"create table ${shardddl1}.${tb2} (a int primary key, b int) DEFAULT CHARSET=latin1;\"" \ + "clean_table" "optimistic" +} + function run() { init_cluster init_database @@ -602,7 +707,9 @@ function run() { DM_${i} sleep 1 done + DM_RENAME_TABLE DM_RENAME_COLUMN_OPTIMISTIC + DM_RECOVER_LOCK } cleanup_data $shardddl diff --git a/tests/shardddl2/conf/dm-master.toml b/tests/shardddl2/conf/dm-master.toml index aa24d7d7d3..e700eeffa5 100644 --- a/tests/shardddl2/conf/dm-master.toml +++ b/tests/shardddl2/conf/dm-master.toml @@ -4,3 +4,4 @@ master-addr = ":8261" advertise-addr = "127.0.0.1:8261" peer-urls = "127.0.0.1:8291" initial-cluster = "master1=http://127.0.0.1:8291" +auto-compaction-retention = "3s" diff --git a/tests/shardddl3/conf/dm-master.toml b/tests/shardddl3/conf/dm-master.toml index aa24d7d7d3..e700eeffa5 100644 --- a/tests/shardddl3/conf/dm-master.toml +++ b/tests/shardddl3/conf/dm-master.toml @@ -4,3 +4,4 @@ master-addr = ":8261" advertise-addr = "127.0.0.1:8261" peer-urls = "127.0.0.1:8291" initial-cluster = "master1=http://127.0.0.1:8291" +auto-compaction-retention = "3s" diff --git a/tests/shardddl3/conf/tidb-alter-pk-config.toml b/tests/shardddl3/conf/tidb-alter-pk-config.toml new file mode 100644 index 0000000000..a1257473a3 --- /dev/null +++ b/tests/shardddl3/conf/tidb-alter-pk-config.toml @@ -0,0 +1,6 @@ +# TiDB Configuration. + +# alter-primary-key is used to control alter primary key feature. Default is false, indicate the alter primary key feature is disabled. +# If it is true, we can add the primary key by "alter table". However, if a table already exists before the switch is turned true and the data type of its primary key column is an integer, +# the primary key cannot be dropped. +alter-primary-key = true diff --git a/tests/shardddl3/run.sh b/tests/shardddl3/run.sh index dc0f04d4b0..af55e6a59c 100644 --- a/tests/shardddl3/run.sh +++ b/tests/shardddl3/run.sh @@ -114,9 +114,15 @@ function DM_078_CASE() { } function DM_078() { + # start a TiDB alter-pk + pkill -hup tidb-server 2>/dev/null || true + wait_process_exit tidb-server + run_tidb_server 4000 $TIDB_PASSWORD $cur/conf/tidb-alter-pk-config.toml + run_case 078 "single-source-pessimistic" "run_sql_source1 \"create table ${shardddl1}.${tb1} (id int unique, a int, b varchar(10));\"" "clean_table" "" run_case 078 "single-source-optimistic" "run_sql_source1 \"create table ${shardddl1}.${tb1} (id int unique, a int, b varchar(10));\"" "clean_table" "" + # don't revert tidb until DM_079 } function DM_079_CASE() { @@ -130,6 +136,11 @@ function DM_079_CASE() { function DM_079() { run_case 079 "single-source-pessimistic" "run_sql_source1 \"create table ${shardddl1}.${tb1} (a int primary key, b varchar(10));\"" "clean_table" "" run_case 079 "single-source-optimistic" "run_sql_source1 \"create table ${shardddl1}.${tb1} (a int primary key, b varchar(10));\"" "clean_table" "" + + # revert tidb + pkill -hup tidb-server 2>/dev/null || true + wait_process_exit tidb-server + run_tidb_server 4000 $TIDB_PASSWORD } function DM_080_CASE() { @@ -919,6 +930,16 @@ function DM_RemoveLock() { "bound" 2 } +function restart_master() { + echo "restart dm-master" + ps aux | grep dm-master |awk '{print $2}'|xargs kill || true + check_port_offline $MASTER_PORT 20 + sleep 2 + + 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 DM_RestartMaster_CASE() { run_sql_source1 "insert into ${shardddl1}.${tb1} values(1,'aaa');" run_sql_source2 "insert into ${shardddl1}.${tb1} values(2,'bbb');" @@ -947,11 +968,7 @@ function DM_RestartMaster_CASE() { 'mysql-replica-02-`shardddl1`.`tb1`' 1 fi - echo "restart dm-master" - ps aux | grep dm-master |awk '{print $2}'|xargs kill || true - check_port_offline $MASTER_PORT 20 - 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 + restart_master if [[ "$1" = "pessimistic" ]]; then run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" \ @@ -984,6 +1001,94 @@ function DM_RestartMaster() { "clean_table" "optimistic" } +function restart_master_on_pos() { + if [ "$1" = "$2" ]; then + restart_master + fi +} + +function DM_DropAddColumn_CASE() { + reset=$2 + + run_sql_source1 "insert into ${shardddl1}.${tb1} values(1,1,1);" + run_sql_source2 "insert into ${shardddl1}.${tb1} values(2,2,2);" + + check_sync_diff $WORK_DIR $cur/conf/diff_config.toml + + run_sql_source1 "alter table ${shardddl1}.${tb1} drop column c;" + run_sql_source1 "insert into ${shardddl1}.${tb1} values(3,3);" + + restart_master_on_pos $reset "1" + + run_sql_source1 "alter table ${shardddl1}.${tb1} drop column b;" + run_sql_source1 "insert into ${shardddl1}.${tb1} values(4);" + + restart_master_on_pos $reset "2" + + run_sql_source2 "alter table ${shardddl1}.${tb1} drop column c;" + run_sql_source2 "insert into ${shardddl1}.${tb1} values(5,5);" + + restart_master_on_pos $reset "3" + + # make sure column c is fully dropped in the downstream + check_log_contain_with_retry 'finish to handle ddls in optimistic shard mode' $WORK_DIR/worker1/log/dm-worker.log + check_log_contain_with_retry 'finish to handle ddls in optimistic shard mode' $WORK_DIR/worker2/log/dm-worker.log + + run_sql_source1 "alter table ${shardddl1}.${tb1} add column c int;" + run_sql_source1 "insert into ${shardddl1}.${tb1} values(6,6);" + + restart_master_on_pos $reset "4" + + # make sure task to step in "Sync" stage + run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "query-status test" \ + "\"stage\": \"Running\"" 3 \ + "\"unit\": \"Sync\"" 2 + + run_sql_source1 "alter table ${shardddl1}.${tb1} add column b int after a;" + + restart_master_on_pos $reset "5" + + run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "query-status test" \ + "because schema conflict detected" 1 \ + "add column b that wasn't fully dropped in downstream" 1 + + check_sync_diff $WORK_DIR $cur/conf/diff_config.toml 3 'fail' + + # try to fix data + echo 'CREATE TABLE `tb1` ( `a` int(11) NOT NULL, `b` int(11) DEFAULT NULL, `c` int(11) DEFAULT NULL, PRIMARY KEY (`a`) /*T![clustered_index] NONCLUSTERED */) ENGINE=InnoDB DEFAULT CHARSET=latin1 COLLATE=latin1_bin' > ${WORK_DIR}/schema.sql + run_dm_ctl $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "operate-schema set test ${WORK_DIR}/schema.sql -s mysql-replica-01 -d ${shardddl1} -t ${tb1}" \ + "\"result\": true" 2 + + # skip this error + run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "handle-error test skip" \ + "\"result\": true" 2 \ + "\"source 'mysql-replica-02' has no error\"" 1 + + run_sql_source1 "update ${shardddl1}.${tb1} set b=1 where a=1;" + run_sql_source1 "update ${shardddl1}.${tb1} set b=3 where a=3;" + run_sql_source1 "update ${shardddl1}.${tb1} set b=4 where a=4;" + run_sql_source1 "update ${shardddl1}.${tb1} set b=6 where a=6;" + run_sql_source2 "alter table ${shardddl1}.${tb1} add column c int" + run_sql_source2 "insert into ${shardddl1}.${tb1} values(7,7,7);" + + check_sync_diff $WORK_DIR $cur/conf/diff_config.toml +} + +function DM_DropAddColumn() { + for i in `seq 0 5` + do + echo "run DM_DropAddColumn case #${i}" + run_case DropAddColumn "double-source-optimistic" \ + "run_sql_source1 \"create table ${shardddl1}.${tb1} (a int primary key, b int, c int);\"; \ + run_sql_source2 \"create table ${shardddl1}.${tb1} (a int primary key, b int, c int);\"" \ + "clean_table" "optimistic" "$i" + done +} + function run() { init_cluster init_database @@ -1001,6 +1106,8 @@ function run() { DM_RemoveLock DM_RestartMaster + + DM_DropAddColumn } cleanup_data $shardddl diff --git a/tests/sharding/conf/dm-master.toml b/tests/sharding/conf/dm-master.toml index 9a36bcbc84..7cecf59ad8 100644 --- a/tests/sharding/conf/dm-master.toml +++ b/tests/sharding/conf/dm-master.toml @@ -1,3 +1,4 @@ # Master Configuration. master-addr = ":8261" advertise-addr = "127.0.0.1:8261" +auto-compaction-retention = "3s" diff --git a/tests/sharding2/conf/dm-master.toml b/tests/sharding2/conf/dm-master.toml index 9a36bcbc84..7cecf59ad8 100644 --- a/tests/sharding2/conf/dm-master.toml +++ b/tests/sharding2/conf/dm-master.toml @@ -1,3 +1,4 @@ # Master Configuration. master-addr = ":8261" advertise-addr = "127.0.0.1:8261" +auto-compaction-retention = "3s" diff --git a/tests/start_task/conf/dm-master.toml b/tests/start_task/conf/dm-master.toml index 9a36bcbc84..7cecf59ad8 100644 --- a/tests/start_task/conf/dm-master.toml +++ b/tests/start_task/conf/dm-master.toml @@ -1,3 +1,4 @@ # Master Configuration. master-addr = ":8261" advertise-addr = "127.0.0.1:8261" +auto-compaction-retention = "3s" diff --git a/tests/tls/conf/dm-master1.toml b/tests/tls/conf/dm-master1.toml index fdae0252af..c321b44c7a 100644 --- a/tests/tls/conf/dm-master1.toml +++ b/tests/tls/conf/dm-master1.toml @@ -9,3 +9,4 @@ ssl-ca = "dir-placeholer/ca.pem" ssl-cert = "dir-placeholer/dm.pem" ssl-key = "dir-placeholer/dm.key" cert-allowed-cn = ["dm"] +auto-compaction-retention = "3s" diff --git a/tests/tls/conf/dm-master2.toml b/tests/tls/conf/dm-master2.toml index d107e4a4cd..6c977f5dc3 100644 --- a/tests/tls/conf/dm-master2.toml +++ b/tests/tls/conf/dm-master2.toml @@ -8,4 +8,5 @@ initial-cluster = "master1=https://127.0.0.1:8291,master2=http://127.0.0.1:8292, ssl-ca = "dir-placeholer/ca.pem" ssl-cert = "dir-placeholer/dm.pem" ssl-key = "dir-placeholer/dm.key" -cert-allowed-cn = ["dm"] \ No newline at end of file +cert-allowed-cn = ["dm"] +auto-compaction-retention = "3s" diff --git a/tests/tls/conf/dm-master3.toml b/tests/tls/conf/dm-master3.toml index 5e85ac017f..aaff43fc16 100644 --- a/tests/tls/conf/dm-master3.toml +++ b/tests/tls/conf/dm-master3.toml @@ -8,4 +8,5 @@ initial-cluster = "master1=http://127.0.0.1:8291,master2=https://127.0.0.1:8292, ssl-ca = "dir-placeholer/ca.pem" ssl-cert = "dir-placeholer/dm.pem" ssl-key = "dir-placeholer/dm.key" -cert-allowed-cn = ["dm"] \ No newline at end of file +cert-allowed-cn = ["dm"] +auto-compaction-retention = "3s" diff --git a/tests/upstream_switch/conf/dm-master.toml b/tests/upstream_switch/conf/dm-master.toml index aa24d7d7d3..e700eeffa5 100644 --- a/tests/upstream_switch/conf/dm-master.toml +++ b/tests/upstream_switch/conf/dm-master.toml @@ -4,3 +4,4 @@ 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"