From d278c7968ca69926b7dd10f0b1e7f110fb5c64fe Mon Sep 17 00:00:00 2001 From: lance6716 Date: Thu, 11 Mar 2021 18:19:44 +0800 Subject: [PATCH 01/46] worker: move relay related function together --- dm/worker/server.go | 54 ++------------------ dm/worker/server_test.go | 14 ++---- dm/worker/worker.go | 103 ++++++++++++++++++++++++++++++--------- dm/worker/worker_test.go | 5 +- 4 files changed, 92 insertions(+), 84 deletions(-) diff --git a/dm/worker/server.go b/dm/worker/server.go index 1588783561..b4480bb2de 100644 --- a/dm/worker/server.go +++ b/dm/worker/server.go @@ -576,30 +576,6 @@ func (s *Server) startWorker(cfg *config.SourceConfig) error { subTaskCfgs = append(subTaskCfgs, &subTaskCfgClone) } - if cfg.EnableRelay { - dctx, dcancel := context.WithTimeout(s.etcdClient.Ctx(), time.Duration(len(subTaskCfgs))*3*time.Second) - defer dcancel() - minLoc, err1 := getMinLocInAllSubTasks(dctx, subTaskCfgs) - if err1 != nil { - return err1 - } - - if minLoc != nil { - log.L().Info("get min location in all subtasks", zap.Stringer("location", *minLoc)) - cfg.RelayBinLogName = binlog.AdjustPosition(minLoc.Position).Name - cfg.RelayBinlogGTID = minLoc.GTIDSetStr() - // set UUIDSuffix when bound to a source - cfg.UUIDSuffix, err = binlog.ExtractSuffix(minLoc.Position.Name) - if err != nil { - return err - } - } else { - // set UUIDSuffix even not checkpoint exist - // so we will still remove relay dir - cfg.UUIDSuffix = binlog.MinUUIDSuffix - } - } - log.L().Info("starting to handle mysql source", zap.String("sourceCfg", cfg.String()), zap.Reflect("subTasks", subTaskCfgs)) w, err := NewWorker(cfg, s.etcdClient, s.cfg.Name) if err != nil { @@ -607,23 +583,13 @@ func (s *Server) startWorker(cfg *config.SourceConfig) error { } s.setWorker(w, false) - startRelay := false - var revRelay int64 if cfg.EnableRelay { - var relayStage ha.Stage - // we get the newest relay stages directly which will omit the relay stage PUT/DELETE event - // because triggering these events is useless now - relayStage, revRelay, err = ha.GetRelayStage(s.etcdClient, cfg.SourceID) - if err != nil { - // TODO: need retry + s.UpdateKeepAliveTTL(s.cfg.RelayKeepAliveTTL) + if err := w.EnableRelay(); err != nil { return err } - startRelay = !relayStage.IsDeleted && relayStage.Expect == pb.Stage_Running - s.UpdateKeepAliveTTL(s.cfg.RelayKeepAliveTTL) } - go func() { - w.Start(startRelay) - }() + go w.Start() isStarted := utils.WaitSomething(50, 100*time.Millisecond, func() bool { return w.closed.Get() == closedFalse @@ -652,16 +618,6 @@ func (s *Server) startWorker(cfg *config.SourceConfig) error { w.observeSubtaskStage(w.ctx, s.etcdClient, revSubTask) }() - if cfg.EnableRelay { - w.wg.Add(1) - go func() { - defer w.wg.Done() - // TODO: handle fatal error from observeRelayStage - //nolint:errcheck - w.observeRelayStage(w.ctx, s.etcdClient, revRelay) - }() - } - log.L().Info("started to handle mysql source", zap.String("sourceCfg", cfg.String())) return nil } @@ -679,7 +635,7 @@ func makeCommonWorkerResponse(reqErr error) *pb.CommonWorkerResponse { // all subTask in subTaskCfgs should have same source // this function return the min location in all subtasks, used for relay's location -func getMinLocInAllSubTasks(ctx context.Context, subTaskCfgs []*config.SubTaskConfig) (minLoc *binlog.Location, err error) { +func getMinLocInAllSubTasks(ctx context.Context, subTaskCfgs map[string]config.SubTaskConfig) (minLoc *binlog.Location, err error) { for _, subTaskCfg := range subTaskCfgs { loc, err := getMinLocForSubTaskFunc(ctx, subTaskCfg) if err != nil { @@ -702,7 +658,7 @@ func getMinLocInAllSubTasks(ctx context.Context, subTaskCfgs []*config.SubTaskCo return minLoc, nil } -func getMinLocForSubTask(ctx context.Context, subTaskCfg *config.SubTaskConfig) (minLoc *binlog.Location, err error) { +func getMinLocForSubTask(ctx context.Context, subTaskCfg config.SubTaskConfig) (minLoc *binlog.Location, err error) { if subTaskCfg.Mode == config.ModeFull { return nil, nil } diff --git a/dm/worker/server_test.go b/dm/worker/server_test.go index 3a05543d39..d61dce8d99 100644 --- a/dm/worker/server_test.go +++ b/dm/worker/server_test.go @@ -532,14 +532,10 @@ func (t *testServer) testStopWorkerWhenLostConnect(c *C, s *Server, ETCD *embed. } func (t *testServer) TestGetMinLocInAllSubTasks(c *C) { - subTaskCfg := []*config.SubTaskConfig{ - { - Name: "test2", - }, { - Name: "test3", - }, { - Name: "test1", - }, + subTaskCfg := map[string]config.SubTaskConfig{ + "test2": {Name: "test2"}, + "test3": {Name: "test3"}, + "test1": {Name: "test1"}, } minLoc, err := getMinLocInAllSubTasks(context.Background(), subTaskCfg) c.Assert(err, IsNil) @@ -666,7 +662,7 @@ func (t *testServer) TestUnifyMasterBinlogPos(c *C) { c.Assert(relay.RelayCatchUpMaster, IsTrue) } -func getFakeLocForSubTask(ctx context.Context, subTaskCfg *config.SubTaskConfig) (minLoc *binlog.Location, err error) { +func getFakeLocForSubTask(ctx context.Context, subTaskCfg config.SubTaskConfig) (minLoc *binlog.Location, err error) { gset1, _ := gtid.ParserGTID(mysql.MySQLFlavor, "ba8f633f-1f15-11eb-b1c7-0242ac110001:1-30") gset2, _ := gtid.ParserGTID(mysql.MySQLFlavor, "ba8f633f-1f15-11eb-b1c7-0242ac110001:1-50") gset3, _ := gtid.ParserGTID(mysql.MySQLFlavor, "ba8f633f-1f15-11eb-b1c7-0242ac110001:1-50,ba8f633f-1f15-11eb-b1c7-0242ac110002:1") diff --git a/dm/worker/worker.go b/dm/worker/worker.go index ec40e46cb9..1c0aece44c 100644 --- a/dm/worker/worker.go +++ b/dm/worker/worker.go @@ -28,6 +28,7 @@ import ( "github.com/pingcap/dm/dm/config" "github.com/pingcap/dm/dm/pb" + "github.com/pingcap/dm/pkg/binlog" "github.com/pingcap/dm/pkg/etcdutil" "github.com/pingcap/dm/pkg/ha" "github.com/pingcap/dm/pkg/log" @@ -68,7 +69,8 @@ type Worker struct { name string } -// NewWorker creates a new Worker +// NewWorker creates a new Worker. The functionality of relay and subtask is disabled by default, need call EnableRelay +// and EnableSubtask later func NewWorker(cfg *config.SourceConfig, etcdClient *clientv3.Client, name string) (w *Worker, err error) { w = &Worker{ cfg: cfg, @@ -89,18 +91,6 @@ func NewWorker(cfg *config.SourceConfig, etcdClient *clientv3.Client, name strin } }(w) - if cfg.EnableRelay { - // initial relay holder, the cfg's password need decrypt - w.relayHolder = NewRelayHolder(cfg) - purger1, err1 := w.relayHolder.Init([]purger.PurgeInterceptor{ - w, - }) - if err1 != nil { - return nil, err1 - } - w.relayPurger = purger1 - } - // initial task status checker if w.cfg.Checker.CheckEnable { tsc := NewTaskStatusChecker(w.cfg.Checker, w) @@ -119,17 +109,7 @@ func NewWorker(cfg *config.SourceConfig, etcdClient *clientv3.Client, name strin } // Start starts working -func (w *Worker) Start(startRelay bool) { - - if w.cfg.EnableRelay && startRelay { - log.L().Info("relay is started") - // start relay - w.relayHolder.Start() - - // start purger - w.relayPurger.Start() - } - +func (w *Worker) Start() { // start task status checker if w.cfg.Checker.CheckEnable { w.taskStatusChecker.Start() @@ -190,6 +170,81 @@ func (w *Worker) Close() { w.l.Info("Stop worker") } +// EnableRelay enables the functionality of start/watch/handle relay +func (w *Worker) EnableRelay() error { + // 1. adjust relay starting position, to the earliest of subtasks + _, subTaskCfgs, _, err := ha.GetSubTaskStageConfig(w.etcdClient, w.cfg.SourceID) + if err != nil { + return err + } + dctx, dcancel := context.WithTimeout(w.etcdClient.Ctx(), time.Duration(len(subTaskCfgs))*3*time.Second) + defer dcancel() + minLoc, err1 := getMinLocInAllSubTasks(dctx, subTaskCfgs) + if err1 != nil { + return err1 + } + + if minLoc != nil { + log.L().Info("get min location in all subtasks", zap.Stringer("location", *minLoc)) + w.cfg.RelayBinLogName = binlog.AdjustPosition(minLoc.Position).Name + w.cfg.RelayBinlogGTID = minLoc.GTIDSetStr() + // set UUIDSuffix when bound to a source + w.cfg.UUIDSuffix, err = binlog.ExtractSuffix(minLoc.Position.Name) + if err != nil { + return err + } + } else { + // set UUIDSuffix even not checkpoint exist + // so we will still remove relay dir + w.cfg.UUIDSuffix = binlog.MinUUIDSuffix + } + + // 2. initial relay holder, the cfg's password need decrypt + w.relayHolder = NewRelayHolder(w.cfg) + relayPurger, err := w.relayHolder.Init([]purger.PurgeInterceptor{ + w, + }) + if err != nil { + return err + } + w.relayPurger = relayPurger + + // 3. get relay stage from etcd and check if need starting + // we get the newest relay stages directly which will omit the relay stage PUT/DELETE event + // because triggering these events is useless now + relayStage, revRelay, err := ha.GetRelayStage(w.etcdClient, w.cfg.SourceID) + if err != nil { + // TODO: need retry + return err + } + startImmediately := !relayStage.IsDeleted && relayStage.Expect == pb.Stage_Running + if startImmediately { + log.L().Info("relay is started") + w.relayHolder.Start() + w.relayPurger.Start() + } + + // 4. watch relay stage + w.wg.Add(1) + go func() { + defer w.wg.Done() + // TODO: handle fatal error from observeRelayStage + //nolint:errcheck + w.observeRelayStage(w.ctx, w.etcdClient, revRelay) + }() + return nil +} + +// EnableSubtask TODO +func (w *Worker) EnableSubtask() { + +} + +// DisableSubtask TODO +func (w *Worker) DisableSubtask() { + +} + // StartSubTask creates a sub task an run it func (w *Worker) StartSubTask(cfg *config.SubTaskConfig, expectStage pb.Stage) error { w.Lock() diff --git a/dm/worker/worker_test.go b/dm/worker/worker_test.go index 25b7074f43..401f505cef 100644 --- a/dm/worker/worker_test.go +++ b/dm/worker/worker_test.go @@ -252,7 +252,7 @@ func (t *testWorkerEtcdCompact) TestWatchSubtaskStageEtcdCompact(c *C) { defer cancel() defer w.Close() go func() { - w.Start(false) + w.Start() }() c.Assert(utils.WaitSomething(50, 100*time.Millisecond, func() bool { return w.closed.Get() == closedFalse @@ -366,7 +366,8 @@ func (t *testWorkerEtcdCompact) TestWatchRelayStageEtcdCompact(c *C) { defer cancel() defer w.Close() go func() { - w.Start(true) + c.Assert(w.EnableRelay(), IsNil) + w.Start() }() c.Assert(utils.WaitSomething(50, 100*time.Millisecond, func() bool { return w.closed.Get() == closedFalse From 8d5db03d33e4cd9da3b6eb19ad5a82e5f3e129c5 Mon Sep 17 00:00:00 2001 From: lance6716 Date: Thu, 11 Mar 2021 21:52:44 +0800 Subject: [PATCH 02/46] try fix CI --- dm/worker/server.go | 10 +++------- dm/worker/worker.go | 7 +++++++ dm/worker/worker_test.go | 29 ++++++++++++++++++++++++++--- 3 files changed, 36 insertions(+), 10 deletions(-) diff --git a/dm/worker/server.go b/dm/worker/server.go index b4480bb2de..361239cca2 100644 --- a/dm/worker/server.go +++ b/dm/worker/server.go @@ -566,14 +566,10 @@ func (s *Server) startWorker(cfg *config.SourceConfig) error { subTaskCfgs := make([]*config.SubTaskConfig, 0, len(subTaskCfgm)) for _, subTaskCfg := range subTaskCfgm { - subTaskCfg.LogLevel = s.cfg.LogLevel - subTaskCfg.LogFile = s.cfg.LogFile - subTaskCfg.LogFormat = s.cfg.LogFormat - subTaskCfgClone := subTaskCfg - if err = copyConfigFromSource(&subTaskCfgClone, cfg); err != nil { - return err + if err2 := copyConfigFromSource(&subTaskCfg, cfg); err2 != nil { + return err2 } - subTaskCfgs = append(subTaskCfgs, &subTaskCfgClone) + subTaskCfgs = append(subTaskCfgs, &subTaskCfg) } log.L().Info("starting to handle mysql source", zap.String("sourceCfg", cfg.String()), zap.Reflect("subTasks", subTaskCfgs)) diff --git a/dm/worker/worker.go b/dm/worker/worker.go index 1c0aece44c..d60041c44f 100644 --- a/dm/worker/worker.go +++ b/dm/worker/worker.go @@ -177,6 +177,13 @@ func (w *Worker) EnableRelay() error { if err != nil { return err } + for k, subTaskCfg := range subTaskCfgs { + if err2 := copyConfigFromSource(&subTaskCfg, w.cfg); err2 != nil { + return err2 + } + subTaskCfgs[k] = subTaskCfg + } + dctx, dcancel := context.WithTimeout(w.etcdClient.Ctx(), time.Duration(len(subTaskCfgs))*3*time.Second) defer dcancel() minLoc, err1 := getMinLocInAllSubTasks(dctx, subTaskCfgs) diff --git a/dm/worker/worker_test.go b/dm/worker/worker_test.go index 401f505cef..0390b81d1b 100644 --- a/dm/worker/worker_test.go +++ b/dm/worker/worker_test.go @@ -47,11 +47,34 @@ func (t *testServer) testWorker(c *C) { NewRelayHolder = NewRealRelayHolder }() - _, err := NewWorker(&cfg, nil, "") - c.Assert(err, ErrorMatches, "init error") + var ( + masterAddr = tempurl.Alloc()[len("http://"):] + keepAliveTTL = int64(1) + ) + etcdDir := c.MkDir() + ETCD, err := createMockETCD(etcdDir, "http://"+masterAddr) + c.Assert(err, IsNil) + defer ETCD.Close() + workerCfg := NewConfig() + c.Assert(workerCfg.Parse([]string{"-config=./dm-worker.toml"}), IsNil) + workerCfg.Join = masterAddr + workerCfg.KeepAliveTTL = keepAliveTTL + workerCfg.RelayKeepAliveTTL = keepAliveTTL + + etcdCli, err := clientv3.New(clientv3.Config{ + Endpoints: GetJoinURLs(workerCfg.Join), + DialTimeout: dialTimeout, + DialKeepAliveTime: keepaliveTime, + DialKeepAliveTimeout: keepaliveTimeout, + }) + c.Assert(err, IsNil) + + w, err := NewWorker(&cfg, etcdCli, "") + c.Assert(err, IsNil) + c.Assert(w.EnableRelay(), ErrorMatches, "init error") NewRelayHolder = NewDummyRelayHolder - w, err := NewWorker(&cfg, nil, "") + w, err = NewWorker(&cfg, etcdCli, "") c.Assert(err, IsNil) c.Assert(w.StatusJSON(context.Background(), ""), HasLen, emptyWorkerStatusInfoJSONLength) //c.Assert(w.closed.Get(), Equals, closedFalse) From ce9ec0ddbe8440ddc7dbe3cf78818fe17219be18 Mon Sep 17 00:00:00 2001 From: lance6716 Date: Thu, 11 Mar 2021 23:01:55 +0800 Subject: [PATCH 03/46] fix some CI --- dm/worker/server_test.go | 6 ++++-- dm/worker/worker_test.go | 20 +++++--------------- 2 files changed, 9 insertions(+), 17 deletions(-) diff --git a/dm/worker/server_test.go b/dm/worker/server_test.go index d61dce8d99..a040770a9f 100644 --- a/dm/worker/server_test.go +++ b/dm/worker/server_test.go @@ -532,6 +532,7 @@ func (t *testServer) testStopWorkerWhenLostConnect(c *C, s *Server, ETCD *embed. } func (t *testServer) TestGetMinLocInAllSubTasks(c *C) { + subTaskCfg := map[string]config.SubTaskConfig{ "test2": {Name: "test2"}, "test3": {Name: "test3"}, @@ -542,8 +543,9 @@ func (t *testServer) TestGetMinLocInAllSubTasks(c *C) { c.Assert(minLoc.Position.Name, Equals, "mysql-binlog.00001") c.Assert(minLoc.Position.Pos, Equals, uint32(12)) - for _, subtask := range subTaskCfg { - subtask.EnableGTID = true + for k, cfg := range subTaskCfg { + cfg.EnableGTID = true + subTaskCfg[k] = cfg } minLoc, err = getMinLocInAllSubTasks(context.Background(), subTaskCfg) diff --git a/dm/worker/worker_test.go b/dm/worker/worker_test.go index 0390b81d1b..adcd7737e2 100644 --- a/dm/worker/worker_test.go +++ b/dm/worker/worker_test.go @@ -400,18 +400,18 @@ func (t *testWorkerEtcdCompact) TestWatchRelayStageEtcdCompact(c *C) { c.Assert(w.relayHolder, NotNil) _, err = ha.PutSourceCfg(etcdCli, sourceCfg) c.Assert(err, IsNil) - _, err = ha.PutRelayStageSourceBound(etcdCli, ha.NewRelayStage(pb.Stage_Running, sourceCfg.SourceID), + rev, err := ha.PutRelayStageSourceBound(etcdCli, ha.NewRelayStage(pb.Stage_Running, sourceCfg.SourceID), ha.NewSourceBound(sourceCfg.SourceID, cfg.Name)) c.Assert(err, IsNil) - rev, err := ha.DeleteSourceCfgRelayStageSourceBound(etcdCli, sourceCfg.SourceID, cfg.Name) - c.Assert(err, IsNil) // check relay stage, should be running c.Assert(utils.WaitSomething(30, 100*time.Millisecond, func() bool { return w.relayHolder.Stage() == pb.Stage_Running }), IsTrue) - // step 3: trigger etcd compaction and check whether we can receive it through watcher + // step 3: trigger etcd compaction and check whether we can receive it through watcher, then we delete relay stage _, err = etcdCli.Compact(ctx, rev) c.Assert(err, IsNil) + rev, err = ha.DeleteSourceCfgRelayStageSourceBound(etcdCli, sourceCfg.SourceID, cfg.Name) + c.Assert(err, IsNil) relayStageCh := make(chan ha.Stage, 10) relayErrCh := make(chan error, 10) ha.WatchRelayStage(ctx, etcdCli, cfg.Name, startRev, relayStageCh, relayErrCh) @@ -421,19 +421,9 @@ func (t *testWorkerEtcdCompact) TestWatchRelayStageEtcdCompact(c *C) { case <-time.After(300 * time.Millisecond): c.Fatal("fail to get etcd error compacted") } - // step 4: watch relay stage from startRev - var wg sync.WaitGroup - ctx1, cancel1 := context.WithCancel(ctx) - wg.Add(1) - go func() { - defer wg.Done() - c.Assert(w.observeRelayStage(ctx1, etcdCli, startRev), IsNil) - }() - // step 5: should stop the running relay + // step 4: should stop the running relay because see deletion after compaction time.Sleep(time.Second) c.Assert(utils.WaitSomething(30, 100*time.Millisecond, func() bool { return w.relayHolder.Stage() == pb.Stage_Stopped }), IsTrue) - cancel1() - wg.Wait() } From 8ba22e1a2ddd55df724f0891cf4bd2589d52e5d9 Mon Sep 17 00:00:00 2001 From: lance6716 Date: Fri, 12 Mar 2021 11:33:32 +0800 Subject: [PATCH 04/46] fix CI --- dm/worker/server.go | 5 +++-- dm/worker/worker_test.go | 2 +- tests/ha_cases/run.sh | 1 + 3 files changed, 5 insertions(+), 3 deletions(-) diff --git a/dm/worker/server.go b/dm/worker/server.go index 361239cca2..e50565ffbc 100644 --- a/dm/worker/server.go +++ b/dm/worker/server.go @@ -566,10 +566,11 @@ func (s *Server) startWorker(cfg *config.SourceConfig) error { subTaskCfgs := make([]*config.SubTaskConfig, 0, len(subTaskCfgm)) for _, subTaskCfg := range subTaskCfgm { - if err2 := copyConfigFromSource(&subTaskCfg, cfg); err2 != nil { + clone := subTaskCfg + if err2 := copyConfigFromSource(&clone, cfg); err2 != nil { return err2 } - subTaskCfgs = append(subTaskCfgs, &subTaskCfg) + subTaskCfgs = append(subTaskCfgs, &clone) } log.L().Info("starting to handle mysql source", zap.String("sourceCfg", cfg.String()), zap.Reflect("subTasks", subTaskCfgs)) diff --git a/dm/worker/worker_test.go b/dm/worker/worker_test.go index adcd7737e2..35226e18cc 100644 --- a/dm/worker/worker_test.go +++ b/dm/worker/worker_test.go @@ -410,7 +410,7 @@ func (t *testWorkerEtcdCompact) TestWatchRelayStageEtcdCompact(c *C) { // step 3: trigger etcd compaction and check whether we can receive it through watcher, then we delete relay stage _, err = etcdCli.Compact(ctx, rev) c.Assert(err, IsNil) - rev, err = ha.DeleteSourceCfgRelayStageSourceBound(etcdCli, sourceCfg.SourceID, cfg.Name) + _, err = ha.DeleteSourceCfgRelayStageSourceBound(etcdCli, sourceCfg.SourceID, cfg.Name) c.Assert(err, IsNil) relayStageCh := make(chan ha.Stage, 10) relayErrCh := make(chan error, 10) diff --git a/tests/ha_cases/run.sh b/tests/ha_cases/run.sh index 9689ed2a82..1ab8d60802 100755 --- a/tests/ha_cases/run.sh +++ b/tests/ha_cases/run.sh @@ -511,6 +511,7 @@ function test_stop_task() { function test_multi_task_reduce_and_restart_worker() { + # here echo "[$(date)] <<<<<< start test_multi_task_reduce_and_restart_worker >>>>>>" test_multi_task_running From 82dfc73416b18fe3996093e268904c2646be7e48 Mon Sep 17 00:00:00 2001 From: lance6716 Date: Fri, 12 Mar 2021 12:05:23 +0800 Subject: [PATCH 05/46] remove some comment --- dm/worker/worker.go | 10 ---------- tests/ha_cases/run.sh | 1 - 2 files changed, 11 deletions(-) diff --git a/dm/worker/worker.go b/dm/worker/worker.go index d60041c44f..fe307ff4c8 100644 --- a/dm/worker/worker.go +++ b/dm/worker/worker.go @@ -242,16 +242,6 @@ func (w *Worker) EnableRelay() error { return nil } -// EnableSubtask TODO -func (w *Worker) EnableSubtask() { - -} - -// DisableSubtask TODO -func (w *Worker) DisableSubtask() { - -} - // StartSubTask creates a sub task an run it func (w *Worker) StartSubTask(cfg *config.SubTaskConfig, expectStage pb.Stage) error { w.Lock() diff --git a/tests/ha_cases/run.sh b/tests/ha_cases/run.sh index 1ab8d60802..9689ed2a82 100755 --- a/tests/ha_cases/run.sh +++ b/tests/ha_cases/run.sh @@ -511,7 +511,6 @@ function test_stop_task() { function test_multi_task_reduce_and_restart_worker() { - # here echo "[$(date)] <<<<<< start test_multi_task_reduce_and_restart_worker >>>>>>" test_multi_task_running From 77cc17e9b4df3b8ee8d3bc16c0791b9a700f2a3f Mon Sep 17 00:00:00 2001 From: lance6716 Date: Fri, 12 Mar 2021 14:07:28 +0800 Subject: [PATCH 06/46] dont shadow --- dm/worker/server.go | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/dm/worker/server.go b/dm/worker/server.go index e50565ffbc..d3e05b7b5f 100644 --- a/dm/worker/server.go +++ b/dm/worker/server.go @@ -582,8 +582,8 @@ func (s *Server) startWorker(cfg *config.SourceConfig) error { if cfg.EnableRelay { s.UpdateKeepAliveTTL(s.cfg.RelayKeepAliveTTL) - if err := w.EnableRelay(); err != nil { - return err + if err2 := w.EnableRelay(); err2 != nil { + return err2 } } go w.Start() From b4b1c4b0462914378db90cf7d9f32ff0a225af3c Mon Sep 17 00:00:00 2001 From: lance6716 Date: Fri, 12 Mar 2021 14:25:09 +0800 Subject: [PATCH 07/46] worker: move subtask related function together --- dm/worker/server.go | 39 ++----------------------------- dm/worker/worker.go | 57 +++++++++++++++++++++++++++++++++++++++++---- 2 files changed, 54 insertions(+), 42 deletions(-) diff --git a/dm/worker/server.go b/dm/worker/server.go index d3e05b7b5f..1134674c02 100644 --- a/dm/worker/server.go +++ b/dm/worker/server.go @@ -557,23 +557,6 @@ func (s *Server) startWorker(cfg *config.SourceConfig) error { return terror.ErrWorkerAlreadyStart.Generate() } - // we get the newest subtask stages directly which will omit the subtask stage PUT/DELETE event - // because triggering these events is useless now - subTaskStages, subTaskCfgm, revSubTask, err := ha.GetSubTaskStageConfig(s.etcdClient, cfg.SourceID) - if err != nil { - return err - } - - subTaskCfgs := make([]*config.SubTaskConfig, 0, len(subTaskCfgm)) - for _, subTaskCfg := range subTaskCfgm { - clone := subTaskCfg - if err2 := copyConfigFromSource(&clone, cfg); err2 != nil { - return err2 - } - subTaskCfgs = append(subTaskCfgs, &clone) - } - - log.L().Info("starting to handle mysql source", zap.String("sourceCfg", cfg.String()), zap.Reflect("subTasks", subTaskCfgs)) w, err := NewWorker(cfg, s.etcdClient, s.cfg.Name) if err != nil { return err @@ -596,27 +579,9 @@ func (s *Server) startWorker(cfg *config.SourceConfig) error { return terror.ErrWorkerNoStart } - for _, subTaskCfg := range subTaskCfgs { - expectStage := subTaskStages[subTaskCfg.Name] - if expectStage.IsDeleted { - continue - } - log.L().Info("start to create subtask", zap.String("sourceID", subTaskCfg.SourceID), zap.String("task", subTaskCfg.Name)) - if err := w.StartSubTask(subTaskCfg, expectStage.Expect); err != nil { - return err - } - } - - w.wg.Add(1) - go func() { - defer w.wg.Done() - // TODO: handle fatal error from observeSubtaskStage - //nolint:errcheck - w.observeSubtaskStage(w.ctx, s.etcdClient, revSubTask) - }() - + err = w.EnableHandleSubtasks() log.L().Info("started to handle mysql source", zap.String("sourceCfg", cfg.String())) - return nil + return err } func makeCommonWorkerResponse(reqErr error) *pb.CommonWorkerResponse { diff --git a/dm/worker/worker.go b/dm/worker/worker.go index fe307ff4c8..d6049b88e3 100644 --- a/dm/worker/worker.go +++ b/dm/worker/worker.go @@ -177,11 +177,8 @@ func (w *Worker) EnableRelay() error { if err != nil { return err } - for k, subTaskCfg := range subTaskCfgs { - if err2 := copyConfigFromSource(&subTaskCfg, w.cfg); err2 != nil { - return err2 - } - subTaskCfgs[k] = subTaskCfg + if err = copyConfigFromSourceForEach(subTaskCfgs, w.cfg); err != nil { + return err } dctx, dcancel := context.WithTimeout(w.etcdClient.Ctx(), time.Duration(len(subTaskCfgs))*3*time.Second) @@ -242,6 +239,45 @@ func (w *Worker) EnableRelay() error { return nil } +// EnableHandleSubtasks enables the functionality of start/watch/handle subtasks +func (w *Worker) EnableHandleSubtasks() error { + // we get the newest subtask stages directly which will omit the subtask stage PUT/DELETE event + // because triggering these events is useless now + subTaskStages, subTaskCfgM, revSubTask, err := ha.GetSubTaskStageConfig(w.etcdClient, w.cfg.SourceID) + if err != nil { + return err + } + + if err = copyConfigFromSourceForEach(subTaskCfgM, w.cfg); err != nil { + return err + } + + log.L().Info("starting to handle mysql source", zap.String("sourceCfg", w.cfg.String()), zap.Any("subTasks", subTaskCfgM)) + + for _, subTaskCfg := range subTaskCfgM { + expectStage := subTaskStages[subTaskCfg.Name] + if expectStage.IsDeleted { + continue + } + log.L().Info("start to create subtask", zap.String("sourceID", subTaskCfg.SourceID), zap.String("task", subTaskCfg.Name)) + // for range of a map will use a same value-address, so we'd better not pass value-address to other function + clone := subTaskCfg + if err := w.StartSubTask(&clone, expectStage.Expect); err != nil { + return err + } + } + + w.wg.Add(1) + go func() { + defer w.wg.Done() + // TODO: handle fatal error from observeSubtaskStage + //nolint:errcheck + w.observeSubtaskStage(w.ctx, w.etcdClient, revSubTask) + }() + + return nil +} + // StartSubTask creates a sub task an run it func (w *Worker) StartSubTask(cfg *config.SubTaskConfig, expectStage pb.Stage) error { w.Lock() @@ -715,6 +751,17 @@ func copyConfigFromSource(cfg *config.SubTaskConfig, sourceCfg *config.SourceCon return nil } +// copyConfigFromSourceForEach do copyConfigFromSource for each value in subTaskCfgM and change subTaskCfgM in-place +func copyConfigFromSourceForEach(subTaskCfgM map[string]config.SubTaskConfig, sourceCfg *config.SourceConfig) error { + for k, subTaskCfg := range subTaskCfgM { + if err2 := copyConfigFromSource(&subTaskCfg, sourceCfg); err2 != nil { + return err2 + } + subTaskCfgM[k] = subTaskCfg + } + return nil +} + // getAllSubTaskStatus returns all subtask status of this worker, note the field // in subtask status is not completed, only includes `Name`, `Stage` and `Result` now func (w *Worker) getAllSubTaskStatus() map[string]*pb.SubTaskStatus { From 55d27d4d0e6fcb3a558d8c6b767889f618ea8d7a Mon Sep 17 00:00:00 2001 From: lance6716 Date: Fri, 12 Mar 2021 19:41:08 +0800 Subject: [PATCH 08/46] worker: enable relay and handling subtask on demand --- dm/worker/relay.go | 12 +-- dm/worker/relay_test.go | 20 ++--- dm/worker/server.go | 3 +- dm/worker/server_test.go | 6 +- dm/worker/task_checker.go | 8 +- dm/worker/task_checker_test.go | 4 +- dm/worker/worker.go | 140 ++++++++++++++++++++++++--------- dm/worker/worker_test.go | 14 ++-- go.mod | 2 + go.sum | 4 +- relay/metrics.go | 1 + 11 files changed, 142 insertions(+), 72 deletions(-) diff --git a/dm/worker/relay.go b/dm/worker/relay.go index ebd9ea37ae..10545579f8 100644 --- a/dm/worker/relay.go +++ b/dm/worker/relay.go @@ -70,7 +70,7 @@ type realRelayHolder struct { l log.Logger - closed sync2.AtomicInt32 + closed sync2.AtomicBool stage pb.Stage result *pb.ProcessResult // the process result, nil when is processing } @@ -85,13 +85,13 @@ func NewRealRelayHolder(sourceCfg *config.SourceConfig) RelayHolder { relay: relay.NewRelay(cfg), l: log.With(zap.String("component", "relay holder")), } - h.closed.Set(closedTrue) + h.closed.Set(true) return h } // Init initializes the holder func (h *realRelayHolder) Init(interceptors []purger.PurgeInterceptor) (purger.Purger, error) { - h.closed.Set(closedFalse) + h.closed.Set(false) // initial relay purger operators := []purger.RelayOperator{ @@ -120,7 +120,7 @@ func (h *realRelayHolder) Start() { // Close closes the holder func (h *realRelayHolder) Close() { - if !h.closed.CompareAndSwap(closedFalse, closedTrue) { + if !h.closed.CompareAndSwap(false, true) { return } @@ -153,7 +153,7 @@ func (h *realRelayHolder) run() { // Status returns relay unit's status func (h *realRelayHolder) Status(ctx context.Context) *pb.RelayStatus { - if h.closed.Get() == closedTrue || h.relay.IsClosed() { + if h.closed.Get() || h.relay.IsClosed() { return &pb.RelayStatus{ Stage: pb.Stage_Stopped, } @@ -168,7 +168,7 @@ func (h *realRelayHolder) Status(ctx context.Context) *pb.RelayStatus { // Error returns relay unit's status func (h *realRelayHolder) Error() *pb.RelayError { - if h.closed.Get() == closedTrue || h.relay.IsClosed() { + if h.closed.Get() || h.relay.IsClosed() { return &pb.RelayError{ Msg: "relay stopped", } diff --git a/dm/worker/relay_test.go b/dm/worker/relay_test.go index 9fc6480a3c..c55f3310fb 100644 --- a/dm/worker/relay_test.go +++ b/dm/worker/relay_test.go @@ -175,13 +175,13 @@ func (t *testRelay) testInit(c *C, holder *realRelayHolder) { func (t *testRelay) testStart(c *C, holder *realRelayHolder) { c.Assert(holder.Stage(), Equals, pb.Stage_New) - c.Assert(holder.closed.Get(), Equals, closedFalse) + c.Assert(holder.closed.Get(), IsFalse) c.Assert(holder.Result(), IsNil) holder.Start() c.Assert(waitRelayStage(holder, pb.Stage_Running, 10), IsTrue) c.Assert(holder.Result(), IsNil) - c.Assert(holder.closed.Get(), Equals, closedFalse) + c.Assert(holder.closed.Get(), IsFalse) // test status status := holder.Status(context.Background()) @@ -193,13 +193,13 @@ func (t *testRelay) testStart(c *C, holder *realRelayHolder) { // test update and pause -> resume t.testUpdate(c, holder) c.Assert(holder.Stage(), Equals, pb.Stage_Paused) - c.Assert(holder.closed.Get(), Equals, closedFalse) + c.Assert(holder.closed.Get(), IsFalse) err := holder.Operate(context.Background(), pb.RelayOp_ResumeRelay) c.Assert(err, IsNil) c.Assert(waitRelayStage(holder, pb.Stage_Running, 10), IsTrue) c.Assert(holder.Result(), IsNil) - c.Assert(holder.closed.Get(), Equals, closedFalse) + c.Assert(holder.closed.Get(), IsFalse) } func (t *testRelay) testClose(c *C, holder *realRelayHolder) { @@ -217,12 +217,12 @@ func (t *testRelay) testClose(c *C, holder *realRelayHolder) { holder.Close() c.Assert(waitRelayStage(holder, pb.Stage_Paused, 10), IsTrue) c.Assert(holder.Result(), DeepEquals, processResult) - c.Assert(holder.closed.Get(), Equals, closedTrue) + c.Assert(holder.closed.Get(), IsTrue) holder.Close() c.Assert(holder.Stage(), Equals, pb.Stage_Paused) c.Assert(holder.Result(), DeepEquals, processResult) - c.Assert(holder.closed.Get(), Equals, closedTrue) + c.Assert(holder.closed.Get(), IsTrue) // todo: very strange, and can't resume status := holder.Status(context.Background()) @@ -237,7 +237,7 @@ func (t *testRelay) testPauseAndResume(c *C, holder *realRelayHolder) { err := holder.Operate(context.Background(), pb.RelayOp_PauseRelay) c.Assert(err, IsNil) c.Assert(holder.Stage(), Equals, pb.Stage_Paused) - c.Assert(holder.closed.Get(), Equals, closedFalse) + c.Assert(holder.closed.Get(), IsFalse) err = holder.pauseRelay(context.Background(), pb.RelayOp_PauseRelay) c.Assert(err, ErrorMatches, ".*current stage is Paused.*") @@ -253,7 +253,7 @@ func (t *testRelay) testPauseAndResume(c *C, holder *realRelayHolder) { c.Assert(err, IsNil) c.Assert(waitRelayStage(holder, pb.Stage_Running, 10), IsTrue) c.Assert(holder.Result(), IsNil) - c.Assert(holder.closed.Get(), Equals, closedFalse) + c.Assert(holder.closed.Get(), IsFalse) err = holder.Operate(context.Background(), pb.RelayOp_ResumeRelay) c.Assert(err, ErrorMatches, ".*current stage is Running.*") @@ -281,7 +281,7 @@ func (t *testRelay) testUpdate(c *C, holder *realRelayHolder) { originStage := holder.Stage() c.Assert(holder.Update(context.Background(), cfg), IsNil) c.Assert(waitRelayStage(holder, originStage, 10), IsTrue) - c.Assert(holder.closed.Get(), Equals, closedFalse) + c.Assert(holder.closed.Get(), IsFalse) r, ok := holder.relay.(*DummyRelay) c.Assert(ok, IsTrue) @@ -296,7 +296,7 @@ func (t *testRelay) testStop(c *C, holder *realRelayHolder) { err := holder.Operate(context.Background(), pb.RelayOp_StopRelay) c.Assert(err, IsNil) c.Assert(holder.Stage(), Equals, pb.Stage_Stopped) - c.Assert(holder.closed.Get(), Equals, closedTrue) + c.Assert(holder.closed.Get(), IsTrue) err = holder.Operate(context.Background(), pb.RelayOp_StopRelay) c.Assert(err, ErrorMatches, ".*current stage is already stopped.*") diff --git a/dm/worker/server.go b/dm/worker/server.go index 1134674c02..588d40873f 100644 --- a/dm/worker/server.go +++ b/dm/worker/server.go @@ -148,6 +148,7 @@ 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 @@ -572,7 +573,7 @@ func (s *Server) startWorker(cfg *config.SourceConfig) error { go w.Start() isStarted := utils.WaitSomething(50, 100*time.Millisecond, func() bool { - return w.closed.Get() == closedFalse + return !w.closed.Get() }) if !isStarted { // TODO: add more mechanism to wait or un-bound the source diff --git a/dm/worker/server_test.go b/dm/worker/server_test.go index a040770a9f..142a8c0523 100644 --- a/dm/worker/server_test.go +++ b/dm/worker/server_test.go @@ -478,20 +478,20 @@ func (t *testServer) testOperateWorker(c *C, s *Server, dir string, start bool) // worker should be started and without error c.Assert(utils.WaitSomething(30, 100*time.Millisecond, func() bool { w := s.getWorker(true) - return w != nil && w.closed.Get() == closedFalse + return w != nil && !w.closed.Get() }), IsTrue) c.Assert(s.getSourceStatus(true).Result, IsNil) } else { // worker should be started before stopped w := s.getWorker(true) c.Assert(w, NotNil) - c.Assert(w.closed.Get() == closedFalse, IsTrue) + 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 c.Assert(utils.WaitSomething(30, 100*time.Millisecond, func() bool { currentWorker := s.getWorker(true) - return currentWorker == nil && w.closed.Get() == closedTrue + return currentWorker == nil && w.closed.Get() }), IsTrue) c.Assert(s.getSourceStatus(true).Result, IsNil) } diff --git a/dm/worker/task_checker.go b/dm/worker/task_checker.go index 69c150ea4d..c4c81d64de 100644 --- a/dm/worker/task_checker.go +++ b/dm/worker/task_checker.go @@ -141,7 +141,7 @@ type realTaskStatusChecker struct { ctx context.Context cancel context.CancelFunc wg sync.WaitGroup - closed sync2.AtomicInt32 + closed sync2.AtomicBool cfg config.CheckerConfig l log.Logger @@ -157,7 +157,7 @@ func NewRealTaskStatusChecker(cfg config.CheckerConfig, w *Worker) TaskStatusChe w: w, bc: newBackoffController(), } - tsc.closed.Set(closedTrue) + tsc.closed.Set(true) return tsc } @@ -180,7 +180,7 @@ func (tsc *realTaskStatusChecker) Start() { // Close implements TaskStatusChecker.Close func (tsc *realTaskStatusChecker) Close() { - if !tsc.closed.CompareAndSwap(closedFalse, closedTrue) { + if !tsc.closed.CompareAndSwap(false, true) { return } @@ -193,7 +193,7 @@ func (tsc *realTaskStatusChecker) Close() { func (tsc *realTaskStatusChecker) run() { // keep running until canceled in `Close`. tsc.ctx, tsc.cancel = context.WithCancel(context.Background()) - tsc.closed.Set(closedFalse) + tsc.closed.Set(false) failpoint.Inject("TaskCheckInterval", func(val failpoint.Value) { interval, err := time.ParseDuration(val.(string)) diff --git a/dm/worker/task_checker_test.go b/dm/worker/task_checker_test.go index fb12fb86ac..bb2d95f5a0 100644 --- a/dm/worker/task_checker_test.go +++ b/dm/worker/task_checker_test.go @@ -91,7 +91,7 @@ func (s *testTaskCheckerSuite) TestCheck(c *check.C) { cfg.MetaDir = dir w, err := NewWorker(&cfg, nil, "") c.Assert(err, check.IsNil) - w.closed.Set(closedFalse) + w.closed.Set(false) tsc := NewRealTaskStatusChecker(config.CheckerConfig{ CheckEnable: true, @@ -208,7 +208,7 @@ func (s *testTaskCheckerSuite) TestCheckTaskIndependent(c *check.C) { cfg.MetaDir = dir w, err := NewWorker(&cfg, nil, "") c.Assert(err, check.IsNil) - w.closed.Set(closedFalse) + w.closed.Set(false) tsc := NewRealTaskStatusChecker(config.CheckerConfig{ CheckEnable: true, diff --git a/dm/worker/worker.go b/dm/worker/worker.go index d6049b88e3..decc6c18b9 100644 --- a/dm/worker/worker.go +++ b/dm/worker/worker.go @@ -37,18 +37,14 @@ import ( "github.com/pingcap/dm/relay/purger" ) -var ( - closedFalse int32 - closedTrue int32 = 1 -) - // Worker manages sub tasks and process units for data migration type Worker struct { - // ensure no other operation can be done when closing (we can use `WatGroup`/`Context` to archive this) + // ensure no other operation can be done when closing (we can use `WatGroup`/`Context` to archive this)/ + // seems only guarding subTaskHolder sync.RWMutex wg sync.WaitGroup - closed sync2.AtomicInt32 + closed sync2.AtomicBool // context created when Worker created, and canceled when closing ctx context.Context @@ -57,10 +53,20 @@ type Worker struct { cfg *config.SourceConfig l log.Logger - subTaskHolder *subTaskHolder - - relayHolder RelayHolder - relayPurger purger.Purger + // subtask functionality + subTaskEnabled sync2.AtomicBool + subTaskCtx context.Context + subTaskCancel context.CancelFunc + subTaskWg sync.WaitGroup + subTaskHolder *subTaskHolder + + // relay functionality + relayEnabled sync2.AtomicBool + relayCtx context.Context + relayCancel context.CancelFunc + relayWg sync.WaitGroup + relayHolder RelayHolder + relayPurger purger.Purger taskStatusChecker TaskStatusChecker @@ -81,7 +87,9 @@ func NewWorker(cfg *config.SourceConfig, etcdClient *clientv3.Client, name strin } // keep running until canceled in `Close`. w.ctx, w.cancel = context.WithCancel(context.Background()) - w.closed.Set(closedTrue) + w.closed.Set(false) + w.subTaskEnabled.Set(false) + w.relayEnabled.Set(false) defer func(w2 *Worker) { if err != nil { // when err != nil, `w` will become nil in this func, so we pass `w` in defer. @@ -121,7 +129,7 @@ func (w *Worker) Start() { w.l.Info("start running") ticker := time.NewTicker(5 * time.Second) - w.closed.Set(closedFalse) + w.closed.Set(false) defer ticker.Stop() for { select { @@ -136,7 +144,7 @@ func (w *Worker) Start() { // Close stops working and releases resources func (w *Worker) Close() { - if w.closed.Get() == closedTrue { + if w.closed.Get() { w.l.Warn("already closed") return } @@ -166,14 +174,21 @@ func (w *Worker) Close() { w.taskStatusChecker.Close() } - w.closed.Set(closedTrue) + w.closed.Set(true) w.l.Info("Stop worker") } // EnableRelay enables the functionality of start/watch/handle relay -func (w *Worker) EnableRelay() error { +func (w *Worker) EnableRelay() (err error) { + if w.relayEnabled.Get() { + w.l.Warn("already enabled relay") + return nil + } + w.relayCtx, w.relayCancel = context.WithCancel(w.ctx) + // 1. adjust relay starting position, to the earliest of subtasks - _, subTaskCfgs, _, err := ha.GetSubTaskStageConfig(w.etcdClient, w.cfg.SourceID) + var subTaskCfgs map[string]config.SubTaskConfig + _, subTaskCfgs, _, err = ha.GetSubTaskStageConfig(w.etcdClient, w.cfg.SourceID) if err != nil { return err } @@ -229,18 +244,52 @@ func (w *Worker) EnableRelay() error { } // 4. watch relay stage - w.wg.Add(1) + w.relayWg.Add(1) go func() { - defer w.wg.Done() + defer w.relayWg.Done() // TODO: handle fatal error from observeRelayStage //nolint:errcheck - w.observeRelayStage(w.ctx, w.etcdClient, revRelay) + w.observeRelayStage(w.relayCtx, w.etcdClient, revRelay) }() + + w.relayEnabled.Set(true) return nil } +// DisableRelay disables the functionality of start/watch/handle relay +func (w *Worker) DisableRelay() { + // TODO: use CAS? + if !w.relayEnabled.Get() { + w.l.Warn("already disabled relay") + return + } + + // TODO: check if running subtask need accessing relay + + w.relayCancel() + w.relayWg.Wait() + + if w.relayHolder != nil { + w.relayHolder.Close() + w.relayHolder = nil + } + + if w.relayPurger != nil { + w.relayPurger.Close() + w.relayPurger = nil + } + + w.relayEnabled.Set(false) +} + // EnableHandleSubtasks enables the functionality of start/watch/handle subtasks func (w *Worker) EnableHandleSubtasks() error { + if w.subTaskEnabled.Get() { + w.l.Warn("already enabled handling subtasks") + return nil + } + w.subTaskCtx, w.subTaskCancel = context.WithCancel(w.ctx) + // we get the newest subtask stages directly which will omit the subtask stage PUT/DELETE event // because triggering these events is useless now subTaskStages, subTaskCfgM, revSubTask, err := ha.GetSubTaskStageConfig(w.etcdClient, w.cfg.SourceID) @@ -260,24 +309,45 @@ func (w *Worker) EnableHandleSubtasks() error { continue } log.L().Info("start to create subtask", zap.String("sourceID", subTaskCfg.SourceID), zap.String("task", subTaskCfg.Name)) - // for range of a map will use a same value-address, so we'd better not pass value-address to other function + // "for range" of a map will use same value address, so we'd better not pass value address to other function clone := subTaskCfg - if err := w.StartSubTask(&clone, expectStage.Expect); err != nil { - return err + if err2 := w.StartSubTask(&clone, expectStage.Expect); err2 != nil { + w.subTaskHolder.closeAllSubTasks() + return err2 } } - w.wg.Add(1) + w.subTaskWg.Add(1) go func() { - defer w.wg.Done() + defer w.subTaskWg.Done() // TODO: handle fatal error from observeSubtaskStage //nolint:errcheck - w.observeSubtaskStage(w.ctx, w.etcdClient, revSubTask) + w.observeSubtaskStage(w.subTaskCtx, w.etcdClient, revSubTask) }() + w.subTaskEnabled.Set(true) return nil } +// DisableHandleSubtasks disables the functionality of start/watch/handle subtasks +func (w *Worker) DisableHandleSubtasks() { + // TODO: use CAS? + if !w.subTaskEnabled.Get() { + w.l.Warn("already disabled handling subtasks") + return + } + + w.subTaskCancel() + w.subTaskWg.Wait() + + w.Lock() + defer w.Unlock() + + // close all sub tasks + w.subTaskHolder.closeAllSubTasks() + w.subTaskEnabled.Set(false) +} + // StartSubTask creates a sub task an run it func (w *Worker) StartSubTask(cfg *config.SubTaskConfig, expectStage pb.Stage) error { w.Lock() @@ -293,7 +363,7 @@ func (w *Worker) StartSubTask(cfg *config.SubTaskConfig, expectStage pb.Stage) e // the unique of subtask should be assured by etcd st := NewSubTask(cfg, w.etcdClient) w.subTaskHolder.recordSubTask(st) - if w.closed.Get() == closedTrue { + if w.closed.Get() { st.fail(terror.ErrWorkerAlreadyClosed.Generate()) return nil } @@ -321,7 +391,7 @@ func (w *Worker) UpdateSubTask(cfg *config.SubTaskConfig) error { w.Lock() defer w.Unlock() - if w.closed.Get() == closedTrue { + if w.closed.Get() { return terror.ErrWorkerAlreadyClosed.Generate() } @@ -339,7 +409,7 @@ func (w *Worker) OperateSubTask(name string, op pb.TaskOp) error { w.Lock() defer w.Unlock() - if w.closed.Get() == closedTrue { + if w.closed.Get() { return terror.ErrWorkerAlreadyClosed.Generate() } @@ -375,7 +445,7 @@ func (w *Worker) QueryStatus(ctx context.Context, name string) []*pb.SubTaskStat w.RLock() defer w.RUnlock() - if w.closed.Get() == closedTrue { + if w.closed.Get() { w.l.Warn("querying status from a closed worker") return nil } @@ -654,7 +724,7 @@ func (w *Worker) operateRelayStage(ctx context.Context, stage ha.Stage) (string, // OperateRelay operates relay unit func (w *Worker) operateRelay(ctx context.Context, op pb.RelayOp) error { - if w.closed.Get() == closedTrue { + if w.closed.Get() { return terror.ErrWorkerAlreadyClosed.Generate() } @@ -668,7 +738,7 @@ func (w *Worker) operateRelay(ctx context.Context, op pb.RelayOp) error { // PurgeRelay purges relay log files func (w *Worker) PurgeRelay(ctx context.Context, req *pb.PurgeRelayRequest) error { - if w.closed.Get() == closedTrue { + if w.closed.Get() { return terror.ErrWorkerAlreadyClosed.Generate() } @@ -682,7 +752,7 @@ func (w *Worker) PurgeRelay(ctx context.Context, req *pb.PurgeRelayRequest) erro // ForbidPurge implements PurgeInterceptor.ForbidPurge func (w *Worker) ForbidPurge() (bool, string) { - if w.closed.Get() == closedTrue { + if w.closed.Get() { return false, "" } @@ -703,7 +773,7 @@ func (w *Worker) OperateSchema(ctx context.Context, req *pb.OperateWorkerSchemaR w.Lock() defer w.Unlock() - if w.closed.Get() == closedTrue { + if w.closed.Get() { return "", terror.ErrWorkerAlreadyClosed.Generate() } @@ -784,7 +854,7 @@ func (w *Worker) HandleError(ctx context.Context, req *pb.HandleWorkerErrorReque w.Lock() defer w.Unlock() - if w.closed.Get() == closedTrue { + if w.closed.Get() { return terror.ErrWorkerAlreadyClosed.Generate() } diff --git a/dm/worker/worker_test.go b/dm/worker/worker_test.go index 35226e18cc..6c6bb2dc7b 100644 --- a/dm/worker/worker_test.go +++ b/dm/worker/worker_test.go @@ -77,19 +77,15 @@ func (t *testServer) testWorker(c *C) { w, err = NewWorker(&cfg, etcdCli, "") c.Assert(err, IsNil) c.Assert(w.StatusJSON(context.Background(), ""), HasLen, emptyWorkerStatusInfoJSONLength) - //c.Assert(w.closed.Get(), Equals, closedFalse) - //go func() { - // w.Start() - //}() // close twice w.Close() - c.Assert(w.closed.Get(), Equals, closedTrue) + c.Assert(w.closed.Get(), IsTrue) c.Assert(w.subTaskHolder.getAllSubTasks(), HasLen, 0) w.Close() - c.Assert(w.closed.Get(), Equals, closedTrue) + c.Assert(w.closed.Get(), IsTrue) c.Assert(w.subTaskHolder.getAllSubTasks(), HasLen, 0) - c.Assert(w.closed.Get(), Equals, closedTrue) + c.Assert(w.closed.Get(), IsTrue) c.Assert(w.StartSubTask(&config.SubTaskConfig{ Name: "testStartTask", @@ -278,7 +274,7 @@ func (t *testWorkerEtcdCompact) TestWatchSubtaskStageEtcdCompact(c *C) { w.Start() }() c.Assert(utils.WaitSomething(50, 100*time.Millisecond, func() bool { - return w.closed.Get() == closedFalse + return !w.closed.Get() }), IsTrue) // step 2: Put a subtask config and subtask stage to this source, then delete it subtaskCfg := config.SubTaskConfig{} @@ -393,7 +389,7 @@ func (t *testWorkerEtcdCompact) TestWatchRelayStageEtcdCompact(c *C) { w.Start() }() c.Assert(utils.WaitSomething(50, 100*time.Millisecond, func() bool { - return w.closed.Get() == closedFalse + return !w.closed.Get() }), IsTrue) // step 2: Put a relay stage to this source, then delete it // put mysql config into relative etcd key adapter to trigger operation event diff --git a/go.mod b/go.mod index f37b036d89..2381031a73 100644 --- a/go.mod +++ b/go.mod @@ -51,3 +51,5 @@ require ( go 1.13 replace github.com/siddontang/go-mysql v1.1.1-0.20200824131207-0c5789dd0bd3 => github.com/lance6716/go-mysql v1.1.1-0.20210303100354-b0e44c2c5623 + +replace github.com/siddontang/go v0.0.0-20180604090527-bdc77568d726 => github.com/lance6716/go v0.0.0-20210312094856-8a1d496ae7d4 diff --git a/go.sum b/go.sum index de74ec2645..79b37a764a 100644 --- a/go.sum +++ b/go.sum @@ -599,6 +599,8 @@ github.com/kr/text v0.1.0 h1:45sCR5RtlFHMR4UwH9sdQ5TC8v0qDQCHnXt+kaKSTVE= github.com/kr/text v0.1.0/go.mod h1:4Jbv+DJW3UT/LiOwJeYQe1efqtUx/iVham/4vfdArNI= github.com/kr/text v0.2.0 h1:5Nx0Ya0ZqY2ygV366QzturHI13Jq95ApcVaJBhpS+AY= github.com/kr/text v0.2.0/go.mod h1:eLer722TekiGuMkidMxC/pM04lWEeraHUUmBw8l2grE= +github.com/lance6716/go v0.0.0-20210312094856-8a1d496ae7d4 h1:19RYJOIPZqTJxJqwJCDdxFaav7Wp9S+gaKUc6hd+fXE= +github.com/lance6716/go v0.0.0-20210312094856-8a1d496ae7d4/go.mod h1:FcZnptvZ/cMvXwOfn5vbaV7T2rv607STeXC3LTYGYp8= github.com/lance6716/go-mysql v1.1.1-0.20210303100354-b0e44c2c5623 h1:HesfLUZvjUaluKJzKetJ6odasdH/WQ/Tc8UO1oAs0WI= github.com/lance6716/go-mysql v1.1.1-0.20210303100354-b0e44c2c5623/go.mod h1:+W4RCzesQDI11HvIkaDjS8yM36SpAnGNQ7jmTLn5BnU= github.com/leodido/go-urn v1.1.0/go.mod h1:+cyI34gQWZcE1eQU7NVgKkkzdXDQHr1dBMtdAPozLkw= @@ -1065,8 +1067,6 @@ github.com/shurcooL/vfsgen v0.0.0-20181020040650-a97a25d856ca h1:3fECS8atRjByiji 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 v0.0.0-20180604090527-bdc77568d726 h1:xT+JlYxNGqyT+XcU8iUrN18JYed2TvG9yN5ULG2jATM= -github.com/siddontang/go v0.0.0-20180604090527-bdc77568d726/go.mod h1:3yhqj7WBBfRhbBlzyOC3gUxftwsU0u8gqevxwIHQpMw= github.com/siddontang/go-log v0.0.0-20180807004314-8d05993dda07 h1:oI+RNwuC9jF2g2lP0u0cVEEZrc/AYBCuFdvwrLWM/6Q= 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= diff --git a/relay/metrics.go b/relay/metrics.go index f58be5d27a..e68797e27c 100644 --- a/relay/metrics.go +++ b/relay/metrics.go @@ -149,6 +149,7 @@ func RegisterMetrics(registry *prometheus.Registry) { registry.MustRegister(relayExitWithErrorCounter) } +// TODO: add a context to stop it func reportRelayLogSpaceInBackground(dirpath string) error { if len(dirpath) == 0 { return terror.ErrRelayLogDirpathEmpty.Generate() From 890a79a4efaf416c41435f0e082a67576bfee824 Mon Sep 17 00:00:00 2001 From: lance6716 Date: Fri, 12 Mar 2021 21:17:02 +0800 Subject: [PATCH 09/46] fix mistakenly set `w.closed` --- dm/worker/worker.go | 10 ++++++---- 1 file changed, 6 insertions(+), 4 deletions(-) diff --git a/dm/worker/worker.go b/dm/worker/worker.go index decc6c18b9..4a44c52d42 100644 --- a/dm/worker/worker.go +++ b/dm/worker/worker.go @@ -87,7 +87,7 @@ func NewWorker(cfg *config.SourceConfig, etcdClient *clientv3.Client, name strin } // keep running until canceled in `Close`. w.ctx, w.cancel = context.WithCancel(context.Background()) - w.closed.Set(false) + w.closed.Set(true) w.subTaskEnabled.Set(false) w.relayEnabled.Set(false) @@ -238,7 +238,7 @@ func (w *Worker) EnableRelay() (err error) { } startImmediately := !relayStage.IsDeleted && relayStage.Expect == pb.Stage_Running if startImmediately { - log.L().Info("relay is started") + w.l.Info("relay is started") w.relayHolder.Start() w.relayPurger.Start() } @@ -253,6 +253,7 @@ func (w *Worker) EnableRelay() (err error) { }() w.relayEnabled.Set(true) + w.l.Info("relay enabled") return nil } @@ -301,14 +302,14 @@ func (w *Worker) EnableHandleSubtasks() error { return err } - log.L().Info("starting to handle mysql source", zap.String("sourceCfg", w.cfg.String()), zap.Any("subTasks", subTaskCfgM)) + w.l.Info("starting to handle mysql source", zap.String("sourceCfg", w.cfg.String()), zap.Any("subTasks", subTaskCfgM)) for _, subTaskCfg := range subTaskCfgM { expectStage := subTaskStages[subTaskCfg.Name] if expectStage.IsDeleted { continue } - log.L().Info("start to create subtask", zap.String("sourceID", subTaskCfg.SourceID), zap.String("task", subTaskCfg.Name)) + w.l.Info("start to create subtask", zap.String("sourceID", subTaskCfg.SourceID), zap.String("task", subTaskCfg.Name)) // "for range" of a map will use same value address, so we'd better not pass value address to other function clone := subTaskCfg if err2 := w.StartSubTask(&clone, expectStage.Expect); err2 != nil { @@ -326,6 +327,7 @@ func (w *Worker) EnableHandleSubtasks() error { }() w.subTaskEnabled.Set(true) + w.l.Info("handling subtask enabled") return nil } From 85e6ab98c3d81f2e8fddc5159892de54cdbb39f1 Mon Sep 17 00:00:00 2001 From: lance6716 Date: Sat, 13 Mar 2021 16:42:12 +0800 Subject: [PATCH 10/46] split call to EnabledXXX --- _utils/terror_gen/errors_release.txt | 2 +- dm/worker/join.go | 7 ++- dm/worker/server.go | 90 +++++++++++++++++----------- dm/worker/server_test.go | 4 +- dm/worker/worker_test.go | 6 +- errors.toml | 4 +- pkg/terror/error_list.go | 2 +- 7 files changed, 74 insertions(+), 41 deletions(-) diff --git a/_utils/terror_gen/errors_release.txt b/_utils/terror_gen/errors_release.txt index b4e15101ff..2b7c482891 100644 --- a/_utils/terror_gen/errors_release.txt +++ b/_utils/terror_gen/errors_release.txt @@ -442,7 +442,7 @@ ErrWorkerWaitRelayCatchupTimeout,[code=40067:class=dm-worker:scope=internal:leve ErrWorkerRelayIsPurging,[code=40068:class=dm-worker:scope=internal:level=high], "Message: relay log purger is purging, cannot start sub task %s, Workaround: Please try again later." ErrWorkerHostPortNotValid,[code=40069:class=dm-worker:scope=internal:level=high], "Message: host:port '%s' not valid, Workaround: Please check configs in worker configuration file." ErrWorkerNoStart,[code=40070:class=dm-worker:scope=internal:level=high], "Message: no mysql source is being handled in the worker" -ErrWorkerAlreadyStart,[code=40071:class=dm-worker:scope=internal:level=high], "Message: mysql source handler worker already started" +ErrWorkerAlreadyStart,[code=40071:class=dm-worker:scope=internal:level=high], "Message: mysql source worker %s has already started with source %s, but get a request with source %s, Workaround: Please try restart this DM-worker" ErrWorkerSourceNotMatch,[code=40072:class=dm-worker:scope=internal:level=high], "Message: source of request does not match with source in worker" ErrWorkerWaitRelayCatchupGTID,[code=40078:class=dm-worker:scope=internal:level=high], "Message: cannot compare gtid between loader and relay, loader gtid: %s, relay gtid: %s" ErrWorkerFailToGetSubtaskConfigFromEtcd,[code=40073:class=dm-worker:scope=internal:level=medium], "Message: there is no relative subtask config for task %s in etcd" diff --git a/dm/worker/join.go b/dm/worker/join.go index f1bd067500..b6d366df20 100644 --- a/dm/worker/join.go +++ b/dm/worker/join.go @@ -16,6 +16,7 @@ package worker import ( "context" "strings" + "sync" "time" "github.com/pingcap/failpoint" @@ -119,9 +120,13 @@ 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 -// this function should not be concurrently called 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 diff --git a/dm/worker/server.go b/dm/worker/server.go index 588d40873f..5a8a797d1b 100644 --- a/dm/worker/server.go +++ b/dm/worker/server.go @@ -115,20 +115,6 @@ func (s *Server) Start() error { } s.setWorker(nil, true) - 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)) - err = s.startWorker(&sourceCfg) - s.setSourceStatus(bound.Source, err, true) - if err != 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 err - } - } s.wg.Add(1) go func() { @@ -144,6 +130,31 @@ func (s *Server) Start() error { s.startKeepAlive() + 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) + 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 + } + w.l.Info("started to handle mysql source", zap.String("sourceCfg", sourceCfg.String())) + } + s.wg.Add(1) go func(ctx context.Context) { defer s.wg.Done() @@ -306,10 +317,19 @@ func (s *Server) observeSourceBound(ctx context.Context, rev int64) error { log.L().Error("fail to stop worker", zap.Error(err)) return err // return if failed to stop the worker. } - err1 = s.startWorker(&cfg) - s.setSourceStatus(bound.Source, err1, true) - if err1 != nil { - log.L().Error("fail to operate sourceBound on worker", zap.Stringer("bound", bound), zap.Bool("is deleted", bound.IsDeleted), zap.Error(err1)) + 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 = w.EnableHandleSubtasks() + } + s.setSourceStatus(bound.Source, err2, true) + 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)) } } } @@ -416,6 +436,7 @@ func (s *Server) stopWorker(sourceID string) error { s.Unlock() return terror.ErrWorkerSourceNotMatch } + // TODO: and when disable relay s.UpdateKeepAliveTTL(s.cfg.KeepAliveTTL) s.setWorker(nil, false) s.Unlock() @@ -471,7 +492,17 @@ func (s *Server) operateSourceBound(bound ha.SourceBound) error { if !ok { return terror.ErrWorkerFailToGetSourceConfigFromEtcd.Generate(bound.Source) } - return s.startWorker(&sourceCfg) + w, err := s.getOrStartWorker(&sourceCfg) + if err != nil { + return err + } + if sourceCfg.EnableRelay { + s.UpdateKeepAliveTTL(s.cfg.RelayKeepAliveTTL) + if err = w.EnableRelay(); err != nil { + return err + } + } + return w.EnableHandleSubtasks() } // QueryStatus implements WorkerServer.QueryStatus @@ -547,29 +578,23 @@ func (s *Server) OperateSchema(ctx context.Context, req *pb.OperateWorkerSchemaR }, nil } -func (s *Server) startWorker(cfg *config.SourceConfig) error { +func (s *Server) getOrStartWorker(cfg *config.SourceConfig) (*Worker, error) { 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)) - return nil + return w, nil } - return terror.ErrWorkerAlreadyStart.Generate() + return nil, terror.ErrWorkerAlreadyStart.Generate(w.name, w.cfg.SourceID, cfg.SourceID) } w, err := NewWorker(cfg, s.etcdClient, s.cfg.Name) if err != nil { - return err + return nil, err } s.setWorker(w, false) - if cfg.EnableRelay { - s.UpdateKeepAliveTTL(s.cfg.RelayKeepAliveTTL) - if err2 := w.EnableRelay(); err2 != nil { - return err2 - } - } go w.Start() isStarted := utils.WaitSomething(50, 100*time.Millisecond, func() bool { @@ -577,12 +602,9 @@ func (s *Server) startWorker(cfg *config.SourceConfig) error { }) if !isStarted { // TODO: add more mechanism to wait or un-bound the source - return terror.ErrWorkerNoStart + return nil, terror.ErrWorkerNoStart } - - err = w.EnableHandleSubtasks() - log.L().Info("started to handle mysql source", zap.String("sourceCfg", cfg.String())) - return err + return w, nil } func makeCommonWorkerResponse(reqErr error) *pb.CommonWorkerResponse { diff --git a/dm/worker/server_test.go b/dm/worker/server_test.go index 142a8c0523..71b5bdab40 100644 --- a/dm/worker/server_test.go +++ b/dm/worker/server_test.go @@ -389,7 +389,9 @@ func (t *testServer) TestWatchSourceBoundEtcdCompact(c *C) { rev, err := ha.DeleteSourceBound(etcdCli, cfg.Name) c.Assert(err, IsNil) // step 2: start source at this worker - c.Assert(s.startWorker(&sourceCfg), IsNil) + w, err := s.getOrStartWorker(&sourceCfg) + c.Assert(err, IsNil) + c.Assert(w.EnableHandleSubtasks(), IsNil) // step 3: trigger etcd compaction and check whether we can receive it through watcher _, err = etcdCli.Compact(ctx, rev) c.Assert(err, IsNil) diff --git a/dm/worker/worker_test.go b/dm/worker/worker_test.go index 6c6bb2dc7b..a267ff834f 100644 --- a/dm/worker/worker_test.go +++ b/dm/worker/worker_test.go @@ -172,7 +172,11 @@ func (t *testServer2) TestTaskAutoResume(c *C) { if s.closed.Get() { return false } - c.Assert(s.startWorker(&sourceConfig), IsNil) + w, err2 := s.getOrStartWorker(&sourceConfig) + c.Assert(err2, IsNil) + // we set sourceConfig.EnableRelay = true above + c.Assert(w.EnableRelay(), IsNil) + c.Assert(w.EnableHandleSubtasks(), IsNil) return true }), IsTrue) // start task diff --git a/errors.toml b/errors.toml index 74f9db3bac..481d556563 100644 --- a/errors.toml +++ b/errors.toml @@ -2663,9 +2663,9 @@ workaround = "" tags = ["internal", "high"] [error.DM-dm-worker-40071] -message = "mysql source handler worker already started" +message = "mysql source worker %s has already started with source %s, but get a request with source %s" description = "" -workaround = "" +workaround = "Please try restart this DM-worker" tags = ["internal", "high"] [error.DM-dm-worker-40072] diff --git a/pkg/terror/error_list.go b/pkg/terror/error_list.go index fe3e4035eb..bd1bfcd62c 100644 --- a/pkg/terror/error_list.go +++ b/pkg/terror/error_list.go @@ -1130,7 +1130,7 @@ var ( ErrWorkerRelayIsPurging = New(codeWorkerRelayIsPurging, ClassDMWorker, ScopeInternal, LevelHigh, "relay log purger is purging, cannot start sub task %s", "Please try again later.") ErrWorkerHostPortNotValid = New(codeWorkerHostPortNotValid, ClassDMWorker, ScopeInternal, LevelHigh, "host:port '%s' not valid", "Please check configs in worker configuration file.") ErrWorkerNoStart = New(codeWorkerNoStart, ClassDMWorker, ScopeInternal, LevelHigh, "no mysql source is being handled in the worker", "") - ErrWorkerAlreadyStart = New(codeWorkerAlreadyStarted, ClassDMWorker, ScopeInternal, LevelHigh, "mysql source handler worker already started", "") + ErrWorkerAlreadyStart = New(codeWorkerAlreadyStarted, ClassDMWorker, ScopeInternal, LevelHigh, "mysql source worker %s has already started with source %s, but get a request with source %s", "Please try restart this DM-worker") ErrWorkerSourceNotMatch = New(codeWorkerSourceNotMatch, ClassDMWorker, ScopeInternal, LevelHigh, "source of request does not match with source in worker", "") ErrWorkerWaitRelayCatchupGTID = New(codeWorkerWaitRelayCatchupGTID, ClassDMWorker, ScopeInternal, LevelHigh, "cannot compare gtid between loader and relay, loader gtid: %s, relay gtid: %s", "") From 67890117889d4cbbb2a0d3b1c773f02e123bae8f Mon Sep 17 00:00:00 2001 From: lance6716 Date: Sat, 13 Mar 2021 20:09:42 +0800 Subject: [PATCH 11/46] fix some concurrent accessing --- dm/worker/server.go | 2 +- dm/worker/server_test.go | 1 + dm/worker/subtask.go | 2 +- dm/worker/task_checker.go | 2 +- dm/worker/worker.go | 30 +++++++++++++++++++----------- syncer/syncer.go | 2 ++ 6 files changed, 25 insertions(+), 14 deletions(-) diff --git a/dm/worker/server.go b/dm/worker/server.go index 5a8a797d1b..264decf604 100644 --- a/dm/worker/server.go +++ b/dm/worker/server.go @@ -525,7 +525,7 @@ func (s *Server) QueryStatus(ctx context.Context, req *pb.QueryStatusRequest) (* } resp.SubTaskStatus = w.QueryStatus(ctx, req.Name) - if w.relayHolder != nil { + if w.relayEnabled.Get() { sourceStatus.RelayStatus = w.relayHolder.Status(ctx) } diff --git a/dm/worker/server_test.go b/dm/worker/server_test.go index 71b5bdab40..3c3a9ca7db 100644 --- a/dm/worker/server_test.go +++ b/dm/worker/server_test.go @@ -504,6 +504,7 @@ func (t *testServer) testRetryConnectMaster(c *C, s *Server, ETCD *embed.Etcd, d time.Sleep(6 * time.Second) // When worker server fail to keepalive with etcd, server should close its worker c.Assert(s.getWorker(true), IsNil) + c.Assert(s.getSourceStatus(true).Result, IsNil) ETCD, err := createMockETCD(dir, "http://"+hostName) c.Assert(err, IsNil) diff --git a/dm/worker/subtask.go b/dm/worker/subtask.go index 14ead538bd..649b684529 100644 --- a/dm/worker/subtask.go +++ b/dm/worker/subtask.go @@ -598,7 +598,7 @@ func (st *SubTask) unitTransWaitCondition(subTaskCtx context.Context) error { st.l.Info("wait condition between two units", zap.Stringer("previous unit", pu.Type()), zap.Stringer("unit", cu.Type())) hub := GetConditionHub() - if hub.w.relayHolder == nil { + if !hub.w.relayEnabled.Get() { return nil } diff --git a/dm/worker/task_checker.go b/dm/worker/task_checker.go index c4c81d64de..243b96ce5d 100644 --- a/dm/worker/task_checker.go +++ b/dm/worker/task_checker.go @@ -406,7 +406,7 @@ func (tsc *realTaskStatusChecker) checkTaskStatus() { } func (tsc *realTaskStatusChecker) check() { - if tsc.w.cfg.EnableRelay { + if tsc.w.relayEnabled.Get() { tsc.checkRelayStatus() } tsc.checkTaskStatus() diff --git a/dm/worker/worker.go b/dm/worker/worker.go index 4a44c52d42..ba4edc1ce4 100644 --- a/dm/worker/worker.go +++ b/dm/worker/worker.go @@ -61,6 +61,7 @@ type Worker struct { subTaskHolder *subTaskHolder // relay functionality + // during relayEnabled == true, relayHolder and relayPurger should not be nil relayEnabled sync2.AtomicBool relayCtx context.Context relayCancel context.CancelFunc @@ -160,12 +161,10 @@ func (w *Worker) Close() { w.subTaskHolder.closeAllSubTasks() if w.relayHolder != nil { - // close relay w.relayHolder.Close() } if w.relayPurger != nil { - // close purger w.relayPurger.Close() } @@ -270,17 +269,26 @@ func (w *Worker) DisableRelay() { w.relayCancel() w.relayWg.Wait() + w.relayEnabled.Set(false) + + // refresh task checker know latest relayEnabled, to avoid accessing relayHolder + if w.cfg.Checker.CheckEnable { + w.l.Info("refresh task checker") + w.taskStatusChecker.Close() + w.taskStatusChecker.Start() + w.l.Info("finish refreshing task checker") + } + if w.relayHolder != nil { - w.relayHolder.Close() + r := w.relayHolder w.relayHolder = nil + r.Close() } - if w.relayPurger != nil { - w.relayPurger.Close() + r := w.relayPurger w.relayPurger = nil + r.Close() } - - w.relayEnabled.Set(false) } // EnableHandleSubtasks enables the functionality of start/watch/handle subtasks @@ -344,10 +352,10 @@ func (w *Worker) DisableHandleSubtasks() { w.Lock() defer w.Unlock() + w.subTaskEnabled.Set(false) // close all sub tasks w.subTaskHolder.closeAllSubTasks() - w.subTaskEnabled.Set(false) } // StartSubTask creates a sub task an run it @@ -377,7 +385,7 @@ func (w *Worker) StartSubTask(cfg *config.SubTaskConfig, expectStage pb.Stage) e } st.cfg = cfg2 - if w.relayPurger != nil && w.relayPurger.Purging() { + if w.relayEnabled.Get() && w.relayPurger.Purging() { // TODO: retry until purged finished st.fail(terror.ErrWorkerRelayIsPurging.Generate(cfg.Name)) return nil @@ -730,7 +738,7 @@ func (w *Worker) operateRelay(ctx context.Context, op pb.RelayOp) error { return terror.ErrWorkerAlreadyClosed.Generate() } - if w.relayHolder != nil { + if w.relayEnabled.Get() { return w.relayHolder.Operate(ctx, op) } @@ -744,7 +752,7 @@ func (w *Worker) PurgeRelay(ctx context.Context, req *pb.PurgeRelayRequest) erro return terror.ErrWorkerAlreadyClosed.Generate() } - if w.relayPurger != nil { + if w.relayEnabled.Get() { return w.relayPurger.Do(ctx, req) } diff --git a/syncer/syncer.go b/syncer/syncer.go index 35ab156bde..4c0950fd8e 100644 --- a/syncer/syncer.go +++ b/syncer/syncer.go @@ -206,10 +206,12 @@ func NewSyncer(cfg *config.SubTaskConfig, etcdClient *clientv3.Client) *Syncer { syncer.done = nil syncer.setTimezone() syncer.addJobFunc = syncer.addJob + // TODO: worker.relayEnabled syncer.enableRelay = cfg.UseRelay syncer.checkpoint = NewRemoteCheckPoint(syncer.tctx, cfg, syncer.checkpointID()) + // TODO: worker.relayEnabled syncer.binlogType = toBinlogType(cfg.UseRelay) syncer.errOperatorHolder = operator.NewHolder(&logger) syncer.readerHub = streamer.GetReaderHub() From df6614c01e3001baed2ca395d996a6f58569c6e4 Mon Sep 17 00:00:00 2001 From: lance6716 Date: Sat, 13 Mar 2021 23:03:25 +0800 Subject: [PATCH 12/46] try fix unstable test --- dm/master/bootstrap.go | 12 ++++++------ dm/worker/server.go | 1 + tests/ha_cases/lib.sh | 2 +- tests/ha_cases/run.sh | 23 +++++------------------ 4 files changed, 13 insertions(+), 25 deletions(-) diff --git a/dm/master/bootstrap.go b/dm/master/bootstrap.go index 5b5d32186a..bd150f5035 100644 --- a/dm/master/bootstrap.go +++ b/dm/master/bootstrap.go @@ -104,16 +104,16 @@ func (s *Server) importFromV10x(ctx context.Context) error { return err } - // 5. create sources. - logger.Info("add source config into cluster") - err = s.addSourcesV1Import(tctx, sourceCfgs) + // 5. upgrade v1.0.x downstream metadata table. + logger.Info("upgrading downstream metadata tables") + err = s.upgradeDBSchemaV1Import(tctx, subtaskCfgs) if err != nil { return err } - // 6. upgrade v1.0.x downstream metadata table. - logger.Info("upgrading downstream metadata tables") - err = s.upgradeDBSchemaV1Import(tctx, subtaskCfgs) + // 6. create sources. + logger.Info("add source config into cluster") + err = s.addSourcesV1Import(tctx, sourceCfgs) if err != nil { return err } diff --git a/dm/worker/server.go b/dm/worker/server.go index 264decf604..de927d8a8f 100644 --- a/dm/worker/server.go +++ b/dm/worker/server.go @@ -439,6 +439,7 @@ func (s *Server) stopWorker(sourceID string) error { // TODO: and when disable relay s.UpdateKeepAliveTTL(s.cfg.KeepAliveTTL) s.setWorker(nil, false) + s.setSourceStatus("", nil, false) s.Unlock() w.Close() return nil diff --git a/tests/ha_cases/lib.sh b/tests/ha_cases/lib.sh index b8300eb3c6..3a82858c5d 100755 --- a/tests/ha_cases/lib.sh +++ b/tests/ha_cases/lib.sh @@ -19,7 +19,7 @@ function load_data() { run_sql "CREATE DATABASE if not exists ${db};" $port $pswd run_sql "DROP TABLE if exists ${db}.t${i};" $port $pswd - run_sql "CREATE TABLE ${db}.t${i}(i TINYINT, j INT UNIQUE KEY);" $port $pswd + run_sql "CREATE TABLE ${db}.t${i}(i SMALLINT, j INT UNIQUE KEY);" $port $pswd for j in $(seq 800); do run_sql "INSERT INTO ${db}.t${i} VALUES ($j,${j}00$j),($j,${j}01$j);" $port $pswd sleep 0.1 diff --git a/tests/ha_cases/run.sh b/tests/ha_cases/run.sh index 9689ed2a82..bab90f49dc 100755 --- a/tests/ha_cases/run.sh +++ b/tests/ha_cases/run.sh @@ -554,13 +554,12 @@ function test_multi_task_reduce_and_restart_worker() { check_port_offline ${worker_ports[$[ $wk - 1] ]} 20 # just one worker was killed should be safe echo "${worker_inuse[$i]} was killed" + for name in ${task_name[@]}; do + run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "query-status $name"\ + "\"stage\": \"Running\"" 4 + done if [ $i = 0 ]; then - for name in ${task_name[@]}; do - run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" \ - "query-status $name"\ - "\"stage\": \"Running\"" 4 - done - # waiting for syncing wait sleep 2 @@ -568,18 +567,6 @@ function test_multi_task_reduce_and_restart_worker() { check_sync_diff $WORK_DIR $cur/conf/diff_config.toml check_sync_diff $WORK_DIR $cur/conf/diff_config_multi_task.toml echo "data checked after one worker was killed" - else - status_str="" - for name in ${task_name[@]}; do - status_str=$status_str$($PWD/bin/dmctl.test DEVEL --master-addr "127.0.0.1":$MASTER_PORT query-status $name) - done - search_str="\"stage\": \"Running\"" - running_count=$(echo $status_str | sed "s/$search_str/$search_str\n/g" | grep -c "$search_str") - if [ $running_count != 8 ]; then - echo "error running worker" - echo $status_str - exit 1 - fi fi done echo "[$(date)] <<<<<< finish test_multi_task_reduce_and_restart_worker >>>>>>" From e2ed64bd8cbe8aa6bf5f0b40bf9510a34508c82c Mon Sep 17 00:00:00 2001 From: lance6716 Date: Sun, 14 Mar 2021 13:03:46 +0800 Subject: [PATCH 13/46] fix some tests --- dm/master/bootstrap.go | 32 +++++++++++++++++++++----------- dm/worker/server.go | 35 ++++++++++++++++++----------------- dm/worker/server_test.go | 18 ++++++++++++++++-- pkg/ha/subtask.go | 11 ----------- pkg/ha/subtask_test.go | 6 +++--- pkg/upgrade/upgrade.go | 11 +++++++++++ 6 files changed, 69 insertions(+), 44 deletions(-) diff --git a/dm/master/bootstrap.go b/dm/master/bootstrap.go index bd150f5035..430e3ce3a9 100644 --- a/dm/master/bootstrap.go +++ b/dm/master/bootstrap.go @@ -56,17 +56,17 @@ func (s *Server) bootstrap(ctx context.Context) error { if err != nil { return terror.ErrMasterFailToImportFromV10x.Delegate(err) } + } else { + uctx := upgrade.Context{ + Context: ctx, + SubTaskConfigs: s.scheduler.GetSubTaskCfgs(), + } + err := upgrade.TryUpgrade(s.etcdClient, uctx) + if err != nil { + return err + } } - uctx := upgrade.Context{ - Context: ctx, - SubTaskConfigs: s.scheduler.GetSubTaskCfgs(), - } - err := upgrade.TryUpgrade(s.etcdClient, uctx) - - if err != nil { - return err - } return nil } @@ -104,12 +104,22 @@ func (s *Server) importFromV10x(ctx context.Context) error { return err } - // 5. upgrade v1.0.x downstream metadata table. + // 5. upgrade v1.0.x downstream metadata table and run v2.0 upgrading routines. + // some v2.0 upgrading routines are also altering schema, if we run them after adding sources, DM worker will + // meet error. logger.Info("upgrading downstream metadata tables") err = s.upgradeDBSchemaV1Import(tctx, subtaskCfgs) if err != nil { return err } + uctx := upgrade.Context{ + Context: ctx, + SubTaskConfigs: subtaskCfgs, + } + err = upgrade.UpgradeNotUpdateVersion(s.etcdClient, uctx) + if err != nil { + return err + } // 6. create sources. logger.Info("add source config into cluster") @@ -127,7 +137,7 @@ func (s *Server) importFromV10x(ctx context.Context) error { // 8. mark the upgrade operation as done. logger.Info("marking upgrade from v1.0.x as done") - _, err = upgrade.PutVersion(s.etcdClient, upgrade.MinVersion) + _, err = upgrade.PutVersion(s.etcdClient, upgrade.CurrentVersion) if err != nil { return err } diff --git a/dm/worker/server.go b/dm/worker/server.go index de927d8a8f..c38a949927 100644 --- a/dm/worker/server.go +++ b/dm/worker/server.go @@ -525,6 +525,7 @@ func (s *Server) QueryStatus(ctx context.Context, req *pb.QueryStatusRequest) (* return resp, nil } + // TODO: return relay status also, to make use of one lock resp.SubTaskStatus = w.QueryStatus(ctx, req.Name) if w.relayEnabled.Get() { sourceStatus.RelayStatus = w.relayHolder.Status(ctx) @@ -674,10 +675,10 @@ func getMinLocForSubTask(ctx context.Context, subTaskCfg config.SubTaskConfig) ( // see https://github.com/pingcap/dm/issues/727 func unifyMasterBinlogPos(resp *pb.QueryStatusResponse, enableGTID bool) { var ( - syncStatus []*pb.SubTaskStatus_Sync - syncMasterBinlog []*mysql.Position - lastestMasterBinlog mysql.Position // not pointer, to make use of zero value and avoid nil check - relayMasterBinlog *mysql.Position + syncStatus []*pb.SubTaskStatus_Sync + syncMasterBinlog []*mysql.Position + latestMasterBinlog mysql.Position // not pointer, to make use of zero value and avoid nil check + relayMasterBinlog *mysql.Position ) // uninitialized mysql.Position is less than any initialized mysql.Position @@ -685,10 +686,10 @@ func unifyMasterBinlogPos(resp *pb.QueryStatusResponse, enableGTID bool) { var err error relayMasterBinlog, err = utils.DecodeBinlogPosition(resp.SourceStatus.RelayStatus.MasterBinlog) if err != nil { - log.L().Error("failed to decode relay's master binlog position", zap.Stringer("response", resp), zap.Error(err)) + log.L().Warn("failed to decode relay's master binlog position", zap.Stringer("response", resp), zap.Error(err)) return } - lastestMasterBinlog = *relayMasterBinlog + latestMasterBinlog = *relayMasterBinlog } for _, stStatus := range resp.SubTaskStatus { @@ -698,11 +699,11 @@ func unifyMasterBinlogPos(resp *pb.QueryStatusResponse, enableGTID bool) { position, err := utils.DecodeBinlogPosition(s.Sync.MasterBinlog) if err != nil { - log.L().Error("failed to decode sync's master binlog position", zap.Stringer("response", resp), zap.Error(err)) + log.L().Warn("failed to decode sync's master binlog position", zap.Stringer("response", resp), zap.Error(err)) return } - if lastestMasterBinlog.Compare(*position) < 0 { - lastestMasterBinlog = *position + if latestMasterBinlog.Compare(*position) < 0 { + latestMasterBinlog = *position } syncMasterBinlog = append(syncMasterBinlog, position) } @@ -710,33 +711,33 @@ func unifyMasterBinlogPos(resp *pb.QueryStatusResponse, enableGTID bool) { // re-check relay if resp.SourceStatus.RelayStatus != nil && resp.SourceStatus.RelayStatus.Stage != pb.Stage_Stopped && - lastestMasterBinlog.Compare(*relayMasterBinlog) != 0 { + latestMasterBinlog.Compare(*relayMasterBinlog) != 0 { - resp.SourceStatus.RelayStatus.MasterBinlog = lastestMasterBinlog.String() + resp.SourceStatus.RelayStatus.MasterBinlog = latestMasterBinlog.String() // if enableGTID, modify output binlog position doesn't affect RelayCatchUpMaster, skip check if !enableGTID { relayPos, err := utils.DecodeBinlogPosition(resp.SourceStatus.RelayStatus.RelayBinlog) if err != nil { - log.L().Error("failed to decode relay binlog position", zap.Stringer("response", resp), zap.Error(err)) + log.L().Warn("failed to decode relay binlog position", zap.Stringer("response", resp), zap.Error(err)) return } - catchUp := lastestMasterBinlog.Compare(*relayPos) == 0 + catchUp := latestMasterBinlog.Compare(*relayPos) == 0 resp.SourceStatus.RelayStatus.RelayCatchUpMaster = catchUp } } // re-check syncer for i, sStatus := range syncStatus { - if lastestMasterBinlog.Compare(*syncMasterBinlog[i]) != 0 { + if latestMasterBinlog.Compare(*syncMasterBinlog[i]) != 0 { syncerPos, err := utils.DecodeBinlogPosition(sStatus.Sync.SyncerBinlog) if err != nil { - log.L().Error("failed to decode syncer binlog position", zap.Stringer("response", resp), zap.Error(err)) + log.L().Warn("failed to decode syncer binlog position", zap.Stringer("response", resp), zap.Error(err)) return } - synced := lastestMasterBinlog.Compare(*syncerPos) == 0 + synced := latestMasterBinlog.Compare(*syncerPos) == 0 - sStatus.Sync.MasterBinlog = lastestMasterBinlog.String() + sStatus.Sync.MasterBinlog = latestMasterBinlog.String() sStatus.Sync.Synced = synced } } diff --git a/dm/worker/server_test.go b/dm/worker/server_test.go index 3c3a9ca7db..bcd33a643d 100644 --- a/dm/worker/server_test.go +++ b/dm/worker/server_test.go @@ -167,8 +167,6 @@ func (t *testServer) TestServer(c *C) { subtaskCfg.MydumperPath = mydumperPath sourceCfg := loadSourceConfigWithoutPassword(c) - _, err = ha.PutSubTaskCfg(s.etcdClient, subtaskCfg) - c.Assert(err, IsNil) _, err = ha.PutSubTaskCfgStage(s.etcdClient, []config.SubTaskConfig{subtaskCfg}, []ha.Stage{ha.NewSubTaskStage(pb.Stage_Running, sourceCfg.SourceID, subtaskCfg.Name)}) c.Assert(err, IsNil) @@ -522,9 +520,25 @@ func (t *testServer) testSubTaskRecover(c *C, s *Server, dir string) { c.Assert(status.Msg, Equals, terror.ErrWorkerNoStart.Error()) t.testOperateWorker(c, s, dir, true) + + utils.WaitSomething(30, 100*time.Millisecond, func() bool { + status, err = workerCli.QueryStatus(context.Background(), &pb.QueryStatusRequest{Name: "sub-task-name"}) + if err != nil { + return false + } + if status.Result == false { + return false + } + if len(status.SubTaskStatus) == 0 || status.SubTaskStatus[0].Stage != pb.Stage_Running { + return false + } + return true + }) + status, err = workerCli.QueryStatus(context.Background(), &pb.QueryStatusRequest{Name: "sub-task-name"}) c.Assert(err, IsNil) c.Assert(status.Result, IsTrue) + c.Assert(status.SubTaskStatus, HasLen, 1) c.Assert(status.SubTaskStatus[0].Stage, Equals, pb.Stage_Running) } diff --git a/pkg/ha/subtask.go b/pkg/ha/subtask.go index eff639a6d0..e4b12d6b8b 100644 --- a/pkg/ha/subtask.go +++ b/pkg/ha/subtask.go @@ -24,17 +24,6 @@ import ( "github.com/pingcap/dm/pkg/terror" ) -// PutSubTaskCfg puts the subtask configs of the specified source and task name into etcd. -// k/k/v: sourceID, taskName -> subtask config. -func PutSubTaskCfg(cli *clientv3.Client, cfgs ...config.SubTaskConfig) (int64, error) { - ops, err := putSubTaskCfgOp(cfgs...) - if err != nil { - return 0, err - } - _, rev, err := etcdutil.DoOpsInOneTxnWithRetry(cli, ops...) - return rev, err -} - // GetSubTaskCfg gets the subtask config of the specified source and task name. // if the config for the source not exist, return with `err == nil` and `revision=0`. // if task name is "", will return all the subtaskConfigs as a map{taskName: subtaskConfig} of the source diff --git a/pkg/ha/subtask_test.go b/pkg/ha/subtask_test.go index 40b2a0905d..2518cebeb5 100644 --- a/pkg/ha/subtask_test.go +++ b/pkg/ha/subtask_test.go @@ -47,7 +47,7 @@ func (t *testForEtcd) TestSubTaskEtcd(c *C) { c.Assert(tsm1, HasLen, 0) // put subtask configs. - rev2, err := PutSubTaskCfg(etcdTestCli, cfg1, cfg2) + rev2, err := PutSubTaskCfgStage(etcdTestCli, []config.SubTaskConfig{cfg1, cfg2}, []Stage{}) c.Assert(err, IsNil) c.Assert(rev2, Greater, rev1) @@ -92,14 +92,14 @@ func (t *testForEtcd) TestSubTaskEtcd(c *C) { c.Assert(tsm4, HasLen, 0) // put subtask config. - rev6, err := PutSubTaskCfg(etcdTestCli, cfg1) + rev6, err := PutSubTaskCfgStage(etcdTestCli, []config.SubTaskConfig{cfg1}, []Stage{}) c.Assert(err, IsNil) c.Assert(rev6, Greater, int64(0)) // update subtask config. cfg3 := cfg1 cfg3.SourceID = "testForRevision" - rev7, err := PutSubTaskCfg(etcdTestCli, cfg3) + rev7, err := PutSubTaskCfgStage(etcdTestCli, []config.SubTaskConfig{cfg3}, []Stage{}) c.Assert(err, IsNil) c.Assert(rev7, Greater, rev6) diff --git a/pkg/upgrade/upgrade.go b/pkg/upgrade/upgrade.go index 161df2e289..387dfba381 100644 --- a/pkg/upgrade/upgrade.go +++ b/pkg/upgrade/upgrade.go @@ -95,6 +95,17 @@ func TryUpgrade(cli *clientv3.Client, uctx Context) error { return err } +// UpgradeNotUpdateVersion does same as TryUpgrade except for PutVersion. This function is called when upgrade from v1.0 +func UpgradeNotUpdateVersion(cli *clientv3.Client, uctx Context) error { + for _, upgrade := range upgrades { + err := upgrade(cli, uctx) + if err != nil { + return err + } + } + return nil +} + // upgradeToVer1 does upgrade operations from Ver0 to Ver1. // in fact, this do nothing now, and just for demonstration. func upgradeToVer1(cli *clientv3.Client, uctx Context) error { From 841fe25cf92d8c24995bd1dd95f6eebf70856546 Mon Sep 17 00:00:00 2001 From: lance6716 Date: Sun, 14 Mar 2021 14:42:57 +0800 Subject: [PATCH 14/46] fix CI --- dm/master/bootstrap.go | 2 +- dm/worker/server.go | 2 +- dm/worker/server_test.go | 4 +++- dm/worker/worker_test.go | 9 ++++----- pkg/upgrade/upgrade.go | 8 ++++---- 5 files changed, 13 insertions(+), 12 deletions(-) diff --git a/dm/master/bootstrap.go b/dm/master/bootstrap.go index 430e3ce3a9..2cc2db7801 100644 --- a/dm/master/bootstrap.go +++ b/dm/master/bootstrap.go @@ -116,7 +116,7 @@ func (s *Server) importFromV10x(ctx context.Context) error { Context: ctx, SubTaskConfigs: subtaskCfgs, } - err = upgrade.UpgradeNotUpdateVersion(s.etcdClient, uctx) + err = upgrade.UntouchVersionUpgrade(s.etcdClient, uctx) if err != nil { return err } diff --git a/dm/worker/server.go b/dm/worker/server.go index c38a949927..c7a1ab7a47 100644 --- a/dm/worker/server.go +++ b/dm/worker/server.go @@ -525,7 +525,7 @@ func (s *Server) QueryStatus(ctx context.Context, req *pb.QueryStatusRequest) (* return resp, nil } - // TODO: return relay status also, to make use of one lock + // TODO: return relay status as well, to make use of one lock resp.SubTaskStatus = w.QueryStatus(ctx, req.Name) if w.relayEnabled.Get() { sourceStatus.RelayStatus = w.relayHolder.Status(ctx) diff --git a/dm/worker/server_test.go b/dm/worker/server_test.go index bcd33a643d..c5495d0df7 100644 --- a/dm/worker/server_test.go +++ b/dm/worker/server_test.go @@ -502,7 +502,6 @@ func (t *testServer) testRetryConnectMaster(c *C, s *Server, ETCD *embed.Etcd, d time.Sleep(6 * time.Second) // When worker server fail to keepalive with etcd, server should close its worker c.Assert(s.getWorker(true), IsNil) - c.Assert(s.getSourceStatus(true).Result, IsNil) ETCD, err := createMockETCD(dir, "http://"+hostName) c.Assert(err, IsNil) @@ -521,6 +520,9 @@ func (t *testServer) testSubTaskRecover(c *C, s *Server, dir string) { t.testOperateWorker(c, s, dir, true) + // because we split starting worker and enabling handling subtasks into two parts, a query-status may occur between + // them, thus get a result of no subtask running + // TODO: use a lock to avoid query-status occur between them utils.WaitSomething(30, 100*time.Millisecond, func() bool { status, err = workerCli.QueryStatus(context.Background(), &pb.QueryStatusRequest{Name: "sub-task-name"}) if err != nil { diff --git a/dm/worker/worker_test.go b/dm/worker/worker_test.go index a267ff834f..a08238bdc4 100644 --- a/dm/worker/worker_test.go +++ b/dm/worker/worker_test.go @@ -42,11 +42,6 @@ func (t *testServer) testWorker(c *C) { cfg.RelayDir = dir cfg.MetaDir = dir - NewRelayHolder = NewDummyRelayHolderWithInitError - defer func() { - NewRelayHolder = NewRealRelayHolder - }() - var ( masterAddr = tempurl.Alloc()[len("http://"):] keepAliveTTL = int64(1) @@ -69,6 +64,10 @@ func (t *testServer) testWorker(c *C) { }) c.Assert(err, IsNil) + NewRelayHolder = NewDummyRelayHolderWithInitError + defer func() { + NewRelayHolder = NewRealRelayHolder + }() w, err := NewWorker(&cfg, etcdCli, "") c.Assert(err, IsNil) c.Assert(w.EnableRelay(), ErrorMatches, "init error") diff --git a/pkg/upgrade/upgrade.go b/pkg/upgrade/upgrade.go index 387dfba381..96c5d4a5ce 100644 --- a/pkg/upgrade/upgrade.go +++ b/pkg/upgrade/upgrade.go @@ -95,8 +95,9 @@ func TryUpgrade(cli *clientv3.Client, uctx Context) error { return err } -// UpgradeNotUpdateVersion does same as TryUpgrade except for PutVersion. This function is called when upgrade from v1.0 -func UpgradeNotUpdateVersion(cli *clientv3.Client, uctx Context) error { +// UntouchVersionUpgrade runs all upgrade functions but doesn't change cluster version. This function is called when +// upgrade from v1.0, with a later PutVersion in caller after success +func UntouchVersionUpgrade(cli *clientv3.Client, uctx Context) error { for _, upgrade := range upgrades { err := upgrade(cli, uctx) if err != nil { @@ -112,8 +113,7 @@ func upgradeToVer1(cli *clientv3.Client, uctx Context) error { return nil } -// upgradeToVer2 does upgrade operations from Ver1 to Ver2 (v2.0.0-rc.3) to upgrade syncer checkpoint schema -// TODO: determine v2.0.0-rc.3 or another version in above line +// upgradeToVer2 does upgrade operations from Ver1 to Ver2 (v2.0.0-GA) to upgrade syncer checkpoint schema func upgradeToVer2(cli *clientv3.Client, uctx Context) error { upgradeTaskName := "upgradeToVer2" logger := log.L().WithFields(zap.String("task", upgradeTaskName)) From 2b64812d0e04e6f22b399675a2f2942cd8e12309 Mon Sep 17 00:00:00 2001 From: lance6716 Date: Sun, 14 Mar 2021 16:42:07 +0800 Subject: [PATCH 15/46] reduce some TODO --- dm/worker/relay.go | 10 +++------- dm/worker/relay_test.go | 5 +++-- dm/worker/server.go | 7 +------ dm/worker/worker.go | 30 +++++++++++++++++------------- dm/worker/worker_test.go | 9 +++++---- relay/metrics.go | 9 ++++++--- relay/relay.go | 2 +- 7 files changed, 36 insertions(+), 36 deletions(-) diff --git a/dm/worker/relay.go b/dm/worker/relay.go index 10545579f8..44b593909e 100644 --- a/dm/worker/relay.go +++ b/dm/worker/relay.go @@ -23,7 +23,6 @@ import ( "github.com/pingcap/dm/dm/config" "github.com/pingcap/dm/dm/pb" - "github.com/pingcap/dm/dm/unit" "github.com/pingcap/dm/pkg/log" "github.com/pingcap/dm/pkg/streamer" "github.com/pingcap/dm/pkg/terror" @@ -34,7 +33,7 @@ import ( // RelayHolder for relay unit type RelayHolder interface { // Init initializes the holder - Init(interceptors []purger.PurgeInterceptor) (purger.Purger, error) + Init(ctx context.Context, interceptors []purger.PurgeInterceptor) (purger.Purger, error) // Start starts run the relay Start() // Close closes the holder @@ -90,7 +89,7 @@ func NewRealRelayHolder(sourceCfg *config.SourceConfig) RelayHolder { } // Init initializes the holder -func (h *realRelayHolder) Init(interceptors []purger.PurgeInterceptor) (purger.Purger, error) { +func (h *realRelayHolder) Init(ctx context.Context, interceptors []purger.PurgeInterceptor) (purger.Purger, error) { h.closed.Set(false) // initial relay purger @@ -99,9 +98,6 @@ func (h *realRelayHolder) Init(interceptors []purger.PurgeInterceptor) (purger.P streamer.GetReaderHub(), } - // TODO: refine the context usage of relay, and it may need to be initialized before handle any subtasks. - ctx, cancel := context.WithTimeout(context.Background(), unit.DefaultInitTimeout) - defer cancel() if err := h.relay.Init(ctx); err != nil { return nil, terror.Annotate(err, "initial relay unit") } @@ -356,7 +352,7 @@ func NewDummyRelayHolderWithInitError(cfg *config.SourceConfig) RelayHolder { } // Init implements interface of RelayHolder -func (d *dummyRelayHolder) Init(interceptors []purger.PurgeInterceptor) (purger.Purger, error) { +func (d *dummyRelayHolder) Init(ctx context.Context, interceptors []purger.PurgeInterceptor) (purger.Purger, error) { // initial relay purger operators := []purger.RelayOperator{ d, diff --git a/dm/worker/relay_test.go b/dm/worker/relay_test.go index c55f3310fb..9535414389 100644 --- a/dm/worker/relay_test.go +++ b/dm/worker/relay_test.go @@ -159,7 +159,8 @@ func (t *testRelay) TestRelay(c *C) { } func (t *testRelay) testInit(c *C, holder *realRelayHolder) { - _, err := holder.Init(nil) + ctx := context.Background() + _, err := holder.Init(ctx, nil) c.Assert(err, IsNil) r, ok := holder.relay.(*DummyRelay) @@ -169,7 +170,7 @@ func (t *testRelay) testInit(c *C, holder *realRelayHolder) { r.InjectInitError(initErr) defer r.InjectInitError(nil) - _, err = holder.Init(nil) + _, err = holder.Init(ctx, nil) c.Assert(err, ErrorMatches, ".*"+initErr.Error()+".*") } diff --git a/dm/worker/server.go b/dm/worker/server.go index c7a1ab7a47..b79ec8062a 100644 --- a/dm/worker/server.go +++ b/dm/worker/server.go @@ -525,12 +525,7 @@ func (s *Server) QueryStatus(ctx context.Context, req *pb.QueryStatusRequest) (* return resp, nil } - // TODO: return relay status as well, to make use of one lock - resp.SubTaskStatus = w.QueryStatus(ctx, req.Name) - if w.relayEnabled.Get() { - sourceStatus.RelayStatus = w.relayHolder.Status(ctx) - } - + resp.SubTaskStatus, sourceStatus.RelayStatus = w.QueryStatus(ctx, req.Name) unifyMasterBinlogPos(resp, w.cfg.EnableGTID) if len(resp.SubTaskStatus) == 0 { diff --git a/dm/worker/worker.go b/dm/worker/worker.go index ba4edc1ce4..fb1bffd4a5 100644 --- a/dm/worker/worker.go +++ b/dm/worker/worker.go @@ -40,7 +40,7 @@ import ( // Worker manages sub tasks and process units for data migration type Worker struct { // ensure no other operation can be done when closing (we can use `WatGroup`/`Context` to archive this)/ - // seems only guarding subTaskHolder + // TODO: check if it only guards subTaskHolder sync.RWMutex wg sync.WaitGroup @@ -219,7 +219,7 @@ func (w *Worker) EnableRelay() (err error) { // 2. initial relay holder, the cfg's password need decrypt w.relayHolder = NewRelayHolder(w.cfg) - relayPurger, err := w.relayHolder.Init([]purger.PurgeInterceptor{ + relayPurger, err := w.relayHolder.Init(w.relayCtx, []purger.PurgeInterceptor{ w, }) if err != nil { @@ -258,8 +258,7 @@ func (w *Worker) EnableRelay() (err error) { // DisableRelay disables the functionality of start/watch/handle relay func (w *Worker) DisableRelay() { - // TODO: use CAS? - if !w.relayEnabled.Get() { + if !w.relayEnabled.CompareAndSwap(true, false) { w.l.Warn("already disabled relay") return } @@ -269,8 +268,6 @@ func (w *Worker) DisableRelay() { w.relayCancel() w.relayWg.Wait() - w.relayEnabled.Set(false) - // refresh task checker know latest relayEnabled, to avoid accessing relayHolder if w.cfg.Checker.CheckEnable { w.l.Info("refresh task checker") @@ -341,8 +338,7 @@ func (w *Worker) EnableHandleSubtasks() error { // DisableHandleSubtasks disables the functionality of start/watch/handle subtasks func (w *Worker) DisableHandleSubtasks() { - // TODO: use CAS? - if !w.subTaskEnabled.Get() { + if !w.subTaskEnabled.CompareAndSwap(true, false) { w.l.Warn("already disabled handling subtasks") return } @@ -352,7 +348,6 @@ func (w *Worker) DisableHandleSubtasks() { w.Lock() defer w.Unlock() - w.subTaskEnabled.Set(false) // close all sub tasks w.subTaskHolder.closeAllSubTasks() @@ -450,20 +445,29 @@ func (w *Worker) OperateSubTask(name string, op pb.TaskOp) error { return err } -// QueryStatus query worker's sub tasks' status -func (w *Worker) QueryStatus(ctx context.Context, name string) []*pb.SubTaskStatus { +// QueryStatus query worker's sub tasks' status. If relay enabled, also return source status +// TODO: `ctx` is used to get upstream master status in every subtasks and source. +// reduce it to only call once and remove `unifyMasterBinlogPos`, also remove below ctx2 +func (w *Worker) QueryStatus(ctx context.Context, name string) ([]*pb.SubTaskStatus, *pb.RelayStatus) { w.RLock() defer w.RUnlock() if w.closed.Get() { w.l.Warn("querying status from a closed worker") - return nil + return nil, nil } // use one timeout for all tasks. increase this value if it's too short. ctx2, cancel2 := context.WithTimeout(ctx, utils.DefaultDBTimeout) defer cancel2() - return w.Status(ctx2, name) + var ( + subtaskStatus = w.Status(ctx2, name) + relayStatus *pb.RelayStatus + ) + if w.relayEnabled.Get() { + relayStatus = w.relayHolder.Status(ctx2) + } + return subtaskStatus, relayStatus } func (w *Worker) resetSubtaskStage(etcdCli *clientv3.Client) (int64, error) { diff --git a/dm/worker/worker_test.go b/dm/worker/worker_test.go index a08238bdc4..3ff1dada35 100644 --- a/dm/worker/worker_test.go +++ b/dm/worker/worker_test.go @@ -186,7 +186,8 @@ func (t *testServer2) TestTaskAutoResume(c *C) { // check task in paused state c.Assert(utils.WaitSomething(100, 100*time.Millisecond, func() bool { - for _, st := range s.getWorker(true).QueryStatus(context.Background(), taskName) { + subtaskStatus, _ := s.getWorker(true).QueryStatus(context.Background(), taskName) + for _, st := range subtaskStatus { if st.Name == taskName && st.Stage == pb.Stage_Paused { return true } @@ -206,7 +207,7 @@ func (t *testServer2) TestTaskAutoResume(c *C) { // check task will be auto resumed c.Assert(utils.WaitSomething(10, 100*time.Millisecond, func() bool { - sts := s.getWorker(true).QueryStatus(context.Background(), taskName) + sts, _ := s.getWorker(true).QueryStatus(context.Background(), taskName) for _, st := range sts { if st.Name == taskName && st.Stage == pb.Stage_Running { return true @@ -326,7 +327,7 @@ func (t *testWorkerEtcdCompact) TestWatchSubtaskStageEtcdCompact(c *C) { c.Assert(utils.WaitSomething(30, 100*time.Millisecond, func() bool { return w.subTaskHolder.findSubTask(subtaskCfg.Name) != nil }), IsTrue) - status := w.QueryStatus(ctx1, subtaskCfg.Name) + status, _ := w.QueryStatus(ctx1, subtaskCfg.Name) c.Assert(status, HasLen, 1) c.Assert(status[0].Name, Equals, subtaskCfg.Name) c.Assert(status[0].Stage, Equals, pb.Stage_Running) @@ -344,7 +345,7 @@ func (t *testWorkerEtcdCompact) TestWatchSubtaskStageEtcdCompact(c *C) { c.Assert(utils.WaitSomething(30, 100*time.Millisecond, func() bool { return w.subTaskHolder.findSubTask(subtaskCfg.Name) != nil }), IsTrue) - status = w.QueryStatus(ctx2, subtaskCfg.Name) + status, _ = w.QueryStatus(ctx2, subtaskCfg.Name) c.Assert(status, HasLen, 1) c.Assert(status[0].Name, Equals, subtaskCfg.Name) c.Assert(status[0].Stage, Equals, pb.Stage_Running) diff --git a/relay/metrics.go b/relay/metrics.go index e68797e27c..5a3a122dd3 100644 --- a/relay/metrics.go +++ b/relay/metrics.go @@ -14,6 +14,7 @@ package relay import ( + "context" "time" "github.com/prometheus/client_golang/prometheus" @@ -149,8 +150,7 @@ func RegisterMetrics(registry *prometheus.Registry) { registry.MustRegister(relayExitWithErrorCounter) } -// TODO: add a context to stop it -func reportRelayLogSpaceInBackground(dirpath string) error { +func reportRelayLogSpaceInBackground(ctx context.Context, dirpath string) error { if len(dirpath) == 0 { return terror.ErrRelayLogDirpathEmpty.Generate() } @@ -159,7 +159,10 @@ func reportRelayLogSpaceInBackground(dirpath string) error { ticker := time.NewTicker(time.Second * 10) defer ticker.Stop() - for range ticker.C { + select { + case <-ctx.Done(): + return + case <-ticker.C: size, err := utils.GetStorageSize(dirpath) if err != nil { log.L().Error("fail to update relay log storage size", log.ShortError(err)) diff --git a/relay/relay.go b/relay/relay.go index 9d37f9ac53..bc22f85a4a 100755 --- a/relay/relay.go +++ b/relay/relay.go @@ -165,7 +165,7 @@ func (r *Relay) Init(ctx context.Context) (err error) { r.relayMetaHub = pkgstreamer.GetRelayMetaHub() r.relayMetaHub.ClearMeta() - return reportRelayLogSpaceInBackground(r.cfg.RelayDir) + return reportRelayLogSpaceInBackground(ctx, r.cfg.RelayDir) } // Process implements the dm.Unit interface. From 0b24d2259d3203fc73bedfe7d99e7cb4ff6b2d8a Mon Sep 17 00:00:00 2001 From: lance6716 Date: Sun, 14 Mar 2021 22:30:41 +0800 Subject: [PATCH 16/46] add test --- dm/config/subtask.go | 2 +- dm/worker/server_test.go | 1 - dm/worker/subtask.go | 4 +- dm/worker/subtask_holder.go | 12 +++ dm/worker/worker.go | 50 +++++++---- dm/worker/worker_test.go | 171 +++++++++++++++++++++++++++++++++++- syncer/syncer.go | 2 - 7 files changed, 216 insertions(+), 26 deletions(-) diff --git a/dm/config/subtask.go b/dm/config/subtask.go index 79ba918c8c..b3c5a70f69 100644 --- a/dm/config/subtask.go +++ b/dm/config/subtask.go @@ -158,7 +158,7 @@ type SubTaskConfig struct { // RelayDir get value from dm-worker config RelayDir string `toml:"relay-dir" json:"relay-dir"` - // UseRelay get value from dm-worker config + // UseRelay get value from dm-worker's relayEnabled UseRelay bool `toml:"use-relay" json:"use-relay"` From DBConfig `toml:"from" json:"from"` To DBConfig `toml:"to" json:"to"` diff --git a/dm/worker/server_test.go b/dm/worker/server_test.go index c5495d0df7..787da5782b 100644 --- a/dm/worker/server_test.go +++ b/dm/worker/server_test.go @@ -522,7 +522,6 @@ func (t *testServer) testSubTaskRecover(c *C, s *Server, dir string) { // because we split starting worker and enabling handling subtasks into two parts, a query-status may occur between // them, thus get a result of no subtask running - // TODO: use a lock to avoid query-status occur between them utils.WaitSomething(30, 100*time.Millisecond, func() bool { status, err = workerCli.QueryStatus(context.Background(), &pb.QueryStatusRequest{Name: "sub-task-name"}) if err != nil { diff --git a/dm/worker/subtask.go b/dm/worker/subtask.go index 649b684529..e798591f5f 100644 --- a/dm/worker/subtask.go +++ b/dm/worker/subtask.go @@ -39,8 +39,8 @@ import ( ) const ( - // the timout to wait for relay catchup when switching from load unit to sync unit. - waitRelayCatchupTimeout = 5 * time.Minute + // the timeout to wait for relay catchup when switching from load unit to sync unit. + waitRelayCatchupTimeout = 30 * time.Second ) // createRealUnits is subtask units initializer diff --git a/dm/worker/subtask_holder.go b/dm/worker/subtask_holder.go index 0123bafa90..e8754ca6fa 100644 --- a/dm/worker/subtask_holder.go +++ b/dm/worker/subtask_holder.go @@ -46,6 +46,18 @@ func (h *subTaskHolder) removeSubTask(name string) { delete(h.subTasks, name) } +// resetAllSubTasks does Close, change cfg.UseRelay then Init the subtasks +func (h *subTaskHolder) resetAllSubTasks(useRelay bool) { + h.mu.Lock() + defer h.mu.Unlock() + for _, st := range h.subTasks { + stage := st.Stage() + st.Close() + st.cfg.UseRelay = useRelay + st.Run(stage) + } +} + // closeAllSubTasks closes all subtask instances. func (h *subTaskHolder) closeAllSubTasks() { h.mu.Lock() diff --git a/dm/worker/worker.go b/dm/worker/worker.go index fb1bffd4a5..d9535810bb 100644 --- a/dm/worker/worker.go +++ b/dm/worker/worker.go @@ -39,8 +39,9 @@ import ( // Worker manages sub tasks and process units for data migration type Worker struct { - // ensure no other operation can be done when closing (we can use `WatGroup`/`Context` to archive this)/ - // TODO: check if it only guards subTaskHolder + // ensure no other operation can be done when closing (we can use `WatGroup`/`Context` to archive this) + // TODO: check what does it guards. Now it's used to guard relayHolder and relayPurger (maybe subTaskHolder?) since + // query-status maybe access them when closing/disable functionalities sync.RWMutex wg sync.WaitGroup @@ -179,6 +180,8 @@ func (w *Worker) Close() { // EnableRelay enables the functionality of start/watch/handle relay func (w *Worker) EnableRelay() (err error) { + w.Lock() + defer w.Unlock() if w.relayEnabled.Get() { w.l.Warn("already enabled relay") return nil @@ -191,7 +194,8 @@ func (w *Worker) EnableRelay() (err error) { if err != nil { return err } - if err = copyConfigFromSourceForEach(subTaskCfgs, w.cfg); err != nil { + // we just use subTaskCfgs to adjust relay's binlog location, so set `enableRelay` arbitrary + if err = copyConfigFromSourceForEach(subTaskCfgs, w.cfg, true); err != nil { return err } @@ -253,18 +257,19 @@ func (w *Worker) EnableRelay() (err error) { w.relayEnabled.Set(true) w.l.Info("relay enabled") + w.subTaskHolder.resetAllSubTasks(true) return nil } // DisableRelay disables the functionality of start/watch/handle relay func (w *Worker) DisableRelay() { + w.Lock() + defer w.Unlock() if !w.relayEnabled.CompareAndSwap(true, false) { w.l.Warn("already disabled relay") return } - // TODO: check if running subtask need accessing relay - w.relayCancel() w.relayWg.Wait() @@ -276,6 +281,8 @@ func (w *Worker) DisableRelay() { w.l.Info("finish refreshing task checker") } + w.subTaskHolder.resetAllSubTasks(false) + if w.relayHolder != nil { r := w.relayHolder w.relayHolder = nil @@ -290,6 +297,8 @@ func (w *Worker) DisableRelay() { // EnableHandleSubtasks enables the functionality of start/watch/handle subtasks func (w *Worker) EnableHandleSubtasks() error { + w.Lock() + defer w.Unlock() if w.subTaskEnabled.Get() { w.l.Warn("already enabled handling subtasks") return nil @@ -303,7 +312,8 @@ func (w *Worker) EnableHandleSubtasks() error { return err } - if err = copyConfigFromSourceForEach(subTaskCfgM, w.cfg); err != nil { + // TODO: make sure relayEnabled is not changing + if err = copyConfigFromSourceForEach(subTaskCfgM, w.cfg, w.relayEnabled.Get()); err != nil { return err } @@ -317,7 +327,7 @@ func (w *Worker) EnableHandleSubtasks() error { w.l.Info("start to create subtask", zap.String("sourceID", subTaskCfg.SourceID), zap.String("task", subTaskCfg.Name)) // "for range" of a map will use same value address, so we'd better not pass value address to other function clone := subTaskCfg - if err2 := w.StartSubTask(&clone, expectStage.Expect); err2 != nil { + if err2 := w.StartSubTask(&clone, expectStage.Expect, false); err2 != nil { w.subTaskHolder.closeAllSubTasks() return err2 } @@ -354,12 +364,14 @@ func (w *Worker) DisableHandleSubtasks() { } // StartSubTask creates a sub task an run it -func (w *Worker) StartSubTask(cfg *config.SubTaskConfig, expectStage pb.Stage) error { - w.Lock() - defer w.Unlock() +func (w *Worker) StartSubTask(cfg *config.SubTaskConfig, expectStage pb.Stage, needLock bool) error { + if needLock { + w.Lock() + defer w.Unlock() + } // copy some config item from dm-worker's source config - err := copyConfigFromSource(cfg, w.cfg) + err := copyConfigFromSource(cfg, w.cfg, w.relayEnabled.Get()) if err != nil { return err } @@ -597,7 +609,7 @@ func (w *Worker) operateSubTaskStage(stage ha.Stage, subTaskCfg config.SubTaskCo if st := w.subTaskHolder.findSubTask(stage.Task); st == nil { // create the subtask for expected running and paused stage. log.L().Info("start to create subtask", zap.String("sourceID", subTaskCfg.SourceID), zap.String("task", subTaskCfg.Name)) - err := w.StartSubTask(&subTaskCfg, stage.Expect) + err := w.StartSubTask(&subTaskCfg, stage.Expect, true) return opErrTypeBeforeOp, err } if stage.Expect == pb.Stage_Running { @@ -799,15 +811,15 @@ func (w *Worker) OperateSchema(ctx context.Context, req *pb.OperateWorkerSchemaR return st.OperateSchema(ctx, req) } -// copyConfigFromSource copies config items from source config to sub task -func copyConfigFromSource(cfg *config.SubTaskConfig, sourceCfg *config.SourceConfig) error { +// copyConfigFromSource copies config items from source config and worker's relayEnabled to sub task +func copyConfigFromSource(cfg *config.SubTaskConfig, sourceCfg *config.SourceConfig, enableRelay bool) error { cfg.From = sourceCfg.From cfg.Flavor = sourceCfg.Flavor cfg.ServerID = sourceCfg.ServerID cfg.RelayDir = sourceCfg.RelayDir cfg.EnableGTID = sourceCfg.EnableGTID - cfg.UseRelay = sourceCfg.EnableRelay + cfg.UseRelay = enableRelay // we can remove this from SubTaskConfig later, because syncer will always read from relay cfg.AutoFixGTID = sourceCfg.AutoFixGTID @@ -836,9 +848,13 @@ func copyConfigFromSource(cfg *config.SubTaskConfig, sourceCfg *config.SourceCon } // copyConfigFromSourceForEach do copyConfigFromSource for each value in subTaskCfgM and change subTaskCfgM in-place -func copyConfigFromSourceForEach(subTaskCfgM map[string]config.SubTaskConfig, sourceCfg *config.SourceConfig) error { +func copyConfigFromSourceForEach( + subTaskCfgM map[string]config.SubTaskConfig, + sourceCfg *config.SourceConfig, + enableRelay bool, +) error { for k, subTaskCfg := range subTaskCfgM { - if err2 := copyConfigFromSource(&subTaskCfg, sourceCfg); err2 != nil { + if err2 := copyConfigFromSource(&subTaskCfg, sourceCfg, enableRelay); err2 != nil { return err2 } subTaskCfgM[k] = subTaskCfg diff --git a/dm/worker/worker_test.go b/dm/worker/worker_test.go index 3ff1dada35..c67bf7ac12 100644 --- a/dm/worker/worker_test.go +++ b/dm/worker/worker_test.go @@ -88,7 +88,7 @@ func (t *testServer) testWorker(c *C) { c.Assert(w.StartSubTask(&config.SubTaskConfig{ Name: "testStartTask", - }, pb.Stage_Running), IsNil) + }, pb.Stage_Running, true), IsNil) task := w.subTaskHolder.findSubTask("testStartTask") c.Assert(task, NotNil) c.Assert(task.Result().String(), Matches, ".*worker already closed.*") @@ -182,7 +182,7 @@ func (t *testServer2) TestTaskAutoResume(c *C) { var subtaskCfg config.SubTaskConfig c.Assert(subtaskCfg.DecodeFile("./subtask.toml", true), IsNil) c.Assert(err, IsNil) - c.Assert(s.getWorker(true).StartSubTask(&subtaskCfg, pb.Stage_Running), IsNil) + c.Assert(s.getWorker(true).StartSubTask(&subtaskCfg, pb.Stage_Running, true), IsNil) // check task in paused state c.Assert(utils.WaitSomething(100, 100*time.Millisecond, func() bool { @@ -218,6 +218,171 @@ func (t *testServer2) TestTaskAutoResume(c *C) { }), IsTrue) } +type testWorkerFunctionalities struct { + createUnitCount int + expectedCreateUnitCount int +} + +var _ = Suite(&testWorkerFunctionalities{}) + +func (t *testWorkerFunctionalities) SetUpSuite(c *C) { + NewRelayHolder = NewDummyRelayHolder + NewSubTask = func(cfg *config.SubTaskConfig, etcdClient *clientv3.Client) *SubTask { + return NewRealSubTask(cfg, etcdClient) + } + createUnits = func(cfg *config.SubTaskConfig, etcdClient *clientv3.Client) []unit.Unit { + t.createUnitCount++ + mockDumper := NewMockUnit(pb.UnitType_Dump) + mockLoader := NewMockUnit(pb.UnitType_Load) + mockSync := NewMockUnit(pb.UnitType_Sync) + return []unit.Unit{mockDumper, mockLoader, mockSync} + } +} + +func (t *testWorkerFunctionalities) TearDownSuite(c *C) { + NewRelayHolder = NewRealRelayHolder + NewSubTask = NewRealSubTask + createUnits = createRealUnits +} + +func (t *testWorkerFunctionalities) TestWorkerFunctionalities(c *C) { + var ( + masterAddr = tempurl.Alloc()[len("http://"):] + keepAliveTTL = int64(1) + ) + etcdDir := c.MkDir() + ETCD, err := createMockETCD(etcdDir, "http://"+masterAddr) + c.Assert(err, IsNil) + defer ETCD.Close() + cfg := NewConfig() + c.Assert(cfg.Parse([]string{"-config=./dm-worker.toml"}), IsNil) + cfg.Join = masterAddr + cfg.KeepAliveTTL = keepAliveTTL + cfg.RelayKeepAliveTTL = keepAliveTTL + + etcdCli, err := clientv3.New(clientv3.Config{ + Endpoints: GetJoinURLs(cfg.Join), + DialTimeout: dialTimeout, + DialKeepAliveTime: keepaliveTime, + DialKeepAliveTimeout: keepaliveTimeout, + }) + c.Assert(err, IsNil) + sourceCfg := loadSourceConfigWithoutPassword(c) + sourceCfg.EnableRelay = false + + subtaskCfg := config.SubTaskConfig{} + err = subtaskCfg.DecodeFile(subtaskSampleFile, true) + c.Assert(err, IsNil) + + // start worker + w, err := NewWorker(&sourceCfg, etcdCli, "") + c.Assert(err, IsNil) + defer w.Close() + go func() { + w.Start() + }() + c.Assert(utils.WaitSomething(50, 100*time.Millisecond, func() bool { + return !w.closed.Get() + }), IsTrue) + + // test 1: when subTaskEnabled is false, switch on relay + c.Assert(w.subTaskEnabled.Get(), IsFalse) + t.testEnableRelay(c, w, etcdCli, sourceCfg, cfg) + + // test2: when subTaskEnabled is false, switch off relay + c.Assert(w.subTaskEnabled.Get(), IsFalse) + t.testDisableRelay(c, w) + + // test3: when relayEnabled is false, switch on subtask + c.Assert(w.relayEnabled.Get(), IsFalse) + + t.testEnableHandleSubtasks(c, w, etcdCli, subtaskCfg, sourceCfg) + + // test4: when subTaskEnabled is true, switch on relay + c.Assert(w.subTaskEnabled.Get(), IsTrue) + + t.testEnableRelay(c, w, etcdCli, sourceCfg, cfg) + c.Assert(w.subTaskHolder.findSubTask(subtaskCfg.Name).cfg.UseRelay, IsTrue) + t.expectedCreateUnitCount++ + c.Assert(t.createUnitCount, Equals, t.expectedCreateUnitCount) + + // test5: when subTaskEnabled is true, switch off relay + c.Assert(w.subTaskEnabled.Get(), IsTrue) + t.testDisableRelay(c, w) + + c.Assert(w.subTaskHolder.findSubTask(subtaskCfg.Name).cfg.UseRelay, IsFalse) + t.expectedCreateUnitCount++ + c.Assert(t.createUnitCount, Equals, t.expectedCreateUnitCount) + + // test6: when relayEnabled is false, switch off subtask + c.Assert(w.relayEnabled.Get(), IsFalse) + + w.DisableHandleSubtasks() + c.Assert(w.subTaskEnabled.Get(), IsFalse) + + // prepare for test7 & 8 + t.testEnableRelay(c, w, etcdCli, sourceCfg, cfg) + // test7: when relayEnabled is true, switch on subtask + c.Assert(w.relayEnabled.Get(), IsTrue) + + subtaskCfg2 := subtaskCfg + subtaskCfg2.Name = "sub-task-name-2" + // we already added subtaskCfg, so below EnableHandleSubtasks will find an extra subtask + t.expectedCreateUnitCount++ + t.testEnableHandleSubtasks(c, w, etcdCli, subtaskCfg2, sourceCfg) + + // test8: when relayEnabled is true, switch off subtask + c.Assert(w.relayEnabled.Get(), IsTrue) + + w.DisableHandleSubtasks() + c.Assert(w.subTaskEnabled.Get(), IsFalse) +} + +func (t *testWorkerFunctionalities) testEnableRelay(c *C, w *Worker, etcdCli *clientv3.Client, + sourceCfg config.SourceConfig, cfg *Config) { + c.Assert(w.EnableRelay(), IsNil) + + c.Assert(w.relayEnabled.Get(), IsTrue) + c.Assert(w.relayHolder.Stage(), Equals, pb.Stage_New) + + _, err := ha.PutSourceCfg(etcdCli, sourceCfg) + c.Assert(err, IsNil) + _, err = ha.PutRelayStageSourceBound(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 { + return w.relayHolder.Stage() == pb.Stage_Running + }), IsTrue) + + _, err = ha.DeleteSourceCfgRelayStageSourceBound(etcdCli, sourceCfg.SourceID, cfg.Name) + c.Assert(err, IsNil) + c.Assert(utils.WaitSomething(30, 100*time.Millisecond, func() bool { + return w.relayHolder.Stage() == pb.Stage_Stopped + }), IsTrue) +} + +func (t *testWorkerFunctionalities) testDisableRelay(c *C, w *Worker) { + w.DisableRelay() + + c.Assert(w.relayEnabled.Get(), IsFalse) + c.Assert(w.relayHolder, IsNil) +} + +func (t *testWorkerFunctionalities) testEnableHandleSubtasks(c *C, w *Worker, etcdCli *clientv3.Client, + subtaskCfg config.SubTaskConfig, sourceCfg config.SourceConfig) { + c.Assert(w.EnableHandleSubtasks(), IsNil) + c.Assert(w.subTaskEnabled.Get(), IsTrue) + + _, err := ha.PutSubTaskCfgStage(etcdCli, []config.SubTaskConfig{subtaskCfg}, + []ha.Stage{ha.NewSubTaskStage(pb.Stage_Running, sourceCfg.SourceID, subtaskCfg.Name)}) + c.Assert(err, IsNil) + c.Assert(utils.WaitSomething(30, 100*time.Millisecond, func() bool { + return w.subTaskHolder.findSubTask(subtaskCfg.Name) != nil + }), IsTrue) + t.expectedCreateUnitCount++ + c.Assert(t.createUnitCount, Equals, t.expectedCreateUnitCount) +} + type testWorkerEtcdCompact struct{} var _ = Suite(&testWorkerEtcdCompact{}) @@ -293,7 +458,7 @@ func (t *testWorkerEtcdCompact) TestWatchSubtaskStageEtcdCompact(c *C) { []ha.Stage{ha.NewSubTaskStage(pb.Stage_Stopped, sourceCfg.SourceID, subtaskCfg.Name)}) c.Assert(err, IsNil) // step 2.1: start a subtask manually - c.Assert(w.StartSubTask(&subtaskCfg, pb.Stage_Running), IsNil) + c.Assert(w.StartSubTask(&subtaskCfg, pb.Stage_Running, true), IsNil) // step 3: trigger etcd compaction and check whether we can receive it through watcher _, err = etcdCli.Compact(ctx, rev) c.Assert(err, IsNil) diff --git a/syncer/syncer.go b/syncer/syncer.go index 4c0950fd8e..35ab156bde 100644 --- a/syncer/syncer.go +++ b/syncer/syncer.go @@ -206,12 +206,10 @@ func NewSyncer(cfg *config.SubTaskConfig, etcdClient *clientv3.Client) *Syncer { syncer.done = nil syncer.setTimezone() syncer.addJobFunc = syncer.addJob - // TODO: worker.relayEnabled syncer.enableRelay = cfg.UseRelay syncer.checkpoint = NewRemoteCheckPoint(syncer.tctx, cfg, syncer.checkpointID()) - // TODO: worker.relayEnabled syncer.binlogType = toBinlogType(cfg.UseRelay) syncer.errOperatorHolder = operator.NewHolder(&logger) syncer.readerHub = streamer.GetReaderHub() From fecd52605dde9c93a40b3555205a776b44ee9e0e Mon Sep 17 00:00:00 2001 From: lance6716 Date: Mon, 15 Mar 2021 09:07:52 +0800 Subject: [PATCH 17/46] improve unit test stability --- dm/worker/worker_test.go | 23 +++++++++++++++++------ 1 file changed, 17 insertions(+), 6 deletions(-) diff --git a/dm/worker/worker_test.go b/dm/worker/worker_test.go index c67bf7ac12..1b0bc723ef 100644 --- a/dm/worker/worker_test.go +++ b/dm/worker/worker_test.go @@ -17,6 +17,7 @@ import ( "context" "fmt" "sync" + "sync/atomic" "time" . "github.com/pingcap/check" @@ -219,8 +220,8 @@ func (t *testServer2) TestTaskAutoResume(c *C) { } type testWorkerFunctionalities struct { - createUnitCount int - expectedCreateUnitCount int + createUnitCount int32 + expectedCreateUnitCount int32 } var _ = Suite(&testWorkerFunctionalities{}) @@ -231,18 +232,20 @@ func (t *testWorkerFunctionalities) SetUpSuite(c *C) { return NewRealSubTask(cfg, etcdClient) } createUnits = func(cfg *config.SubTaskConfig, etcdClient *clientv3.Client) []unit.Unit { - t.createUnitCount++ + atomic.AddInt32(&t.createUnitCount, 1) mockDumper := NewMockUnit(pb.UnitType_Dump) mockLoader := NewMockUnit(pb.UnitType_Load) mockSync := NewMockUnit(pb.UnitType_Sync) return []unit.Unit{mockDumper, mockLoader, mockSync} } + getMinLocForSubTaskFunc = getFakeLocForSubTask } func (t *testWorkerFunctionalities) TearDownSuite(c *C) { NewRelayHolder = NewRealRelayHolder NewSubTask = NewRealSubTask createUnits = createRealUnits + getMinLocForSubTaskFunc = getMinLocForSubTask } func (t *testWorkerFunctionalities) TestWorkerFunctionalities(c *C) { @@ -304,7 +307,9 @@ func (t *testWorkerFunctionalities) TestWorkerFunctionalities(c *C) { t.testEnableRelay(c, w, etcdCli, sourceCfg, cfg) c.Assert(w.subTaskHolder.findSubTask(subtaskCfg.Name).cfg.UseRelay, IsTrue) t.expectedCreateUnitCount++ - c.Assert(t.createUnitCount, Equals, t.expectedCreateUnitCount) + c.Assert(utils.WaitSomething(30, 100*time.Millisecond, func() bool { + return atomic.LoadInt32(&t.createUnitCount) == t.expectedCreateUnitCount + }), IsTrue) // test5: when subTaskEnabled is true, switch off relay c.Assert(w.subTaskEnabled.Get(), IsTrue) @@ -312,7 +317,9 @@ func (t *testWorkerFunctionalities) TestWorkerFunctionalities(c *C) { c.Assert(w.subTaskHolder.findSubTask(subtaskCfg.Name).cfg.UseRelay, IsFalse) t.expectedCreateUnitCount++ - c.Assert(t.createUnitCount, Equals, t.expectedCreateUnitCount) + c.Assert(utils.WaitSomething(30, 100*time.Millisecond, func() bool { + return atomic.LoadInt32(&t.createUnitCount) == t.expectedCreateUnitCount + }), IsTrue) // test6: when relayEnabled is false, switch off subtask c.Assert(w.relayEnabled.Get(), IsFalse) @@ -330,6 +337,8 @@ func (t *testWorkerFunctionalities) TestWorkerFunctionalities(c *C) { // we already added subtaskCfg, so below EnableHandleSubtasks will find an extra subtask t.expectedCreateUnitCount++ t.testEnableHandleSubtasks(c, w, etcdCli, subtaskCfg2, sourceCfg) + c.Assert(w.subTaskHolder.findSubTask(subtaskCfg.Name).cfg.UseRelay, IsTrue) + c.Assert(w.subTaskHolder.findSubTask(subtaskCfg2.Name).cfg.UseRelay, IsTrue) // test8: when relayEnabled is true, switch off subtask c.Assert(w.relayEnabled.Get(), IsTrue) @@ -380,7 +389,9 @@ func (t *testWorkerFunctionalities) testEnableHandleSubtasks(c *C, w *Worker, et return w.subTaskHolder.findSubTask(subtaskCfg.Name) != nil }), IsTrue) t.expectedCreateUnitCount++ - c.Assert(t.createUnitCount, Equals, t.expectedCreateUnitCount) + c.Assert(utils.WaitSomething(30, 100*time.Millisecond, func() bool { + return atomic.LoadInt32(&t.createUnitCount) == t.expectedCreateUnitCount + }), IsTrue) } type testWorkerEtcdCompact struct{} From 8821a4066e9681ddea76cab48ff70a7a022805cd Mon Sep 17 00:00:00 2001 From: lance6716 Date: Mon, 15 Mar 2021 15:13:13 +0800 Subject: [PATCH 18/46] worker, ha: separate etcd message for subtask and relay --- dm/worker/metrics.go | 1 + dm/worker/server.go | 198 +++++++++++++++++++++++++++++++++++---- dm/worker/server_test.go | 2 +- pkg/ha/bound.go | 24 +++-- pkg/ha/ops.go | 10 +- pkg/ha/relay.go | 62 ++++++++++++ 6 files changed, 266 insertions(+), 31 deletions(-) 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..eea4a583be 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,6 +131,33 @@ 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) + 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 observeSourceBound + //nolint:errcheck + s.observeRelayConfig(ctx, revRelay) + }(s.ctx) + bound, sourceCfg, revBound, err := ha.GetSourceBoundConfig(s.etcdClient, s.cfg.Name) if err != nil { return err @@ -143,12 +171,6 @@ func (s *Server) Start() 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,84 @@ 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.stopWorker("") + if err != nil { + log.L().Error("fail to stop worker", zap.Error(err)) + return err // return if failed to stop the worker. + } + } else { + if w := s.getWorker(true); w != nil && w.cfg.SourceID == relaySource.SourceID { + 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(relaySource) + s.setSourceStatus(relaySource.SourceID, err2, true) + if err2 != nil { + w.l.Error("fail to recover observeRelayConfig", + zap.String("relay source", relaySource.SourceID), + zap.Error(err2)) + } + s.UpdateKeepAliveTTL(s.cfg.RelayKeepAliveTTL) + if err2 = w.EnableRelay(); 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 { @@ -319,12 +418,6 @@ func (s *Server) observeSourceBound(ctx context.Context, rev int64) error { } 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 = w.EnableHandleSubtasks() } s.setSourceStatus(bound.Source, err2, true) @@ -436,7 +529,6 @@ func (s *Server) stopWorker(sourceID string) error { s.Unlock() return terror.ErrWorkerSourceNotMatch } - // TODO: and when disable relay s.UpdateKeepAliveTTL(s.cfg.KeepAliveTTL) s.setWorker(nil, false) s.setSourceStatus("", nil, false) @@ -480,9 +572,54 @@ 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) + // TODO: will worker be modified on other goroutine? + w := s.getWorker(true) + w.DisableHandleSubtasks() + if !w.relayEnabled.Get() { + return s.stopWorker(bound.Source) + } + return nil } scm, _, err := ha.GetSourceCfg(s.etcdClient, bound.Source, bound.Revision) if err != nil { @@ -497,13 +634,34 @@ func (s *Server) operateSourceBound(bound ha.SourceBound) error { 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) operateRelaySource(relaySource ha.RelaySource) error { + if relaySource.IsDeleted { + // TODO: will worker be modified on other goroutine? + w := s.getWorker(true) + s.UpdateKeepAliveTTL(s.cfg.KeepAliveTTL) + w.DisableRelay() + if !w.subTaskEnabled.Get() { + return s.stopWorker(relaySource.Source) } + return nil } - return w.EnableHandleSubtasks() + 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) + if err != nil { + return err + } + return w.EnableRelay() } // QueryStatus implements WorkerServer.QueryStatus diff --git a/dm/worker/server_test.go b/dm/worker/server_test.go index 787da5782b..26ed8a7ab3 100644 --- a/dm/worker/server_test.go +++ b/dm/worker/server_test.go @@ -488,7 +488,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/pkg/ha/bound.go b/pkg/ha/bound.go index 8b8e32dd28..dfc50dfc12 100644 --- a/pkg/ha/bound.go +++ b/pkg/ha/bound.go @@ -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 @@ -114,13 +114,14 @@ 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)) + 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...) _, rev, err := etcdutil.DoOpsInOneTxnWithRetry(cli, ops...) return rev, err } @@ -351,8 +352,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 enabled 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. @@ -371,6 +376,9 @@ func putSourceBoundOp(bound SourceBound) ([]clientv3.Op, error) { op1 := clientv3.OpPut(key1, value) key2 := common.UpstreamLastBoundWorkerKeyAdapter.Encode(bound.Worker) op2 := clientv3.OpPut(key2, value) + // TODO: move this to start-relay, and wait until worker has enabled relay + key3 := common.UpstreamRelayWorkerKeyAdapter.Encode(bound.Worker) + op3 := clientv3.OpPut(key3, bound.Source) - return []clientv3.Op{op1, op2}, nil + return []clientv3.Op{op1, op2, op3}, nil } diff --git a/pkg/ha/ops.go b/pkg/ha/ops.go index 03cced27b8..a60e644b8b 100644 --- a/pkg/ha/ops.go +++ b/pkg/ha/ops.go @@ -33,7 +33,7 @@ func PutRelayStageSourceBound(cli *clientv3.Client, stage Stage, bound SourceBou if err != nil { return 0, err } - ops := make([]clientv3.Op, 0, 3) + ops := make([]clientv3.Op, 0, len(ops1)+len(op2)) ops = append(ops, ops1...) ops = append(ops, op2...) _, rev, err := etcdutil.DoOpsInOneTxnWithRetry(cli, ops...) @@ -49,7 +49,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/relay.go b/pkg/ha/relay.go index 9103ef88ea..b7bbbfdfda 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,6 +28,16 @@ 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. func PutRelayConfig(cli *clientv3.Client, source string, workers ...string) (int64, error) { @@ -140,3 +151,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 + } + } + } + } + +} From f07796451bef033d58e619f6c32cf3d410f447f3 Mon Sep 17 00:00:00 2001 From: lance6716 Date: Tue, 16 Mar 2021 10:17:13 +0800 Subject: [PATCH 19/46] we enabled relay for every source in last commit, fix it --- dm/master/scheduler/scheduler.go | 49 ++++++++++++++----- dm/master/scheduler/scheduler_test.go | 5 +- dm/master/server_test.go | 2 + dm/worker/join.go | 11 ----- dm/worker/server.go | 3 +- dm/worker/server_test.go | 2 +- dm/worker/subtask.go | 2 + dm/worker/subtask_holder.go | 8 +++ dm/worker/worker_test.go | 4 +- pkg/ha/bound.go | 13 ++--- pkg/ha/keepalive.go | 14 ++++-- pkg/ha/ops.go | 9 ++-- pkg/ha/ops_test.go | 2 +- pkg/ha/relay.go | 1 + tests/ha_cases/run.sh | 5 +- tests/import_goroutine_leak/conf/source1.yaml | 2 +- tests/import_goroutine_leak/run.sh | 2 +- 17 files changed, 88 insertions(+), 46 deletions(-) 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..007c42f979 100644 --- a/dm/master/server_test.go +++ b/dm/master/server_test.go @@ -919,6 +919,8 @@ func (t *testMaster) TestOperateWorkerRelayTask(c *check.C) { // 1. test pause-relay successfully resp, err = server.OperateWorkerRelayTask(context.Background(), pauseReq) c.Assert(err, check.IsNil) + fmt.Println(resp) + // you didn't create relay stage c.Assert(resp.Result, check.IsTrue) for _, source := range sources { t.relayStageMatch(c, server.scheduler, source, pb.Stage_Paused) diff --git a/dm/worker/join.go b/dm/worker/join.go index b6d366df20..18b4c8b394 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" @@ -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/server.go b/dm/worker/server.go index eea4a583be..e19bc54273 100644 --- a/dm/worker/server.go +++ b/dm/worker/server.go @@ -153,7 +153,7 @@ func (s *Server) Start() error { s.wg.Add(1) go func(ctx context.Context) { defer s.wg.Done() - // TODO: handle fatal error from observeSourceBound + // TODO: handle fatal error from observeRelayConfig //nolint:errcheck s.observeRelayConfig(ctx, revRelay) }(s.ctx) @@ -661,6 +661,7 @@ func (s *Server) operateRelaySource(relaySource ha.RelaySource) error { if err != nil { return err } + s.UpdateKeepAliveTTL(s.cfg.RelayKeepAliveTTL) return w.EnableRelay() } diff --git a/dm/worker/server_test.go b/dm/worker/server_test.go index 26ed8a7ab3..194ad44baf 100644 --- a/dm/worker/server_test.go +++ b/dm/worker/server_test.go @@ -472,7 +472,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 diff --git a/dm/worker/subtask.go b/dm/worker/subtask.go index e798591f5f..596fd683ec 100644 --- a/dm/worker/subtask.go +++ b/dm/worker/subtask.go @@ -171,6 +171,7 @@ func (st *SubTask) Init() error { return terror.Annotatef(err, "fail to get fresh status of subtask %s %s", st.cfg.Name, u.Type()) } else if !isFresh { skipIdx = i + // TODO: why after print it, nothing continued? st.l.Info("continue unit", zap.Stringer("unit", u.Type())) break } @@ -215,6 +216,7 @@ func (st *SubTask) run() { st.fail(err) return } else if ctx.Err() != nil { + st.l.Error("exit SubTask.run because of", log.ShortError(ctx.Err())) return } diff --git a/dm/worker/subtask_holder.go b/dm/worker/subtask_holder.go index e8754ca6fa..b6c5fc737e 100644 --- a/dm/worker/subtask_holder.go +++ b/dm/worker/subtask_holder.go @@ -14,7 +14,12 @@ package worker import ( + "context" "sync" + + "go.uber.org/zap" + + "github.com/pingcap/dm/pkg/log" ) // subTaskHolder holds subtask instances. @@ -53,7 +58,10 @@ 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 + log.L().Warn("lance test", zap.Any("stage", stage)) st.Run(stage) } } diff --git a/dm/worker/worker_test.go b/dm/worker/worker_test.go index 1b0bc723ef..4be727848b 100644 --- a/dm/worker/worker_test.go +++ b/dm/worker/worker_test.go @@ -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/pkg/ha/bound.go b/pkg/ha/bound.go index dfc50dfc12..b8108301e7 100644 --- a/pkg/ha/bound.go +++ b/pkg/ha/bound.go @@ -113,7 +113,8 @@ 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) { +// 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 { @@ -122,6 +123,9 @@ func ReplaceSourceBound(cli *clientv3.Client, source, oldWorker, newWorker strin 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 } @@ -353,7 +357,7 @@ 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 { - // TODO: move this to stop-relay, and wait until worker has enabled relay + // 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)), @@ -376,9 +380,6 @@ func putSourceBoundOp(bound SourceBound) ([]clientv3.Op, error) { op1 := clientv3.OpPut(key1, value) key2 := common.UpstreamLastBoundWorkerKeyAdapter.Encode(bound.Worker) op2 := clientv3.OpPut(key2, value) - // TODO: move this to start-relay, and wait until worker has enabled relay - key3 := common.UpstreamRelayWorkerKeyAdapter.Encode(bound.Worker) - op3 := clientv3.OpPut(key3, bound.Source) - return []clientv3.Op{op1, op2, op3}, nil + return []clientv3.Op{op1, op2}, nil } 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 a60e644b8b..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, len(ops1)+len(op2)) + 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 } 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 b7bbbfdfda..2baaf8f985 100644 --- a/pkg/ha/relay.go +++ b/pkg/ha/relay.go @@ -40,6 +40,7 @@ type RelaySource struct { // 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 { 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/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 From ac1ebbcf298bbf0503e1e26dcd9d02a0942e0d68 Mon Sep 17 00:00:00 2001 From: lance6716 Date: Mon, 22 Mar 2021 14:15:13 +0800 Subject: [PATCH 20/46] fix checkpoint wasn't flush --- dm/worker/subtask_holder.go | 5 ----- syncer/syncer.go | 11 ++++++++--- 2 files changed, 8 insertions(+), 8 deletions(-) diff --git a/dm/worker/subtask_holder.go b/dm/worker/subtask_holder.go index b6c5fc737e..7627db23af 100644 --- a/dm/worker/subtask_holder.go +++ b/dm/worker/subtask_holder.go @@ -16,10 +16,6 @@ package worker import ( "context" "sync" - - "go.uber.org/zap" - - "github.com/pingcap/dm/pkg/log" ) // subTaskHolder holds subtask instances. @@ -61,7 +57,6 @@ func (h *subTaskHolder) resetAllSubTasks(useRelay bool) { // TODO: make a st.reset st.ctx, st.cancel = context.WithCancel(context.Background()) st.cfg.UseRelay = useRelay - log.L().Warn("lance test", zap.Any("stage", stage)) st.Run(stage) } } diff --git a/syncer/syncer.go b/syncer/syncer.go index 35ab156bde..fdaa3b3162 100644 --- a/syncer/syncer.go +++ b/syncer/syncer.go @@ -863,8 +863,13 @@ 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() + if err != nil && !utils.IsContextCanceledError(err) { + // is its a user cancel, still need to flush checkpoint, because we might already put shard info, and other + // worker has dropped a column so we must continue on this position after resuming + 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 +890,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) } From e59bbfada9983d6bbeeb63ea47ec03abead1a18b Mon Sep 17 00:00:00 2001 From: lance6716 Date: Mon, 22 Mar 2021 14:24:28 +0800 Subject: [PATCH 21/46] fix etcd port conflict --- dm/worker/server_test.go | 9 ++++++--- 1 file changed, 6 insertions(+), 3 deletions(-) diff --git a/dm/worker/server_test.go b/dm/worker/server_test.go index 194ad44baf..50d2fd7e0d 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 @@ -349,7 +351,7 @@ func (t *testServer) TestHandleSourceBoundAfterError(c *C) { 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 +361,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 From cbf709d56889f94b8635a3fe2e7d22cc9fa7fd23 Mon Sep 17 00:00:00 2001 From: lance6716 Date: Mon, 22 Mar 2021 17:37:15 +0800 Subject: [PATCH 22/46] leave some problems to be fixed in future --- syncer/syncer.go | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/syncer/syncer.go b/syncer/syncer.go index fdaa3b3162..3e234406f3 100644 --- a/syncer/syncer.go +++ b/syncer/syncer.go @@ -864,9 +864,11 @@ 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 { err := s.execError.Get() - if err != nil && !utils.IsContextCanceledError(err) { - // is its a user cancel, still need to flush checkpoint, because we might already put shard info, and other - // worker has dropped a column so we must continue on this position after resuming + // 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)) From 5395abb859b34d31bc30c578251033533b5ee63b Mon Sep 17 00:00:00 2001 From: lance6716 Date: Wed, 17 Mar 2021 15:38:07 +0800 Subject: [PATCH 23/46] *: add start-relay/stop-relay command --- _utils/terror_gen/errors_release.txt | 8 +- dm/ctl/ctl.go | 2 + dm/ctl/master/start_relay.go | 84 ++ dm/ctl/master/stop_relay.go | 84 ++ dm/master/scheduler/scheduler.go | 306 ++++++- dm/master/scheduler/scheduler_test.go | 94 ++- dm/master/server.go | 41 +- dm/pb/dmmaster.pb.go | 791 ++++++++++++++++--- dm/proto/dmmaster.proto | 19 + dm/worker/server.go | 5 +- dm/worker/server_test.go | 4 +- dm/worker/worker.go | 2 + dm/worker/worker_test.go | 3 +- errors.toml | 30 +- pkg/ha/bound.go | 2 - pkg/ha/relay.go | 36 + pkg/ha/stage.go | 6 + pkg/ha/worker.go | 10 +- pkg/ha/worker_test.go | 2 +- pkg/terror/error_list.go | 12 +- syncer/syncer.go | 7 + tests/all_mode/run.sh | 11 +- tests/dmctl_basic/check_list/pause_relay.sh | 2 +- tests/dmctl_basic/check_list/purge_relay.sh | 2 +- tests/dmctl_basic/check_list/query_status.sh | 2 +- tests/dmctl_basic/check_list/start_relay.sh | 34 + tests/dmctl_basic/check_list/stop_relay.sh | 34 + tests/dmctl_basic/run.sh | 25 +- tests/dmctl_command/run.sh | 2 +- tests/drop_column_with_index/run.sh | 4 + tests/full_mode/run.sh | 10 +- tests/gtid/run.sh | 3 + tests/ha/run.sh | 13 +- tests/ha_cases/lib.sh | 24 +- tests/ha_cases/run.sh | 94 ++- tests/ha_master/run.sh | 6 + tests/handle_error/run.sh | 8 +- tests/http_apis/run.sh | 4 +- tests/import_v10x/run.sh | 2 +- tests/incremental_mode/run.sh | 30 +- tests/initial_unit/run.sh | 10 +- tests/online_ddl/run.sh | 14 +- tests/only_dml/run.sh | 7 + tests/relay_interrupt/run.sh | 6 +- tests/sequence_sharding_optimistic/run.sh | 6 + tests/sharding2/run.sh | 10 +- 46 files changed, 1660 insertions(+), 251 deletions(-) create mode 100644 dm/ctl/master/start_relay.go create mode 100644 dm/ctl/master/stop_relay.go create mode 100644 tests/dmctl_basic/check_list/start_relay.sh create mode 100644 tests/dmctl_basic/check_list/stop_relay.sh diff --git a/_utils/terror_gen/errors_release.txt b/_utils/terror_gen/errors_release.txt index 1c8f20f246..c72e8e1af7 100644 --- a/_utils/terror_gen/errors_release.txt +++ b/_utils/terror_gen/errors_release.txt @@ -482,7 +482,7 @@ ErrSchedulerWorkerInvalidTrans,[code=46006:class=scheduler:scope=internal:level= ErrSchedulerSourceCfgExist,[code=46007:class=scheduler:scope=internal:level=medium], "Message: source config with ID %s already exists" ErrSchedulerSourceCfgNotExist,[code=46008:class=scheduler:scope=internal:level=medium], "Message: source config with ID %s not exists" ErrSchedulerSourcesUnbound,[code=46009:class=dm-master:scope=internal:level=medium], "Message: sources %v have not bound" -ErrSchedulerSourceOpTaskExist,[code=46010:class=dm-master:scope=internal:level=medium], "Message: source with name %s need to operate with tasks %v exist" +ErrSchedulerSourceOpTaskExist,[code=46010:class=dm-master:scope=internal:level=medium], "Message: source with name %s need to operate has existing tasks %v, Workaround: Please `stop-task` first." ErrSchedulerRelayStageInvalidUpdate,[code=46011:class=scheduler:scope=internal:level=medium], "Message: invalid new expectant relay stage %s" ErrSchedulerRelayStageSourceNotExist,[code=46012:class=scheduler:scope=internal:level=medium], "Message: sources %v need to update expectant relay stage not exist" ErrSchedulerMultiTask,[code=46013:class=scheduler:scope=internal:level=medium], "Message: the scheduler cannot perform multiple different tasks %v in one operation" @@ -491,7 +491,11 @@ ErrSchedulerSubTaskStageInvalidUpdate,[code=46015:class=dm-master:scope=internal ErrSchedulerSubTaskOpTaskNotExist,[code=46016:class=dm-master:scope=internal:level=medium], "Message: subtasks with name %s need to be operate not exist, Workaround: Please use `query-status` command to see tasks." ErrSchedulerSubTaskOpSourceNotExist,[code=46017:class=dm-master:scope=internal:level=medium], "Message: sources %v need to be operate not exist" ErrSchedulerTaskNotExist,[code=46018:class=scheduler:scope=internal:level=medium], "Message: task with name %s not exist, Workaround: Please use `query-status` command to see tasks." -ErrSchedulerRequireNotRunning,[code=46019:class=scheduler:scope=internal:level=high], "Message: tasks %v on source %s should not be running, Workaround: Please use `pause-task [-s source ...] task` to pause them first" +ErrSchedulerRequireNotRunning,[code=46019:class=scheduler:scope=internal:level=high], "Message: tasks %v on source %s should not be running, Workaround: Please use `pause-task [-s source ...] task` to pause them first." +ErrSchedulerRelayWorkersBusy,[code=46020:class=scheduler:scope=internal:level=high], "Message: these workers %s have started relay for sources %s respectively, Workaround: Please use `stop-relay` to stop them, or change your topology." +ErrSchedulerRelayWorkersWrongBound,[code=46021:class=scheduler:scope=internal:level=high], "Message: these workers %s have bound for another sources %s respectively, Workaround: Please `start-relay` on free or same source workers." +ErrSchedulerRelayWorkersWrongRelay,[code=46022:class=scheduler:scope=internal:level=high], "Message: these workers %s have started relay for another sources %s respectively, Workaround: Please correct sources in `stop-relay`." +ErrSchedulerSourceOpRelayExist,[code=46023:class=scheduler:scope=internal:level=high], "Message: source with name %s need to operate has existing relay workers %s, Workaround: Please `stop-relay` first." ErrCtlGRPCCreateConn,[code=48001:class=dmctl:scope=internal:level=high], "Message: can not create grpc connection, Workaround: Please check your network connection." ErrCtlInvalidTLSCfg,[code=48002:class=dmctl:scope=internal:level=medium], "Message: invalid TLS config, Workaround: Please check the `ssl-ca`, `ssl-cert` and `ssl-key` config in command line." ErrNotSet,[code=50000:class=not-set:scope=not-set:level=high] diff --git a/dm/ctl/ctl.go b/dm/ctl/ctl.go index 4a7a18646c..e40b848b54 100644 --- a/dm/ctl/ctl.go +++ b/dm/ctl/ctl.go @@ -75,6 +75,8 @@ func NewRootCmd() *cobra.Command { master.NewGetCfgCmd(), master.NewHandleErrorCmd(), master.NewTransferSourceCmd(), + master.NewStartRelayCmd(), + master.NewStopRelayCmd(), ) // copied from (*cobra.Command).InitDefaultHelpCmd helpCmd := &cobra.Command{ diff --git a/dm/ctl/master/start_relay.go b/dm/ctl/master/start_relay.go new file mode 100644 index 0000000000..da4bfdc1aa --- /dev/null +++ b/dm/ctl/master/start_relay.go @@ -0,0 +1,84 @@ +// 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 master + +import ( + "context" + "errors" + "os" + + "github.com/pingcap/dm/dm/ctl/common" + "github.com/pingcap/dm/dm/pb" + + "github.com/spf13/cobra" +) + +// NewStartRelayCmd creates a StartRelay command +func NewStartRelayCmd() *cobra.Command { + cmd := &cobra.Command{ + Use: "start-relay <-s source-id> [...worker-name]", + Short: "Starts workers pulling relay log for a source.", + RunE: startRelayFunc, + } + return cmd +} + +func startRelayFunc(cmd *cobra.Command, _ []string) (err error) { + sources, err := common.GetSourceArgs(cmd) + if err != nil { + return err + } + + if len(cmd.Flags().Args()) == 0 { + cmd.SetOut(os.Stdout) + if len(sources) == 0 { + // all args empty + common.PrintCmdUsage(cmd) + } else { + common.PrintLines("must specify at least one worker") + } + err = errors.New("please check output to see error") + return + } + + if len(sources) != 1 { + common.PrintLines("must specify one source (`-s` / `--source`)") + err = errors.New("please check output to see error") + return + } + + workers := cmd.Flags().Args() + + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + + resp := &pb.OperateRelayResponse{} + err = common.SendRequest( + ctx, + "OperateRelay", + &pb.OperateRelayRequest{ + Op: pb.RelayOpV2_StartRelayV2, + Source: sources[0], + Worker: workers, + }, + &resp, + ) + + if err != nil { + return + } + + common.PrettyPrintResponse(resp) + return +} diff --git a/dm/ctl/master/stop_relay.go b/dm/ctl/master/stop_relay.go new file mode 100644 index 0000000000..660a8467bc --- /dev/null +++ b/dm/ctl/master/stop_relay.go @@ -0,0 +1,84 @@ +// 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 master + +import ( + "context" + "errors" + "os" + + "github.com/pingcap/dm/dm/ctl/common" + "github.com/pingcap/dm/dm/pb" + + "github.com/spf13/cobra" +) + +// NewStopRelayCmd creates a StartRelay command +func NewStopRelayCmd() *cobra.Command { + cmd := &cobra.Command{ + Use: "stop-relay <-s source-id> [...worker-name]", + Short: "Stops workers pulling relay log for a source.", + RunE: stopRelayFunc, + } + return cmd +} + +func stopRelayFunc(cmd *cobra.Command, _ []string) (err error) { + sources, err := common.GetSourceArgs(cmd) + if err != nil { + return err + } + + if len(cmd.Flags().Args()) == 0 { + cmd.SetOut(os.Stdout) + if len(sources) == 0 { + // all args empty + common.PrintCmdUsage(cmd) + } else { + common.PrintLines("must specify at least one worker") + } + err = errors.New("please check output to see error") + return + } + + if len(sources) != 1 { + common.PrintLines("must specify one source (`-s` / `--source`)") + err = errors.New("please check output to see error") + return + } + + workers := cmd.Flags().Args() + + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + + resp := &pb.OperateRelayResponse{} + err = common.SendRequest( + ctx, + "OperateRelay", + &pb.OperateRelayRequest{ + Op: pb.RelayOpV2_StopRelayV2, + Source: sources[0], + Worker: workers, + }, + &resp, + ) + + if err != nil { + return + } + + common.PrettyPrintResponse(resp) + return +} diff --git a/dm/master/scheduler/scheduler.go b/dm/master/scheduler/scheduler.go index f5baf55420..9e85716191 100644 --- a/dm/master/scheduler/scheduler.go +++ b/dm/master/scheduler/scheduler.go @@ -136,6 +136,14 @@ type Scheduler struct { // - remove/stop subtask by user request (calling `RemoveSubTasks`). expectSubTaskStages map[string]map[string]ha.Stage + // a source has its relay workers. source-id -> set(worker-name) + // add: + // - start-relay + // - recover from etcd (calling `recoverRelayConfigs`) + // delete: + // - stop-relay + relayWorkers map[string]map[string]struct{} + securityCfg config.Security } @@ -151,6 +159,7 @@ func NewScheduler(pLogger *log.Logger, securityCfg config.Security) *Scheduler { lastBound: make(map[string]ha.SourceBound), expectRelayStages: make(map[string]ha.Stage), expectSubTaskStages: make(map[string]map[string]ha.Stage), + relayWorkers: make(map[string]map[string]struct{}), securityCfg: securityCfg, } } @@ -179,6 +188,10 @@ func (s *Scheduler) Start(pCtx context.Context, etcdCli *clientv3.Client) error if err != nil { return err } + err = s.recoverRelayConfigs(etcdCli) + if err != nil { + return err + } rev, err := s.recoverWorkersBounds(etcdCli) if err != nil { return err @@ -279,7 +292,7 @@ func (s *Scheduler) RemoveSourceCfg(source string) error { return terror.ErrSchedulerSourceCfgNotExist.Generate(source) } - // 2. check whether any subtask exists for the source. + // 2. check whether any subtask or relay config exists for the source. existingSubtasksM := make(map[string]struct{}) for task, cfg := range s.subTaskCfgs { for source2 := range cfg { @@ -292,13 +305,17 @@ func (s *Scheduler) RemoveSourceCfg(source string) error { if len(existingSubtasks) > 0 { return terror.ErrSchedulerSourceOpTaskExist.Generate(source, existingSubtasks) } + relayWorkers := s.relayWorkers[source] + if len(relayWorkers) != 0 { + return terror.ErrSchedulerSourceOpRelayExist.Generate(source, strMapToSlice(relayWorkers)) + } // 3. find worker name by source ID. var ( workerName string // empty should be fine below. worker *Worker ) - if w, ok := s.bounds[source]; ok { + if w, ok2 := s.bounds[source]; ok2 { worker = w workerName = w.BaseInfo().Name } @@ -375,18 +392,26 @@ func (s *Scheduler) TransferSource(source, worker string) error { return nil } - // 2. check new worker is free + // 2. check new worker is free and not started relay for another source stage := w.Stage() if stage != WorkerFree { return terror.ErrSchedulerWorkerInvalidTrans.Generate(worker, stage, WorkerBound) } + for source2, workers := range s.relayWorkers { + if source2 == source { + continue + } + if _, ok2 := workers[worker]; ok2 { + return terror.ErrSchedulerRelayWorkersBusy.Generate(worker, source2) + } + } // 3. if no old worker, bound it directly if !hasOldWorker { 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, s.sourceCfgs[source].EnableRelay) + err := s.boundSourceToWorker(source, w) if err == nil { delete(s.unbounds, source) } @@ -685,7 +710,7 @@ func (s *Scheduler) RemoveWorker(name string) error { } // delete the info in etcd. - _, err := ha.DeleteWorkerInfo(s.etcdCli, name) + _, err := ha.DeleteWorkerInfoRelayConfig(s.etcdCli, name) if err != nil { return err } @@ -748,6 +773,170 @@ func (s *Scheduler) UnboundSources() []string { return IDs } +// StartRelay puts etcd key-value pairs to start relay on some workers +func (s *Scheduler) StartRelay(source string, workers []string) error { + s.mu.Lock() + defer s.mu.Unlock() + + if !s.started { + return terror.ErrSchedulerNotStarted.Generate() + } + + // TODO: remove old relay config, where caused by EnableRelay + + // 1. precheck + if _, ok := s.sourceCfgs[source]; !ok { + return terror.ErrSchedulerSourceCfgNotExist.Generate(source) + } + startedWorkers := s.relayWorkers[source] + if startedWorkers == nil { + startedWorkers = map[string]struct{}{} + s.relayWorkers[source] = startedWorkers + } + var ( + notExistWorkers []string + // below two list means the worker that requested start-relay has bound to another source + boundWorkers, boundSources []string + alreadyStarted []string + ) + for _, worker := range workers { + if _, ok := s.workers[worker]; !ok { + notExistWorkers = append(notExistWorkers, worker) + } + if _, ok := startedWorkers[worker]; ok { + alreadyStarted = append(alreadyStarted, worker) + } + + for source2, w := range s.bounds { + if source2 == source { + continue + } + if w.BaseInfo().Name == worker { + boundWorkers = append(boundWorkers, worker) + boundSources = append(boundSources, source2) + } + } + } + if len(notExistWorkers) > 0 { + return terror.ErrSchedulerWorkerNotExist.Generate(notExistWorkers) + } + if len(boundWorkers) > 0 { + return terror.ErrSchedulerRelayWorkersWrongBound.Generate(boundWorkers, boundSources) + } + if len(alreadyStarted) > 0 { + s.logger.Warn("some workers already started relay", + zap.String("source", source), + zap.Strings("already started workers", alreadyStarted)) + } + + // currently we forbid one worker starting multiple relay + // worker -> source + oldSource := map[string]string{} + for source2, workers2 := range s.relayWorkers { + if source2 == source { + continue + } + for w := range workers2 { + oldSource[w] = source2 + } + } + var ( + busyWorkers, busySources []string + ) + for _, w := range workers { + source2 := oldSource[w] + if source2 != "" { + busyWorkers = append(busyWorkers, w) + busySources = append(busySources, source2) + } + } + if len(busyWorkers) > 0 { + return terror.ErrSchedulerRelayWorkersBusy.Generate(busyWorkers, busySources) + } + + // 2. put etcd and update memory cache + // if there's no relay stage, create a running one. otherwise we should respect paused stage + if len(startedWorkers) == 0 { + stage := ha.NewRelayStage(pb.Stage_Running, source) + if _, err := ha.PutRelayStage(s.etcdCli, stage); err != nil { + return err + } + s.expectRelayStages[source] = stage + } + if _, err := ha.PutRelayConfig(s.etcdCli, source, workers...); err != nil { + return err + } + for _, w := range workers { + s.relayWorkers[source][w] = struct{}{} + } + return nil +} + +// StopRelay deletes etcd key-value pairs to stop relay on some workers +func (s *Scheduler) StopRelay(source string, workers []string) error { + s.mu.Lock() + defer s.mu.Unlock() + + if !s.started { + return terror.ErrSchedulerNotStarted.Generate() + } + + // 1. precheck + if _, ok := s.sourceCfgs[source]; !ok { + return terror.ErrSchedulerSourceCfgNotExist.Generate(source) + } + startedWorkers := s.relayWorkers[source] + var ( + notExistWorkers []string + unmatchedWorkers, unmatchedSources []string + alreadyStopped []string + ) + for _, worker := range workers { + if _, ok := s.workers[worker]; !ok { + notExistWorkers = append(notExistWorkers, worker) + } + if _, ok := startedWorkers[worker]; !ok { + alreadyStopped = append(alreadyStopped, worker) + } + for source2, workers2 := range s.relayWorkers { + if source2 == source { + continue + } + if _, ok := workers2[worker]; ok { + unmatchedWorkers = append(unmatchedWorkers, worker) + unmatchedSources = append(unmatchedSources, source2) + } + } + } + if len(notExistWorkers) > 0 { + return terror.ErrSchedulerWorkerNotExist.Generate(notExistWorkers) + } + if len(unmatchedWorkers) > 0 { + return terror.ErrSchedulerRelayWorkersWrongRelay.Generate(unmatchedWorkers, unmatchedSources) + } + if len(alreadyStopped) > 0 { + s.logger.Warn("some workers already stopped relay", + zap.String("source", source), + zap.Strings("already stopped workers", alreadyStopped)) + } + + // 2. delete from etcd and update memory cache + if _, err := ha.DeleteRelayConfig(s.etcdCli, workers...); err != nil { + return err + } + for _, w := range workers { + delete(s.relayWorkers[source], w) + } + if len(s.relayWorkers[source]) == 0 { + if _, err := ha.DeleteRelayStage(s.etcdCli, source); err != nil { + return err + } + delete(s.relayWorkers, source) + delete(s.expectRelayStages, source) + } + return nil +} + // UpdateExpectRelayStage updates the current expect relay stage. // now, only support updates: // - from `Running` to `Paused`. @@ -976,6 +1165,16 @@ func (s *Scheduler) recoverSubTasks(cli *clientv3.Client) error { return nil } +// recoverRelayConfigs recovers history relay configs for each worker from etcd. +func (s *Scheduler) recoverRelayConfigs(cli *clientv3.Client) error { + relayWorkers, _, err := ha.GetAllRelayConfig(cli) + if err != nil { + return err + } + s.relayWorkers = relayWorkers + return nil +} + // recoverWorkersBounds recovers history DM-worker info and status from etcd. // and it also recovers the bound/unbound relationship. func (s *Scheduler) recoverWorkersBounds(cli *clientv3.Client) (int64, error) { @@ -1043,13 +1242,6 @@ 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 @@ -1194,9 +1386,14 @@ 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 + // also put identical relay config for this worker + for source, workers := range s.relayWorkers { + if _, ok2 := workers[w.BaseInfo().Name]; ok2 { + _, err := ha.PutRelayConfig(s.etcdCli, source, w.BaseInfo().Name) + if err != nil { + return err + } + break } } // TODO: When dm-worker keepalive is broken, it will turn off its own running source @@ -1274,12 +1471,37 @@ func (s *Scheduler) handleWorkerOffline(ev ha.WorkerEvent, toLock bool) error { func (s *Scheduler) tryBoundForWorker(w *Worker) (bounded bool, err error) { // 1. check if last bound is still available. // if lastBound not found, or this source has been bounded to another worker (we also check that source still exists - // here), randomly pick one from unbounds. + // here), use its relay source. if no relay source, randomly pick one from unbounds. // NOTE: if worker isn't in lastBound, we'll get "zero" SourceBound and it's OK, because "zero" string is not in // unbounds source := s.lastBound[w.baseInfo.Name].Source if _, ok := s.unbounds[source]; !ok { source = "" + } + + // try to find its relay source (currently only one relay source) + if source == "" { + for source2, workers := range s.relayWorkers { + if _, ok2 := workers[w.BaseInfo().Name]; ok2 { + source = source2 + break + } + } + } + // found a relay source + if source != "" { + // currently worker can only handle same relay source and source bound, so we don't try bound another source + if oldWorker, ok := s.bounds[source]; ok { + s.logger.Info("worker has started relay for a source, but that source is bound to another worker, so we let this worker free", + zap.String("worker", w.BaseInfo().Name), + zap.String("relay source", source), + zap.String("bound worker for its relay source", oldWorker.BaseInfo().Name)) + return false, nil + } + } + + // randomly pick one from unbounds + if source == "" { for source = range s.unbounds { break // got a source. } @@ -1301,7 +1523,7 @@ func (s *Scheduler) tryBoundForWorker(w *Worker) (bounded bool, err error) { }() // 3. try to bound them. - err = s.boundSourceToWorker(source, w, s.sourceCfgs[source].EnableRelay) + err = s.boundSourceToWorker(source, w) if err != nil { return false, err } @@ -1310,8 +1532,9 @@ func (s *Scheduler) tryBoundForWorker(w *Worker) (bounded bool, err error) { // tryBoundForSource tries to bound a source to a random Free worker. // returns (true, nil) after bounded. +// called should update the s.unbounds func (s *Scheduler) tryBoundForSource(source string) (bool, error) { - // 1. try to find history workers, then random Free worker. + // 1. try to find history workers... var worker *Worker for workerName, bound := range s.lastBound { if bound.Source == source { @@ -1326,7 +1549,21 @@ func (s *Scheduler) tryBoundForSource(source string) (bool, error) { } } } - + // then a relay worker for this source... + if worker == nil { + for workerName := range s.relayWorkers[source] { + w, ok := s.workers[workerName] + if !ok { + // a not found worker, should not happened + continue + } + if w.Stage() == WorkerFree { + worker = w + break + } + } + } + // and then a random Free worker. if worker == nil { for _, w := range s.workers { if w.Stage() == WorkerFree { @@ -1342,7 +1579,7 @@ func (s *Scheduler) tryBoundForSource(source string) (bool, error) { } // 2. try to bound them. - err := s.boundSourceToWorker(source, worker, s.sourceCfgs[source].EnableRelay) + err := s.boundSourceToWorker(source, worker) if err != nil { return false, err } @@ -1351,32 +1588,11 @@ 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, enableRelay bool) error { +func (s *Scheduler) boundSourceToWorker(source string, w *Worker) 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 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.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) - } + _, err = ha.PutSourceBound(s.etcdCli, bound) if err != nil { return err } @@ -1407,6 +1623,9 @@ func (s *Scheduler) recordWorker(info ha.WorkerInfo) (*Worker, error) { // this func is used when removing the worker. // NOTE: trigger scheduler when the worker become offline, not when deleted. func (s *Scheduler) deleteWorker(name string) { + for _, workers := range s.relayWorkers { + delete(workers, name) + } w, ok := s.workers[name] if !ok { return @@ -1420,6 +1639,7 @@ func (s *Scheduler) deleteWorker(name string) { // - update the stage of worker to `Bound`. // - record the bound relationship and last bound relationship in the scheduler. // this func is called after the bound relationship existed in etcd. +// TODO: update s.unbounds here func (s *Scheduler) updateStatusForBound(w *Worker, b ha.SourceBound) error { err := w.ToBound(b) if err != nil { diff --git a/dm/master/scheduler/scheduler_test.go b/dm/master/scheduler/scheduler_test.go index 3c7362df2d..7e492629ae 100644 --- a/dm/master/scheduler/scheduler_test.go +++ b/dm/master/scheduler/scheduler_test.go @@ -199,7 +199,8 @@ func (t *testScheduler) testSchedulerProgress(c *C, restart int) { }), IsTrue) t.sourceBounds(c, s, []string{sourceID1}, []string{}) t.workerBound(c, s, ha.NewSourceBound(sourceID1, workerName1)) - // expect relay stage become Running after the first bound. + c.Assert(s.StartRelay(sourceID1, []string{workerName1}), IsNil) + // expect relay stage become Running after the start relay. t.relayStageMatch(c, s, sourceID1, pb.Stage_Running) rebuildScheduler(ctx) @@ -380,6 +381,7 @@ func (t *testScheduler) testSchedulerProgress(c *C, restart int) { // source2 should bound to worker2. t.workerBound(c, s, ha.NewSourceBound(sourceID2, workerName2)) t.sourceBounds(c, s, []string{sourceID1, sourceID2}, []string{}) + c.Assert(s.StartRelay(sourceID2, []string{workerName2}), IsNil) t.relayStageMatch(c, s, sourceID2, pb.Stage_Running) rebuildScheduler(ctx) @@ -447,6 +449,7 @@ func (t *testScheduler) testSchedulerProgress(c *C, restart int) { rebuildScheduler(ctx) // CASE 4.7: remove source2. + c.Assert(s.StopRelay(sourceID2, []string{workerName2}), IsNil) c.Assert(s.RemoveSourceCfg(sourceID2), IsNil) c.Assert(terror.ErrSchedulerSourceCfgNotExist.Equal(s.RemoveSourceCfg(sourceID2)), IsTrue) // already removed. // source2 removed. @@ -1053,9 +1056,9 @@ func (t *testScheduler) TestTransferSource(c *C) { s.sourceCfgs[sourceID2] = config.SourceConfig{} worker1.ToFree() - c.Assert(s.boundSourceToWorker(sourceID1, worker1, false), IsNil) + c.Assert(s.boundSourceToWorker(sourceID1, worker1), IsNil) worker2.ToFree() - c.Assert(s.boundSourceToWorker(sourceID2, worker2, false), IsNil) + c.Assert(s.boundSourceToWorker(sourceID2, worker2), IsNil) c.Assert(s.bounds[sourceID1], DeepEquals, worker1) c.Assert(s.bounds[sourceID2], DeepEquals, worker2) @@ -1115,3 +1118,88 @@ func (t *testScheduler) TestTransferSource(c *C) { c.Assert(s.bounds[sourceID1], DeepEquals, worker4) c.Assert(worker1.Stage(), Equals, WorkerFree) } + +func (t *testScheduler) TestStartStopSource(c *C) { + defer clearTestInfoOperation(c) + + var ( + logger = log.L() + s = NewScheduler(&logger, config.Security{}) + sourceID1 = "mysql-replica-1" + sourceID2 = "mysql-replica-2" + sourceID3 = "mysql-replica-3" + sourceID4 = "mysql-replica-4" + workerName1 = "dm-worker-1" + workerName2 = "dm-worker-2" + workerName3 = "dm-worker-3" + workerName4 = "dm-worker-4" + ) + + worker1 := &Worker{baseInfo: ha.WorkerInfo{Name: workerName1}} + worker2 := &Worker{baseInfo: ha.WorkerInfo{Name: workerName2}} + worker3 := &Worker{baseInfo: ha.WorkerInfo{Name: workerName3}} + worker4 := &Worker{baseInfo: ha.WorkerInfo{Name: workerName4}} + + // step 1: start an empty scheduler + s.started = true + s.etcdCli = etcdTestCli + s.workers[workerName1] = worker1 + s.workers[workerName2] = worker2 + s.workers[workerName3] = worker3 + s.workers[workerName4] = worker4 + s.sourceCfgs[sourceID1] = config.SourceConfig{} + s.sourceCfgs[sourceID2] = config.SourceConfig{} + + worker1.ToFree() + c.Assert(s.boundSourceToWorker(sourceID1, worker1), IsNil) + worker2.ToFree() + c.Assert(s.boundSourceToWorker(sourceID2, worker2), IsNil) + + c.Assert(s.bounds[sourceID1], DeepEquals, worker1) + c.Assert(s.bounds[sourceID2], DeepEquals, worker2) + + worker3.ToFree() + worker4.ToFree() + + // test not exist source + c.Assert(terror.ErrSchedulerSourceCfgNotExist.Equal(s.StartRelay(sourceID3, []string{workerName1})), IsTrue) + c.Assert(terror.ErrSchedulerSourceCfgNotExist.Equal(s.StopRelay(sourceID4, []string{workerName1})), IsTrue) + + // start-relay success on bound-same-source and free worker + c.Assert(s.StartRelay(sourceID1, []string{workerName1}), IsNil) + c.Assert(s.StartRelay(sourceID1, []string{workerName1}), IsNil) + c.Assert(s.expectRelayStages, HasLen, 1) + c.Assert(s.expectRelayStages, HasKey, sourceID1) + c.Assert(s.StartRelay(sourceID1, []string{workerName3}), IsNil) + c.Assert(s.relayWorkers, HasLen, 1) + c.Assert(s.relayWorkers[sourceID1], HasLen, 2) + c.Assert(s.relayWorkers[sourceID1], HasKey, workerName1) + c.Assert(s.relayWorkers[sourceID1], HasKey, workerName3) + + // failed on bound-not-same-source worker and not exist worker + c.Assert(terror.ErrSchedulerRelayWorkersWrongBound.Equal(s.StartRelay(sourceID1, []string{workerName2})), IsTrue) + c.Assert(terror.ErrSchedulerWorkerNotExist.Equal(s.StartRelay(sourceID1, []string{"not-exist"})), IsTrue) + + // failed on one worker multiple relay source + c.Assert(terror.ErrSchedulerRelayWorkersBusy.Equal(s.StartRelay(sourceID2, []string{workerName3})), IsTrue) + + // start another relay worker + c.Assert(s.StartRelay(sourceID2, []string{workerName2}), IsNil) + c.Assert(s.expectRelayStages, HasLen, 2) + c.Assert(s.expectRelayStages, HasKey, sourceID2) + c.Assert(s.relayWorkers[sourceID2], HasLen, 1) + c.Assert(s.relayWorkers[sourceID2], HasKey, workerName2) + + // failed on not-same-source worker and not exist worker + c.Assert(terror.ErrSchedulerRelayWorkersWrongRelay.Equal(s.StopRelay(sourceID1, []string{workerName2})), IsTrue) + c.Assert(terror.ErrSchedulerWorkerNotExist.Equal(s.StopRelay(sourceID1, []string{"not-exist"})), IsTrue) + + // stop-relay success + c.Assert(s.StopRelay(sourceID1, []string{workerName1}), IsNil) + c.Assert(s.StopRelay(sourceID1, []string{workerName1}), IsNil) + c.Assert(s.StopRelay(sourceID1, []string{workerName3}), IsNil) + c.Assert(s.expectRelayStages, HasLen, 1) + c.Assert(s.expectRelayStages, HasKey, sourceID2) + c.Assert(s.relayWorkers, HasLen, 1) + c.Assert(s.relayWorkers, HasKey, sourceID2) +} diff --git a/dm/master/server.go b/dm/master/server.go index 7a45942074..a5923ec2f4 100644 --- a/dm/master/server.go +++ b/dm/master/server.go @@ -1102,7 +1102,7 @@ func (s *Server) OperateSource(ctx context.Context, req *pb.OperateSourceRequest for _, sid := range toRemove { boundM[sid] = s.scheduler.GetWorkerBySource(sid) - err := s.scheduler.RemoveSourceCfg(sid) + err3 := s.scheduler.RemoveSourceCfg(sid) // TODO(lance6716): // user could not copy-paste same command if encounter error halfway: // `operate-source stop correct-id-1 wrong-id-2` @@ -1111,8 +1111,8 @@ func (s *Server) OperateSource(ctx context.Context, req *pb.OperateSourceRequest // not exist, error // find a way to distinguish this scenario and wrong source id // or give a command to show existing source id - if err != nil { - resp.Msg = err.Error() + if err3 != nil { + resp.Msg = err3.Error() return resp, nil } } @@ -1126,6 +1126,13 @@ func (s *Server) OperateSource(ctx context.Context, req *pb.OperateSourceRequest } resp.Result = true + // tell user he should use `start-relay` to manually specify relay workers + for _, cfg := range cfgs { + if cfg.EnableRelay { + resp.Msg = "Please use `start-relay` to specify which workers should pull relay log of relay-enabled sources." + } + } + var noWorkerMsg string switch req.Op { case pb.SourceOp_StartSource, pb.SourceOp_ShowSource: @@ -2004,6 +2011,34 @@ func (s *Server) TransferSource(ctx context.Context, req *pb.TransferSourceReque return resp2, nil } +// OperateRelay implements MasterServer.OperateRelay +func (s *Server) OperateRelay(ctx context.Context, req *pb.OperateRelayRequest) (*pb.OperateRelayResponse, error) { + var ( + resp2 = &pb.OperateRelayResponse{} + err error + ) + shouldRet := s.sharedLogic(ctx, req, &resp2, &err) + if shouldRet { + return resp2, err + } + + switch req.Op { + case pb.RelayOpV2_StartRelayV2: + err = s.scheduler.StartRelay(req.Source, req.Worker) + case pb.RelayOpV2_StopRelayV2: + err = s.scheduler.StopRelay(req.Source, req.Worker) + default: + // should not happen + return resp2, fmt.Errorf("only support start-relay or stop-relay, op: %s", req.Op.String()) + } + if err != nil { + resp2.Msg = err.Error() + return resp2, nil + } + resp2.Result = true + return resp2, nil +} + // sharedLogic does some shared logic for each RPC implementation // arguments with `Pointer` suffix should be pointer to that variable its name indicated // return `true` means caller should return with variable that `xxPointer` modified diff --git a/dm/pb/dmmaster.pb.go b/dm/pb/dmmaster.pb.go index 6072a79779..04d767c437 100644 --- a/dm/pb/dmmaster.pb.go +++ b/dm/pb/dmmaster.pb.go @@ -124,6 +124,34 @@ func (CfgType) EnumDescriptor() ([]byte, []int) { return fileDescriptor_f9bef11f2a341f03, []int{2} } +type RelayOpV2 int32 + +const ( + RelayOpV2_InvalidRelayOpV2 RelayOpV2 = 0 + RelayOpV2_StartRelayV2 RelayOpV2 = 1 + RelayOpV2_StopRelayV2 RelayOpV2 = 2 +) + +var RelayOpV2_name = map[int32]string{ + 0: "InvalidRelayOpV2", + 1: "StartRelayV2", + 2: "StopRelayV2", +} + +var RelayOpV2_value = map[string]int32{ + "InvalidRelayOpV2": 0, + "StartRelayV2": 1, + "StopRelayV2": 2, +} + +func (x RelayOpV2) String() string { + return proto.EnumName(RelayOpV2_name, int32(x)) +} + +func (RelayOpV2) EnumDescriptor() ([]byte, []int) { + return fileDescriptor_f9bef11f2a341f03, []int{3} +} + type StartTaskRequest struct { Task string `protobuf:"bytes,1,opt,name=task,proto3" json:"task,omitempty"` Sources []string `protobuf:"bytes,2,rep,name=sources,proto3" json:"sources,omitempty"` @@ -2949,10 +2977,123 @@ func (m *TransferSourceResponse) GetMsg() string { return "" } +type OperateRelayRequest struct { + Op RelayOpV2 `protobuf:"varint,1,opt,name=op,proto3,enum=pb.RelayOpV2" json:"op,omitempty"` + Source string `protobuf:"bytes,2,opt,name=source,proto3" json:"source,omitempty"` + Worker []string `protobuf:"bytes,3,rep,name=worker,proto3" json:"worker,omitempty"` +} + +func (m *OperateRelayRequest) Reset() { *m = OperateRelayRequest{} } +func (m *OperateRelayRequest) String() string { return proto.CompactTextString(m) } +func (*OperateRelayRequest) ProtoMessage() {} +func (*OperateRelayRequest) Descriptor() ([]byte, []int) { + return fileDescriptor_f9bef11f2a341f03, []int{47} +} +func (m *OperateRelayRequest) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *OperateRelayRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_OperateRelayRequest.Marshal(b, m, deterministic) + } else { + b = b[:cap(b)] + n, err := m.MarshalToSizedBuffer(b) + if err != nil { + return nil, err + } + return b[:n], nil + } +} +func (m *OperateRelayRequest) XXX_Merge(src proto.Message) { + xxx_messageInfo_OperateRelayRequest.Merge(m, src) +} +func (m *OperateRelayRequest) XXX_Size() int { + return m.Size() +} +func (m *OperateRelayRequest) XXX_DiscardUnknown() { + xxx_messageInfo_OperateRelayRequest.DiscardUnknown(m) +} + +var xxx_messageInfo_OperateRelayRequest proto.InternalMessageInfo + +func (m *OperateRelayRequest) GetOp() RelayOpV2 { + if m != nil { + return m.Op + } + return RelayOpV2_InvalidRelayOpV2 +} + +func (m *OperateRelayRequest) GetSource() string { + if m != nil { + return m.Source + } + return "" +} + +func (m *OperateRelayRequest) GetWorker() []string { + if m != nil { + return m.Worker + } + return nil +} + +type OperateRelayResponse struct { + Result bool `protobuf:"varint,1,opt,name=result,proto3" json:"result,omitempty"` + Msg string `protobuf:"bytes,2,opt,name=msg,proto3" json:"msg,omitempty"` +} + +func (m *OperateRelayResponse) Reset() { *m = OperateRelayResponse{} } +func (m *OperateRelayResponse) String() string { return proto.CompactTextString(m) } +func (*OperateRelayResponse) ProtoMessage() {} +func (*OperateRelayResponse) Descriptor() ([]byte, []int) { + return fileDescriptor_f9bef11f2a341f03, []int{48} +} +func (m *OperateRelayResponse) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *OperateRelayResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_OperateRelayResponse.Marshal(b, m, deterministic) + } else { + b = b[:cap(b)] + n, err := m.MarshalToSizedBuffer(b) + if err != nil { + return nil, err + } + return b[:n], nil + } +} +func (m *OperateRelayResponse) XXX_Merge(src proto.Message) { + xxx_messageInfo_OperateRelayResponse.Merge(m, src) +} +func (m *OperateRelayResponse) XXX_Size() int { + return m.Size() +} +func (m *OperateRelayResponse) XXX_DiscardUnknown() { + xxx_messageInfo_OperateRelayResponse.DiscardUnknown(m) +} + +var xxx_messageInfo_OperateRelayResponse proto.InternalMessageInfo + +func (m *OperateRelayResponse) GetResult() bool { + if m != nil { + return m.Result + } + return false +} + +func (m *OperateRelayResponse) GetMsg() string { + if m != nil { + return m.Msg + } + return "" +} + func init() { proto.RegisterEnum("pb.SourceOp", SourceOp_name, SourceOp_value) proto.RegisterEnum("pb.LeaderOp", LeaderOp_name, LeaderOp_value) proto.RegisterEnum("pb.CfgType", CfgType_name, CfgType_value) + proto.RegisterEnum("pb.RelayOpV2", RelayOpV2_name, RelayOpV2_value) proto.RegisterType((*StartTaskRequest)(nil), "pb.StartTaskRequest") proto.RegisterType((*StartTaskResponse)(nil), "pb.StartTaskResponse") proto.RegisterType((*OperateTaskRequest)(nil), "pb.OperateTaskRequest") @@ -3000,134 +3141,141 @@ func init() { proto.RegisterType((*HandleErrorResponse)(nil), "pb.HandleErrorResponse") proto.RegisterType((*TransferSourceRequest)(nil), "pb.TransferSourceRequest") proto.RegisterType((*TransferSourceResponse)(nil), "pb.TransferSourceResponse") + proto.RegisterType((*OperateRelayRequest)(nil), "pb.OperateRelayRequest") + proto.RegisterType((*OperateRelayResponse)(nil), "pb.OperateRelayResponse") } func init() { proto.RegisterFile("dmmaster.proto", fileDescriptor_f9bef11f2a341f03) } var fileDescriptor_f9bef11f2a341f03 = []byte{ - // 1940 bytes of a gzipped FileDescriptorProto + // 2018 bytes of a gzipped FileDescriptorProto 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xbc, 0x59, 0x5f, 0x6f, 0xdb, 0xc8, - 0x11, 0x17, 0x25, 0xc5, 0x96, 0x47, 0xb6, 0x4e, 0x5e, 0xdb, 0x32, 0xc3, 0xf8, 0x14, 0xdf, 0xf6, - 0x2e, 0x30, 0x8c, 0x22, 0x46, 0xdc, 0x3e, 0x1d, 0x70, 0x05, 0x2e, 0x56, 0x2e, 0x67, 0x54, 0xa9, - 0xaf, 0x74, 0x82, 0xf6, 0x50, 0xa0, 0x38, 0x8a, 0x5a, 0xc9, 0x84, 0x29, 0x92, 0x21, 0x29, 0xbb, - 0x46, 0x70, 0x2f, 0xfd, 0x00, 0xfd, 0x83, 0x3e, 0xdc, 0x63, 0x1f, 0xfa, 0x65, 0xfa, 0x78, 0x40, - 0x81, 0xa2, 0x8f, 0x45, 0xd2, 0xaf, 0x51, 0xa0, 0xd8, 0xd9, 0x25, 0xb9, 0xfc, 0x23, 0xb7, 0x0a, - 0x50, 0xbf, 0x71, 0x76, 0x56, 0x33, 0xbf, 0xf9, 0xb3, 0xb3, 0x33, 0x2b, 0xe8, 0x8c, 0x67, 0x33, - 0x2b, 0x8a, 0x59, 0xf8, 0x38, 0x08, 0xfd, 0xd8, 0x27, 0xf5, 0x60, 0x64, 0x74, 0xc6, 0xb3, 0x6b, - 0x3f, 0xbc, 0x4c, 0xd6, 0x8c, 0xbd, 0xa9, 0xef, 0x4f, 0x5d, 0x76, 0x64, 0x05, 0xce, 0x91, 0xe5, - 0x79, 0x7e, 0x6c, 0xc5, 0x8e, 0xef, 0x45, 0x82, 0x4b, 0xbf, 0x81, 0xee, 0x79, 0x6c, 0x85, 0xf1, - 0x4b, 0x2b, 0xba, 0x34, 0xd9, 0xeb, 0x39, 0x8b, 0x62, 0x42, 0xa0, 0x19, 0x5b, 0xd1, 0xa5, 0xae, - 0xed, 0x6b, 0x07, 0x6b, 0x26, 0x7e, 0x13, 0x1d, 0x56, 0x23, 0x7f, 0x1e, 0xda, 0x2c, 0xd2, 0xeb, - 0xfb, 0x8d, 0x83, 0x35, 0x33, 0x21, 0x49, 0x1f, 0x20, 0x64, 0x33, 0xff, 0x8a, 0xbd, 0x60, 0xb1, - 0xa5, 0x37, 0xf6, 0xb5, 0x83, 0x96, 0xa9, 0xac, 0xd0, 0xd7, 0xb0, 0xa9, 0x68, 0x88, 0x02, 0xdf, - 0x8b, 0x18, 0xe9, 0xc1, 0x4a, 0xc8, 0xa2, 0xb9, 0x1b, 0xa3, 0x92, 0x96, 0x29, 0x29, 0xd2, 0x85, - 0xc6, 0x2c, 0x9a, 0xea, 0x75, 0xd4, 0xcc, 0x3f, 0xc9, 0x71, 0xa6, 0xb8, 0xb1, 0xdf, 0x38, 0x68, - 0x1f, 0xeb, 0x8f, 0x83, 0xd1, 0xe3, 0x13, 0x7f, 0x36, 0xf3, 0xbd, 0x5f, 0xa0, 0x9d, 0x89, 0xd0, - 0x14, 0x12, 0xfd, 0x35, 0x90, 0xb3, 0x80, 0x85, 0x56, 0xcc, 0x54, 0xb3, 0x0c, 0xa8, 0xfb, 0x01, - 0xea, 0xeb, 0x1c, 0x03, 0x17, 0xc2, 0x99, 0x67, 0x81, 0x59, 0xf7, 0x03, 0x6e, 0xb2, 0x67, 0xcd, - 0x98, 0x54, 0x8c, 0xdf, 0xaa, 0xc9, 0x8d, 0x9c, 0xc9, 0xf4, 0xf7, 0x1a, 0x6c, 0xe5, 0x14, 0x48, - 0xab, 0x6e, 0xd3, 0x90, 0x59, 0x5c, 0xaf, 0xb2, 0xb8, 0x51, 0x69, 0x71, 0xf3, 0x7f, 0xb5, 0xf8, - 0x73, 0xd8, 0x7c, 0x15, 0x8c, 0x0b, 0x06, 0x2f, 0x15, 0x47, 0x1a, 0x02, 0x51, 0x45, 0xdc, 0x49, - 0xa0, 0xbe, 0x80, 0xde, 0xcf, 0xe7, 0x2c, 0xbc, 0x39, 0x8f, 0xad, 0x78, 0x1e, 0x0d, 0x9d, 0x28, - 0x56, 0xb0, 0x63, 0x40, 0xb4, 0xea, 0x80, 0x14, 0xb0, 0x5f, 0xc1, 0x6e, 0x49, 0xce, 0xd2, 0x06, - 0x3c, 0x29, 0x1a, 0xb0, 0xcb, 0x0d, 0x50, 0xe4, 0x96, 0xf1, 0x9f, 0xc0, 0xd6, 0xf9, 0x85, 0x7f, - 0x3d, 0x18, 0x0c, 0x87, 0xbe, 0x7d, 0x19, 0xbd, 0x9f, 0xe3, 0xff, 0xac, 0xc1, 0xaa, 0x94, 0x40, - 0x3a, 0x50, 0x3f, 0x1d, 0xc8, 0xdf, 0xd5, 0x4f, 0x07, 0xa9, 0xa4, 0xba, 0x22, 0x89, 0x40, 0x73, - 0xe6, 0x8f, 0x99, 0x4c, 0x19, 0xfc, 0x26, 0xdb, 0x70, 0xcf, 0xbf, 0xf6, 0x58, 0xa8, 0x37, 0x71, - 0x51, 0x10, 0x7c, 0xe7, 0x60, 0x30, 0x8c, 0xf4, 0x7b, 0xa8, 0x10, 0xbf, 0xb9, 0x3f, 0xa2, 0x1b, - 0xcf, 0x66, 0x63, 0x7d, 0x05, 0x57, 0x25, 0x45, 0x0c, 0x68, 0xcd, 0x3d, 0xc9, 0x59, 0x45, 0x4e, - 0x4a, 0x53, 0x1b, 0xb6, 0xf3, 0x66, 0x2e, 0xed, 0xdb, 0x8f, 0xe0, 0x9e, 0xcb, 0x7f, 0x2a, 0x3d, - 0xdb, 0xe6, 0x9e, 0x95, 0xe2, 0x4c, 0xc1, 0xa1, 0x2e, 0x6c, 0xbf, 0xf2, 0xf8, 0x67, 0xb2, 0x2e, - 0x9d, 0x59, 0x74, 0x09, 0x85, 0xf5, 0x90, 0x05, 0xae, 0x65, 0xb3, 0x33, 0xb4, 0x58, 0x68, 0xc9, - 0xad, 0x91, 0x7d, 0x68, 0x4f, 0xfc, 0xd0, 0x66, 0x26, 0x96, 0x21, 0x59, 0x94, 0xd4, 0x25, 0xfa, - 0x39, 0xec, 0x14, 0xb4, 0x2d, 0x6b, 0x13, 0x35, 0xe1, 0xbe, 0x2c, 0x02, 0x49, 0x7a, 0xbb, 0xd6, - 0x4d, 0x82, 0xfa, 0x81, 0x52, 0x0a, 0xd0, 0x5a, 0xe4, 0xca, 0x5a, 0xb0, 0x38, 0x17, 0xbe, 0xd3, - 0xc0, 0xa8, 0x12, 0x2a, 0xc1, 0xdd, 0x2a, 0xf5, 0xff, 0x5b, 0x61, 0xbe, 0xd3, 0x60, 0xf7, 0xab, - 0x79, 0x38, 0xad, 0x32, 0x56, 0xb1, 0x47, 0xcb, 0x5f, 0x0e, 0x06, 0xb4, 0x1c, 0xcf, 0xb2, 0x63, - 0xe7, 0x8a, 0x49, 0x54, 0x29, 0x8d, 0xb9, 0xed, 0xcc, 0x44, 0x74, 0x1a, 0x26, 0x7e, 0xf3, 0xfd, - 0x13, 0xc7, 0x65, 0x78, 0xf4, 0x45, 0x2a, 0xa7, 0x34, 0x66, 0xee, 0x7c, 0x34, 0x70, 0x42, 0xfd, - 0x1e, 0x72, 0x24, 0x45, 0x7f, 0x03, 0x7a, 0x19, 0xd8, 0x9d, 0x94, 0xaf, 0x47, 0xd0, 0x3d, 0xb9, - 0x60, 0xf6, 0xe5, 0x7f, 0x29, 0xba, 0xf4, 0x33, 0xd8, 0x54, 0xf6, 0x2d, 0x9d, 0x68, 0x17, 0xb0, - 0x2d, 0x73, 0xe2, 0x1c, 0x15, 0x27, 0xaa, 0xf6, 0x94, 0x6c, 0x58, 0xe7, 0x68, 0x05, 0x3b, 0x4b, - 0x07, 0xdb, 0xf7, 0x26, 0xce, 0x54, 0xe6, 0x98, 0xa4, 0xb8, 0x8b, 0x05, 0xfe, 0xd3, 0x81, 0xbc, - 0xd7, 0x52, 0x9a, 0xce, 0x61, 0xa7, 0xa0, 0xe9, 0x4e, 0xfc, 0xf8, 0x0c, 0x76, 0x4c, 0x36, 0x75, - 0x78, 0x23, 0x93, 0x6c, 0xb9, 0xf5, 0x16, 0xb0, 0xc6, 0xe3, 0x90, 0x45, 0x91, 0x54, 0x9b, 0x90, - 0xf4, 0x29, 0xf4, 0x8a, 0x62, 0x96, 0xf6, 0xf5, 0x4f, 0x60, 0xfb, 0x6c, 0x32, 0x71, 0x1d, 0x8f, - 0xbd, 0x60, 0xb3, 0x51, 0x0e, 0x49, 0x7c, 0x13, 0xa4, 0x48, 0xf8, 0x77, 0x55, 0xd3, 0xc0, 0xeb, - 0x4a, 0xe1, 0xf7, 0x4b, 0x43, 0xf8, 0x71, 0x1a, 0xee, 0x21, 0xb3, 0xc6, 0x19, 0x84, 0x52, 0xb8, - 0x05, 0x5b, 0x84, 0x1b, 0x15, 0xe7, 0x7f, 0xb5, 0xb4, 0xe2, 0xdf, 0x69, 0x00, 0x2f, 0xb0, 0x9d, - 0x3c, 0xf5, 0x26, 0x7e, 0xa5, 0xf3, 0x0d, 0x68, 0xcd, 0xd0, 0xae, 0xd3, 0x01, 0xfe, 0xb2, 0x69, - 0xa6, 0x34, 0xbf, 0x83, 0x2c, 0xd7, 0x49, 0xcb, 0xad, 0x20, 0xf8, 0x2f, 0x02, 0xc6, 0xc2, 0x57, - 0xe6, 0x50, 0x14, 0x9b, 0x35, 0x33, 0xa5, 0x79, 0xeb, 0x68, 0xbb, 0x0e, 0xf3, 0x62, 0xe4, 0x8a, - 0x5b, 0x4a, 0x59, 0xa1, 0x23, 0x00, 0x11, 0xc8, 0x85, 0x78, 0x08, 0x34, 0x79, 0xf4, 0x93, 0x10, - 0xf0, 0x6f, 0x8e, 0x23, 0x8a, 0xad, 0x69, 0x72, 0x41, 0x0a, 0x02, 0xab, 0x07, 0xa6, 0x9b, 0xac, - 0x2b, 0x92, 0xa2, 0x43, 0xe8, 0xf2, 0x7e, 0x41, 0x38, 0x4d, 0xc4, 0x2c, 0x71, 0x8d, 0x96, 0x65, - 0x75, 0x55, 0x7f, 0x98, 0xe8, 0x6e, 0x64, 0xba, 0xe9, 0xcf, 0x84, 0x34, 0xe1, 0xc5, 0x85, 0xd2, - 0x0e, 0x60, 0x55, 0xb4, 0xed, 0xa2, 0xfe, 0xb7, 0x8f, 0x3b, 0x3c, 0x9c, 0x99, 0xeb, 0xcd, 0x84, - 0x9d, 0xc8, 0x13, 0x5e, 0xb8, 0x4d, 0x9e, 0x68, 0xf9, 0x73, 0xf2, 0x32, 0xd7, 0x99, 0x09, 0x9b, - 0xfe, 0x45, 0x83, 0x55, 0x21, 0x26, 0x22, 0x8f, 0x61, 0xc5, 0x45, 0xab, 0x51, 0x54, 0xfb, 0x78, - 0x1b, 0x73, 0xaa, 0xe0, 0x8b, 0x2f, 0x6b, 0xa6, 0xdc, 0xc5, 0xf7, 0x0b, 0x58, 0xe8, 0x05, 0x65, - 0xbf, 0x6a, 0x2d, 0xdf, 0x2f, 0x76, 0xf1, 0xfd, 0x42, 0x2d, 0x7a, 0x48, 0xd9, 0xaf, 0x5a, 0xc3, - 0xf7, 0x8b, 0x5d, 0x4f, 0x5b, 0xb0, 0x22, 0x72, 0x89, 0x8f, 0x0c, 0x28, 0x37, 0x77, 0x02, 0x7b, - 0x39, 0xb8, 0xad, 0x14, 0x56, 0x2f, 0x07, 0xab, 0x95, 0xaa, 0xef, 0xe5, 0xd4, 0xb7, 0x12, 0x35, - 0x3c, 0x3d, 0x78, 0xf8, 0x92, 0x6c, 0x14, 0x04, 0x65, 0x40, 0x54, 0x95, 0x4b, 0x97, 0xbd, 0x4f, - 0x60, 0x55, 0x80, 0xcf, 0xb5, 0x38, 0xd2, 0xd5, 0x66, 0xc2, 0xa3, 0x7f, 0xd7, 0xb2, 0x5a, 0x6e, - 0x5f, 0xb0, 0x99, 0xb5, 0xb8, 0x96, 0x23, 0x3b, 0x1b, 0x4f, 0x4a, 0x6d, 0xe0, 0xc2, 0xf1, 0x84, - 0x1f, 0xb9, 0xb1, 0x15, 0x5b, 0x23, 0x2b, 0x4a, 0x2f, 0xd1, 0x84, 0xe6, 0xd6, 0xc7, 0xd6, 0xc8, - 0x65, 0xf2, 0x0e, 0x15, 0x04, 0x1e, 0x0e, 0xd4, 0xa7, 0xaf, 0xc8, 0xc3, 0x81, 0x14, 0xdf, 0x3d, - 0x71, 0xe7, 0xd1, 0x85, 0xbe, 0x2a, 0x8e, 0x34, 0x12, 0x1c, 0x0d, 0x6f, 0x0c, 0xf5, 0x16, 0x2e, - 0xe2, 0xb7, 0x7a, 0x73, 0x48, 0xbb, 0xee, 0xe4, 0xe6, 0x38, 0x84, 0xed, 0xe7, 0x2c, 0x3e, 0x9f, - 0x8f, 0xf8, 0xd5, 0x7a, 0x32, 0x99, 0xde, 0x72, 0x71, 0xd0, 0x57, 0xb0, 0x53, 0xd8, 0xbb, 0x34, - 0x44, 0x02, 0x4d, 0x7b, 0x32, 0x4d, 0x1c, 0x8e, 0xdf, 0x74, 0x00, 0x1b, 0xcf, 0x59, 0xac, 0xe8, - 0x7e, 0xa8, 0x5c, 0x15, 0xb2, 0x4d, 0x3b, 0x99, 0x4c, 0x5f, 0xde, 0x04, 0xec, 0x96, 0x7b, 0x63, - 0x08, 0x9d, 0x44, 0xca, 0xd2, 0xa8, 0xba, 0xd0, 0xb0, 0x27, 0x69, 0x83, 0x67, 0x4f, 0xa6, 0x74, - 0x07, 0xb6, 0x9e, 0x33, 0x79, 0x2e, 0x33, 0x64, 0xf4, 0x00, 0xbd, 0xa5, 0x2c, 0x4b, 0x55, 0x52, - 0x80, 0x96, 0x09, 0xf8, 0xa3, 0x06, 0xe4, 0x4b, 0xcb, 0x1b, 0xbb, 0xec, 0x59, 0x18, 0xfa, 0xe1, - 0xc2, 0xae, 0x16, 0xb9, 0xef, 0x95, 0xa4, 0x7b, 0xb0, 0x36, 0x72, 0x3c, 0xd7, 0x9f, 0x7e, 0xe5, - 0x47, 0x32, 0x4b, 0xb3, 0x05, 0x4c, 0xb1, 0xd7, 0x6e, 0x3a, 0xb9, 0xf0, 0x6f, 0x1a, 0xc1, 0x56, - 0x0e, 0xd2, 0x9d, 0x24, 0xd8, 0x73, 0xd8, 0x79, 0x19, 0x5a, 0x5e, 0x34, 0x61, 0x61, 0xbe, 0xf9, - 0xca, 0xee, 0x13, 0x4d, 0xbd, 0x4f, 0x94, 0xb2, 0x23, 0x34, 0x4b, 0x8a, 0x37, 0x27, 0x45, 0x41, - 0xcb, 0x1a, 0x70, 0x38, 0x82, 0x56, 0xd2, 0xe2, 0x91, 0x2d, 0xf8, 0xe0, 0xd4, 0xbb, 0xb2, 0x5c, - 0x67, 0x9c, 0x2c, 0x75, 0x6b, 0xe4, 0x03, 0x68, 0xe3, 0x5b, 0x8b, 0x58, 0xea, 0x6a, 0xa4, 0x0b, - 0xeb, 0x62, 0xa8, 0x97, 0x2b, 0x75, 0xd2, 0x01, 0x38, 0x8f, 0xfd, 0x40, 0xd2, 0x0d, 0xa4, 0x2f, - 0xfc, 0x6b, 0x49, 0x37, 0x0f, 0x7f, 0x0a, 0xad, 0xa4, 0xaf, 0x50, 0x74, 0x24, 0x4b, 0xdd, 0x1a, - 0xd9, 0x84, 0x8d, 0x67, 0x57, 0x8e, 0x1d, 0xa7, 0x4b, 0x1a, 0xd9, 0x85, 0xad, 0x13, 0xcb, 0xb3, - 0x99, 0x9b, 0x67, 0xd4, 0x0f, 0x7f, 0x09, 0xab, 0x32, 0xf5, 0x39, 0x34, 0x29, 0x8b, 0x93, 0xdd, - 0x1a, 0x59, 0x87, 0x16, 0x3f, 0x88, 0x48, 0x69, 0x1c, 0x86, 0xc8, 0x4b, 0xa4, 0x11, 0xa6, 0x08, - 0x09, 0xd2, 0x02, 0x26, 0x42, 0x44, 0xba, 0x79, 0xfc, 0xef, 0x0d, 0x58, 0x11, 0x3f, 0x20, 0x5f, - 0xc3, 0x5a, 0xfa, 0xc0, 0x44, 0xf0, 0x92, 0x29, 0xbe, 0x68, 0x19, 0x3b, 0x85, 0x55, 0xe1, 0x79, - 0xfa, 0xf0, 0xb7, 0x7f, 0xfb, 0xd7, 0x9f, 0xea, 0xf7, 0xe9, 0xf6, 0x91, 0x15, 0x38, 0xd1, 0xd1, - 0xd5, 0x13, 0xcb, 0x0d, 0x2e, 0xac, 0x27, 0x47, 0x3c, 0x75, 0xa3, 0x4f, 0xb5, 0x43, 0x32, 0x81, - 0xb6, 0xf2, 0xce, 0x43, 0x7a, 0x5c, 0x4c, 0xf9, 0x65, 0xc9, 0xd8, 0x2d, 0xad, 0x4b, 0x05, 0x8f, - 0x50, 0xc1, 0xbe, 0xf1, 0xa0, 0x4a, 0xc1, 0xd1, 0x1b, 0x7e, 0xf2, 0xbf, 0xe5, 0x7a, 0x3e, 0x03, - 0xc8, 0xde, 0x5e, 0x08, 0xa2, 0x2d, 0x3d, 0xe7, 0x18, 0xbd, 0xe2, 0xb2, 0x54, 0x52, 0x23, 0x2e, - 0xb4, 0x95, 0x67, 0x0a, 0x62, 0x14, 0xde, 0x2d, 0x94, 0x77, 0x15, 0xe3, 0x41, 0x25, 0x4f, 0x4a, - 0xfa, 0x18, 0xe1, 0xf6, 0xc9, 0x5e, 0x01, 0x6e, 0x84, 0x5b, 0x25, 0x5e, 0x72, 0x02, 0xeb, 0xea, - 0x6b, 0x00, 0x41, 0xeb, 0x2b, 0x9e, 0x41, 0x0c, 0xbd, 0xcc, 0x48, 0x21, 0x7f, 0x01, 0x1b, 0xb9, - 0xf9, 0x9b, 0xe0, 0xe6, 0xaa, 0x07, 0x00, 0xe3, 0x7e, 0x05, 0x27, 0x95, 0xf3, 0x35, 0xf4, 0xca, - 0xf3, 0x32, 0x7a, 0xf1, 0x43, 0x25, 0x28, 0xe5, 0x99, 0xd5, 0xe8, 0x2f, 0x62, 0xa7, 0xa2, 0xcf, - 0xa0, 0x5b, 0x9c, 0x2b, 0x09, 0xba, 0x6f, 0xc1, 0x18, 0x6c, 0xec, 0x55, 0x33, 0x53, 0x81, 0x9f, - 0xc2, 0x5a, 0x3a, 0x06, 0x8a, 0x44, 0x2d, 0x4e, 0x8f, 0x22, 0x51, 0x4b, 0xb3, 0x22, 0xad, 0x91, - 0x29, 0x6c, 0xe4, 0x26, 0x33, 0xe1, 0xaf, 0xaa, 0xb1, 0x50, 0xf8, 0xab, 0x72, 0x8c, 0xa3, 0x1f, - 0x61, 0x80, 0x1f, 0x18, 0xbd, 0x62, 0x80, 0x45, 0xb5, 0xe3, 0xa9, 0x78, 0x0a, 0x9d, 0xfc, 0x10, - 0x45, 0xee, 0x8b, 0x87, 0x86, 0x8a, 0xf9, 0xcc, 0x30, 0xaa, 0x58, 0x29, 0xe6, 0x10, 0x36, 0x72, - 0xb3, 0x90, 0xc4, 0x5c, 0x31, 0x5e, 0x49, 0xcc, 0x55, 0x83, 0x13, 0xfd, 0x21, 0x62, 0x7e, 0x74, - 0xf8, 0x71, 0x01, 0xb3, 0x6c, 0xa9, 0x8e, 0xde, 0xf0, 0x3b, 0xf5, 0xdb, 0x24, 0x39, 0x2f, 0x53, - 0x3f, 0x89, 0x32, 0x94, 0xf3, 0x53, 0x6e, 0x9e, 0xca, 0xf9, 0x29, 0x3f, 0x33, 0xd1, 0x4f, 0x50, - 0xe7, 0x43, 0xc3, 0x28, 0xe8, 0x14, 0x2d, 0xe7, 0xd1, 0x1b, 0x3f, 0xc0, 0x63, 0xfb, 0x2b, 0x80, - 0xac, 0x69, 0x14, 0xc7, 0xb6, 0xd4, 0xb7, 0x8a, 0x63, 0x5b, 0xee, 0x2d, 0x69, 0x1f, 0x75, 0xe8, - 0xa4, 0x57, 0x6d, 0x17, 0x99, 0x64, 0x11, 0x17, 0xcd, 0x58, 0x2e, 0xe2, 0x6a, 0xf3, 0x98, 0x8f, - 0x78, 0xae, 0xfd, 0xa2, 0xfb, 0xa8, 0xc5, 0x30, 0x76, 0x8a, 0x11, 0xc7, 0x6d, 0xdc, 0x08, 0x17, - 0xfb, 0x97, 0xac, 0x2d, 0x12, 0x7a, 0xaa, 0xba, 0x2a, 0xa1, 0xa7, 0xb2, 0x87, 0x4a, 0x2a, 0x1d, - 0xe9, 0x17, 0xf5, 0xcc, 0x47, 0x6a, 0xb1, 0x23, 0x2f, 0x61, 0x45, 0xf4, 0x39, 0x64, 0x53, 0x0a, - 0x53, 0xe4, 0x13, 0x75, 0x49, 0x0a, 0xfe, 0x01, 0x0a, 0xfe, 0x90, 0xdc, 0x56, 0x42, 0xc9, 0x37, - 0xd0, 0x56, 0x5a, 0x03, 0x51, 0xa7, 0xcb, 0xed, 0x8b, 0xa8, 0xd3, 0x15, 0x3d, 0xc4, 0x42, 0x2f, - 0x31, 0xbe, 0x0b, 0x8f, 0xc5, 0x09, 0xac, 0xab, 0xad, 0x93, 0x28, 0x7a, 0x15, 0x3d, 0x96, 0xa1, - 0x97, 0x19, 0xe9, 0x81, 0x38, 0x85, 0x4e, 0xbe, 0x07, 0x10, 0x67, 0xab, 0xb2, 0xc1, 0x10, 0x67, - 0xab, 0xba, 0x65, 0xa0, 0xb5, 0xa7, 0xfa, 0x5f, 0xdf, 0xf6, 0xb5, 0xef, 0xdf, 0xf6, 0xb5, 0x7f, - 0xbe, 0xed, 0x6b, 0x7f, 0x78, 0xd7, 0xaf, 0x7d, 0xff, 0xae, 0x5f, 0xfb, 0xc7, 0xbb, 0x7e, 0x6d, - 0xb4, 0x82, 0x7f, 0xec, 0xfc, 0xe8, 0x3f, 0x01, 0x00, 0x00, 0xff, 0xff, 0x79, 0xf4, 0x4e, 0x15, - 0x1c, 0x1a, 0x00, 0x00, + 0x11, 0x17, 0x25, 0xc5, 0x96, 0x47, 0xb6, 0x4e, 0x5e, 0xcb, 0x32, 0xc3, 0x38, 0x8a, 0x6f, 0x7b, + 0x17, 0x18, 0x46, 0x11, 0x23, 0x6e, 0x9f, 0x0e, 0xb8, 0xa2, 0x17, 0x2b, 0x97, 0x33, 0xaa, 0xd4, + 0x57, 0x3a, 0xb9, 0xf6, 0x50, 0xa0, 0x38, 0x4a, 0x5a, 0xc9, 0x84, 0x29, 0x92, 0x21, 0x29, 0xbb, + 0x46, 0x70, 0x2f, 0xfd, 0x00, 0xfd, 0x83, 0x3e, 0xdc, 0x63, 0x1f, 0xfa, 0x4d, 0xfa, 0xd4, 0xc7, + 0x00, 0x05, 0x8a, 0x3e, 0x16, 0x49, 0x3f, 0x48, 0xb1, 0xb3, 0x4b, 0x72, 0xf9, 0x47, 0x6e, 0x15, + 0xe0, 0xfc, 0xc6, 0x99, 0x59, 0xcd, 0xfc, 0xe6, 0xcf, 0xce, 0xce, 0xae, 0xa0, 0x35, 0x9e, 0xcd, + 0xac, 0x30, 0x62, 0xc1, 0x23, 0x3f, 0xf0, 0x22, 0x8f, 0x54, 0xfd, 0xa1, 0xd1, 0x1a, 0xcf, 0xae, + 0xbc, 0xe0, 0x22, 0xe6, 0x19, 0xbb, 0x53, 0xcf, 0x9b, 0x3a, 0xec, 0xd0, 0xf2, 0xed, 0x43, 0xcb, + 0x75, 0xbd, 0xc8, 0x8a, 0x6c, 0xcf, 0x0d, 0x85, 0x94, 0x7e, 0x03, 0xed, 0xb3, 0xc8, 0x0a, 0xa2, + 0x17, 0x56, 0x78, 0x61, 0xb2, 0x57, 0x73, 0x16, 0x46, 0x84, 0x40, 0x3d, 0xb2, 0xc2, 0x0b, 0x5d, + 0xdb, 0xd3, 0xf6, 0xd7, 0x4c, 0xfc, 0x26, 0x3a, 0xac, 0x86, 0xde, 0x3c, 0x18, 0xb1, 0x50, 0xaf, + 0xee, 0xd5, 0xf6, 0xd7, 0xcc, 0x98, 0x24, 0x3d, 0x80, 0x80, 0xcd, 0xbc, 0x4b, 0xf6, 0x9c, 0x45, + 0x96, 0x5e, 0xdb, 0xd3, 0xf6, 0x1b, 0xa6, 0xc2, 0xa1, 0xaf, 0x60, 0x53, 0xb1, 0x10, 0xfa, 0x9e, + 0x1b, 0x32, 0xd2, 0x85, 0x95, 0x80, 0x85, 0x73, 0x27, 0x42, 0x23, 0x0d, 0x53, 0x52, 0xa4, 0x0d, + 0xb5, 0x59, 0x38, 0xd5, 0xab, 0x68, 0x99, 0x7f, 0x92, 0xa3, 0xd4, 0x70, 0x6d, 0xaf, 0xb6, 0xdf, + 0x3c, 0xd2, 0x1f, 0xf9, 0xc3, 0x47, 0xc7, 0xde, 0x6c, 0xe6, 0xb9, 0xbf, 0x44, 0x3f, 0x63, 0xa5, + 0x09, 0x24, 0xfa, 0x1b, 0x20, 0xa7, 0x3e, 0x0b, 0xac, 0x88, 0xa9, 0x6e, 0x19, 0x50, 0xf5, 0x7c, + 0xb4, 0xd7, 0x3a, 0x02, 0xae, 0x84, 0x0b, 0x4f, 0x7d, 0xb3, 0xea, 0xf9, 0xdc, 0x65, 0xd7, 0x9a, + 0x31, 0x69, 0x18, 0xbf, 0x55, 0x97, 0x6b, 0x19, 0x97, 0xe9, 0x1f, 0x34, 0xd8, 0xca, 0x18, 0x90, + 0x5e, 0xdd, 0x64, 0x21, 0xf5, 0xb8, 0x5a, 0xe6, 0x71, 0xad, 0xd4, 0xe3, 0xfa, 0xff, 0xeb, 0xf1, + 0x67, 0xb0, 0xf9, 0xd2, 0x1f, 0xe7, 0x1c, 0x5e, 0x2a, 0x8f, 0x34, 0x00, 0xa2, 0xaa, 0xb8, 0x95, + 0x44, 0x7d, 0x0e, 0xdd, 0x5f, 0xcc, 0x59, 0x70, 0x7d, 0x16, 0x59, 0xd1, 0x3c, 0x1c, 0xd8, 0x61, + 0xa4, 0x60, 0xc7, 0x84, 0x68, 0xe5, 0x09, 0xc9, 0x61, 0xbf, 0x84, 0x9d, 0x82, 0x9e, 0xa5, 0x1d, + 0x78, 0x9c, 0x77, 0x60, 0x87, 0x3b, 0xa0, 0xe8, 0x2d, 0xe2, 0x3f, 0x86, 0xad, 0xb3, 0x73, 0xef, + 0xaa, 0xdf, 0x1f, 0x0c, 0xbc, 0xd1, 0x45, 0xf8, 0x7e, 0x81, 0xff, 0x8b, 0x06, 0xab, 0x52, 0x03, + 0x69, 0x41, 0xf5, 0xa4, 0x2f, 0x7f, 0x57, 0x3d, 0xe9, 0x27, 0x9a, 0xaa, 0x8a, 0x26, 0x02, 0xf5, + 0x99, 0x37, 0x66, 0xb2, 0x64, 0xf0, 0x9b, 0x74, 0xe0, 0x8e, 0x77, 0xe5, 0xb2, 0x40, 0xaf, 0x23, + 0x53, 0x10, 0x7c, 0x65, 0xbf, 0x3f, 0x08, 0xf5, 0x3b, 0x68, 0x10, 0xbf, 0x79, 0x3c, 0xc2, 0x6b, + 0x77, 0xc4, 0xc6, 0xfa, 0x0a, 0x72, 0x25, 0x45, 0x0c, 0x68, 0xcc, 0x5d, 0x29, 0x59, 0x45, 0x49, + 0x42, 0xd3, 0x11, 0x74, 0xb2, 0x6e, 0x2e, 0x1d, 0xdb, 0x0f, 0xe1, 0x8e, 0xc3, 0x7f, 0x2a, 0x23, + 0xdb, 0xe4, 0x91, 0x95, 0xea, 0x4c, 0x21, 0xa1, 0x0e, 0x74, 0x5e, 0xba, 0xfc, 0x33, 0xe6, 0xcb, + 0x60, 0xe6, 0x43, 0x42, 0x61, 0x3d, 0x60, 0xbe, 0x63, 0x8d, 0xd8, 0x29, 0x7a, 0x2c, 0xac, 0x64, + 0x78, 0x64, 0x0f, 0x9a, 0x13, 0x2f, 0x18, 0x31, 0x13, 0xdb, 0x90, 0x6c, 0x4a, 0x2a, 0x8b, 0x7e, + 0x06, 0xdb, 0x39, 0x6b, 0xcb, 0xfa, 0x44, 0x4d, 0xb8, 0x2b, 0x9b, 0x40, 0x5c, 0xde, 0x8e, 0x75, + 0x1d, 0xa3, 0xbe, 0xa7, 0xb4, 0x02, 0xf4, 0x16, 0xa5, 0xb2, 0x17, 0x2c, 0xae, 0x85, 0xef, 0x34, + 0x30, 0xca, 0x94, 0x4a, 0x70, 0x37, 0x6a, 0xfd, 0x7e, 0x3b, 0xcc, 0x77, 0x1a, 0xec, 0x7c, 0x39, + 0x0f, 0xa6, 0x65, 0xce, 0x2a, 0xfe, 0x68, 0xd9, 0xc3, 0xc1, 0x80, 0x86, 0xed, 0x5a, 0xa3, 0xc8, + 0xbe, 0x64, 0x12, 0x55, 0x42, 0x63, 0x6d, 0xdb, 0x33, 0x91, 0x9d, 0x9a, 0x89, 0xdf, 0x7c, 0xfd, + 0xc4, 0x76, 0x18, 0x6e, 0x7d, 0x51, 0xca, 0x09, 0x8d, 0x95, 0x3b, 0x1f, 0xf6, 0xed, 0x40, 0xbf, + 0x83, 0x12, 0x49, 0xd1, 0xdf, 0x82, 0x5e, 0x04, 0x76, 0x2b, 0xed, 0xeb, 0x21, 0xb4, 0x8f, 0xcf, + 0xd9, 0xe8, 0xe2, 0x7f, 0x34, 0x5d, 0xfa, 0x29, 0x6c, 0x2a, 0xeb, 0x96, 0x2e, 0xb4, 0x73, 0xe8, + 0xc8, 0x9a, 0x38, 0x43, 0xc3, 0xb1, 0xa9, 0x5d, 0xa5, 0x1a, 0xd6, 0x39, 0x5a, 0x21, 0x4e, 0xcb, + 0x61, 0xe4, 0xb9, 0x13, 0x7b, 0x2a, 0x6b, 0x4c, 0x52, 0x3c, 0xc4, 0x02, 0xff, 0x49, 0x5f, 0x9e, + 0x6b, 0x09, 0x4d, 0xe7, 0xb0, 0x9d, 0xb3, 0x74, 0x2b, 0x71, 0x7c, 0x0a, 0xdb, 0x26, 0x9b, 0xda, + 0x7c, 0x90, 0x89, 0x97, 0xdc, 0x78, 0x0a, 0x58, 0xe3, 0x71, 0xc0, 0xc2, 0x50, 0x9a, 0x8d, 0x49, + 0xfa, 0x04, 0xba, 0x79, 0x35, 0x4b, 0xc7, 0xfa, 0x27, 0xd0, 0x39, 0x9d, 0x4c, 0x1c, 0xdb, 0x65, + 0xcf, 0xd9, 0x6c, 0x98, 0x41, 0x12, 0x5d, 0xfb, 0x09, 0x12, 0xfe, 0x5d, 0x36, 0x34, 0xf0, 0xbe, + 0x92, 0xfb, 0xfd, 0xd2, 0x10, 0x7e, 0x9c, 0xa4, 0x7b, 0xc0, 0xac, 0x71, 0x0a, 0xa1, 0x90, 0x6e, + 0x21, 0x16, 0xe9, 0x46, 0xc3, 0xd9, 0x5f, 0x2d, 0x6d, 0xf8, 0xf7, 0x1a, 0xc0, 0x73, 0x1c, 0x27, + 0x4f, 0xdc, 0x89, 0x57, 0x1a, 0x7c, 0x03, 0x1a, 0x33, 0xf4, 0xeb, 0xa4, 0x8f, 0xbf, 0xac, 0x9b, + 0x09, 0xcd, 0xcf, 0x20, 0xcb, 0xb1, 0x93, 0x76, 0x2b, 0x08, 0xfe, 0x0b, 0x9f, 0xb1, 0xe0, 0xa5, + 0x39, 0x10, 0xcd, 0x66, 0xcd, 0x4c, 0x68, 0x3e, 0x3a, 0x8e, 0x1c, 0x9b, 0xb9, 0x11, 0x4a, 0xc5, + 0x29, 0xa5, 0x70, 0xe8, 0x10, 0x40, 0x24, 0x72, 0x21, 0x1e, 0x02, 0x75, 0x9e, 0xfd, 0x38, 0x05, + 0xfc, 0x9b, 0xe3, 0x08, 0x23, 0x6b, 0x1a, 0x1f, 0x90, 0x82, 0xc0, 0xee, 0x81, 0xe5, 0x26, 0xfb, + 0x8a, 0xa4, 0xe8, 0x00, 0xda, 0x7c, 0x5e, 0x10, 0x41, 0x13, 0x39, 0x8b, 0x43, 0xa3, 0xa5, 0x55, + 0x5d, 0x36, 0x1f, 0xc6, 0xb6, 0x6b, 0xa9, 0x6d, 0xfa, 0x73, 0xa1, 0x4d, 0x44, 0x71, 0xa1, 0xb6, + 0x7d, 0x58, 0x15, 0x63, 0xbb, 0xe8, 0xff, 0xcd, 0xa3, 0x16, 0x4f, 0x67, 0x1a, 0x7a, 0x33, 0x16, + 0xc7, 0xfa, 0x44, 0x14, 0x6e, 0xd2, 0x27, 0x46, 0xfe, 0x8c, 0xbe, 0x34, 0x74, 0x66, 0x2c, 0xa6, + 0x7f, 0xd5, 0x60, 0x55, 0xa8, 0x09, 0xc9, 0x23, 0x58, 0x71, 0xd0, 0x6b, 0x54, 0xd5, 0x3c, 0xea, + 0x60, 0x4d, 0xe5, 0x62, 0xf1, 0x45, 0xc5, 0x94, 0xab, 0xf8, 0x7a, 0x01, 0x0b, 0xa3, 0xa0, 0xac, + 0x57, 0xbd, 0xe5, 0xeb, 0xc5, 0x2a, 0xbe, 0x5e, 0x98, 0xc5, 0x08, 0x29, 0xeb, 0x55, 0x6f, 0xf8, + 0x7a, 0xb1, 0xea, 0x49, 0x03, 0x56, 0x44, 0x2d, 0xf1, 0x2b, 0x03, 0xea, 0xcd, 0xec, 0xc0, 0x6e, + 0x06, 0x6e, 0x23, 0x81, 0xd5, 0xcd, 0xc0, 0x6a, 0x24, 0xe6, 0xbb, 0x19, 0xf3, 0x8d, 0xd8, 0x0c, + 0x2f, 0x0f, 0x9e, 0xbe, 0xb8, 0x1a, 0x05, 0x41, 0x19, 0x10, 0xd5, 0xe4, 0xd2, 0x6d, 0xef, 0x63, + 0x58, 0x15, 0xe0, 0x33, 0x23, 0x8e, 0x0c, 0xb5, 0x19, 0xcb, 0xe8, 0x3f, 0xb5, 0xb4, 0x97, 0x8f, + 0xce, 0xd9, 0xcc, 0x5a, 0xdc, 0xcb, 0x51, 0x9c, 0x5e, 0x4f, 0x0a, 0x63, 0xe0, 0xc2, 0xeb, 0x09, + 0xdf, 0x72, 0x63, 0x2b, 0xb2, 0x86, 0x56, 0x98, 0x1c, 0xa2, 0x31, 0xcd, 0xbd, 0x8f, 0xac, 0xa1, + 0xc3, 0xe4, 0x19, 0x2a, 0x08, 0xdc, 0x1c, 0x68, 0x4f, 0x5f, 0x91, 0x9b, 0x03, 0x29, 0xbe, 0x7a, + 0xe2, 0xcc, 0xc3, 0x73, 0x7d, 0x55, 0x6c, 0x69, 0x24, 0x38, 0x1a, 0x3e, 0x18, 0xea, 0x0d, 0x64, + 0xe2, 0xb7, 0x7a, 0x72, 0x48, 0xbf, 0x6e, 0xe5, 0xe4, 0x38, 0x80, 0xce, 0x33, 0x16, 0x9d, 0xcd, + 0x87, 0xfc, 0x68, 0x3d, 0x9e, 0x4c, 0x6f, 0x38, 0x38, 0xe8, 0x4b, 0xd8, 0xce, 0xad, 0x5d, 0x1a, + 0x22, 0x81, 0xfa, 0x68, 0x32, 0x8d, 0x03, 0x8e, 0xdf, 0xb4, 0x0f, 0x1b, 0xcf, 0x58, 0xa4, 0xd8, + 0x7e, 0xa0, 0x1c, 0x15, 0x72, 0x4c, 0x3b, 0x9e, 0x4c, 0x5f, 0x5c, 0xfb, 0xec, 0x86, 0x73, 0x63, + 0x00, 0xad, 0x58, 0xcb, 0xd2, 0xa8, 0xda, 0x50, 0x1b, 0x4d, 0x92, 0x01, 0x6f, 0x34, 0x99, 0xd2, + 0x6d, 0xd8, 0x7a, 0xc6, 0xe4, 0xbe, 0x4c, 0x91, 0xd1, 0x7d, 0x8c, 0x96, 0xc2, 0x96, 0xa6, 0xa4, + 0x02, 0x2d, 0x55, 0xf0, 0x27, 0x0d, 0xc8, 0x17, 0x96, 0x3b, 0x76, 0xd8, 0xd3, 0x20, 0xf0, 0x82, + 0x85, 0x53, 0x2d, 0x4a, 0xdf, 0xab, 0x48, 0x77, 0x61, 0x6d, 0x68, 0xbb, 0x8e, 0x37, 0xfd, 0xd2, + 0x0b, 0x65, 0x95, 0xa6, 0x0c, 0x2c, 0xb1, 0x57, 0x4e, 0x72, 0x73, 0xe1, 0xdf, 0x34, 0x84, 0xad, + 0x0c, 0xa4, 0x5b, 0x29, 0xb0, 0x67, 0xb0, 0xfd, 0x22, 0xb0, 0xdc, 0x70, 0xc2, 0x82, 0xec, 0xf0, + 0x95, 0x9e, 0x27, 0x9a, 0x7a, 0x9e, 0x28, 0x6d, 0x47, 0x58, 0x96, 0x14, 0x1f, 0x4e, 0xf2, 0x8a, + 0x96, 0x3e, 0xa0, 0xc7, 0xc9, 0xb3, 0x43, 0x66, 0xfc, 0xbe, 0xaf, 0x64, 0x65, 0x43, 0xb9, 0x15, + 0x7c, 0x75, 0x14, 0x0f, 0x82, 0x12, 0x69, 0x75, 0x01, 0x52, 0x91, 0x9a, 0x18, 0xe9, 0x4f, 0x93, + 0x16, 0xf5, 0x9e, 0xb3, 0xf4, 0xc1, 0x10, 0x1a, 0xf1, 0x28, 0x4a, 0xb6, 0xe0, 0x83, 0x13, 0xf7, + 0xd2, 0x72, 0xec, 0x71, 0xcc, 0x6a, 0x57, 0xc8, 0x07, 0xd0, 0xc4, 0x37, 0x21, 0xc1, 0x6a, 0x6b, + 0xa4, 0x0d, 0xeb, 0xe2, 0xf1, 0x41, 0x72, 0xaa, 0xa4, 0x05, 0x70, 0x16, 0x79, 0xbe, 0xa4, 0x6b, + 0x48, 0x9f, 0x7b, 0x57, 0x92, 0xae, 0x1f, 0xfc, 0x0c, 0x1a, 0xf1, 0xfc, 0xa3, 0xd8, 0x88, 0x59, + 0xed, 0x0a, 0xd9, 0x84, 0x8d, 0xa7, 0x97, 0xf6, 0x28, 0x4a, 0x58, 0x1a, 0xd9, 0x81, 0xad, 0x63, + 0xcb, 0x1d, 0x31, 0x27, 0x2b, 0xa8, 0x1e, 0xfc, 0x0a, 0x56, 0xe5, 0x16, 0xe5, 0xd0, 0xa4, 0x2e, + 0x4e, 0xb6, 0x2b, 0x64, 0x1d, 0x1a, 0xbc, 0x61, 0x20, 0xa5, 0x71, 0x18, 0x62, 0xff, 0x20, 0x8d, + 0x30, 0x45, 0xe9, 0x20, 0x2d, 0x60, 0x22, 0x44, 0xa4, 0xeb, 0x07, 0x7d, 0x58, 0x4b, 0xb2, 0x41, + 0x3a, 0xd0, 0x96, 0xba, 0x13, 0x5e, 0xbb, 0xc2, 0x7d, 0xc7, 0x60, 0x20, 0xef, 0xab, 0xa3, 0xb6, + 0x26, 0xc2, 0xe3, 0xf9, 0x31, 0xa3, 0x7a, 0xf4, 0xb7, 0x16, 0xac, 0x08, 0xb3, 0xe4, 0x6b, 0x58, + 0x4b, 0x9e, 0xd3, 0x08, 0x1e, 0xa9, 0xf9, 0xf7, 0x3b, 0x63, 0x3b, 0xc7, 0x15, 0xf9, 0xa3, 0x0f, + 0x7e, 0xf7, 0x8f, 0xff, 0xfc, 0xb9, 0x7a, 0x97, 0x76, 0x0e, 0x2d, 0xdf, 0x0e, 0x0f, 0x2f, 0x1f, + 0x5b, 0x8e, 0x7f, 0x6e, 0x3d, 0x3e, 0xe4, 0x1b, 0x35, 0xfc, 0x44, 0x3b, 0x20, 0x13, 0x68, 0x2a, + 0xaf, 0x5a, 0xa4, 0xcb, 0xd5, 0x14, 0xdf, 0xd1, 0x8c, 0x9d, 0x02, 0x5f, 0x1a, 0x78, 0x88, 0x06, + 0xf6, 0x8c, 0x7b, 0x65, 0x06, 0x0e, 0x5f, 0xf3, 0x3e, 0xf7, 0x2d, 0xb7, 0xf3, 0x29, 0x40, 0xfa, + 0xd2, 0x44, 0x10, 0x6d, 0xe1, 0xf1, 0xca, 0xe8, 0xe6, 0xd9, 0xd2, 0x48, 0x85, 0x38, 0xd0, 0x54, + 0x1e, 0x65, 0x88, 0x91, 0x7b, 0xa5, 0x51, 0x5e, 0x91, 0x8c, 0x7b, 0xa5, 0x32, 0xa9, 0xe9, 0x23, + 0x84, 0xdb, 0x23, 0xbb, 0x39, 0xb8, 0x21, 0x2e, 0x95, 0x78, 0xc9, 0x31, 0xac, 0xab, 0x6f, 0x1f, + 0x04, 0xbd, 0x2f, 0x79, 0xf4, 0x31, 0xf4, 0xa2, 0x20, 0x81, 0xfc, 0x39, 0x6c, 0x64, 0x5e, 0x1b, + 0x08, 0x2e, 0x2e, 0x7b, 0xee, 0x30, 0xee, 0x96, 0x48, 0x12, 0x3d, 0x5f, 0x43, 0xb7, 0xf8, 0x3a, + 0x80, 0x51, 0xbc, 0xaf, 0x24, 0xa5, 0x78, 0x43, 0x37, 0x7a, 0x8b, 0xc4, 0x89, 0xea, 0x53, 0x68, + 0xe7, 0x6f, 0xd1, 0x04, 0xc3, 0xb7, 0xe0, 0xd2, 0x6f, 0xec, 0x96, 0x0b, 0x13, 0x85, 0x9f, 0xc0, + 0x5a, 0x72, 0xe9, 0x15, 0x85, 0x9a, 0xbf, 0x2b, 0x8b, 0x42, 0x2d, 0xdc, 0x8c, 0x69, 0x85, 0x4c, + 0x61, 0x23, 0x73, 0x0f, 0x15, 0xf1, 0x2a, 0xbb, 0x04, 0x8b, 0x78, 0x95, 0x5e, 0x5a, 0xe9, 0x87, + 0x98, 0xe0, 0x7b, 0x46, 0x37, 0x9f, 0x60, 0xd1, 0xdb, 0x79, 0x29, 0x9e, 0x40, 0x2b, 0x7b, 0x65, + 0x24, 0x77, 0x45, 0x03, 0x2d, 0xb9, 0x8d, 0x1a, 0x46, 0x99, 0x28, 0xc1, 0x1c, 0xc0, 0x46, 0xe6, + 0xe6, 0x27, 0x31, 0x97, 0x5c, 0x26, 0x25, 0xe6, 0xb2, 0x6b, 0x22, 0xfd, 0x21, 0x62, 0x7e, 0x78, + 0xf0, 0x51, 0x0e, 0xb3, 0x1c, 0x20, 0x0f, 0x5f, 0xf3, 0x09, 0xe2, 0xdb, 0xb8, 0x38, 0x2f, 0x92, + 0x38, 0x89, 0x66, 0x96, 0x89, 0x53, 0xe6, 0xf6, 0x98, 0x89, 0x53, 0xf6, 0x86, 0x48, 0x3f, 0x46, + 0x9b, 0x0f, 0x0c, 0x23, 0x67, 0x53, 0x0c, 0xd8, 0x87, 0xaf, 0x3d, 0x1f, 0xb7, 0xed, 0xaf, 0x01, + 0xd2, 0x11, 0x59, 0x6c, 0xdb, 0xc2, 0x94, 0x2e, 0xb6, 0x6d, 0x71, 0x92, 0xa6, 0x3d, 0xb4, 0xa1, + 0x93, 0x6e, 0xb9, 0x5f, 0x64, 0x92, 0x66, 0x5c, 0x8c, 0x9e, 0x99, 0x8c, 0xab, 0xa3, 0x72, 0x36, + 0xe3, 0x99, 0x61, 0x93, 0xee, 0xa1, 0x15, 0xc3, 0xd8, 0xce, 0x67, 0x1c, 0x97, 0x71, 0x27, 0x1c, + 0x9c, 0xd6, 0xd2, 0x21, 0x50, 0xd8, 0x29, 0x9b, 0x21, 0x85, 0x9d, 0xd2, 0x89, 0x31, 0xee, 0x74, + 0xa4, 0x97, 0xb7, 0x33, 0x1f, 0xaa, 0xcd, 0x8e, 0xbc, 0x80, 0x15, 0x31, 0xd5, 0x91, 0x4d, 0xa9, + 0x4c, 0xd1, 0x4f, 0x54, 0x96, 0x54, 0xfc, 0x03, 0x54, 0x7c, 0x9f, 0xdc, 0xd4, 0x42, 0xc9, 0x37, + 0xd0, 0x54, 0x06, 0x21, 0xd1, 0xa7, 0x8b, 0xc3, 0x9a, 0xe8, 0xd3, 0x25, 0x13, 0xd3, 0xc2, 0x28, + 0x31, 0xbe, 0x0a, 0xb7, 0xc5, 0x31, 0xac, 0xab, 0x83, 0xa2, 0x68, 0x7a, 0x25, 0x13, 0xa5, 0xa1, + 0x17, 0x05, 0xc9, 0x86, 0x38, 0x81, 0x56, 0x76, 0xe2, 0x11, 0x7b, 0xab, 0x74, 0x9c, 0x12, 0x7b, + 0xab, 0x7c, 0x40, 0xa2, 0x15, 0x8e, 0x47, 0x1d, 0x49, 0x88, 0x7a, 0x04, 0x65, 0x9a, 0x92, 0x5e, + 0x14, 0xc4, 0x4a, 0x9e, 0xe8, 0x7f, 0x7f, 0xdb, 0xd3, 0xde, 0xbc, 0xed, 0x69, 0xff, 0x7e, 0xdb, + 0xd3, 0xfe, 0xf8, 0xae, 0x57, 0x79, 0xf3, 0xae, 0x57, 0xf9, 0xd7, 0xbb, 0x5e, 0x65, 0xb8, 0x82, + 0xff, 0x85, 0xfd, 0xe8, 0xbf, 0x01, 0x00, 0x00, 0xff, 0xff, 0x8a, 0x3d, 0xfc, 0xd4, 0x4f, 0x1b, + 0x00, 0x00, } // Reference imports to suppress errors if they are not otherwise used. @@ -3175,6 +3323,7 @@ type MasterClient interface { HandleError(ctx context.Context, in *HandleErrorRequest, opts ...grpc.CallOption) (*HandleErrorResponse, error) GetMasterCfg(ctx context.Context, in *GetMasterCfgRequest, opts ...grpc.CallOption) (*GetMasterCfgResponse, error) TransferSource(ctx context.Context, in *TransferSourceRequest, opts ...grpc.CallOption) (*TransferSourceResponse, error) + OperateRelay(ctx context.Context, in *OperateRelayRequest, opts ...grpc.CallOption) (*OperateRelayResponse, error) } type masterClient struct { @@ -3365,6 +3514,15 @@ func (c *masterClient) TransferSource(ctx context.Context, in *TransferSourceReq return out, nil } +func (c *masterClient) OperateRelay(ctx context.Context, in *OperateRelayRequest, opts ...grpc.CallOption) (*OperateRelayResponse, error) { + out := new(OperateRelayResponse) + err := c.cc.Invoke(ctx, "/pb.Master/OperateRelay", in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + // MasterServer is the server API for Master service. type MasterServer interface { StartTask(context.Context, *StartTaskRequest) (*StartTaskResponse, error) @@ -3400,6 +3558,7 @@ type MasterServer interface { HandleError(context.Context, *HandleErrorRequest) (*HandleErrorResponse, error) GetMasterCfg(context.Context, *GetMasterCfgRequest) (*GetMasterCfgResponse, error) TransferSource(context.Context, *TransferSourceRequest) (*TransferSourceResponse, error) + OperateRelay(context.Context, *OperateRelayRequest) (*OperateRelayResponse, error) } // UnimplementedMasterServer can be embedded to have forward compatible implementations. @@ -3466,6 +3625,9 @@ func (*UnimplementedMasterServer) GetMasterCfg(ctx context.Context, req *GetMast func (*UnimplementedMasterServer) TransferSource(ctx context.Context, req *TransferSourceRequest) (*TransferSourceResponse, error) { return nil, status.Errorf(codes.Unimplemented, "method TransferSource not implemented") } +func (*UnimplementedMasterServer) OperateRelay(ctx context.Context, req *OperateRelayRequest) (*OperateRelayResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method OperateRelay not implemented") +} func RegisterMasterServer(s *grpc.Server, srv MasterServer) { s.RegisterService(&_Master_serviceDesc, srv) @@ -3831,6 +3993,24 @@ func _Master_TransferSource_Handler(srv interface{}, ctx context.Context, dec fu return interceptor(ctx, in, info, handler) } +func _Master_OperateRelay_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(OperateRelayRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(MasterServer).OperateRelay(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: "/pb.Master/OperateRelay", + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(MasterServer).OperateRelay(ctx, req.(*OperateRelayRequest)) + } + return interceptor(ctx, in, info, handler) +} + var _Master_serviceDesc = grpc.ServiceDesc{ ServiceName: "pb.Master", HandlerType: (*MasterServer)(nil), @@ -3915,6 +4095,10 @@ var _Master_serviceDesc = grpc.ServiceDesc{ MethodName: "TransferSource", Handler: _Master_TransferSource_Handler, }, + { + MethodName: "OperateRelay", + Handler: _Master_OperateRelay_Handler, + }, }, Streams: []grpc.StreamDesc{}, Metadata: "dmmaster.proto", @@ -6174,6 +6358,90 @@ func (m *TransferSourceResponse) MarshalToSizedBuffer(dAtA []byte) (int, error) return len(dAtA) - i, nil } +func (m *OperateRelayRequest) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *OperateRelayRequest) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *OperateRelayRequest) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + _ = i + var l int + _ = l + if len(m.Worker) > 0 { + for iNdEx := len(m.Worker) - 1; iNdEx >= 0; iNdEx-- { + i -= len(m.Worker[iNdEx]) + copy(dAtA[i:], m.Worker[iNdEx]) + i = encodeVarintDmmaster(dAtA, i, uint64(len(m.Worker[iNdEx]))) + i-- + dAtA[i] = 0x1a + } + } + if len(m.Source) > 0 { + i -= len(m.Source) + copy(dAtA[i:], m.Source) + i = encodeVarintDmmaster(dAtA, i, uint64(len(m.Source))) + i-- + dAtA[i] = 0x12 + } + if m.Op != 0 { + i = encodeVarintDmmaster(dAtA, i, uint64(m.Op)) + i-- + dAtA[i] = 0x8 + } + return len(dAtA) - i, nil +} + +func (m *OperateRelayResponse) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *OperateRelayResponse) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *OperateRelayResponse) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + _ = i + var l int + _ = l + if len(m.Msg) > 0 { + i -= len(m.Msg) + copy(dAtA[i:], m.Msg) + i = encodeVarintDmmaster(dAtA, i, uint64(len(m.Msg))) + i-- + dAtA[i] = 0x12 + } + if m.Result { + i-- + if m.Result { + dAtA[i] = 1 + } else { + dAtA[i] = 0 + } + i-- + dAtA[i] = 0x8 + } + return len(dAtA) - i, nil +} + func encodeVarintDmmaster(dAtA []byte, offset int, v uint64) int { offset -= sovDmmaster(v) base := offset @@ -7192,6 +7460,44 @@ func (m *TransferSourceResponse) Size() (n int) { return n } +func (m *OperateRelayRequest) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Op != 0 { + n += 1 + sovDmmaster(uint64(m.Op)) + } + l = len(m.Source) + if l > 0 { + n += 1 + l + sovDmmaster(uint64(l)) + } + if len(m.Worker) > 0 { + for _, s := range m.Worker { + l = len(s) + n += 1 + l + sovDmmaster(uint64(l)) + } + } + return n +} + +func (m *OperateRelayResponse) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Result { + n += 2 + } + l = len(m.Msg) + if l > 0 { + n += 1 + l + sovDmmaster(uint64(l)) + } + return n +} + func sovDmmaster(x uint64) (n int) { return (math_bits.Len64(x|1) + 6) / 7 } @@ -13501,6 +13807,247 @@ func (m *TransferSourceResponse) Unmarshal(dAtA []byte) error { } return nil } +func (m *OperateRelayRequest) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowDmmaster + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: OperateRelayRequest: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: OperateRelayRequest: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Op", wireType) + } + m.Op = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowDmmaster + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.Op |= RelayOpV2(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Source", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowDmmaster + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLengthDmmaster + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthDmmaster + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Source = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Worker", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowDmmaster + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLengthDmmaster + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthDmmaster + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Worker = append(m.Worker, string(dAtA[iNdEx:postIndex])) + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipDmmaster(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthDmmaster + } + if (iNdEx + skippy) < 0 { + return ErrInvalidLengthDmmaster + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *OperateRelayResponse) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowDmmaster + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: OperateRelayResponse: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: OperateRelayResponse: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Result", wireType) + } + var v int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowDmmaster + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + m.Result = bool(v != 0) + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Msg", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowDmmaster + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLengthDmmaster + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthDmmaster + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Msg = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipDmmaster(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthDmmaster + } + if (iNdEx + skippy) < 0 { + return ErrInvalidLengthDmmaster + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} func skipDmmaster(dAtA []byte) (n int, err error) { l := len(dAtA) iNdEx := 0 diff --git a/dm/proto/dmmaster.proto b/dm/proto/dmmaster.proto index e6203e1160..84f3df286e 100644 --- a/dm/proto/dmmaster.proto +++ b/dm/proto/dmmaster.proto @@ -106,6 +106,8 @@ service Master { rpc GetMasterCfg(GetMasterCfgRequest) returns(GetMasterCfgResponse) {} rpc TransferSource(TransferSourceRequest) returns(TransferSourceResponse) {} + + rpc OperateRelay(OperateRelayRequest) returns(OperateRelayResponse) {} } message StartTaskRequest { @@ -437,4 +439,21 @@ message TransferSourceRequest { message TransferSourceResponse { bool result = 1; string msg = 2; +} + +message OperateRelayRequest { + RelayOpV2 op = 1; + string source = 2; + repeated string worker = 3; +} + +message OperateRelayResponse { + bool result = 1; + string msg = 2; +} + +enum RelayOpV2 { + InvalidRelayOpV2 = 0; + StartRelayV2 = 1; + StopRelayV2 = 2; } \ No newline at end of file diff --git a/dm/worker/server.go b/dm/worker/server.go index e19bc54273..8ac5574fa4 100644 --- a/dm/worker/server.go +++ b/dm/worker/server.go @@ -473,6 +473,7 @@ func (s *Server) Close() { s.wg.Wait() } +// is needLock is false, we should make sure Server has been locked in caller func (s *Server) getWorker(needLock bool) *Worker { if needLock { s.Lock() @@ -481,6 +482,7 @@ func (s *Server) getWorker(needLock bool) *Worker { return s.worker } +// is needLock is false, we should make sure Server has been locked in caller func (s *Server) setWorker(worker *Worker, needLock bool) { if needLock { s.Lock() @@ -532,8 +534,8 @@ func (s *Server) stopWorker(sourceID string) error { s.UpdateKeepAliveTTL(s.cfg.KeepAliveTTL) s.setWorker(nil, false) s.setSourceStatus("", nil, false) - s.Unlock() w.Close() + s.Unlock() return nil } @@ -639,7 +641,6 @@ func (s *Server) operateSourceBound(bound ha.SourceBound) error { func (s *Server) operateRelaySource(relaySource ha.RelaySource) error { if relaySource.IsDeleted { - // TODO: will worker be modified on other goroutine? w := s.getWorker(true) s.UpdateKeepAliveTTL(s.cfg.KeepAliveTTL) w.DisableRelay() diff --git a/dm/worker/server_test.go b/dm/worker/server_test.go index 50d2fd7e0d..44e4ffd41b 100644 --- a/dm/worker/server_test.go +++ b/dm/worker/server_test.go @@ -489,7 +489,9 @@ func (t *testServer) testOperateWorker(c *C, s *Server, dir string, start bool) w := s.getWorker(true) c.Assert(w, NotNil) c.Assert(w.closed.Get(), IsFalse) - _, err := ha.DeleteSourceCfgRelayStageSourceBound(s.etcdClient, sourceCfg.SourceID, s.cfg.Name) + _, err := ha.DeleteRelayConfig(s.etcdClient, w.name) + c.Assert(err, IsNil) + _, err = ha.DeleteSourceCfgRelayStageSourceBound(s.etcdClient, sourceCfg.SourceID, s.cfg.Name) c.Assert(err, IsNil) // worker should be closed and without error c.Assert(utils.WaitSomething(30, 100*time.Millisecond, func() bool { diff --git a/dm/worker/worker.go b/dm/worker/worker.go index 06175d9cec..d080e74519 100644 --- a/dm/worker/worker.go +++ b/dm/worker/worker.go @@ -154,6 +154,8 @@ func (w *Worker) Close() { // cancel status output ticker and wait for return w.cancel() w.wg.Wait() + w.relayWg.Wait() + w.subTaskWg.Wait() w.Lock() defer w.Unlock() diff --git a/dm/worker/worker_test.go b/dm/worker/worker_test.go index 4be727848b..0336f4a836 100644 --- a/dm/worker/worker_test.go +++ b/dm/worker/worker_test.go @@ -123,7 +123,7 @@ func (t *testServer2) TestTaskAutoResume(c *C) { taskName = "sub-task-name" port = 8263 ) - hostName := "127.0.0.1:8261" + hostName := "127.0.0.1:18261" etcdDir := c.MkDir() ETCD, err := createMockETCD(etcdDir, "http://"+hostName) c.Assert(err, IsNil) @@ -131,6 +131,7 @@ func (t *testServer2) TestTaskAutoResume(c *C) { cfg := NewConfig() c.Assert(cfg.Parse([]string{"-config=./dm-worker.toml"}), IsNil) + cfg.Join = hostName sourceConfig := loadSourceConfigWithoutPassword(c) sourceConfig.Checker.CheckEnable = true sourceConfig.Checker.CheckInterval = config.Duration{Duration: 40 * time.Millisecond} diff --git a/errors.toml b/errors.toml index 4a77f15f7e..25fa6ebec3 100644 --- a/errors.toml +++ b/errors.toml @@ -2903,9 +2903,9 @@ workaround = "" tags = ["internal", "medium"] [error.DM-dm-master-46010] -message = "source with name %s need to operate with tasks %v exist" +message = "source with name %s need to operate has existing tasks %v" description = "" -workaround = "" +workaround = "Please `stop-task` first." tags = ["internal", "medium"] [error.DM-scheduler-46011] @@ -2959,7 +2959,31 @@ tags = ["internal", "medium"] [error.DM-scheduler-46019] message = "tasks %v on source %s should not be running" description = "" -workaround = "Please use `pause-task [-s source ...] task` to pause them first" +workaround = "Please use `pause-task [-s source ...] task` to pause them first." +tags = ["internal", "high"] + +[error.DM-scheduler-46020] +message = "these workers %s have started relay for sources %s respectively" +description = "" +workaround = "Please use `stop-relay` to stop them, or change your topology." +tags = ["internal", "high"] + +[error.DM-scheduler-46021] +message = "these workers %s have bound for another sources %s respectively" +description = "" +workaround = "Please `start-relay` on free or same source workers." +tags = ["internal", "high"] + +[error.DM-scheduler-46022] +message = "these workers %s have started relay for another sources %s respectively" +description = "" +workaround = "Please correct sources in `stop-relay`." +tags = ["internal", "high"] + +[error.DM-scheduler-46023] +message = "source with name %s need to operate has existing relay workers %s" +description = "" +workaround = "Please `stop-relay` first." tags = ["internal", "high"] [error.DM-dmctl-48001] diff --git a/pkg/ha/bound.go b/pkg/ha/bound.go index b8108301e7..2019fcf589 100644 --- a/pkg/ha/bound.go +++ b/pkg/ha/bound.go @@ -357,10 +357,8 @@ 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 { - // 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)), } } diff --git a/pkg/ha/relay.go b/pkg/ha/relay.go index 2baaf8f985..cfd4a24c30 100644 --- a/pkg/ha/relay.go +++ b/pkg/ha/relay.go @@ -60,6 +60,42 @@ func DeleteRelayConfig(cli *clientv3.Client, workers ...string) (int64, error) { return rev, err } +// GetAllRelayConfig gets all source and its relay worker. +// k/v: source ID -> set(workers). +func GetAllRelayConfig(cli *clientv3.Client) (map[string]map[string]struct{}, int64, error) { + ctx, cancel := context.WithTimeout(cli.Ctx(), etcdutil.DefaultRequestTimeout) + defer cancel() + + resp, err := cli.Get(ctx, common.UpstreamRelayWorkerKeyAdapter.Path(), clientv3.WithPrefix()) + if err != nil { + return nil, 0, err + } + + ret := map[string]map[string]struct{}{} + for _, kv := range resp.Kvs { + source := string(kv.Value) + keys, err2 := common.UpstreamRelayWorkerKeyAdapter.Decode(string(kv.Key)) + if err2 != nil { + return nil, 0, err2 + } + if len(keys) != 1 { + // should not happened + return nil, 0, terror.Annotate(err, "illegal key of UpstreamRelayWorkerKeyAdapter") + } + worker := keys[0] + var ( + ok bool + workers = map[string]struct{}{} + ) + if workers, ok = ret[source]; !ok { + workers = map[string]struct{}{} + ret[source] = workers + } + workers[worker] = struct{}{} + } + return ret, resp.Header.Revision, nil +} + // GetRelayConfig returns the source config which the given worker need to pull relay log from etcd, with revision func GetRelayConfig(cli *clientv3.Client, worker string) (*config.SourceConfig, int64, error) { var ( diff --git a/pkg/ha/stage.go b/pkg/ha/stage.go index 584e61fb57..97d7587969 100644 --- a/pkg/ha/stage.go +++ b/pkg/ha/stage.go @@ -99,6 +99,12 @@ func PutRelayStage(cli *clientv3.Client, stages ...Stage) (int64, error) { return rev, err } +// DeleteRelayStage deleted the relay stage of this source +func DeleteRelayStage(cli *clientv3.Client, source string) (int64, error) { + _, rev, err := etcdutil.DoOpsInOneTxnWithRetry(cli, deleteRelayStageOp(source)) + return rev, err +} + // PutSubTaskStage puts the stage of the subtask into etcd. // k/v: sourceID, task -> the running stage of the subtask. func PutSubTaskStage(cli *clientv3.Client, stages ...Stage) (int64, error) { diff --git a/pkg/ha/worker.go b/pkg/ha/worker.go index 44372ce801..762041a058 100644 --- a/pkg/ha/worker.go +++ b/pkg/ha/worker.go @@ -94,8 +94,12 @@ func GetAllWorkerInfo(cli *clientv3.Client) (map[string]WorkerInfo, int64, error return ifm, resp.Header.Revision, nil } -// DeleteWorkerInfo deletes the specified DM-worker information. -func DeleteWorkerInfo(cli *clientv3.Client, worker string) (int64, error) { - _, rev, err := etcdutil.DoOpsInOneTxnWithRetry(cli, clientv3.OpDelete(common.WorkerRegisterKeyAdapter.Encode(worker))) +// DeleteWorkerInfoRelayConfig deletes the specified DM-worker information and its relay config. +func DeleteWorkerInfoRelayConfig(cli *clientv3.Client, worker string) (int64, error) { + ops := []clientv3.Op{ + clientv3.OpDelete(common.WorkerRegisterKeyAdapter.Encode(worker)), + clientv3.OpDelete(common.UpstreamRelayWorkerKeyAdapter.Encode(worker)), + } + _, rev, err := etcdutil.DoOpsInOneTxnWithRetry(cli, ops...) return rev, err } diff --git a/pkg/ha/worker_test.go b/pkg/ha/worker_test.go index d731349879..15a3699aeb 100644 --- a/pkg/ha/worker_test.go +++ b/pkg/ha/worker_test.go @@ -62,7 +62,7 @@ func (t *testForEtcd) TestWorkerInfoEtcd(c *C) { c.Assert(ifm[worker2], DeepEquals, info2) // delete info1. - rev4, err := DeleteWorkerInfo(etcdTestCli, worker1) + rev4, err := DeleteWorkerInfoRelayConfig(etcdTestCli, worker1) c.Assert(err, IsNil) c.Assert(rev4, Greater, rev3) diff --git a/pkg/terror/error_list.go b/pkg/terror/error_list.go index 7e1b0a5122..cf496a5140 100644 --- a/pkg/terror/error_list.go +++ b/pkg/terror/error_list.go @@ -605,6 +605,10 @@ const ( codeSchedulerSubTaskOpSourceNotExist codeSchedulerTaskNotExist codeSchedulerRequireNotRunning + codeSchedulerRelayWorkersBusy + codeSchedulerRelayWorkersBound + codeSchedulerRelayWorkersWrongRelay + codeSchedulerSourceOpRelayExist ) // dmctl error code @@ -1186,7 +1190,7 @@ var ( ErrSchedulerSourceCfgExist = New(codeSchedulerSourceCfgExist, ClassScheduler, ScopeInternal, LevelMedium, "source config with ID %s already exists", "") ErrSchedulerSourceCfgNotExist = New(codeSchedulerSourceCfgNotExist, ClassScheduler, ScopeInternal, LevelMedium, "source config with ID %s not exists", "") ErrSchedulerSourcesUnbound = New(codeSchedulerSourcesUnbound, ClassDMMaster, ScopeInternal, LevelMedium, "sources %v have not bound", "") - ErrSchedulerSourceOpTaskExist = New(codeSchedulerSourceOpTaskExist, ClassDMMaster, ScopeInternal, LevelMedium, "source with name %s need to operate with tasks %v exist", "") + ErrSchedulerSourceOpTaskExist = New(codeSchedulerSourceOpTaskExist, ClassDMMaster, ScopeInternal, LevelMedium, "source with name %s need to operate has existing tasks %v", "Please `stop-task` first.") ErrSchedulerRelayStageInvalidUpdate = New(codeSchedulerRelayStageInvalidUpdate, ClassScheduler, ScopeInternal, LevelMedium, "invalid new expectant relay stage %s", "") ErrSchedulerRelayStageSourceNotExist = New(codeSchedulerRelayStageSourceNotExist, ClassScheduler, ScopeInternal, LevelMedium, "sources %v need to update expectant relay stage not exist", "") ErrSchedulerMultiTask = New(codeSchedulerMultiTask, ClassScheduler, ScopeInternal, LevelMedium, "the scheduler cannot perform multiple different tasks %v in one operation", "") @@ -1195,7 +1199,11 @@ var ( ErrSchedulerSubTaskOpTaskNotExist = New(codeSchedulerSubTaskOpTaskNotExist, ClassDMMaster, ScopeInternal, LevelMedium, "subtasks with name %s need to be operate not exist", "Please use `query-status` command to see tasks.") ErrSchedulerSubTaskOpSourceNotExist = New(codeSchedulerSubTaskOpSourceNotExist, ClassDMMaster, ScopeInternal, LevelMedium, "sources %v need to be operate not exist", "") ErrSchedulerTaskNotExist = New(codeSchedulerTaskNotExist, ClassScheduler, ScopeInternal, LevelMedium, "task with name %s not exist", "Please use `query-status` command to see tasks.") - ErrSchedulerRequireNotRunning = New(codeSchedulerRequireNotRunning, ClassScheduler, ScopeInternal, LevelHigh, "tasks %v on source %s should not be running", "Please use `pause-task [-s source ...] task` to pause them first") + ErrSchedulerRequireNotRunning = New(codeSchedulerRequireNotRunning, ClassScheduler, ScopeInternal, LevelHigh, "tasks %v on source %s should not be running", "Please use `pause-task [-s source ...] task` to pause them first.") + ErrSchedulerRelayWorkersBusy = New(codeSchedulerRelayWorkersBusy, ClassScheduler, ScopeInternal, LevelHigh, "these workers %s have started relay for sources %s respectively", "Please use `stop-relay` to stop them, or change your topology.") + ErrSchedulerRelayWorkersWrongBound = New(codeSchedulerRelayWorkersBound, ClassScheduler, ScopeInternal, LevelHigh, "these workers %s have bound for another sources %s respectively", "Please `start-relay` on free or same source workers.") + ErrSchedulerRelayWorkersWrongRelay = New(codeSchedulerRelayWorkersWrongRelay, ClassScheduler, ScopeInternal, LevelHigh, "these workers %s have started relay for another sources %s respectively", "Please correct sources in `stop-relay`.") + ErrSchedulerSourceOpRelayExist = New(codeSchedulerSourceOpRelayExist, ClassScheduler, ScopeInternal, LevelHigh, "source with name %s need to operate has existing relay workers %s", "Please `stop-relay` first.") // dmctl ErrCtlGRPCCreateConn = New(codeCtlGRPCCreateConn, ClassDMCtl, ScopeInternal, LevelHigh, "can not create grpc connection", "Please check your network connection.") diff --git a/syncer/syncer.go b/syncer/syncer.go index 3e234406f3..a1f5b6698d 100644 --- a/syncer/syncer.go +++ b/syncer/syncer.go @@ -495,7 +495,14 @@ func (s *Syncer) Process(ctx context.Context, pr chan pb.ProcessResult) { defer cancel() // create new done chan + // use lock of Syncer to avoid Close while Process + s.Lock() + if s.isClosed() { + s.Unlock() + return + } s.done = make(chan struct{}) + s.Unlock() runFatalChan := make(chan *pb.ProcessError, s.cfg.WorkerCount+1) s.runFatalChan = runFatalChan diff --git a/tests/all_mode/run.sh b/tests/all_mode/run.sh index 1bf58fe0c1..535c6ecdde 100755 --- a/tests/all_mode/run.sh +++ b/tests/all_mode/run.sh @@ -189,14 +189,21 @@ function run() { check_rpc_alive $cur/../bin/check_master_online 127.0.0.1:$MASTER_PORT run_dm_worker $WORK_DIR/worker1 $WORKER1_PORT $cur/conf/dm-worker1.toml check_rpc_alive $cur/../bin/check_worker_online 127.0.0.1:$WORKER1_PORT - run_dm_worker $WORK_DIR/worker2 $WORKER2_PORT $cur/conf/dm-worker2.toml - check_rpc_alive $cur/../bin/check_worker_online 127.0.0.1:$WORKER2_PORT + # operate mysql config to worker cp $cur/conf/source1.yaml $WORK_DIR/source1.yaml cp $cur/conf/source2.yaml $WORK_DIR/source2.yaml sed -i "/relay-binlog-name/i\relay-dir: $WORK_DIR/worker1/relay_log" $WORK_DIR/source1.yaml sed -i "/relay-binlog-name/i\relay-dir: $WORK_DIR/worker2/relay_log" $WORK_DIR/source2.yaml + # make sure source1 is bound to worker1 dmctl_operate_source create $WORK_DIR/source1.yaml $SOURCE_ID1 + + run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "start-relay -s $SOURCE_ID1 worker1" \ + "\"result\": true" 1 + + run_dm_worker $WORK_DIR/worker2 $WORKER2_PORT $cur/conf/dm-worker2.toml + check_rpc_alive $cur/../bin/check_worker_online 127.0.0.1:$WORKER2_PORT dmctl_operate_source create $WORK_DIR/source2.yaml $SOURCE_ID2 # start DM task only diff --git a/tests/dmctl_basic/check_list/pause_relay.sh b/tests/dmctl_basic/check_list/pause_relay.sh index c3cc018d8d..9b0b131879 100644 --- a/tests/dmctl_basic/check_list/pause_relay.sh +++ b/tests/dmctl_basic/check_list/pause_relay.sh @@ -6,7 +6,7 @@ function pause_relay_wrong_arg() { "pause-relay <-s source ...> \[flags\]" 1 } -function pause_relay_wihout_worker() { +function pause_relay_without_worker() { run_dm_ctl $WORK_DIR "127.0.0.1:$MASTER_PORT" \ "pause-relay" \ "must specify at least one source" 1 diff --git a/tests/dmctl_basic/check_list/purge_relay.sh b/tests/dmctl_basic/check_list/purge_relay.sh index ed38b34321..c4dc8a21f8 100644 --- a/tests/dmctl_basic/check_list/purge_relay.sh +++ b/tests/dmctl_basic/check_list/purge_relay.sh @@ -6,7 +6,7 @@ function purge_relay_wrong_arg() { "purge-relay <-s source> <-f filename> \[--sub-dir directory\] \[flags\]" 1 } -function purge_relay_wihout_worker() { +function purge_relay_without_worker() { run_dm_ctl $WORK_DIR "127.0.0.1:$MASTER_PORT" \ "purge-relay" \ "must specify at least one source (\`-s\` \/ \`--source\`)" 1 diff --git a/tests/dmctl_basic/check_list/query_status.sh b/tests/dmctl_basic/check_list/query_status.sh index f8dc82be27..bfd2d1371c 100644 --- a/tests/dmctl_basic/check_list/query_status.sh +++ b/tests/dmctl_basic/check_list/query_status.sh @@ -24,7 +24,7 @@ function query_status_with_tasks() { "query-status -s $SOURCE_ID1,$SOURCE_ID2" \ "\"result\": true" 3 \ "\"unit\": \"Sync\"" 2 \ - "\"stage\": \"Running\"" 4 + "\"stage\": \"Running\"" 2 run_dm_ctl $WORK_DIR "127.0.0.1:$MASTER_PORT" \ "query-status" \ "\"result\": true" 1 \ diff --git a/tests/dmctl_basic/check_list/start_relay.sh b/tests/dmctl_basic/check_list/start_relay.sh new file mode 100644 index 0000000000..f79fff3f53 --- /dev/null +++ b/tests/dmctl_basic/check_list/start_relay.sh @@ -0,0 +1,34 @@ +#!/bin/bash + +function start_relay_empty_arg() { + run_dm_ctl $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "start-relay" \ + "start-relay <-s source-id> \[...worker-name\]" 1 +} + +function start_relay_wrong_arg() { + run_dm_ctl $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "start-relay wrong_arg" \ + "must specify one source (\`-s\` \/ \`--source\`)" 1 +} + +function start_relay_without_worker() { + run_dm_ctl $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "start-relay -s $SOURCE_ID1" \ + "must specify at least one worker" 1 +} + +function start_relay_success() { + run_dm_ctl $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "start-relay -s $SOURCE_ID1 worker1" \ + "\"result\": true" 1 + run_dm_ctl $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "start-relay -s $SOURCE_ID2 worker2" \ + "\"result\": true" 1 +} + +function start_relay_fail() { + run_dm_ctl $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "start-relay -s $SOURCE_ID1 worker2" \ + "these workers \[worker2\] have bound for another sources \[$SOURCE_ID2\] respectively" 1 +} diff --git a/tests/dmctl_basic/check_list/stop_relay.sh b/tests/dmctl_basic/check_list/stop_relay.sh new file mode 100644 index 0000000000..6a0bc26da8 --- /dev/null +++ b/tests/dmctl_basic/check_list/stop_relay.sh @@ -0,0 +1,34 @@ +#!/bin/bash + +function stop_relay_empty_arg() { + run_dm_ctl $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "stop-relay" \ + "stop-relay <-s source-id> \[...worker-name\]" 1 +} + +function stop_relay_wrong_arg() { + run_dm_ctl $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "stop-relay wrong_arg" \ + "must specify one source (\`-s\` \/ \`--source\`)" 1 +} + +function stop_relay_without_worker() { + run_dm_ctl $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "stop-relay -s $SOURCE_ID1" \ + "must specify at least one worker" 1 +} + +function stop_relay_success() { + run_dm_ctl $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "stop-relay -s $SOURCE_ID1 worker1" \ + "\"result\": true" 1 + run_dm_ctl $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "stop-relay -s $SOURCE_ID2 worker2" \ + "\"result\": true" 1 +} + +function stop_relay_fail() { + run_dm_ctl $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "stop-relay -s $SOURCE_ID1 worker2" \ + "these workers \[worker2\] have started relay for another sources \[$SOURCE_ID2\] respectively" 1 +} diff --git a/tests/dmctl_basic/run.sh b/tests/dmctl_basic/run.sh index a3070b83f1..c74e0e4d90 100755 --- a/tests/dmctl_basic/run.sh +++ b/tests/dmctl_basic/run.sh @@ -18,7 +18,7 @@ function usage_and_arg_test() { echo "pause_relay_wrong_arg" pause_relay_wrong_arg - pause_relay_wihout_worker + pause_relay_without_worker echo "resume_relay_wrong_arg" resume_relay_wrong_arg @@ -59,7 +59,7 @@ function usage_and_arg_test() { # echo "purge_relay_wrong_arg" purge_relay_wrong_arg - purge_relay_wihout_worker + purge_relay_without_worker purge_relay_filename_with_multi_workers echo "operate_source_empty_arg" @@ -72,6 +72,16 @@ function usage_and_arg_test() { transfer_source_empty_arg transfer_source_less_arg transfer_source_more_arg + + echo "start_relay_empty_arg" + start_relay_empty_arg + start_relay_wrong_arg + start_relay_without_worker + + echo "stop_relay_empty_arg" + stop_relay_empty_arg + stop_relay_wrong_arg + stop_relay_without_worker } function recover_max_binlog_size() { @@ -160,6 +170,9 @@ function run() { transfer_source_valid $SOURCE_ID1 worker1 # transfer to self transfer_source_invalid $SOURCE_ID1 worker2 + start_relay_success + start_relay_fail + echo "pause_relay_success" pause_relay_success query_status_stopped_relay @@ -187,6 +200,10 @@ function run() { "\"stage\": \"Running\"" 4 # update_task_not_paused $TASK_CONF + # stop relay because get_config_to_file will stop source + stop_relay_fail + stop_relay_success + echo "get_config" get_config_wrong_arg get_config_to_file @@ -194,7 +211,7 @@ function run() { # retry to wait for recovered from etcd ready run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" \ "query-status test" \ - "\"stage\": \"Running\"" 4 # relay enabled + "\"stage\": \"Running\"" 2 echo "show_ddl_locks_no_locks" show_ddl_locks_no_locks $TASK_NAME @@ -213,6 +230,8 @@ function run() { # update_task_success_single_worker $TASK_CONF $SOURCE_ID1 # update_task_success $TASK_CONF + start_relay_success + run_sql_file $cur/data/db1.increment.sql $MYSQL_HOST1 $MYSQL_PORT1 $MYSQL_PASSWORD1 run_sql_file $cur/data/db2.increment.sql $MYSQL_HOST2 $MYSQL_PORT2 $MYSQL_PASSWORD2 resume_task_success $TASK_NAME diff --git a/tests/dmctl_command/run.sh b/tests/dmctl_command/run.sh index 15a3603606..669f98f62d 100644 --- a/tests/dmctl_command/run.sh +++ b/tests/dmctl_command/run.sh @@ -6,7 +6,7 @@ cur=$( cd "$( dirname "${BASH_SOURCE[0]}" )" && pwd ) source $cur/../_utils/test_prepare WORK_DIR=$TEST_DIR/$TEST_NAME -help_cnt=35 +help_cnt=37 function run() { # check dmctl output with help flag diff --git a/tests/drop_column_with_index/run.sh b/tests/drop_column_with_index/run.sh index 8c4f7b77a9..1f5421d41f 100755 --- a/tests/drop_column_with_index/run.sh +++ b/tests/drop_column_with_index/run.sh @@ -22,6 +22,10 @@ function run() { sed -i "/relay-binlog-name/i\relay-dir: $WORK_DIR/worker1/relay_log" $WORK_DIR/source1.yaml dmctl_operate_source create $WORK_DIR/source1.yaml $SOURCE_ID1 + run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "start-relay -s $SOURCE_ID1 worker1" \ + "\"result\": true" 1 + # start DM task only dmctl_start_task_standalone "$cur/conf/dm-task.yaml" "--remove-meta" diff --git a/tests/full_mode/run.sh b/tests/full_mode/run.sh index 355a4f495e..5b136a75a0 100755 --- a/tests/full_mode/run.sh +++ b/tests/full_mode/run.sh @@ -27,16 +27,20 @@ function fail_acquire_global_lock() { check_rpc_alive $cur/../bin/check_master_online 127.0.0.1:$MASTER_PORT run_dm_worker $WORK_DIR/worker1 $WORKER1_PORT $cur/conf/dm-worker1.toml check_rpc_alive $cur/../bin/check_worker_online 127.0.0.1:$WORKER1_PORT - run_dm_worker $WORK_DIR/worker2 $WORKER2_PORT $cur/conf/dm-worker2.toml - check_rpc_alive $cur/../bin/check_worker_online 127.0.0.1:$WORKER2_PORT - cp $cur/conf/source1.yaml $WORK_DIR/source1.yaml cp $cur/conf/source2.yaml $WORK_DIR/source2.yaml sed -i "/relay-binlog-name/i\relay-dir: $WORK_DIR/worker1/relay_log" $WORK_DIR/source1.yaml sed -i "/relay-binlog-name/i\relay-dir: $WORK_DIR/worker2/relay_log" $WORK_DIR/source2.yaml dmctl_operate_source create $WORK_DIR/source1.yaml $SOURCE_ID1 + + run_dm_worker $WORK_DIR/worker2 $WORKER2_PORT $cur/conf/dm-worker2.toml + check_rpc_alive $cur/../bin/check_worker_online 127.0.0.1:$WORKER2_PORT dmctl_operate_source create $WORK_DIR/source2.yaml $SOURCE_ID2 + run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "start-relay -s $SOURCE_ID2 worker2" \ + "\"result\": true" 1 + cp $cur/conf/dm-task.yaml $WORK_DIR/dm-task.yaml sed -i '/timezone/i\ignore-checking-items: ["dump_privilege"]' $WORK_DIR/dm-task.yaml run_dm_ctl $WORK_DIR "127.0.0.1:$MASTER_PORT" \ diff --git a/tests/gtid/run.sh b/tests/gtid/run.sh index fe4b52cefa..2f99fa137a 100755 --- a/tests/gtid/run.sh +++ b/tests/gtid/run.sh @@ -23,6 +23,9 @@ function run() { run_dm_worker $WORK_DIR/worker1 $WORKER1_PORT $cur/conf/dm-worker1.toml check_rpc_alive $cur/../bin/check_worker_online 127.0.0.1:$WORKER1_PORT dmctl_operate_source create $WORK_DIR/source1.yaml $SOURCE_ID1 + run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "start-relay -s $SOURCE_ID1 worker1" \ + "\"result\": true" 1 run_dm_worker $WORK_DIR/worker2 $WORKER2_PORT $cur/conf/dm-worker2.toml check_rpc_alive $cur/../bin/check_worker_online 127.0.0.1:$WORKER2_PORT diff --git a/tests/ha/run.sh b/tests/ha/run.sh index 70b093fb63..431c47a999 100755 --- a/tests/ha/run.sh +++ b/tests/ha/run.sh @@ -22,16 +22,21 @@ function run() { run_dm_worker $WORK_DIR/worker1 $WORKER1_PORT $cur/conf/dm-worker1.toml check_rpc_alive $cur/../bin/check_worker_online 127.0.0.1:$WORKER1_PORT - run_dm_worker $WORK_DIR/worker2 $WORKER2_PORT $cur/conf/dm-worker2.toml - check_rpc_alive $cur/../bin/check_worker_online 127.0.0.1:$WORKER2_PORT echo "operate mysql config to worker" cp $cur/conf/source1.yaml $WORK_DIR/source1.yaml cp $cur/conf/source2.yaml $WORK_DIR/source2.yaml sed -i "/relay-binlog-name/i\relay-dir: $WORK_DIR/worker1/relay_log" $WORK_DIR/source1.yaml sed -i "/relay-binlog-name/i\relay-dir: $WORK_DIR/worker2/relay_log" $WORK_DIR/source2.yaml dmctl_operate_source create $WORK_DIR/source1.yaml $SOURCE_ID1 + + run_dm_worker $WORK_DIR/worker2 $WORKER2_PORT $cur/conf/dm-worker2.toml + check_rpc_alive $cur/../bin/check_worker_online 127.0.0.1:$WORKER2_PORT dmctl_operate_source create $WORK_DIR/source2.yaml $SOURCE_ID2 + run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "start-relay -s $SOURCE_ID2 worker2" \ + "\"result\": true" 1 + # join master3 run_dm_master $WORK_DIR/master3 $MASTER_PORT3 $cur/conf/dm-master3.toml check_rpc_alive $cur/../bin/check_master_online 127.0.0.1:$MASTER_PORT3 @@ -67,6 +72,10 @@ function run() { run_dm_worker $WORK_DIR/worker3 $WORKER3_PORT $cur/conf/dm-worker3.toml check_rpc_alive $cur/../bin/check_worker_online 127.0.0.1:$WORKER3_PORT + run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "start-relay -s $SOURCE_ID2 worker2" \ + "\"result\": true" 1 + sleep 8 echo "wait for the task to be scheduled and keep paused" check_http_alive 127.0.0.1:$MASTER_PORT/apis/${API_VERSION}/status/test '"stage": "Paused"' 10 diff --git a/tests/ha_cases/lib.sh b/tests/ha_cases/lib.sh index 3a82858c5d..a865192252 100755 --- a/tests/ha_cases/lib.sh +++ b/tests/ha_cases/lib.sh @@ -70,19 +70,18 @@ function start_cluster() { check_rpc_alive $cur/../bin/check_master_online 127.0.0.1:$MASTER_PORT2 check_rpc_alive $cur/../bin/check_master_online 127.0.0.1:$MASTER_PORT3 + echo "start worker and operate mysql config to worker" run_dm_worker $WORK_DIR/worker1 $WORKER1_PORT $cur/conf/dm-worker1.toml check_rpc_alive $cur/../bin/check_worker_online 127.0.0.1:$WORKER1_PORT - run_dm_worker $WORK_DIR/worker2 $WORKER2_PORT $cur/conf/dm-worker2.toml - check_rpc_alive $cur/../bin/check_worker_online 127.0.0.1:$WORKER2_PORT - echo "operate mysql config to worker" cp $cur/conf/source1.yaml $WORK_DIR/source1.yaml cp $cur/conf/source2.yaml $WORK_DIR/source2.yaml sed -i "/relay-binlog-name/i\relay-dir: $WORK_DIR/worker1/relay_log" $WORK_DIR/source1.yaml sed -i "/relay-binlog-name/i\relay-dir: $WORK_DIR/worker2/relay_log" $WORK_DIR/source2.yaml dmctl_operate_source create $WORK_DIR/source1.yaml $SOURCE_ID1 + run_dm_worker $WORK_DIR/worker2 $WORKER2_PORT $cur/conf/dm-worker2.toml + check_rpc_alive $cur/../bin/check_worker_online 127.0.0.1:$WORKER2_PORT dmctl_operate_source create $WORK_DIR/source2.yaml $SOURCE_ID2 - echo "start DM task" dmctl_start_task } @@ -119,23 +118,26 @@ function start_multi_tasks_cluster() { check_rpc_alive $cur/../bin/check_master_online 127.0.0.1:$MASTER_PORT2 check_rpc_alive $cur/../bin/check_master_online 127.0.0.1:$MASTER_PORT3 + echo "operate mysql config to worker" + cp $cur/conf/source1.yaml $WORK_DIR/source1.yaml + cp $cur/conf/source2.yaml $WORK_DIR/source2.yaml + sed -i "/relay-binlog-name/i\relay-dir: $WORK_DIR/worker1/relay_log" $WORK_DIR/source1.yaml + sed -i "/relay-binlog-name/i\relay-dir: $WORK_DIR/worker2/relay_log" $WORK_DIR/source2.yaml + + # make sure source_i bound to worker_i run_dm_worker $WORK_DIR/worker1 $WORKER1_PORT $cur/conf/dm-worker1.toml check_rpc_alive $cur/../bin/check_worker_online 127.0.0.1:$WORKER1_PORT + dmctl_operate_source create $WORK_DIR/source1.yaml $SOURCE_ID1 run_dm_worker $WORK_DIR/worker2 $WORKER2_PORT $cur/conf/dm-worker2.toml check_rpc_alive $cur/../bin/check_worker_online 127.0.0.1:$WORKER2_PORT + dmctl_operate_source create $WORK_DIR/source2.yaml $SOURCE_ID2 + run_dm_worker $WORK_DIR/worker3 $WORKER3_PORT $cur/conf/dm-worker3.toml check_rpc_alive $cur/../bin/check_worker_online 127.0.0.1:$WORKER3_PORT run_dm_worker $WORK_DIR/worker4 $WORKER4_PORT $cur/conf/dm-worker4.toml check_rpc_alive $cur/../bin/check_worker_online 127.0.0.1:$WORKER4_PORT run_dm_worker $WORK_DIR/worker5 $WORKER5_PORT $cur/conf/dm-worker5.toml check_rpc_alive $cur/../bin/check_worker_online 127.0.0.1:$WORKER5_PORT - echo "operate mysql config to worker" - cp $cur/conf/source1.yaml $WORK_DIR/source1.yaml - cp $cur/conf/source2.yaml $WORK_DIR/source2.yaml - sed -i "/relay-binlog-name/i\relay-dir: $WORK_DIR/worker1/relay_log" $WORK_DIR/source1.yaml - sed -i "/relay-binlog-name/i\relay-dir: $WORK_DIR/worker2/relay_log" $WORK_DIR/source2.yaml - dmctl_operate_source create $WORK_DIR/source1.yaml $SOURCE_ID1 - dmctl_operate_source create $WORK_DIR/source2.yaml $SOURCE_ID2 echo "start DM task" diff --git a/tests/ha_cases/run.sh b/tests/ha_cases/run.sh index e23dabd7d1..2d2f40eda4 100755 --- a/tests/ha_cases/run.sh +++ b/tests/ha_cases/run.sh @@ -19,7 +19,7 @@ function test_running() { # make sure task to step in "Sync" stage run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT3" \ "query-status test" \ - "\"stage\": \"Running\"" 4 \ + "\"stage\": \"Running\"" 2 \ "\"unit\": \"Sync\"" 2 echo "use sync_diff_inspector to check full dump loader" @@ -49,11 +49,11 @@ function test_multi_task_running() { # make sure task to step in "Sync" stage run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT3" \ "query-status test" \ - "\"stage\": \"Running\"" 4 \ + "\"stage\": \"Running\"" 2 \ "\"unit\": \"Sync\"" 2 run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT3" \ "query-status test2" \ - "\"stage\": \"Running\"" 4 \ + "\"stage\": \"Running\"" 2 \ "\"unit\": \"Sync\"" 2 echo "use sync_diff_inspector to check full dump loader" @@ -155,11 +155,11 @@ function test_kill_master() { echo "check master2,3 are running" run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT2" \ "query-status test" \ - "\"stage\": \"Running\"" 4 + "\"stage\": \"Running\"" 2 run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT3" \ "query-status test" \ - "\"stage\": \"Running\"" 4 + "\"stage\": \"Running\"" 2 run_sql_file_withdb $cur/data/db1.increment2.sql $MYSQL_HOST1 $MYSQL_PORT1 $MYSQL_PASSWORD1 $ha_test run_sql_file_withdb $cur/data/db2.increment2.sql $MYSQL_HOST2 $MYSQL_PORT2 $MYSQL_PASSWORD2 $ha_test @@ -175,14 +175,17 @@ function test_kill_and_isolate_worker() { echo "[$(date)] <<<<<< start test_kill_and_isolate_worker >>>>>>" test_running + run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "start-relay -s $SOURCE_ID2 worker2" \ + "\"result\": true" 1 echo "kill dm-worker2" ps aux | grep dm-worker2 |awk '{print $2}'|xargs kill || true check_port_offline $WORKER2_PORT 20 rm -rf $WORK_DIR/worker2/relay_log run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT1" \ - "query-status test" \ - "\"result\": false" 1 + "query-status test" \ + "\"result\": false" 1 run_dm_worker $WORK_DIR/worker3 $WORKER3_PORT $cur/conf/dm-worker3.toml check_rpc_alive $cur/../bin/check_worker_online 127.0.0.1:$WORKER3_PORT @@ -193,6 +196,10 @@ function test_kill_and_isolate_worker() { run_dm_worker $WORK_DIR/worker4 $WORKER4_PORT $cur/conf/dm-worker4.toml check_rpc_alive $cur/../bin/check_worker_online 127.0.0.1:$WORKER4_PORT + run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "start-relay -s $SOURCE_ID2 worker3 worker4" \ + "\"result\": true" 1 + echo "restart dm-worker3" ps aux | grep dm-worker3 |awk '{print $2}'|xargs kill || true check_port_offline $WORKER3_PORT 20 @@ -208,33 +215,33 @@ function test_kill_and_isolate_worker() { isolate_worker 4 "isolate" run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT1" \ "query-status test" \ - "\"stage\": \"Running\"" 4 + "\"stage\": \"Running\"" 3 echo "isolate dm-worker3" isolate_worker 3 "isolate" run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT1" \ "query-status test" \ - "\"stage\": \"Running\"" 2 \ + "\"stage\": \"Running\"" 1 \ "\"result\": false" 1 echo "disable isolate dm-worker4" isolate_worker 4 "disable_isolate" run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT1" \ "query-status test" \ - "\"stage\": \"Running\"" 4 + "\"stage\": \"Running\"" 3 echo "query-status from all dm-master" run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT1" \ "query-status test" \ - "\"stage\": \"Running\"" 4 + "\"stage\": \"Running\"" 3 run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT2" \ "query-status test" \ - "\"stage\": \"Running\"" 4 + "\"stage\": \"Running\"" 3 run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT3" \ "query-status test" \ - "\"stage\": \"Running\"" 4 + "\"stage\": \"Running\"" 3 run_dm_ctl $WORK_DIR "127.0.0.1:$MASTER_PORT" \ "pause-task test"\ @@ -278,7 +285,7 @@ function test_kill_master_in_sync() { check_http_alive 127.0.0.1:$MASTER_PORT2/apis/${API_VERSION}/status/test '"stage": "Running"' 10 run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT2" \ "query-status test" \ - "\"stage\": \"Running\"" 4 + "\"stage\": \"Running\"" 2 # waiting for syncing wait @@ -300,11 +307,17 @@ function test_kill_worker_in_sync() { echo "kill dm-worker1" ps aux | grep dm-worker1 |awk '{print $2}'|xargs kill || true - echo "kill dm-worker2" - ps aux | grep dm-worker2 |awk '{print $2}'|xargs kill || true echo "start worker3" run_dm_worker $WORK_DIR/worker3 $WORKER3_PORT $cur/conf/dm-worker3.toml check_rpc_alive $cur/../bin/check_worker_online 127.0.0.1:$WORKER3_PORT + + # start-relay halfway + run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "start-relay -s $SOURCE_ID1 worker3" \ + "\"result\": true" 1 + + echo "kill dm-worker2" + ps aux | grep dm-worker2 |awk '{print $2}'|xargs kill || true echo "start worker4" run_dm_worker $WORK_DIR/worker4 $WORKER4_PORT $cur/conf/dm-worker4.toml check_rpc_alive $cur/../bin/check_worker_online 127.0.0.1:$WORKER4_PORT @@ -316,15 +329,15 @@ function test_kill_worker_in_sync() { echo "query-status from all dm-master" run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT1" \ "query-status test" \ - "\"stage\": \"Running\"" 4 + "\"stage\": \"Running\"" 3 run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT2" \ "query-status test" \ - "\"stage\": \"Running\"" 4 + "\"stage\": \"Running\"" 3 run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT3" \ "query-status test" \ - "\"stage\": \"Running\"" 4 + "\"stage\": \"Running\"" 3 # waiting for syncing wait @@ -401,7 +414,7 @@ function test_standalone_running() { # test should still running run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" \ "query-status test"\ - "\"stage\": \"Running\"" 2 + "\"stage\": \"Running\"" 1 echo "[$(date)] <<<<<< finish test_standalone_running >>>>>>" } @@ -411,6 +424,13 @@ function test_pause_task() { echo "[$(date)] <<<<<< start test_pause_task >>>>>>" test_multi_task_running + run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "start-relay -s $SOURCE_ID1 worker1" \ + "\"result\": true" 1 + run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "start-relay -s $SOURCE_ID2 worker2" \ + "\"result\": true" 1 + echo "start dumping SQLs into source" load_data $MYSQL_PORT1 $MYSQL_PASSWORD1 "a" & load_data $MYSQL_PORT2 $MYSQL_PASSWORD2 "b" & @@ -465,6 +485,13 @@ function test_stop_task() { echo "[$(date)] <<<<<< start test_stop_task >>>>>>" test_multi_task_running + run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "start-relay -s $SOURCE_ID1 worker1" \ + "\"result\": true" 1 + run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "start-relay -s $SOURCE_ID2 worker2" \ + "\"result\": true" 1 + echo "start dumping SQLs into source" load_data $MYSQL_PORT1 $MYSQL_PASSWORD1 "a" & load_data $MYSQL_PORT2 $MYSQL_PASSWORD2 "b" & @@ -557,7 +584,7 @@ function test_multi_task_reduce_and_restart_worker() { for name in ${task_name[@]}; do run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" \ "query-status $name"\ - "\"stage\": \"Running\"" 4 + "\"stage\": \"Running\"" 2 done if [ $i = 0 ]; then # waiting for syncing @@ -694,6 +721,7 @@ function test_last_bound() { echo "[$(date)] <<<<<< start test_last_bound >>>>>>" test_running + # now in start_cluster, we ensure source_i is bound to worker_i worker1bound=$($PWD/bin/dmctl.test DEVEL --master-addr "127.0.0.1:$MASTER_PORT1" list-member --name worker1 \ | grep 'source' | awk -F: '{print $2}') echo "worker1bound $worker1bound" @@ -701,6 +729,16 @@ function test_last_bound() { | grep 'source' | awk -F: '{print $2}') echo "worker2bound $worker2bound" + run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "start-relay -s $SOURCE_ID1 worker1" \ + "\"result\": true" 1 + run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "start-relay -s $SOURCE_ID2 worker2" \ + "\"result\": true" 1 + run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "query-status test" \ + "\"stage\": \"Running\"" 4 + kill_2_worker_ensure_unbound 1 2 # start 1 then 2 @@ -723,6 +761,20 @@ function test_last_bound() { # kill 12, start 34, kill 34 kill_2_worker_ensure_unbound 1 2 start_2_worker_ensure_bound 3 4 + worker3bound=$($PWD/bin/dmctl.test DEVEL --master-addr "127.0.0.1:$MASTER_PORT1" list-member --name worker3 \ + | grep 'source' | awk -F: '{print $2}' | cut -d'"' -f 2) + worker4bound=$($PWD/bin/dmctl.test DEVEL --master-addr "127.0.0.1:$MASTER_PORT1" list-member --name worker4 \ + | grep 'source' | awk -F: '{print $2}' | cut -d'"' -f 2) + run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "start-relay -s $worker3bound worker3" \ + "\"result\": true" 1 + run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "start-relay -s $worker4bound worker4" \ + "\"result\": true" 1 + run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "query-status test" \ + "\"stage\": \"Running\"" 4 + # let other workers rather then 1 2 forward the syncer's progress 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 diff --git a/tests/ha_master/run.sh b/tests/ha_master/run.sh index 633db79310..56dcd27026 100755 --- a/tests/ha_master/run.sh +++ b/tests/ha_master/run.sh @@ -308,6 +308,12 @@ function run() { test_evict_leader test_list_member # TICASE-942, 944, 945, 946, 947 + worker1bound=$($PWD/bin/dmctl.test DEVEL --master-addr "127.0.0.1:$MASTER_PORT1" list-member --name worker1 \ + | grep 'source' | awk -F: '{print $2}' | cut -d'"' -f 2) + run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "start-relay -s $worker1bound worker1" \ + "\"result\": true" 1 + echo "start DM task" dmctl_start_task "$cur/conf/dm-task.yaml" "--remove-meta" diff --git a/tests/handle_error/run.sh b/tests/handle_error/run.sh index bbb2bfb7c4..c573ea4ddb 100644 --- a/tests/handle_error/run.sh +++ b/tests/handle_error/run.sh @@ -118,7 +118,7 @@ function DM_SKIP_ERROR_SHARDING_CASE() { run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" \ "query-status test" \ - "\"stage\": \"Running\"" 4 + "\"stage\": \"Running\"" 2 run_sql_tidb_with_retry "select count(1) from ${db}.${tb}" "count(1): 8" } @@ -279,7 +279,7 @@ function DM_REPLACE_ERROR_SHARDING_CASE() { run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" \ "query-status test" \ - "\"stage\": \"Running\"" 4 \ + "\"stage\": \"Running\"" 2 \ run_sql_tidb_with_retry "select count(1) from ${db}.${tb}" "count(1): 8" } @@ -348,7 +348,7 @@ function DM_REPLACE_ERROR_MULTIPLE_CASE() { run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" \ "query-status test" \ - "\"stage\": \"Running\"" 4 \ + "\"stage\": \"Running\"" 2 \ run_sql_tidb_with_retry "select count(1) from ${db}.${tb};" "count(1): 2" } @@ -1391,7 +1391,7 @@ function DM_SKIP_INCOMPATIBLE_DDL_CASE() { run_sql_source1 "CREATE FUNCTION ${db}.hello (s CHAR(20)) RETURNS CHAR(50) DETERMINISTIC RETURN CONCAT('Hello, ',s,'!');" run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" \ "query-status test" \ - "\"stage\": \"Running\"" 2 + "\"stage\": \"Running\"" 1 run_sql_source1 "/*!50003 drop function ${db}.hello*/;" run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" \ diff --git a/tests/http_apis/run.sh b/tests/http_apis/run.sh index 0b574026d2..1f4980deab 100644 --- a/tests/http_apis/run.sh +++ b/tests/http_apis/run.sh @@ -50,7 +50,7 @@ function run() { sleep 1 curl -X GET 127.0.0.1:$MASTER_PORT/apis/${API_VERSION}/status/test > $WORK_DIR/status.log - check_log_contains $WORK_DIR/status.log "\"stage\": \"Running\"" 2 + check_log_contains $WORK_DIR/status.log "\"stage\": \"Running\"" 1 check_log_contains $WORK_DIR/status.log "\"name\": \"test\"" 1 echo "get sub task configs" @@ -72,7 +72,7 @@ function run() { sleep 1 curl -X GET 127.0.0.1:$MASTER_PORT/apis/${API_VERSION}/status/test > $WORK_DIR/status.log - check_log_contains $WORK_DIR/status.log "\"stage\": \"Running\"" 2 + check_log_contains $WORK_DIR/status.log "\"stage\": \"Running\"" 1 check_log_contains $WORK_DIR/status.log "\"name\": \"test\"" 1 sleep 1 diff --git a/tests/import_v10x/run.sh b/tests/import_v10x/run.sh index f5455337dc..918b8ed163 100644 --- a/tests/import_v10x/run.sh +++ b/tests/import_v10x/run.sh @@ -71,7 +71,7 @@ function run() { # check task running. run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" \ "query-status test" \ - "\"stage\": \"Running\"" 3 + "\"stage\": \"Running\"" 2 # check task config, just a simple match run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" \ diff --git a/tests/incremental_mode/run.sh b/tests/incremental_mode/run.sh index eee31b4619..be68eab8c0 100755 --- a/tests/incremental_mode/run.sh +++ b/tests/incremental_mode/run.sh @@ -43,8 +43,19 @@ function run() { dmctl_operate_source create $WORK_DIR/source1.yaml $SOURCE_ID1 dmctl_operate_source create $WORK_DIR/source2.yaml $SOURCE_ID2 - # relay should be started after source bounded - sleep 1 + worker1bound=$($PWD/bin/dmctl.test DEVEL --master-addr "127.0.0.1:$MASTER_PORT1" list-member --name worker1 \ + | grep 'source' | awk -F: '{print $2}' | cut -d'"' -f 2) + worker2bound=$($PWD/bin/dmctl.test DEVEL --master-addr "127.0.0.1:$MASTER_PORT1" list-member --name worker2 \ + | grep 'source' | awk -F: '{print $2}' | cut -d'"' -f 2) + run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "start-relay -s $worker1bound worker1" \ + "\"result\": true" 1 + run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "start-relay -s $worker2bound worker2" \ + "\"result\": true" 1 + + # relay should be started after start-relay + sleep 2 # and now default keepalive TTL is 30 minutes killall -9 dm-worker.test sleep 3 @@ -112,11 +123,26 @@ function run() { "operate-source update $WORK_DIR/source2.yaml" \ "Update worker config is not supported by dm-ha now" 1 # update mysql config is not supported by dm-ha now, so we stop and start source again to update source config + + run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "stop-relay -s $worker1bound worker1" \ + "\"result\": true" 1 + run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "stop-relay -s $worker2bound worker2" \ + "\"result\": true" 1 + dmctl_operate_source stop $WORK_DIR/source1.yaml $SOURCE_ID1 dmctl_operate_source stop $WORK_DIR/source2.yaml $SOURCE_ID2 dmctl_operate_source create $WORK_DIR/source1.yaml $SOURCE_ID1 dmctl_operate_source create $WORK_DIR/source2.yaml $SOURCE_ID2 + run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "start-relay -s $worker1bound worker1" \ + "\"result\": true" 1 + run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "start-relay -s $worker2bound worker2" \ + "\"result\": true" 1 + echo "start task in incremental mode" cat $cur/conf/dm-task.yaml > $WORK_DIR/dm-task.yaml sed -i "s/task-mode-placeholder/incremental/g" $WORK_DIR/dm-task.yaml diff --git a/tests/initial_unit/run.sh b/tests/initial_unit/run.sh index 53161bb20a..a81ffe5194 100644 --- a/tests/initial_unit/run.sh +++ b/tests/initial_unit/run.sh @@ -18,9 +18,9 @@ function prepare_data() { function run() { failpoints=( - # 1152 is ErrAbortingConnection - "github.com/pingcap/dm/syncer/LoadCheckpointFailed=return(1152)" - "github.com/pingcap/dm/syncer/GetMasterStatusFailed=return(1152)" + # 1152 is ErrAbortingConnection + "github.com/pingcap/dm/syncer/LoadCheckpointFailed=return(1152)" + "github.com/pingcap/dm/syncer/GetMasterStatusFailed=return(1152)" ) for(( i=0;i<${#failpoints[@]};i++)) do @@ -43,6 +43,10 @@ function run() { sed -i "/relay-binlog-name/i\relay-dir: $WORK_DIR/worker1/relay_log" $WORK_DIR/source1.yaml dmctl_operate_source create $WORK_DIR/source1.yaml $SOURCE_ID1 + run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "start-relay -s $SOURCE_ID1 worker1" \ + "\"result\": true" 1 + echo "start task and query status, the sync unit will initial failed" task_conf="$cur/conf/dm-task.yaml" run_dm_ctl $WORK_DIR "127.0.0.1:$MASTER_PORT" \ diff --git a/tests/online_ddl/run.sh b/tests/online_ddl/run.sh index 59f4a42546..6436bc62b8 100755 --- a/tests/online_ddl/run.sh +++ b/tests/online_ddl/run.sh @@ -16,18 +16,24 @@ function real_run() { run_dm_master $WORK_DIR/master $MASTER_PORT $cur/conf/dm-master.toml check_rpc_alive $cur/../bin/check_master_online 127.0.0.1:$MASTER_PORT + run_dm_worker $WORK_DIR/worker1 $WORKER1_PORT $cur/conf/dm-worker1.toml check_rpc_alive $cur/../bin/check_worker_online 127.0.0.1:$WORKER1_PORT - run_dm_worker $WORK_DIR/worker2 $WORKER2_PORT $cur/conf/dm-worker2.toml - check_rpc_alive $cur/../bin/check_worker_online 127.0.0.1:$WORKER2_PORT # operate mysql config to worker cp $cur/conf/source1.yaml $WORK_DIR/source1.yaml cp $cur/conf/source2.yaml $WORK_DIR/source2.yaml sed -i "/relay-binlog-name/i\relay-dir: $WORK_DIR/worker1/relay_log" $WORK_DIR/source1.yaml sed -i "/relay-binlog-name/i\relay-dir: $WORK_DIR/worker2/relay_log" $WORK_DIR/source2.yaml dmctl_operate_source create $WORK_DIR/source1.yaml $SOURCE_ID1 + + run_dm_worker $WORK_DIR/worker2 $WORKER2_PORT $cur/conf/dm-worker2.toml + check_rpc_alive $cur/../bin/check_worker_online 127.0.0.1:$WORKER2_PORT dmctl_operate_source create $WORK_DIR/source2.yaml $SOURCE_ID2 + run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "start-relay -s $SOURCE_ID2 worker2" \ + "\"result\": true" 1 + # start DM task only cp $cur/conf/dm-task.yaml $WORK_DIR/dm-task-${online_ddl_scheme}.yaml sed -i "s/online-ddl-scheme-placeholder/${online_ddl_scheme}/g" $WORK_DIR/dm-task-${online_ddl_scheme}.yaml @@ -49,6 +55,10 @@ function real_run() { run_dm_worker $WORK_DIR/worker3 $WORKER3_PORT $cur/conf/dm-worker3.toml check_rpc_alive $cur/../bin/check_worker_online 127.0.0.1:$WORKER3_PORT + run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "start-relay -s $SOURCE_ID2 worker3" \ + "\"result\": true" 1 + echo "wait and check task running" run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" \ "query-status test" \ diff --git a/tests/only_dml/run.sh b/tests/only_dml/run.sh index 808eb69629..f40a0a3415 100755 --- a/tests/only_dml/run.sh +++ b/tests/only_dml/run.sh @@ -64,6 +64,13 @@ function run() { check_rpc_alive $cur/../bin/check_worker_online 127.0.0.1:$WORKER2_PORT dmctl_operate_source create $WORK_DIR/source2.yaml $SOURCE_ID2 + run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "start-relay -s $SOURCE_ID1 worker1" \ + "\"result\": true" 1 + run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "start-relay -s $SOURCE_ID2 worker2" \ + "\"result\": true" 1 + # start a task in all mode, and when enter incremental mode, we only execute DML dmctl_start_task $cur/conf/dm-task.yaml check_sync_diff $WORK_DIR $cur/conf/diff_config.toml diff --git a/tests/relay_interrupt/run.sh b/tests/relay_interrupt/run.sh index fab8593b85..e350e921fe 100644 --- a/tests/relay_interrupt/run.sh +++ b/tests/relay_interrupt/run.sh @@ -47,8 +47,12 @@ function run() { sed -i "/relay-binlog-name/i\relay-dir: $WORK_DIR/worker1/relay_log" $WORK_DIR/source1.yaml dmctl_operate_source create $WORK_DIR/source1.yaml $SOURCE_ID1 + run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "start-relay -s $SOURCE_ID1 worker1" \ + "\"result\": true" 1 + echo "query status, relay log failed" - run_dm_ctl $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" \ "query-status -s $SOURCE_ID1" \ "no sub task started" 1 \ "ERROR" 1 diff --git a/tests/sequence_sharding_optimistic/run.sh b/tests/sequence_sharding_optimistic/run.sh index d0b9ab7127..4f6759ec80 100755 --- a/tests/sequence_sharding_optimistic/run.sh +++ b/tests/sequence_sharding_optimistic/run.sh @@ -33,6 +33,12 @@ run() { dmctl_operate_source create $WORK_DIR/source1.yaml $SOURCE_ID1 dmctl_operate_source create $WORK_DIR/source2.yaml $SOURCE_ID2 + worker1bound=$($PWD/bin/dmctl.test DEVEL --master-addr "127.0.0.1:$MASTER_PORT1" list-member --name worker1 \ + | grep 'source' | awk -F: '{print $2}' | cut -d'"' -f 2) + run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "start-relay -s $worker1bound worker1" \ + "\"result\": true" 1 + # try to get schema for the table, the subtask has not started. curl -X PUT ${API_URL} -d '{"op":1, "task":"sequence_sharding_optimistic", "sources": ["mysql-replica-01"], "database":"sharding_seq_opt", "table":"t1"}' > ${WORK_DIR}/get_schema.log check_log_contains ${WORK_DIR}/get_schema.log "sub task with name sequence_sharding_optimistic not found" 1 diff --git a/tests/sharding2/run.sh b/tests/sharding2/run.sh index f0b95343b5..8ebe6cac87 100755 --- a/tests/sharding2/run.sh +++ b/tests/sharding2/run.sh @@ -14,18 +14,24 @@ function run() { run_dm_master $WORK_DIR/master $MASTER_PORT $cur/conf/dm-master.toml check_rpc_alive $cur/../bin/check_master_online 127.0.0.1:$MASTER_PORT + run_dm_worker $WORK_DIR/worker1 $WORKER1_PORT $cur/conf/dm-worker1.toml check_rpc_alive $cur/../bin/check_worker_online 127.0.0.1:$WORKER1_PORT - run_dm_worker $WORK_DIR/worker2 $WORKER2_PORT $cur/conf/dm-worker2.toml - check_rpc_alive $cur/../bin/check_worker_online 127.0.0.1:$WORKER2_PORT # operate mysql config to worker cp $cur/conf/source1.yaml $WORK_DIR/source1.yaml cp $cur/conf/source2.yaml $WORK_DIR/source2.yaml sed -i "/relay-binlog-name/i\relay-dir: $WORK_DIR/worker1/relay_log" $WORK_DIR/source1.yaml sed -i "/relay-binlog-name/i\relay-dir: $WORK_DIR/worker2/relay_log" $WORK_DIR/source2.yaml dmctl_operate_source create $WORK_DIR/source1.yaml $SOURCE_ID1 + + run_dm_worker $WORK_DIR/worker2 $WORKER2_PORT $cur/conf/dm-worker2.toml + check_rpc_alive $cur/../bin/check_worker_online 127.0.0.1:$WORKER2_PORT dmctl_operate_source create $WORK_DIR/source2.yaml $SOURCE_ID2 + run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "start-relay -s $SOURCE_ID2 worker2" \ + "\"result\": true" 1 + # start DM task only dmctl_start_task From 2d75c112e7d263c18d09193be09134363a22dbff Mon Sep 17 00:00:00 2001 From: lance6716 Date: Mon, 22 Mar 2021 13:33:35 +0800 Subject: [PATCH 24/46] fix CI --- pkg/ha/relay.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pkg/ha/relay.go b/pkg/ha/relay.go index cfd4a24c30..6e995c2fb4 100644 --- a/pkg/ha/relay.go +++ b/pkg/ha/relay.go @@ -85,7 +85,7 @@ func GetAllRelayConfig(cli *clientv3.Client) (map[string]map[string]struct{}, in worker := keys[0] var ( ok bool - workers = map[string]struct{}{} + workers map[string]struct{} ) if workers, ok = ret[source]; !ok { workers = map[string]struct{}{} From 4a1989dbd1d965162ca967003d0f9e127acf91ba Mon Sep 17 00:00:00 2001 From: lance6716 Date: Mon, 22 Mar 2021 18:54:40 +0800 Subject: [PATCH 25/46] fix unstable test --- dm/worker/server_test.go | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/dm/worker/server_test.go b/dm/worker/server_test.go index 44e4ffd41b..eb46789bd6 100644 --- a/dm/worker/server_test.go +++ b/dm/worker/server_test.go @@ -507,7 +507,8 @@ func (t *testServer) testRetryConnectMaster(c *C, s *Server, ETCD *embed.Etcd, d time.Sleep(6 * time.Second) // When worker server fail to keepalive with etcd, server should close its worker c.Assert(s.getWorker(true), IsNil) - c.Assert(s.getSourceStatus(true).Result, IsNil) + // source status could be nil or context canceled from handleSourceBound, skip this test + //c.Assert(s.getSourceStatus(true).Result, IsNil) ETCD, err := createMockETCD(dir, "http://"+hostName) c.Assert(err, IsNil) time.Sleep(3 * time.Second) From 2b8c2457cadf38e43bf3167abed4670265e04d98 Mon Sep 17 00:00:00 2001 From: lance6716 Date: Wed, 24 Mar 2021 14:32:40 +0800 Subject: [PATCH 26/46] fix github CI --- tests/upstream_switch/case.sh | 31 +++++++++++++++++-------------- 1 file changed, 17 insertions(+), 14 deletions(-) diff --git a/tests/upstream_switch/case.sh b/tests/upstream_switch/case.sh index 411e7f54f0..8378c37a88 100644 --- a/tests/upstream_switch/case.sh +++ b/tests/upstream_switch/case.sh @@ -65,7 +65,7 @@ function setup_replica() { change_master_to_gtid $master_8_host $slave_8_host } -function run_dm_components() { +function run_dm_components_and_create_sources() { echo "-------run_dm_components--------" pkill -9 dm-master || true @@ -77,22 +77,25 @@ function run_dm_components() { "alive" 1 run_dm_worker $WORK_DIR/worker1 $WORKER1_PORT $CUR/conf/dm-worker1.toml + run_dm_ctl $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "operate-source create $CUR/conf/source1.yaml" \ + "\"result\": true" 2 run_dm_worker $WORK_DIR/worker2 $WORKER2_PORT $CUR/conf/dm-worker2.toml + run_dm_ctl $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "operate-source create $CUR/conf/source2.yaml" \ + "\"result\": true" 2 run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" \ "list-member" \ "alive" 1 \ - "free" 2 + "bound" 2 } -function create_sources() { - echo "-------create_sources--------" +function start_relay() { + echo "-------start_relay--------" - run_dm_ctl $WORK_DIR "127.0.0.1:$MASTER_PORT" \ - "operate-source create $CUR/conf/source1.yaml" \ - "\"result\": true" 2 - run_dm_ctl $WORK_DIR "127.0.0.1:$MASTER_PORT" \ - "operate-source create $CUR/conf/source2.yaml" \ - "\"result\": true" 2 + run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "start-relay -s mysql-replica-02 worker2" \ + "\"result\": true" 1 } function gen_full_data() { @@ -178,8 +181,8 @@ function clean_task() { "stop-task task_pessimistic" \ "\result\": true" 3 run_dm_ctl $WORK_DIR "127.0.0.1:$MASTER_PORT" \ - "pause-relay -s mysql-replica-02" \ - "\result\": true" 2 + "stop-relay -s mysql-replica-02 worker2" \ + "\result\": true" 1 run_dm_ctl $WORK_DIR "127.0.0.1:$MASTER_PORT" \ "operate-source stop mysql-replica-01" \ "\result\": true" 2 @@ -201,8 +204,8 @@ function test() { prepare_binlogs setup_replica gen_full_data - run_dm_components - create_sources + run_dm_components_and_create_sources + start_relay start_task gen_incr_data verify_result From 6e0b336e39dea3dee2726e9a5adf29c6fb36e5ee Mon Sep 17 00:00:00 2001 From: lance6716 Date: Mon, 29 Mar 2021 18:19:01 +0800 Subject: [PATCH 27/46] try resolve confilct --- dm/config/source_config.go | 20 +++++++++----------- dm/master/scheduler/scheduler.go | 10 +--------- pkg/ha/bound.go | 6 +----- 3 files changed, 11 insertions(+), 25 deletions(-) diff --git a/dm/config/source_config.go b/dm/config/source_config.go index 7dda5c9b0c..22cf5db1f2 100644 --- a/dm/config/source_config.go +++ b/dm/config/source_config.go @@ -177,17 +177,15 @@ func (c *SourceConfig) Verify() error { } var err error - if c.EnableRelay { - if len(c.RelayBinLogName) > 0 { - if !binlog.VerifyFilename(c.RelayBinLogName) { - return terror.ErrWorkerRelayBinlogName.Generate(c.RelayBinLogName) - } + if len(c.RelayBinLogName) > 0 { + if !binlog.VerifyFilename(c.RelayBinLogName) { + return terror.ErrWorkerRelayBinlogName.Generate(c.RelayBinLogName) } - if len(c.RelayBinlogGTID) > 0 { - _, err = gtid.ParserGTID(c.Flavor, c.RelayBinlogGTID) - if err != nil { - return terror.WithClass(terror.Annotatef(err, "relay-binlog-gtid %s", c.RelayBinlogGTID), terror.ClassDMWorker) - } + } + if len(c.RelayBinlogGTID) > 0 { + _, err = gtid.ParserGTID(c.Flavor, c.RelayBinlogGTID) + if err != nil { + return terror.WithClass(terror.Annotatef(err, "relay-binlog-gtid %s", c.RelayBinlogGTID), terror.ClassDMWorker) } } @@ -254,7 +252,7 @@ func (c *SourceConfig) Adjust(ctx context.Context, db *sql.DB) (err error) { } } - if c.EnableRelay && len(c.RelayDir) == 0 { + if len(c.RelayDir) == 0 { c.RelayDir = defaultRelayDir } diff --git a/dm/master/scheduler/scheduler.go b/dm/master/scheduler/scheduler.go index df453d7d26..3714699f63 100644 --- a/dm/master/scheduler/scheduler.go +++ b/dm/master/scheduler/scheduler.go @@ -437,8 +437,7 @@ func (s *Scheduler) TransferSource(source, worker string) error { failpoint.Inject("failToReplaceSourceBound", func(_ failpoint.Value) { failpoint.Return(errors.New("failToPutSourceBound")) }) - enableRelay := s.sourceCfgs[source].EnableRelay - _, err := ha.ReplaceSourceBound(s.etcdCli, source, oldWorker.BaseInfo().Name, worker, enableRelay) + _, err := ha.ReplaceSourceBound(s.etcdCli, source, oldWorker.BaseInfo().Name, worker) if err != nil { return err } @@ -1242,13 +1241,6 @@ 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 diff --git a/pkg/ha/bound.go b/pkg/ha/bound.go index 213822dbdc..43b9bd1b01 100644 --- a/pkg/ha/bound.go +++ b/pkg/ha/bound.go @@ -113,8 +113,7 @@ 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 -// 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) { +func ReplaceSourceBound(cli *clientv3.Client, source, oldWorker, newWorker string) (int64, error) { deleteOps := deleteSourceBoundOp(oldWorker) putOps, err := putSourceBoundOp(NewSourceBound(source, newWorker)) if err != nil { @@ -123,9 +122,6 @@ func ReplaceSourceBound(cli *clientv3.Client, source, oldWorker, newWorker strin 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 } From 9e899586a81ca3c48de2ea6eb6f84eb0e98e425e Mon Sep 17 00:00:00 2001 From: lance6716 Date: Mon, 29 Mar 2021 19:42:37 +0800 Subject: [PATCH 28/46] fix CI --- dm/config/source_config_test.go | 3 ++- dm/master/bootstrap_test.go | 1 + tests/shardddl3/run.sh | 2 +- 3 files changed, 4 insertions(+), 2 deletions(-) diff --git a/dm/config/source_config_test.go b/dm/config/source_config_test.go index 59e536cb50..ca83372888 100644 --- a/dm/config/source_config_test.go +++ b/dm/config/source_config_test.go @@ -164,12 +164,13 @@ func (t *testConfig) TestConfigVerify(c *C) { ".*not valid.*", }, { + // after support `start-relay`, we always check Relay related config func() *SourceConfig { cfg := newConfig() cfg.RelayBinLogName = "mysql-binlog" return cfg }, - "", + ".*not valid.*", }, { func() *SourceConfig { diff --git a/dm/master/bootstrap_test.go b/dm/master/bootstrap_test.go index 1a01fdc26c..b4ce9ccd65 100644 --- a/dm/master/bootstrap_test.go +++ b/dm/master/bootstrap_test.go @@ -79,6 +79,7 @@ func (t *testMaster) TestCollectSourceConfigFilesV1Import(c *C) { cfg1.From.Port = port cfg1.From.User = user cfg1.From.Password = password + cfg1.RelayDir = "relay-dir" cfg2 := cfg1.Clone() cfg2.SourceID = "mysql-replica-02" diff --git a/tests/shardddl3/run.sh b/tests/shardddl3/run.sh index 5975a83b5f..2019170782 100644 --- a/tests/shardddl3/run.sh +++ b/tests/shardddl3/run.sh @@ -1042,7 +1042,7 @@ function DM_DropAddColumn_CASE() { # 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 \ + "\"stage\": \"Running\"" 2 \ "\"unit\": \"Sync\"" 2 run_sql_source1 "alter table ${shardddl1}.${tb1} add column b int after a;" From 01cd66a2ce610126a4a03f52b54cc9cd9fb0b5d6 Mon Sep 17 00:00:00 2001 From: lance6716 Date: Wed, 24 Mar 2021 19:03:28 +0800 Subject: [PATCH 29/46] *: `query-status -s` and purge-relay send to all relay worker --- dm/config/source_config.go | 2 + dm/master/scheduler/scheduler.go | 27 ++++- dm/master/scheduler/scheduler_test.go | 23 ++++ dm/master/server.go | 154 +++++++++++++++--------- dm/worker/worker.go | 39 +++++- tests/new_relay/conf/diff_config.toml | 40 ++++++ tests/new_relay/conf/dm-master.toml | 3 + tests/new_relay/conf/dm-task.yaml | 44 +++++++ tests/new_relay/conf/dm-worker1.toml | 2 + tests/new_relay/conf/dm-worker2.toml | 2 + tests/new_relay/conf/dm-worker3.toml | 2 + tests/new_relay/conf/source1.yaml | 13 ++ tests/new_relay/data/db1.increment.sql | 27 +++++ tests/new_relay/data/db1.increment2.sql | 2 + tests/new_relay/data/db1.prepare.sql | 5 + tests/new_relay/run.sh | 62 ++++++++++ tests/others_integration.txt | 1 + 17 files changed, 383 insertions(+), 65 deletions(-) create mode 100644 tests/new_relay/conf/diff_config.toml create mode 100644 tests/new_relay/conf/dm-master.toml create mode 100644 tests/new_relay/conf/dm-task.yaml create mode 100644 tests/new_relay/conf/dm-worker1.toml create mode 100644 tests/new_relay/conf/dm-worker2.toml create mode 100644 tests/new_relay/conf/dm-worker3.toml create mode 100644 tests/new_relay/conf/source1.yaml create mode 100644 tests/new_relay/data/db1.increment.sql create mode 100644 tests/new_relay/data/db1.increment2.sql create mode 100644 tests/new_relay/data/db1.prepare.sql create mode 100755 tests/new_relay/run.sh diff --git a/dm/config/source_config.go b/dm/config/source_config.go index 22cf5db1f2..aa967a764c 100644 --- a/dm/config/source_config.go +++ b/dm/config/source_config.go @@ -9,6 +9,7 @@ import ( "math" "math/rand" "strings" + "time" "github.com/BurntSushi/toml" "github.com/siddontang/go-mysql/mysql" @@ -291,6 +292,7 @@ func (c *SourceConfig) AdjustServerID(ctx context.Context, db *sql.DB) error { return terror.WithScope(err, terror.ScopeUpstream) } + rand.Seed(time.Now().UnixNano()) for i := 0; i < 5; i++ { randomValue := uint32(rand.Intn(100000)) randomServerID := defaultBaseServerID + randomValue diff --git a/dm/master/scheduler/scheduler.go b/dm/master/scheduler/scheduler.go index 3714699f63..1ecaf02b75 100644 --- a/dm/master/scheduler/scheduler.go +++ b/dm/master/scheduler/scheduler.go @@ -781,8 +781,6 @@ func (s *Scheduler) StartRelay(source string, workers []string) error { return terror.ErrSchedulerNotStarted.Generate() } - // TODO: remove old relay config, where caused by EnableRelay - // 1. precheck if _, ok := s.sourceCfgs[source]; !ok { return terror.ErrSchedulerSourceCfgNotExist.Generate(source) @@ -936,6 +934,31 @@ func (s *Scheduler) StopRelay(source string, workers []string) error { return nil } +func (s *Scheduler) GetRelayWorkers(source string) ([]*Worker, error) { + s.mu.RLock() + defer s.mu.RUnlock() + + if !s.started { + return nil, terror.ErrSchedulerNotStarted.Generate() + } + + workers := s.relayWorkers[source] + var ret []*Worker + for w := range workers { + worker, ok := s.workers[w] + if !ok { + // should not happen + s.logger.Error("worker instance for relay worker not found", zap.String("worker", w)) + continue + } + ret = append(ret, worker) + } + sort.Slice(ret, func(i, j int) bool { + return ret[i].baseInfo.Name < ret[j].baseInfo.Name + }) + return ret, nil +} + // UpdateExpectRelayStage updates the current expect relay stage. // now, only support updates: // - from `Running` to `Paused`. diff --git a/dm/master/scheduler/scheduler_test.go b/dm/master/scheduler/scheduler_test.go index 7e492629ae..268d820b0b 100644 --- a/dm/master/scheduler/scheduler_test.go +++ b/dm/master/scheduler/scheduler_test.go @@ -1164,6 +1164,12 @@ func (t *testScheduler) TestStartStopSource(c *C) { // test not exist source c.Assert(terror.ErrSchedulerSourceCfgNotExist.Equal(s.StartRelay(sourceID3, []string{workerName1})), IsTrue) c.Assert(terror.ErrSchedulerSourceCfgNotExist.Equal(s.StopRelay(sourceID4, []string{workerName1})), IsTrue) + noWorkerSources := []string{sourceID1, sourceID2, sourceID3, sourceID4} + for _, source := range noWorkerSources { + workers, err := s.GetRelayWorkers(source) + c.Assert(err, IsNil) + c.Assert(workers, HasLen, 0) + } // start-relay success on bound-same-source and free worker c.Assert(s.StartRelay(sourceID1, []string{workerName1}), IsNil) @@ -1175,6 +1181,9 @@ func (t *testScheduler) TestStartStopSource(c *C) { c.Assert(s.relayWorkers[sourceID1], HasLen, 2) c.Assert(s.relayWorkers[sourceID1], HasKey, workerName1) c.Assert(s.relayWorkers[sourceID1], HasKey, workerName3) + workers, err := s.GetRelayWorkers(sourceID1) + c.Assert(err, IsNil) + c.Assert(workers, DeepEquals, []*Worker{worker1, worker3}) // failed on bound-not-same-source worker and not exist worker c.Assert(terror.ErrSchedulerRelayWorkersWrongBound.Equal(s.StartRelay(sourceID1, []string{workerName2})), IsTrue) @@ -1189,11 +1198,22 @@ func (t *testScheduler) TestStartStopSource(c *C) { c.Assert(s.expectRelayStages, HasKey, sourceID2) c.Assert(s.relayWorkers[sourceID2], HasLen, 1) c.Assert(s.relayWorkers[sourceID2], HasKey, workerName2) + workers, err = s.GetRelayWorkers(sourceID2) + c.Assert(err, IsNil) + c.Assert(workers, DeepEquals, []*Worker{worker2}) // failed on not-same-source worker and not exist worker c.Assert(terror.ErrSchedulerRelayWorkersWrongRelay.Equal(s.StopRelay(sourceID1, []string{workerName2})), IsTrue) c.Assert(terror.ErrSchedulerWorkerNotExist.Equal(s.StopRelay(sourceID1, []string{"not-exist"})), IsTrue) + // nothing changed + workers, err = s.GetRelayWorkers(sourceID1) + c.Assert(err, IsNil) + c.Assert(workers, DeepEquals, []*Worker{worker1, worker3}) + workers, err = s.GetRelayWorkers(sourceID2) + c.Assert(err, IsNil) + c.Assert(workers, DeepEquals, []*Worker{worker2}) + // stop-relay success c.Assert(s.StopRelay(sourceID1, []string{workerName1}), IsNil) c.Assert(s.StopRelay(sourceID1, []string{workerName1}), IsNil) @@ -1202,4 +1222,7 @@ func (t *testScheduler) TestStartStopSource(c *C) { c.Assert(s.expectRelayStages, HasKey, sourceID2) c.Assert(s.relayWorkers, HasLen, 1) c.Assert(s.relayWorkers, HasKey, sourceID2) + workers, err = s.GetRelayWorkers(sourceID1) + c.Assert(err, IsNil) + c.Assert(workers, HasLen, 0) } diff --git a/dm/master/server.go b/dm/master/server.go index 6707d7b6ac..fb345cafd7 100644 --- a/dm/master/server.go +++ b/dm/master/server.go @@ -677,18 +677,24 @@ func (s *Server) QueryStatus(ctx context.Context, req *pb.QueryStatusListRequest Msg: err.Error(), }, nil } - workerRespCh := s.getStatusFromWorkers(ctx, sources, req.Name) - workerRespMap := make(map[string]*pb.QueryStatusResponse, len(sources)) - for len(workerRespCh) > 0 { - workerResp := <-workerRespCh - workerRespMap[workerResp.SourceStatus.Source] = workerResp + queryRelayWorker := false + if len(req.GetSources()) > 0 { + // if user specified sources, query relay workers instead of task workers + queryRelayWorker = true + } + + resps := s.getStatusFromWorkers(ctx, sources, req.Name, queryRelayWorker) + + workerRespMap := make(map[string][]*pb.QueryStatusResponse, len(sources)) + for _, workerResp := range resps { + workerRespMap[workerResp.SourceStatus.Source] = append(workerRespMap[workerResp.SourceStatus.Source], workerResp) } sort.Strings(sources) workerResps := make([]*pb.QueryStatusResponse, 0, len(sources)) for _, worker := range sources { - workerResps = append(workerResps, workerRespMap[worker]) + workerResps = append(workerResps, workerRespMap[worker]...) } resp := &pb.QueryStatusListResponse{ Result: true, @@ -791,37 +797,42 @@ func (s *Server) PurgeWorkerRelay(ctx context.Context, req *pb.PurgeWorkerRelayR workerRespCh := make(chan *pb.CommonWorkerResponse, len(req.Sources)) var wg sync.WaitGroup for _, source := range req.Sources { - wg.Add(1) - go func(source string) { - defer wg.Done() - worker := s.scheduler.GetWorkerBySource(source) - if worker == nil { - workerRespCh <- errorCommonWorkerResponse(fmt.Sprintf("source %s relevant worker-client not found", source), source, "") - return - } - resp, err := worker.SendRequest(ctx, workerReq, s.cfg.RPCTimeout) - var workerResp *pb.CommonWorkerResponse - if err != nil { - workerResp = errorCommonWorkerResponse(err.Error(), source, worker.BaseInfo().Name) - } else { - workerResp = resp.PurgeRelay - } - workerResp.Source = source - workerRespCh <- workerResp - }(source) + workers, err := s.scheduler.GetRelayWorkers(source) + if err != nil { + return nil, err + } + if len(workers) == 0 { + workerRespCh <- errorCommonWorkerResponse(fmt.Sprintf("relay worker for source %s not found, please `start-relay` first", source), source, "") + continue + } + for _, worker := range workers { + wg.Add(1) + go func(worker *scheduler.Worker) { + defer wg.Done() + resp, err3 := worker.SendRequest(ctx, workerReq, s.cfg.RPCTimeout) + var workerResp *pb.CommonWorkerResponse + if err3 != nil { + workerResp = errorCommonWorkerResponse(err3.Error(), source, worker.BaseInfo().Name) + } else { + workerResp = resp.PurgeRelay + } + workerResp.Source = source + workerRespCh <- workerResp + }(worker) + } } wg.Wait() - workerRespMap := make(map[string]*pb.CommonWorkerResponse, len(req.Sources)) + workerRespMap := make(map[string][]*pb.CommonWorkerResponse, len(req.Sources)) for len(workerRespCh) > 0 { workerResp := <-workerRespCh - workerRespMap[workerResp.Source] = workerResp + workerRespMap[workerResp.Source] = append(workerRespMap[workerResp.Source], workerResp) } sort.Strings(req.Sources) workerResps := make([]*pb.CommonWorkerResponse, 0, len(req.Sources)) for _, worker := range req.Sources { - workerResps = append(workerResps, workerRespMap[worker]) + workerResps = append(workerResps, workerRespMap[worker]...) } return &pb.PurgeWorkerRelayResponse{ @@ -879,14 +890,23 @@ func (s *Server) getTaskResources(task string) []string { } // getStatusFromWorkers does RPC request to get status from dm-workers -func (s *Server) getStatusFromWorkers(ctx context.Context, sources []string, taskName string) chan *pb.QueryStatusResponse { +func (s *Server) getStatusFromWorkers(ctx context.Context, sources []string, taskName string, relayWorker bool) []*pb.QueryStatusResponse { workerReq := &workerrpc.Request{ Type: workerrpc.CmdQueryStatus, QueryStatus: &pb.QueryStatusRequest{Name: taskName}, } - workerRespCh := make(chan *pb.QueryStatusResponse, len(sources)) + var ( + workerResps = make([]*pb.QueryStatusResponse, 0, len(sources)) + workerRespMu sync.Mutex + ) + + setWorkerResp := func(resp *pb.QueryStatusResponse) { + workerRespMu.Lock() + workerResps = append(workerResps, resp) + workerRespMu.Unlock() + } - handleErr := func(err error, source string) bool { + handleErr := func(err error, source string) { log.L().Error("response error", zap.Error(err)) resp := &pb.QueryStatusResponse{ Result: false, @@ -895,43 +915,61 @@ func (s *Server) getStatusFromWorkers(ctx context.Context, sources []string, tas Source: source, }, } - workerRespCh <- resp - return false + setWorkerResp(resp) } var wg sync.WaitGroup for _, source := range sources { - wg.Add(1) - go s.ap.Emit(ctx, 0, func(args ...interface{}) { - defer wg.Done() - sourceID, _ := args[0].(string) - worker := s.scheduler.GetWorkerBySource(sourceID) + var ( + workers []*scheduler.Worker + err2 error + ) + if relayWorker { + workers, err2 = s.scheduler.GetRelayWorkers(source) + if err2 != nil { + handleErr(err2, source) + continue + } + } else { + // subtask workers + worker := s.scheduler.GetWorkerBySource(source) if worker == nil { - err := terror.ErrMasterWorkerArgsExtractor.Generatef("%s relevant worker-client not found", sourceID) - handleErr(err, sourceID) - return + err := terror.ErrMasterWorkerArgsExtractor.Generatef("%s relevant worker-client not found", source) + handleErr(err, source) + continue } - resp, err := worker.SendRequest(ctx, workerReq, s.cfg.RPCTimeout) - var workerStatus *pb.QueryStatusResponse - if err != nil { - workerStatus = &pb.QueryStatusResponse{ - Result: false, - Msg: err.Error(), - SourceStatus: &pb.SourceStatus{}, + workers = append(workers, worker) + } + + for _, worker := range workers { + wg.Add(1) + go s.ap.Emit(ctx, 0, func(args ...interface{}) { + defer wg.Done() + sourceID := args[0].(string) + w, _ := args[1].(*scheduler.Worker) + + resp, err := w.SendRequest(ctx, workerReq, s.cfg.RPCTimeout) + var workerStatus *pb.QueryStatusResponse + if err != nil { + workerStatus = &pb.QueryStatusResponse{ + Result: false, + Msg: err.Error(), + SourceStatus: &pb.SourceStatus{}, + } + } else { + workerStatus = resp.QueryStatus } - } else { - workerStatus = resp.QueryStatus - } - workerStatus.SourceStatus.Source = sourceID - workerRespCh <- workerStatus - }, func(args ...interface{}) { - defer wg.Done() - sourceID, _ := args[0].(string) - handleErr(terror.ErrMasterNoEmitToken.Generate(sourceID), sourceID) - }, source) + workerStatus.SourceStatus.Source = sourceID + setWorkerResp(workerStatus) + }, func(args ...interface{}) { + defer wg.Done() + sourceID, _ := args[0].(string) + handleErr(terror.ErrMasterNoEmitToken.Generate(sourceID), sourceID) + }, source, worker) + } } wg.Wait() - return workerRespCh + return workerResps } // TODO: refine the call stack of this API, query worker configs that we needed only diff --git a/dm/worker/worker.go b/dm/worker/worker.go index d080e74519..ed24c24c77 100644 --- a/dm/worker/worker.go +++ b/dm/worker/worker.go @@ -32,6 +32,7 @@ import ( "github.com/pingcap/dm/pkg/etcdutil" "github.com/pingcap/dm/pkg/ha" "github.com/pingcap/dm/pkg/log" + "github.com/pingcap/dm/pkg/streamer" "github.com/pingcap/dm/pkg/terror" "github.com/pingcap/dm/pkg/utils" "github.com/pingcap/dm/relay/purger" @@ -182,6 +183,7 @@ func (w *Worker) Close() { // EnableRelay enables the functionality of start/watch/handle relay func (w *Worker) EnableRelay() (err error) { + w.l.Info("enter EnableRelay") w.Lock() defer w.Unlock() if w.relayEnabled.Get() { @@ -239,7 +241,7 @@ func (w *Worker) EnableRelay() (err error) { } startImmediately := !relayStage.IsDeleted && relayStage.Expect == pb.Stage_Running if startImmediately { - w.l.Info("relay is started") + w.l.Info("start relay for existing relay stage") w.relayHolder.Start() w.relayPurger.Start() } @@ -261,6 +263,7 @@ func (w *Worker) EnableRelay() (err error) { // DisableRelay disables the functionality of start/watch/handle relay func (w *Worker) DisableRelay() { + w.l.Info("enter DisableRelay") w.Lock() defer w.Unlock() if !w.relayEnabled.CompareAndSwap(true, false) { @@ -295,6 +298,7 @@ func (w *Worker) DisableRelay() { // EnableHandleSubtasks enables the functionality of start/watch/handle subtasks func (w *Worker) EnableHandleSubtasks() error { + w.l.Info("enter EnableHandleSubtasks") w.Lock() defer w.Unlock() if w.subTaskEnabled.Get() { @@ -341,6 +345,7 @@ func (w *Worker) EnableHandleSubtasks() error { // DisableHandleSubtasks disables the functionality of start/watch/handle subtasks func (w *Worker) DisableHandleSubtasks() { + w.l.Info("enter DisableHandleSubtasks") if !w.subTaskEnabled.CompareAndSwap(true, false) { w.l.Warn("already disabled handling subtasks") return @@ -765,6 +770,7 @@ func (w *Worker) operateRelay(ctx context.Context, op pb.RelayOp) error { } if w.relayEnabled.Get() { + // TODO: lock the worker? return w.relayHolder.Operate(ctx, op) } @@ -778,12 +784,35 @@ func (w *Worker) PurgeRelay(ctx context.Context, req *pb.PurgeRelayRequest) erro return terror.ErrWorkerAlreadyClosed.Generate() } - if w.relayEnabled.Get() { - return w.relayPurger.Do(ctx, req) + if !w.relayEnabled.Get() { + w.l.Warn("enable-relay is false, ignore purge relay") + return nil } - w.l.Warn("enable-relay is false, ignore purge relay") - return nil + if !w.subTaskEnabled.Get() { + w.l.Info("worker received purge-relay but didn't handling subtasks, read global checkpoint to decided active relay log") + + uuid := w.relayHolder.Status(ctx).RelaySubDir + + _, subTaskCfgs, _, err := w.fetchSubTasksAndAdjust() + if err != nil { + return err + } + for _, subTaskCfg := range subTaskCfgs { + loc, err2 := getMinLocForSubTaskFunc(ctx, subTaskCfg) + if err2 != nil { + return err2 + } + w.l.Info("update active relay log with", + zap.String("task name", subTaskCfg.Name), + zap.String("uuid", uuid), + zap.String("binlog name", loc.Position.Name)) + if err3 := streamer.GetReaderHub().UpdateActiveRelayLog(subTaskCfg.Name, uuid, loc.Position.Name); err3 != nil { + w.l.Error("Error when update active relay log", zap.Error(err3)) + } + } + } + return w.relayPurger.Do(ctx, req) } // ForbidPurge implements PurgeInterceptor.ForbidPurge diff --git a/tests/new_relay/conf/diff_config.toml b/tests/new_relay/conf/diff_config.toml new file mode 100644 index 0000000000..a10afcf774 --- /dev/null +++ b/tests/new_relay/conf/diff_config.toml @@ -0,0 +1,40 @@ +# diff Configuration. + +log-level = "info" + +chunk-size = 1000 + +check-thread-count = 4 + +sample-percent = 100 + +use-checksum = true + +fix-sql-file = "fix.sql" + +# tables need to check. +[[check-tables]] +schema = "new_relay" +tables = ["~t.*"] + +[[table-config]] +schema = "new_relay" +table = "t1" + +[[table-config.source-tables]] +instance-id = "source-1" +schema = "new_relay" +table = "t1" + +[[source-db]] +host = "127.0.0.1" +port = 3306 +user = "root" +password = "123456" +instance-id = "source-1" + +[target-db] +host = "127.0.0.1" +port = 4000 +user = "test" +password = "123456" diff --git a/tests/new_relay/conf/dm-master.toml b/tests/new_relay/conf/dm-master.toml new file mode 100644 index 0000000000..9a36bcbc84 --- /dev/null +++ b/tests/new_relay/conf/dm-master.toml @@ -0,0 +1,3 @@ +# Master Configuration. +master-addr = ":8261" +advertise-addr = "127.0.0.1:8261" diff --git a/tests/new_relay/conf/dm-task.yaml b/tests/new_relay/conf/dm-task.yaml new file mode 100644 index 0000000000..3abc7a87f8 --- /dev/null +++ b/tests/new_relay/conf/dm-task.yaml @@ -0,0 +1,44 @@ +--- +name: test +task-mode: all +is-sharding: false +meta-schema: "dm_meta" +# enable-heartbeat: true +heartbeat-update-interval: 1 +heartbeat-report-interval: 1 +timezone: "Asia/Shanghai" +clean-dump-file: false + +target-database: + host: "127.0.0.1" + port: 4000 + user: "root" + password: "" + +mysql-instances: + - source-id: "mysql-replica-01" + block-allow-list: "instance" + mydumper-config-name: "global" + loader-config-name: "global" + syncer-config-name: "global" + +block-allow-list: + instance: + do-dbs: ["new_relay"] + +mydumpers: + global: + threads: 4 + chunk-filesize: 64 + skip-tz-utc: true + extra-args: "" + +loaders: + global: + pool-size: 16 + dir: "./dumped_data" + +syncers: + global: + worker-count: 16 + batch: 100 diff --git a/tests/new_relay/conf/dm-worker1.toml b/tests/new_relay/conf/dm-worker1.toml new file mode 100644 index 0000000000..7a72ea72bf --- /dev/null +++ b/tests/new_relay/conf/dm-worker1.toml @@ -0,0 +1,2 @@ +name = "worker1" +join = "127.0.0.1:8261" diff --git a/tests/new_relay/conf/dm-worker2.toml b/tests/new_relay/conf/dm-worker2.toml new file mode 100644 index 0000000000..010e21c73e --- /dev/null +++ b/tests/new_relay/conf/dm-worker2.toml @@ -0,0 +1,2 @@ +name = "worker2" +join = "127.0.0.1:8261" diff --git a/tests/new_relay/conf/dm-worker3.toml b/tests/new_relay/conf/dm-worker3.toml new file mode 100644 index 0000000000..ab7e1b9cb3 --- /dev/null +++ b/tests/new_relay/conf/dm-worker3.toml @@ -0,0 +1,2 @@ +name = "worker3" +join = "127.0.0.1:8261" diff --git a/tests/new_relay/conf/source1.yaml b/tests/new_relay/conf/source1.yaml new file mode 100644 index 0000000000..406e7ae254 --- /dev/null +++ b/tests/new_relay/conf/source1.yaml @@ -0,0 +1,13 @@ +source-id: mysql-replica-01 +flavor: 'mysql' +enable-gtid: true +relay-binlog-name: '' +relay-binlog-gtid: '' +enable-relay: true +from: + host: 127.0.0.1 + user: root + password: /Q7B9DizNLLTTfiZHv9WoEAKamfpIUs= + port: 3306 +checker: + check-enable: false diff --git a/tests/new_relay/data/db1.increment.sql b/tests/new_relay/data/db1.increment.sql new file mode 100644 index 0000000000..0ff0143c4e --- /dev/null +++ b/tests/new_relay/data/db1.increment.sql @@ -0,0 +1,27 @@ +use new_relay; +insert into t1 (id, name) values (3, 'Eddard Stark'); +update t1 set name = 'Arya Stark' where id = 1; +update t1 set name = 'Catelyn Stark' where name = 'catelyn'; +flush logs; + +-- test multi column index with generated column +alter table t1 add column info json; +alter table t1 add column gen_id int as (info->"$.id"); +alter table t1 add index multi_col(`id`, `gen_id`); +insert into t1 (id, name, info) values (4, 'gentest', '{"id": 123}'); +insert into t1 (id, name, info) values (5, 'gentest', '{"id": 124}'); +update t1 set info = '{"id": 120}' where id = 1; +update t1 set info = '{"id": 121}' where id = 2; +flush logs; +update t1 set info = '{"id": 122}' where id = 3; + +-- test genColumnCache is reset after ddl +alter table t1 add column info2 varchar(40); +insert into t1 (id, name, info) values (6, 'gentest', '{"id": 125, "test cache": false}'); +alter table t1 add unique key gen_idx(`gen_id`); +update t1 set name = 'gentestxx' where gen_id = 123; + +insert into t1 (id, name, info) values (7, 'gentest', '{"id": 126}'); +update t1 set name = 'gentestxxxxxx' where gen_id = 124; +-- delete with unique key +delete from t1 where gen_id > 124; diff --git a/tests/new_relay/data/db1.increment2.sql b/tests/new_relay/data/db1.increment2.sql new file mode 100644 index 0000000000..c249e20991 --- /dev/null +++ b/tests/new_relay/data/db1.increment2.sql @@ -0,0 +1,2 @@ +insert into t1 (id, name, info) values (8, 'gentest', '{"id": 128}'); +insert into t1 (id, name, info) values (9, 'gentest', '{"id": 129}'), (10, 'gentest', '{"id": 130}'); diff --git a/tests/new_relay/data/db1.prepare.sql b/tests/new_relay/data/db1.prepare.sql new file mode 100644 index 0000000000..540b577fdc --- /dev/null +++ b/tests/new_relay/data/db1.prepare.sql @@ -0,0 +1,5 @@ +drop database if exists `new_relay`; +create database `new_relay`; +use `new_relay`; +create table t1 (id int, name varchar(20), primary key(`id`)); +insert into t1 (id, name) values (1, 'arya'), (2, 'catelyn'); diff --git a/tests/new_relay/run.sh b/tests/new_relay/run.sh new file mode 100755 index 0000000000..8150fd4112 --- /dev/null +++ b/tests/new_relay/run.sh @@ -0,0 +1,62 @@ +#!/bin/bash + +set -eu + +cur=$( cd "$( dirname "${BASH_SOURCE[0]}" )" && pwd ) +source $cur/../_utils/test_prepare +WORK_DIR=$TEST_DIR/$TEST_NAME +TASK_NAME="test" + +API_VERSION="v1alpha1" + +function run() { + run_sql_file $cur/data/db1.prepare.sql $MYSQL_HOST1 $MYSQL_PORT1 $MYSQL_PASSWORD1 + check_contains 'Query OK, 2 rows affected' + + run_dm_master $WORK_DIR/master $MASTER_PORT $cur/conf/dm-master.toml + check_rpc_alive $cur/../bin/check_master_online 127.0.0.1:$MASTER_PORT + run_dm_worker $WORK_DIR/worker1 $WORKER1_PORT $cur/conf/dm-worker1.toml + check_rpc_alive $cur/../bin/check_worker_online 127.0.0.1:$WORKER1_PORT + run_dm_worker $WORK_DIR/worker2 $WORKER2_PORT $cur/conf/dm-worker2.toml + check_rpc_alive $cur/../bin/check_worker_online 127.0.0.1:$WORKER2_PORT + run_dm_worker $WORK_DIR/worker3 $WORKER3_PORT $cur/conf/dm-worker3.toml + check_rpc_alive $cur/../bin/check_worker_online 127.0.0.1:$WORKER3_PORT + + cp $cur/conf/source1.yaml $WORK_DIR/source1.yaml + sed -i "/relay-binlog-name/i\relay-dir: $WORK_DIR/worker1/relay_log" $WORK_DIR/source1.yaml + dmctl_operate_source create $WORK_DIR/source1.yaml $SOURCE_ID1 + + run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "start-relay -s $SOURCE_ID1 worker1 worker2" \ + "\"result\": true" 1 + run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "transfer-source $SOURCE_ID1 worker1" \ + "\"result\": true" 1 + + run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "query-status -s $SOURCE_ID1" \ + "\"result\": true" 3 \ + "\"worker\": \"worker1\"" 1 \ + "\"worker\": \"worker2\"" 1 + + dmctl_start_task_standalone $cur/conf/dm-task.yaml "--remove-meta" + check_sync_diff $WORK_DIR $cur/conf/diff_config.toml + + run_sql_file $cur/data/db1.increment.sql $MYSQL_HOST1 $MYSQL_PORT1 $MYSQL_PASSWORD1 + + # kill dm-worker1, now worker2 is preferred to scheduled + pkill -hup dm-worker1.toml 2>/dev/null || true + wait_process_exit dm-worker1.toml + + run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "query-status -s $SOURCE_ID1" \ + "fail me" 1 +} + +cleanup_data $TEST_NAME +# also cleanup dm processes in case of last run failed +cleanup_process $* +run $* +cleanup_process $* + +echo "[$(date)] <<<<<< test case $TEST_NAME success! >>>>>>" diff --git a/tests/others_integration.txt b/tests/others_integration.txt index b4085e81b1..e187b8c4c3 100644 --- a/tests/others_integration.txt +++ b/tests/others_integration.txt @@ -1,3 +1,4 @@ +new_relay import_v10x tls sharding2 From 734ecefc51af2948549c8a109c93c11ea4fc76a0 Mon Sep 17 00:00:00 2001 From: lance6716 Date: Thu, 25 Mar 2021 12:06:36 +0800 Subject: [PATCH 30/46] *: add test --- dm/master/scheduler/scheduler.go | 22 ++++++++++++- dm/master/server.go | 42 +++++++++++++++++-------- dm/worker/config.go | 4 +++ pkg/streamer/hub.go | 1 + relay/relay.go | 18 +++++++++-- syncer/streamer_controller.go | 1 + tests/_utils/wait_process_exit | 2 +- tests/all_mode/run.sh | 4 +-- tests/ha_cases/run.sh | 3 +- tests/new_relay/data/db1.increment2.sql | 6 ++-- tests/new_relay/data/db1.increment3.sql | 2 ++ tests/new_relay/run.sh | 42 ++++++++++++++++++++++--- 12 files changed, 119 insertions(+), 28 deletions(-) create mode 100644 tests/new_relay/data/db1.increment3.sql diff --git a/dm/master/scheduler/scheduler.go b/dm/master/scheduler/scheduler.go index 1ecaf02b75..3fd5946e20 100644 --- a/dm/master/scheduler/scheduler.go +++ b/dm/master/scheduler/scheduler.go @@ -1502,10 +1502,17 @@ func (s *Scheduler) tryBoundForWorker(w *Worker) (bounded bool, err error) { } // try to find its relay source (currently only one relay source) - if source == "" { + if source != "" { + s.logger.Info("found history source when worker bound", + zap.String("worker", w.BaseInfo().Name), + zap.String("source", source)) + } else { for source2, workers := range s.relayWorkers { if _, ok2 := workers[w.BaseInfo().Name]; ok2 { source = source2 + s.logger.Info("found relay source when worker bound", + zap.String("worker", w.BaseInfo().Name), + zap.String("source", source)) break } } @@ -1525,6 +1532,9 @@ func (s *Scheduler) tryBoundForWorker(w *Worker) (bounded bool, err error) { // randomly pick one from unbounds if source == "" { for source = range s.unbounds { + s.logger.Info("found unbound source when worker bound", + zap.String("worker", w.BaseInfo().Name), + zap.String("source", source)) break // got a source. } } @@ -1567,6 +1577,9 @@ func (s *Scheduler) tryBoundForSource(source string) (bool, error) { } if w.Stage() == WorkerFree { worker = w + s.logger.Info("found free history worker when source bound", + zap.String("worker", workerName), + zap.String("source", source)) break } } @@ -1577,10 +1590,14 @@ func (s *Scheduler) tryBoundForSource(source string) (bool, error) { w, ok := s.workers[workerName] if !ok { // a not found worker, should not happened + s.logger.Info("worker instance not found for relay worker", zap.String("worker", workerName)) continue } if w.Stage() == WorkerFree { worker = w + s.logger.Info("found relay worker when source bound", + zap.String("worker", workerName), + zap.String("source", source)) break } } @@ -1590,6 +1607,9 @@ func (s *Scheduler) tryBoundForSource(source string) (bool, error) { for _, w := range s.workers { if w.Stage() == WorkerFree { worker = w + s.logger.Info("found free worker when source bound", + zap.String("worker", w.BaseInfo().Name), + zap.String("source", source)) break } } diff --git a/dm/master/server.go b/dm/master/server.go index fb345cafd7..3f82b20bca 100644 --- a/dm/master/server.go +++ b/dm/master/server.go @@ -794,7 +794,16 @@ func (s *Server) PurgeWorkerRelay(ctx context.Context, req *pb.PurgeWorkerRelayR }, } - workerRespCh := make(chan *pb.CommonWorkerResponse, len(req.Sources)) + var ( + workerResps = make([]*pb.CommonWorkerResponse, 0, len(req.Sources)) + workerRespMu sync.Mutex + ) + setWorkerResp := func(resp *pb.CommonWorkerResponse) { + workerRespMu.Lock() + workerResps = append(workerResps, resp) + workerRespMu.Unlock() + } + var wg sync.WaitGroup for _, source := range req.Sources { workers, err := s.scheduler.GetRelayWorkers(source) @@ -802,7 +811,7 @@ func (s *Server) PurgeWorkerRelay(ctx context.Context, req *pb.PurgeWorkerRelayR return nil, err } if len(workers) == 0 { - workerRespCh <- errorCommonWorkerResponse(fmt.Sprintf("relay worker for source %s not found, please `start-relay` first", source), source, "") + setWorkerResp(errorCommonWorkerResponse(fmt.Sprintf("relay worker for source %s not found, please `start-relay` first", source), source, "")) continue } for _, worker := range workers { @@ -817,27 +826,26 @@ func (s *Server) PurgeWorkerRelay(ctx context.Context, req *pb.PurgeWorkerRelayR workerResp = resp.PurgeRelay } workerResp.Source = source - workerRespCh <- workerResp + setWorkerResp(workerResp) }(worker) } } wg.Wait() workerRespMap := make(map[string][]*pb.CommonWorkerResponse, len(req.Sources)) - for len(workerRespCh) > 0 { - workerResp := <-workerRespCh + for _, workerResp := range workerResps { workerRespMap[workerResp.Source] = append(workerRespMap[workerResp.Source], workerResp) } sort.Strings(req.Sources) - workerResps := make([]*pb.CommonWorkerResponse, 0, len(req.Sources)) + returnResps := make([]*pb.CommonWorkerResponse, 0, len(req.Sources)) for _, worker := range req.Sources { - workerResps = append(workerResps, workerRespMap[worker]...) + returnResps = append(returnResps, workerRespMap[worker]...) } return &pb.PurgeWorkerRelayResponse{ Result: true, - Sources: workerResps, + Sources: returnResps, }, nil } @@ -895,18 +903,18 @@ func (s *Server) getStatusFromWorkers(ctx context.Context, sources []string, tas Type: workerrpc.CmdQueryStatus, QueryStatus: &pb.QueryStatusRequest{Name: taskName}, } + var ( workerResps = make([]*pb.QueryStatusResponse, 0, len(sources)) workerRespMu sync.Mutex ) - setWorkerResp := func(resp *pb.QueryStatusResponse) { workerRespMu.Lock() workerResps = append(workerResps, resp) workerRespMu.Unlock() } - handleErr := func(err error, source string) { + handleErr := func(err error, source string, worker string) { log.L().Error("response error", zap.Error(err)) resp := &pb.QueryStatusResponse{ Result: false, @@ -915,6 +923,9 @@ func (s *Server) getStatusFromWorkers(ctx context.Context, sources []string, tas Source: source, }, } + if worker != "" { + resp.SourceStatus.Worker = worker + } setWorkerResp(resp) } @@ -927,7 +938,7 @@ func (s *Server) getStatusFromWorkers(ctx context.Context, sources []string, tas if relayWorker { workers, err2 = s.scheduler.GetRelayWorkers(source) if err2 != nil { - handleErr(err2, source) + handleErr(err2, source, "") continue } } else { @@ -935,7 +946,7 @@ func (s *Server) getStatusFromWorkers(ctx context.Context, sources []string, tas worker := s.scheduler.GetWorkerBySource(source) if worker == nil { err := terror.ErrMasterWorkerArgsExtractor.Generatef("%s relevant worker-client not found", source) - handleErr(err, source) + handleErr(err, source, "") continue } workers = append(workers, worker) @@ -964,7 +975,12 @@ func (s *Server) getStatusFromWorkers(ctx context.Context, sources []string, tas }, func(args ...interface{}) { defer wg.Done() sourceID, _ := args[0].(string) - handleErr(terror.ErrMasterNoEmitToken.Generate(sourceID), sourceID) + w, _ := args[1].(*scheduler.Worker) + workerName := "" + if w != nil { + workerName = w.BaseInfo().Name + } + handleErr(terror.ErrMasterNoEmitToken.Generate(sourceID), sourceID, workerName) }, source, worker) } } diff --git a/dm/worker/config.go b/dm/worker/config.go index 4d65ef7a12..b6e2fd2730 100644 --- a/dm/worker/config.go +++ b/dm/worker/config.go @@ -45,6 +45,10 @@ func init() { i := val.(int) defaultKeepAliveTTL = int64(i) }) + failpoint.Inject("defaultRelayKeepAliveTTL", func(val failpoint.Value) { + i := val.(int) + defaultRelayKeepAliveTTL = int64(i) + }) } // NewConfig creates a new base config for worker. diff --git a/pkg/streamer/hub.go b/pkg/streamer/hub.go index 562b3b0ad8..7c75ffd52a 100644 --- a/pkg/streamer/hub.go +++ b/pkg/streamer/hub.go @@ -143,6 +143,7 @@ func (h *ReaderHub) EarliestActiveRelayLog() *RelayLogInfo { return rli } +// TODO: no use now, remove later // RelayMetaHub holds information for relay metas type RelayMetaHub struct { mu sync.RWMutex diff --git a/relay/relay.go b/relay/relay.go index bc22f85a4a..96891a4871 100755 --- a/relay/relay.go +++ b/relay/relay.go @@ -248,9 +248,21 @@ func (r *Relay) process(ctx context.Context) error { if err2 != nil { return err2 } - err2 = r.SaveMeta(mysql.Position{Name: neededBinlogName, Pos: binlog.MinPosition.Pos}, neededBinlogGset) - if err2 != nil { - return err2 + uuidWithSuffix := r.meta.UUID() // only change after switch + uuid, _, err3 := utils.ParseSuffixForUUID(uuidWithSuffix) + if err3 != nil { + r.logger.Error("parse suffix for UUID when relay meta oudated", zap.String("UUID", uuidWithSuffix), zap.Error(err)) + return err3 + } + + pos := &mysql.Position{Name: neededBinlogName, Pos: binlog.MinPosition.Pos} + err = r.meta.AddDir(uuid, pos, neededBinlogGset, r.cfg.UUIDSuffix) + if err != nil { + return err + } + err = r.meta.Load() + if err != nil { + return err } } } diff --git a/syncer/streamer_controller.go b/syncer/streamer_controller.go index 09f10133c8..188c64ede9 100644 --- a/syncer/streamer_controller.go +++ b/syncer/streamer_controller.go @@ -252,6 +252,7 @@ func (c *StreamerController) GetEvent(tctx *tcontext.Context) (event *replicatio if err != nil { if err != context.Canceled && err != context.DeadlineExceeded { c.Lock() + tctx.L().Error("meet error when get binlog event", zap.Error(err)) c.meetError = true c.Unlock() } diff --git a/tests/_utils/wait_process_exit b/tests/_utils/wait_process_exit index 1b300fbd08..f80cdd2baa 100755 --- a/tests/_utils/wait_process_exit +++ b/tests/_utils/wait_process_exit @@ -5,7 +5,7 @@ process=$1 while true do - pgrep $process > /dev/null 2>&1 + pgrep -f $process > /dev/null 2>&1 ret=$? if [ "$ret" != "0" ]; then echo "process $process already exit" diff --git a/tests/all_mode/run.sh b/tests/all_mode/run.sh index 535c6ecdde..c99b62f4de 100755 --- a/tests/all_mode/run.sh +++ b/tests/all_mode/run.sh @@ -219,7 +219,7 @@ function run() { check_log_contain_with_retry '\\"tidb_txn_mode\\":\\"optimistic\\"' $WORK_DIR/worker2/log/dm-worker.log # restart dm-worker1 - pkill -hup dm-worker1.toml 2>/dev/null || true + pkill -hup -f dm-worker1.toml 2>/dev/null || true wait_process_exit dm-worker1.toml run_dm_worker $WORK_DIR/worker1 $WORKER1_PORT $cur/conf/dm-worker1.toml check_rpc_alive $cur/../bin/check_worker_online 127.0.0.1:$WORKER1_PORT @@ -229,7 +229,7 @@ function run() { "worker1" 1 # restart dm-worker2 - pkill -hup dm-worker2.toml 2>/dev/null || true + pkill -hup -f dm-worker2.toml 2>/dev/null || true wait_process_exit dm-worker2.toml run_dm_worker $WORK_DIR/worker2 $WORKER2_PORT $cur/conf/dm-worker2.toml check_rpc_alive $cur/../bin/check_worker_online 127.0.0.1:$WORKER2_PORT diff --git a/tests/ha_cases/run.sh b/tests/ha_cases/run.sh index 2d2f40eda4..d0ae884fd2 100755 --- a/tests/ha_cases/run.sh +++ b/tests/ha_cases/run.sh @@ -369,7 +369,6 @@ function test_standalone_running() { check_sync_diff $WORK_DIR $cur/conf/diff-standalone-config.toml cp $cur/conf/source2.yaml $WORK_DIR/source2.yaml - sed -i "/relay-binlog-name/i\relay-dir: $WORK_DIR/worker2/relay_log" $WORK_DIR/source2.yaml dmctl_operate_source create $WORK_DIR/source2.yaml $SOURCE_ID2 run_dm_ctl $WORK_DIR "127.0.0.1:$MASTER_PORT" \ "start-task $cur/conf/standalone-task2.yaml" \ @@ -396,7 +395,7 @@ function test_standalone_running() { echo "kill $worker_name" ps aux | grep dm-worker${worker_idx} |awk '{print $2}'|xargs kill || true check_port_offline ${worker_ports[$worker_idx]} 20 - rm -rf $WORK_DIR/worker${worker_idx}/relay_log + rm -rf $WORK_DIR/worker${worker_idx}/relay-dir # test running, test2 fail run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" \ diff --git a/tests/new_relay/data/db1.increment2.sql b/tests/new_relay/data/db1.increment2.sql index c249e20991..7c144ac5f6 100644 --- a/tests/new_relay/data/db1.increment2.sql +++ b/tests/new_relay/data/db1.increment2.sql @@ -1,2 +1,4 @@ -insert into t1 (id, name, info) values (8, 'gentest', '{"id": 128}'); -insert into t1 (id, name, info) values (9, 'gentest', '{"id": 129}'), (10, 'gentest', '{"id": 130}'); +flush logs; +insert into new_relay.t1 (id, name, info) values (8, 'gentest', '{"id": 128}'); +insert into new_relay.t1 (id, name, info) values (9, 'gentest', '{"id": 129}'), (10, 'gentest', '{"id": 130}'); +alter table new_relay.t1 add column notused int; diff --git a/tests/new_relay/data/db1.increment3.sql b/tests/new_relay/data/db1.increment3.sql new file mode 100644 index 0000000000..a3eb51c204 --- /dev/null +++ b/tests/new_relay/data/db1.increment3.sql @@ -0,0 +1,2 @@ +flush logs; +alter table new_relay.t1 add column notused2 int; diff --git a/tests/new_relay/run.sh b/tests/new_relay/run.sh index 8150fd4112..530c9629d1 100755 --- a/tests/new_relay/run.sh +++ b/tests/new_relay/run.sh @@ -6,6 +6,7 @@ cur=$( cd "$( dirname "${BASH_SOURCE[0]}" )" && pwd ) source $cur/../_utils/test_prepare WORK_DIR=$TEST_DIR/$TEST_NAME TASK_NAME="test" +SQL_RESULT_FILE="$TEST_DIR/sql_res.$TEST_NAME.txt" API_VERSION="v1alpha1" @@ -23,7 +24,6 @@ function run() { check_rpc_alive $cur/../bin/check_worker_online 127.0.0.1:$WORKER3_PORT cp $cur/conf/source1.yaml $WORK_DIR/source1.yaml - sed -i "/relay-binlog-name/i\relay-dir: $WORK_DIR/worker1/relay_log" $WORK_DIR/source1.yaml dmctl_operate_source create $WORK_DIR/source1.yaml $SOURCE_ID1 run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" \ @@ -44,13 +44,47 @@ function run() { run_sql_file $cur/data/db1.increment.sql $MYSQL_HOST1 $MYSQL_PORT1 $MYSQL_PASSWORD1 - # kill dm-worker1, now worker2 is preferred to scheduled - pkill -hup dm-worker1.toml 2>/dev/null || true + # subtask is preferred to scheduled to another relay worker + pkill -hup -f dm-worker1.toml 2>/dev/null || true wait_process_exit dm-worker1.toml + # worker1 is down, worker2 has running relay and sync unit + run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "query-status -s $SOURCE_ID1" \ + "connect: connection refused" 1 \ + "\"stage\": \"Running\"" 2 + + run_sql_file $cur/data/db1.increment2.sql $MYSQL_HOST1 $MYSQL_PORT1 $MYSQL_PASSWORD1 + check_sync_diff $WORK_DIR $cur/conf/diff_config.toml + + # after restarting, worker will purge relay log directory because checkpoint is newer than relay.meta + run_dm_worker $WORK_DIR/worker1 $WORKER1_PORT $cur/conf/dm-worker1.toml + check_rpc_alive $cur/../bin/check_worker_online 127.0.0.1:$WORKER1_PORT + + run_sql_file $cur/data/db1.increment3.sql $MYSQL_HOST1 $MYSQL_PORT1 $MYSQL_PASSWORD1 + check_sync_diff $WORK_DIR $cur/conf/diff_config.toml run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" \ "query-status -s $SOURCE_ID1" \ - "fail me" 1 + "\"result\": true" 3 \ + "\"worker\": \"worker1\"" 1 \ + "\"worker\": \"worker2\"" 1 + + # test purge-relay for all relay workers + run_sql_source1 "show binary logs\G" + max_binlog_name=$(grep Log_name "$SQL_RESULT_FILE"| tail -n 1 | awk -F":" '{print $NF}') + server_uuid_1=$(tail -n 1 $WORK_DIR/worker1/relay-dir/server-uuid.index) + relay_log_count_1=$(($(ls $WORK_DIR/worker1/relay-dir/$server_uuid_1 | wc -l) - 1)) + server_uuid_2=$(tail -n 1 $WORK_DIR/worker2/relay-dir/server-uuid.index) + relay_log_count_2=$(($(ls $WORK_DIR/worker2/relay-dir/$server_uuid_2 | wc -l) - 1)) + [ "$relay_log_count_1" -ne 1 ] + [ "$relay_log_count_2" -ne 1 ] + run_dm_ctl $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "purge-relay --filename $max_binlog_name -s $SOURCE_ID1" \ + "\"result\": true" 3 + new_relay_log_count_1=$(($(ls $WORK_DIR/worker1/relay-dir/$server_uuid | wc -l) - 1)) + new_relay_log_count_2=$(($(ls $WORK_DIR/worker2/relay-dir/$server_uuid | wc -l) - 1)) + [ "$new_relay_log_count_1" -eq 1 ] + [ "$new_relay_log_count_2" -eq 1 ] } cleanup_data $TEST_NAME From 6f65c369d4f585f8146f182adce44d5c1dbd0418 Mon Sep 17 00:00:00 2001 From: lance6716 Date: Thu, 25 Mar 2021 12:12:30 +0800 Subject: [PATCH 31/46] fix CI --- dm/config/source_config.go | 4 +++ dm/master/scheduler/scheduler.go | 1 + pkg/streamer/hub.go | 55 ++------------------------------ relay/relay.go | 12 +------ tests/new_relay/run.sh | 4 +-- 5 files changed, 10 insertions(+), 66 deletions(-) diff --git a/dm/config/source_config.go b/dm/config/source_config.go index aa967a764c..93559a2bf3 100644 --- a/dm/config/source_config.go +++ b/dm/config/source_config.go @@ -8,6 +8,7 @@ import ( "io/ioutil" "math" "math/rand" + "path/filepath" "strings" "time" @@ -256,6 +257,9 @@ func (c *SourceConfig) Adjust(ctx context.Context, db *sql.DB) (err error) { if len(c.RelayDir) == 0 { c.RelayDir = defaultRelayDir } + if filepath.IsAbs(c.RelayDir) { + log.L().Warn("using an absolute relay path, relay log can't work when starting multiple relay worker") + } return nil } diff --git a/dm/master/scheduler/scheduler.go b/dm/master/scheduler/scheduler.go index 3fd5946e20..be8f987070 100644 --- a/dm/master/scheduler/scheduler.go +++ b/dm/master/scheduler/scheduler.go @@ -934,6 +934,7 @@ func (s *Scheduler) StopRelay(source string, workers []string) error { return nil } +// GetRelayWorkers returns all alive worker instances for a relay source func (s *Scheduler) GetRelayWorkers(source string) ([]*Worker, error) { s.mu.RLock() defer s.mu.RUnlock() diff --git a/pkg/streamer/hub.go b/pkg/streamer/hub.go index 7c75ffd52a..994f270f40 100644 --- a/pkg/streamer/hub.go +++ b/pkg/streamer/hub.go @@ -18,19 +18,14 @@ import ( "strings" "sync" - "github.com/siddontang/go-mysql/mysql" - "github.com/pingcap/dm/pkg/binlog" - "github.com/pingcap/dm/pkg/gtid" "github.com/pingcap/dm/pkg/terror" "github.com/pingcap/dm/pkg/utils" ) var ( - readerHub *ReaderHub // singleton instance - relayMetaHub *RelayMetaHub - relayMetaOnce sync.Once - once sync.Once + readerHub *ReaderHub // singleton instance + once sync.Once ) // RelayLogInfo represents information for relay log @@ -142,49 +137,3 @@ func (h *ReaderHub) EarliestActiveRelayLog() *RelayLogInfo { _, rli := h.rlih.earliest() return rli } - -// TODO: no use now, remove later -// RelayMetaHub holds information for relay metas -type RelayMetaHub struct { - mu sync.RWMutex - meta Meta -} - -// GetRelayMetaHub gets singleton instance of RelayMetaHub -func GetRelayMetaHub() *RelayMetaHub { - relayMetaOnce.Do(func() { - relayMetaHub = &RelayMetaHub{} - }) - return relayMetaHub -} - -// GetMeta gets all metas -func (r *RelayMetaHub) GetMeta() Meta { - r.mu.Lock() - defer r.mu.Unlock() - return r.meta -} - -// SetMeta sets meta -func (r *RelayMetaHub) SetMeta(uuid string, pos mysql.Position, gset gtid.Set) { - gs := "" - if gset != nil { - gs = gset.String() - } - meta := Meta{ - BinLogPos: pos.Pos, - BinLogName: pos.Name, - BinlogGTID: gs, - UUID: uuid, - } - r.mu.Lock() - defer r.mu.Unlock() - r.meta = meta -} - -// ClearMeta clears meta -func (r *RelayMetaHub) ClearMeta() { - r.mu.Lock() - defer r.mu.Unlock() - r.meta = Meta{} -} diff --git a/relay/relay.go b/relay/relay.go index 96891a4871..d324f41413 100755 --- a/relay/relay.go +++ b/relay/relay.go @@ -118,8 +118,6 @@ type Relay struct { sync.RWMutex info *pkgstreamer.RelayLogInfo } - - relayMetaHub *pkgstreamer.RelayMetaHub } // NewRealRelay creates an instance of Relay. @@ -162,9 +160,6 @@ func (r *Relay) Init(ctx context.Context) (err error) { return err } - r.relayMetaHub = pkgstreamer.GetRelayMetaHub() - r.relayMetaHub.ClearMeta() - return reportRelayLogSpaceInBackground(ctx, r.cfg.RelayDir) } @@ -791,17 +786,12 @@ func (r *Relay) IsClosed() bool { // SaveMeta save relay meta and update meta in RelayLogInfo func (r *Relay) SaveMeta(pos mysql.Position, gset gtid.Set) error { - if err := r.meta.Save(pos, gset); err != nil { - return err - } - r.relayMetaHub.SetMeta(r.meta.UUID(), pos, gset) - return nil + return r.meta.Save(pos, gset) } // ResetMeta reset relay meta func (r *Relay) ResetMeta() { r.meta = NewLocalMeta(r.cfg.Flavor, r.cfg.RelayDir) - r.relayMetaHub.ClearMeta() } // FlushMeta flush relay meta diff --git a/tests/new_relay/run.sh b/tests/new_relay/run.sh index 530c9629d1..c49eefaf03 100755 --- a/tests/new_relay/run.sh +++ b/tests/new_relay/run.sh @@ -81,8 +81,8 @@ function run() { run_dm_ctl $WORK_DIR "127.0.0.1:$MASTER_PORT" \ "purge-relay --filename $max_binlog_name -s $SOURCE_ID1" \ "\"result\": true" 3 - new_relay_log_count_1=$(($(ls $WORK_DIR/worker1/relay-dir/$server_uuid | wc -l) - 1)) - new_relay_log_count_2=$(($(ls $WORK_DIR/worker2/relay-dir/$server_uuid | wc -l) - 1)) + new_relay_log_count_1=$(($(ls $WORK_DIR/worker1/relay-dir/$server_uuid_1 | wc -l) - 1)) + new_relay_log_count_2=$(($(ls $WORK_DIR/worker2/relay-dir/$server_uuid_2 | wc -l) - 1)) [ "$new_relay_log_count_1" -eq 1 ] [ "$new_relay_log_count_2" -eq 1 ] } From 502a0cad40a88d8fa1fa0ea229e9e9d7a7a67024 Mon Sep 17 00:00:00 2001 From: lance6716 Date: Thu, 25 Mar 2021 14:57:13 +0800 Subject: [PATCH 32/46] fix unit test --- dm/master/server.go | 4 ++++ dm/master/server_test.go | 49 +++++++++++++++++++++++++++++++++++----- pkg/ha/source.go | 3 ++- 3 files changed, 49 insertions(+), 7 deletions(-) diff --git a/dm/master/server.go b/dm/master/server.go index 3f82b20bca..f2536fd70f 100644 --- a/dm/master/server.go +++ b/dm/master/server.go @@ -815,6 +815,10 @@ func (s *Server) PurgeWorkerRelay(ctx context.Context, req *pb.PurgeWorkerRelayR continue } for _, worker := range workers { + if worker == nil { + setWorkerResp(errorCommonWorkerResponse(fmt.Sprintf("relay worker instance for source %s not found, please `start-relay` first", source), source, "")) + continue + } wg.Add(1) go func(worker *scheduler.Worker) { defer wg.Done() diff --git a/dm/master/server_test.go b/dm/master/server_test.go index 6c71a88777..d7243c5bcb 100644 --- a/dm/master/server_test.go +++ b/dm/master/server_test.go @@ -283,6 +283,39 @@ func testMockScheduler(ctx context.Context, wg *sync.WaitGroup, c *check.C, sour return scheduler2, cancels } +func testMockSchedulerForRelay(ctx context.Context, wg *sync.WaitGroup, c *check.C, sources, workers []string, password string, workerClients map[string]workerrpc.Client) (*scheduler.Scheduler, []context.CancelFunc) { + logger := log.L() + scheduler2 := scheduler.NewScheduler(&logger, config.Security{}) + err := scheduler2.Start(ctx, etcdTestCli) + c.Assert(err, check.IsNil) + cancels := make([]context.CancelFunc, 0, 2) + for i := range workers { + // add worker to scheduler's workers map + name := workers[i] + c.Assert(scheduler2.AddWorker(name, workers[i]), check.IsNil) + scheduler2.SetWorkerClientForTest(name, workerClients[workers[i]]) + // operate mysql config on this worker + cfg := config.NewSourceConfig() + cfg.SourceID = sources[i] + cfg.From.Password = password + c.Assert(scheduler2.AddSourceCfg(*cfg), check.IsNil, check.Commentf("all sources: %v", sources)) + wg.Add(1) + ctx1, cancel1 := context.WithCancel(ctx) + cancels = append(cancels, cancel1) + go func(ctx context.Context, workerName string) { + defer wg.Done() + c.Assert(ha.KeepAlive(ctx, etcdTestCli, workerName, keepAliveTTL), check.IsNil) + }(ctx1, name) + c.Assert(scheduler2.StartRelay(sources[i], []string{workers[i]}), check.IsNil) + c.Assert(utils.WaitSomething(30, 100*time.Millisecond, func() bool { + relayWorkers, err := scheduler2.GetRelayWorkers(sources[i]) + c.Assert(err, check.IsNil) + return len(relayWorkers) == 1 && relayWorkers[0].BaseInfo().Name == name + }), check.IsTrue) + } + return scheduler2, cancels +} + func (t *testMaster) TestQueryStatus(c *check.C) { ctrl := gomock.NewController(c) defer ctrl.Finish() @@ -323,7 +356,7 @@ func (t *testMaster) TestQueryStatus(c *check.C) { t.workerClients[worker] = newMockRPCClient(mockWorkerClient) } ctx, cancel = context.WithCancel(context.Background()) - server.scheduler, _ = testMockScheduler(ctx, &wg, c, sources, workers, "", t.workerClients) + server.scheduler, _ = testMockSchedulerForRelay(ctx, &wg, c, sources, workers, "", t.workerClients) resp, err = server.QueryStatus(context.Background(), &pb.QueryStatusListRequest{ Sources: sources, }) @@ -837,6 +870,10 @@ func (t *testMaster) TestPurgeWorkerRelay(c *check.C) { } } + var wg sync.WaitGroup + ctx, cancel := context.WithCancel(context.Background()) + server.scheduler, _ = testMockSchedulerForRelay(ctx, &wg, c, nil, nil, "", t.workerClients) + // test PurgeWorkerRelay with invalid dm-worker[s] resp, err := server.PurgeWorkerRelay(context.Background(), &pb.PurgeWorkerRelayRequest{ Sources: []string{"invalid-source1", "invalid-source2"}, @@ -848,14 +885,14 @@ func (t *testMaster) TestPurgeWorkerRelay(c *check.C) { c.Assert(resp.Sources, check.HasLen, 2) for _, w := range resp.Sources { c.Assert(w.Result, check.IsFalse) - c.Assert(w.Msg, check.Matches, ".*relevant worker-client not found") + c.Assert(w.Msg, check.Matches, "relay worker for source .* not found.*") } + clearSchedulerEnv(c, cancel, &wg) - var wg sync.WaitGroup - ctx, cancel := context.WithCancel(context.Background()) + ctx, cancel = context.WithCancel(context.Background()) // test PurgeWorkerRelay successfully mockPurgeRelay(true) - server.scheduler, _ = testMockScheduler(ctx, &wg, c, sources, workers, "", t.workerClients) + server.scheduler, _ = testMockSchedulerForRelay(ctx, &wg, c, sources, workers, "", t.workerClients) resp, err = server.PurgeWorkerRelay(context.Background(), &pb.PurgeWorkerRelayRequest{ Sources: sources, Time: now, @@ -872,7 +909,7 @@ func (t *testMaster) TestPurgeWorkerRelay(c *check.C) { ctx, cancel = context.WithCancel(context.Background()) // test PurgeWorkerRelay with error response mockPurgeRelay(false) - server.scheduler, _ = testMockScheduler(ctx, &wg, c, sources, workers, "", t.workerClients) + server.scheduler, _ = testMockSchedulerForRelay(ctx, &wg, c, sources, workers, "", t.workerClients) resp, err = server.PurgeWorkerRelay(context.Background(), &pb.PurgeWorkerRelayRequest{ Sources: sources, Time: now, diff --git a/pkg/ha/source.go b/pkg/ha/source.go index afa31d8f1d..e31122cee5 100644 --- a/pkg/ha/source.go +++ b/pkg/ha/source.go @@ -108,8 +108,9 @@ func ClearTestInfoOperation(cli *clientv3.Client) error { clearBound := clientv3.OpDelete(common.UpstreamBoundWorkerKeyAdapter.Path(), clientv3.WithPrefix()) clearLastBound := clientv3.OpDelete(common.UpstreamLastBoundWorkerKeyAdapter.Path(), clientv3.WithPrefix()) clearRelayStage := clientv3.OpDelete(common.StageRelayKeyAdapter.Path(), clientv3.WithPrefix()) + clearRelayConfig := clientv3.OpDelete(common.UpstreamRelayWorkerKeyAdapter.Path(), clientv3.WithPrefix()) clearSubTaskStage := clientv3.OpDelete(common.StageSubTaskKeyAdapter.Path(), clientv3.WithPrefix()) _, _, err := etcdutil.DoOpsInOneTxnWithRetry(cli, clearSource, clearTask, clearSubTask, clearWorkerInfo, clearBound, - clearLastBound, clearWorkerKeepAlive, clearRelayStage, clearSubTaskStage) + clearLastBound, clearWorkerKeepAlive, clearRelayStage, clearRelayConfig, clearSubTaskStage) return err } From 6c12434a2d8557611cccea3a3c6095e794af010a Mon Sep 17 00:00:00 2001 From: lance6716 Date: Thu, 25 Mar 2021 17:11:19 +0800 Subject: [PATCH 33/46] fix more --- dm/master/server.go | 47 +++++++++++--------- dm/master/server_test.go | 6 +-- tests/_utils/test_prepare | 14 ++++++ tests/_utils/wait_process_exit | 9 ++-- tests/all_mode/run.sh | 4 +- tests/dmctl_basic/check_list/query_status.sh | 2 +- tests/new_relay/run.sh | 2 +- tests/start_task/run.sh | 2 +- 8 files changed, 52 insertions(+), 34 deletions(-) diff --git a/dm/master/server.go b/dm/master/server.go index f2536fd70f..963fc0b50e 100644 --- a/dm/master/server.go +++ b/dm/master/server.go @@ -633,18 +633,16 @@ func extractSources(s *Server, req hasWokers) ([]string, error) { var sources []string if len(req.GetSources()) > 0 { - // query specified dm-workers - invalidWorkers := make([]string, 0, len(req.GetSources())) - for _, source := range req.GetSources() { - w := s.scheduler.GetWorkerBySource(source) - if w == nil || w.Stage() == scheduler.WorkerOffline { - invalidWorkers = append(invalidWorkers, source) + sources = req.GetSources() + var invalidSource []string + for _, source := range sources { + if s.scheduler.GetSourceCfgByID(source) == nil { + invalidSource = append(invalidSource, source) } } - if len(invalidWorkers) > 0 { - return nil, errors.Errorf("%s relevant worker-client not found", strings.Join(invalidWorkers, ", ")) + if len(invalidSource) > 0 { + return nil, errors.Errorf("sources %s haven't been added", invalidSource) } - sources = req.GetSources() } else if len(req.GetName()) > 0 { // query specified task's sources sources = s.getTaskResources(req.GetName()) @@ -820,7 +818,7 @@ func (s *Server) PurgeWorkerRelay(ctx context.Context, req *pb.PurgeWorkerRelayR continue } wg.Add(1) - go func(worker *scheduler.Worker) { + go func(worker *scheduler.Worker, source string) { defer wg.Done() resp, err3 := worker.SendRequest(ctx, workerReq, s.cfg.RPCTimeout) var workerResp *pb.CommonWorkerResponse @@ -831,7 +829,7 @@ func (s *Server) PurgeWorkerRelay(ctx context.Context, req *pb.PurgeWorkerRelayR } workerResp.Source = source setWorkerResp(workerResp) - }(worker) + }(worker, source) } } wg.Wait() @@ -936,8 +934,9 @@ func (s *Server) getStatusFromWorkers(ctx context.Context, sources []string, tas var wg sync.WaitGroup for _, source := range sources { var ( - workers []*scheduler.Worker - err2 error + workers []*scheduler.Worker + workerNameSet = make(map[string]struct{}, 0) + err2 error ) if relayWorker { workers, err2 = s.scheduler.GetRelayWorkers(source) @@ -945,15 +944,21 @@ func (s *Server) getStatusFromWorkers(ctx context.Context, sources []string, tas handleErr(err2, source, "") continue } - } else { - // subtask workers - worker := s.scheduler.GetWorkerBySource(source) - if worker == nil { - err := terror.ErrMasterWorkerArgsExtractor.Generatef("%s relevant worker-client not found", source) - handleErr(err, source, "") - continue + // returned workers is not duplicated + for _, w := range workers { + workerNameSet[w.BaseInfo().Name] = struct{}{} } - workers = append(workers, worker) + } + + // subtask workers + taskWorker := s.scheduler.GetWorkerBySource(source) + if taskWorker == nil && len(workers) == 0 { + err := terror.ErrMasterWorkerArgsExtractor.Generatef("%s relevant worker-client not found", source) + handleErr(err, source, "") + continue + } + if _, ok := workerNameSet[taskWorker.BaseInfo().Name]; !ok { + workers = append(workers, taskWorker) } for _, worker := range workers { diff --git a/dm/master/server_test.go b/dm/master/server_test.go index d7243c5bcb..b056eb6b42 100644 --- a/dm/master/server_test.go +++ b/dm/master/server_test.go @@ -308,8 +308,8 @@ func testMockSchedulerForRelay(ctx context.Context, wg *sync.WaitGroup, c *check }(ctx1, name) c.Assert(scheduler2.StartRelay(sources[i], []string{workers[i]}), check.IsNil) c.Assert(utils.WaitSomething(30, 100*time.Millisecond, func() bool { - relayWorkers, err := scheduler2.GetRelayWorkers(sources[i]) - c.Assert(err, check.IsNil) + relayWorkers, err2 := scheduler2.GetRelayWorkers(sources[i]) + c.Assert(err2, check.IsNil) return len(relayWorkers) == 1 && relayWorkers[0].BaseInfo().Name == name }), check.IsTrue) } @@ -369,7 +369,7 @@ func (t *testMaster) TestQueryStatus(c *check.C) { }) c.Assert(err, check.IsNil) c.Assert(resp.Result, check.IsFalse) - c.Assert(resp.Msg, check.Matches, ".*relevant worker-client not found") + c.Assert(resp.Msg, check.Matches, "sources .* haven't been added") // query with invalid task name resp, err = server.QueryStatus(context.Background(), &pb.QueryStatusListRequest{ diff --git a/tests/_utils/test_prepare b/tests/_utils/test_prepare index 34a9163e94..7521a6864d 100644 --- a/tests/_utils/test_prepare +++ b/tests/_utils/test_prepare @@ -20,6 +20,20 @@ function cleanup_process() { wait_process_exit dm-syncer.test } +function wait_pattern_exit() { + pattern=$1 + while true + do + c=$(pgrep -f $pattern | xargs ps) + if [[ $c == "" ]]; then + echo "wait pattern $pattern exit..." + break + fi + echo $c + sleep 0.2 + done +} + if [ "$RESET_MASTER" = true ]; then run_sql "RESET MASTER" $MYSQL_PORT1 $MYSQL_PASSWORD1 run_sql "RESET MASTER" $MYSQL_PORT2 $MYSQL_PASSWORD2 diff --git a/tests/_utils/wait_process_exit b/tests/_utils/wait_process_exit index f80cdd2baa..0b871a2106 100755 --- a/tests/_utils/wait_process_exit +++ b/tests/_utils/wait_process_exit @@ -5,12 +5,11 @@ process=$1 while true do - pgrep -f $process > /dev/null 2>&1 - ret=$? - if [ "$ret" != "0" ]; then - echo "process $process already exit" + c=$(pgrep $process | xargs ps) + if [[ $c == "" ]]; then + echo "wait process $process exit..." break fi + echo $c sleep 0.2 - echo "wait process $process exit..." done diff --git a/tests/all_mode/run.sh b/tests/all_mode/run.sh index c99b62f4de..c030525ce7 100755 --- a/tests/all_mode/run.sh +++ b/tests/all_mode/run.sh @@ -220,7 +220,7 @@ function run() { # restart dm-worker1 pkill -hup -f dm-worker1.toml 2>/dev/null || true - wait_process_exit dm-worker1.toml + wait_pattern_exit dm-worker1.toml run_dm_worker $WORK_DIR/worker1 $WORKER1_PORT $cur/conf/dm-worker1.toml check_rpc_alive $cur/../bin/check_worker_online 127.0.0.1:$WORKER1_PORT # make sure worker1 have bound a source, and the source should same with bound before @@ -230,7 +230,7 @@ function run() { # restart dm-worker2 pkill -hup -f dm-worker2.toml 2>/dev/null || true - wait_process_exit dm-worker2.toml + wait_pattern_exit dm-worker2.toml run_dm_worker $WORK_DIR/worker2 $WORKER2_PORT $cur/conf/dm-worker2.toml check_rpc_alive $cur/../bin/check_worker_online 127.0.0.1:$WORKER2_PORT diff --git a/tests/dmctl_basic/check_list/query_status.sh b/tests/dmctl_basic/check_list/query_status.sh index bfd2d1371c..a27f38ad00 100644 --- a/tests/dmctl_basic/check_list/query_status.sh +++ b/tests/dmctl_basic/check_list/query_status.sh @@ -9,7 +9,7 @@ function query_status_wrong_arg() { function query_status_wrong_params() { run_dm_ctl $WORK_DIR "127.0.0.1:$MASTER_PORT" \ "query-status -s source-x task-y" \ - "source-x relevant worker-client not found" 1 + "sources \[source-x\] haven't been added" 1 } function query_status_with_no_tasks() { diff --git a/tests/new_relay/run.sh b/tests/new_relay/run.sh index c49eefaf03..db78bbf870 100755 --- a/tests/new_relay/run.sh +++ b/tests/new_relay/run.sh @@ -46,7 +46,7 @@ function run() { # subtask is preferred to scheduled to another relay worker pkill -hup -f dm-worker1.toml 2>/dev/null || true - wait_process_exit dm-worker1.toml + wait_pattern_exit dm-worker1.toml # worker1 is down, worker2 has running relay and sync unit run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" \ "query-status -s $SOURCE_ID1" \ diff --git a/tests/start_task/run.sh b/tests/start_task/run.sh index 1c439ddf63..60a3eaaf96 100644 --- a/tests/start_task/run.sh +++ b/tests/start_task/run.sh @@ -93,7 +93,7 @@ function run() { echo "check un-accessible DM-worker exists" run_dm_ctl $WORK_DIR "127.0.0.1:$MASTER_PORT" \ "query-status -s 127.0.0.1:8888" \ - "127.0.0.1:8888 relevant worker-client not found" 1 + "sources \[127.0.0.1:8888\] haven't been added" 1 echo "start task and will failed" task_conf="$cur/conf/dm-task.yaml" From 27f98fcc41d518386a6aa74f3ee7805351b38851 Mon Sep 17 00:00:00 2001 From: lance6716 Date: Thu, 25 Mar 2021 18:07:16 +0800 Subject: [PATCH 34/46] revert some changes --- dm/master/server.go | 2 +- tests/_utils/wait_process_exit | 9 +++++---- 2 files changed, 6 insertions(+), 5 deletions(-) diff --git a/dm/master/server.go b/dm/master/server.go index 963fc0b50e..6c3ecf7d7d 100644 --- a/dm/master/server.go +++ b/dm/master/server.go @@ -935,7 +935,7 @@ func (s *Server) getStatusFromWorkers(ctx context.Context, sources []string, tas for _, source := range sources { var ( workers []*scheduler.Worker - workerNameSet = make(map[string]struct{}, 0) + workerNameSet = make(map[string]struct{}) err2 error ) if relayWorker { diff --git a/tests/_utils/wait_process_exit b/tests/_utils/wait_process_exit index 0b871a2106..1b300fbd08 100755 --- a/tests/_utils/wait_process_exit +++ b/tests/_utils/wait_process_exit @@ -5,11 +5,12 @@ process=$1 while true do - c=$(pgrep $process | xargs ps) - if [[ $c == "" ]]; then - echo "wait process $process exit..." + pgrep $process > /dev/null 2>&1 + ret=$? + if [ "$ret" != "0" ]; then + echo "process $process already exit" break fi - echo $c sleep 0.2 + echo "wait process $process exit..." done From 691ddea0bad456d074c20e858a6e55a7ab7b0d2a Mon Sep 17 00:00:00 2001 From: lance6716 Date: Thu, 25 Mar 2021 18:51:27 +0800 Subject: [PATCH 35/46] fix CI --- dm/master/server.go | 13 ++++++++----- tests/_utils/test_prepare | 9 +++++---- 2 files changed, 13 insertions(+), 9 deletions(-) diff --git a/dm/master/server.go b/dm/master/server.go index 6c3ecf7d7d..27c111d929 100644 --- a/dm/master/server.go +++ b/dm/master/server.go @@ -950,16 +950,19 @@ func (s *Server) getStatusFromWorkers(ctx context.Context, sources []string, tas } } - // subtask workers + // subtask workers may have been found in relay workers taskWorker := s.scheduler.GetWorkerBySource(source) - if taskWorker == nil && len(workers) == 0 { + if taskWorker != nil { + if _, ok := workerNameSet[taskWorker.BaseInfo().Name]; !ok { + workers = append(workers, taskWorker) + } + } + + if len(workers) == 0 { err := terror.ErrMasterWorkerArgsExtractor.Generatef("%s relevant worker-client not found", source) handleErr(err, source, "") continue } - if _, ok := workerNameSet[taskWorker.BaseInfo().Name]; !ok { - workers = append(workers, taskWorker) - } for _, worker := range workers { wg.Add(1) diff --git a/tests/_utils/test_prepare b/tests/_utils/test_prepare index 7521a6864d..9670a7c691 100644 --- a/tests/_utils/test_prepare +++ b/tests/_utils/test_prepare @@ -24,13 +24,14 @@ function wait_pattern_exit() { pattern=$1 while true do - c=$(pgrep -f $pattern | xargs ps) - if [[ $c == "" ]]; then - echo "wait pattern $pattern exit..." + pgrep -f $pattern > /dev/null 2>&1 + ret=$? + if [ "$ret" != "0" ]; then + echo "pattern $pattern already exit" break fi - echo $c sleep 0.2 + echo "wait pattern $pattern exit..." done } From 73dd7b5bed2a06553087088374b15720fad2b252 Mon Sep 17 00:00:00 2001 From: lance6716 Date: Thu, 25 Mar 2021 19:46:27 +0800 Subject: [PATCH 36/46] try debug CI --- dm/master/server_test.go | 1 + pkg/ha/keepalive.go | 1 + tests/_utils/test_prepare | 6 ++---- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/dm/master/server_test.go b/dm/master/server_test.go index b056eb6b42..6e6edf232f 100644 --- a/dm/master/server_test.go +++ b/dm/master/server_test.go @@ -304,6 +304,7 @@ func testMockSchedulerForRelay(ctx context.Context, wg *sync.WaitGroup, c *check cancels = append(cancels, cancel1) go func(ctx context.Context, workerName string) { defer wg.Done() + // TODO: why this will get context cancel? c.Assert(ha.KeepAlive(ctx, etcdTestCli, workerName, keepAliveTTL), check.IsNil) }(ctx1, name) c.Assert(scheduler2.StartRelay(sources[i], []string{workers[i]}), check.IsNil) diff --git a/pkg/ha/keepalive.go b/pkg/ha/keepalive.go index be4f2e7035..d34c1c6303 100644 --- a/pkg/ha/keepalive.go +++ b/pkg/ha/keepalive.go @@ -154,6 +154,7 @@ func KeepAlive(ctx context.Context, cli *clientv3.Client, workerName string, kee oldLeaseID := leaseID leaseID, err = grantAndPutKV(k, workerEventJSON, newTTL) if err != nil { + log.L().Error("meet error when grantAndPutKV keepalive TTL", zap.Error(err)) keepAliveCancel() // make go vet happy return err } diff --git a/tests/_utils/test_prepare b/tests/_utils/test_prepare index 9670a7c691..65264e66af 100644 --- a/tests/_utils/test_prepare +++ b/tests/_utils/test_prepare @@ -24,11 +24,9 @@ function wait_pattern_exit() { pattern=$1 while true do - pgrep -f $pattern > /dev/null 2>&1 - ret=$? - if [ "$ret" != "0" ]; then + if [ "pgrep -f $pattern" != "0" ]; then echo "pattern $pattern already exit" - break + return 0 fi sleep 0.2 echo "wait pattern $pattern exit..." From 49af33dead3413d0bc30d9e6b23397504c89bc1d Mon Sep 17 00:00:00 2001 From: lance6716 Date: Fri, 26 Mar 2021 10:23:11 +0800 Subject: [PATCH 37/46] fix CI --- pkg/utils/db.go | 27 --------------------------- relay/relay.go | 6 ++++-- 2 files changed, 4 insertions(+), 29 deletions(-) diff --git a/pkg/utils/db.go b/pkg/utils/db.go index 600d694832..7ae0b63ef4 100644 --- a/pkg/utils/db.go +++ b/pkg/utils/db.go @@ -78,33 +78,6 @@ func GetAllServerID(ctx context.Context, db *sql.DB) (map[uint32]struct{}, error return serverIDs, nil } -// ReuseServerID reuse given server ID or get a new server ID -func ReuseServerID(ctx context.Context, serverID uint32, db *sql.DB) (uint32, error) { - serverIDs, err := GetAllServerID(ctx, db) - if err != nil { - return 0, err - } - - if _, ok := serverIDs[serverID]; !ok && serverID > 0 { - // reuse given server ID - return serverID, nil - } - - rand.Seed(time.Now().UnixNano()) - for i := 0; i < 99999; i++ { - randomValue := uint32(rand.Intn(100000)) - randomServerID := uint32(defaultBaseServerID) + randomValue - if _, ok := serverIDs[randomServerID]; ok { - continue - } - - return randomServerID, nil - } - - // should never happened unless the master has too many slave. - return 0, terror.ErrInvalidServerID.Generatef("can't find a random available server ID") -} - // GetRandomServerID gets a random server ID which is not used func GetRandomServerID(ctx context.Context, db *sql.DB) (uint32, error) { rand.Seed(time.Now().UnixNano()) diff --git a/relay/relay.go b/relay/relay.go index d324f41413..6b8e5b987a 100755 --- a/relay/relay.go +++ b/relay/relay.go @@ -728,7 +728,8 @@ func (r *Relay) setUpReader(ctx context.Context) (reader.Reader, error) { ctx2, cancel := context.WithTimeout(ctx, utils.DefaultDBTimeout) defer cancel() - randomServerID, err := utils.ReuseServerID(ctx2, r.cfg.ServerID, r.db) + // always use a new random serverID + randomServerID, err := utils.GetRandomServerID(ctx2, r.db) if err != nil { // should never happened unless the master has too many slave return nil, terror.Annotate(err, "fail to get random server id for relay reader") @@ -993,7 +994,8 @@ func (r *Relay) setSyncConfig() error { func (r *Relay) adjustGTID(ctx context.Context, gset gtid.Set) (gtid.Set, error) { // setup a TCP binlog reader (because no relay can be used when upgrading). syncCfg := r.syncerCfg - randomServerID, err := utils.ReuseServerID(ctx, r.cfg.ServerID, r.db) + // always use a new random serverID + randomServerID, err := utils.GetRandomServerID(ctx, r.db) if err != nil { return nil, terror.Annotate(err, "fail to get random server id when relay adjust gtid") } From 9d79abcab2d2ec799c0b36265642fb3475751ba9 Mon Sep 17 00:00:00 2001 From: lance6716 Date: Fri, 26 Mar 2021 14:01:29 +0800 Subject: [PATCH 38/46] fix unstable behaviour --- dm/worker/server.go | 2 ++ 1 file changed, 2 insertions(+) diff --git a/dm/worker/server.go b/dm/worker/server.go index 80796abbc3..9c8b76ef17 100644 --- a/dm/worker/server.go +++ b/dm/worker/server.go @@ -530,6 +530,7 @@ func (s *Server) getSourceStatus(needLock bool) pb.SourceStatus { return s.sourceStatus } +// TODO: move some call to setWorker/getOrStartWorker func (s *Server) setSourceStatus(source string, err error, needLock bool) { if needLock { s.Lock() @@ -809,6 +810,7 @@ func (s *Server) getOrStartWorker(cfg *config.SourceConfig, needLock bool) (*Wor return nil, terror.ErrWorkerAlreadyStart.Generate(w.name, w.cfg.SourceID, cfg.SourceID) } + log.L().Info("will start a now worker", zap.String("sourceID", cfg.SourceID)) w, err := NewWorker(cfg, s.etcdClient, s.cfg.Name) if err != nil { return nil, err From 99fd149ccec091a676b84802e44bc848c0674f63 Mon Sep 17 00:00:00 2001 From: lance6716 Date: Tue, 30 Mar 2021 14:15:39 +0800 Subject: [PATCH 39/46] address comments from previous PR, and refine comment --- dm/master/scheduler/scheduler.go | 2 +- dm/master/server_test.go | 2 - dm/worker/server.go | 93 ++++++++++++++++---------------- 3 files changed, 48 insertions(+), 49 deletions(-) diff --git a/dm/master/scheduler/scheduler.go b/dm/master/scheduler/scheduler.go index be8f987070..64bc7dc557 100644 --- a/dm/master/scheduler/scheduler.go +++ b/dm/master/scheduler/scheduler.go @@ -1682,7 +1682,7 @@ func (s *Scheduler) deleteWorker(name string) { // - update the stage of worker to `Bound`. // - record the bound relationship and last bound relationship in the scheduler. // this func is called after the bound relationship existed in etcd. -// TODO: update s.unbounds here +// TODO: we could only let updateStatusForBound and updateStatusForUnbound to update s.unbounds/bounds/lastBound func (s *Scheduler) updateStatusForBound(w *Worker, b ha.SourceBound) error { err := w.ToBound(b) if err != nil { diff --git a/dm/master/server_test.go b/dm/master/server_test.go index 6e6edf232f..6eb4f84325 100644 --- a/dm/master/server_test.go +++ b/dm/master/server_test.go @@ -958,8 +958,6 @@ func (t *testMaster) TestOperateWorkerRelayTask(c *check.C) { // 1. test pause-relay successfully resp, err = server.OperateWorkerRelayTask(context.Background(), pauseReq) c.Assert(err, check.IsNil) - fmt.Println(resp) - // you didn't create relay stage c.Assert(resp.Result, check.IsTrue) for _, source := range sources { t.relayStageMatch(c, server.scheduler, source, pb.Stage_Paused) diff --git a/dm/worker/server.go b/dm/worker/server.go index 9c8b76ef17..69f023c4f6 100644 --- a/dm/worker/server.go +++ b/dm/worker/server.go @@ -137,15 +137,7 @@ func (s *Server) Start() error { } 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 { + if err2 := s.enableRelay(relaySource, true); err2 != nil { return err2 } } @@ -164,17 +156,10 @@ func (s *Server) Start() error { } 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, 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 err2 = w.EnableHandleSubtasks(); err2 != nil { + if err2 := s.enableHandleSubtasks(&sourceCfg, true); err2 != nil { return err2 } - w.l.Info("started to handle mysql source", zap.String("sourceCfg", sourceCfg.String())) + log.L().Info("started to handle mysql source", zap.String("sourceCfg", sourceCfg.String())) } s.wg.Add(1) @@ -350,15 +335,9 @@ func (s *Server) observeRelayConfig(ctx context.Context, rev int64) error { 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() + log.L().Info("will recover observeRelayConfig", + zap.String("relay source", relaySource.SourceID)) + return s.enableRelay(relaySource, false) }() if err2 != nil { return err2 @@ -442,15 +421,9 @@ func (s *Server) observeSourceBound(ctx context.Context, rev int64) error { 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 + log.L().Info("will recover observeSourceBound", + zap.String("relay source", cfg.SourceID)) + return s.enableHandleSubtasks(&cfg, false) }() if err2 != nil { return err2 @@ -504,7 +477,7 @@ func (s *Server) Close() { s.wg.Wait() } -// is needLock is false, we should make sure Server has been locked in caller +// if needLock is false, we should make sure Server has been locked in caller func (s *Server) getWorker(needLock bool) *Worker { if needLock { s.Lock() @@ -513,7 +486,7 @@ func (s *Server) getWorker(needLock bool) *Worker { return s.worker } -// is needLock is false, we should make sure Server has been locked in caller +// if needLock is false, we should make sure Server has been locked in caller func (s *Server) setWorker(worker *Worker, needLock bool) { if needLock { s.Lock() @@ -662,12 +635,25 @@ func (s *Server) operateSourceBound(bound ha.SourceBound) error { if !ok { return terror.ErrWorkerFailToGetSourceConfigFromEtcd.Generate(bound.Source) } - w, err := s.getOrStartWorker(&sourceCfg, true) - s.setSourceStatus(bound.Source, err, true) + return s.enableHandleSubtasks(&sourceCfg, true) +} + +func (s *Server) enableHandleSubtasks(sourceCfg *config.SourceConfig, needLock bool) error { + if needLock { + s.Lock() + defer s.Unlock() + } + + w, err := s.getOrStartWorker(sourceCfg, false) + s.setSourceStatus(sourceCfg.SourceID, err, false) if err != nil { return err } - return w.EnableHandleSubtasks() + if err2 := w.EnableHandleSubtasks(); err2 != nil { + s.setSourceStatus(sourceCfg.SourceID, err2, false) + return err2 + } + return nil } func (s *Server) disableHandleSubtasks(source string) error { @@ -700,13 +686,28 @@ func (s *Server) operateRelaySource(relaySource ha.RelaySource) error { 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 + return s.enableRelay(&sourceCfg, true) +} + +func (s *Server) enableRelay(sourceCfg *config.SourceConfig, needLock bool) error { + if needLock { + s.Lock() + defer s.Unlock() + } + + w, err2 := s.getOrStartWorker(sourceCfg, false) + s.setSourceStatus(sourceCfg.SourceID, err2, false) + 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) - return w.EnableRelay() + if err2 = w.EnableRelay(); err2 != nil { + s.setSourceStatus(sourceCfg.SourceID, err2, false) + return err2 + } + return nil } func (s *Server) disableRelay(source string) error { From b5f57068f1864982113baa2c148c919373d8460d Mon Sep 17 00:00:00 2001 From: lance6716 Date: Wed, 31 Mar 2021 11:57:02 +0800 Subject: [PATCH 40/46] debug CI --- dm/master/shardddl/optimist.go | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/dm/master/shardddl/optimist.go b/dm/master/shardddl/optimist.go index 9fabc1494e..0aa1f72749 100644 --- a/dm/master/shardddl/optimist.go +++ b/dm/master/shardddl/optimist.go @@ -327,6 +327,12 @@ func (o *Optimist) recoverLocks( lockJoined, lockTTS := o.buildLockJoinedAndTTS(ifm) // build lock and restore table info o.logger.Info("rebuild locks and tables") + o.logger.Debug("(REMOVE ME) will rebuild locks and tables", + zap.Any("info map", ifm), + zap.Any("operation map", opm), + zap.Any("column map", colm), + zap.Any("lockJoined", lockJoined), + zap.Any("lockTTS", lockTTS)) o.lk.RebuildLocksAndTables(o.cli, ifm, colm, lockJoined, lockTTS) // sort infos by revision infos := sortInfos(ifm) From 57addc163f14410b544f703185c586482d321a18 Mon Sep 17 00:00:00 2001 From: lance6716 Date: Wed, 31 Mar 2021 17:42:25 +0800 Subject: [PATCH 41/46] test if all relay workers are down, non-relay workers could sync --- tests/new_relay/data/db1.increment4.sql | 2 ++ tests/new_relay/run.sh | 14 ++++++++++++++ 2 files changed, 16 insertions(+) create mode 100644 tests/new_relay/data/db1.increment4.sql diff --git a/tests/new_relay/data/db1.increment4.sql b/tests/new_relay/data/db1.increment4.sql new file mode 100644 index 0000000000..095c538596 --- /dev/null +++ b/tests/new_relay/data/db1.increment4.sql @@ -0,0 +1,2 @@ +alter table new_relay.t1 drop column notused, drop column notused2; +insert into new_relay.t1 (id, name, info) values (11, 'gentest', '{"id": 131}'), (12, 'gentest', '{"id": 132}'); diff --git a/tests/new_relay/run.sh b/tests/new_relay/run.sh index db78bbf870..a76cd68868 100755 --- a/tests/new_relay/run.sh +++ b/tests/new_relay/run.sh @@ -85,6 +85,20 @@ function run() { new_relay_log_count_2=$(($(ls $WORK_DIR/worker2/relay-dir/$server_uuid_2 | wc -l) - 1)) [ "$new_relay_log_count_1" -eq 1 ] [ "$new_relay_log_count_2" -eq 1 ] + + pkill -hup -f dm-worker1.toml 2>/dev/null || true + wait_pattern_exit dm-worker1.toml + pkill -hup -f dm-worker2.toml 2>/dev/null || true + wait_pattern_exit dm-worker2.toml + + # if all relay workers are offline, relay-not-enabled worker should continue to sync + run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "query-status -s $SOURCE_ID1" \ + "\"result\": true" 2 \ + "\"worker\": \"worker3\"" 1 + + run_sql_file $cur/data/db1.increment4.sql $MYSQL_HOST1 $MYSQL_PORT1 $MYSQL_PASSWORD1 + check_sync_diff $WORK_DIR $cur/conf/diff_config.toml } cleanup_data $TEST_NAME From 4f66d996699090a48ecd9d7aaa5952bb2bf0d7a3 Mon Sep 17 00:00:00 2001 From: lance6716 Date: Wed, 31 Mar 2021 17:44:00 +0800 Subject: [PATCH 42/46] remove debug log --- dm/master/shardddl/optimist.go | 6 ------ 1 file changed, 6 deletions(-) diff --git a/dm/master/shardddl/optimist.go b/dm/master/shardddl/optimist.go index 0aa1f72749..9fabc1494e 100644 --- a/dm/master/shardddl/optimist.go +++ b/dm/master/shardddl/optimist.go @@ -327,12 +327,6 @@ func (o *Optimist) recoverLocks( lockJoined, lockTTS := o.buildLockJoinedAndTTS(ifm) // build lock and restore table info o.logger.Info("rebuild locks and tables") - o.logger.Debug("(REMOVE ME) will rebuild locks and tables", - zap.Any("info map", ifm), - zap.Any("operation map", opm), - zap.Any("column map", colm), - zap.Any("lockJoined", lockJoined), - zap.Any("lockTTS", lockTTS)) o.lk.RebuildLocksAndTables(o.cli, ifm, colm, lockJoined, lockTTS) // sort infos by revision infos := sortInfos(ifm) From 8ed8363cca746285791b17b9a6ad123fa3a00484 Mon Sep 17 00:00:00 2001 From: lance6716 Date: Tue, 6 Apr 2021 13:49:43 +0800 Subject: [PATCH 43/46] fix history worker is prior to free relay worker --- dm/master/scheduler/scheduler.go | 54 ++++++++++++++++++++++---------- 1 file changed, 38 insertions(+), 16 deletions(-) diff --git a/dm/master/scheduler/scheduler.go b/dm/master/scheduler/scheduler.go index 87e30f2474..a7c84f5be4 100644 --- a/dm/master/scheduler/scheduler.go +++ b/dm/master/scheduler/scheduler.go @@ -1565,31 +1565,34 @@ func (s *Scheduler) tryBoundForWorker(w *Worker) (bounded bool, err error) { // returns (true, nil) after bounded. // called should update the s.unbounds func (s *Scheduler) tryBoundForSource(source string) (bool, error) { - // 1. try to find history workers... var worker *Worker - for workerName, bound := range s.lastBound { - if bound.Source == source { - w, ok := s.workers[workerName] - if !ok { - // a not found worker - continue - } - if w.Stage() == WorkerFree { - worker = w - s.logger.Info("found free history worker when source bound", - zap.String("worker", workerName), - zap.String("source", source)) - break + relayWorkers := s.relayWorkers[source] + // 1. try to find a history worker in relay workers... + if len(relayWorkers) > 0 { + for workerName, bound := range s.lastBound { + if bound.Source == source { + w, ok := s.workers[workerName] + if !ok { + // a not found worker + continue + } + if _, ok2 := relayWorkers[workerName]; ok2 && w.Stage() == WorkerFree { + worker = w + s.logger.Info("found history relay worker when source bound", + zap.String("worker", workerName), + zap.String("source", source)) + break + } } } } // then a relay worker for this source... if worker == nil { - for workerName := range s.relayWorkers[source] { + for workerName := range relayWorkers { w, ok := s.workers[workerName] if !ok { // a not found worker, should not happened - s.logger.Info("worker instance not found for relay worker", zap.String("worker", workerName)) + s.logger.Warn("worker instance not found for relay worker", zap.String("worker", workerName)) continue } if w.Stage() == WorkerFree { @@ -1601,6 +1604,25 @@ func (s *Scheduler) tryBoundForSource(source string) (bool, error) { } } } + // then a history worker for this source... + if worker == nil { + for workerName, bound := range s.lastBound { + if bound.Source == source { + w, ok := s.workers[workerName] + if !ok { + // a not found worker + continue + } + if w.Stage() == WorkerFree { + worker = w + s.logger.Info("found history worker when source bound", + zap.String("worker", workerName), + zap.String("source", source)) + break + } + } + } + } // and then a random Free worker. if worker == nil { for _, w := range s.workers { From 85c6c4a6c86c0bdf9237712aa83144c0eb8472e6 Mon Sep 17 00:00:00 2001 From: lance6716 Date: Wed, 7 Apr 2021 17:35:35 +0800 Subject: [PATCH 44/46] refine log --- dm/worker/server.go | 4 ++-- dm/worker/worker.go | 2 ++ 2 files changed, 4 insertions(+), 2 deletions(-) diff --git a/dm/worker/server.go b/dm/worker/server.go index 69f023c4f6..b2852b2a7a 100644 --- a/dm/worker/server.go +++ b/dm/worker/server.go @@ -579,7 +579,7 @@ OUTER: } } } - log.L().Info("worker server is closed, handleSourceBound will quit now") + log.L().Info("handleSourceBound will quit now") return nil } @@ -702,11 +702,11 @@ func (s *Server) enableRelay(sourceCfg *config.SourceConfig, needLock bool) erro // because no re-assigned mechanism exists for keepalived DM-worker yet. return err2 } - s.UpdateKeepAliveTTL(s.cfg.RelayKeepAliveTTL) if err2 = w.EnableRelay(); err2 != nil { s.setSourceStatus(sourceCfg.SourceID, err2, false) return err2 } + s.UpdateKeepAliveTTL(s.cfg.RelayKeepAliveTTL) return nil } diff --git a/dm/worker/worker.go b/dm/worker/worker.go index ed24c24c77..3d2583a0d6 100644 --- a/dm/worker/worker.go +++ b/dm/worker/worker.go @@ -294,6 +294,7 @@ func (w *Worker) DisableRelay() { w.relayPurger = nil r.Close() } + w.l.Info("relay disabled") } // EnableHandleSubtasks enables the functionality of start/watch/handle subtasks @@ -359,6 +360,7 @@ func (w *Worker) DisableHandleSubtasks() { // close all sub tasks w.subTaskHolder.closeAllSubTasks() + w.l.Info("handling subtask enabled") } // fetchSubTasksAndAdjust gets source's subtask stages and configs, adjust some values by worker's config and status From 3748f6ce51ef1e3872fababe33f672e86f06447d Mon Sep 17 00:00:00 2001 From: lance6716 Date: Thu, 8 Apr 2021 11:10:13 +0800 Subject: [PATCH 45/46] fix too long keepalive --- tests/ha_cases/run.sh | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/ha_cases/run.sh b/tests/ha_cases/run.sh index d0ae884fd2..22ed5f6671 100755 --- a/tests/ha_cases/run.sh +++ b/tests/ha_cases/run.sh @@ -172,6 +172,7 @@ function test_kill_master() { function test_kill_and_isolate_worker() { + export GO_FAILPOINTS="github.com/pingcap/dm/dm/worker/defaultKeepAliveTTL=return(1)" echo "[$(date)] <<<<<< start test_kill_and_isolate_worker >>>>>>" test_running @@ -265,6 +266,7 @@ function test_kill_and_isolate_worker() { echo "use sync_diff_inspector to check increment2 data now!" check_sync_diff $WORK_DIR $cur/conf/diff_config.toml echo "[$(date)] <<<<<< finish test_kill_and_isolate_worker >>>>>>" + export GO_FAILPOINTS="" } # usage: test_kill_master_in_sync leader From 0e08badc3fa9fb6810fc5e73ca6cf8be08a7bdb3 Mon Sep 17 00:00:00 2001 From: lance6716 Date: Thu, 8 Apr 2021 17:29:40 +0800 Subject: [PATCH 46/46] try fix components closeing order --- cmd/dm-worker/main.go | 8 +++--- dm/worker/server.go | 58 ++++++++++++++++++++++++++++--------------- tests/ha_cases/run.sh | 5 +++- 3 files changed, 46 insertions(+), 25 deletions(-) diff --git a/cmd/dm-worker/main.go b/cmd/dm-worker/main.go index 07490056d8..1322c19b5f 100644 --- a/cmd/dm-worker/main.go +++ b/cmd/dm-worker/main.go @@ -21,15 +21,15 @@ import ( "strings" "syscall" + "github.com/pingcap/errors" + globalLog "github.com/pingcap/log" + "go.uber.org/zap" + "github.com/pingcap/dm/dm/ctl/common" "github.com/pingcap/dm/dm/worker" "github.com/pingcap/dm/pkg/log" "github.com/pingcap/dm/pkg/terror" "github.com/pingcap/dm/pkg/utils" - globalLog "github.com/pingcap/log" - - "github.com/pingcap/errors" - "go.uber.org/zap" ) func main() { diff --git a/dm/worker/server.go b/dm/worker/server.go index b2852b2a7a..4505b99e21 100644 --- a/dm/worker/server.go +++ b/dm/worker/server.go @@ -187,20 +187,50 @@ func (s *Server) Start() error { // NOTE: don't need to set tls config, because rootLis already use tls s.svr = grpc.NewServer() pb.RegisterWorkerServer(s.svr, s) + + grpcExitCh := make(chan struct{}, 1) s.wg.Add(1) go func() { - defer s.wg.Done() err2 := s.svr.Serve(grpcL) if err2 != nil && !common.IsErrNetClosing(err2) && err2 != cmux.ErrListenerClosed { log.L().Error("gRPC server returned", log.ShortError(err2)) } + grpcExitCh <- struct{}{} }() + go func(ctx context.Context) { + defer s.wg.Done() + select { + case <-ctx.Done(): + if s.svr != nil { + // GracefulStop can not cancel active stream RPCs + // and the stream RPC may block on Recv or Send + // so we use Stop instead to cancel all active RPCs + s.svr.Stop() + } + case <-grpcExitCh: + } + }(s.ctx) + + httpExitCh := make(chan struct{}, 1) s.wg.Add(1) go func() { - defer s.wg.Done() InitStatus(httpL) // serve status + httpExitCh <- struct{}{} }() + go func(ctx context.Context) { + defer s.wg.Done() + select { + case <-ctx.Done(): + if s.rootLis != nil { + err2 := s.rootLis.Close() + if err2 != nil && !common.IsErrNetClosing(err2) { + log.L().Error("fail to close net listener", log.ShortError(err2)) + } + } + case <-httpExitCh: + } + }(s.ctx) s.closed.Set(false) log.L().Info("listening gRPC API and status request", zap.String("address", s.cfg.WorkerAddr)) @@ -443,27 +473,16 @@ func (s *Server) observeSourceBound(ctx context.Context, rev int64) error { } func (s *Server) doClose() { + s.cancel() + // close server in advance, stop receiving source bound and relay bound + s.wg.Wait() + s.Lock() defer s.Unlock() if s.closed.Get() { return } - - if s.rootLis != nil { - err := s.rootLis.Close() - if err != nil && !common.IsErrNetClosing(err) { - log.L().Error("fail to close net listener", log.ShortError(err)) - } - } - if s.svr != nil { - // GracefulStop can not cancel active stream RPCs - // and the stream RPC may block on Recv or Send - // so we use Stop instead to cancel all active RPCs - s.svr.Stop() - } - // close worker and wait for return - s.cancel() if w := s.getWorker(false); w != nil { w.Close() } @@ -472,9 +491,8 @@ func (s *Server) doClose() { // Close close the RPC server, this function can be called multiple times func (s *Server) Close() { - s.doClose() s.stopKeepAlive() - s.wg.Wait() + s.doClose() } // if needLock is false, we should make sure Server has been locked in caller @@ -811,7 +829,7 @@ func (s *Server) getOrStartWorker(cfg *config.SourceConfig, needLock bool) (*Wor return nil, terror.ErrWorkerAlreadyStart.Generate(w.name, w.cfg.SourceID, cfg.SourceID) } - log.L().Info("will start a now worker", zap.String("sourceID", cfg.SourceID)) + log.L().Info("will start a new worker", zap.String("sourceID", cfg.SourceID)) w, err := NewWorker(cfg, s.etcdClient, s.cfg.Name) if err != nil { return nil, err diff --git a/tests/ha_cases/run.sh b/tests/ha_cases/run.sh index 22ed5f6671..aca1daa7d5 100755 --- a/tests/ha_cases/run.sh +++ b/tests/ha_cases/run.sh @@ -172,7 +172,10 @@ function test_kill_master() { function test_kill_and_isolate_worker() { - export GO_FAILPOINTS="github.com/pingcap/dm/dm/worker/defaultKeepAliveTTL=return(1)" + inject_points=("github.com/pingcap/dm/dm/worker/defaultKeepAliveTTL=return(1)" + "github.com/pingcap/dm/dm/worker/defaultRelayKeepAliveTTL=return(2)" + ) + export GO_FAILPOINTS="$(join_string \; ${inject_points[@]})" echo "[$(date)] <<<<<< start test_kill_and_isolate_worker >>>>>>" test_running