diff --git a/cmd/dm-syncer/main.go b/cmd/dm-syncer/main.go index bd25129814..c84fabbaae 100644 --- a/cmd/dm-syncer/main.go +++ b/cmd/dm-syncer/main.go @@ -63,7 +63,7 @@ func main() { log.L().Info("", zap.Stringer("dm-syncer conf", conf)) }) - sync := syncer.NewSyncer(conf) + sync := syncer.NewSyncer(conf, nil) // do not support shard DDL for singleton syncer. sc := make(chan os.Signal, 1) signal.Notify(sc, syscall.SIGHUP, syscall.SIGINT, syscall.SIGTERM, syscall.SIGQUIT) diff --git a/dm/common/common.go b/dm/common/common.go index 0ad9d82a77..d3e0be3991 100644 --- a/dm/common/common.go +++ b/dm/common/common.go @@ -39,6 +39,13 @@ var ( // UpstreamSubTaskKeyAdapter used to store SubTask which are subscribing data from MySQL source. // k/v: Encode(source-id, task-name) -> SubTaskConfig UpstreamSubTaskKeyAdapter KeyAdapter = keyHexEncoderDecoder("/dm-master/upstream/subtask/") + + // ShardDDLPessimismInfoKeyAdapter used to store shard DDL info in pessimistic model. + // k/v: Encode(task-name, source-id) -> shard DDL info + ShardDDLPessimismInfoKeyAdapter KeyAdapter = keyHexEncoderDecoder("/dm-master/shardddl-pessimism/info/") + // ShardDDLPessimismOperationKeyAdapter used to store shard DDL lock in pessimistic model. + // k/v: Encode(task-name, source-id) -> shard DDL operation + ShardDDLPessimismOperationKeyAdapter KeyAdapter = keyHexEncoderDecoder("/dm-master/shardddl-pessimism/operation/") ) func keyAdapterKeysLen(s KeyAdapter) int { diff --git a/dm/master/server.go b/dm/master/server.go index 03f8a38fe2..98fbef7268 100644 --- a/dm/master/server.go +++ b/dm/master/server.go @@ -16,17 +16,24 @@ package master import ( "context" "fmt" - "io" "net/http" "sort" "strings" "sync" "time" + "github.com/pingcap/errors" + "github.com/siddontang/go/sync2" + "go.etcd.io/etcd/clientv3" + "go.etcd.io/etcd/embed" + "go.uber.org/zap" + "google.golang.org/grpc" + "github.com/pingcap/dm/checker" "github.com/pingcap/dm/dm/common" "github.com/pingcap/dm/dm/config" "github.com/pingcap/dm/dm/master/coordinator" + "github.com/pingcap/dm/dm/master/shardddl" operator "github.com/pingcap/dm/dm/master/sql-operator" "github.com/pingcap/dm/dm/master/workerrpc" "github.com/pingcap/dm/dm/pb" @@ -36,13 +43,6 @@ import ( "github.com/pingcap/dm/pkg/log" "github.com/pingcap/dm/pkg/terror" "github.com/pingcap/dm/pkg/tracing" - "github.com/pingcap/dm/syncer" - "github.com/pingcap/errors" - "github.com/siddontang/go/sync2" - "go.etcd.io/etcd/clientv3" - "go.etcd.io/etcd/embed" - "go.uber.org/zap" - "google.golang.org/grpc" ) const ( @@ -81,8 +81,8 @@ type Server struct { // task-name -> source-list taskSources map[string][]string - // DDL lock keeper - lockKeeper *LockKeeper + // shard DDL pessimist + pessimist *shardddl.Pessimist // SQL operator holder sqlOperatorHolder *operator.Holder @@ -103,11 +103,12 @@ func NewServer(cfg *Config) *Server { coordinator: coordinator.NewCoordinator(), workerClients: make(map[string]workerrpc.Client), taskSources: make(map[string][]string), - lockKeeper: NewLockKeeper(), sqlOperatorHolder: operator.NewHolder(), idGen: tracing.NewIDGen(), ap: NewAgentPool(&RateLimitConfig{rate: cfg.RPCRateLimit, burst: cfg.RPCRateBurst}), } + logger := log.L() + server.pessimist = shardddl.NewPessimist(&logger, server.getTaskResources) server.closed.Set(true) return &server @@ -175,6 +176,12 @@ func (s *Server) Start(ctx context.Context) (err error) { return } + // start the shard DDL pessimist. + err = s.pessimist.Start(ctx, s.etcdClient) + if err != nil { + return + } + s.closed.Set(false) // the server started now. s.bgFunWg.Add(1) @@ -198,13 +205,6 @@ func (s *Server) Start(ctx context.Context) (err error) { } }() - s.bgFunWg.Add(1) - go func() { - defer s.bgFunWg.Done() - // fetch DDL info from dm-workers to sync sharding DDL - s.fetchWorkerDDLInfo(ctx) - }() - log.L().Info("listening gRPC API and status request", zap.String("address", s.cfg.MasterAddr)) return } @@ -246,6 +246,8 @@ func (s *Server) Close() { // wait for background functions returned s.bgFunWg.Wait() + s.pessimist.Close() + if s.election != nil { s.election.Close() } @@ -752,7 +754,7 @@ func (s *Server) ShowDDLLocks(ctx context.Context, req *pb.ShowDDLLocksRequest) Result: true, } - locks := s.lockKeeper.Locks() + locks := s.pessimist.Locks() resp.Locks = make([]*pb.DDLLock, 0, len(locks)) for _, lock := range locks { if len(req.Task) > 0 && req.Task != lock.Task { @@ -772,7 +774,7 @@ func (s *Server) ShowDDLLocks(ctx context.Context, req *pb.ShowDDLLocksRequest) ID: lock.ID, Task: lock.Task, Owner: lock.Owner, - DDLs: lock.Stmts, + DDLs: lock.DDLs, Synced: make([]string, 0, len(ready)), Unsynced: make([]string, 0, len(ready)), } @@ -793,83 +795,22 @@ func (s *Server) ShowDDLLocks(ctx context.Context, req *pb.ShowDDLLocksRequest) } // UnlockDDLLock implements MasterServer.UnlockDDLLock +// TODO(csuzhangxc): implement this later. func (s *Server) UnlockDDLLock(ctx context.Context, req *pb.UnlockDDLLockRequest) (*pb.UnlockDDLLockResponse, error) { log.L().Info("", zap.String("lock ID", req.ID), zap.Stringer("payload", req), zap.String("request", "UnlockDDLLock")) - - workerResps, err := s.resolveDDLLock(ctx, req.ID, req.ReplaceOwner, req.Sources) - resp := &pb.UnlockDDLLockResponse{ - Result: true, - Workers: workerResps, - } - if err != nil { - resp.Result = false - resp.Msg = errors.ErrorStack(err) - log.L().Error("fail to unlock ddl", zap.String("ID", req.ID), zap.String("request", "UnlockDDLLock"), zap.Error(err)) - - if req.ForceRemove { - s.lockKeeper.RemoveLock(req.ID) - log.L().Warn("force to remove DDL lock", zap.String("request", "UnlockDDLLock"), zap.String("ID", req.ID)) - } - } else { - log.L().Info("unlock ddl successfully", zap.String("ID", req.ID), zap.String("request", "UnlockDDLLock")) - } - - return resp, nil + return &pb.UnlockDDLLockResponse{ + Result: false, + Msg: "not implement", + }, nil } // BreakWorkerDDLLock implements MasterServer.BreakWorkerDDLLock +// TODO(csuzhangxc): implement this later. func (s *Server) BreakWorkerDDLLock(ctx context.Context, req *pb.BreakWorkerDDLLockRequest) (*pb.BreakWorkerDDLLockResponse, error) { log.L().Info("", zap.String("lock ID", req.RemoveLockID), zap.Stringer("payload", req), zap.String("request", "BreakWorkerDDLLock")) - - request := &workerrpc.Request{ - Type: workerrpc.CmdBreakDDLLock, - BreakDDLLock: &pb.BreakDDLLockRequest{ - Task: req.Task, - RemoveLockID: req.RemoveLockID, - ExecDDL: req.ExecDDL, - SkipDDL: req.SkipDDL, - }, - } - - workerRespCh := make(chan *pb.CommonWorkerResponse, len(req.Sources)) - var wg sync.WaitGroup - for _, source := range req.Sources { - wg.Add(1) - go func(sourceID string) { - defer wg.Done() - worker := s.coordinator.GetWorkerBySourceID(sourceID) - if worker == nil || worker.State() == coordinator.WorkerClosed { - workerRespCh <- errorCommonWorkerResponse(fmt.Sprintf("worker %s relevant worker-client not found", sourceID), sourceID) - return - } - resp, err := worker.SendRequest(ctx, request, s.cfg.RPCTimeout) - workerResp := &pb.CommonWorkerResponse{} - if err != nil { - workerResp = errorCommonWorkerResponse(errors.ErrorStack(err), sourceID) - } else { - workerResp = resp.BreakDDLLock - } - workerResp.Source = sourceID - workerRespCh <- workerResp - }(source) - } - wg.Wait() - - workerRespMap := make(map[string]*pb.CommonWorkerResponse, len(req.Sources)) - for len(workerRespCh) > 0 { - workerResp := <-workerRespCh - 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]) - } - return &pb.BreakWorkerDDLLockResponse{ - Result: true, - Sources: workerResps, + Result: false, + Msg: "not implement", }, nil } @@ -1294,297 +1235,6 @@ func (s *Server) checkTaskAndWorkerMatch(taskname string, targetWorker string) b return false } -// fetchWorkerDDLInfo fetches DDL info from all dm-workers -// and sends DDL lock info back to dm-workers -func (s *Server) fetchWorkerDDLInfo(ctx context.Context) { - var wg sync.WaitGroup - - request := &workerrpc.Request{Type: workerrpc.CmdFetchDDLInfo} - for source, w := range s.coordinator.GetRunningMysqlSource() { - wg.Add(1) - go func(source string, worker *coordinator.Worker) { - defer wg.Done() - var doRetry bool - - for { - if doRetry { - select { - case <-ctx.Done(): - return - case <-time.After(fetchDDLInfoRetryTimeout): - } - } - doRetry = false // reset - - select { - case <-ctx.Done(): - return - default: - resp, err := worker.SendRequest(ctx, request, s.cfg.RPCTimeout) - if err != nil { - log.L().Error("create FetchDDLInfo stream", zap.String("source", source), log.ShortError(err)) - doRetry = true - continue - } - stream := resp.FetchDDLInfo - for { - in, err := stream.Recv() - if err == io.EOF { - doRetry = true - break - } - select { - case <-ctx.Done(): // check whether canceled again - return - default: - } - if err != nil { - log.L().Error("receive ddl info", zap.String("source", source), log.ShortError(err)) - doRetry = true - break - } - log.L().Info("receive ddl info", zap.Stringer("ddl info", in), zap.String("source", source)) - - workers := s.getTaskResources(in.Task) - if len(workers) == 0 { - // should happen only when starting and before updateTaskWorkers return - log.L().Error("try to sync shard DDL, but with no workers", zap.String("task", in.Task)) - doRetry = true - break - } - if !s.containWorker(workers, source) { - // should not happen - log.L().Error("try to sync shard DDL, but worker is not in workers", zap.String("task", in.Task), zap.String("source", source), zap.Strings("workers", workers)) - doRetry = true - break - } - - lockID, synced, remain, err := s.lockKeeper.TrySync(in.Task, in.Schema, in.Table, source, in.DDLs, workers) - if err != nil { - log.L().Error("fail to sync lock", zap.String("source", source), log.ShortError(err)) - doRetry = true - break - } - - out := &pb.DDLLockInfo{ - Task: in.Task, - ID: lockID, - } - err = stream.Send(out) - if err != nil { - log.L().Error("fail to send ddl lock info", zap.Stringer("ddl lock info", out), zap.String("source", source), log.ShortError(err)) - doRetry = true - break - } - - if !synced { - // still need wait other workers to sync - log.L().Info("shard DDL is in syncing", zap.String("lock ID", lockID), zap.Int("remain worker count", remain)) - continue - } - - log.L().Info("shard DDL was synced", zap.String("lock ID", lockID)) - - // resolve DDL lock - wg.Add(1) - go func(lockID string) { - defer wg.Done() - resps, err := s.resolveDDLLock(ctx, lockID, "", nil) - if err == nil { - log.L().Info("resolve DDL lock successfully", zap.String("lock ID", lockID)) - } else { - log.L().Error("fail to resolve DDL lock", zap.String("lock ID", lockID), zap.Reflect("responses", resps), zap.Error(err)) - lock := s.lockKeeper.FindLock(lockID) - if lock != nil { - lock.AutoRetry.Set(true) // need auto-retry resolve at intervals - } - } - }(lockID) - } - stream.CloseSend() - } - } - - }(source, w) - } - - wg.Wait() -} - -// resolveDDLLock resolves DDL lock -// requests DDL lock's owner to execute the DDL -// requests DDL lock's non-owner dm-workers to ignore (skip) the DDL -func (s *Server) resolveDDLLock(ctx context.Context, lockID string, replaceOwner string, prefWorkers []string) ([]*pb.CommonWorkerResponse, error) { - lock := s.lockKeeper.FindLock(lockID) - if lock == nil { - // should not happen even when dm-master restarted - return nil, terror.ErrMasterLockNotFound.Generate(lockID) - } - - if lock.Resolving.Get() { - return nil, terror.ErrMasterLockIsResolving.Generate(lockID) - } - lock.Resolving.Set(true) - defer lock.Resolving.Set(false) //reset - - ready := lock.Ready() // Ready contain all dm-sources and whether they were synced - - // request the owner to execute DDL - owner := lock.Owner - if len(replaceOwner) > 0 { - owner = replaceOwner - } - worker := s.coordinator.GetWorkerBySourceID(owner) - if worker == nil { - return nil, terror.ErrMasterWorkerCliNotFound.Generate(owner) - } - if _, ok := ready[owner]; !ok { - return nil, terror.ErrMasterWorkerNotWaitLock.Generate(owner, lockID) - } - - // try send handle SQLs request to owner if exists - key, oper := s.sqlOperatorHolder.Get(lock.Task, lock.DDLs()) - if oper != nil { - ownerReq := &workerrpc.Request{ - Type: workerrpc.CmdHandleSubTaskSQLs, - HandleSubTaskSQLs: &pb.HandleSubTaskSQLsRequest{ - Name: oper.Req.Name, - Op: oper.Req.Op, - Args: oper.Req.Args, - BinlogPos: oper.Req.BinlogPos, - SqlPattern: oper.Req.SqlPattern, - }, - } - resp, err := worker.SendRequest(ctx, ownerReq, s.cfg.RPCTimeout) - if err != nil { - return nil, terror.Annotatef(err, "send handle SQLs request %s to DDL lock %s owner %s fail", ownerReq.HandleSubTaskSQLs, lockID, owner) - } - ownerResp := resp.HandleSubTaskSQLs - if !ownerResp.Result { - return nil, terror.ErrMasterHandleSQLReqFail.Generate(lockID, owner, ownerReq.HandleSubTaskSQLs, ownerResp.Msg) - } - log.L().Info("sent handle --sharding DDL request", zap.Stringer("payload", ownerReq.HandleSubTaskSQLs), zap.String("owner", owner), zap.String("lock ID", lockID)) - s.sqlOperatorHolder.Remove(lock.Task, key) // remove SQL operator after sent to owner - } - - // If send ExecuteDDL request failed, we will generate more tracer group id, - // this is acceptable if each ExecuteDDL request successes at last. - // TODO: we need a better way to combine brain split tracing events into one - // single group. - traceGID := s.idGen.NextID("resolveDDLLock", 0) - log.L().Info("requesting to execute DDL", zap.String("owner", owner), zap.String("lock ID", lockID)) - ownerReq := &workerrpc.Request{ - Type: workerrpc.CmdExecDDL, - ExecDDL: &pb.ExecDDLRequest{ - Task: lock.Task, - LockID: lockID, - Exec: true, - TraceGID: traceGID, - DDLs: lock.ddls, - }, - } - // use a longer timeout for executing DDL in DM-source. - // now, we ignore `invalid connection` for `ADD INDEX`, use a longer timout to ensure the DDL lock removed. - ownerTimeout := time.Duration(syncer.MaxDDLConnectionTimeoutMinute)*time.Minute + 30*time.Second - resp, err := worker.SendRequest(ctx, ownerReq, ownerTimeout) - ownerResp := &pb.CommonWorkerResponse{} - if err != nil { - ownerResp = errorCommonWorkerResponse(errors.ErrorStack(err), owner) - } else { - ownerResp = resp.ExecDDL - } - ownerResp.Source = owner - if !ownerResp.Result { - // owner execute DDL fail, do not continue - return []*pb.CommonWorkerResponse{ - ownerResp, - }, terror.ErrMasterOwnerExecDDL.Generate(owner) - } - - // request other dm-sources to ignore DDL - sources := make([]string, 0, len(ready)) - if len(prefWorkers) > 0 { - sources = prefWorkers - } else { - for worker := range ready { - sources = append(sources, worker) - } - } - - request := &workerrpc.Request{ - Type: workerrpc.CmdExecDDL, - ExecDDL: &pb.ExecDDLRequest{ - Task: lock.Task, - LockID: lockID, - Exec: false, // ignore and skip DDL - TraceGID: traceGID, - DDLs: lock.ddls, - }, - } - workerRespCh := make(chan *pb.CommonWorkerResponse, len(sources)) - var wg sync.WaitGroup - for _, source := range sources { - if source == owner { - continue // owner has executed DDL - } - - wg.Add(1) - go func(source string) { - defer wg.Done() - worker := s.coordinator.GetWorkerBySourceID(source) - if worker == nil { - workerRespCh <- errorCommonWorkerResponse(fmt.Sprintf("source %s relevant source-client not found", source), source) - return - } - if _, ok := ready[source]; !ok { - workerRespCh <- errorCommonWorkerResponse(fmt.Sprintf("source %s not waiting for DDL lock %s", owner, lockID), source) - return - } - - log.L().Info("request to skip DDL", zap.String("not owner source", source), zap.String("lock ID", lockID)) - resp, err2 := worker.SendRequest(ctx, request, s.cfg.RPCTimeout) - var workerResp *pb.CommonWorkerResponse - if err2 != nil { - workerResp = errorCommonWorkerResponse(errors.ErrorStack(err2), "") - } else { - workerResp = resp.ExecDDL - } - workerResp.Source = source - workerRespCh <- workerResp - }(source) - } - wg.Wait() - - workerRespMap := make(map[string]*pb.CommonWorkerResponse, len(sources)) - var success = true - for len(workerRespCh) > 0 { - workerResp := <-workerRespCh - workerRespMap[workerResp.Source] = workerResp - if !workerResp.Result { - success = false - } - } - - sort.Strings(sources) - workerResps := make([]*pb.CommonWorkerResponse, 0, len(sources)+1) - workerResps = append(workerResps, ownerResp) - for _, source := range sources { - workerResp, ok := workerRespMap[source] - if ok { - workerResps = append(workerResps, workerResp) - } - } - - // owner has ExecuteDDL successfully, we remove the Lock - // if some dm-sources ExecuteDDL occurred error, we should use dmctl to handle dm-source directly - s.lockKeeper.RemoveLock(lockID) - - if !success { - err = terror.ErrMasterPartWorkerExecDDLFail.Generate(lockID) - } - return workerResps, err -} - // UpdateMasterConfig implements MasterServer.UpdateConfig func (s *Server) UpdateMasterConfig(ctx context.Context, req *pb.UpdateMasterConfigRequest) (*pb.UpdateMasterConfigResponse, error) { log.L().Info("", zap.Stringer("payload", req), zap.String("request", "UpdateMasterConfig")) diff --git a/dm/master/server_test.go b/dm/master/server_test.go index 29173fc391..d03736495d 100644 --- a/dm/master/server_test.go +++ b/dm/master/server_test.go @@ -17,11 +17,9 @@ import ( "bytes" "context" "fmt" - "io" "io/ioutil" "net/http" "strings" - "sync" "testing" "time" @@ -311,67 +309,6 @@ func (t *testMaster) TestQueryStatus(c *check.C) { // TODO: test query with correct task name, this needs to add task first } -func (t *testMaster) TestShowDDLLocks(c *check.C) { - server := testDefaultMasterServer(c) - - resp, err := server.ShowDDLLocks(context.Background(), &pb.ShowDDLLocksRequest{}) - c.Assert(err, check.IsNil) - c.Assert(resp.Result, check.IsTrue) - c.Assert(resp.Locks, check.HasLen, 0) - - sources, _ := extractWorkerSource(server.cfg.Deploy) - - // prepare ddl lock keeper, mainly use code from ddl_lock_test.go - sqls := []string{"stmt"} - cases := []struct { - task string - schema string - table string - }{ - {"testA", "test_db1", "test_tbl1"}, - {"testB", "test_db2", "test_tbl2"}, - } - lk := NewLockKeeper() - var wg sync.WaitGroup - for _, tc := range cases { - wg.Add(1) - go func(task, schema, table string) { - defer wg.Done() - id, synced, remain, err2 := lk.TrySync(task, schema, table, sources[0], sqls, sources) - c.Assert(err2, check.IsNil) - c.Assert(synced, check.IsFalse) - c.Assert(remain, check.Greater, 0) // multi-goroutines TrySync concurrently, can only confirm remain > 0 - c.Assert(lk.FindLock(id), check.NotNil) - }(tc.task, tc.schema, tc.table) - } - wg.Wait() - server.lockKeeper = lk - - // test query with task name - resp, err = server.ShowDDLLocks(context.Background(), &pb.ShowDDLLocksRequest{ - Task: "testA", - Sources: sources, - }) - c.Assert(err, check.IsNil) - c.Assert(resp.Result, check.IsTrue) - c.Assert(resp.Locks, check.HasLen, 1) - c.Assert(resp.Locks[0].ID, check.Equals, genDDLLockID("testA", "test_db1", "test_tbl1")) - - // test specify a mismatch worker - resp, err = server.ShowDDLLocks(context.Background(), &pb.ShowDDLLocksRequest{ - Sources: []string{"invalid-source"}, - }) - c.Assert(err, check.IsNil) - c.Assert(resp.Result, check.IsTrue) - c.Assert(resp.Locks, check.HasLen, 0) - - // test query all ddl locks - resp, err = server.ShowDDLLocks(context.Background(), &pb.ShowDDLLocksRequest{}) - c.Assert(err, check.IsNil) - c.Assert(resp.Result, check.IsTrue) - c.Assert(resp.Locks, check.HasLen, 2) -} - func (t *testMaster) TestCheckTask(c *check.C) { ctrl := gomock.NewController(c) defer ctrl.Finish() @@ -763,232 +700,6 @@ func (t *testMaster) TestUpdateTask(c *check.C) { } } -func (t *testMaster) TestUnlockDDLLock(c *check.C) { - ctrl := gomock.NewController(c) - defer ctrl.Finish() - - var ( - sqls = []string{"stmt"} - task = "testA" - schema = "test_db" - table = "test_table" - traceGIDIdx = 1 - ) - - server := testDefaultMasterServer(c) - sources, workers := extractWorkerSource(server.cfg.Deploy) - - mockResolveDDLLock := func(task, lockID, owner, traceGID string, ownerFail, nonOwnerFail bool) { - for _, worker := range workers { - mockWorkerClient := pbmock.NewMockWorkerClient(ctrl) - - if ownerFail && worker != owner { - continue - } - - exec := false - if owner == worker { - exec = true - } - - ret := []interface{}{ - &pb.CommonWorkerResponse{Result: true}, - nil, - } - if (ownerFail && worker == owner) || (nonOwnerFail && worker != owner) { - ret[0] = &pb.CommonWorkerResponse{Result: false} - ret[1] = errors.New(errExecDDLFailed) - } - - mockWorkerClient.EXPECT().ExecuteDDL( - gomock.Any(), - &pb.ExecDDLRequest{ - Task: task, - LockID: lockID, - Exec: exec, - TraceGID: traceGID, - DDLs: sqls, - }, - ).Return(ret...) - - server.workerClients[worker] = newMockRPCClient(mockWorkerClient) - } - } - - prepareDDLLock := func() { - // prepare ddl lock keeper, mainly use code from ddl_lock_test.go - lk := NewLockKeeper() - var wg sync.WaitGroup - for _, s := range sources { - wg.Add(1) - go func(source string) { - defer wg.Done() - id, _, _, err := lk.TrySync(task, schema, table, source, sqls, sources) - c.Assert(err, check.IsNil) - c.Assert(lk.FindLock(id), check.NotNil) - }(s) - } - wg.Wait() - server.lockKeeper = lk - } - - // test UnlockDDLLock successfully - prepareDDLLock() - lockID := genDDLLockID(task, schema, table) - traceGID := fmt.Sprintf("resolveDDLLock.%d", traceGIDIdx) - traceGIDIdx++ - mockResolveDDLLock(task, lockID, workers[0], traceGID, false, false) - server.coordinator = testMockCoordinator(c, sources, workers, "", server.workerClients) - resp, err := server.UnlockDDLLock(context.Background(), &pb.UnlockDDLLockRequest{ - ID: lockID, - ReplaceOwner: sources[0], - }) - c.Assert(err, check.IsNil) - c.Assert(resp.Result, check.IsTrue) - - // test UnlockDDLLock but DDL owner executed failed - prepareDDLLock() - lockID = genDDLLockID(task, schema, table) - traceGID = fmt.Sprintf("resolveDDLLock.%d", traceGIDIdx) - traceGIDIdx++ - mockResolveDDLLock(task, lockID, workers[0], traceGID, true, false) - server.coordinator = testMockCoordinator(c, sources, workers, "", server.workerClients) - resp, err = server.UnlockDDLLock(context.Background(), &pb.UnlockDDLLockRequest{ - ID: lockID, - ReplaceOwner: sources[0], - }) - c.Assert(err, check.IsNil) - c.Assert(resp.Result, check.IsFalse) - c.Assert(server.lockKeeper.FindLock(lockID), check.NotNil) - - // retry UnlockDDLLock with force remove, but still DDL owner executed failed - traceGID = fmt.Sprintf("resolveDDLLock.%d", traceGIDIdx) - traceGIDIdx++ - mockResolveDDLLock(task, lockID, workers[0], traceGID, true, false) - server.coordinator = testMockCoordinator(c, sources, workers, "", server.workerClients) - resp, err = server.UnlockDDLLock(context.Background(), &pb.UnlockDDLLockRequest{ - ID: lockID, - ReplaceOwner: sources[0], - ForceRemove: true, - }) - c.Assert(err, check.IsNil) - c.Assert(resp.Result, check.IsFalse) - c.Assert(server.lockKeeper.FindLock(lockID), check.IsNil) - - // test UnlockDDLLock, DDL owner executed successfully but other workers failed - prepareDDLLock() - lockID = genDDLLockID(task, schema, table) - traceGID = fmt.Sprintf("resolveDDLLock.%d", traceGIDIdx) - traceGIDIdx++ - mockResolveDDLLock(task, lockID, workers[0], traceGID, false, true) - server.coordinator = testMockCoordinator(c, sources, workers, "", server.workerClients) - resp, err = server.UnlockDDLLock(context.Background(), &pb.UnlockDDLLockRequest{ - ID: lockID, - ReplaceOwner: sources[0], - }) - c.Assert(err, check.IsNil) - c.Assert(resp.Result, check.IsFalse) - c.Assert(server.lockKeeper.FindLock(lockID), check.IsNil) - - // TODO: add SQL operator test -} - -func (t *testMaster) TestBreakWorkerDDLLock(c *check.C) { - ctrl := gomock.NewController(c) - defer ctrl.Finish() - - server := testDefaultMasterServer(c) - var ( - task = "test" - schema = "test_db" - table = "test_table" - lockID = genDDLLockID(task, schema, table) - ) - sources, workers := extractWorkerSource(server.cfg.Deploy) - - // mock BreakDDLLock request - mockBreakDDLLock := func(rpcSuccess bool) { - for _, deploy := range server.cfg.Deploy { - rets := make([]interface{}, 0, 2) - if rpcSuccess { - rets = []interface{}{ - &pb.CommonWorkerResponse{ - Result: true, - Source: deploy.Source, - }, - nil, - } - } else { - rets = []interface{}{ - nil, - errors.New(errGRPCFailed), - } - } - mockWorkerClient := pbmock.NewMockWorkerClient(ctrl) - mockWorkerClient.EXPECT().BreakDDLLock( - gomock.Any(), - &pb.BreakDDLLockRequest{ - Task: "test", - RemoveLockID: lockID, - SkipDDL: true, - }, - ).Return(rets...) - server.workerClients[deploy.Worker] = newMockRPCClient(mockWorkerClient) - } - } - - // test BreakWorkerDDLLock with invalid dm-worker[s] - resp, err := server.BreakWorkerDDLLock(context.Background(), &pb.BreakWorkerDDLLockRequest{ - Task: task, - Sources: []string{"invalid-source1", "invalid-source2"}, - RemoveLockID: lockID, - SkipDDL: true, - }) - c.Assert(err, check.IsNil) - c.Assert(resp.Result, check.IsTrue) - 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") - } - - // test BreakWorkerDDLLock successfully - mockBreakDDLLock(true) - c.Assert(len(sources), check.Equals, len(workers)) - server.coordinator = testMockCoordinator(c, sources, workers, "", server.workerClients) - - resp, err = server.BreakWorkerDDLLock(context.Background(), &pb.BreakWorkerDDLLockRequest{ - Task: task, - Sources: sources, - RemoveLockID: lockID, - SkipDDL: true, - }) - c.Assert(err, check.IsNil) - c.Assert(resp.Result, check.IsTrue) - c.Assert(resp.Sources, check.HasLen, 2) - for _, w := range resp.Sources { - c.Assert(w.Result, check.IsTrue) - } - - // test BreakWorkerDDLLock with error response - mockBreakDDLLock(false) - server.coordinator = testMockCoordinator(c, sources, workers, "", server.workerClients) - - resp, err = server.BreakWorkerDDLLock(context.Background(), &pb.BreakWorkerDDLLockRequest{ - Task: task, - Sources: sources, - RemoveLockID: lockID, - SkipDDL: true, - }) - c.Assert(err, check.IsNil) - c.Assert(resp.Result, check.IsTrue) - c.Assert(resp.Sources, check.HasLen, 2) - for _, w := range resp.Sources { - c.Assert(w.Result, check.IsFalse) - c.Assert(w.Msg, check.Matches, errGRPCFailedReg) - } -} - func (t *testMaster) TestPurgeWorkerRelay(c *check.C) { ctrl := gomock.NewController(c) defer ctrl.Finish() @@ -1229,87 +940,6 @@ func (t *testMaster) TestOperateWorkerRelayTask(c *check.C) { } } -func (t *testMaster) TestFetchWorkerDDLInfo(c *check.C) { - ctrl := gomock.NewController(c) - defer ctrl.Finish() - - server := testDefaultMasterServer(c) - sources, workers := extractWorkerSource(server.cfg.Deploy) - server.taskSources = map[string][]string{"test": sources} - var ( - task = "test" - schema = "test_db" - table = "test_table" - ddls = []string{"stmt"} - traceGID = fmt.Sprintf("resolveDDLLock.%d", 1) - lockID = genDDLLockID(task, schema, table) - wg sync.WaitGroup - ) - - // mock FetchDDLInfo stream API - for _, deploy := range server.cfg.Deploy { - stream := pbmock.NewMockWorker_FetchDDLInfoClient(ctrl) - stream.EXPECT().Recv().Return(&pb.DDLInfo{ - Task: task, - Schema: schema, - Table: table, - DDLs: ddls, - }, nil) - // This will lead to a select on ctx.Done and time.After(fetchDDLInfoRetryTimeout), - // so we have enough time to cancel the context. - stream.EXPECT().Recv().Return(nil, io.EOF).MaxTimes(1) - stream.EXPECT().Send(&pb.DDLLockInfo{Task: task, ID: lockID}).Return(nil) - stream.EXPECT().CloseSend().Return(nil) - - mockWorkerClient := pbmock.NewMockWorkerClient(ctrl) - mockWorkerClient.EXPECT().FetchDDLInfo(gomock.Any()).Return(stream, nil) - // which worker is DDLLock owner is not determined, so we mock a exec/non-exec - // ExecDDLRequest to each worker client, with at most one time call. - mockWorkerClient.EXPECT().ExecuteDDL( - gomock.Any(), - &pb.ExecDDLRequest{ - Task: task, - LockID: lockID, - Exec: true, - TraceGID: traceGID, - DDLs: ddls, - }, - ).Return(&pb.CommonWorkerResponse{Result: true}, nil).MaxTimes(1) - mockWorkerClient.EXPECT().ExecuteDDL( - gomock.Any(), - &pb.ExecDDLRequest{ - Task: task, - LockID: lockID, - Exec: false, - TraceGID: traceGID, - DDLs: ddls, - }, - ).Return(&pb.CommonWorkerResponse{Result: true}, nil).MaxTimes(1) - - server.workerClients[deploy.Worker] = newMockRPCClient(mockWorkerClient) - } - - server.coordinator = testMockCoordinator(c, sources, workers, "", server.workerClients) - ctx, cancel := context.WithCancel(context.Background()) - wg.Add(1) - go func() { - defer wg.Done() - server.fetchWorkerDDLInfo(ctx) - }() - go func() { - for { - select { - case <-time.After(time.Millisecond * 10): - if server.lockKeeper.FindLock(lockID) == nil { - cancel() - return - } - } - } - }() - wg.Wait() -} - func (t *testMaster) TestServer(c *check.C) { cfg := NewConfig() c.Assert(cfg.Parse([]string{"-config=./dm-master.toml"}), check.IsNil) diff --git a/dm/master/shardddl/pessimist.go b/dm/master/shardddl/pessimist.go new file mode 100644 index 0000000000..3c4608811f --- /dev/null +++ b/dm/master/shardddl/pessimist.go @@ -0,0 +1,376 @@ +// Copyright 2020 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package shardddl + +import ( + "context" + "sync" + + "go.etcd.io/etcd/clientv3" + "go.uber.org/zap" + + "github.com/pingcap/dm/pkg/log" + "github.com/pingcap/dm/pkg/shardddl/pessimism" +) + +// Pessimist used to coordinate the shard DDL migration in pessimism mode. +type Pessimist struct { + mu sync.Mutex + + logger log.Logger + + closed bool + cancel context.CancelFunc + wg sync.WaitGroup + + cli *clientv3.Client + lk *pessimism.LockKeeper + + // taskSources used to get all sources relative to the given task. + taskSources func(task string) []string +} + +// NewPessimist creates a new Pessimist instance. +func NewPessimist(pLogger *log.Logger, taskSources func(task string) []string) *Pessimist { + return &Pessimist{ + logger: pLogger.WithFields(zap.String("component", "shard DDL pessimist")), + closed: true, // mark as closed before started. + lk: pessimism.NewLockKeeper(), + taskSources: taskSources, + } +} + +// Start starts the shard DDL coordination in pessimism mode. +func (p *Pessimist) Start(pCtx context.Context, etcdCli *clientv3.Client) error { + p.logger.Info("the shard DDL pessimist is starting") + + p.mu.Lock() + defer p.mu.Unlock() + + p.lk.Clear() // clear all previous locks to support re-Start. + + // get the history shard DDL info. + // for the sequence of coordinate a shard DDL lock, see `/pkg/shardddl/pessimism/doc.go`. + ifm, rev1, err := pessimism.GetAllInfo(etcdCli) + if err != nil { + return err + } + p.logger.Info("get history shard DDL info", zap.Reflect("info", ifm), zap.Int64("revision", rev1)) + + // get the history shard DDL lock operation. + // the newly operations after this GET will be received through the WATCH with `rev2`, + // and call `Lock.MarkDone` multiple times is fine. + opm, rev2, err := pessimism.GetAllOperations(etcdCli) + if err != nil { + return err + } + p.logger.Info("get history shard DDL lock operation", zap.Reflect("operation", opm), zap.Int64("revision", rev2)) + + // recover the shard DDL lock based on history shard DDL info & lock operation. + err = p.recoverLocks(ifm, opm) + if err != nil { + return err + } + + ctx, cancel := context.WithCancel(pCtx) + + // watch for the shard DDL info and handle them. + infoCh := make(chan pessimism.Info, 10) + p.wg.Add(2) + go func() { + defer func() { + p.wg.Done() + close(infoCh) + }() + pessimism.WatchInfoPut(ctx, etcdCli, rev1, infoCh) + }() + go func() { + defer p.wg.Done() + p.handleInfoPut(ctx, infoCh) + }() + + // watch for the shard DDL lock operation and handle them. + opCh := make(chan pessimism.Operation, 10) + p.wg.Add(2) + go func() { + defer func() { + p.wg.Done() + close(opCh) + }() + pessimism.WatchOperationPut(ctx, etcdCli, "", "", rev2, opCh) + }() + go func() { + defer p.wg.Done() + p.handleOperationPut(ctx, opCh) + }() + + p.closed = false // started now. + p.cancel = cancel + p.cli = etcdCli + p.logger.Info("the shard DDL pessimist has started") + return nil +} + +// Close closes the Pessimist instance. +func (p *Pessimist) Close() { + p.mu.Lock() + defer p.mu.Unlock() + + if p.closed { + return + } + + if p.cancel != nil { + p.cancel() + p.cancel = nil + } + + p.wg.Wait() + p.closed = true // closed now. + p.logger.Info("the shard DDL pessimist has closed") +} + +// Locks return all shard DDL locks current exist. +func (p *Pessimist) Locks() map[string]*pessimism.Lock { + return p.lk.Locks() +} + +// recoverLocks recovers shard DDL locks based on shard DDL info and shard DDL lock operation. +func (p *Pessimist) recoverLocks(ifm map[string]map[string]pessimism.Info, opm map[string]map[string]pessimism.Operation) error { + // construct locks based on the shard DDL info. + for task, ifs := range ifm { + sources := p.taskSources(task) + for _, info := range ifs { + _, _, _, err := p.lk.TrySync(info, sources) + if err != nil { + return err + } + } + } + + // update locks based on the lock operation. + for _, ops := range opm { + for source, op := range ops { + lock := p.lk.FindLock(op.ID) + if lock == nil { + p.logger.Warn("no shard DDL lock exists for the operation", zap.Stringer("operation", op)) + continue + } + + // if any operation exists, the lock must have been synced. + lock.ForceSynced() + + if op.Done { + lock.MarkDone(source) + } + if op.Exec { + // restore the role of `owner` based on `exec` operation. + // This is needed because `TrySync` can only set `owner` for the first call of the lock. + p.logger.Info("restore the role of owner for the shard DDL lock", zap.String("lock", op.ID), zap.String("from", lock.Owner), zap.String("to", op.Source)) + lock.Owner = op.Source + } + } + } + + // try to handle locks. + for _, lock := range p.lk.Locks() { + synced, remain := lock.IsSynced() + if !synced { + p.logger.Info("restored an un-synced shard DDL lock", zap.String("lock", lock.ID), zap.Int("remain", remain)) + continue + } + err := p.handleLock(lock.ID) + if err != nil { + return err + } + } + + return nil +} + +// handleInfoPut handles the shard DDL lock info PUTed. +func (p *Pessimist) handleInfoPut(ctx context.Context, infoCh <-chan pessimism.Info) { + for { + select { + case <-ctx.Done(): + return + case info, ok := <-infoCh: + if !ok { + return + } + p.logger.Info("receive a shard DDL info", zap.Stringer("info", info)) + lockID, synced, remain, err := p.lk.TrySync(info, p.taskSources(info.Task)) + if err != nil { + // TODO: add & update metrics. + p.logger.Error("fail to try sync shard DDL lock", zap.Stringer("info", info), log.ShortError(err)) + continue + } else if !synced { + p.logger.Info("the shard DDL lock has not synced", zap.String("lock", lockID), zap.Int("remain", remain)) + continue + } + p.logger.Info("the shard DDL lock has synced", zap.String("lock", lockID)) + + err = p.handleLock(lockID) + if err != nil { + // TODO: add & update metrics. + p.logger.Error("fail to handle the shard DDL lock", zap.String("lock", lockID), log.ShortError(err)) + continue + } + } + } +} + +// handleOperationPut handles the shard DDL lock operations PUTed. +func (p *Pessimist) handleOperationPut(ctx context.Context, opCh <-chan pessimism.Operation) { + for { + select { + case <-ctx.Done(): + return + case op, ok := <-opCh: + if !ok { + return + } + p.logger.Info("receive a shard DDL lock operation", zap.Stringer("operation", op)) + if !op.Done { + p.logger.Info("the shard DDL lock operation has not done", zap.Stringer("operation", op)) + continue + } + + lock := p.lk.FindLock(op.ID) + if lock == nil { + p.logger.Warn("no lock for the shard DDL lock operation exist", zap.Stringer("operation", op)) + continue + } else if synced, _ := lock.IsSynced(); !synced { + // this should not happen in normal case. + p.logger.Warn("the lock for the shard DDL lock operation has not synced", zap.Stringer("operation", op)) + continue + } + + // update the `done` status of the lock and check whether is resolved. + lock.MarkDone(op.Source) + if lock.IsResolved() { + p.logger.Info("the lock for the shard DDL lock operation has been resolved", zap.Stringer("operation", op)) + // remove all operations for this shard DDL lock. + err := p.deleteOps(lock) + if err != nil { + // TODO: add & update metrics. + p.logger.Error("fail to delete the shard DDL lock operations", zap.String("lock", lock.ID), log.ShortError(err)) + } + p.lk.RemoveLock(lock.ID) + p.logger.Info("the lock info for the shard DDL lock operation has been cleared", zap.Stringer("operation", op)) + continue + } + + // one of the non-owner dm-worker instance has done the operation, + // still need to wait for more `done` from other non-owner dm-worker instances. + if op.Source != lock.Owner { + p.logger.Info("the shard DDL lock operation of a non-owner has done", zap.Stringer("operation", op), zap.String("owner", lock.Owner)) + continue + } + + // the owner has done the operation, put `skip` operation for non-owner dm-worker instances. + // no need to `skipDone`, all of them should be not done just after the owner has done. + err := p.putOpsForNonOwner(lock, false) + if err != nil { + // TODO: add & update metrics. + p.logger.Error("fail to put skip shard DDL lock operations for non-owner", zap.String("lock", lock.ID), log.ShortError(err)) + } + } + } +} + +// handleLock handles a single shard DDL lock. +func (p *Pessimist) handleLock(lockID string) error { + lock := p.lk.FindLock(lockID) + if lock == nil { + return nil + } + if synced, _ := lock.IsSynced(); !synced { + return nil // do not handle un-synced lock now. + } + + // check whether the lock has resolved. + if lock.IsResolved() { + // remove all operations for this shard DDL lock. + // this is to handle the case where dm-master exit before deleting operations for them. + err := p.deleteOps(lock) + if err != nil { + return err + } + p.lk.RemoveLock(lock.ID) + return nil + } + + // check whether the owner has done. + if lock.IsDone(lock.Owner) { + // try to put the skip operation for non-owner dm-worker instances, + // this is to handle the case where dm-master exit before putting operations for them. + // use `skipDone` to avoid overwriting any existing operations. + err := p.putOpsForNonOwner(lock, true) + if err != nil { + return err + } + return nil + } + + // put `exec=true` for the owner and skip it if already existing. + return p.putOpForOwner(lock, true) +} + +// putOpForOwner PUTs the shard DDL lock operation for the owner into etcd. +func (p *Pessimist) putOpForOwner(lock *pessimism.Lock, skipDone bool) error { + op := pessimism.NewOperation(lock.ID, lock.Task, lock.Owner, lock.DDLs, true, false) + rev, succ, err := pessimism.PutOperations(p.cli, skipDone, op) + if err != nil { + return err + } + p.logger.Info("put exec shard DDL lock operation for the owner", zap.String("lock", lock.ID), zap.String("owner", lock.Owner), zap.Bool("already exist", !succ), zap.Int64("revision", rev)) + return nil +} + +// putOpsForNonOwner PUTs shard DDL lock operations for non-owner dm-worker instances into etcd. +func (p *Pessimist) putOpsForNonOwner(lock *pessimism.Lock, skipDone bool) error { + ready := lock.Ready() + sources := make([]string, 0, len(ready)-1) + ops := make([]pessimism.Operation, 0, len(ready)-1) + for source := range ready { + if source != lock.Owner { + sources = append(sources, source) + ops = append(ops, pessimism.NewOperation(lock.ID, lock.Task, source, lock.DDLs, false, false)) + } + } + rev, succ, err := pessimism.PutOperations(p.cli, skipDone, ops...) + if err != nil { + return err + } + p.logger.Info("put skip shard DDL lock operations for non-owner", zap.String("lock", lock.ID), zap.Strings("non-owner", sources), zap.Bool("already exist", !succ), zap.Int64("revision", rev)) + return nil +} + +// deleteOps DELETEs shard DDL lock operations relative to the lock. +func (p *Pessimist) deleteOps(lock *pessimism.Lock) error { + ready := lock.Ready() + ops := make([]pessimism.Operation, 0, len(ready)) + for source := range ready { + // When deleting operations, we do not verify the value of the operation now, + // so simply set `exec=false` and `done=true`. + ops = append(ops, pessimism.NewOperation(lock.ID, lock.Task, source, lock.DDLs, false, true)) + } + rev, err := pessimism.DeleteOperations(p.cli, ops...) + if err != nil { + return err + } + p.logger.Info("delete shard DDL lock operations", zap.String("lock", lock.ID), zap.Int64("revision", rev)) + return err +} diff --git a/dm/master/shardddl/pessimist_test.go b/dm/master/shardddl/pessimist_test.go new file mode 100644 index 0000000000..e1f0f2122f --- /dev/null +++ b/dm/master/shardddl/pessimist_test.go @@ -0,0 +1,279 @@ +// Copyright 2020 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package shardddl + +import ( + "context" + "testing" + "time" + + . "github.com/pingcap/check" + "go.etcd.io/etcd/clientv3" + "go.etcd.io/etcd/integration" + + "github.com/pingcap/dm/dm/common" + "github.com/pingcap/dm/pkg/log" + "github.com/pingcap/dm/pkg/shardddl/pessimism" + "github.com/pingcap/dm/pkg/utils" +) + +var ( + etcdTestCli *clientv3.Client +) + +type testPessimist struct{} + +var _ = Suite(&testPessimist{}) + +func TestPessimist(t *testing.T) { + log.InitLogger(&log.Config{}) + + mockCluster := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1}) + defer mockCluster.Terminate(t) + + etcdTestCli = mockCluster.RandClient() + + TestingT(t) +} + +// clear keys in etcd test cluster. +func clearTestInfoOperation(c *C) { + clearInfo := clientv3.OpDelete(common.ShardDDLPessimismInfoKeyAdapter.Path(), clientv3.WithPrefix()) + clearOp := clientv3.OpDelete(common.ShardDDLPessimismOperationKeyAdapter.Path(), clientv3.WithPrefix()) + _, err := etcdTestCli.Txn(context.Background()).Then(clearInfo, clearOp).Commit() + c.Assert(err, IsNil) +} + +func (t *testPessimist) TestPessimist(c *C) { + defer clearTestInfoOperation(c) + + var ( + watchTimeout = 500 * time.Millisecond + task1 = "task-1" + task2 = "task-2" + source1 = "mysql-replica-1" + source2 = "mysql-replica-2" + source3 = "mysql-replica-3" + schema, table = "foo", "bar" + DDLs = []string{"ALTER TABLE bar ADD COLUMN c1 INT"} + ID1 = "task-1-`foo`.`bar`" + ID2 = "task-2-`foo`.`bar`" + i11 = pessimism.NewInfo(task1, source1, schema, table, DDLs) + i12 = pessimism.NewInfo(task1, source2, schema, table, DDLs) + i21 = pessimism.NewInfo(task2, source1, schema, table, DDLs) + i22 = pessimism.NewInfo(task2, source2, schema, table, DDLs) + i23 = pessimism.NewInfo(task2, source3, schema, table, DDLs) + + sources = func(task string) []string { + switch task { + case task1: + return []string{source1, source2} + case task2: + return []string{source1, source2, source3} + default: + c.Fatalf("unsupported task %s", task) + } + return []string{} + } + logger = log.L() + p = NewPessimist(&logger, sources) + ) + + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + + // CASE 1: start without any previous kv and no etcd operation. + c.Assert(p.Start(ctx, etcdTestCli), IsNil) + c.Assert(p.Locks(), HasLen, 0) + p.Close() + p.Close() // close multiple times. + + // CASE 2: start again without any previous kv. + c.Assert(p.Start(ctx, etcdTestCli), IsNil) + c.Assert(p.Locks(), HasLen, 0) + + // PUT i11, will create a lock but not synced. + _, err := pessimism.PutInfo(etcdTestCli, i11) + c.Assert(err, IsNil) + c.Assert(utils.WaitSomething(30, 10*time.Millisecond, func() bool { + return len(p.Locks()) == 1 + }), IsTrue) + c.Assert(p.Locks(), HasKey, ID1) + synced, remain := p.Locks()[ID1].IsSynced() + c.Assert(synced, IsFalse) + c.Assert(remain, Equals, 1) + + // PUT i12, the lock will be synced, then an operation PUT for the owner will be triggered. + rev1, err := pessimism.PutInfo(etcdTestCli, i12) + c.Assert(err, IsNil) + c.Assert(utils.WaitSomething(30, 10*time.Millisecond, func() bool { + synced, _ = p.Locks()[ID1].IsSynced() + return synced + }), IsTrue) + + // wait exec operation for the owner become available. + opCh := make(chan pessimism.Operation, 10) + ctx2, cancel2 := context.WithTimeout(ctx, watchTimeout) + pessimism.WatchOperationPut(ctx2, etcdTestCli, task1, source1, rev1, opCh) + cancel2() + close(opCh) + c.Assert(len(opCh), Equals, 1) + op11 := <-opCh + c.Assert(op11.Exec, IsTrue) + c.Assert(op11.Done, IsFalse) + + // mark exec operation for the owner as `done` (and delete the info). + op11c := op11 + op11c.Done = true + rev2, err := pessimism.PutOperationDeleteInfo(etcdTestCli, op11c, i11) + c.Assert(err, IsNil) + c.Assert(utils.WaitSomething(30, 10*time.Millisecond, func() bool { + return p.Locks()[ID1].IsDone(source1) + }), IsTrue) + + // wait skip operation for the non-owner become available. + opCh = make(chan pessimism.Operation, 10) + ctx2, cancel2 = context.WithTimeout(ctx, watchTimeout) + pessimism.WatchOperationPut(ctx2, etcdTestCli, task1, source2, rev2, opCh) + cancel2() + close(opCh) + c.Assert(len(opCh), Equals, 1) + op12 := <-opCh + c.Assert(op12.Exec, IsFalse) + c.Assert(op12.Done, IsFalse) + + // mark skip operation for the non-owner as `done` (and delete the info). + // the lock should become resolved and deleted. + op12c := op12 + op12c.Done = true + _, err = pessimism.PutOperationDeleteInfo(etcdTestCli, op12c, i12) + c.Assert(err, IsNil) + c.Assert(utils.WaitSomething(30, 10*time.Millisecond, func() bool { + _, ok := p.Locks()[ID1] + return !ok + }), IsTrue) + c.Assert(p.Locks(), HasLen, 0) + + // PUT i21, i22, this will create a lock. + _, err = pessimism.PutInfo(etcdTestCli, i21) + c.Assert(err, IsNil) + _, err = pessimism.PutInfo(etcdTestCli, i22) + c.Assert(err, IsNil) + c.Assert(utils.WaitSomething(30, 10*time.Millisecond, func() bool { + lock := p.Locks()[ID2] + if lock == nil { + return false + } + _, remain = lock.IsSynced() + return remain == 1 + }), IsTrue) + + p.Close() // close the Pessimist. + + // CASE 3: start again with some previous shard DDL info and the lock is un-synced. + c.Assert(p.Start(ctx, etcdTestCli), IsNil) + c.Assert(p.Locks(), HasLen, 1) + c.Assert(p.Locks(), HasKey, ID2) + synced, remain = p.Locks()[ID2].IsSynced() + c.Assert(synced, IsFalse) + c.Assert(remain, Equals, 1) + + // PUT i23, then the lock will become synced. + rev3, err := pessimism.PutInfo(etcdTestCli, i23) + c.Assert(err, IsNil) + c.Assert(utils.WaitSomething(30, 10*time.Millisecond, func() bool { + synced, _ = p.Locks()[ID2].IsSynced() + return synced + }), IsTrue) + + // wait exec operation for the owner become available. + opCh = make(chan pessimism.Operation, 10) + ctx2, cancel2 = context.WithTimeout(ctx, watchTimeout) + pessimism.WatchOperationPut(ctx2, etcdTestCli, task2, source1, rev3, opCh) + cancel2() + close(opCh) + c.Logf("watch operation PUT with revision %d", rev3) + c.Assert(len(opCh), Equals, 1) + op21 := <-opCh + c.Assert(op21.Exec, IsTrue) + c.Assert(op21.Done, IsFalse) + + p.Close() // close the Pessimist. + + // CASE 4: start again with some previous shard DDL info and non-`done` operation. + c.Assert(p.Start(ctx, etcdTestCli), IsNil) + c.Assert(p.Locks(), HasLen, 1) + c.Assert(p.Locks(), HasKey, ID2) + synced, _ = p.Locks()[ID2].IsSynced() + c.Assert(synced, IsTrue) + c.Assert(p.Locks()[ID2].IsDone(source1), IsFalse) + + // mark exec operation for the owner as `done` (and delete the info). + op21c := op21 + op21c.Done = true + _, err = pessimism.PutOperationDeleteInfo(etcdTestCli, op21c, i21) + c.Assert(err, IsNil) + c.Assert(utils.WaitSomething(30, 10*time.Millisecond, func() bool { + return p.Locks()[ID2].IsDone(source1) + }), IsTrue) + + p.Close() // close the Pessimist. + + // CASE 5: start again with some previous shard DDL info and `done` operation for the owner. + c.Assert(p.Start(ctx, etcdTestCli), IsNil) + c.Assert(p.Locks(), HasLen, 1) + c.Assert(p.Locks(), HasKey, ID2) + synced, _ = p.Locks()[ID2].IsSynced() + c.Assert(synced, IsTrue) + c.Assert(p.Locks()[ID2].IsDone(source1), IsTrue) + c.Assert(p.Locks()[ID2].IsDone(source2), IsFalse) + + // mark exec operation for one non-owner as `done` (and delete the info). + op22c := pessimism.NewOperation(ID2, task2, source2, DDLs, false, true) + _, err = pessimism.PutOperationDeleteInfo(etcdTestCli, op22c, i22) + c.Assert(err, IsNil) + c.Assert(utils.WaitSomething(30, 10*time.Millisecond, func() bool { + return p.Locks()[ID2].IsDone(source2) + }), IsTrue) + + p.Close() // close the Pessimist. + + // CASE 6: start again with some previous shard DDL info and `done` operation for the owner and non-owner. + c.Assert(p.Start(ctx, etcdTestCli), IsNil) + c.Assert(p.Locks(), HasLen, 1) + c.Assert(p.Locks(), HasKey, ID2) + synced, _ = p.Locks()[ID2].IsSynced() + c.Assert(synced, IsTrue) + c.Assert(p.Locks()[ID2].IsDone(source1), IsTrue) + c.Assert(p.Locks()[ID2].IsDone(source2), IsTrue) + c.Assert(p.Locks()[ID2].IsDone(source3), IsFalse) + + // mark skip operation for the non-owner as `done` (and delete the info). + // the lock should become resolved and deleted. + op23c := pessimism.NewOperation(ID2, task2, source3, DDLs, false, true) + _, err = pessimism.PutOperationDeleteInfo(etcdTestCli, op23c, i23) + c.Assert(err, IsNil) + c.Assert(utils.WaitSomething(30, 10*time.Millisecond, func() bool { + _, ok := p.Locks()[ID2] + return !ok + }), IsTrue) + c.Assert(p.Locks(), HasLen, 0) + + p.Close() // close the Pessimist. + + // CASE 7: start again after all shard DDL locks have been resolved. + c.Assert(p.Start(ctx, etcdTestCli), IsNil) + c.Assert(p.Locks(), HasLen, 0) + p.Close() // close the Pessimist. +} diff --git a/dm/master/workerrpc/interface.go b/dm/master/workerrpc/interface.go index 85297f2f79..c845de8c5a 100644 --- a/dm/master/workerrpc/interface.go +++ b/dm/master/workerrpc/interface.go @@ -61,8 +61,6 @@ type Request struct { QueryWorkerConfig *pb.QueryWorkerConfigRequest HandleSubTaskSQLs *pb.HandleSubTaskSQLsRequest - ExecDDL *pb.ExecDDLRequest - BreakDDLLock *pb.BreakDDLLockRequest SwitchRelayMaster *pb.SwitchRelayMasterRequest OperateRelay *pb.OperateRelayRequest @@ -94,8 +92,6 @@ type Response struct { UpdateRelay *pb.CommonWorkerResponse MigrateRelay *pb.CommonWorkerResponse MysqlTask *pb.MysqlWorkerResponse - - FetchDDLInfo pb.Worker_FetchDDLInfoClient } // Client is a client that sends RPC. diff --git a/dm/master/workerrpc/rawgrpc.go b/dm/master/workerrpc/rawgrpc.go index aa5ec10560..4255900491 100644 --- a/dm/master/workerrpc/rawgrpc.go +++ b/dm/master/workerrpc/rawgrpc.go @@ -109,10 +109,6 @@ func callRPC(ctx context.Context, client pb.WorkerClient, req *Request) (*Respon resp.QueryWorkerConfig, err = client.QueryWorkerConfig(ctx, req.QueryWorkerConfig) case CmdHandleSubTaskSQLs: resp.HandleSubTaskSQLs, err = client.HandleSQLs(ctx, req.HandleSubTaskSQLs) - case CmdExecDDL: - resp.ExecDDL, err = client.ExecuteDDL(ctx, req.ExecDDL) - case CmdBreakDDLLock: - resp.BreakDDLLock, err = client.BreakDDLLock(ctx, req.BreakDDLLock) case CmdSwitchRelayMaster: resp.SwitchRelayMaster, err = client.SwitchRelayMaster(ctx, req.SwitchRelayMaster) case CmdOperateRelay: @@ -123,8 +119,6 @@ func callRPC(ctx context.Context, client pb.WorkerClient, req *Request) (*Respon resp.UpdateRelay, err = client.UpdateRelayConfig(ctx, req.UpdateRelay) case CmdMigrateRelay: resp.MigrateRelay, err = client.MigrateRelay(ctx, req.MigrateRelay) - case CmdFetchDDLInfo: - resp.FetchDDLInfo, err = client.FetchDDLInfo(ctx) case CmdOperateMysqlTask: resp.MysqlTask, err = client.OperateMysqlWorker(ctx, req.MysqlTask) default: diff --git a/dm/pb/dmworker.pb.go b/dm/pb/dmworker.pb.go index 5513a28404..5a503bfcc5 100644 --- a/dm/pb/dmworker.pb.go +++ b/dm/pb/dmworker.pb.go @@ -2389,284 +2389,6 @@ func (m *ProcessError) GetError() *TError { return nil } -// DDLInfo represents a occurred DDL in a sub task -// it been sent from dm-worker to dm-master -// dm-master uses it to contruct a DDL lock and do syncing with other dm-workers -// add more fields if needed -type DDLInfo struct { - Task string `protobuf:"bytes,1,opt,name=task,proto3" json:"task,omitempty"` - Schema string `protobuf:"bytes,2,opt,name=schema,proto3" json:"schema,omitempty"` - Table string `protobuf:"bytes,3,opt,name=table,proto3" json:"table,omitempty"` - DDLs []string `protobuf:"bytes,4,rep,name=DDLs,proto3" json:"DDLs,omitempty"` -} - -func (m *DDLInfo) Reset() { *m = DDLInfo{} } -func (m *DDLInfo) String() string { return proto.CompactTextString(m) } -func (*DDLInfo) ProtoMessage() {} -func (*DDLInfo) Descriptor() ([]byte, []int) { - return fileDescriptor_51a1b9e17fd67b10, []int{31} -} -func (m *DDLInfo) XXX_Unmarshal(b []byte) error { - return m.Unmarshal(b) -} -func (m *DDLInfo) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { - if deterministic { - return xxx_messageInfo_DDLInfo.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 *DDLInfo) XXX_Merge(src proto.Message) { - xxx_messageInfo_DDLInfo.Merge(m, src) -} -func (m *DDLInfo) XXX_Size() int { - return m.Size() -} -func (m *DDLInfo) XXX_DiscardUnknown() { - xxx_messageInfo_DDLInfo.DiscardUnknown(m) -} - -var xxx_messageInfo_DDLInfo proto.InternalMessageInfo - -func (m *DDLInfo) GetTask() string { - if m != nil { - return m.Task - } - return "" -} - -func (m *DDLInfo) GetSchema() string { - if m != nil { - return m.Schema - } - return "" -} - -func (m *DDLInfo) GetTable() string { - if m != nil { - return m.Table - } - return "" -} - -func (m *DDLInfo) GetDDLs() []string { - if m != nil { - return m.DDLs - } - return nil -} - -// DDLLockInfo represents a DDL lock -// it been sent from dm-master to dm-worker -// add more fields if needed -type DDLLockInfo struct { - Task string `protobuf:"bytes,1,opt,name=task,proto3" json:"task,omitempty"` - ID string `protobuf:"bytes,2,opt,name=ID,proto3" json:"ID,omitempty"` -} - -func (m *DDLLockInfo) Reset() { *m = DDLLockInfo{} } -func (m *DDLLockInfo) String() string { return proto.CompactTextString(m) } -func (*DDLLockInfo) ProtoMessage() {} -func (*DDLLockInfo) Descriptor() ([]byte, []int) { - return fileDescriptor_51a1b9e17fd67b10, []int{32} -} -func (m *DDLLockInfo) XXX_Unmarshal(b []byte) error { - return m.Unmarshal(b) -} -func (m *DDLLockInfo) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { - if deterministic { - return xxx_messageInfo_DDLLockInfo.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 *DDLLockInfo) XXX_Merge(src proto.Message) { - xxx_messageInfo_DDLLockInfo.Merge(m, src) -} -func (m *DDLLockInfo) XXX_Size() int { - return m.Size() -} -func (m *DDLLockInfo) XXX_DiscardUnknown() { - xxx_messageInfo_DDLLockInfo.DiscardUnknown(m) -} - -var xxx_messageInfo_DDLLockInfo proto.InternalMessageInfo - -func (m *DDLLockInfo) GetTask() string { - if m != nil { - return m.Task - } - return "" -} - -func (m *DDLLockInfo) GetID() string { - if m != nil { - return m.ID - } - return "" -} - -// ExecDDLRequest represents a request for a dm-worker to execute (or ignore) a DDL -type ExecDDLRequest struct { - Task string `protobuf:"bytes,1,opt,name=task,proto3" json:"task,omitempty"` - LockID string `protobuf:"bytes,2,opt,name=lockID,proto3" json:"lockID,omitempty"` - Exec bool `protobuf:"varint,3,opt,name=exec,proto3" json:"exec,omitempty"` - TraceGID string `protobuf:"bytes,4,opt,name=traceGID,proto3" json:"traceGID,omitempty"` - DDLs []string `protobuf:"bytes,5,rep,name=DDLs,proto3" json:"DDLs,omitempty"` -} - -func (m *ExecDDLRequest) Reset() { *m = ExecDDLRequest{} } -func (m *ExecDDLRequest) String() string { return proto.CompactTextString(m) } -func (*ExecDDLRequest) ProtoMessage() {} -func (*ExecDDLRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_51a1b9e17fd67b10, []int{33} -} -func (m *ExecDDLRequest) XXX_Unmarshal(b []byte) error { - return m.Unmarshal(b) -} -func (m *ExecDDLRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { - if deterministic { - return xxx_messageInfo_ExecDDLRequest.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 *ExecDDLRequest) XXX_Merge(src proto.Message) { - xxx_messageInfo_ExecDDLRequest.Merge(m, src) -} -func (m *ExecDDLRequest) XXX_Size() int { - return m.Size() -} -func (m *ExecDDLRequest) XXX_DiscardUnknown() { - xxx_messageInfo_ExecDDLRequest.DiscardUnknown(m) -} - -var xxx_messageInfo_ExecDDLRequest proto.InternalMessageInfo - -func (m *ExecDDLRequest) GetTask() string { - if m != nil { - return m.Task - } - return "" -} - -func (m *ExecDDLRequest) GetLockID() string { - if m != nil { - return m.LockID - } - return "" -} - -func (m *ExecDDLRequest) GetExec() bool { - if m != nil { - return m.Exec - } - return false -} - -func (m *ExecDDLRequest) GetTraceGID() string { - if m != nil { - return m.TraceGID - } - return "" -} - -func (m *ExecDDLRequest) GetDDLs() []string { - if m != nil { - return m.DDLs - } - return nil -} - -// BreakDDLLockRequest represents a request for a dm-worker to force to break the DDL lock -// task: sub task's name -// removeLockID: DDLLockInfo's ID which need to remove -// execDDL: execute DDL which is blocking -// skipDDL: skip DDL which is blocking -// execDDL and skipDDL can not specify both at the same time, but can specify neither -type BreakDDLLockRequest struct { - Task string `protobuf:"bytes,1,opt,name=task,proto3" json:"task,omitempty"` - RemoveLockID string `protobuf:"bytes,2,opt,name=removeLockID,proto3" json:"removeLockID,omitempty"` - ExecDDL bool `protobuf:"varint,3,opt,name=execDDL,proto3" json:"execDDL,omitempty"` - SkipDDL bool `protobuf:"varint,4,opt,name=skipDDL,proto3" json:"skipDDL,omitempty"` -} - -func (m *BreakDDLLockRequest) Reset() { *m = BreakDDLLockRequest{} } -func (m *BreakDDLLockRequest) String() string { return proto.CompactTextString(m) } -func (*BreakDDLLockRequest) ProtoMessage() {} -func (*BreakDDLLockRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_51a1b9e17fd67b10, []int{34} -} -func (m *BreakDDLLockRequest) XXX_Unmarshal(b []byte) error { - return m.Unmarshal(b) -} -func (m *BreakDDLLockRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { - if deterministic { - return xxx_messageInfo_BreakDDLLockRequest.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 *BreakDDLLockRequest) XXX_Merge(src proto.Message) { - xxx_messageInfo_BreakDDLLockRequest.Merge(m, src) -} -func (m *BreakDDLLockRequest) XXX_Size() int { - return m.Size() -} -func (m *BreakDDLLockRequest) XXX_DiscardUnknown() { - xxx_messageInfo_BreakDDLLockRequest.DiscardUnknown(m) -} - -var xxx_messageInfo_BreakDDLLockRequest proto.InternalMessageInfo - -func (m *BreakDDLLockRequest) GetTask() string { - if m != nil { - return m.Task - } - return "" -} - -func (m *BreakDDLLockRequest) GetRemoveLockID() string { - if m != nil { - return m.RemoveLockID - } - return "" -} - -func (m *BreakDDLLockRequest) GetExecDDL() bool { - if m != nil { - return m.ExecDDL - } - return false -} - -func (m *BreakDDLLockRequest) GetSkipDDL() bool { - if m != nil { - return m.SkipDDL - } - return false -} - // SwitchRelayMasterRequest represents a request for switching a dm-worker's relay unit to another master server type SwitchRelayMasterRequest struct { } @@ -2675,7 +2397,7 @@ func (m *SwitchRelayMasterRequest) Reset() { *m = SwitchRelayMasterReque func (m *SwitchRelayMasterRequest) String() string { return proto.CompactTextString(m) } func (*SwitchRelayMasterRequest) ProtoMessage() {} func (*SwitchRelayMasterRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_51a1b9e17fd67b10, []int{35} + return fileDescriptor_51a1b9e17fd67b10, []int{31} } func (m *SwitchRelayMasterRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2713,7 +2435,7 @@ 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_51a1b9e17fd67b10, []int{36} + return fileDescriptor_51a1b9e17fd67b10, []int{32} } func (m *OperateRelayRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2760,7 +2482,7 @@ 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_51a1b9e17fd67b10, []int{37} + return fileDescriptor_51a1b9e17fd67b10, []int{33} } func (m *OperateRelayResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2833,7 +2555,7 @@ func (m *PurgeRelayRequest) Reset() { *m = PurgeRelayRequest{} } func (m *PurgeRelayRequest) String() string { return proto.CompactTextString(m) } func (*PurgeRelayRequest) ProtoMessage() {} func (*PurgeRelayRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_51a1b9e17fd67b10, []int{38} + return fileDescriptor_51a1b9e17fd67b10, []int{34} } func (m *PurgeRelayRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2897,7 +2619,7 @@ func (m *QueryWorkerConfigRequest) Reset() { *m = QueryWorkerConfigReque func (m *QueryWorkerConfigRequest) String() string { return proto.CompactTextString(m) } func (*QueryWorkerConfigRequest) ProtoMessage() {} func (*QueryWorkerConfigRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_51a1b9e17fd67b10, []int{39} + return fileDescriptor_51a1b9e17fd67b10, []int{35} } func (m *QueryWorkerConfigRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2937,7 +2659,7 @@ func (m *QueryWorkerConfigResponse) Reset() { *m = QueryWorkerConfigResp func (m *QueryWorkerConfigResponse) String() string { return proto.CompactTextString(m) } func (*QueryWorkerConfigResponse) ProtoMessage() {} func (*QueryWorkerConfigResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_51a1b9e17fd67b10, []int{40} + return fileDescriptor_51a1b9e17fd67b10, []int{36} } func (m *QueryWorkerConfigResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3003,7 +2725,7 @@ func (m *MysqlWorkerRequest) Reset() { *m = MysqlWorkerRequest{} } func (m *MysqlWorkerRequest) String() string { return proto.CompactTextString(m) } func (*MysqlWorkerRequest) ProtoMessage() {} func (*MysqlWorkerRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_51a1b9e17fd67b10, []int{41} + return fileDescriptor_51a1b9e17fd67b10, []int{37} } func (m *MysqlWorkerRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3055,7 +2777,7 @@ func (m *MysqlWorkerResponse) Reset() { *m = MysqlWorkerResponse{} } func (m *MysqlWorkerResponse) String() string { return proto.CompactTextString(m) } func (*MysqlWorkerResponse) ProtoMessage() {} func (*MysqlWorkerResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_51a1b9e17fd67b10, []int{42} + return fileDescriptor_51a1b9e17fd67b10, []int{38} } func (m *MysqlWorkerResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3137,10 +2859,6 @@ func init() { proto.RegisterType((*ProcessResult)(nil), "pb.ProcessResult") proto.RegisterType((*TError)(nil), "pb.TError") proto.RegisterType((*ProcessError)(nil), "pb.ProcessError") - proto.RegisterType((*DDLInfo)(nil), "pb.DDLInfo") - proto.RegisterType((*DDLLockInfo)(nil), "pb.DDLLockInfo") - proto.RegisterType((*ExecDDLRequest)(nil), "pb.ExecDDLRequest") - proto.RegisterType((*BreakDDLLockRequest)(nil), "pb.BreakDDLLockRequest") proto.RegisterType((*SwitchRelayMasterRequest)(nil), "pb.SwitchRelayMasterRequest") proto.RegisterType((*OperateRelayRequest)(nil), "pb.OperateRelayRequest") proto.RegisterType((*OperateRelayResponse)(nil), "pb.OperateRelayResponse") @@ -3154,151 +2872,138 @@ func init() { func init() { proto.RegisterFile("dmworker.proto", fileDescriptor_51a1b9e17fd67b10) } var fileDescriptor_51a1b9e17fd67b10 = []byte{ - // 2293 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xb4, 0x59, 0xcd, 0x73, 0x23, 0x47, - 0x15, 0xd7, 0x8c, 0x3e, 0x2c, 0x3d, 0xc9, 0xde, 0x71, 0x7b, 0xb3, 0x51, 0xc4, 0xc6, 0x98, 0xd9, - 0x54, 0xe2, 0xe8, 0xe0, 0x4a, 0x0c, 0x14, 0x55, 0x40, 0x08, 0x59, 0xc9, 0xeb, 0x75, 0x90, 0x77, - 0xed, 0x91, 0x17, 0xb8, 0xa5, 0xc6, 0xa3, 0xb6, 0x3c, 0xe5, 0xd1, 0xcc, 0xec, 0x7c, 0xd8, 0xf1, - 0x0d, 0xce, 0x54, 0x51, 0x5c, 0x29, 0x8e, 0x14, 0x37, 0xfe, 0x04, 0x6e, 0x1c, 0xe0, 0x98, 0x23, - 0x37, 0xa8, 0xdd, 0x7f, 0x83, 0x03, 0xf5, 0x5e, 0xf7, 0xcc, 0xf4, 0x58, 0x1f, 0x9b, 0xc3, 0x72, - 0x51, 0xcd, 0xfb, 0xe8, 0xd7, 0xaf, 0x7f, 0xef, 0xf5, 0x7b, 0xdd, 0x2d, 0xd8, 0x98, 0xcc, 0x6e, - 0x82, 0xe8, 0x8a, 0x47, 0x7b, 0x61, 0x14, 0x24, 0x01, 0xd3, 0xc3, 0x73, 0xf3, 0x63, 0xd8, 0x1a, - 0x27, 0x76, 0x94, 0x8c, 0xd3, 0xf3, 0x33, 0x3b, 0xbe, 0xb2, 0xf8, 0xcb, 0x94, 0xc7, 0x09, 0x63, - 0x50, 0x4b, 0xec, 0xf8, 0xaa, 0xab, 0xed, 0x68, 0xbb, 0x2d, 0x8b, 0xbe, 0xcd, 0x3d, 0x60, 0x2f, - 0xc2, 0x89, 0x9d, 0x70, 0x8b, 0x7b, 0xf6, 0x6d, 0xa6, 0xd9, 0x85, 0x35, 0x27, 0xf0, 0x13, 0xee, - 0x27, 0x52, 0x39, 0x23, 0xcd, 0x31, 0x6c, 0x1d, 0xbb, 0xd3, 0xe8, 0xee, 0x80, 0x6d, 0x80, 0xc7, - 0xae, 0xef, 0x05, 0xd3, 0x67, 0xf6, 0x8c, 0xcb, 0x31, 0x0a, 0x87, 0x3d, 0x84, 0x96, 0xa0, 0x4e, - 0x82, 0xb8, 0xab, 0xef, 0x68, 0xbb, 0xeb, 0x56, 0xc1, 0x30, 0x0f, 0xe1, 0x9d, 0xe7, 0x21, 0x47, - 0xa3, 0x77, 0x3c, 0xee, 0x81, 0x1e, 0x84, 0x64, 0x6e, 0x63, 0x1f, 0xf6, 0xc2, 0xf3, 0x3d, 0x14, - 0x3e, 0x0f, 0x2d, 0x3d, 0x08, 0x71, 0x35, 0x3e, 0x4e, 0xa6, 0x8b, 0xd5, 0xe0, 0xb7, 0x79, 0x0d, - 0x0f, 0xee, 0x1a, 0x8a, 0xc3, 0xc0, 0x8f, 0xf9, 0x4a, 0x4b, 0x0f, 0xa0, 0x11, 0xf1, 0x38, 0xf5, - 0x12, 0xb2, 0xd5, 0xb4, 0x24, 0x85, 0xfc, 0x38, 0x48, 0x23, 0x87, 0x77, 0xab, 0x34, 0x87, 0xa4, - 0x98, 0x01, 0xd5, 0x59, 0x3c, 0xed, 0xd6, 0x88, 0x89, 0x9f, 0x66, 0x1f, 0xee, 0x0b, 0x14, 0xbf, - 0x05, 0xe2, 0xbb, 0xc0, 0x4e, 0x53, 0x1e, 0xdd, 0x8e, 0x13, 0x3b, 0x49, 0x63, 0x45, 0xd3, 0x2f, - 0xa0, 0x13, 0xab, 0xf9, 0x08, 0x36, 0x49, 0xf3, 0x20, 0x8a, 0x82, 0x68, 0x95, 0xe2, 0x9f, 0x34, - 0xe8, 0x3e, 0xb5, 0xfd, 0x89, 0x97, 0xcd, 0x3f, 0x3e, 0x1d, 0xad, 0xb2, 0xcc, 0xde, 0x23, 0x34, - 0x74, 0x42, 0xa3, 0x85, 0x68, 0x8c, 0x4f, 0x47, 0x05, 0xac, 0x76, 0x34, 0x8d, 0xbb, 0xd5, 0x9d, - 0x2a, 0xaa, 0xe3, 0x37, 0x46, 0xef, 0x3c, 0x8f, 0x9e, 0x58, 0x76, 0xc1, 0xc0, 0xd8, 0xc7, 0x2f, - 0xbd, 0x13, 0x3b, 0x49, 0x78, 0xe4, 0x77, 0xeb, 0x22, 0xf6, 0x05, 0xc7, 0xfc, 0x35, 0xdc, 0x1f, - 0x04, 0xb3, 0x59, 0xe0, 0xff, 0x8a, 0xf2, 0x34, 0x0f, 0x49, 0x01, 0xbb, 0xb6, 0x04, 0x76, 0x7d, - 0x11, 0xec, 0xd5, 0x02, 0xf6, 0xbf, 0x6b, 0xb0, 0x55, 0xc2, 0xf2, 0x6d, 0x59, 0x66, 0x3f, 0x82, - 0xf5, 0x58, 0x42, 0x49, 0xa6, 0xbb, 0xb5, 0x9d, 0xea, 0x6e, 0x7b, 0x7f, 0x93, 0xb0, 0x52, 0x05, - 0x56, 0x59, 0x8f, 0x7d, 0x0a, 0xed, 0x08, 0x37, 0x86, 0x1c, 0x86, 0x68, 0xb4, 0xf7, 0xef, 0xe1, - 0x30, 0xab, 0x60, 0x5b, 0xaa, 0x8e, 0xf9, 0x37, 0x4d, 0x66, 0x84, 0x8c, 0xf3, 0x5b, 0x5b, 0xc4, - 0x0f, 0xa0, 0x23, 0x9d, 0x23, 0xcb, 0x72, 0x0d, 0x86, 0xb2, 0x06, 0x31, 0x63, 0x49, 0x8b, 0xed, - 0x01, 0x90, 0xab, 0x62, 0x8c, 0x58, 0xc0, 0x46, 0xbe, 0x00, 0x31, 0x42, 0xd1, 0x30, 0xff, 0xa2, - 0x41, 0x7b, 0x70, 0xc9, 0x9d, 0x0c, 0x81, 0x07, 0xd0, 0x08, 0xed, 0x38, 0xe6, 0x93, 0xcc, 0x6f, - 0x41, 0xb1, 0xfb, 0x50, 0x4f, 0x82, 0xc4, 0xf6, 0xc8, 0xed, 0xba, 0x25, 0x08, 0x4a, 0x9e, 0xd4, - 0x71, 0x78, 0x1c, 0x5f, 0xa4, 0x1e, 0x39, 0x5f, 0xb7, 0x14, 0x0e, 0x5a, 0xbb, 0xb0, 0x5d, 0x8f, - 0x4f, 0x28, 0xef, 0xea, 0x96, 0xa4, 0xb0, 0x42, 0xdd, 0xd8, 0x91, 0xef, 0xfa, 0x53, 0x72, 0xb1, - 0x6e, 0x65, 0x24, 0x8e, 0x98, 0xf0, 0xc4, 0x76, 0xbd, 0x6e, 0x63, 0x47, 0xdb, 0xed, 0x58, 0x92, - 0x32, 0x3b, 0x00, 0xc3, 0x74, 0x16, 0x4a, 0xd0, 0x7f, 0xaf, 0x01, 0x8c, 0x02, 0x7b, 0x22, 0x9d, - 0xfe, 0x00, 0xd6, 0x2f, 0x5c, 0xdf, 0x8d, 0x2f, 0xf9, 0xe4, 0xf1, 0x6d, 0xc2, 0x63, 0xf2, 0xbd, - 0x6a, 0x95, 0x99, 0xe8, 0x2c, 0x79, 0x2d, 0x54, 0x74, 0x52, 0x51, 0x38, 0xac, 0x07, 0xcd, 0x30, - 0x0a, 0xa6, 0x11, 0x8f, 0x63, 0x19, 0x87, 0x9c, 0xc6, 0xb1, 0x33, 0x9e, 0xd8, 0xa2, 0xe8, 0xc9, - 0x4d, 0xa4, 0x70, 0xcc, 0xdf, 0x69, 0xb0, 0x3e, 0xbe, 0xb4, 0xa3, 0x89, 0xeb, 0x4f, 0x0f, 0xa3, - 0x20, 0xa5, 0xb2, 0x94, 0xd8, 0xd1, 0x94, 0x67, 0x35, 0x58, 0x52, 0xb8, 0x43, 0x87, 0xc3, 0x11, - 0xce, 0x4f, 0x3b, 0x14, 0xbf, 0x71, 0xe6, 0x0b, 0x37, 0x8a, 0x13, 0xdc, 0xa0, 0x72, 0xe6, 0x8c, - 0xa6, 0x84, 0xb9, 0xf5, 0x1d, 0x82, 0xb0, 0x4a, 0x09, 0x43, 0x14, 0x8e, 0x49, 0x7d, 0x29, 0xa9, - 0x93, 0x24, 0xa7, 0xcd, 0x3f, 0x57, 0x01, 0xc6, 0xb7, 0xbe, 0x23, 0xe1, 0xd9, 0x81, 0x36, 0x2d, - 0xf3, 0xe0, 0x9a, 0xfb, 0x49, 0x06, 0x8e, 0xca, 0x42, 0x63, 0x44, 0x9e, 0x85, 0x19, 0x30, 0x39, - 0x8d, 0xe5, 0x23, 0xe2, 0x0e, 0xf7, 0x13, 0x14, 0x56, 0x49, 0x58, 0x30, 0x98, 0x09, 0x9d, 0x99, - 0x1d, 0x27, 0x3c, 0x2a, 0x41, 0x53, 0xe2, 0xb1, 0x3e, 0x18, 0x2a, 0x7d, 0x98, 0xb8, 0x13, 0x59, - 0x68, 0xe6, 0xf8, 0x68, 0x8f, 0x16, 0x91, 0xd9, 0x6b, 0x08, 0x7b, 0x2a, 0x0f, 0xed, 0xa9, 0x34, - 0xd9, 0x5b, 0x13, 0xf6, 0xee, 0xf2, 0xd1, 0xde, 0xb9, 0x17, 0x38, 0x57, 0xae, 0x3f, 0x25, 0xd8, - 0x9b, 0x04, 0x55, 0x89, 0xc7, 0x3e, 0x03, 0x23, 0xf5, 0x23, 0x1e, 0x07, 0xde, 0x35, 0x9f, 0x50, - 0xf4, 0xe2, 0x6e, 0x4b, 0xa9, 0x18, 0x6a, 0x5c, 0xad, 0x39, 0x55, 0x25, 0x42, 0x20, 0xb6, 0x8c, - 0x8c, 0xd0, 0x36, 0x80, 0x28, 0xb3, 0x67, 0xb7, 0x21, 0xef, 0xb6, 0x45, 0xce, 0x14, 0x1c, 0xf3, - 0x1f, 0x3a, 0xb4, 0x95, 0xb2, 0x32, 0x07, 0xa5, 0xf6, 0x2d, 0xa1, 0xd4, 0x97, 0x40, 0xb9, 0x93, - 0x15, 0xb3, 0xf4, 0x7c, 0xe8, 0x46, 0x32, 0xb1, 0x54, 0x56, 0xae, 0x51, 0x8a, 0x9d, 0xca, 0x62, - 0xbb, 0x70, 0x4f, 0x21, 0x95, 0xc8, 0xdd, 0x65, 0xb3, 0x3d, 0x60, 0xc4, 0x1a, 0xd8, 0x89, 0x73, - 0xf9, 0x22, 0x3c, 0x26, 0x6f, 0x28, 0x7c, 0x4d, 0x6b, 0x81, 0x84, 0x7d, 0x17, 0xea, 0x71, 0x62, - 0x4f, 0x39, 0x45, 0x2e, 0xeb, 0x63, 0xc8, 0xb0, 0x04, 0x9f, 0x7d, 0x9c, 0x57, 0xd0, 0x26, 0x55, - 0x31, 0x8a, 0xc5, 0x49, 0x14, 0x60, 0x6d, 0xb1, 0x48, 0x90, 0x15, 0x55, 0xf3, 0xbf, 0x3a, 0xac, - 0x97, 0xea, 0xfa, 0xc2, 0xb6, 0x99, 0xcf, 0xa8, 0x2f, 0x99, 0x71, 0x07, 0x6a, 0xa9, 0xef, 0x26, - 0x84, 0xd4, 0xc6, 0x7e, 0x07, 0xe5, 0x2f, 0x7c, 0x37, 0xc1, 0x60, 0x59, 0x24, 0x51, 0x7c, 0xaa, - 0xbd, 0xc1, 0x27, 0xf6, 0x09, 0x6c, 0x15, 0x99, 0x32, 0x1c, 0x8e, 0x46, 0x81, 0x73, 0x75, 0x34, - 0x94, 0xe8, 0x2d, 0x12, 0x31, 0x26, 0x5a, 0x00, 0x65, 0xfc, 0xd3, 0x8a, 0x68, 0x02, 0x1f, 0x41, - 0xdd, 0xc1, 0xea, 0x4c, 0x28, 0xc9, 0x56, 0xa4, 0x94, 0xeb, 0xa7, 0x15, 0x4b, 0xc8, 0xd9, 0x07, - 0x50, 0x9b, 0xa4, 0xb3, 0x50, 0x62, 0x45, 0x15, 0xbf, 0xa8, 0x97, 0x4f, 0x2b, 0x16, 0x49, 0x51, - 0xcb, 0x0b, 0xec, 0x49, 0xb7, 0x55, 0x68, 0x15, 0x65, 0x14, 0xb5, 0x50, 0x8a, 0x5a, 0x98, 0xc2, - 0x94, 0xce, 0x52, 0xab, 0xa8, 0x26, 0xa8, 0x85, 0xd2, 0xc7, 0x4d, 0x68, 0xc4, 0xa2, 0x1a, 0xff, - 0x0c, 0x36, 0x4b, 0xe8, 0x8f, 0xdc, 0x98, 0xa0, 0x12, 0xe2, 0xae, 0xb6, 0xac, 0xf9, 0x66, 0xe3, - 0xb7, 0x01, 0x68, 0x4d, 0xa2, 0x83, 0xc9, 0x4e, 0xa8, 0x15, 0x07, 0x85, 0xf7, 0xa1, 0x85, 0x6b, - 0x59, 0x21, 0xc6, 0x45, 0x2c, 0x13, 0x87, 0xd0, 0x21, 0xef, 0x4f, 0x47, 0x4b, 0x34, 0xd8, 0x3e, - 0xdc, 0x17, 0x7d, 0x29, 0x3f, 0xd3, 0xba, 0x89, 0x1b, 0xf8, 0x72, 0x63, 0x2d, 0x94, 0x61, 0xc5, - 0xe4, 0x68, 0x6e, 0x7c, 0x3a, 0xca, 0x4a, 0x76, 0x46, 0x9b, 0x3f, 0x84, 0x16, 0xce, 0x28, 0xa6, - 0xdb, 0x85, 0x06, 0x09, 0x32, 0x1c, 0x8c, 0x1c, 0x4e, 0xe9, 0x90, 0x25, 0xe5, 0x08, 0x43, 0xd1, - 0x98, 0x17, 0x2c, 0xe4, 0x8f, 0x3a, 0x74, 0xd4, 0xce, 0xff, 0xff, 0x4a, 0x72, 0xa6, 0x1c, 0x90, - 0xb3, 0x3c, 0xfc, 0x30, 0xcb, 0x43, 0xe5, 0x44, 0x51, 0xc4, 0xac, 0x48, 0xc3, 0x47, 0x32, 0x0d, - 0x1b, 0xa4, 0xb6, 0x9e, 0xa5, 0x61, 0xa6, 0x25, 0xb2, 0xf0, 0x91, 0xcc, 0xc2, 0xb5, 0x42, 0x29, - 0x0f, 0x60, 0x9e, 0x84, 0x8f, 0x64, 0x12, 0x36, 0x0b, 0xa5, 0x1c, 0xd4, 0x3c, 0x07, 0xd7, 0xa0, - 0x4e, 0xe0, 0x99, 0x3f, 0x06, 0x43, 0x85, 0x86, 0x32, 0xf0, 0x43, 0x29, 0x2c, 0x01, 0xaf, 0x9e, - 0x9c, 0xe4, 0xd8, 0x97, 0xb0, 0x5e, 0xda, 0xc2, 0x58, 0xb8, 0xdd, 0x78, 0x60, 0xfb, 0x0e, 0xf7, - 0xf2, 0x73, 0x90, 0xc2, 0x51, 0x42, 0xaa, 0x17, 0x96, 0xa5, 0x89, 0x52, 0x48, 0x95, 0xd3, 0x4c, - 0xb5, 0x74, 0x9a, 0xf9, 0xab, 0x06, 0x8d, 0x33, 0x11, 0xc4, 0x2e, 0xac, 0x1d, 0x44, 0xd1, 0x20, - 0x98, 0x88, 0x38, 0xd6, 0xad, 0x8c, 0xc4, 0x14, 0xc3, 0x4f, 0xcf, 0x8e, 0x63, 0x79, 0xea, 0xca, - 0x69, 0x29, 0x1b, 0x3b, 0x41, 0xc8, 0xe5, 0xb1, 0x2b, 0xa7, 0xa5, 0x6c, 0xc4, 0xaf, 0xb9, 0x27, - 0x8f, 0x5d, 0x39, 0x8d, 0xb3, 0x1d, 0xf3, 0x38, 0xc6, 0x04, 0x11, 0x95, 0x28, 0x23, 0x71, 0x94, - 0x65, 0xdf, 0x0c, 0xec, 0x34, 0xe6, 0xb2, 0xe9, 0xe6, 0xb4, 0xc9, 0xa1, 0xa3, 0x2e, 0x8f, 0x7d, - 0x0f, 0x6a, 0xd4, 0xd3, 0xc4, 0x85, 0x8c, 0x62, 0x43, 0x02, 0x91, 0x44, 0xf8, 0x9b, 0xa5, 0xaf, - 0x5e, 0xec, 0xb2, 0x9d, 0x2c, 0x1c, 0x55, 0x8a, 0xa8, 0xb8, 0xc6, 0x95, 0x02, 0xf1, 0x15, 0xac, - 0x0d, 0x87, 0xa3, 0x23, 0xff, 0x22, 0x58, 0x74, 0xf5, 0xa2, 0x3e, 0xeb, 0x5c, 0xf2, 0x99, 0x9d, - 0x1f, 0x9d, 0x89, 0xa2, 0xa3, 0xa9, 0x7d, 0xee, 0x65, 0xf7, 0x3c, 0x41, 0xe4, 0xe7, 0xac, 0x5a, - 0x71, 0xce, 0x32, 0x3f, 0x85, 0x76, 0x56, 0x6e, 0x97, 0x4d, 0xb2, 0x01, 0xfa, 0xd1, 0x50, 0x4e, - 0xa0, 0x1f, 0x0d, 0xcd, 0xdf, 0x68, 0xb0, 0x71, 0xf0, 0x35, 0x77, 0x86, 0xc3, 0xd1, 0x8a, 0x6b, - 0x21, 0xfa, 0xe6, 0x89, 0x02, 0x2f, 0x7d, 0xf3, 0xb2, 0x9a, 0x5e, 0xe3, 0x5f, 0x73, 0x87, 0x5c, - 0x6b, 0x5a, 0xf4, 0x4d, 0x87, 0xad, 0xc8, 0x76, 0xf8, 0xe1, 0xd1, 0x50, 0xb6, 0xdc, 0x9c, 0xce, - 0xbd, 0xae, 0x2b, 0x5e, 0xff, 0x56, 0x83, 0xad, 0xc7, 0x11, 0xb7, 0xaf, 0xa4, 0xef, 0xab, 0xfc, - 0x30, 0xa1, 0x13, 0xf1, 0x59, 0x70, 0xcd, 0x47, 0xaa, 0x37, 0x25, 0x1e, 0xe6, 0x00, 0x17, 0x2b, - 0x92, 0x6e, 0x65, 0x24, 0x4a, 0xe2, 0x2b, 0x37, 0x44, 0x49, 0x4d, 0x48, 0x24, 0x69, 0xf6, 0xa0, - 0x3b, 0xbe, 0x71, 0x13, 0xe7, 0x92, 0x2a, 0x94, 0x68, 0xe1, 0xd2, 0x0f, 0x73, 0x1f, 0xb6, 0xe4, - 0xb5, 0xbd, 0xf4, 0xa8, 0xf0, 0x1d, 0xe5, 0xce, 0xde, 0xce, 0x6f, 0x20, 0xe2, 0x9e, 0x6a, 0xa6, - 0x70, 0xbf, 0x3c, 0x46, 0x5e, 0x9b, 0x56, 0x0d, 0x7a, 0x0b, 0x37, 0xfd, 0x1b, 0xd8, 0x3c, 0x49, - 0xa3, 0x69, 0xd9, 0xd1, 0x1e, 0x34, 0x5d, 0xdf, 0x76, 0x12, 0xf7, 0x9a, 0xcb, 0xcd, 0x9e, 0xd3, - 0x84, 0xb1, 0x2b, 0x9f, 0x29, 0xaa, 0x16, 0x7d, 0x8b, 0xd3, 0xba, 0xc7, 0xa9, 0xf4, 0xe6, 0xa7, - 0x75, 0x41, 0x93, 0x2b, 0xe2, 0xb8, 0x55, 0x93, 0xae, 0x10, 0x85, 0xf8, 0xd1, 0x25, 0x51, 0x5c, - 0xa2, 0x07, 0x81, 0x7f, 0xe1, 0x4e, 0x33, 0xfc, 0x6e, 0xe0, 0xbd, 0x05, 0xb2, 0xb7, 0x76, 0x8f, - 0x54, 0x5e, 0x83, 0x6a, 0xe5, 0xd7, 0xa0, 0x2f, 0x81, 0x1d, 0xdf, 0xc6, 0x2f, 0xbd, 0xec, 0x66, - 0x2f, 0xe0, 0x78, 0xa8, 0x84, 0x80, 0xda, 0x83, 0x10, 0x17, 0x31, 0x70, 0xc8, 0xc3, 0x6c, 0x5e, - 0x41, 0x99, 0x9f, 0xc3, 0x56, 0xc9, 0xd6, 0x1b, 0xdc, 0x9f, 0x2b, 0x0f, 0xfd, 0xaf, 0xa0, 0x21, - 0x1e, 0x75, 0xd8, 0x3a, 0xb4, 0x8e, 0xfc, 0x6b, 0xdb, 0x73, 0x27, 0xcf, 0x43, 0xa3, 0xc2, 0x9a, - 0x50, 0x1b, 0x27, 0x41, 0x68, 0x68, 0xac, 0x05, 0xf5, 0x13, 0xac, 0x47, 0x86, 0xce, 0x00, 0x1a, - 0x58, 0xac, 0x67, 0xdc, 0xa8, 0x22, 0x9b, 0xde, 0xcb, 0x8c, 0x1a, 0xb2, 0xc5, 0x4b, 0x8e, 0x51, - 0x67, 0x1b, 0x00, 0x5f, 0xa4, 0x49, 0x20, 0xd5, 0x1a, 0xfd, 0x3e, 0xd4, 0xe9, 0x9d, 0x84, 0x0c, - 0xfe, 0xe2, 0xe8, 0xc4, 0xa8, 0xb0, 0x36, 0xac, 0x59, 0x07, 0x27, 0xa3, 0x2f, 0x06, 0x07, 0x86, - 0x86, 0x63, 0x8f, 0x9e, 0x7d, 0x79, 0x30, 0x38, 0x33, 0xf4, 0xfe, 0x2f, 0xc9, 0xe4, 0x14, 0xe1, - 0xec, 0x48, 0x5f, 0x88, 0x36, 0x2a, 0x6c, 0x0d, 0xaa, 0xcf, 0xf8, 0x8d, 0xa1, 0xd1, 0xe0, 0xd4, - 0xc7, 0x4b, 0xab, 0xf0, 0x87, 0x5c, 0x9b, 0x18, 0x55, 0x14, 0xa0, 0xc3, 0x21, 0x9f, 0x18, 0x35, - 0xd6, 0x81, 0xe6, 0x13, 0x79, 0x0b, 0x35, 0xea, 0xfd, 0xe7, 0xd0, 0xcc, 0x9a, 0x2d, 0xbb, 0x07, - 0x6d, 0x69, 0x1a, 0x59, 0x46, 0x05, 0xd7, 0x41, 0x2d, 0xd5, 0xd0, 0xd0, 0x45, 0x6c, 0x9b, 0x86, - 0x8e, 0x5f, 0xd8, 0x1b, 0x8d, 0x2a, 0xb9, 0x7d, 0xeb, 0x3b, 0x46, 0x0d, 0x15, 0x29, 0x81, 0x8d, - 0x49, 0xff, 0x27, 0xd0, 0xca, 0x2b, 0x2f, 0x3a, 0xfb, 0xc2, 0xbf, 0xf2, 0x83, 0x1b, 0x9f, 0x78, - 0x62, 0x81, 0x58, 0xbc, 0xc6, 0xa7, 0x23, 0x43, 0xc3, 0x09, 0xc9, 0xfe, 0x13, 0x3a, 0xcf, 0x18, - 0x7a, 0xff, 0x18, 0xd6, 0xe4, 0xf6, 0x62, 0x0c, 0x36, 0xa4, 0x33, 0x92, 0x63, 0x54, 0x30, 0x0e, - 0xb8, 0x0e, 0x31, 0x95, 0x86, 0x78, 0xd2, 0x12, 0x05, 0xad, 0xa3, 0x39, 0x81, 0xad, 0x60, 0x54, - 0xfb, 0x16, 0x34, 0xb3, 0x54, 0x61, 0x5b, 0x70, 0x4f, 0xda, 0xcb, 0x58, 0x46, 0x05, 0x47, 0x50, - 0xa0, 0x04, 0xcb, 0xd0, 0xc8, 0x61, 0x0a, 0x97, 0x48, 0x7a, 0x43, 0xc7, 0x49, 0x70, 0x4e, 0xa9, - 0x51, 0xdd, 0xff, 0x77, 0x13, 0x1a, 0x82, 0x60, 0x03, 0xe8, 0xa8, 0xcf, 0xa2, 0xec, 0x5d, 0x79, - 0xb4, 0xb9, 0xfb, 0x50, 0xda, 0xeb, 0xd2, 0xe1, 0x64, 0xc1, 0x9b, 0x95, 0x59, 0x61, 0x47, 0xb0, - 0x51, 0x7e, 0x62, 0x64, 0xef, 0xa1, 0xf6, 0xc2, 0xf7, 0xcb, 0x5e, 0x6f, 0x91, 0x28, 0x37, 0x75, - 0x00, 0xeb, 0xa5, 0x57, 0x43, 0x46, 0xf3, 0x2e, 0x7a, 0x48, 0x5c, 0xe9, 0xd1, 0xcf, 0xa1, 0xad, - 0x3c, 0x82, 0xb1, 0x07, 0xa8, 0x3a, 0xff, 0xc2, 0xd8, 0x7b, 0x77, 0x8e, 0x9f, 0x5b, 0xf8, 0x0c, - 0xa0, 0x78, 0x80, 0x62, 0xef, 0xe4, 0x8a, 0xea, 0xc3, 0x63, 0xef, 0xc1, 0x5d, 0x76, 0x3e, 0xfc, - 0x09, 0x80, 0x7c, 0x7d, 0x3c, 0x1d, 0xc5, 0xec, 0x21, 0xea, 0x2d, 0x7b, 0x8d, 0x5c, 0xb9, 0x90, - 0x7d, 0xe8, 0x3c, 0xe1, 0x89, 0x73, 0x99, 0xb5, 0x70, 0xba, 0xab, 0x28, 0xed, 0xb6, 0xd7, 0x96, - 0x0c, 0x24, 0xcc, 0xca, 0xae, 0xf6, 0x89, 0xc6, 0x7e, 0x0a, 0x80, 0xf9, 0x99, 0x26, 0x1c, 0xdb, - 0x0f, 0xa3, 0x83, 0x44, 0xa9, 0xd9, 0xae, 0x9c, 0x71, 0x00, 0x1d, 0xb5, 0x2f, 0x8a, 0x8c, 0x58, - 0xd0, 0x29, 0x57, 0x1a, 0x39, 0x86, 0xcd, 0xb9, 0xce, 0x26, 0x50, 0x58, 0xd6, 0xf0, 0xde, 0xe4, - 0x93, 0xda, 0xd8, 0x84, 0x4f, 0x0b, 0xda, 0xa3, 0x30, 0xb2, 0xa8, 0x07, 0x9a, 0x15, 0xf6, 0x39, - 0x40, 0xd1, 0xa6, 0x44, 0x44, 0xe7, 0xda, 0xd6, 0x4a, 0x2f, 0x0e, 0x61, 0x53, 0xf9, 0x5f, 0x40, - 0xec, 0x2e, 0x91, 0x5a, 0xf3, 0x7f, 0x17, 0xac, 0x34, 0x64, 0xc9, 0x47, 0x6c, 0xb5, 0x37, 0x09, - 0x74, 0x96, 0xb5, 0xb3, 0xde, 0xfb, 0x4b, 0xa4, 0x2a, 0x44, 0xea, 0x9f, 0x10, 0x02, 0xa2, 0x05, - 0x7f, 0x4b, 0xbc, 0x61, 0x85, 0x4c, 0x82, 0xa7, 0xb4, 0x1d, 0xb1, 0xc4, 0xf9, 0x9e, 0x26, 0x76, - 0xcf, 0x82, 0xfe, 0x64, 0x56, 0x1e, 0x77, 0xff, 0xf9, 0x6a, 0x5b, 0xfb, 0xe6, 0xd5, 0xb6, 0xf6, - 0x9f, 0x57, 0xdb, 0xda, 0x1f, 0x5e, 0x6f, 0x57, 0xbe, 0x79, 0xbd, 0x5d, 0xf9, 0xd7, 0xeb, 0xed, - 0xca, 0x79, 0x83, 0xfe, 0x92, 0xf9, 0xfe, 0xff, 0x02, 0x00, 0x00, 0xff, 0xff, 0xbc, 0x82, 0x0a, - 0xfd, 0xa4, 0x19, 0x00, 0x00, + // 2094 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xb4, 0x58, 0x4f, 0x73, 0xe3, 0x58, + 0x11, 0xb7, 0x24, 0xdb, 0xb1, 0xdb, 0x4e, 0x46, 0x79, 0xc9, 0x66, 0x3d, 0x66, 0x36, 0x18, 0xcd, + 0xd6, 0x6e, 0xd6, 0x87, 0x14, 0x04, 0x28, 0xaa, 0xa0, 0x96, 0x65, 0xc7, 0xc9, 0xcc, 0x64, 0x71, + 0x66, 0x12, 0x39, 0x03, 0xdc, 0x28, 0xc5, 0x7e, 0x71, 0x54, 0x91, 0x25, 0x8d, 0xfe, 0x24, 0xe4, + 0x33, 0x50, 0x45, 0x71, 0xa5, 0x38, 0x52, 0x54, 0x71, 0xe0, 0x23, 0x70, 0xe3, 0x00, 0xc7, 0x3d, + 0x72, 0xa4, 0x66, 0xbe, 0x06, 0x07, 0xaa, 0xfb, 0x3d, 0x49, 0x4f, 0xf1, 0x9f, 0xd9, 0xc3, 0xec, + 0xc5, 0xe5, 0xfe, 0xf3, 0xfa, 0xf5, 0xfb, 0x75, 0xab, 0xfb, 0xbd, 0x86, 0x8d, 0xc9, 0xec, 0x36, + 0x88, 0xae, 0x79, 0xb4, 0x1f, 0x46, 0x41, 0x12, 0x30, 0x3d, 0xbc, 0xb0, 0x3e, 0x83, 0xad, 0x51, + 0xe2, 0x44, 0xc9, 0x28, 0xbd, 0x38, 0x77, 0xe2, 0x6b, 0x9b, 0xbf, 0x4e, 0x79, 0x9c, 0x30, 0x06, + 0xd5, 0xc4, 0x89, 0xaf, 0x3b, 0x5a, 0x4f, 0xdb, 0x6b, 0xda, 0xf4, 0xdf, 0xda, 0x07, 0xf6, 0x2a, + 0x9c, 0x38, 0x09, 0xb7, 0xb9, 0xe7, 0xdc, 0x65, 0x9a, 0x1d, 0x58, 0x1b, 0x07, 0x7e, 0xc2, 0xfd, + 0x44, 0x2a, 0x67, 0xa4, 0x35, 0x82, 0xad, 0x13, 0x77, 0x1a, 0xdd, 0x5f, 0xb0, 0x0b, 0xf0, 0xc4, + 0xf5, 0xbd, 0x60, 0xfa, 0xc2, 0x99, 0x71, 0xb9, 0x46, 0xe1, 0xb0, 0x47, 0xd0, 0x14, 0xd4, 0x69, + 0x10, 0x77, 0xf4, 0x9e, 0xb6, 0xb7, 0x6e, 0x17, 0x0c, 0xeb, 0x19, 0x7c, 0xf0, 0x32, 0xe4, 0x68, + 0xf4, 0x9e, 0xc7, 0x5d, 0xd0, 0x83, 0x90, 0xcc, 0x6d, 0x1c, 0xc0, 0x7e, 0x78, 0xb1, 0x8f, 0xc2, + 0x97, 0xa1, 0xad, 0x07, 0x21, 0x9e, 0xc6, 0xc7, 0xcd, 0x74, 0x71, 0x1a, 0xfc, 0x6f, 0xdd, 0xc0, + 0xce, 0x7d, 0x43, 0x71, 0x18, 0xf8, 0x31, 0x5f, 0x69, 0x69, 0x07, 0xea, 0x11, 0x8f, 0x53, 0x2f, + 0x21, 0x5b, 0x0d, 0x5b, 0x52, 0xc8, 0x8f, 0x83, 0x34, 0x1a, 0xf3, 0x8e, 0x41, 0x7b, 0x48, 0x8a, + 0x99, 0x60, 0xcc, 0xe2, 0x69, 0xa7, 0x4a, 0x4c, 0xfc, 0x6b, 0xf5, 0x61, 0x5b, 0xa0, 0xf8, 0x0d, + 0x10, 0xdf, 0x03, 0x76, 0x96, 0xf2, 0xe8, 0x6e, 0x94, 0x38, 0x49, 0x1a, 0x2b, 0x9a, 0x7e, 0x01, + 0x9d, 0x38, 0xcd, 0xa7, 0xb0, 0x49, 0x9a, 0x47, 0x51, 0x14, 0x44, 0xab, 0x14, 0xff, 0xac, 0x41, + 0xe7, 0xb9, 0xe3, 0x4f, 0xbc, 0x6c, 0xff, 0xd1, 0xd9, 0x70, 0x95, 0x65, 0xf6, 0x90, 0xd0, 0xd0, + 0x09, 0x8d, 0x26, 0xa2, 0x31, 0x3a, 0x1b, 0x16, 0xb0, 0x3a, 0xd1, 0x34, 0xee, 0x18, 0x3d, 0x03, + 0xd5, 0xf1, 0x3f, 0x46, 0xef, 0x22, 0x8f, 0x9e, 0x38, 0x76, 0xc1, 0xc0, 0xd8, 0xc7, 0xaf, 0xbd, + 0x53, 0x27, 0x49, 0x78, 0xe4, 0x77, 0x6a, 0x22, 0xf6, 0x05, 0xc7, 0xfa, 0x0d, 0x6c, 0x0f, 0x82, + 0xd9, 0x2c, 0xf0, 0x7f, 0x4d, 0x79, 0x9a, 0x87, 0xa4, 0x80, 0x5d, 0x5b, 0x02, 0xbb, 0xbe, 0x08, + 0x76, 0xa3, 0x80, 0xfd, 0x9f, 0x1a, 0x6c, 0x95, 0xb0, 0x7c, 0x5f, 0x96, 0xd9, 0x4f, 0x60, 0x3d, + 0x96, 0x50, 0x92, 0xe9, 0x4e, 0xb5, 0x67, 0xec, 0xb5, 0x0e, 0x36, 0x09, 0x2b, 0x55, 0x60, 0x97, + 0xf5, 0xd8, 0x0f, 0xa0, 0x15, 0xe1, 0x87, 0x21, 0x97, 0x21, 0x1a, 0xad, 0x83, 0x07, 0xb8, 0xcc, + 0x2e, 0xd8, 0xb6, 0xaa, 0x63, 0xfd, 0x43, 0x93, 0x19, 0x21, 0xe3, 0xfc, 0xde, 0x0e, 0xf1, 0x23, + 0x68, 0x4b, 0xe7, 0xc8, 0xb2, 0x3c, 0x83, 0xa9, 0x9c, 0x41, 0xec, 0x58, 0xd2, 0x62, 0xfb, 0x00, + 0xe4, 0xaa, 0x58, 0x23, 0x0e, 0xb0, 0x91, 0x1f, 0x40, 0xac, 0x50, 0x34, 0xac, 0xbf, 0x6a, 0xd0, + 0x1a, 0x5c, 0xf1, 0x71, 0x86, 0xc0, 0x0e, 0xd4, 0x43, 0x27, 0x8e, 0xf9, 0x24, 0xf3, 0x5b, 0x50, + 0x6c, 0x1b, 0x6a, 0x49, 0x90, 0x38, 0x1e, 0xb9, 0x5d, 0xb3, 0x05, 0x41, 0xc9, 0x93, 0x8e, 0xc7, + 0x3c, 0x8e, 0x2f, 0x53, 0x8f, 0x9c, 0xaf, 0xd9, 0x0a, 0x07, 0xad, 0x5d, 0x3a, 0xae, 0xc7, 0x27, + 0x94, 0x77, 0x35, 0x5b, 0x52, 0x58, 0xa1, 0x6e, 0x9d, 0xc8, 0x77, 0xfd, 0x29, 0xb9, 0x58, 0xb3, + 0x33, 0x12, 0x57, 0x4c, 0x78, 0xe2, 0xb8, 0x5e, 0xa7, 0xde, 0xd3, 0xf6, 0xda, 0xb6, 0xa4, 0xac, + 0x36, 0xc0, 0x61, 0x3a, 0x0b, 0x25, 0xe8, 0x7f, 0xd0, 0x00, 0x86, 0x81, 0x33, 0x91, 0x4e, 0x7f, + 0x0c, 0xeb, 0x97, 0xae, 0xef, 0xc6, 0x57, 0x7c, 0xf2, 0xe4, 0x2e, 0xe1, 0x31, 0xf9, 0x6e, 0xd8, + 0x65, 0x26, 0x3a, 0x4b, 0x5e, 0x0b, 0x15, 0x9d, 0x54, 0x14, 0x0e, 0xeb, 0x42, 0x23, 0x8c, 0x82, + 0x69, 0xc4, 0xe3, 0x58, 0xc6, 0x21, 0xa7, 0x71, 0xed, 0x8c, 0x27, 0x8e, 0x28, 0x7a, 0xf2, 0x23, + 0x52, 0x38, 0xd6, 0xef, 0x35, 0x58, 0x1f, 0x5d, 0x39, 0xd1, 0xc4, 0xf5, 0xa7, 0xcf, 0xa2, 0x20, + 0xa5, 0xb2, 0x94, 0x38, 0xd1, 0x94, 0x67, 0x35, 0x58, 0x52, 0xf8, 0x85, 0x1e, 0x1e, 0x0e, 0x71, + 0x7f, 0xfa, 0x42, 0xf1, 0x3f, 0xee, 0x7c, 0xe9, 0x46, 0x71, 0x82, 0x1f, 0xa8, 0xdc, 0x39, 0xa3, + 0x29, 0x61, 0xee, 0xfc, 0x31, 0x41, 0x68, 0x50, 0xc2, 0x10, 0x85, 0x6b, 0x52, 0x5f, 0x4a, 0x6a, + 0x24, 0xc9, 0x69, 0xeb, 0x2f, 0x06, 0xc0, 0xe8, 0xce, 0x1f, 0x4b, 0x78, 0x7a, 0xd0, 0xa2, 0x63, + 0x1e, 0xdd, 0x70, 0x3f, 0xc9, 0xc0, 0x51, 0x59, 0x68, 0x8c, 0xc8, 0xf3, 0x30, 0x03, 0x26, 0xa7, + 0xb1, 0x7c, 0x44, 0x7c, 0xcc, 0xfd, 0x04, 0x85, 0x06, 0x09, 0x0b, 0x06, 0xb3, 0xa0, 0x3d, 0x73, + 0xe2, 0x84, 0x47, 0x25, 0x68, 0x4a, 0x3c, 0xd6, 0x07, 0x53, 0xa5, 0x9f, 0x25, 0xee, 0x44, 0x16, + 0x9a, 0x39, 0x3e, 0xda, 0xa3, 0x43, 0x64, 0xf6, 0xea, 0xc2, 0x9e, 0xca, 0x43, 0x7b, 0x2a, 0x4d, + 0xf6, 0xd6, 0x84, 0xbd, 0xfb, 0x7c, 0xb4, 0x77, 0xe1, 0x05, 0xe3, 0x6b, 0xd7, 0x9f, 0x12, 0xec, + 0x0d, 0x82, 0xaa, 0xc4, 0x63, 0x9f, 0x83, 0x99, 0xfa, 0x11, 0x8f, 0x03, 0xef, 0x86, 0x4f, 0x28, + 0x7a, 0x71, 0xa7, 0xa9, 0x54, 0x0c, 0x35, 0xae, 0xf6, 0x9c, 0xaa, 0x12, 0x21, 0x10, 0x9f, 0x8c, + 0x8c, 0xd0, 0x2e, 0x80, 0x28, 0xb3, 0xe7, 0x77, 0x21, 0xef, 0xb4, 0x44, 0xce, 0x14, 0x1c, 0xeb, + 0x5f, 0x3a, 0xb4, 0x94, 0xb2, 0x32, 0x07, 0xa5, 0xf6, 0x0d, 0xa1, 0xd4, 0x97, 0x40, 0xd9, 0xcb, + 0x8a, 0x59, 0x7a, 0x71, 0xe8, 0x46, 0x32, 0xb1, 0x54, 0x56, 0xae, 0x51, 0x8a, 0x9d, 0xca, 0x62, + 0x7b, 0xf0, 0x40, 0x21, 0x95, 0xc8, 0xdd, 0x67, 0xb3, 0x7d, 0x60, 0xc4, 0x1a, 0x38, 0xc9, 0xf8, + 0xea, 0x55, 0x78, 0x42, 0xde, 0x50, 0xf8, 0x1a, 0xf6, 0x02, 0x09, 0xfb, 0x2e, 0xd4, 0xe2, 0xc4, + 0x99, 0x72, 0x8a, 0x5c, 0xd6, 0xc7, 0x90, 0x61, 0x0b, 0x3e, 0xfb, 0x2c, 0xaf, 0xa0, 0x0d, 0xaa, + 0x62, 0x14, 0x8b, 0xd3, 0x28, 0xc0, 0xda, 0x62, 0x93, 0x20, 0x2b, 0xaa, 0xd6, 0xff, 0x74, 0x58, + 0x2f, 0xd5, 0xf5, 0x85, 0x6d, 0x33, 0xdf, 0x51, 0x5f, 0xb2, 0x63, 0x0f, 0xaa, 0xa9, 0xef, 0x26, + 0x84, 0xd4, 0xc6, 0x41, 0x1b, 0xe5, 0xaf, 0x7c, 0x37, 0xc1, 0x60, 0xd9, 0x24, 0x51, 0x7c, 0xaa, + 0xbe, 0xc3, 0x27, 0xf6, 0x7d, 0xd8, 0x2a, 0x32, 0xe5, 0xf0, 0x70, 0x38, 0x0c, 0xc6, 0xd7, 0xc7, + 0x87, 0x12, 0xbd, 0x45, 0x22, 0xc6, 0x44, 0x0b, 0xa0, 0x8c, 0x7f, 0x5e, 0x11, 0x4d, 0xe0, 0x53, + 0xa8, 0x8d, 0xb1, 0x3a, 0x13, 0x4a, 0xb2, 0x15, 0x29, 0xe5, 0xfa, 0x79, 0xc5, 0x16, 0x72, 0xf6, + 0x31, 0x54, 0x27, 0xe9, 0x2c, 0x94, 0x58, 0x51, 0xc5, 0x2f, 0xea, 0xe5, 0xf3, 0x8a, 0x4d, 0x52, + 0xd4, 0xf2, 0x02, 0x67, 0xd2, 0x69, 0x16, 0x5a, 0x45, 0x19, 0x45, 0x2d, 0x94, 0xa2, 0x16, 0xa6, + 0x30, 0xa5, 0xb3, 0xd4, 0x2a, 0xaa, 0x09, 0x6a, 0xa1, 0xf4, 0x49, 0x03, 0xea, 0xb1, 0xa8, 0xc6, + 0x3f, 0x87, 0xcd, 0x12, 0xfa, 0x43, 0x37, 0x26, 0xa8, 0x84, 0xb8, 0xa3, 0x2d, 0x6b, 0xbe, 0xd9, + 0xfa, 0x5d, 0x00, 0x3a, 0x93, 0xe8, 0x60, 0xb2, 0x13, 0x6a, 0xc5, 0x45, 0xe1, 0x23, 0x68, 0xe2, + 0x59, 0x56, 0x88, 0xf1, 0x10, 0xcb, 0xc4, 0x21, 0xb4, 0xc9, 0xfb, 0xb3, 0xe1, 0x12, 0x0d, 0x76, + 0x00, 0xdb, 0xa2, 0x2f, 0xe5, 0x77, 0x5a, 0x37, 0x71, 0x03, 0x5f, 0x7e, 0x58, 0x0b, 0x65, 0x58, + 0x31, 0x39, 0x9a, 0x1b, 0x9d, 0x0d, 0xb3, 0x92, 0x9d, 0xd1, 0xd6, 0x8f, 0xa1, 0x89, 0x3b, 0x8a, + 0xed, 0xf6, 0xa0, 0x4e, 0x82, 0x0c, 0x07, 0x33, 0x87, 0x53, 0x3a, 0x64, 0x4b, 0x39, 0xc2, 0x50, + 0x34, 0xe6, 0x05, 0x07, 0xf9, 0x93, 0x0e, 0x6d, 0xb5, 0xf3, 0x7f, 0x5b, 0x49, 0xce, 0x94, 0x0b, + 0x72, 0x96, 0x87, 0x9f, 0x64, 0x79, 0xa8, 0xdc, 0x28, 0x8a, 0x98, 0x15, 0x69, 0xf8, 0x58, 0xa6, + 0x61, 0x9d, 0xd4, 0xd6, 0xb3, 0x34, 0xcc, 0xb4, 0x44, 0x16, 0x3e, 0x96, 0x59, 0xb8, 0x56, 0x28, + 0xe5, 0x01, 0xcc, 0x93, 0xf0, 0xb1, 0x4c, 0xc2, 0x46, 0xa1, 0x94, 0x83, 0x9a, 0xe7, 0xe0, 0x1a, + 0xd4, 0x08, 0x3c, 0xeb, 0xa7, 0x60, 0xaa, 0xd0, 0x50, 0x06, 0x7e, 0x22, 0x85, 0x25, 0xe0, 0xd5, + 0x9b, 0x93, 0x5c, 0xfb, 0x1a, 0xd6, 0x4b, 0x9f, 0x30, 0x16, 0x6e, 0x37, 0x1e, 0x38, 0xfe, 0x98, + 0x7b, 0xf9, 0x3d, 0x48, 0xe1, 0x28, 0x21, 0xd5, 0x0b, 0xcb, 0xd2, 0x44, 0x29, 0xa4, 0xca, 0x6d, + 0xc6, 0x28, 0xdd, 0x66, 0xfe, 0xae, 0x41, 0xfd, 0x5c, 0x04, 0xb1, 0x03, 0x6b, 0x47, 0x51, 0x34, + 0x08, 0x26, 0x22, 0x8e, 0x35, 0x3b, 0x23, 0x31, 0xc5, 0xf0, 0xaf, 0xe7, 0xc4, 0xb1, 0xbc, 0x75, + 0xe5, 0xb4, 0x94, 0x8d, 0xc6, 0x41, 0xc8, 0xe5, 0xb5, 0x2b, 0xa7, 0xa5, 0x6c, 0xc8, 0x6f, 0xb8, + 0x27, 0xaf, 0x5d, 0x39, 0x8d, 0xbb, 0x9d, 0xf0, 0x38, 0xc6, 0x04, 0x11, 0x95, 0x28, 0x23, 0x71, + 0x95, 0xed, 0xdc, 0x0e, 0x9c, 0x34, 0xe6, 0xb2, 0xe9, 0xe6, 0xb4, 0xc5, 0xa1, 0xad, 0x1e, 0x8f, + 0x7d, 0x0f, 0xaa, 0xd4, 0xd3, 0xc4, 0x83, 0x8c, 0x62, 0x43, 0x02, 0x91, 0x44, 0xf8, 0x9b, 0xa5, + 0xaf, 0x5e, 0x7c, 0x65, 0xbd, 0x2c, 0x1c, 0x06, 0x45, 0x54, 0x3c, 0xe3, 0x4a, 0x81, 0xe8, 0x42, + 0x67, 0x74, 0xeb, 0x26, 0xe3, 0x2b, 0xfa, 0x0c, 0x44, 0x9f, 0x90, 0xef, 0x20, 0xeb, 0x00, 0xb6, + 0xe4, 0xdb, 0xb0, 0xf4, 0x72, 0xfd, 0x8e, 0xf2, 0x30, 0x6c, 0xe5, 0xd7, 0x5c, 0xf1, 0x18, 0xb2, + 0x52, 0xd8, 0x2e, 0xaf, 0x91, 0x77, 0xf3, 0x55, 0x8b, 0xde, 0xc3, 0x73, 0xf2, 0x16, 0x36, 0x4f, + 0xd3, 0x68, 0x5a, 0x76, 0xb4, 0x0b, 0x0d, 0xd7, 0x77, 0xc6, 0x89, 0x7b, 0xc3, 0x65, 0x46, 0xe5, + 0x34, 0xbd, 0x33, 0x5d, 0xf9, 0x16, 0x36, 0x6c, 0xfa, 0x2f, 0xae, 0x84, 0x1e, 0xa7, 0xef, 0x3b, + 0xbf, 0x12, 0x0a, 0x9a, 0x5c, 0x11, 0x3d, 0xbd, 0x2a, 0x5d, 0x21, 0x0a, 0xf1, 0xa3, 0x97, 0x88, + 0x78, 0xa9, 0x0d, 0x02, 0xff, 0xd2, 0x9d, 0x66, 0xf8, 0xdd, 0xc2, 0xc3, 0x05, 0xb2, 0xf7, 0xf6, + 0x58, 0x51, 0x46, 0x0e, 0xd5, 0xf2, 0xc8, 0xe1, 0x2b, 0x60, 0x27, 0x77, 0xf1, 0x6b, 0x2f, 0x7b, + 0x3e, 0x0a, 0x38, 0x1e, 0x29, 0x21, 0xa0, 0x1a, 0x24, 0xc4, 0x45, 0x0c, 0xc6, 0xe4, 0x61, 0xb6, + 0xaf, 0xa0, 0xac, 0x2f, 0x60, 0xab, 0x64, 0xeb, 0x1d, 0xee, 0xcf, 0xe5, 0x60, 0xff, 0xb7, 0x50, + 0x17, 0x93, 0x03, 0xb6, 0x0e, 0xcd, 0x63, 0xff, 0xc6, 0xf1, 0xdc, 0xc9, 0xcb, 0xd0, 0xac, 0xb0, + 0x06, 0x54, 0x47, 0x49, 0x10, 0x9a, 0x1a, 0x6b, 0x42, 0xed, 0x14, 0x93, 0xde, 0xd4, 0x19, 0x40, + 0x1d, 0x2b, 0xc2, 0x8c, 0x9b, 0x06, 0xb2, 0x69, 0x28, 0x63, 0x56, 0x91, 0x2d, 0xc6, 0x05, 0x66, + 0x8d, 0x6d, 0x00, 0x7c, 0x99, 0x26, 0x81, 0x54, 0xab, 0xf7, 0xfb, 0x50, 0xa3, 0xc7, 0x38, 0x19, + 0xfc, 0xe5, 0xf1, 0xa9, 0x59, 0x61, 0x2d, 0x58, 0xb3, 0x8f, 0x4e, 0x87, 0x5f, 0x0e, 0x8e, 0x4c, + 0x0d, 0xd7, 0x1e, 0xbf, 0xf8, 0xea, 0x68, 0x70, 0x6e, 0xea, 0xfd, 0x5f, 0x91, 0xc9, 0x29, 0xc2, + 0xd9, 0x96, 0xbe, 0x10, 0x6d, 0x56, 0xd8, 0x1a, 0x18, 0x2f, 0xf8, 0xad, 0xa9, 0xd1, 0xe2, 0xd4, + 0xc7, 0x97, 0x91, 0xf0, 0x87, 0x5c, 0x9b, 0x98, 0x06, 0x0a, 0xd0, 0xe1, 0x90, 0x4f, 0xcc, 0x2a, + 0x6b, 0x43, 0xe3, 0xa9, 0x7c, 0xea, 0x98, 0xb5, 0xfe, 0x4b, 0x68, 0x64, 0x15, 0x9d, 0x3d, 0x80, + 0x96, 0x34, 0x8d, 0x2c, 0xb3, 0x82, 0xe7, 0xa0, 0xba, 0x6d, 0x6a, 0xe8, 0x22, 0xd6, 0x66, 0x53, + 0xc7, 0x7f, 0x58, 0x80, 0x4d, 0x83, 0xdc, 0xbe, 0xf3, 0xc7, 0x66, 0x15, 0x15, 0x29, 0x81, 0xcd, + 0x49, 0xff, 0x67, 0xd0, 0xcc, 0x3f, 0x6f, 0x74, 0xf6, 0x95, 0x7f, 0xed, 0x07, 0xb7, 0x3e, 0xf1, + 0xc4, 0x01, 0x8f, 0x7e, 0xc7, 0xb1, 0x9f, 0x99, 0x1a, 0x6e, 0x48, 0xf6, 0x9f, 0x52, 0xd3, 0x34, + 0xf5, 0xfe, 0x09, 0xac, 0xc9, 0xcf, 0x8b, 0x31, 0xd8, 0x90, 0xce, 0x48, 0x8e, 0x59, 0xc1, 0x38, + 0xe0, 0x39, 0xc4, 0x56, 0x1a, 0xe2, 0x49, 0x47, 0x14, 0xb4, 0x8e, 0xe6, 0x04, 0xb6, 0x82, 0x61, + 0xf4, 0x6d, 0x68, 0x64, 0xa9, 0xc2, 0xb6, 0xe0, 0x81, 0xb4, 0x97, 0xb1, 0xcc, 0x0a, 0xae, 0xa0, + 0x40, 0x09, 0x96, 0xa9, 0x91, 0xc3, 0x14, 0x2e, 0x91, 0xf4, 0xa6, 0x8e, 0x9b, 0xe0, 0x9e, 0x52, + 0xc3, 0x38, 0xf8, 0xdb, 0x1a, 0xd4, 0x05, 0xc1, 0x06, 0xd0, 0x56, 0x67, 0x6f, 0xec, 0x43, 0xd9, + 0x3f, 0xef, 0x4f, 0xe3, 0xba, 0x1d, 0xea, 0x80, 0x0b, 0x06, 0x23, 0x56, 0x85, 0x1d, 0xc3, 0x46, + 0x79, 0x8e, 0xc5, 0x1e, 0xa2, 0xf6, 0xc2, 0x21, 0x59, 0xb7, 0xbb, 0x48, 0x94, 0x9b, 0x3a, 0x82, + 0xf5, 0xd2, 0x68, 0x8a, 0xd1, 0xbe, 0x8b, 0xa6, 0x55, 0x2b, 0x3d, 0xfa, 0x05, 0xb4, 0x94, 0x49, + 0x0b, 0xdb, 0x41, 0xd5, 0xf9, 0x31, 0x56, 0xf7, 0xc3, 0x39, 0x7e, 0x6e, 0xe1, 0x73, 0x80, 0x62, + 0xca, 0xc1, 0x3e, 0xc8, 0x15, 0xd5, 0xe9, 0x56, 0x77, 0xe7, 0x3e, 0x3b, 0x5f, 0xfe, 0x14, 0x40, + 0x8e, 0xb8, 0xce, 0x86, 0x31, 0x7b, 0x84, 0x7a, 0xcb, 0x46, 0x5e, 0x2b, 0x0f, 0x72, 0x02, 0x9b, + 0x73, 0x2d, 0x42, 0x98, 0x5b, 0xd6, 0x39, 0x56, 0x9a, 0x1b, 0x40, 0x5b, 0xed, 0x10, 0x22, 0xdc, + 0x0b, 0xfa, 0x8c, 0x30, 0xb2, 0xa8, 0x99, 0x58, 0x15, 0xf6, 0x05, 0x40, 0x51, 0xef, 0x05, 0x34, + 0x73, 0xf5, 0x7f, 0xa5, 0x17, 0xcf, 0x60, 0x53, 0x99, 0xe2, 0x8a, 0x34, 0x15, 0x31, 0x9a, 0x1f, + 0xee, 0xae, 0x34, 0x64, 0xcb, 0x91, 0xa3, 0x5a, 0xe4, 0x05, 0x3a, 0xcb, 0xfa, 0x42, 0xf7, 0xa3, + 0x25, 0x52, 0x15, 0x22, 0x75, 0x64, 0x2c, 0x20, 0x5a, 0x30, 0x44, 0x7e, 0xc7, 0x09, 0x99, 0x04, + 0x4f, 0xa9, 0xdf, 0xe2, 0x88, 0xf3, 0xcd, 0x41, 0xa4, 0xe1, 0x82, 0x42, 0x6f, 0x55, 0x9e, 0x74, + 0xfe, 0xfd, 0x66, 0x57, 0xfb, 0xfa, 0xcd, 0xae, 0xf6, 0xdf, 0x37, 0xbb, 0xda, 0x1f, 0xdf, 0xee, + 0x56, 0xbe, 0x7e, 0xbb, 0x5b, 0xf9, 0xcf, 0xdb, 0xdd, 0xca, 0x45, 0x9d, 0x06, 0xe8, 0x3f, 0xfc, + 0x7f, 0x00, 0x00, 0x00, 0xff, 0xff, 0x79, 0x63, 0x20, 0x05, 0x52, 0x17, 0x00, 0x00, } // Reference imports to suppress errors if they are not otherwise used. @@ -3319,15 +3024,6 @@ type WorkerClient interface { QueryStatus(ctx context.Context, in *QueryStatusRequest, opts ...grpc.CallOption) (*QueryStatusResponse, error) QueryError(ctx context.Context, in *QueryErrorRequest, opts ...grpc.CallOption) (*QueryErrorResponse, error) HandleSQLs(ctx context.Context, in *HandleSubTaskSQLsRequest, opts ...grpc.CallOption) (*CommonWorkerResponse, error) - // FetchDDLInfo fetches DDL info from dm-worker by dm-master - // and sends the relevant DDL lock info back to dm-worker - // NOTE: split to two RPCs if needed - FetchDDLInfo(ctx context.Context, opts ...grpc.CallOption) (Worker_FetchDDLInfoClient, error) - // ExecuteDDL request a dm-worker to execute or ignore (skip) a DDL - ExecuteDDL(ctx context.Context, in *ExecDDLRequest, opts ...grpc.CallOption) (*CommonWorkerResponse, error) - // BreakDDLLock request a dm-worker to break a DDL lock - // including remove DDLLockInfo and/or execute/skip DDL - BreakDDLLock(ctx context.Context, in *BreakDDLLockRequest, opts ...grpc.CallOption) (*CommonWorkerResponse, error) // SwitchRelayMaster request a dm-worker's relay unit switching to another master server SwitchRelayMaster(ctx context.Context, in *SwitchRelayMasterRequest, opts ...grpc.CallOption) (*CommonWorkerResponse, error) // OperateRelay operates relay unit @@ -3403,55 +3099,6 @@ func (c *workerClient) HandleSQLs(ctx context.Context, in *HandleSubTaskSQLsRequ return out, nil } -func (c *workerClient) FetchDDLInfo(ctx context.Context, opts ...grpc.CallOption) (Worker_FetchDDLInfoClient, error) { - stream, err := c.cc.NewStream(ctx, &_Worker_serviceDesc.Streams[0], "/pb.Worker/FetchDDLInfo", opts...) - if err != nil { - return nil, err - } - x := &workerFetchDDLInfoClient{stream} - return x, nil -} - -type Worker_FetchDDLInfoClient interface { - Send(*DDLLockInfo) error - Recv() (*DDLInfo, error) - grpc.ClientStream -} - -type workerFetchDDLInfoClient struct { - grpc.ClientStream -} - -func (x *workerFetchDDLInfoClient) Send(m *DDLLockInfo) error { - return x.ClientStream.SendMsg(m) -} - -func (x *workerFetchDDLInfoClient) Recv() (*DDLInfo, error) { - m := new(DDLInfo) - if err := x.ClientStream.RecvMsg(m); err != nil { - return nil, err - } - return m, nil -} - -func (c *workerClient) ExecuteDDL(ctx context.Context, in *ExecDDLRequest, opts ...grpc.CallOption) (*CommonWorkerResponse, error) { - out := new(CommonWorkerResponse) - err := c.cc.Invoke(ctx, "/pb.Worker/ExecuteDDL", in, out, opts...) - if err != nil { - return nil, err - } - return out, nil -} - -func (c *workerClient) BreakDDLLock(ctx context.Context, in *BreakDDLLockRequest, opts ...grpc.CallOption) (*CommonWorkerResponse, error) { - out := new(CommonWorkerResponse) - err := c.cc.Invoke(ctx, "/pb.Worker/BreakDDLLock", in, out, opts...) - if err != nil { - return nil, err - } - return out, nil -} - func (c *workerClient) SwitchRelayMaster(ctx context.Context, in *SwitchRelayMasterRequest, opts ...grpc.CallOption) (*CommonWorkerResponse, error) { out := new(CommonWorkerResponse) err := c.cc.Invoke(ctx, "/pb.Worker/SwitchRelayMaster", in, out, opts...) @@ -3523,15 +3170,6 @@ type WorkerServer interface { QueryStatus(context.Context, *QueryStatusRequest) (*QueryStatusResponse, error) QueryError(context.Context, *QueryErrorRequest) (*QueryErrorResponse, error) HandleSQLs(context.Context, *HandleSubTaskSQLsRequest) (*CommonWorkerResponse, error) - // FetchDDLInfo fetches DDL info from dm-worker by dm-master - // and sends the relevant DDL lock info back to dm-worker - // NOTE: split to two RPCs if needed - FetchDDLInfo(Worker_FetchDDLInfoServer) error - // ExecuteDDL request a dm-worker to execute or ignore (skip) a DDL - ExecuteDDL(context.Context, *ExecDDLRequest) (*CommonWorkerResponse, error) - // BreakDDLLock request a dm-worker to break a DDL lock - // including remove DDLLockInfo and/or execute/skip DDL - BreakDDLLock(context.Context, *BreakDDLLockRequest) (*CommonWorkerResponse, error) // SwitchRelayMaster request a dm-worker's relay unit switching to another master server SwitchRelayMaster(context.Context, *SwitchRelayMasterRequest) (*CommonWorkerResponse, error) // OperateRelay operates relay unit @@ -3567,15 +3205,6 @@ func (*UnimplementedWorkerServer) QueryError(ctx context.Context, req *QueryErro func (*UnimplementedWorkerServer) HandleSQLs(ctx context.Context, req *HandleSubTaskSQLsRequest) (*CommonWorkerResponse, error) { return nil, status.Errorf(codes.Unimplemented, "method HandleSQLs not implemented") } -func (*UnimplementedWorkerServer) FetchDDLInfo(srv Worker_FetchDDLInfoServer) error { - return status.Errorf(codes.Unimplemented, "method FetchDDLInfo not implemented") -} -func (*UnimplementedWorkerServer) ExecuteDDL(ctx context.Context, req *ExecDDLRequest) (*CommonWorkerResponse, error) { - return nil, status.Errorf(codes.Unimplemented, "method ExecuteDDL not implemented") -} -func (*UnimplementedWorkerServer) BreakDDLLock(ctx context.Context, req *BreakDDLLockRequest) (*CommonWorkerResponse, error) { - return nil, status.Errorf(codes.Unimplemented, "method BreakDDLLock not implemented") -} func (*UnimplementedWorkerServer) SwitchRelayMaster(ctx context.Context, req *SwitchRelayMasterRequest) (*CommonWorkerResponse, error) { return nil, status.Errorf(codes.Unimplemented, "method SwitchRelayMaster not implemented") } @@ -3710,68 +3339,6 @@ func _Worker_HandleSQLs_Handler(srv interface{}, ctx context.Context, dec func(i return interceptor(ctx, in, info, handler) } -func _Worker_FetchDDLInfo_Handler(srv interface{}, stream grpc.ServerStream) error { - return srv.(WorkerServer).FetchDDLInfo(&workerFetchDDLInfoServer{stream}) -} - -type Worker_FetchDDLInfoServer interface { - Send(*DDLInfo) error - Recv() (*DDLLockInfo, error) - grpc.ServerStream -} - -type workerFetchDDLInfoServer struct { - grpc.ServerStream -} - -func (x *workerFetchDDLInfoServer) Send(m *DDLInfo) error { - return x.ServerStream.SendMsg(m) -} - -func (x *workerFetchDDLInfoServer) Recv() (*DDLLockInfo, error) { - m := new(DDLLockInfo) - if err := x.ServerStream.RecvMsg(m); err != nil { - return nil, err - } - return m, nil -} - -func _Worker_ExecuteDDL_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { - in := new(ExecDDLRequest) - if err := dec(in); err != nil { - return nil, err - } - if interceptor == nil { - return srv.(WorkerServer).ExecuteDDL(ctx, in) - } - info := &grpc.UnaryServerInfo{ - Server: srv, - FullMethod: "/pb.Worker/ExecuteDDL", - } - handler := func(ctx context.Context, req interface{}) (interface{}, error) { - return srv.(WorkerServer).ExecuteDDL(ctx, req.(*ExecDDLRequest)) - } - return interceptor(ctx, in, info, handler) -} - -func _Worker_BreakDDLLock_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { - in := new(BreakDDLLockRequest) - if err := dec(in); err != nil { - return nil, err - } - if interceptor == nil { - return srv.(WorkerServer).BreakDDLLock(ctx, in) - } - info := &grpc.UnaryServerInfo{ - Server: srv, - FullMethod: "/pb.Worker/BreakDDLLock", - } - handler := func(ctx context.Context, req interface{}) (interface{}, error) { - return srv.(WorkerServer).BreakDDLLock(ctx, req.(*BreakDDLLockRequest)) - } - return interceptor(ctx, in, info, handler) -} - func _Worker_SwitchRelayMaster_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { in := new(SwitchRelayMasterRequest) if err := dec(in); err != nil { @@ -3926,14 +3493,6 @@ var _Worker_serviceDesc = grpc.ServiceDesc{ MethodName: "HandleSQLs", Handler: _Worker_HandleSQLs_Handler, }, - { - MethodName: "ExecuteDDL", - Handler: _Worker_ExecuteDDL_Handler, - }, - { - MethodName: "BreakDDLLock", - Handler: _Worker_BreakDDLLock_Handler, - }, { MethodName: "SwitchRelayMaster", Handler: _Worker_SwitchRelayMaster_Handler, @@ -3963,14 +3522,7 @@ var _Worker_serviceDesc = grpc.ServiceDesc{ Handler: _Worker_OperateMysqlWorker_Handler, }, }, - Streams: []grpc.StreamDesc{ - { - StreamName: "FetchDDLInfo", - Handler: _Worker_FetchDDLInfo_Handler, - ServerStreams: true, - ClientStreams: true, - }, - }, + Streams: []grpc.StreamDesc{}, Metadata: "dmworker.proto", } @@ -5628,216 +5180,6 @@ func (m *ProcessError) MarshalToSizedBuffer(dAtA []byte) (int, error) { return len(dAtA) - i, nil } -func (m *DDLInfo) 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 *DDLInfo) MarshalTo(dAtA []byte) (int, error) { - size := m.Size() - return m.MarshalToSizedBuffer(dAtA[:size]) -} - -func (m *DDLInfo) MarshalToSizedBuffer(dAtA []byte) (int, error) { - i := len(dAtA) - _ = i - var l int - _ = l - if len(m.DDLs) > 0 { - for iNdEx := len(m.DDLs) - 1; iNdEx >= 0; iNdEx-- { - i -= len(m.DDLs[iNdEx]) - copy(dAtA[i:], m.DDLs[iNdEx]) - i = encodeVarintDmworker(dAtA, i, uint64(len(m.DDLs[iNdEx]))) - i-- - dAtA[i] = 0x22 - } - } - if len(m.Table) > 0 { - i -= len(m.Table) - copy(dAtA[i:], m.Table) - i = encodeVarintDmworker(dAtA, i, uint64(len(m.Table))) - i-- - dAtA[i] = 0x1a - } - if len(m.Schema) > 0 { - i -= len(m.Schema) - copy(dAtA[i:], m.Schema) - i = encodeVarintDmworker(dAtA, i, uint64(len(m.Schema))) - i-- - dAtA[i] = 0x12 - } - if len(m.Task) > 0 { - i -= len(m.Task) - copy(dAtA[i:], m.Task) - i = encodeVarintDmworker(dAtA, i, uint64(len(m.Task))) - i-- - dAtA[i] = 0xa - } - return len(dAtA) - i, nil -} - -func (m *DDLLockInfo) 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 *DDLLockInfo) MarshalTo(dAtA []byte) (int, error) { - size := m.Size() - return m.MarshalToSizedBuffer(dAtA[:size]) -} - -func (m *DDLLockInfo) MarshalToSizedBuffer(dAtA []byte) (int, error) { - i := len(dAtA) - _ = i - var l int - _ = l - if len(m.ID) > 0 { - i -= len(m.ID) - copy(dAtA[i:], m.ID) - i = encodeVarintDmworker(dAtA, i, uint64(len(m.ID))) - i-- - dAtA[i] = 0x12 - } - if len(m.Task) > 0 { - i -= len(m.Task) - copy(dAtA[i:], m.Task) - i = encodeVarintDmworker(dAtA, i, uint64(len(m.Task))) - i-- - dAtA[i] = 0xa - } - return len(dAtA) - i, nil -} - -func (m *ExecDDLRequest) 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 *ExecDDLRequest) MarshalTo(dAtA []byte) (int, error) { - size := m.Size() - return m.MarshalToSizedBuffer(dAtA[:size]) -} - -func (m *ExecDDLRequest) MarshalToSizedBuffer(dAtA []byte) (int, error) { - i := len(dAtA) - _ = i - var l int - _ = l - if len(m.DDLs) > 0 { - for iNdEx := len(m.DDLs) - 1; iNdEx >= 0; iNdEx-- { - i -= len(m.DDLs[iNdEx]) - copy(dAtA[i:], m.DDLs[iNdEx]) - i = encodeVarintDmworker(dAtA, i, uint64(len(m.DDLs[iNdEx]))) - i-- - dAtA[i] = 0x2a - } - } - if len(m.TraceGID) > 0 { - i -= len(m.TraceGID) - copy(dAtA[i:], m.TraceGID) - i = encodeVarintDmworker(dAtA, i, uint64(len(m.TraceGID))) - i-- - dAtA[i] = 0x22 - } - if m.Exec { - i-- - if m.Exec { - dAtA[i] = 1 - } else { - dAtA[i] = 0 - } - i-- - dAtA[i] = 0x18 - } - if len(m.LockID) > 0 { - i -= len(m.LockID) - copy(dAtA[i:], m.LockID) - i = encodeVarintDmworker(dAtA, i, uint64(len(m.LockID))) - i-- - dAtA[i] = 0x12 - } - if len(m.Task) > 0 { - i -= len(m.Task) - copy(dAtA[i:], m.Task) - i = encodeVarintDmworker(dAtA, i, uint64(len(m.Task))) - i-- - dAtA[i] = 0xa - } - return len(dAtA) - i, nil -} - -func (m *BreakDDLLockRequest) 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 *BreakDDLLockRequest) MarshalTo(dAtA []byte) (int, error) { - size := m.Size() - return m.MarshalToSizedBuffer(dAtA[:size]) -} - -func (m *BreakDDLLockRequest) MarshalToSizedBuffer(dAtA []byte) (int, error) { - i := len(dAtA) - _ = i - var l int - _ = l - if m.SkipDDL { - i-- - if m.SkipDDL { - dAtA[i] = 1 - } else { - dAtA[i] = 0 - } - i-- - dAtA[i] = 0x20 - } - if m.ExecDDL { - i-- - if m.ExecDDL { - dAtA[i] = 1 - } else { - dAtA[i] = 0 - } - i-- - dAtA[i] = 0x18 - } - if len(m.RemoveLockID) > 0 { - i -= len(m.RemoveLockID) - copy(dAtA[i:], m.RemoveLockID) - i = encodeVarintDmworker(dAtA, i, uint64(len(m.RemoveLockID))) - i-- - dAtA[i] = 0x12 - } - if len(m.Task) > 0 { - i -= len(m.Task) - copy(dAtA[i:], m.Task) - i = encodeVarintDmworker(dAtA, i, uint64(len(m.Task))) - i-- - dAtA[i] = 0xa - } - return len(dAtA) - i, nil -} - func (m *SwitchRelayMasterRequest) Marshal() (dAtA []byte, err error) { size := m.Size() dAtA = make([]byte, size) @@ -6930,103 +6272,6 @@ func (m *ProcessError) Size() (n int) { return n } -func (m *DDLInfo) Size() (n int) { - if m == nil { - return 0 - } - var l int - _ = l - l = len(m.Task) - if l > 0 { - n += 1 + l + sovDmworker(uint64(l)) - } - l = len(m.Schema) - if l > 0 { - n += 1 + l + sovDmworker(uint64(l)) - } - l = len(m.Table) - if l > 0 { - n += 1 + l + sovDmworker(uint64(l)) - } - if len(m.DDLs) > 0 { - for _, s := range m.DDLs { - l = len(s) - n += 1 + l + sovDmworker(uint64(l)) - } - } - return n -} - -func (m *DDLLockInfo) Size() (n int) { - if m == nil { - return 0 - } - var l int - _ = l - l = len(m.Task) - if l > 0 { - n += 1 + l + sovDmworker(uint64(l)) - } - l = len(m.ID) - if l > 0 { - n += 1 + l + sovDmworker(uint64(l)) - } - return n -} - -func (m *ExecDDLRequest) Size() (n int) { - if m == nil { - return 0 - } - var l int - _ = l - l = len(m.Task) - if l > 0 { - n += 1 + l + sovDmworker(uint64(l)) - } - l = len(m.LockID) - if l > 0 { - n += 1 + l + sovDmworker(uint64(l)) - } - if m.Exec { - n += 2 - } - l = len(m.TraceGID) - if l > 0 { - n += 1 + l + sovDmworker(uint64(l)) - } - if len(m.DDLs) > 0 { - for _, s := range m.DDLs { - l = len(s) - n += 1 + l + sovDmworker(uint64(l)) - } - } - return n -} - -func (m *BreakDDLLockRequest) Size() (n int) { - if m == nil { - return 0 - } - var l int - _ = l - l = len(m.Task) - if l > 0 { - n += 1 + l + sovDmworker(uint64(l)) - } - l = len(m.RemoveLockID) - if l > 0 { - n += 1 + l + sovDmworker(uint64(l)) - } - if m.ExecDDL { - n += 2 - } - if m.SkipDDL { - n += 2 - } - return n -} - func (m *SwitchRelayMasterRequest) Size() (n int) { if m == nil { return 0 @@ -11835,662 +11080,6 @@ func (m *ProcessError) Unmarshal(dAtA []byte) error { } return nil } -func (m *DDLInfo) 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 ErrIntOverflowDmworker - } - 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: DDLInfo: wiretype end group for non-group") - } - if fieldNum <= 0 { - return fmt.Errorf("proto: DDLInfo: illegal tag %d (wire type %d)", fieldNum, wire) - } - switch fieldNum { - case 1: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Task", wireType) - } - var stringLen uint64 - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowDmworker - } - 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 ErrInvalidLengthDmworker - } - postIndex := iNdEx + intStringLen - if postIndex < 0 { - return ErrInvalidLengthDmworker - } - if postIndex > l { - return io.ErrUnexpectedEOF - } - m.Task = string(dAtA[iNdEx:postIndex]) - iNdEx = postIndex - case 2: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Schema", wireType) - } - var stringLen uint64 - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowDmworker - } - 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 ErrInvalidLengthDmworker - } - postIndex := iNdEx + intStringLen - if postIndex < 0 { - return ErrInvalidLengthDmworker - } - if postIndex > l { - return io.ErrUnexpectedEOF - } - m.Schema = string(dAtA[iNdEx:postIndex]) - iNdEx = postIndex - case 3: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Table", wireType) - } - var stringLen uint64 - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowDmworker - } - 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 ErrInvalidLengthDmworker - } - postIndex := iNdEx + intStringLen - if postIndex < 0 { - return ErrInvalidLengthDmworker - } - if postIndex > l { - return io.ErrUnexpectedEOF - } - m.Table = string(dAtA[iNdEx:postIndex]) - iNdEx = postIndex - case 4: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field DDLs", wireType) - } - var stringLen uint64 - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowDmworker - } - 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 ErrInvalidLengthDmworker - } - postIndex := iNdEx + intStringLen - if postIndex < 0 { - return ErrInvalidLengthDmworker - } - if postIndex > l { - return io.ErrUnexpectedEOF - } - m.DDLs = append(m.DDLs, string(dAtA[iNdEx:postIndex])) - iNdEx = postIndex - default: - iNdEx = preIndex - skippy, err := skipDmworker(dAtA[iNdEx:]) - if err != nil { - return err - } - if skippy < 0 { - return ErrInvalidLengthDmworker - } - if (iNdEx + skippy) < 0 { - return ErrInvalidLengthDmworker - } - if (iNdEx + skippy) > l { - return io.ErrUnexpectedEOF - } - iNdEx += skippy - } - } - - if iNdEx > l { - return io.ErrUnexpectedEOF - } - return nil -} -func (m *DDLLockInfo) 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 ErrIntOverflowDmworker - } - 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: DDLLockInfo: wiretype end group for non-group") - } - if fieldNum <= 0 { - return fmt.Errorf("proto: DDLLockInfo: illegal tag %d (wire type %d)", fieldNum, wire) - } - switch fieldNum { - case 1: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Task", wireType) - } - var stringLen uint64 - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowDmworker - } - 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 ErrInvalidLengthDmworker - } - postIndex := iNdEx + intStringLen - if postIndex < 0 { - return ErrInvalidLengthDmworker - } - if postIndex > l { - return io.ErrUnexpectedEOF - } - m.Task = string(dAtA[iNdEx:postIndex]) - iNdEx = postIndex - case 2: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field ID", wireType) - } - var stringLen uint64 - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowDmworker - } - 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 ErrInvalidLengthDmworker - } - postIndex := iNdEx + intStringLen - if postIndex < 0 { - return ErrInvalidLengthDmworker - } - if postIndex > l { - return io.ErrUnexpectedEOF - } - m.ID = string(dAtA[iNdEx:postIndex]) - iNdEx = postIndex - default: - iNdEx = preIndex - skippy, err := skipDmworker(dAtA[iNdEx:]) - if err != nil { - return err - } - if skippy < 0 { - return ErrInvalidLengthDmworker - } - if (iNdEx + skippy) < 0 { - return ErrInvalidLengthDmworker - } - if (iNdEx + skippy) > l { - return io.ErrUnexpectedEOF - } - iNdEx += skippy - } - } - - if iNdEx > l { - return io.ErrUnexpectedEOF - } - return nil -} -func (m *ExecDDLRequest) 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 ErrIntOverflowDmworker - } - 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: ExecDDLRequest: wiretype end group for non-group") - } - if fieldNum <= 0 { - return fmt.Errorf("proto: ExecDDLRequest: illegal tag %d (wire type %d)", fieldNum, wire) - } - switch fieldNum { - case 1: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Task", wireType) - } - var stringLen uint64 - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowDmworker - } - 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 ErrInvalidLengthDmworker - } - postIndex := iNdEx + intStringLen - if postIndex < 0 { - return ErrInvalidLengthDmworker - } - if postIndex > l { - return io.ErrUnexpectedEOF - } - m.Task = string(dAtA[iNdEx:postIndex]) - iNdEx = postIndex - case 2: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field LockID", wireType) - } - var stringLen uint64 - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowDmworker - } - 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 ErrInvalidLengthDmworker - } - postIndex := iNdEx + intStringLen - if postIndex < 0 { - return ErrInvalidLengthDmworker - } - if postIndex > l { - return io.ErrUnexpectedEOF - } - m.LockID = string(dAtA[iNdEx:postIndex]) - iNdEx = postIndex - case 3: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field Exec", wireType) - } - var v int - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowDmworker - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - v |= int(b&0x7F) << shift - if b < 0x80 { - break - } - } - m.Exec = bool(v != 0) - case 4: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field TraceGID", wireType) - } - var stringLen uint64 - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowDmworker - } - 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 ErrInvalidLengthDmworker - } - postIndex := iNdEx + intStringLen - if postIndex < 0 { - return ErrInvalidLengthDmworker - } - if postIndex > l { - return io.ErrUnexpectedEOF - } - m.TraceGID = string(dAtA[iNdEx:postIndex]) - iNdEx = postIndex - case 5: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field DDLs", wireType) - } - var stringLen uint64 - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowDmworker - } - 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 ErrInvalidLengthDmworker - } - postIndex := iNdEx + intStringLen - if postIndex < 0 { - return ErrInvalidLengthDmworker - } - if postIndex > l { - return io.ErrUnexpectedEOF - } - m.DDLs = append(m.DDLs, string(dAtA[iNdEx:postIndex])) - iNdEx = postIndex - default: - iNdEx = preIndex - skippy, err := skipDmworker(dAtA[iNdEx:]) - if err != nil { - return err - } - if skippy < 0 { - return ErrInvalidLengthDmworker - } - if (iNdEx + skippy) < 0 { - return ErrInvalidLengthDmworker - } - if (iNdEx + skippy) > l { - return io.ErrUnexpectedEOF - } - iNdEx += skippy - } - } - - if iNdEx > l { - return io.ErrUnexpectedEOF - } - return nil -} -func (m *BreakDDLLockRequest) 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 ErrIntOverflowDmworker - } - 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: BreakDDLLockRequest: wiretype end group for non-group") - } - if fieldNum <= 0 { - return fmt.Errorf("proto: BreakDDLLockRequest: illegal tag %d (wire type %d)", fieldNum, wire) - } - switch fieldNum { - case 1: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Task", wireType) - } - var stringLen uint64 - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowDmworker - } - 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 ErrInvalidLengthDmworker - } - postIndex := iNdEx + intStringLen - if postIndex < 0 { - return ErrInvalidLengthDmworker - } - if postIndex > l { - return io.ErrUnexpectedEOF - } - m.Task = string(dAtA[iNdEx:postIndex]) - iNdEx = postIndex - case 2: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field RemoveLockID", wireType) - } - var stringLen uint64 - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowDmworker - } - 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 ErrInvalidLengthDmworker - } - postIndex := iNdEx + intStringLen - if postIndex < 0 { - return ErrInvalidLengthDmworker - } - if postIndex > l { - return io.ErrUnexpectedEOF - } - m.RemoveLockID = string(dAtA[iNdEx:postIndex]) - iNdEx = postIndex - case 3: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field ExecDDL", wireType) - } - var v int - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowDmworker - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - v |= int(b&0x7F) << shift - if b < 0x80 { - break - } - } - m.ExecDDL = bool(v != 0) - case 4: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field SkipDDL", wireType) - } - var v int - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowDmworker - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - v |= int(b&0x7F) << shift - if b < 0x80 { - break - } - } - m.SkipDDL = bool(v != 0) - default: - iNdEx = preIndex - skippy, err := skipDmworker(dAtA[iNdEx:]) - if err != nil { - return err - } - if skippy < 0 { - return ErrInvalidLengthDmworker - } - if (iNdEx + skippy) < 0 { - return ErrInvalidLengthDmworker - } - if (iNdEx + skippy) > l { - return io.ErrUnexpectedEOF - } - iNdEx += skippy - } - } - - if iNdEx > l { - return io.ErrUnexpectedEOF - } - return nil -} func (m *SwitchRelayMasterRequest) Unmarshal(dAtA []byte) error { l := len(dAtA) iNdEx := 0 diff --git a/dm/pbmock/dmworker.go b/dm/pbmock/dmworker.go index 8af632f3c6..caa5087a85 100644 --- a/dm/pbmock/dmworker.go +++ b/dm/pbmock/dmworker.go @@ -1,5 +1,5 @@ // Code generated by MockGen. DO NOT EDIT. -// Source: github.com/pingcap/dm/dm/pb (interfaces: WorkerClient,Worker_FetchDDLInfoClient,WorkerServer,Worker_FetchDDLInfoServer) +// Source: github.com/pingcap/dm/dm/pb (interfaces: WorkerClient,WorkerServer) // Package pbmock is a generated GoMock package. package pbmock @@ -9,7 +9,6 @@ import ( gomock "github.com/golang/mock/gomock" pb "github.com/pingcap/dm/dm/pb" grpc "google.golang.org/grpc" - metadata "google.golang.org/grpc/metadata" reflect "reflect" ) @@ -36,66 +35,6 @@ func (m *MockWorkerClient) EXPECT() *MockWorkerClientMockRecorder { return m.recorder } -// BreakDDLLock mocks base method -func (m *MockWorkerClient) BreakDDLLock(arg0 context.Context, arg1 *pb.BreakDDLLockRequest, arg2 ...grpc.CallOption) (*pb.CommonWorkerResponse, error) { - m.ctrl.T.Helper() - varargs := []interface{}{arg0, arg1} - for _, a := range arg2 { - varargs = append(varargs, a) - } - ret := m.ctrl.Call(m, "BreakDDLLock", varargs...) - ret0, _ := ret[0].(*pb.CommonWorkerResponse) - ret1, _ := ret[1].(error) - return ret0, ret1 -} - -// BreakDDLLock indicates an expected call of BreakDDLLock -func (mr *MockWorkerClientMockRecorder) BreakDDLLock(arg0, arg1 interface{}, arg2 ...interface{}) *gomock.Call { - mr.mock.ctrl.T.Helper() - varargs := append([]interface{}{arg0, arg1}, arg2...) - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "BreakDDLLock", reflect.TypeOf((*MockWorkerClient)(nil).BreakDDLLock), varargs...) -} - -// ExecuteDDL mocks base method -func (m *MockWorkerClient) ExecuteDDL(arg0 context.Context, arg1 *pb.ExecDDLRequest, arg2 ...grpc.CallOption) (*pb.CommonWorkerResponse, error) { - m.ctrl.T.Helper() - varargs := []interface{}{arg0, arg1} - for _, a := range arg2 { - varargs = append(varargs, a) - } - ret := m.ctrl.Call(m, "ExecuteDDL", varargs...) - ret0, _ := ret[0].(*pb.CommonWorkerResponse) - ret1, _ := ret[1].(error) - return ret0, ret1 -} - -// ExecuteDDL indicates an expected call of ExecuteDDL -func (mr *MockWorkerClientMockRecorder) ExecuteDDL(arg0, arg1 interface{}, arg2 ...interface{}) *gomock.Call { - mr.mock.ctrl.T.Helper() - varargs := append([]interface{}{arg0, arg1}, arg2...) - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "ExecuteDDL", reflect.TypeOf((*MockWorkerClient)(nil).ExecuteDDL), varargs...) -} - -// FetchDDLInfo mocks base method -func (m *MockWorkerClient) FetchDDLInfo(arg0 context.Context, arg1 ...grpc.CallOption) (pb.Worker_FetchDDLInfoClient, error) { - m.ctrl.T.Helper() - varargs := []interface{}{arg0} - for _, a := range arg1 { - varargs = append(varargs, a) - } - ret := m.ctrl.Call(m, "FetchDDLInfo", varargs...) - ret0, _ := ret[0].(pb.Worker_FetchDDLInfoClient) - ret1, _ := ret[1].(error) - return ret0, ret1 -} - -// FetchDDLInfo indicates an expected call of FetchDDLInfo -func (mr *MockWorkerClientMockRecorder) FetchDDLInfo(arg0 interface{}, arg1 ...interface{}) *gomock.Call { - mr.mock.ctrl.T.Helper() - varargs := append([]interface{}{arg0}, arg1...) - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "FetchDDLInfo", reflect.TypeOf((*MockWorkerClient)(nil).FetchDDLInfo), varargs...) -} - // HandleSQLs mocks base method func (m *MockWorkerClient) HandleSQLs(arg0 context.Context, arg1 *pb.HandleSubTaskSQLsRequest, arg2 ...grpc.CallOption) (*pb.CommonWorkerResponse, error) { m.ctrl.T.Helper() @@ -356,143 +295,6 @@ func (mr *MockWorkerClientMockRecorder) UpdateSubTask(arg0, arg1 interface{}, ar return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "UpdateSubTask", reflect.TypeOf((*MockWorkerClient)(nil).UpdateSubTask), varargs...) } -// MockWorker_FetchDDLInfoClient is a mock of Worker_FetchDDLInfoClient interface -type MockWorker_FetchDDLInfoClient struct { - ctrl *gomock.Controller - recorder *MockWorker_FetchDDLInfoClientMockRecorder -} - -// MockWorker_FetchDDLInfoClientMockRecorder is the mock recorder for MockWorker_FetchDDLInfoClient -type MockWorker_FetchDDLInfoClientMockRecorder struct { - mock *MockWorker_FetchDDLInfoClient -} - -// NewMockWorker_FetchDDLInfoClient creates a new mock instance -func NewMockWorker_FetchDDLInfoClient(ctrl *gomock.Controller) *MockWorker_FetchDDLInfoClient { - mock := &MockWorker_FetchDDLInfoClient{ctrl: ctrl} - mock.recorder = &MockWorker_FetchDDLInfoClientMockRecorder{mock} - return mock -} - -// EXPECT returns an object that allows the caller to indicate expected use -func (m *MockWorker_FetchDDLInfoClient) EXPECT() *MockWorker_FetchDDLInfoClientMockRecorder { - return m.recorder -} - -// CloseSend mocks base method -func (m *MockWorker_FetchDDLInfoClient) CloseSend() error { - m.ctrl.T.Helper() - ret := m.ctrl.Call(m, "CloseSend") - ret0, _ := ret[0].(error) - return ret0 -} - -// CloseSend indicates an expected call of CloseSend -func (mr *MockWorker_FetchDDLInfoClientMockRecorder) CloseSend() *gomock.Call { - mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "CloseSend", reflect.TypeOf((*MockWorker_FetchDDLInfoClient)(nil).CloseSend)) -} - -// Context mocks base method -func (m *MockWorker_FetchDDLInfoClient) Context() context.Context { - m.ctrl.T.Helper() - ret := m.ctrl.Call(m, "Context") - ret0, _ := ret[0].(context.Context) - return ret0 -} - -// Context indicates an expected call of Context -func (mr *MockWorker_FetchDDLInfoClientMockRecorder) Context() *gomock.Call { - mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Context", reflect.TypeOf((*MockWorker_FetchDDLInfoClient)(nil).Context)) -} - -// Header mocks base method -func (m *MockWorker_FetchDDLInfoClient) Header() (metadata.MD, error) { - m.ctrl.T.Helper() - ret := m.ctrl.Call(m, "Header") - ret0, _ := ret[0].(metadata.MD) - ret1, _ := ret[1].(error) - return ret0, ret1 -} - -// Header indicates an expected call of Header -func (mr *MockWorker_FetchDDLInfoClientMockRecorder) Header() *gomock.Call { - mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Header", reflect.TypeOf((*MockWorker_FetchDDLInfoClient)(nil).Header)) -} - -// Recv mocks base method -func (m *MockWorker_FetchDDLInfoClient) Recv() (*pb.DDLInfo, error) { - m.ctrl.T.Helper() - ret := m.ctrl.Call(m, "Recv") - ret0, _ := ret[0].(*pb.DDLInfo) - ret1, _ := ret[1].(error) - return ret0, ret1 -} - -// Recv indicates an expected call of Recv -func (mr *MockWorker_FetchDDLInfoClientMockRecorder) Recv() *gomock.Call { - mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Recv", reflect.TypeOf((*MockWorker_FetchDDLInfoClient)(nil).Recv)) -} - -// RecvMsg mocks base method -func (m *MockWorker_FetchDDLInfoClient) RecvMsg(arg0 interface{}) error { - m.ctrl.T.Helper() - ret := m.ctrl.Call(m, "RecvMsg", arg0) - ret0, _ := ret[0].(error) - return ret0 -} - -// RecvMsg indicates an expected call of RecvMsg -func (mr *MockWorker_FetchDDLInfoClientMockRecorder) RecvMsg(arg0 interface{}) *gomock.Call { - mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "RecvMsg", reflect.TypeOf((*MockWorker_FetchDDLInfoClient)(nil).RecvMsg), arg0) -} - -// Send mocks base method -func (m *MockWorker_FetchDDLInfoClient) Send(arg0 *pb.DDLLockInfo) error { - m.ctrl.T.Helper() - ret := m.ctrl.Call(m, "Send", arg0) - ret0, _ := ret[0].(error) - return ret0 -} - -// Send indicates an expected call of Send -func (mr *MockWorker_FetchDDLInfoClientMockRecorder) Send(arg0 interface{}) *gomock.Call { - mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Send", reflect.TypeOf((*MockWorker_FetchDDLInfoClient)(nil).Send), arg0) -} - -// SendMsg mocks base method -func (m *MockWorker_FetchDDLInfoClient) SendMsg(arg0 interface{}) error { - m.ctrl.T.Helper() - ret := m.ctrl.Call(m, "SendMsg", arg0) - ret0, _ := ret[0].(error) - return ret0 -} - -// SendMsg indicates an expected call of SendMsg -func (mr *MockWorker_FetchDDLInfoClientMockRecorder) SendMsg(arg0 interface{}) *gomock.Call { - mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "SendMsg", reflect.TypeOf((*MockWorker_FetchDDLInfoClient)(nil).SendMsg), arg0) -} - -// Trailer mocks base method -func (m *MockWorker_FetchDDLInfoClient) Trailer() metadata.MD { - m.ctrl.T.Helper() - ret := m.ctrl.Call(m, "Trailer") - ret0, _ := ret[0].(metadata.MD) - return ret0 -} - -// Trailer indicates an expected call of Trailer -func (mr *MockWorker_FetchDDLInfoClientMockRecorder) Trailer() *gomock.Call { - mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Trailer", reflect.TypeOf((*MockWorker_FetchDDLInfoClient)(nil).Trailer)) -} - // MockWorkerServer is a mock of WorkerServer interface type MockWorkerServer struct { ctrl *gomock.Controller @@ -516,50 +318,6 @@ func (m *MockWorkerServer) EXPECT() *MockWorkerServerMockRecorder { return m.recorder } -// BreakDDLLock mocks base method -func (m *MockWorkerServer) BreakDDLLock(arg0 context.Context, arg1 *pb.BreakDDLLockRequest) (*pb.CommonWorkerResponse, error) { - m.ctrl.T.Helper() - ret := m.ctrl.Call(m, "BreakDDLLock", arg0, arg1) - ret0, _ := ret[0].(*pb.CommonWorkerResponse) - ret1, _ := ret[1].(error) - return ret0, ret1 -} - -// BreakDDLLock indicates an expected call of BreakDDLLock -func (mr *MockWorkerServerMockRecorder) BreakDDLLock(arg0, arg1 interface{}) *gomock.Call { - mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "BreakDDLLock", reflect.TypeOf((*MockWorkerServer)(nil).BreakDDLLock), arg0, arg1) -} - -// ExecuteDDL mocks base method -func (m *MockWorkerServer) ExecuteDDL(arg0 context.Context, arg1 *pb.ExecDDLRequest) (*pb.CommonWorkerResponse, error) { - m.ctrl.T.Helper() - ret := m.ctrl.Call(m, "ExecuteDDL", arg0, arg1) - ret0, _ := ret[0].(*pb.CommonWorkerResponse) - ret1, _ := ret[1].(error) - return ret0, ret1 -} - -// ExecuteDDL indicates an expected call of ExecuteDDL -func (mr *MockWorkerServerMockRecorder) ExecuteDDL(arg0, arg1 interface{}) *gomock.Call { - mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "ExecuteDDL", reflect.TypeOf((*MockWorkerServer)(nil).ExecuteDDL), arg0, arg1) -} - -// FetchDDLInfo mocks base method -func (m *MockWorkerServer) FetchDDLInfo(arg0 pb.Worker_FetchDDLInfoServer) error { - m.ctrl.T.Helper() - ret := m.ctrl.Call(m, "FetchDDLInfo", arg0) - ret0, _ := ret[0].(error) - return ret0 -} - -// FetchDDLInfo indicates an expected call of FetchDDLInfo -func (mr *MockWorkerServerMockRecorder) FetchDDLInfo(arg0 interface{}) *gomock.Call { - mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "FetchDDLInfo", reflect.TypeOf((*MockWorkerServer)(nil).FetchDDLInfo), arg0) -} - // HandleSQLs mocks base method func (m *MockWorkerServer) HandleSQLs(arg0 context.Context, arg1 *pb.HandleSubTaskSQLsRequest) (*pb.CommonWorkerResponse, error) { m.ctrl.T.Helper() @@ -754,137 +512,3 @@ func (mr *MockWorkerServerMockRecorder) UpdateSubTask(arg0, arg1 interface{}) *g mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "UpdateSubTask", reflect.TypeOf((*MockWorkerServer)(nil).UpdateSubTask), arg0, arg1) } - -// MockWorker_FetchDDLInfoServer is a mock of Worker_FetchDDLInfoServer interface -type MockWorker_FetchDDLInfoServer struct { - ctrl *gomock.Controller - recorder *MockWorker_FetchDDLInfoServerMockRecorder -} - -// MockWorker_FetchDDLInfoServerMockRecorder is the mock recorder for MockWorker_FetchDDLInfoServer -type MockWorker_FetchDDLInfoServerMockRecorder struct { - mock *MockWorker_FetchDDLInfoServer -} - -// NewMockWorker_FetchDDLInfoServer creates a new mock instance -func NewMockWorker_FetchDDLInfoServer(ctrl *gomock.Controller) *MockWorker_FetchDDLInfoServer { - mock := &MockWorker_FetchDDLInfoServer{ctrl: ctrl} - mock.recorder = &MockWorker_FetchDDLInfoServerMockRecorder{mock} - return mock -} - -// EXPECT returns an object that allows the caller to indicate expected use -func (m *MockWorker_FetchDDLInfoServer) EXPECT() *MockWorker_FetchDDLInfoServerMockRecorder { - return m.recorder -} - -// Context mocks base method -func (m *MockWorker_FetchDDLInfoServer) Context() context.Context { - m.ctrl.T.Helper() - ret := m.ctrl.Call(m, "Context") - ret0, _ := ret[0].(context.Context) - return ret0 -} - -// Context indicates an expected call of Context -func (mr *MockWorker_FetchDDLInfoServerMockRecorder) Context() *gomock.Call { - mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Context", reflect.TypeOf((*MockWorker_FetchDDLInfoServer)(nil).Context)) -} - -// Recv mocks base method -func (m *MockWorker_FetchDDLInfoServer) Recv() (*pb.DDLLockInfo, error) { - m.ctrl.T.Helper() - ret := m.ctrl.Call(m, "Recv") - ret0, _ := ret[0].(*pb.DDLLockInfo) - ret1, _ := ret[1].(error) - return ret0, ret1 -} - -// Recv indicates an expected call of Recv -func (mr *MockWorker_FetchDDLInfoServerMockRecorder) Recv() *gomock.Call { - mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Recv", reflect.TypeOf((*MockWorker_FetchDDLInfoServer)(nil).Recv)) -} - -// RecvMsg mocks base method -func (m *MockWorker_FetchDDLInfoServer) RecvMsg(arg0 interface{}) error { - m.ctrl.T.Helper() - ret := m.ctrl.Call(m, "RecvMsg", arg0) - ret0, _ := ret[0].(error) - return ret0 -} - -// RecvMsg indicates an expected call of RecvMsg -func (mr *MockWorker_FetchDDLInfoServerMockRecorder) RecvMsg(arg0 interface{}) *gomock.Call { - mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "RecvMsg", reflect.TypeOf((*MockWorker_FetchDDLInfoServer)(nil).RecvMsg), arg0) -} - -// Send mocks base method -func (m *MockWorker_FetchDDLInfoServer) Send(arg0 *pb.DDLInfo) error { - m.ctrl.T.Helper() - ret := m.ctrl.Call(m, "Send", arg0) - ret0, _ := ret[0].(error) - return ret0 -} - -// Send indicates an expected call of Send -func (mr *MockWorker_FetchDDLInfoServerMockRecorder) Send(arg0 interface{}) *gomock.Call { - mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Send", reflect.TypeOf((*MockWorker_FetchDDLInfoServer)(nil).Send), arg0) -} - -// SendHeader mocks base method -func (m *MockWorker_FetchDDLInfoServer) SendHeader(arg0 metadata.MD) error { - m.ctrl.T.Helper() - ret := m.ctrl.Call(m, "SendHeader", arg0) - ret0, _ := ret[0].(error) - return ret0 -} - -// SendHeader indicates an expected call of SendHeader -func (mr *MockWorker_FetchDDLInfoServerMockRecorder) SendHeader(arg0 interface{}) *gomock.Call { - mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "SendHeader", reflect.TypeOf((*MockWorker_FetchDDLInfoServer)(nil).SendHeader), arg0) -} - -// SendMsg mocks base method -func (m *MockWorker_FetchDDLInfoServer) SendMsg(arg0 interface{}) error { - m.ctrl.T.Helper() - ret := m.ctrl.Call(m, "SendMsg", arg0) - ret0, _ := ret[0].(error) - return ret0 -} - -// SendMsg indicates an expected call of SendMsg -func (mr *MockWorker_FetchDDLInfoServerMockRecorder) SendMsg(arg0 interface{}) *gomock.Call { - mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "SendMsg", reflect.TypeOf((*MockWorker_FetchDDLInfoServer)(nil).SendMsg), arg0) -} - -// SetHeader mocks base method -func (m *MockWorker_FetchDDLInfoServer) SetHeader(arg0 metadata.MD) error { - m.ctrl.T.Helper() - ret := m.ctrl.Call(m, "SetHeader", arg0) - ret0, _ := ret[0].(error) - return ret0 -} - -// SetHeader indicates an expected call of SetHeader -func (mr *MockWorker_FetchDDLInfoServerMockRecorder) SetHeader(arg0 interface{}) *gomock.Call { - mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "SetHeader", reflect.TypeOf((*MockWorker_FetchDDLInfoServer)(nil).SetHeader), arg0) -} - -// SetTrailer mocks base method -func (m *MockWorker_FetchDDLInfoServer) SetTrailer(arg0 metadata.MD) { - m.ctrl.T.Helper() - m.ctrl.Call(m, "SetTrailer", arg0) -} - -// SetTrailer indicates an expected call of SetTrailer -func (mr *MockWorker_FetchDDLInfoServerMockRecorder) SetTrailer(arg0 interface{}) *gomock.Call { - mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "SetTrailer", reflect.TypeOf((*MockWorker_FetchDDLInfoServer)(nil).SetTrailer), arg0) -} diff --git a/dm/proto/dmworker.proto b/dm/proto/dmworker.proto index 79bcf01447..dfec20796c 100644 --- a/dm/proto/dmworker.proto +++ b/dm/proto/dmworker.proto @@ -12,17 +12,6 @@ service Worker { rpc QueryError (QueryErrorRequest) returns (QueryErrorResponse) {} rpc HandleSQLs (HandleSubTaskSQLsRequest) returns (CommonWorkerResponse) {} - // FetchDDLInfo fetches DDL info from dm-worker by dm-master - // and sends the relevant DDL lock info back to dm-worker - // NOTE: split to two RPCs if needed - rpc FetchDDLInfo (stream DDLLockInfo) returns (stream DDLInfo) {} - - // ExecuteDDL request a dm-worker to execute or ignore (skip) a DDL - rpc ExecuteDDL (ExecDDLRequest) returns (CommonWorkerResponse) {} - - // BreakDDLLock request a dm-worker to break a DDL lock - // including remove DDLLockInfo and/or execute/skip DDL - rpc BreakDDLLock (BreakDDLLockRequest) returns (CommonWorkerResponse) {} // SwitchRelayMaster request a dm-worker's relay unit switching to another master server rpc SwitchRelayMaster (SwitchRelayMasterRequest) returns (CommonWorkerResponse) {} @@ -363,47 +352,6 @@ message ProcessError { TError error = 3; } -// DDLInfo represents a occurred DDL in a sub task -// it been sent from dm-worker to dm-master -// dm-master uses it to contruct a DDL lock and do syncing with other dm-workers -// add more fields if needed -message DDLInfo { - string task = 1; // sub task's name - string schema = 2; // DDL's schema - string table = 3; // DDL's table - repeated string DDLs = 4; // DDL statement -} - -// DDLLockInfo represents a DDL lock -// it been sent from dm-master to dm-worker -// add more fields if needed -message DDLLockInfo { - string task = 1; // sub task's name - string ID = 2; // DDL lock ID -} - -// ExecDDLRequest represents a request for a dm-worker to execute (or ignore) a DDL -message ExecDDLRequest { - string task = 1; // sub task's name - string lockID = 2; // DDL lock ID - bool exec = 3; // true for execute, false for ignore (skip) - string traceGID = 4; // trace group ID - repeated string DDLs = 5; // DDL statement -} - -// BreakDDLLockRequest represents a request for a dm-worker to force to break the DDL lock -// task: sub task's name -// removeLockID: DDLLockInfo's ID which need to remove -// execDDL: execute DDL which is blocking -// skipDDL: skip DDL which is blocking -// execDDL and skipDDL can not specify both at the same time, but can specify neither -message BreakDDLLockRequest { - string task = 1; - string removeLockID = 2; - bool execDDL = 3; - bool skipDDL = 4; -} - // SwitchRelayMasterRequest represents a request for switching a dm-worker's relay unit to another master server message SwitchRelayMasterRequest { } diff --git a/dm/worker/server.go b/dm/worker/server.go index 5ce5bd1d7e..01211e92f2 100644 --- a/dm/worker/server.go +++ b/dm/worker/server.go @@ -15,7 +15,6 @@ package worker import ( "context" - "io" "net" "sync" "time" @@ -420,89 +419,6 @@ func (s *Server) QueryError(ctx context.Context, req *pb.QueryErrorRequest) (*pb return resp, nil } -// FetchDDLInfo implements WorkerServer.FetchDDLInfo -// we do ping-pong send-receive on stream for DDL (lock) info -// if error occurred in Send / Recv, just retry in client -func (s *Server) FetchDDLInfo(stream pb.Worker_FetchDDLInfoServer) error { - log.L().Info("", zap.String("request", "FetchDDLInfo")) - w := s.getWorker(true) - if w == nil { - log.L().Error("fail to call StartSubTask, because mysql worker has not been started") - return terror.ErrWorkerNoStart.Generate() - } - - var ddlInfo *pb.DDLInfo - for { - // try fetch pending to sync DDL info from worker - ddlInfo = w.FetchDDLInfo(stream.Context()) - if ddlInfo == nil { - return nil // worker closed or context canceled - } - log.L().Info("", zap.String("request", "FetchDDLInfo"), zap.Stringer("ddl info", ddlInfo)) - // send DDLInfo to dm-master - err := stream.Send(ddlInfo) - if err != nil { - log.L().Error("fail to send DDLInfo to RPC stream", zap.String("request", "FetchDDLInfo"), zap.Stringer("ddl info", ddlInfo), log.ShortError(err)) - return err - } - - // receive DDLLockInfo from dm-master - in, err := stream.Recv() - if err == io.EOF { - return nil - } - if err != nil { - log.L().Error("fail to receive DDLLockInfo from RPC stream", zap.String("request", "FetchDDLInfo"), zap.Stringer("ddl info", ddlInfo), log.ShortError(err)) - return err - } - log.L().Info("receive DDLLockInfo", zap.String("request", "FetchDDLInfo"), zap.Stringer("ddl lock info", in)) - - //ddlInfo = nil // clear and protect to put it back - - err = w.RecordDDLLockInfo(in) - if err != nil { - // if error occurred when recording DDLLockInfo, log an error - // user can handle this case using dmctl - log.L().Error("fail to record DDLLockInfo", zap.String("request", "FetchDDLInfo"), zap.Stringer("ddl lock info", in), zap.Error(err)) - } - } - // TODO: check whether this interface need to store message in ETCD -} - -// ExecuteDDL implements WorkerServer.ExecuteDDL -func (s *Server) ExecuteDDL(ctx context.Context, req *pb.ExecDDLRequest) (*pb.CommonWorkerResponse, error) { - log.L().Info("", zap.String("request", "ExecuteDDL"), zap.Stringer("payload", req)) - w := s.getWorker(true) - if w == nil { - log.L().Error("fail to call StartSubTask, because mysql worker has not been started") - return makeCommonWorkerResponse(terror.ErrWorkerNoStart.Generate()), nil - } - - err := w.ExecuteDDL(ctx, req) - if err != nil { - log.L().Error("fail to execute ddl", zap.String("request", "ExecuteDDL"), zap.Stringer("payload", req), zap.Error(err)) - } - // TODO: check whether this interface need to store message in ETCD - return makeCommonWorkerResponse(err), nil -} - -// BreakDDLLock implements WorkerServer.BreakDDLLock -func (s *Server) BreakDDLLock(ctx context.Context, req *pb.BreakDDLLockRequest) (*pb.CommonWorkerResponse, error) { - log.L().Info("", zap.String("request", "BreakDDLLock"), zap.Stringer("payload", req)) - w := s.getWorker(true) - if w == nil { - log.L().Error("fail to call StartSubTask, because mysql worker has not been started") - return makeCommonWorkerResponse(terror.ErrWorkerNoStart.Generate()), nil - } - - err := w.BreakDDLLock(ctx, req) - if err != nil { - log.L().Error("fail to break ddl lock", zap.String("request", "BreakDDLLock"), zap.Stringer("payload", req), zap.Error(err)) - } - // TODO: check whether this interface need to store message in ETCD - return makeCommonWorkerResponse(err), nil -} - // HandleSQLs implements WorkerServer.HandleSQLs func (s *Server) HandleSQLs(ctx context.Context, req *pb.HandleSubTaskSQLsRequest) (*pb.CommonWorkerResponse, error) { log.L().Info("", zap.String("request", "HandleSQLs"), zap.Stringer("payload", req)) @@ -701,7 +617,7 @@ func (s *Server) startWorker(cfg *config.MysqlConfig) error { log.L().Info("start workers", zap.Reflect("subTasks", subTaskCfgs)) - w, err := NewWorker(cfg) + w, err := NewWorker(cfg, s.etcdClient) if err != nil { return err } diff --git a/dm/worker/status.go b/dm/worker/status.go index 069f1757d9..700f10ea2b 100644 --- a/dm/worker/status.go +++ b/dm/worker/status.go @@ -78,9 +78,9 @@ func (w *Worker) Status(stName string) []*pb.SubTaskStatus { } } else { var lockID = "" - lockInfo := st.DDLLockInfo() - if lockInfo != nil { - lockID = lockInfo.ID + op := st.ShardDDLOperation() + if op != nil { + lockID = op.ID } cu := st.CurrUnit() diff --git a/dm/worker/subtask.go b/dm/worker/subtask.go index 5540b94e4b..94f416a6d4 100644 --- a/dm/worker/subtask.go +++ b/dm/worker/subtask.go @@ -18,24 +18,25 @@ import ( "sync" "time" + "github.com/pingcap/failpoint" + "github.com/siddontang/go/sync2" + "go.etcd.io/etcd/clientv3" + "go.uber.org/zap" + "github.com/pingcap/dm/dm/config" "github.com/pingcap/dm/dm/pb" "github.com/pingcap/dm/dm/unit" "github.com/pingcap/dm/loader" "github.com/pingcap/dm/mydumper" "github.com/pingcap/dm/pkg/log" + "github.com/pingcap/dm/pkg/shardddl/pessimism" "github.com/pingcap/dm/pkg/terror" "github.com/pingcap/dm/pkg/utils" "github.com/pingcap/dm/syncer" - - "github.com/golang/protobuf/proto" - "github.com/pingcap/failpoint" - "github.com/siddontang/go/sync2" - "go.uber.org/zap" ) // createUnits creates process units base on task mode -func createUnits(cfg *config.SubTaskConfig) []unit.Unit { +func createUnits(cfg *config.SubTaskConfig, etcdClient *clientv3.Client) []unit.Unit { failpoint.Inject("mockCreateUnitsDumpOnly", func(_ failpoint.Value) { log.L().Info("create mock worker units with dump unit only", zap.String("failpoint", "mockCreateUnitsDumpOnly")) failpoint.Return([]unit.Unit{mydumper.NewMydumper(cfg)}) @@ -46,13 +47,13 @@ func createUnits(cfg *config.SubTaskConfig) []unit.Unit { case config.ModeAll: us = append(us, mydumper.NewMydumper(cfg)) us = append(us, loader.NewLoader(cfg)) - us = append(us, syncer.NewSyncer(cfg)) + us = append(us, syncer.NewSyncer(cfg, etcdClient)) case config.ModeFull: // NOTE: maybe need another checker in the future? us = append(us, mydumper.NewMydumper(cfg)) us = append(us, loader.NewLoader(cfg)) case config.ModeIncrement: - us = append(us, syncer.NewSyncer(cfg)) + us = append(us, syncer.NewSyncer(cfg, etcdClient)) default: log.L().Error("unsupported task mode", zap.String("subtask", cfg.Name), zap.String("task mode", cfg.Mode)) } @@ -79,25 +80,22 @@ type SubTask struct { stage pb.Stage // stage of current sub task result *pb.ProcessResult // the process result, nil when is processing - // only support sync one DDL lock one time, refine if needed - DDLInfo chan *pb.DDLInfo // DDL info pending to sync - ddlLockInfo *pb.DDLLockInfo // DDL lock info which waiting other dm-workers to sync - cacheDDLInfo *pb.DDLInfo + etcdClient *clientv3.Client } // NewSubTask creates a new SubTask -func NewSubTask(cfg *config.SubTaskConfig) *SubTask { - return NewSubTaskWithStage(cfg, pb.Stage_New) +func NewSubTask(cfg *config.SubTaskConfig, etcdClient *clientv3.Client) *SubTask { + return NewSubTaskWithStage(cfg, pb.Stage_New, etcdClient) } // NewSubTaskWithStage creates a new SubTask with stage -func NewSubTaskWithStage(cfg *config.SubTaskConfig, stage pb.Stage) *SubTask { +func NewSubTaskWithStage(cfg *config.SubTaskConfig, stage pb.Stage, etcdClient *clientv3.Client) *SubTask { st := SubTask{ - cfg: cfg, - units: createUnits(cfg), - stage: stage, - l: log.With(zap.String("subtask", cfg.Name)), - DDLInfo: make(chan *pb.DDLInfo, 1), + cfg: cfg, + units: createUnits(cfg, etcdClient), + stage: stage, + l: log.With(zap.String("subtask", cfg.Name)), + etcdClient: etcdClient, } taskState.WithLabelValues(st.cfg.Name).Set(float64(st.stage)) return &st @@ -197,9 +195,6 @@ func (st *SubTask) run() { st.wg.Add(1) go st.fetchResult(pr) go cu.Process(st.ctx, pr) - - st.wg.Add(1) - go st.fetchUnitDDLInfo(st.ctx) } // fetchResult fetches units process result @@ -437,9 +432,6 @@ func (st *SubTask) Resume() error { st.wg.Add(1) go st.fetchResult(pr) go cu.Resume(st.ctx, pr) - - st.wg.Add(1) - go st.fetchUnitDDLInfo(st.ctx) return nil } @@ -460,78 +452,6 @@ func (st *SubTask) Update(cfg *config.SubTaskConfig) error { return nil } -// fetchUnitDDLInfo fetches DDL info from current processing unit -// when unit switched, returns and starts fetching again for new unit -func (st *SubTask) fetchUnitDDLInfo(ctx context.Context) { - defer st.wg.Done() - - cu := st.CurrUnit() - syncer2, ok := cu.(*syncer.Syncer) - if !ok { - return - } - - // discard previous saved DDLInfo - // when process unit resuming, un-resolved DDL will send again - for len(st.DDLInfo) > 0 { - <-st.DDLInfo - } - - for { - select { - case <-ctx.Done(): - return - case info, ok := <-syncer2.DDLInfo(): - if !ok { - return - } - select { - case <-ctx.Done(): - return - case st.DDLInfo <- info: - } - } - } -} - -// ExecuteDDL requests current unit to execute a DDL -func (st *SubTask) ExecuteDDL(ctx context.Context, req *pb.ExecDDLRequest) error { - // NOTE: check current stage? - cu := st.CurrUnit() - syncer2, ok := cu.(*syncer.Syncer) - if !ok { - return terror.ErrWorkerExecDDLSyncerOnly.Generate(cu.Type().String()) - } - chResp, err := syncer2.ExecuteDDL(ctx, req) - if err != nil { - return err - } - - // also any timeout - timeout := time.Duration(syncer.MaxDDLConnectionTimeoutMinute)*time.Minute + 30*time.Second - ctxTimeout, cancel := context.WithTimeout(context.Background(), timeout) - defer cancel() - select { - case err = <-chResp: // block until complete ddl execution - return err - case <-ctx.Done(): - return ctx.Err() - case <-ctxTimeout.Done(): - return terror.ErrWorkerExecDDLTimeout.Generate(timeout) - } -} - -// SaveDDLLockInfo saves a DDLLockInfo -func (st *SubTask) SaveDDLLockInfo(info *pb.DDLLockInfo) error { - st.Lock() - defer st.Unlock() - if st.ddlLockInfo != nil { - return terror.ErrWorkerDDLLockInfoExists.Generate(info.Task) - } - st.ddlLockInfo = info - return nil -} - // SetSyncerSQLOperator sets an operator to syncer. func (st *SubTask) SetSyncerSQLOperator(ctx context.Context, req *pb.HandleSubTaskSQLsRequest) error { syncUnit, ok := st.currUnit.(*syncer.Syncer) @@ -547,20 +467,6 @@ func (st *SubTask) SetSyncerSQLOperator(ctx context.Context, req *pb.HandleSubTa return syncUnit.SetSQLOperator(req) } -// ClearDDLLockInfo clears current DDLLockInfo -func (st *SubTask) ClearDDLLockInfo() { - st.Lock() - defer st.Unlock() - st.ddlLockInfo = nil -} - -// DDLLockInfo returns current DDLLockInfo, maybe nil -func (st *SubTask) DDLLockInfo() *pb.DDLLockInfo { - st.RLock() - defer st.RUnlock() - return proto.Clone(st.ddlLockInfo).(*pb.DDLLockInfo) -} - // UpdateFromConfig updates config for `From` func (st *SubTask) UpdateFromConfig(cfg *config.SubTaskConfig) error { st.Lock() @@ -592,29 +498,32 @@ func (st *SubTask) CheckUnit() bool { return flag } -// SaveDDLInfo saves a CacheDDLInfo. -func (st *SubTask) SaveDDLInfo(info *pb.DDLInfo) error { - st.Lock() - defer st.Unlock() - if st.cacheDDLInfo != nil { - return terror.ErrWorkerCacheDDLInfoExists.Generate(info.Task) +// ShardDDLInfo returns the current shard DDL info. +func (st *SubTask) ShardDDLInfo() *pessimism.Info { + st.RLock() + defer st.RUnlock() + + cu := st.CurrUnit() + syncer2, ok := cu.(*syncer.Syncer) + if !ok { + return nil } - st.cacheDDLInfo = info - return nil + + return syncer2.ShardDDLInfo() } -// GetDDLInfo returns current CacheDDLInfo. -func (st *SubTask) GetDDLInfo() *pb.DDLInfo { +// ShardDDLOperation returns the current shard DDL lock operation. +func (st *SubTask) ShardDDLOperation() *pessimism.Operation { st.RLock() defer st.RUnlock() - return proto.Clone(st.cacheDDLInfo).(*pb.DDLInfo) -} -// ClearDDLInfo clears current CacheDDLInfo. -func (st *SubTask) ClearDDLInfo() { - st.Lock() - defer st.Unlock() - st.cacheDDLInfo = nil + cu := st.CurrUnit() + syncer2, ok := cu.(*syncer.Syncer) + if !ok { + return nil + } + + return syncer2.ShardDDLOperation() } // unitTransWaitCondition waits when transferring from current unit to next unit. diff --git a/dm/worker/subtask_test.go b/dm/worker/subtask_test.go index fa06a9d5cc..6d0864e3f7 100644 --- a/dm/worker/subtask_test.go +++ b/dm/worker/subtask_test.go @@ -37,10 +37,10 @@ func (t *testSubTask) TestCreateUnits(c *C) { cfg := &config.SubTaskConfig{ Mode: "xxx", } - c.Assert(createUnits(cfg), HasLen, 0) + c.Assert(createUnits(cfg, nil), HasLen, 0) cfg.Mode = config.ModeFull - unitsFull := createUnits(cfg) + unitsFull := createUnits(cfg, nil) c.Assert(unitsFull, HasLen, 2) _, ok := unitsFull[0].(*mydumper.Mydumper) c.Assert(ok, IsTrue) @@ -48,13 +48,13 @@ func (t *testSubTask) TestCreateUnits(c *C) { c.Assert(ok, IsTrue) cfg.Mode = config.ModeIncrement - unitsIncr := createUnits(cfg) + unitsIncr := createUnits(cfg, nil) c.Assert(unitsIncr, HasLen, 1) _, ok = unitsIncr[0].(*syncer.Syncer) c.Assert(ok, IsTrue) cfg.Mode = config.ModeAll - unitsAll := createUnits(cfg) + unitsAll := createUnits(cfg, nil) c.Assert(unitsAll, HasLen, 3) _, ok = unitsAll[0].(*mydumper.Mydumper) c.Assert(ok, IsTrue) @@ -148,7 +148,7 @@ func (t *testSubTask) TestSubTaskNormalUsage(c *C) { Mode: config.ModeFull, } - st := NewSubTask(cfg) + st := NewSubTask(cfg, nil) c.Assert(st.Stage(), DeepEquals, pb.Stage_New) // test empty and fail @@ -261,7 +261,7 @@ func (t *testSubTask) TestPauseAndResumeSubtask(c *C) { Mode: config.ModeFull, } - st := NewSubTask(cfg) + st := NewSubTask(cfg, nil) c.Assert(st.Stage(), DeepEquals, pb.Stage_New) mockDumper := NewMockUnit(pb.UnitType_Dump) @@ -398,7 +398,7 @@ func (t *testSubTask) TestSubtaskWithStage(c *C) { Mode: config.ModeFull, } - st := NewSubTaskWithStage(cfg, pb.Stage_Paused) + st := NewSubTaskWithStage(cfg, pb.Stage_Paused, nil) c.Assert(st.Stage(), DeepEquals, pb.Stage_Paused) mockDumper := NewMockUnit(pb.UnitType_Dump) @@ -425,7 +425,7 @@ func (t *testSubTask) TestSubtaskWithStage(c *C) { c.Fatalf("result %+v is not right after closing", st.Result()) } - st = NewSubTaskWithStage(cfg, pb.Stage_Finished) + st = NewSubTaskWithStage(cfg, pb.Stage_Finished, nil) c.Assert(st.Stage(), DeepEquals, pb.Stage_Finished) st.units = []unit.Unit{mockDumper, mockLoader} @@ -446,53 +446,3 @@ func (t *testSubTask) TestSubtaskWithStage(c *C) { c.Assert(st.CurrUnit(), Equals, nil) c.Assert(st.Result(), IsNil) } - -func (t *testSubTask) TestDDLLockInfo(c *C) { - cfg := &config.SubTaskConfig{ - Name: "testSubtaskScene", - Mode: config.ModeFull, - } - - st := NewSubTaskWithStage(cfg, pb.Stage_Paused) - c.Assert(st.Stage(), DeepEquals, pb.Stage_Paused) - c.Assert(st.DDLLockInfo(), IsNil) - - ddlLock := &pb.DDLLockInfo{ - ID: "xxx", - } - - c.Assert(st.SaveDDLLockInfo(ddlLock), IsNil) - c.Assert(st.DDLLockInfo(), DeepEquals, ddlLock) - c.Assert(st.SaveDDLLockInfo(&pb.DDLLockInfo{}), ErrorMatches, ".*already exists.*") - - st.ClearDDLLockInfo() - c.Assert(st.DDLLockInfo(), IsNil) - - c.Assert(st.SaveDDLLockInfo(ddlLock), IsNil) - c.Assert(st.DDLLockInfo(), DeepEquals, ddlLock) -} - -func (t *testSubTask) TestDDLInfo(c *C) { - cfg := &config.SubTaskConfig{ - Name: "testSubtaskScene", - Mode: config.ModeFull, - } - - st := NewSubTaskWithStage(cfg, pb.Stage_Paused) - c.Assert(st.Stage(), DeepEquals, pb.Stage_Paused) - c.Assert(st.GetDDLInfo(), IsNil) - - ddlInfo := &pb.DDLInfo{ - Task: "xxx", - } - - c.Assert(st.SaveDDLInfo(ddlInfo), IsNil) - c.Assert(st.GetDDLInfo(), DeepEquals, ddlInfo) - c.Assert(st.SaveDDLInfo(&pb.DDLInfo{}), ErrorMatches, ".*already exists.*") - - st.ClearDDLInfo() - c.Assert(st.GetDDLInfo(), IsNil) - - c.Assert(st.SaveDDLInfo(ddlInfo), IsNil) - c.Assert(st.GetDDLInfo(), DeepEquals, ddlInfo) -} diff --git a/dm/worker/task_checker_test.go b/dm/worker/task_checker_test.go index 46fefd8ffc..9541666409 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) { c.Assert(cfg.LoadFromFile("./dm-mysql.toml"), check.IsNil) cfg.RelayDir = dir cfg.MetaDir = dir - w, err := NewWorker(cfg) + w, err := NewWorker(cfg, nil) c.Assert(err, check.IsNil) w.closed.Set(closedFalse) @@ -210,7 +210,7 @@ func (s *testTaskCheckerSuite) TestCheckTaskIndependent(c *check.C) { c.Assert(cfg.LoadFromFile("./dm-mysql.toml"), check.IsNil) cfg.RelayDir = dir cfg.MetaDir = dir - w, err := NewWorker(cfg) + w, err := NewWorker(cfg, nil) c.Assert(err, check.IsNil) w.closed.Set(closedFalse) diff --git a/dm/worker/worker.go b/dm/worker/worker.go index 2dc597bf4d..81a2b56ded 100644 --- a/dm/worker/worker.go +++ b/dm/worker/worker.go @@ -16,12 +16,12 @@ package worker import ( "context" "fmt" - "reflect" "sync" "time" "github.com/golang/protobuf/proto" "github.com/siddontang/go/sync2" + "go.etcd.io/etcd/clientv3" "go.uber.org/zap" "github.com/pingcap/dm/dm/config" @@ -33,9 +33,6 @@ import ( ) var ( - // sub tasks may changed, so we re-FetchDDLInfo at intervals - reFetchInterval = 10 * time.Second - closedFalse int32 closedTrue int32 = 1 ) @@ -64,15 +61,18 @@ type Worker struct { taskStatusChecker TaskStatusChecker configFile string + + etcdClient *clientv3.Client } // NewWorker creates a new Worker -func NewWorker(cfg *config.MysqlConfig) (w *Worker, err error) { +func NewWorker(cfg *config.MysqlConfig, etcdClient *clientv3.Client) (w *Worker, err error) { w = &Worker{ cfg: cfg, tracer: tracing.InitTracerHub(cfg.Tracer), subTaskHolder: newSubTaskHolder(), l: log.With(zap.String("component", "worker controller")), + etcdClient: etcdClient, } w.ctx, w.cancel = context.WithCancel(context.Background()) w.closed.Set(closedTrue) @@ -220,7 +220,7 @@ func (w *Worker) StartSubTask(cfg *config.SubTaskConfig) error { } w.l.Info("started sub task", zap.Stringer("config", cfgDecrypted)) - st := NewSubTask(cfgDecrypted) + st := NewSubTask(cfgDecrypted, w.etcdClient) w.subTaskHolder.recordSubTask(st) st.Run() return nil @@ -320,163 +320,6 @@ func (w *Worker) HandleSQLs(ctx context.Context, req *pb.HandleSubTaskSQLsReques return st.SetSyncerSQLOperator(ctx, req) } -// FetchDDLInfo fetches all sub tasks' DDL info which pending to sync -func (w *Worker) FetchDDLInfo(ctx context.Context) *pb.DDLInfo { - if w.closed.Get() == closedTrue { - w.l.Warn("fetching DDLInfo from a closed worker") - return nil - } - - // sub tasks can be changed by StartSubTask / StopSubTask, so we retry doFetch at intervals - // maybe we can refine later and just re-do when sub tasks changed really - result := make(chan *pb.DDLInfo, 1) - for { - newCtx, cancel := context.WithTimeout(ctx, reFetchInterval) - - var wg sync.WaitGroup - wg.Add(1) - go func() { - defer wg.Done() - w.doFetchDDLInfo(newCtx, result) - cancel() // cancel when doFetchDDLInfo returned - }() - - <-newCtx.Done() // wait for timeout or canceled - - if ctx.Err() == context.Canceled { - return nil // canceled from external - } - - wg.Wait() - if len(result) > 0 { - return <-result - } - } -} - -// doFetchDDLInfo does fetching DDL info for all sub tasks concurrently -func (w *Worker) doFetchDDLInfo(ctx context.Context, ch chan<- *pb.DDLInfo) { - sts := w.subTaskHolder.getAllSubTasks() - cases := make([]reflect.SelectCase, 0, len(sts)+1) - for _, st := range sts { - cases = append(cases, reflect.SelectCase{ - Dir: reflect.SelectRecv, - Chan: reflect.ValueOf(st.DDLInfo), // select all sub tasks' chan - }) - } - - cases = append(cases, reflect.SelectCase{ - Dir: reflect.SelectRecv, - Chan: reflect.ValueOf(ctx.Done()), // also need select context - }) - - _, value, ok := reflect.Select(cases) - if !ok { - for _, st := range sts { - // NOTE: Can you guarantee that each DDLInfo you get is different? - if st.GetDDLInfo() == nil { - continue - } - ch <- st.GetDDLInfo() - break - } - return - } - - v, ok := value.Interface().(*pb.DDLInfo) - if !ok { - return // should not go here - } - - st := w.subTaskHolder.findSubTask(v.Task) - if st != nil { - st.SaveDDLInfo(v) - w.l.Info("save DDLInfo into subTasks") - ch <- v - } else { - w.l.Warn("can not find specified subtask", zap.String("task", v.Task)) - } -} - -// RecordDDLLockInfo records the current DDL lock info which pending to sync -func (w *Worker) RecordDDLLockInfo(info *pb.DDLLockInfo) error { - if w.closed.Get() == closedTrue { - return terror.ErrWorkerAlreadyClosed.Generate() - } - - st := w.subTaskHolder.findSubTask(info.Task) - if st == nil { - return terror.ErrWorkerSubTaskNotFound.Generatef("sub task for DDLLockInfo %+v not found", info) - } - return st.SaveDDLLockInfo(info) -} - -// ExecuteDDL executes (or ignores) DDL (in sharding DDL lock, requested by dm-master) -func (w *Worker) ExecuteDDL(ctx context.Context, req *pb.ExecDDLRequest) error { - if w.closed.Get() == closedTrue { - return terror.ErrWorkerAlreadyClosed.Generate() - } - - st := w.subTaskHolder.findSubTask(req.Task) - if st == nil { - return terror.ErrWorkerSubTaskNotFound.Generate(req.Task) - } - - info := st.DDLLockInfo() - if info == nil || info.ID != req.LockID { - return terror.ErrWorkerDDLLockInfoNotFound.Generate(req.LockID) - } - - err := st.ExecuteDDL(ctx, req) - if err == nil { - st.ClearDDLLockInfo() // remove DDL lock info - st.ClearDDLInfo() - w.l.Info("ExecuteDDL remove cacheDDLInfo") - } - return err -} - -// BreakDDLLock breaks current blocking DDL lock and/or remove current DDLLockInfo -func (w *Worker) BreakDDLLock(ctx context.Context, req *pb.BreakDDLLockRequest) error { - if w.closed.Get() == closedTrue { - return terror.ErrWorkerAlreadyClosed.Generate() - } - - st := w.subTaskHolder.findSubTask(req.Task) - if st == nil { - return terror.ErrWorkerSubTaskNotFound.Generate(req.Task) - } - - if len(req.RemoveLockID) > 0 { - info := st.DDLLockInfo() - if info == nil || info.ID != req.RemoveLockID { - return terror.ErrWorkerDDLLockInfoNotFound.Generate(req.RemoveLockID) - } - st.ClearDDLLockInfo() // remove DDL lock info - st.ClearDDLInfo() - w.l.Info("BreakDDLLock remove cacheDDLInfo") - } - - if req.ExecDDL && req.SkipDDL { - return terror.ErrWorkerExecSkipDDLConflict.Generate() - } - - if !req.ExecDDL && !req.SkipDDL { - return nil // not need to execute or skip - } - - execReq := &pb.ExecDDLRequest{ - Task: req.Task, - LockID: req.RemoveLockID, // force to operate, even if lockID mismatch - Exec: false, - } - if req.ExecDDL { - execReq.Exec = true - } - - return st.ExecuteDDL(ctx, execReq) -} - // SwitchRelayMaster switches relay unit's master server func (w *Worker) SwitchRelayMaster(ctx context.Context, req *pb.SwitchRelayMasterRequest) error { if w.closed.Get() == closedTrue { diff --git a/dm/worker/worker_test.go b/dm/worker/worker_test.go index 7da8810a3f..28ae8002dd 100644 --- a/dm/worker/worker_test.go +++ b/dm/worker/worker_test.go @@ -44,11 +44,11 @@ func (t *testServer) testWorker(c *C) { NewRelayHolder = NewRealRelayHolder }() - _, err := NewWorker(cfg) + _, err := NewWorker(cfg, nil) c.Assert(err, ErrorMatches, "init error") NewRelayHolder = NewDummyRelayHolder - w, err := NewWorker(cfg) + w, err := NewWorker(cfg, nil) c.Assert(err, IsNil) c.Assert(w.StatusJSON(""), HasLen, emptyWorkerStatusInfoJSONLength) //c.Assert(w.closed.Get(), Equals, closedFalse) diff --git a/pkg/shardddl/pessimism/doc.go b/pkg/shardddl/pessimism/doc.go new file mode 100644 index 0000000000..2fbf03b568 --- /dev/null +++ b/pkg/shardddl/pessimism/doc.go @@ -0,0 +1,67 @@ +// Copyright 2020 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +// The sequence of coordinate a shard DDL lock: +// 1. DM-master GET all history `Info` & construct/sync shard DDL locks (revision-M1). +// - also PUT the `Operation` of synced locks as step-4. +// 2. DM-worker construct `Info` based on received shard DDL & PUT the `Info` into etcd (revision-W1). +// 3. DM-master get `Info` through WATCH (after revision-M1) & try to construct/sync the `Lock`. +// 4. DM-master PUT the `Operation` into etcd (revision-M2). +// 5. DM-worker get `Operation` through WATCH (after revision-W1). +// 6. DM-worker exec the `Operation` (execute/skip the shard DDL statements). +// 7. DM-worker flush the checkpoint. +// 8. DM-worker PUT `done=true` of `Operation` & DELETE the `Info` (in one txn, revision-W2). +// 9. DM-master get `done=true` of `Operation` through WATCH (after revision-M1). +// 10. DM-master DELETE the `Operation` after all DM-worker finished the operation (revision-M3). +// +// for step-4: +// 1. DM-master PUT `exec=true` `Operation` for the shard DDL lock owner. +// 2. DM-master wait for the owner to finish the `Operation`. +// 3. DM-master PUT `exec=false` `Operation` for other DM-workers. +// +// the order of revision: +// * revision-M1 < revision-W1 < revision-M2 < revision-W2 < revision-M3. +// +// the operation on `Info`: +// * PUT & DELETE by DM-worker (in revision-W1 & revision-W2). +// * GET & WATCH by DM-master (for all revision). +// +// the operation on `Operation`: +// * PUT & DELETE by DM-master (in revision-M2 & revision-M3). +// * PUT (update) by DM-worker (in revision-W2). +// * GET by DM-worker (after revision-W1). +// * WATCH by DM-master (after revision-M1). +// +// re-entrant of DM-worker: +// * before step-6: +// * just re-run the sequence of the flow is fine. +// * in step-6: +// * operation has canceled: like `before step-6`. +// * operation has done: need to tolerate the re-execution of DDL statements, +// such as ignore `ErrColumnExists` for `ADD COLUMN`. +// TODO(csuzhangxc): flush checkpoint before execute/skip shard DDL statements. +// * in step-7: +// * operation has canceled: like `operation has done` in step-6. +// * operation has done: need to use `unlock-ddl-lock` resolve the lock manually. +// TODO(csuzhangxc): if we can find out this case correctly, it may be handled automatically later. +// * in step-8: +// * operation has canceled: like `operation has done` in step-7. +// * operation has done: just re-run the sequence of the flow is fine. +// * after step-8: +// * just re-run the sequence of the flow is fine. +// +// re-entrant of DM-master: +// * just re-run the sequence of the flow is fine. +// * do not overwrite the result of `Operation` PUTed by DM-worker in step-4. + +package pessimism diff --git a/pkg/shardddl/pessimism/info.go b/pkg/shardddl/pessimism/info.go new file mode 100644 index 0000000000..bf1b98dbe1 --- /dev/null +++ b/pkg/shardddl/pessimism/info.go @@ -0,0 +1,167 @@ +// Copyright 2020 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package pessimism + +import ( + "context" + "encoding/json" + + "go.etcd.io/etcd/clientv3" + "go.etcd.io/etcd/mvcc/mvccpb" + "go.uber.org/zap" + + "github.com/pingcap/dm/dm/common" + "github.com/pingcap/dm/pkg/etcdutil" + "github.com/pingcap/dm/pkg/log" +) + +// Info represents the shard DDL information. +// This information should be persistent in etcd so can be retrieved after the DM-master leader restarted or changed. +// NOTE: `Task` and `Source` are redundant in the etcd key path for convenient. +type Info struct { + Task string `json:"task"` // data migration task name + Source string `json:"source"` // upstream source ID + Schema string `json:"schema"` // schema name of the DDL + Table string `json:"table"` // table name of the DDL + DDLs []string `json:"ddls"` // DDL statements +} + +// NewInfo creates a new Info instance. +func NewInfo(task, source, schema, table string, DDLs []string) Info { + return Info{ + Task: task, + Source: source, + Schema: schema, + Table: table, + DDLs: DDLs, + } +} + +// String implements Stringer interface. +func (i Info) String() string { + s, _ := i.toJSON() + return s +} + +// toJSON returns the string of JSON represent. +func (i Info) toJSON() (string, error) { + data, err := json.Marshal(i) + if err != nil { + return "", err + } + return string(data), nil +} + +// infoFromJSON constructs Info from its JSON represent. +func infoFromJSON(s string) (i Info, err error) { + err = json.Unmarshal([]byte(s), &i) + return +} + +// PutInfo puts the shard DDL info into etcd. +// NOTE: +// In some cases before the lock resolved, the same DDL info may be PUT multiple times: +// 1. start-task after stop-task. +// 2. resume-task after paused manually or automatically. +// 3. the task scheduled to another DM-worker instance (just like case-1). +// Then we need to ensure re-PUT is safe: +// 1. DM-master can construct the lock and do the coordination correctly. +// 2. DM-worker can re-PUT and comply with the coordination correctly. +// This function should often be called by DM-worker. +func PutInfo(cli *clientv3.Client, info Info) (int64, error) { + value, err := info.toJSON() + if err != nil { + return 0, err + } + key := common.ShardDDLPessimismInfoKeyAdapter.Encode(info.Task, info.Source) + + ctx, cancel := context.WithTimeout(cli.Ctx(), etcdutil.DefaultRequestTimeout) + defer cancel() + + resp, err := cli.Put(ctx, key, value) + if err != nil { + return 0, err + } + return resp.Header.Revision, nil +} + +// GetAllInfo gets all shard DDL info in etcd currently. +// k/k/v: task-name -> source-ID -> DDL info. +// This function should often be called by DM-master. +func GetAllInfo(cli *clientv3.Client) (map[string]map[string]Info, int64, error) { + ctx, cancel := context.WithTimeout(cli.Ctx(), etcdutil.DefaultRequestTimeout) + defer cancel() + + resp, err := cli.Get(ctx, common.ShardDDLPessimismInfoKeyAdapter.Path(), clientv3.WithPrefix()) + if err != nil { + return nil, 0, err + } + + ifm := make(map[string]map[string]Info) + for _, kv := range resp.Kvs { + info, err2 := infoFromJSON(string(kv.Value)) + if err2 != nil { + return nil, 0, err2 + } + + if _, ok := ifm[info.Task]; !ok { + ifm[info.Task] = make(map[string]Info) + } + ifm[info.Task][info.Source] = info + } + + return ifm, resp.Header.Revision, nil +} + +// WatchInfoPut watches PUT operations for info. +// This function should often be called by DM-master. +func WatchInfoPut(ctx context.Context, cli *clientv3.Client, revision int64, outCh chan<- Info) { + ch := cli.Watch(ctx, common.ShardDDLPessimismInfoKeyAdapter.Path(), + clientv3.WithPrefix(), clientv3.WithRev(revision)) + + for { + select { + case <-ctx.Done(): + return + case resp := <-ch: + if resp.Canceled { + return + } + + for _, ev := range resp.Events { + if ev.Type != mvccpb.PUT { + continue + } + + info, err := infoFromJSON(string(ev.Kv.Value)) + if err != nil { + // this should not happen. + log.L().Error("fail to construct shard DDL info from json", zap.ByteString("json", ev.Kv.Value)) + continue + } + select { + case outCh <- info: + case <-ctx.Done(): + return + } + } + } + } +} + +// deleteInfoOp returns a DELETE etcd operation for info. +// This operation should often be sent by DM-worker. +func deleteInfoOp(info Info) clientv3.Op { + return clientv3.OpDelete(common.ShardDDLPessimismInfoKeyAdapter.Encode(info.Task, info.Source)) +} diff --git a/pkg/shardddl/pessimism/info_test.go b/pkg/shardddl/pessimism/info_test.go new file mode 100644 index 0000000000..3bf90ac81a --- /dev/null +++ b/pkg/shardddl/pessimism/info_test.go @@ -0,0 +1,152 @@ +// Copyright 2020 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package pessimism + +import ( + "context" + "sync" + "testing" + "time" + + . "github.com/pingcap/check" + "go.etcd.io/etcd/clientv3" + "go.etcd.io/etcd/integration" + + "github.com/pingcap/dm/dm/common" +) + +var ( + etcdTestCli *clientv3.Client +) + +func TestInfo(t *testing.T) { + mockCluster := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1}) + defer mockCluster.Terminate(t) + + etcdTestCli = mockCluster.RandClient() + + TestingT(t) +} + +// clear keys in etcd test cluster. +func clearTestInfoOperation(c *C) { + clearInfo := clientv3.OpDelete(common.ShardDDLPessimismInfoKeyAdapter.Path(), clientv3.WithPrefix()) + clearOp := clientv3.OpDelete(common.ShardDDLPessimismOperationKeyAdapter.Path(), clientv3.WithPrefix()) + _, err := etcdTestCli.Txn(context.Background()).Then(clearInfo, clearOp).Commit() + c.Assert(err, IsNil) +} + +type testForEtcd struct{} + +var _ = Suite(&testForEtcd{}) + +func (t *testForEtcd) TestInfoJSON(c *C) { + i1 := NewInfo("test", "mysql-replica-1", "foo", "bar", []string{ + "ALTER TABLE bar ADD COLUMN c1 INT", + "ALTER TABLE bar ADD COLUMN c2 INT", + }) + + j, err := i1.toJSON() + c.Assert(err, IsNil) + c.Assert(j, Equals, `{"task":"test","source":"mysql-replica-1","schema":"foo","table":"bar","ddls":["ALTER TABLE bar ADD COLUMN c1 INT","ALTER TABLE bar ADD COLUMN c2 INT"]}`) + c.Assert(j, Equals, i1.String()) + + i2, err := infoFromJSON(j) + c.Assert(err, IsNil) + c.Assert(i2, DeepEquals, i1) +} + +func (t *testForEtcd) TestInfoEtcd(c *C) { + defer clearTestInfoOperation(c) + + var ( + source1 = "mysql-replica-1" + source2 = "mysql-replica-2" + task1 = "task-1" + task2 = "task-2" + i11 = NewInfo(task1, source1, "foo", "bar", []string{ + "ALTER TABLE bar ADD COLUMN c1 INT", + }) + i12 = NewInfo(task1, source2, "foo", "bar", []string{ + "ALTER TABLE bar ADD COLUMN c2 INT", + }) + i21 = NewInfo(task2, source1, "foo", "bar", []string{ + "ALTER TABLE bar ADD COLUMN c3 INT", + }) + ) + + // put the same key twice. + rev1, err := PutInfo(etcdTestCli, i11) + c.Assert(err, IsNil) + rev2, err := PutInfo(etcdTestCli, i11) + c.Assert(err, IsNil) + c.Assert(rev2, Greater, rev1) + + // get with only 1 info. + ifm, rev3, err := GetAllInfo(etcdTestCli) + c.Assert(err, IsNil) + c.Assert(rev3, Equals, rev2) + c.Assert(ifm, HasLen, 1) + c.Assert(ifm, HasKey, task1) + c.Assert(ifm[task1], HasLen, 1) + c.Assert(ifm[task1][source1], DeepEquals, i11) + + // put another key and get again with 2 info. + rev4, err := PutInfo(etcdTestCli, i12) + c.Assert(err, IsNil) + ifm, _, err = GetAllInfo(etcdTestCli) + c.Assert(err, IsNil) + c.Assert(ifm, HasLen, 1) + c.Assert(ifm, HasKey, task1) + c.Assert(ifm[task1], HasLen, 2) + c.Assert(ifm[task1][source1], DeepEquals, i11) + c.Assert(ifm[task1][source2], DeepEquals, i12) + + // start the watcher. + wch := make(chan Info, 10) + var wg sync.WaitGroup + wg.Add(1) + go func() { + defer wg.Done() + ctx, cancel := context.WithTimeout(context.Background(), 500*time.Millisecond) + defer cancel() + WatchInfoPut(ctx, etcdTestCli, rev4+1, wch) // revision+1 + close(wch) // close the chan + }() + + // put another key for a different task. + _, err = PutInfo(etcdTestCli, i21) + c.Assert(err, IsNil) + wg.Wait() + + // watch should only get i21. + c.Assert(len(wch), Equals, 1) + c.Assert(<-wch, DeepEquals, i21) + + // delete i12. + deleteOp := deleteInfoOp(i12) + _, err = etcdTestCli.Txn(context.Background()).Then(deleteOp).Commit() + c.Assert(err, IsNil) + + // get again. + ifm, _, err = GetAllInfo(etcdTestCli) + c.Assert(err, IsNil) + c.Assert(ifm, HasLen, 2) + c.Assert(ifm, HasKey, task1) + c.Assert(ifm, HasKey, task2) + c.Assert(ifm[task1], HasLen, 1) + c.Assert(ifm[task1][source1], DeepEquals, i11) + c.Assert(ifm[task2], HasLen, 1) + c.Assert(ifm[task2][source1], DeepEquals, i21) +} diff --git a/pkg/shardddl/pessimism/keeper.go b/pkg/shardddl/pessimism/keeper.go new file mode 100644 index 0000000000..b589bc92e0 --- /dev/null +++ b/pkg/shardddl/pessimism/keeper.go @@ -0,0 +1,98 @@ +// Copyright 2020 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package pessimism + +import ( + "fmt" + "sync" + + "github.com/pingcap/tidb-tools/pkg/dbutil" +) + +// LockKeeper used to keep and handle DDL lock conveniently. +// The lock information do not need to be persistent, and can be re-constructed from the shard DDL info. +type LockKeeper struct { + mu sync.RWMutex + locks map[string]*Lock // lockID -> Lock +} + +// NewLockKeeper creates a new LockKeeper instance. +func NewLockKeeper() *LockKeeper { + return &LockKeeper{ + locks: make(map[string]*Lock), + } +} + +// TrySync tries to sync the lock. +func (lk *LockKeeper) TrySync(info Info, sources []string) (string, bool, int, error) { + var ( + lockID = genDDLLockID(info) + l *Lock + ok bool + ) + + lk.mu.Lock() + defer lk.mu.Unlock() + + if l, ok = lk.locks[lockID]; !ok { + lk.locks[lockID] = NewLock(lockID, info.Task, info.Source, info.DDLs, sources) + l = lk.locks[lockID] + } + + synced, remain, err := l.TrySync(info.Source, info.DDLs, sources) + return lockID, synced, remain, err +} + +// RemoveLock removes a lock. +func (lk *LockKeeper) RemoveLock(lockID string) bool { + lk.mu.Lock() + defer lk.mu.Unlock() + + _, ok := lk.locks[lockID] + delete(lk.locks, lockID) + return ok +} + +// FindLock finds a lock. +func (lk *LockKeeper) FindLock(lockID string) *Lock { + lk.mu.RLock() + defer lk.mu.RUnlock() + + return lk.locks[lockID] +} + +// Locks return a copy of all Locks. +func (lk *LockKeeper) Locks() map[string]*Lock { + lk.mu.RLock() + defer lk.mu.RUnlock() + + locks := make(map[string]*Lock, len(lk.locks)) + for k, v := range lk.locks { + locks[k] = v + } + return locks +} + +// Clear clears all Locks. +func (lk *LockKeeper) Clear() { + lk.mu.Lock() + defer lk.mu.Unlock() + + lk.locks = make(map[string]*Lock) +} + +// genDDLLockID generates DDL lock ID from its info. +func genDDLLockID(info Info) string { + return fmt.Sprintf("%s-%s", info.Task, dbutil.TableName(info.Schema, info.Table)) +} diff --git a/pkg/shardddl/pessimism/keeper_test.go b/pkg/shardddl/pessimism/keeper_test.go new file mode 100644 index 0000000000..da532c9438 --- /dev/null +++ b/pkg/shardddl/pessimism/keeper_test.go @@ -0,0 +1,84 @@ +// Copyright 2020 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package pessimism + +import ( + . "github.com/pingcap/check" +) + +type testLockKeeper struct{} + +var _ = Suite(&testLockKeeper{}) + +func (t *testLockKeeper) TestLockKeeper(c *C) { + var ( + lk = NewLockKeeper() + schema = "foo" + table = "bar" + DDLs = []string{"ALTER TABLE bar ADD COLUMN c1 INT"} + task1 = "task1" + task2 = "task2" + source1 = "mysql-replica-1" + source2 = "mysql-replica-2" + info11 = NewInfo(task1, source1, schema, table, DDLs) + info12 = NewInfo(task1, source2, schema, table, DDLs) + info21 = NewInfo(task2, source1, schema, table, DDLs) + ) + + // lock with 2 sources. + lockID1, synced, remain, err := lk.TrySync(info11, []string{source1, source2}) + c.Assert(err, IsNil) + c.Assert(lockID1, Equals, "task1-`foo`.`bar`") + c.Assert(synced, IsFalse) + c.Assert(remain, Equals, 1) + lockID1, synced, remain, err = lk.TrySync(info12, []string{source1, source2}) + c.Assert(err, IsNil) + c.Assert(lockID1, Equals, "task1-`foo`.`bar`") + c.Assert(synced, IsTrue) + c.Assert(remain, Equals, 0) + + // lock with only 1 source. + lockID2, synced, remain, err := lk.TrySync(info21, []string{source1}) + c.Assert(err, IsNil) + c.Assert(lockID2, Equals, "task2-`foo`.`bar`") + c.Assert(synced, IsTrue) + c.Assert(remain, Equals, 0) + + // find lock. + lock1 := lk.FindLock(lockID1) + c.Assert(lock1, NotNil) + c.Assert(lock1.ID, Equals, lockID1) + lock2 := lk.FindLock(lockID2) + c.Assert(lock2, NotNil) + c.Assert(lock2.ID, Equals, lockID2) + lockIDNotExists := "lock-not-exists" + c.Assert(lk.FindLock(lockIDNotExists), IsNil) + + // all locks. + locks := lk.Locks() + c.Assert(locks, HasLen, 2) + c.Assert(locks[lockID1], Equals, lock1) // compare pointer + c.Assert(locks[lockID2], Equals, lock2) + + // remove lock. + c.Assert(lk.RemoveLock(lockID1), IsTrue) + c.Assert(lk.RemoveLock(lockIDNotExists), IsFalse) + c.Assert(lk.Locks(), HasLen, 1) + + // clear locks. + lk.Clear() + + // no locks exist. + c.Assert(lk.Locks(), HasLen, 0) +} diff --git a/pkg/shardddl/pessimism/lock.go b/pkg/shardddl/pessimism/lock.go new file mode 100644 index 0000000000..a4883960b4 --- /dev/null +++ b/pkg/shardddl/pessimism/lock.go @@ -0,0 +1,153 @@ +// Copyright 2020 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package pessimism + +import ( + "sync" + + "github.com/pingcap/dm/pkg/terror" + "github.com/pingcap/dm/pkg/utils" +) + +// Lock represents the shard DDL lock in memory. +// This information does not need to be persistent, and can be re-constructed from the shard DDL info. +type Lock struct { + mu sync.RWMutex + + ID string // lock's ID + Task string // lock's corresponding task name + Owner string // Owner's source ID (not DM-worker's name) + DDLs []string // DDL statements + remain int // remain count of sources needed to receive DDL info + + // whether the DDL info received from the source. + // if all of them have been ready, then we call the lock `synced`. + ready map[string]bool + + // whether the operations have done (exec/skip the shard DDL). + // if all of them have done, then we call the lock `resolved`. + done map[string]bool +} + +// NewLock creates a new Lock instance. +func NewLock(ID, task, owner string, DDLs, sources []string) *Lock { + l := &Lock{ + ID: ID, + Task: task, + Owner: owner, + DDLs: DDLs, + remain: len(sources), + ready: make(map[string]bool), + done: make(map[string]bool), + } + for _, s := range sources { + l.ready[s] = false + l.done[s] = false + } + + return l +} + +// TrySync tries to sync the lock, does decrease on remain, re-entrant. +// new upstream sources may join when the DDL lock is in syncing, +// so we need to merge these new sources. +func (l *Lock) TrySync(caller string, DDLs, sources []string) (bool, int, error) { + l.mu.Lock() + defer l.mu.Unlock() + + // check DDL statement first. + if !utils.CompareShardingDDLs(DDLs, l.DDLs) { + return l.remain <= 0, l.remain, terror.ErrMasterShardingDDLDiff.Generate(l.DDLs, DDLs) + } + + // try to merge any newly joined sources. + for _, s := range sources { + if _, ok := l.ready[s]; !ok { + l.remain++ + l.ready[s] = false + l.done[s] = false // mark as not-done for newly joined sources. + } + } + + // only `sync` once. + if synced, ok := l.ready[caller]; ok && !synced { + l.remain-- + l.ready[caller] = true + } + + return l.remain <= 0, l.remain, nil +} + +// ForceSynced forces to mark the lock as synced. +func (l *Lock) ForceSynced() { + l.mu.Lock() + defer l.mu.Unlock() + + for source := range l.ready { + l.ready[source] = true + } + l.remain = 0 +} + +// IsSynced returns whether the lock has synced. +func (l *Lock) IsSynced() (bool, int) { + l.mu.RLock() + defer l.mu.RUnlock() + return l.remain <= 0, l.remain +} + +// Ready returns the sources sync status or whether they are ready. +func (l *Lock) Ready() map[string]bool { + l.mu.RLock() + defer l.mu.RUnlock() + + ret := make(map[string]bool, len(l.ready)) + for k, v := range l.ready { + ret[k] = v + } + return ret +} + +// MarkDone marks the operation of the source as done. +// NOTE: we do not support revert the `done` after marked now. +func (l *Lock) MarkDone(source string) { + l.mu.Lock() + defer l.mu.Unlock() + + if _, ok := l.done[source]; !ok { + return // do not add it if not exists. + } + l.done[source] = true +} + +// IsDone returns whether the operation has done. +func (l *Lock) IsDone(source string) bool { + l.mu.RLock() + defer l.mu.RUnlock() + + return l.done[source] +} + +// IsResolved returns whether the lock has resolved (all operations have done). +func (l *Lock) IsResolved() bool { + l.mu.RLock() + defer l.mu.RUnlock() + + for _, done := range l.done { + if !done { + return false + } + } + return true +} diff --git a/pkg/shardddl/pessimism/lock_test.go b/pkg/shardddl/pessimism/lock_test.go new file mode 100644 index 0000000000..9390b44d28 --- /dev/null +++ b/pkg/shardddl/pessimism/lock_test.go @@ -0,0 +1,132 @@ +// Copyright 2020 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package pessimism + +import ( + . "github.com/pingcap/check" + + "github.com/pingcap/dm/pkg/terror" +) + +type testLock struct{} + +var _ = Suite(&testLock{}) + +func (t *testLock) TestLock(c *C) { + var ( + ID = "test-`foo`.`bar`" + task = "test" + source1 = "mysql-replica-1" + source2 = "mysql-replica-2" + source3 = "mysql-replica-3" + DDLs = []string{ + "ALTER TABLE bar ADD COLUMN c1 INT", + "ALTER TABLE bar ADD COLUMN c2 INT", + } + ) + + // create the lock with only 1 source. + l1 := NewLock(ID, task, source1, DDLs, []string{source1}) + + // DDLs mismatch. + synced, remain, err := l1.TrySync(source1, DDLs[1:], []string{source1}) + c.Assert(terror.ErrMasterShardingDDLDiff.Equal(err), IsTrue) + c.Assert(synced, IsFalse) + c.Assert(remain, Equals, 1) + c.Assert(l1.Ready(), DeepEquals, map[string]bool{source1: false}) + synced, _ = l1.IsSynced() + c.Assert(synced, IsFalse) + c.Assert(l1.IsDone(source1), IsFalse) + c.Assert(l1.IsResolved(), IsFalse) + + // synced. + synced, remain, err = l1.TrySync(source1, DDLs, []string{source1}) + c.Assert(err, IsNil) + c.Assert(synced, IsTrue) + c.Assert(remain, Equals, 0) + c.Assert(l1.Ready(), DeepEquals, map[string]bool{source1: true}) + synced, _ = l1.IsSynced() + c.Assert(synced, IsTrue) + c.Assert(l1.IsDone(source1), IsFalse) + c.Assert(l1.IsResolved(), IsFalse) + + // mark done. + l1.MarkDone(source1) + c.Assert(l1.IsDone(source1), IsTrue) + c.Assert(l1.IsResolved(), IsTrue) + + // create the lock with 2 sources. + l2 := NewLock(ID, task, source1, DDLs, []string{source1, source2}) + + // join a new source. + synced, remain, err = l2.TrySync(source1, DDLs, []string{source2, source3}) + c.Assert(err, IsNil) + c.Assert(synced, IsFalse) + c.Assert(remain, Equals, 2) + c.Assert(l2.Ready(), DeepEquals, map[string]bool{ + source1: true, + source2: false, + source3: false, + }) + + // sync other sources. + synced, remain, err = l2.TrySync(source2, DDLs, []string{}) + c.Assert(err, IsNil) + c.Assert(synced, IsFalse) + c.Assert(remain, Equals, 1) + c.Assert(l2.Ready(), DeepEquals, map[string]bool{ + source1: true, + source2: true, + source3: false, + }) + synced, remain, err = l2.TrySync(source3, DDLs, nil) + c.Assert(err, IsNil) + c.Assert(synced, IsTrue) + c.Assert(remain, Equals, 0) + c.Assert(l2.Ready(), DeepEquals, map[string]bool{ + source1: true, + source2: true, + source3: true, + }) + + // done none. + c.Assert(l2.IsDone(source1), IsFalse) + c.Assert(l2.IsDone(source2), IsFalse) + c.Assert(l2.IsDone(source3), IsFalse) + c.Assert(l2.IsResolved(), IsFalse) + + // done some. + l2.MarkDone(source1) + l2.MarkDone(source2) + c.Assert(l2.IsDone(source1), IsTrue) + c.Assert(l2.IsDone(source2), IsTrue) + c.Assert(l2.IsDone(source3), IsFalse) + c.Assert(l2.IsResolved(), IsFalse) + + // done all. + l2.MarkDone(source3) + c.Assert(l2.IsDone(source3), IsTrue) + c.Assert(l2.IsResolved(), IsTrue) + + // mark on not existing source has no effect. + l2.MarkDone("not-exist-source") + c.Assert(l2.IsResolved(), IsTrue) + + // create the lock with 2 sources. + l3 := NewLock(ID, task, source1, DDLs, []string{source1, source2}) + l3.ForceSynced() + synced, remain = l3.IsSynced() + c.Assert(synced, IsTrue) + c.Assert(remain, Equals, 0) +} diff --git a/pkg/shardddl/pessimism/operation.go b/pkg/shardddl/pessimism/operation.go new file mode 100644 index 0000000000..12506e283a --- /dev/null +++ b/pkg/shardddl/pessimism/operation.go @@ -0,0 +1,226 @@ +// Copyright 2020 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package pessimism + +import ( + "context" + "encoding/json" + + "go.etcd.io/etcd/clientv3" + "go.etcd.io/etcd/clientv3/clientv3util" + "go.etcd.io/etcd/mvcc/mvccpb" + "go.uber.org/zap" + + "github.com/pingcap/dm/dm/common" + "github.com/pingcap/dm/pkg/etcdutil" + "github.com/pingcap/dm/pkg/log" +) + +// Operation represents a shard DDL coordinate operation. +// This information should be persistent in etcd so can be retrieved after the DM-master leader restarted or changed. +// NOTE: `Task` and `Source` are redundant in the etcd key path for convenient. +type Operation struct { + ID string `json:"id"` // the corresponding DDL lock ID + Task string `json:"task"` // data migration task name + Source string `json:"source"` // upstream source ID + DDLs []string `json:"ddls"` // DDL statements + Exec bool `json:"exec"` // execute or skip the DDL statements + Done bool `json:"done"` // whether the `Exec` operation has done +} + +// NewOperation creates a new Operation instance. +func NewOperation(ID, task, source string, DDLs []string, exec, done bool) Operation { + return Operation{ + ID: ID, + Task: task, + Source: source, + DDLs: DDLs, + Exec: exec, + Done: done, + } +} + +// String implements Stringer interface. +func (o Operation) String() string { + s, _ := o.toJSON() + return s +} + +// toJSON returns the string of JSON represent. +func (o Operation) toJSON() (string, error) { + data, err := json.Marshal(o) + if err != nil { + return "", err + } + return string(data), nil +} + +// operationFromJSON constructs Operation from its JSON represent. +func operationFromJSON(s string) (o Operation, err error) { + err = json.Unmarshal([]byte(s), &o) + return +} + +// PutOperations puts the shard DDL operations into etcd. +// if `skipDone` is `true`: +// - PUT: all of kvs ("not exist" or "the `done` field is not `true`") +// - skip PUT: any of kvs ("exist" and "the `done` field is `true`") +// NOTE: +// `clientv3.Value` has a strange behavior for *not-exist* kv, +// see https://github.com/etcd-io/etcd/issues/10566. +// In addition, etcd compare has no `OR` operator now, +// see https://github.com/etcd-io/etcd/issues/10571. +// So, it's hard to do our `skipDone` logic in one txn. +// We break the logic into two txn, but this may lead to problem when PUT operations concurrently. +// This function should often be called by DM-master. +func PutOperations(cli *clientv3.Client, skipDone bool, ops ...Operation) (rev int64, putted bool, err error) { + cmpsNotExist := make([]clientv3.Cmp, 0, len(ops)) + cmpsNotDone := make([]clientv3.Cmp, 0, len(ops)) + opsPut := make([]clientv3.Op, 0, len(ops)) + for _, op := range ops { + value, err2 := op.toJSON() + if err2 != nil { + return 0, false, err2 + } + + key := common.ShardDDLPessimismOperationKeyAdapter.Encode(op.Task, op.Source) + opsPut = append(opsPut, clientv3.OpPut(key, value)) + + if skipDone { + opDone := op + opDone.Done = true // set `done` to `true`. + valueDone, err3 := opDone.toJSON() + if err3 != nil { + return 0, false, err3 + } + cmpsNotExist = append(cmpsNotExist, clientv3util.KeyMissing(key)) + cmpsNotDone = append(cmpsNotDone, clientv3.Compare(clientv3.Value(key), "!=", valueDone)) + } + } + + ctx, cancel := context.WithTimeout(cli.Ctx(), etcdutil.DefaultRequestTimeout) + defer cancel() + + // txn 1: try to PUT if all of kvs "not exist". + resp, err := cli.Txn(ctx).If(cmpsNotExist...).Then(opsPut...).Commit() + if err != nil { + return 0, false, err + } else if resp.Succeeded { + return resp.Header.Revision, resp.Succeeded, nil + } + + // txn 2: try to PUT if all of kvs "the `done` field is not `true`. + // FIXME: if any "not `done`" kv putted after txn 1, this txn 2 will fail, but this is not what we want. + resp, err = cli.Txn(ctx).If(cmpsNotDone...).Then(opsPut...).Commit() + if err != nil { + return 0, false, err + } + return resp.Header.Revision, resp.Succeeded, nil +} + +// DeleteOperations deletes the shard DDL operations in etcd. +// This function should often be called by DM-master. +func DeleteOperations(cli *clientv3.Client, ops ...Operation) (int64, error) { + opsDel := make([]clientv3.Op, 0, len(ops)) + for _, op := range ops { + key := common.ShardDDLPessimismOperationKeyAdapter.Encode(op.Task, op.Source) + opsDel = append(opsDel, clientv3.OpDelete(key)) + } + + ctx, cancel := context.WithTimeout(cli.Ctx(), etcdutil.DefaultRequestTimeout) + defer cancel() + + resp, err := cli.Txn(ctx).Then(opsDel...).Commit() + if err != nil { + return 0, err + } + return resp.Header.Revision, nil +} + +// GetAllOperations gets all DDL lock operation in etcd currently. +// k/k/v: task-name -> source-ID -> lock operation. +func GetAllOperations(cli *clientv3.Client) (map[string]map[string]Operation, int64, error) { + ctx, cancel := context.WithTimeout(cli.Ctx(), etcdutil.DefaultRequestTimeout) + defer cancel() + + resp, err := cli.Get(ctx, common.ShardDDLPessimismOperationKeyAdapter.Path(), clientv3.WithPrefix()) + if err != nil { + return nil, 0, err + } + + opm := make(map[string]map[string]Operation) + for _, kv := range resp.Kvs { + op, err2 := operationFromJSON(string(kv.Value)) + if err2 != nil { + return nil, 0, err2 + } + + if _, ok := opm[op.Task]; !ok { + opm[op.Task] = make(map[string]Operation) + } + opm[op.Task][op.Source] = op + } + + return opm, resp.Header.Revision, nil +} + +// WatchOperationPut watches PUT operations for DDL lock operation. +// If want to watch all operations, pass empty string for `task` and `source`. +// This function can be called by DM-worker and DM-master. +// TODO(csuzhangxc): report error and do some retry. +func WatchOperationPut(ctx context.Context, cli *clientv3.Client, task, source string, revision int64, outCh chan<- Operation) { + ch := cli.Watch(ctx, common.ShardDDLPessimismOperationKeyAdapter.Encode(task, source), + clientv3.WithPrefix(), clientv3.WithRev(revision)) + + for { + select { + case <-ctx.Done(): + return + case resp := <-ch: + if resp.Canceled { + return + } + + for _, ev := range resp.Events { + if ev.Type != mvccpb.PUT { + continue + } + + op, err := operationFromJSON(string(ev.Kv.Value)) + if err != nil { + // this should not happen. + log.L().Error("fail to construct shard DDL operation from json", zap.ByteString("json", ev.Kv.Value)) + continue + } + select { + case outCh <- op: + case <-ctx.Done(): + return + } + } + } + } +} + +// putOperationOp returns a PUT etcd operation for Operation. +// This operation should often be sent by DM-worker. +func putOperationOp(o Operation) (clientv3.Op, error) { + value, err := o.toJSON() + if err != nil { + return clientv3.Op{}, err + } + key := common.ShardDDLPessimismOperationKeyAdapter.Encode(o.Task, o.Source) + + return clientv3.OpPut(key, value), nil +} diff --git a/pkg/shardddl/pessimism/operation_test.go b/pkg/shardddl/pessimism/operation_test.go new file mode 100644 index 0000000000..edc1d594ee --- /dev/null +++ b/pkg/shardddl/pessimism/operation_test.go @@ -0,0 +1,177 @@ +// Copyright 2020 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package pessimism + +import ( + "context" + "time" + + . "github.com/pingcap/check" +) + +func (t *testForEtcd) TestOperationJSON(c *C) { + o1 := NewOperation("test-ID", "test", "mysql-replica-1", []string{ + "ALTER TABLE bar ADD COLUMN c1 INT", + }, true, false) + + j, err := o1.toJSON() + c.Assert(err, IsNil) + c.Assert(j, Equals, `{"id":"test-ID","task":"test","source":"mysql-replica-1","ddls":["ALTER TABLE bar ADD COLUMN c1 INT"],"exec":true,"done":false}`) + c.Assert(j, Equals, o1.String()) + + o2, err := operationFromJSON(j) + c.Assert(err, IsNil) + c.Assert(o2, DeepEquals, o1) +} + +func (t *testForEtcd) TestOperationEtcd(c *C) { + defer clearTestInfoOperation(c) + + var ( + task1 = "test1" + task2 = "test2" + ID1 = "test1-`foo`.`bar`" + ID2 = "test2-`foo`.`bar`" + source1 = "mysql-replica-1" + source2 = "mysql-replica-2" + source3 = "mysql-replica-3" + DDLs = []string{"ALTER TABLE bar ADD COLUMN c1 INT"} + op11 = NewOperation(ID1, task1, source1, DDLs, true, false) + op12 = NewOperation(ID1, task1, source2, DDLs, true, false) + op13 = NewOperation(ID1, task1, source3, DDLs, true, false) + op21 = NewOperation(ID2, task2, source1, DDLs, false, true) + ) + + // put the same keys twice. + rev1, succ, err := PutOperations(etcdTestCli, false, op11, op12) + c.Assert(err, IsNil) + c.Assert(succ, IsTrue) + rev2, succ, err := PutOperations(etcdTestCli, false, op11, op12) + c.Assert(err, IsNil) + c.Assert(succ, IsTrue) + c.Assert(rev2, Greater, rev1) + + // start the watcher with the same revision as the last PUT for the specified task and source. + wch := make(chan Operation, 10) + ctx, cancel := context.WithTimeout(context.Background(), 500*time.Millisecond) + WatchOperationPut(ctx, etcdTestCli, task1, source1, rev2, wch) + cancel() + close(wch) + + // watch should only get op11. + c.Assert(len(wch), Equals, 1) + c.Assert(<-wch, DeepEquals, op11) + + // put for another task. + rev3, succ, err := PutOperations(etcdTestCli, false, op21) + c.Assert(err, IsNil) + c.Assert(succ, IsTrue) + + // start the watch with an older revision for all tasks and sources. + wch = make(chan Operation, 10) + ctx, cancel = context.WithTimeout(context.Background(), 500*time.Millisecond) + WatchOperationPut(ctx, etcdTestCli, "", "", rev2, wch) + cancel() + close(wch) + + // watch should get 3 operations. + c.Assert(len(wch), Equals, 3) + c.Assert(<-wch, DeepEquals, op11) + c.Assert(<-wch, DeepEquals, op12) + c.Assert(<-wch, DeepEquals, op21) + + // get all operations. + opm, rev4, err := GetAllOperations(etcdTestCli) + c.Assert(err, IsNil) + c.Assert(rev4, Equals, rev3) + c.Assert(opm, HasLen, 2) + c.Assert(opm, HasKey, task1) + c.Assert(opm, HasKey, task2) + c.Assert(opm[task1], HasLen, 2) + c.Assert(opm[task1][source1], DeepEquals, op11) + c.Assert(opm[task1][source2], DeepEquals, op12) + c.Assert(opm[task2], HasLen, 1) + c.Assert(opm[task2][source1], DeepEquals, op21) + + // put for `skipDone` with `done` in etcd, the operations should not be skipped. + // case: all of kvs "the `done` field is not `true`". + rev5, succ, err := PutOperations(etcdTestCli, true, op11, op12) + c.Assert(err, IsNil) + c.Assert(succ, IsTrue) + c.Assert(rev5, Greater, rev4) + + // delete op11. + rev6, err := DeleteOperations(etcdTestCli, op11) + c.Assert(err, IsNil) + c.Assert(rev6, Greater, rev5) + + // get again, op11 should be deleted. + opm, _, err = GetAllOperations(etcdTestCli) + c.Assert(err, IsNil) + c.Assert(opm[task1], HasLen, 1) + c.Assert(opm[task1][source2], DeepEquals, op12) + + // put for `skipDone` with `done` in etcd, the operations should not be skipped. + // case: all of kvs "not exist". + rev7, succ, err := PutOperations(etcdTestCli, true, op11, op13) + c.Assert(err, IsNil) + c.Assert(succ, IsTrue) + c.Assert(rev7, Greater, rev6) + + // get again, op11 and op13 should be putted. + opm, _, err = GetAllOperations(etcdTestCli) + c.Assert(err, IsNil) + c.Assert(opm[task1], HasLen, 3) + c.Assert(opm[task1][source1], DeepEquals, op11) + c.Assert(opm[task1][source2], DeepEquals, op12) + c.Assert(opm[task1][source3], DeepEquals, op13) + + // update op12 to `done`. + op12c := op12 + op12c.Done = true + putOp, err := putOperationOp(op12c) + c.Assert(err, IsNil) + txnResp, err := etcdTestCli.Txn(context.Background()).Then(putOp).Commit() + c.Assert(err, IsNil) + + // delete op13. + rev8, err := DeleteOperations(etcdTestCli, op13) + c.Assert(err, IsNil) + c.Assert(rev8, Greater, txnResp.Header.Revision) + + // put for `skipDone` with `done` in etcd, the operations should be skipped. + // case: any of kvs ("exist" and "the `done` field is `true`"). + rev9, succ, err := PutOperations(etcdTestCli, true, op12, op13) + c.Assert(err, IsNil) + c.Assert(succ, IsFalse) + c.Assert(rev9, Equals, rev8) + + // get again, op13 not putted. + opm, _, err = GetAllOperations(etcdTestCli) + c.Assert(err, IsNil) + c.Assert(opm[task1], HasLen, 2) + c.Assert(opm[task1][source1], DeepEquals, op11) + c.Assert(opm[task1][source2], DeepEquals, op12c) + + // FIXME: the right result: + // the operations should *NOT* be skipped. + // case: + // - some of kvs "exist" and "the `done` field is not `true`" + // - some of kvs "not exist" + // after FIXED, this test case will fail and need to be updated. + rev10, succ, err := PutOperations(etcdTestCli, true, op11, op13) + c.Assert(err, IsNil) + c.Assert(succ, IsFalse) + c.Assert(rev10, Equals, rev9) +} diff --git a/pkg/shardddl/pessimism/ops.go b/pkg/shardddl/pessimism/ops.go new file mode 100644 index 0000000000..6eb0e132b6 --- /dev/null +++ b/pkg/shardddl/pessimism/ops.go @@ -0,0 +1,43 @@ +// Copyright 2020 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package pessimism + +import ( + "context" + + "go.etcd.io/etcd/clientv3" + + "github.com/pingcap/dm/pkg/etcdutil" +) + +// TODO(csuzhangxc): assign terror code before merged into the master branch. + +// PutOperationDeleteInfo puts an operation and deletes an info in one txn. +// This function should often be called by DM-worker. +func PutOperationDeleteInfo(cli *clientv3.Client, op Operation, info Info) (int64, error) { + putOp, err := putOperationOp(op) + if err != nil { + return 0, nil + } + delOp := deleteInfoOp(info) + + ctx, cancel := context.WithTimeout(cli.Ctx(), etcdutil.DefaultRequestTimeout) + defer cancel() + + resp, err := cli.Txn(ctx).Then(putOp, delOp).Commit() + if err != nil { + return 0, err + } + return resp.Header.Revision, nil +} diff --git a/pkg/shardddl/pessimism/ops_test.go b/pkg/shardddl/pessimism/ops_test.go new file mode 100644 index 0000000000..d0e6ebbbaf --- /dev/null +++ b/pkg/shardddl/pessimism/ops_test.go @@ -0,0 +1,62 @@ +// Copyright 2020 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package pessimism + +import ( + . "github.com/pingcap/check" +) + +func (t *testForEtcd) TestPutOperationDeleteInfo(c *C) { + defer clearTestInfoOperation(c) + + var ( + task = "test" + source = "mysql-replica-1" + DDLs = []string{"ALTER TABLE bar ADD COLUMN c1 INT"} + info = NewInfo(task, source, "foo", "bar", DDLs) + op = NewOperation("test-ID", task, source, DDLs, true, false) + ) + + // put info. + _, err := PutInfo(etcdTestCli, info) + c.Assert(err, IsNil) + + // verify the info exists. + ifm, _, err := GetAllInfo(etcdTestCli) + c.Assert(err, IsNil) + c.Assert(ifm, HasLen, 1) + c.Assert(ifm, HasKey, task) + c.Assert(ifm[task][source], DeepEquals, info) + + // verify no operations exist. + opm, _, err := GetAllOperations(etcdTestCli) + c.Assert(err, IsNil) + c.Assert(opm, HasLen, 0) + + // put operation & delete info. + _, err = PutOperationDeleteInfo(etcdTestCli, op, info) + c.Assert(err, IsNil) + + // verify no info exit. + ifm, _, err = GetAllInfo(etcdTestCli) + c.Assert(err, IsNil) + c.Assert(ifm, HasLen, 0) + + // verify the operation exists. + opm, _, err = GetAllOperations(etcdTestCli) + c.Assert(err, IsNil) + c.Assert(opm, HasLen, 1) + c.Assert(opm, HasKey, task) + c.Assert(opm[task][source], DeepEquals, op) +} diff --git a/pkg/tracing/tracer_syncer.go b/pkg/tracing/tracer_syncer.go index e9a5e93402..9e7073ceb9 100644 --- a/pkg/tracing/tracer_syncer.go +++ b/pkg/tracing/tracer_syncer.go @@ -64,7 +64,7 @@ func (t *Tracer) FinishedSyncerJobState(err error) pb.SyncerJobState { } // CollectSyncerJobEvent collects syncer job event and returns traceID -func (t *Tracer) CollectSyncerJobEvent(traceID string, traceGID string, opType int32, pos, currentPos mysql.Position, queueBucket, sql string, ddls []string, args []interface{}, req *pb.ExecDDLRequest, state pb.SyncerJobState) (*pb.SyncerJobEvent, error) { +func (t *Tracer) CollectSyncerJobEvent(traceID string, traceGID string, opType int32, pos, currentPos mysql.Position, queueBucket, sql string, ddls []string, args []interface{}, state pb.SyncerJobState) (*pb.SyncerJobEvent, error) { base, err := t.collectBaseEvent("", traceID, traceGID, pb.TraceType_JobEvent) if err != nil { return nil, err @@ -83,12 +83,6 @@ func (t *Tracer) CollectSyncerJobEvent(traceID string, traceGID string, opType i if t.cfg.Checksum && args != nil { event.ArgsChecksum, _ = DataChecksum(args) } - if req != nil { - event.DdlInfo = &pb.ExecDDLInfo{ - LockID: req.LockID, - Exec: req.Exec, - } - } job := &Job{ Tp: EventSyncerJob, Event: event, diff --git a/pkg/tracing/tracer_test.go b/pkg/tracing/tracer_test.go index 9e531a5b22..7f7a78c617 100644 --- a/pkg/tracing/tracer_test.go +++ b/pkg/tracing/tracer_test.go @@ -286,12 +286,12 @@ ForEnd: ts.server.CheckEvent(event.Base.TraceID, &TraceEvent{Type: pb.TraceType_BinlogEvent, Event: event}, c) event2, err2 := ts.tracer.CollectSyncerJobEvent(event.Base.TraceID, "", 1, mysql.Position{Name: "bin|000001.000004", Pos: 1626}, mysql.Position{Name: "bin|000001.000004", Pos: 1873}, - "q_1", "REPLACE INTO `test`.`t_target` (`id`,`ct`,`name`) VALUES (?,?,?);", []string{}, []interface{}{1, "2019-03-12 12:13:00", "test"}, nil, pb.SyncerJobState_queued) + "q_1", "REPLACE INTO `test`.`t_target` (`id`,`ct`,`name`) VALUES (?,?,?);", []string{}, []interface{}{1, "2019-03-12 12:13:00", "test"}, pb.SyncerJobState_queued) c.Assert(err2, tc.IsNil) ts.server.CheckEvent(event2.Base.TraceID, &TraceEvent{Type: pb.TraceType_JobEvent, Event: event2}, c) event2, err2 = ts.tracer.CollectSyncerJobEvent(event.Base.TraceID, "", 1, mysql.Position{Name: "bin|000001.000004", Pos: 1626}, mysql.Position{Name: "bin|000001.000004", Pos: 1873}, - "q_1", "REPLACE INTO `test`.`t_target` (`id`,`ct`,`name`) VALUES (?,?,?);", []string{}, []interface{}{1, "2019-03-12 12:13:00", "test"}, nil, pb.SyncerJobState_success) + "q_1", "REPLACE INTO `test`.`t_target` (`id`,`ct`,`name`) VALUES (?,?,?);", []string{}, []interface{}{1, "2019-03-12 12:13:00", "test"}, pb.SyncerJobState_success) c.Assert(err2, tc.IsNil) ts.server.CheckEvent(event2.Base.TraceID, &TraceEvent{Type: pb.TraceType_JobEvent, Event: event2}, c) diff --git a/syncer/db_test.go b/syncer/db_test.go index 008cb217f4..d435d55612 100644 --- a/syncer/db_test.go +++ b/syncer/db_test.go @@ -236,7 +236,7 @@ func (s *testDBSuite) TestTimezone(c *C) { for _, testCase := range testCases { s.cfg.Timezone = testCase.timezone - syncer := NewSyncer(s.cfg) + syncer := NewSyncer(s.cfg, nil) syncer.genRouter() s.resetBinlogSyncer(c) diff --git a/syncer/ddl_exec_info.go b/syncer/ddl_exec_info.go deleted file mode 100644 index 650fae942f..0000000000 --- a/syncer/ddl_exec_info.go +++ /dev/null @@ -1,159 +0,0 @@ -// Copyright 2019 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 syncer - -import ( - "context" - "sync" - "time" - - "github.com/siddontang/go/sync2" - - "github.com/pingcap/dm/dm/pb" - "github.com/pingcap/dm/pkg/terror" -) - -var ( - ddlExecIdle = "idle" - ddlExecSending = "sending" - ddlExecClosed = "closed" -) - -// DDLExecItem wraps request and response for a sharding DDL execution -type DDLExecItem struct { - req *pb.ExecDDLRequest - resp chan error -} - -// newDDLExecItem creates a new DDLExecItem -func newDDLExecItem(req *pb.ExecDDLRequest) *DDLExecItem { - item := &DDLExecItem{ - req: req, - resp: make(chan error, 1), // one elem buffered - } - return item -} - -// DDLExecInfo used by syncer to execute or ignore sharding DDL -// it's specific to syncer, and can not be used by other process unit -type DDLExecInfo struct { - sync.RWMutex - status sync2.AtomicString - ch chan *DDLExecItem // item.req.Exec: true for exec, false for ignore - cancel chan struct{} // chan used to cancel sending - ddls []string // DDL which is blocking -} - -// NewDDLExecInfo creates a new DDLExecInfo -func NewDDLExecInfo() *DDLExecInfo { - i := &DDLExecInfo{ - ch: make(chan *DDLExecItem), // un-buffered - cancel: make(chan struct{}), - } - i.status.Set(ddlExecIdle) - return i -} - -// Renew renews the chan -func (i *DDLExecInfo) Renew() { - i.Lock() - defer i.Unlock() - - i.cancelAndWaitSending() - - if i.status.Get() != ddlExecClosed { - close(i.ch) - } - - i.ch = make(chan *DDLExecItem) - i.cancel = make(chan struct{}) - i.ddls = nil - i.status.Set(ddlExecIdle) -} - -// Close closes the chan -func (i *DDLExecInfo) Close() { - i.Lock() - defer i.Unlock() - if i.status.CompareAndSwap(ddlExecClosed, ddlExecClosed) { - return - } - - i.cancelAndWaitSending() - - close(i.ch) - i.ddls = nil - i.status.Set(ddlExecClosed) -} - -// NOTE: in caller, should do lock -func (i *DDLExecInfo) cancelAndWaitSending() { - // close the un-closed cancel chan - select { - case <-i.cancel: - default: - close(i.cancel) - } - - // wait Send to return - timer := time.NewTicker(1 * time.Millisecond) - defer timer.Stop() - for range timer.C { - if !i.status.CompareAndSwap(ddlExecSending, ddlExecSending) { - return // return from select and for - } - } -} - -// Send sends an item (with request) to the chan -func (i *DDLExecInfo) Send(ctx context.Context, item *DDLExecItem) error { - i.RLock() - if !i.status.CompareAndSwap(ddlExecIdle, ddlExecSending) { - i.RUnlock() - return terror.ErrSyncerUnitDDLExecChanCloseOrBusy.Generate() - } - i.RUnlock() - defer i.status.Set(ddlExecIdle) - - select { - case <-ctx.Done(): - return terror.ErrSyncerUnitDDLChanDone.Generate() - case i.ch <- item: - return nil - case <-i.cancel: - return terror.ErrSyncerUnitDDLChanCanceled.Generate() - } -} - -// Chan returns a receive only DDLExecItem chan -func (i *DDLExecInfo) Chan(ddls []string) <-chan *DDLExecItem { - i.Lock() - i.ddls = ddls - i.Unlock() - return i.ch -} - -// BlockingDDLs returns current blocking DDL -func (i *DDLExecInfo) BlockingDDLs() []string { - i.RLock() - defer i.RUnlock() - return i.ddls -} - -// ClearBlockingDDL clears current blocking DDL -func (i *DDLExecInfo) ClearBlockingDDL() { - i.Lock() - defer i.Unlock() - i.ddls = nil -} diff --git a/syncer/ddl_exec_info_test.go b/syncer/ddl_exec_info_test.go deleted file mode 100644 index 42b0ad47ee..0000000000 --- a/syncer/ddl_exec_info_test.go +++ /dev/null @@ -1,79 +0,0 @@ -// Copyright 2019 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 syncer - -import ( - "context" - "sync" - "time" - - . "github.com/pingcap/check" - - "github.com/pingcap/dm/dm/pb" -) - -var _ = Suite(&testDDLExecInfoSuite{}) - -type testDDLExecInfoSuite struct{} - -func (t *testDDLExecInfoSuite) TestDDLExecItem(c *C) { - ddlExecInfo := NewDDLExecInfo() - c.Assert(ddlExecInfo.status.Get(), Equals, ddlExecIdle) - - ddlExecInfo.Renew() - c.Assert(ddlExecInfo.status.Get(), Equals, ddlExecIdle) - - ddls := []string{"create database test"} - - var wg sync.WaitGroup - wg.Add(1) - go func() { - defer wg.Done() - - select { - case <-ddlExecInfo.Chan(ddls): - case <-time.After(time.Second): - c.Fatal("timeout") - } - }() - - for i := 0; i < 3; i++ { - if len(ddlExecInfo.BlockingDDLs()) == 0 { - time.Sleep(100 * time.Millisecond) - continue - } - - break - } - - c.Assert(ddlExecInfo.BlockingDDLs(), DeepEquals, ddls) - - ddlExecInfo.ClearBlockingDDL() - c.Assert(ddlExecInfo.BlockingDDLs(), IsNil) - - ctx, cancel := context.WithTimeout(context.Background(), time.Second) - defer cancel() - - err := ddlExecInfo.Send(ctx, newDDLExecItem(new(pb.ExecDDLRequest))) - c.Assert(err, IsNil) - c.Assert(ddlExecInfo.status.Get(), Equals, ddlExecIdle) - - ddlExecInfo.Close() - c.Assert(ddlExecInfo.status.Get(), Equals, ddlExecClosed) - - ddlExecInfo.Renew() - c.Assert(ddlExecInfo.status.Get(), Equals, ddlExecIdle) - - wg.Wait() -} diff --git a/syncer/ddl_test.go b/syncer/ddl_test.go index ac5a609ecc..eb22fb06ae 100644 --- a/syncer/ddl_test.go +++ b/syncer/ddl_test.go @@ -208,7 +208,7 @@ func (s *testSyncerSuite) TestresolveDDLSQL(c *C) { }, } var err error - syncer := NewSyncer(cfg) + syncer := NewSyncer(cfg, nil) syncer.bwList, err = filter.New(syncer.cfg.CaseSensitive, syncer.cfg.BWList) c.Assert(err, IsNil) @@ -338,7 +338,7 @@ func (s *testSyncerSuite) TestParseDDLSQL(c *C) { }, } var err error - syncer := NewSyncer(cfg) + syncer := NewSyncer(cfg, nil) syncer.bwList, err = filter.New(syncer.cfg.CaseSensitive, syncer.cfg.BWList) c.Assert(err, IsNil) diff --git a/syncer/error_test.go b/syncer/error_test.go index 6a4660a064..772374744b 100644 --- a/syncer/error_test.go +++ b/syncer/error_test.go @@ -70,7 +70,7 @@ func (s *testSyncerSuite) TestOriginError(c *C) { func (s *testSyncerSuite) TestHandleSpecialDDLError(c *C) { var ( - syncer = NewSyncer(s.cfg) + syncer = NewSyncer(s.cfg, nil) tctx = tcontext.Background() conn2 = &DBConn{resetBaseConnFn: func(*context.Context, *conn.BaseConn) (*conn.BaseConn, error) { return nil, nil diff --git a/syncer/job.go b/syncer/job.go index c67cec147b..34f38ff8eb 100644 --- a/syncer/job.go +++ b/syncer/job.go @@ -71,7 +71,6 @@ type job struct { pos mysql.Position currentPos mysql.Position // exactly binlog position of current SQL gtidSet gtid.Set - ddlExecItem *DDLExecItem ddls []string traceID string traceGID string @@ -104,19 +103,18 @@ func newJob(tp opType, sourceSchema, sourceTable, targetSchema, targetTable, sql } } -func newDDLJob(ddlInfo *shardingDDLInfo, ddls []string, pos, cmdPos mysql.Position, currentGtidSet gtid.Set, ddlExecItem *DDLExecItem, traceID string) *job { +func newDDLJob(ddlInfo *shardingDDLInfo, ddls []string, pos, cmdPos mysql.Position, currentGtidSet gtid.Set, traceID string) *job { var gs gtid.Set if currentGtidSet != nil { gs = currentGtidSet.Clone() } j := &job{ - tp: ddl, - ddls: ddls, - pos: pos, - currentPos: cmdPos, - gtidSet: gs, - ddlExecItem: ddlExecItem, - traceID: traceID, + tp: ddl, + ddls: ddls, + pos: pos, + currentPos: cmdPos, + gtidSet: gs, + traceID: traceID, } if ddlInfo != nil { @@ -126,10 +124,6 @@ func newDDLJob(ddlInfo *shardingDDLInfo, ddls []string, pos, cmdPos mysql.Positi j.targetTable = ddlInfo.tableNames[1][0].Name } - if ddlExecItem != nil && ddlExecItem.req != nil { - j.traceGID = ddlExecItem.req.TraceGID - } - return j } diff --git a/syncer/job_test.go b/syncer/job_test.go index 9ef3af43dd..e536793389 100644 --- a/syncer/job_test.go +++ b/syncer/job_test.go @@ -17,8 +17,6 @@ import ( . "github.com/pingcap/check" "github.com/pingcap/tidb-tools/pkg/filter" "github.com/siddontang/go-mysql/mysql" - - "github.com/pingcap/dm/dm/pb" ) var _ = Suite(&testJobSuite{}) @@ -83,12 +81,6 @@ func (t *testJobSuite) TestJob(c *C) { }, } - ddlExecItem := &DDLExecItem{ - req: &pb.ExecDDLRequest{ - TraceGID: "abc", - }, - } - testCases := []struct { job *job jobStr string @@ -97,7 +89,7 @@ func (t *testJobSuite) TestJob(c *C) { newJob(insert, "test", "t1", "test", "t1", "insert into test.t1 values(?)", []interface{}{1}, "1", mysql.Position{}, mysql.Position{}, nil, ""), "tp: insert, sql: insert into test.t1 values(?), args: [1], key: 1, ddls: [], last_pos: (, 0), current_pos: (, 0), gtid:", }, { - newDDLJob(ddlInfo, []string{"create database test"}, mysql.Position{}, mysql.Position{}, nil, ddlExecItem, ""), + newDDLJob(ddlInfo, []string{"create database test"}, mysql.Position{}, mysql.Position{}, nil, ""), "tp: ddl, sql: , args: [], key: , ddls: [create database test], last_pos: (, 0), current_pos: (, 0), gtid:", }, { newXIDJob(mysql.Position{}, mysql.Position{}, nil, ""), diff --git a/syncer/shardddl/pessimist.go b/syncer/shardddl/pessimist.go new file mode 100644 index 0000000000..b89e4fea30 --- /dev/null +++ b/syncer/shardddl/pessimist.go @@ -0,0 +1,137 @@ +// Copyright 2020 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package shardddl + +import ( + "context" + "sync" + + "go.etcd.io/etcd/clientv3" + "go.uber.org/zap" + + "github.com/pingcap/dm/pkg/log" + "github.com/pingcap/dm/pkg/shardddl/pessimism" +) + +// Pessimist used to coordinate the shard DDL migration in pessimism mode. +type Pessimist struct { + mu sync.RWMutex + + logger log.Logger + cli *clientv3.Client + task string + source string + + // the shard DDL info which is pending to handle. + pendingInfo *pessimism.Info + // the shard DDL lock operation which is pending to handle. + pendingOp *pessimism.Operation +} + +// NewPessimist creates a new Pessimist instance. +func NewPessimist(pLogger *log.Logger, cli *clientv3.Client, task, source string) *Pessimist { + return &Pessimist{ + logger: pLogger.WithFields(zap.String("component", "shard DDL pessimist")), + cli: cli, + task: task, + source: source, + } +} + +// Reset resets the internal state of the pessimist. +func (p *Pessimist) Reset() { + p.mu.Lock() + defer p.mu.Unlock() + + p.pendingInfo = nil + p.pendingOp = nil +} + +// ConstructInfo constructs a shard DDL info. +func (p *Pessimist) ConstructInfo(schema, table string, DDLs []string) pessimism.Info { + return pessimism.NewInfo(p.task, p.source, schema, table, DDLs) +} + +// PutInfo puts the shard DDL info into etcd and returns the revision. +func (p *Pessimist) PutInfo(info pessimism.Info) (int64, error) { + rev, err := pessimism.PutInfo(p.cli, info) + if err != nil { + return 0, err + } + + p.mu.Lock() + p.pendingInfo = &info + p.mu.Unlock() + + return rev, nil +} + +// GetOperation gets the shard DDL lock operation relative to the shard DDL info. +func (p *Pessimist) GetOperation(ctx context.Context, info pessimism.Info, rev int64) (pessimism.Operation, error) { + ctx2, cancel2 := context.WithCancel(ctx) + defer cancel2() + + ch := make(chan pessimism.Operation, 1) + go pessimism.WatchOperationPut(ctx2, p.cli, info.Task, info.Source, rev, ch) + + select { + case op := <-ch: + p.mu.Lock() + p.pendingOp = &op + p.mu.Unlock() + return op, nil + case <-ctx.Done(): + return pessimism.Operation{}, ctx.Err() + } +} + +// DoneOperationDeleteInfo marks the shard DDL lock operation as done and delete the shard DDL info. +func (p *Pessimist) DoneOperationDeleteInfo(op pessimism.Operation, info pessimism.Info) error { + op.Done = true // mark the operation as `done`. + _, err := pessimism.PutOperationDeleteInfo(p.cli, op, info) + if err != nil { + return err + } + + p.mu.Lock() + p.pendingInfo = nil + p.pendingOp = nil + p.mu.Unlock() + + return err +} + +// PendingInfo returns the shard DDL info which is pending to handle. +func (p *Pessimist) PendingInfo() *pessimism.Info { + p.mu.RLock() + defer p.mu.RUnlock() + + if p.pendingInfo == nil { + return nil + } + info := *p.pendingInfo + return &info +} + +// PendingOperation returns the shard DDL lock operation which is pending to handle. +func (p *Pessimist) PendingOperation() *pessimism.Operation { + p.mu.RLock() + defer p.mu.RUnlock() + + if p.pendingOp == nil { + return nil + } + op := *p.pendingOp + return &op +} diff --git a/syncer/shardddl/pessimist_test.go b/syncer/shardddl/pessimist_test.go new file mode 100644 index 0000000000..e2e8e3453b --- /dev/null +++ b/syncer/shardddl/pessimist_test.go @@ -0,0 +1,141 @@ +// Copyright 2020 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package shardddl + +import ( + "context" + "testing" + + . "github.com/pingcap/check" + "go.etcd.io/etcd/clientv3" + "go.etcd.io/etcd/integration" + + "github.com/pingcap/dm/dm/common" + "github.com/pingcap/dm/pkg/log" + "github.com/pingcap/dm/pkg/shardddl/pessimism" +) + +var ( + etcdTestCli *clientv3.Client +) + +type testPessimist struct{} + +var _ = Suite(&testPessimist{}) + +func TestPessimist(t *testing.T) { + log.InitLogger(&log.Config{}) + + mockCluster := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1}) + defer mockCluster.Terminate(t) + + etcdTestCli = mockCluster.RandClient() + + TestingT(t) +} + +// clear keys in etcd test cluster. +func clearTestInfoOperation(c *C) { + clearInfo := clientv3.OpDelete(common.ShardDDLPessimismInfoKeyAdapter.Path(), clientv3.WithPrefix()) + clearOp := clientv3.OpDelete(common.ShardDDLPessimismOperationKeyAdapter.Path(), clientv3.WithPrefix()) + _, err := etcdTestCli.Txn(context.Background()).Then(clearInfo, clearOp).Commit() + c.Assert(err, IsNil) +} + +func (t *testPessimist) TestPessimist(c *C) { + defer clearTestInfoOperation(c) + + var ( + task = "task" + source = "mysql-replicate-1" + schema, table = "foo", "bar" + DDLs = []string{"ALTER TABLE bar ADD COLUMN c1 INT"} + ID = "task-`foo`.`bar`" + op = pessimism.NewOperation(ID, task, source, DDLs, true, false) + + logger = log.L() + p = NewPessimist(&logger, etcdTestCli, task, source) + info = p.ConstructInfo(schema, table, DDLs) + ) + + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + + // no info and operation in pending + c.Assert(p.PendingInfo(), IsNil) + c.Assert(p.PendingOperation(), IsNil) + + // put shard DDL info. + rev1, err := p.PutInfo(info) + c.Assert(err, IsNil) + c.Assert(rev1, Greater, int64(0)) + + // have info in pending + info2 := p.PendingInfo() + c.Assert(info2, NotNil) + c.Assert(*info2, DeepEquals, info) + + // put the lock operation. + rev2, putted, err := pessimism.PutOperations(etcdTestCli, false, op) + c.Assert(err, IsNil) + c.Assert(rev2, Greater, rev1) + c.Assert(putted, IsTrue) + + // wait for the lock operation. + op2, err := p.GetOperation(ctx, info, rev1) + c.Assert(err, IsNil) + c.Assert(op2, DeepEquals, op) + + // have operation in pending. + op3 := p.PendingOperation() + c.Assert(op3, NotNil) + c.Assert(*op3, DeepEquals, op) + + // mark the operation as done and delete the info. + c.Assert(p.DoneOperationDeleteInfo(op, info), IsNil) + + // verify the operation and info. + opc := op2 + opc.Done = true + opm, _, err := pessimism.GetAllOperations(etcdTestCli) + c.Assert(err, IsNil) + c.Assert(opm, HasLen, 1) + c.Assert(opm[task], HasLen, 1) + c.Assert(opm[task][source], DeepEquals, opc) + ifm, _, err := pessimism.GetAllInfo(etcdTestCli) + c.Assert(err, IsNil) + c.Assert(ifm, HasLen, 0) + + // no info and operation in pending now. + c.Assert(p.PendingInfo(), IsNil) + c.Assert(p.PendingOperation(), IsNil) + + // put info again, but do not complete the flow. + _, err = p.PutInfo(info) + c.Assert(err, IsNil) + c.Assert(p.PendingInfo(), NotNil) + + // put the lock operation again. + rev3, _, err := pessimism.PutOperations(etcdTestCli, false, op) + c.Assert(err, IsNil) + // wait for the lock operation. + _, err = p.GetOperation(ctx, info, rev3) + c.Assert(err, IsNil) + c.Assert(p.PendingOperation(), NotNil) + + // reset the pessimist. + p.Reset() + c.Assert(p.PendingInfo(), IsNil) + c.Assert(p.PendingOperation(), IsNil) +} diff --git a/syncer/status.go b/syncer/status.go index 2944db2688..c399420453 100644 --- a/syncer/status.go +++ b/syncer/status.go @@ -70,7 +70,12 @@ func (s *Syncer) Status() interface{} { if s.cfg.IsSharding { st.UnresolvedGroups = s.sgk.UnresolvedGroups() - st.BlockingDDLs = s.ddlExecInfo.BlockingDDLs() } + + pendingShardInfo := s.pessimist.PendingInfo() + if pendingShardInfo != nil { + st.BlockingDDLs = pendingShardInfo.DDLs + } + return st } diff --git a/syncer/syncer.go b/syncer/syncer.go index 75576de3f6..0d083368c1 100644 --- a/syncer/syncer.go +++ b/syncer/syncer.go @@ -35,6 +35,7 @@ import ( "github.com/siddontang/go-mysql/mysql" "github.com/siddontang/go-mysql/replication" "github.com/siddontang/go/sync2" + "go.etcd.io/etcd/clientv3" "go.uber.org/zap" "github.com/pingcap/dm/dm/config" @@ -47,11 +48,13 @@ import ( "github.com/pingcap/dm/pkg/gtid" "github.com/pingcap/dm/pkg/log" "github.com/pingcap/dm/pkg/schema" + "github.com/pingcap/dm/pkg/shardddl/pessimism" "github.com/pingcap/dm/pkg/streamer" "github.com/pingcap/dm/pkg/terror" "github.com/pingcap/dm/pkg/tracing" "github.com/pingcap/dm/pkg/utils" sm "github.com/pingcap/dm/syncer/safe-mode" + "github.com/pingcap/dm/syncer/shardddl" operator "github.com/pingcap/dm/syncer/sql-operator" ) @@ -95,8 +98,7 @@ type Syncer struct { syncCfg replication.BinlogSyncerConfig sgk *ShardingGroupKeeper // keeper to keep all sharding (sub) group in this syncer - ddlInfoCh chan *pb.DDLInfo // DDL info pending to sync, only support sync one DDL lock one time, refine if needed - ddlExecInfo *DDLExecInfo // DDL execute (ignore) info + pessimist *shardddl.Pessimist // shard DDL pessimist injectEventCh chan *replication.BinlogEvent // extra binlog event chan, used to inject binlog event into the main for loop binlogType BinlogType @@ -166,6 +168,7 @@ type Syncer struct { readerHub *streamer.ReaderHub + // TODO: re-implement tracer flow for binlog event later. tracer *tracing.Tracer currentPosMu struct { @@ -177,10 +180,13 @@ type Syncer struct { } // NewSyncer creates a new Syncer. -func NewSyncer(cfg *config.SubTaskConfig) *Syncer { - syncer := new(Syncer) +func NewSyncer(cfg *config.SubTaskConfig, etcdClient *clientv3.Client) *Syncer { + logger := log.With(zap.String("task", cfg.Name), zap.String("unit", "binlog replication")) + syncer := &Syncer{ + pessimist: shardddl.NewPessimist(&logger, etcdClient, cfg.Name, cfg.SourceID), + } syncer.cfg = cfg - syncer.tctx = tcontext.Background().WithLogger(log.With(zap.String("task", cfg.Name), zap.String("unit", "binlog replication"))) + syncer.tctx = tcontext.Background().WithLogger(logger) syncer.jobsClosed.Set(true) // not open yet syncer.closed.Set(false) syncer.lastBinlogSizeCount.Set(0) @@ -206,8 +212,6 @@ func NewSyncer(cfg *config.SubTaskConfig) *Syncer { if cfg.IsSharding { // only need to sync DDL in sharding mode syncer.sgk = NewShardingGroupKeeper(syncer.tctx, cfg) - syncer.ddlInfoCh = make(chan *pb.DDLInfo, 1) - syncer.ddlExecInfo = NewDDLExecInfo() } var err error @@ -436,7 +440,7 @@ func (s *Syncer) reset() { if s.cfg.IsSharding { // every time start to re-sync from resume, we reset status to make it like a fresh syncing s.sgk.ResetGroups() - s.ddlExecInfo.Renew() + s.pessimist.Reset() } } @@ -510,9 +514,6 @@ func (s *Syncer) Process(ctx context.Context, pr chan pb.ProcessResult) { go func() { defer wg.Done() <-newCtx.Done() // ctx or newCtx - if s.ddlExecInfo != nil { - s.ddlExecInfo.Close() // let Run can return - } }() wg.Add(1) @@ -691,7 +692,6 @@ func (s *Syncer) saveTablePoint(db, table string, pos mysql.Position) { func (s *Syncer) addJob(job *job) error { var ( queueBucket int - execDDLReq *pb.ExecDDLRequest ) switch job.tp { case xid: @@ -713,9 +713,6 @@ func (s *Syncer) addJob(job *job) error { s.jobWg.Add(1) queueBucket = s.cfg.WorkerCount s.jobs[queueBucket] <- job - if job.ddlExecItem != nil { - execDDLReq = job.ddlExecItem.req - } case insert, update, del: s.jobWg.Add(1) queueBucket = int(utils.GenHashKey(job.key)) % s.cfg.WorkerCount @@ -723,13 +720,6 @@ func (s *Syncer) addJob(job *job) error { s.jobs[queueBucket] <- job } - if s.tracer.Enable() { - _, err := s.tracer.CollectSyncerJobEvent(job.traceID, job.traceGID, int32(job.tp), job.pos, job.currentPos, s.queueBucketMapping[queueBucket], job.sql, job.ddls, job.args, execDDLReq, pb.SyncerJobState_queued) - if err != nil { - s.tctx.L().Error("fail to collect binlog replication job event", log.ShortError(err)) - } - } - wait := s.checkWait(job) if wait { s.jobWg.Wait() @@ -834,7 +824,8 @@ func (s *Syncer) syncDDL(tctx *tcontext.Context, queueBucket string, db *DBConn, return } - if sqlJob.ddlExecItem != nil && sqlJob.ddlExecItem.req != nil && !sqlJob.ddlExecItem.req.Exec { + shardOp := s.pessimist.PendingOperation() + if shardOp != nil && !shardOp.Exec { tctx.L().Info("ignore sharding DDLs", zap.Strings("ddls", sqlJob.ddls)) } else { var affected int @@ -843,18 +834,6 @@ func (s *Syncer) syncDDL(tctx *tcontext.Context, queueBucket string, db *DBConn, err = s.handleSpecialDDLError(tctx, err, sqlJob.ddls, affected, db) err = terror.WithScope(err, terror.ScopeDownstream) } - - if s.tracer.Enable() { - syncerJobState := s.tracer.FinishedSyncerJobState(err) - var execDDLReq *pb.ExecDDLRequest - if sqlJob.ddlExecItem != nil { - execDDLReq = sqlJob.ddlExecItem.req - } - _, traceErr := s.tracer.CollectSyncerJobEvent(sqlJob.traceID, sqlJob.traceGID, int32(sqlJob.tp), sqlJob.pos, sqlJob.currentPos, queueBucket, sqlJob.sql, sqlJob.ddls, nil, execDDLReq, syncerJobState) - if traceErr != nil { - tctx.L().Error("fail to collect binlog replication job event", log.ShortError(traceErr)) - } - } } if err != nil { s.appendExecErrors(&ExecErrorContext{ @@ -866,10 +845,16 @@ func (s *Syncer) syncDDL(tctx *tcontext.Context, queueBucket string, db *DBConn, if s.cfg.IsSharding { // for sharding DDL syncing, send result back - if sqlJob.ddlExecItem != nil { - sqlJob.ddlExecItem.resp <- err + shardInfo := s.pessimist.PendingInfo() + if shardInfo == nil { + // no need to do the shard DDL handle for `CREATE DATABASE/TABLE` now. + s.tctx.L().Warn("skip shard DDL handle in sharding mode", zap.Strings("ddl", sqlJob.ddls)) + } else if shardOp == nil { + // TODO(csuzhangxc): add terror. + err = fmt.Errorf("missing shard DDL lock operation for shard DDL info (%s)", shardInfo) + } else { + err = s.pessimist.DoneOperationDeleteInfo(*shardOp, *shardInfo) } - s.ddlExecInfo.ClearBlockingDDL() } s.jobWg.Done() if err != nil { @@ -927,15 +912,6 @@ func (s *Syncer) sync(tctx *tcontext.Context, queueBucket string, db *DBConn, jo errCtx := &ExecErrorContext{err, jobs[affected].currentPos, fmt.Sprintf("%v", jobs)} s.appendExecErrors(errCtx) } - if s.tracer.Enable() { - syncerJobState := s.tracer.FinishedSyncerJobState(err) - for _, job := range jobs { - _, err2 := s.tracer.CollectSyncerJobEvent(job.traceID, job.traceGID, int32(job.tp), job.pos, job.currentPos, queueBucket, job.sql, job.ddls, nil, nil, syncerJobState) - if err2 != nil { - tctx.L().Error("fail to collect binlog replication job event", log.ShortError(err2)) - } - } - } return err } @@ -1334,16 +1310,6 @@ func (s *Syncer) handleRotateEvent(ev *replication.RotateEvent, ec eventContext) } *ec.latestOp = rotate - if s.tracer.Enable() { - // Cannot convert this into a common method like `ec.CollectSyncerBinlogEvent()` - // since CollectSyncerBinlogEvent relies on a fixed stack trace level - // (must track 3 callers up). - _, err := s.tracer.CollectSyncerBinlogEvent(ec.traceSource, ec.safeMode.Enable(), ec.tryReSync, *ec.lastPos, *ec.currentPos, int32(ec.header.EventType), int32(*ec.latestOp)) - if err != nil { - s.tctx.L().Error("fail to collect binlog replication job event", zap.String("event", "rotate"), log.ShortError(err)) - } - } - s.tctx.L().Info("", zap.String("event", "rotate"), log.WrapStringerField("position", ec.currentPos)) return nil } @@ -1475,14 +1441,6 @@ func (s *Syncer) handleRowsEvent(ev *replication.RowsEvent, ec eventContext) err return nil } - if s.tracer.Enable() { - traceEvent, traceErr := s.tracer.CollectSyncerBinlogEvent(ec.traceSource, ec.safeMode.Enable(), ec.tryReSync, *ec.lastPos, *ec.currentPos, int32(ec.header.EventType), int32(*ec.latestOp)) - if traceErr != nil { - s.tctx.L().Error("fail to collect binlog replication job event", zap.String("event", "row"), log.ShortError(traceErr)) - } - *ec.traceID = traceEvent.Base.TraceID - } - for i := range sqls { var arg []interface{} var key []string @@ -1656,14 +1614,6 @@ func (s *Syncer) handleQueryEvent(ev *replication.QueryEvent, ec eventContext) e return s.recordSkipSQLsPos(*ec.lastPos, nil) } - if s.tracer.Enable() { - traceEvent, traceErr := s.tracer.CollectSyncerBinlogEvent(ec.traceSource, ec.safeMode.Enable(), ec.tryReSync, *ec.lastPos, *ec.currentPos, int32(ec.header.EventType), int32(*ec.latestOp)) - if traceErr != nil { - s.tctx.L().Error("fail to collect binlog replication job event", zap.String("event", "query"), log.ShortError(traceErr)) - } - *ec.traceID = traceEvent.Base.TraceID - } - if !s.cfg.IsSharding { s.tctx.L().Info("start to handle ddls in normal mode", zap.String("event", "query"), zap.Strings("ddls", needHandleDDLs), zap.ByteString("raw statement", ev.Query), log.WrapStringerField("position", ec.currentPos)) // try apply SQL operator before addJob. now, one query event only has one DDL job, if updating to multi DDL jobs, refine this. @@ -1675,7 +1625,7 @@ func (s *Syncer) handleQueryEvent(ev *replication.QueryEvent, ec eventContext) e s.tctx.L().Info("replace ddls to preset ddls by sql operator in normal mode", zap.String("event", "query"), zap.Strings("preset ddls", appliedSQLs), zap.Strings("ddls", needHandleDDLs), zap.ByteString("raw statement", ev.Query), log.WrapStringerField("position", ec.currentPos)) needHandleDDLs = appliedSQLs // maybe nil } - job := newDDLJob(nil, needHandleDDLs, *ec.lastPos, *ec.currentPos, nil, nil, *ec.traceID) + job := newDDLJob(nil, needHandleDDLs, *ec.lastPos, *ec.currentPos, nil, *ec.traceID) err = s.addJobFunc(job) if err != nil { return err @@ -1711,7 +1661,6 @@ func (s *Syncer) handleQueryEvent(ev *replication.QueryEvent, ec eventContext) e active bool remain int source string - ddlExecItem *DDLExecItem ) // for sharding DDL, the firstPos should be the `Pos` of the binlog, not the `End_log_pos` // so when restarting before sharding DDLs synced, this binlog can be re-sync again to trigger the TrySync @@ -1803,35 +1752,22 @@ func (s *Syncer) handleQueryEvent(ev *replication.QueryEvent, ec eventContext) e // NOTE: if we need singleton Syncer (without dm-master) to support sharding DDL sync // we should add another config item to differ, and do not save DDLInfo, and not wait for ddlExecInfo - ddlInfo1 := &pb.DDLInfo{ - Task: s.cfg.Name, - Schema: ddlInfo.tableNames[1][0].Schema, // use target schema / table name - Table: ddlInfo.tableNames[1][0].Name, - DDLs: needHandleDDLs, + // construct & send shard DDL info into etcd, DM-master will handle it. + shardInfo := s.pessimist.ConstructInfo(ddlInfo.tableNames[1][0].Schema, ddlInfo.tableNames[1][0].Name, needHandleDDLs) + rev, err2 := s.pessimist.PutInfo(shardInfo) + if err2 != nil { + return err2 } - s.ddlInfoCh <- ddlInfo1 // save DDLInfo, and dm-worker will fetch it + shardLockResolving.WithLabelValues(s.cfg.Name).Set(1) // block and wait DDL lock to be synced + s.tctx.L().Info("putted shard DDL info", zap.Stringer("info", shardInfo)) - // block and wait DDL lock to be synced - shardLockResolving.WithLabelValues(s.cfg.Name).Set(1) - for { - var ok bool - ddlExecItem, ok = <-s.ddlExecInfo.Chan(needHandleDDLs) - if !ok { - // chan closed - shardLockResolving.WithLabelValues(s.cfg.Name).Set(0) - s.tctx.L().Warn("canceled from external", zap.String("event", "query"), zap.String("source", source), zap.Strings("ddls", needHandleDDLs), zap.ByteString("raw statement", ev.Query), zap.Stringer("start position", startPos), log.WrapStringerField("end position", ec.currentPos)) - return nil - } else if len(ddlExecItem.req.DDLs) != 0 && !reflect.DeepEqual(ddlExecItem.req.DDLs, needHandleDDLs) { - // ignore un-cleared cached/duplicate DDL execute request - // check `len(ddlExecItem.req.DDLs) != 0` to support old DM-master and `break-ddl-lock` - s.tctx.L().Warn("ignore mismatched DDL execute request", zap.String("source", source), zap.Strings("expect", needHandleDDLs), zap.Strings("request", ddlExecItem.req.DDLs)) - continue - } - shardLockResolving.WithLabelValues(s.cfg.Name).Set(0) - break + shardOp, err2 := s.pessimist.GetOperation(ec.tctx.Ctx, shardInfo, rev) + shardLockResolving.WithLabelValues(s.cfg.Name).Set(0) + if err2 != nil { + return err2 } - if ddlExecItem.req.Exec { + if shardOp.Exec { failpoint.Inject("ShardSyncedExecutionExit", func() { s.tctx.L().Warn("exit triggered", zap.String("failpoint", "ShardSyncedExecutionExit")) s.flushCheckPoints() @@ -1849,9 +1785,9 @@ func (s *Syncer) handleQueryEvent(ev *replication.QueryEvent, ec eventContext) e } }) - s.tctx.L().Info("execute DDL job", zap.String("event", "query"), zap.String("source", source), zap.Strings("ddls", ddlInfo1.DDLs), zap.ByteString("raw statement", ev.Query), zap.Stringer("start position", startPos), log.WrapStringerField("end position", ec.currentPos), zap.Reflect("request", ddlExecItem.req)) + s.tctx.L().Info("execute DDL job", zap.String("event", "query"), zap.String("source", source), zap.ByteString("raw statement", ev.Query), zap.Stringer("start position", startPos), log.WrapStringerField("end position", ec.currentPos), zap.Stringer("operation", shardOp)) } else { - s.tctx.L().Info("ignore DDL job", zap.String("event", "query"), zap.String("source", source), zap.Strings("ddls", ddlInfo1.DDLs), zap.ByteString("raw statement", ev.Query), zap.Stringer("start position", startPos), log.WrapStringerField("end position", ec.currentPos), zap.Reflect("request", ddlExecItem.req)) + s.tctx.L().Info("ignore DDL job", zap.String("event", "query"), zap.String("source", source), zap.ByteString("raw statement", ev.Query), zap.Stringer("start position", startPos), log.WrapStringerField("end position", ec.currentPos), zap.Stringer("operation", shardOp)) } } @@ -1866,7 +1802,7 @@ func (s *Syncer) handleQueryEvent(ev *replication.QueryEvent, ec eventContext) e s.tctx.L().Info("replace ddls to preset ddls by sql operator in shard mode", zap.String("event", "query"), zap.Strings("preset ddls", appliedSQLs), zap.Strings("ddls", needHandleDDLs), zap.ByteString("raw statement", ev.Query), zap.Stringer("start position", startPos), log.WrapStringerField("end position", ec.currentPos)) needHandleDDLs = appliedSQLs // maybe nil } - job := newDDLJob(ddlInfo, needHandleDDLs, *ec.lastPos, *ec.currentPos, nil, ddlExecItem, *ec.traceID) + job := newDDLJob(ddlInfo, needHandleDDLs, *ec.lastPos, *ec.currentPos, nil, *ec.traceID) err = s.addJobFunc(job) if err != nil { return err @@ -2185,11 +2121,6 @@ func (s *Syncer) Close() { s.stopSync() - if s.ddlInfoCh != nil { - close(s.ddlInfoCh) - s.ddlInfoCh = nil - } - s.closeDBs() s.checkpoint.Close() @@ -2394,26 +2325,6 @@ func (s *Syncer) checkpointID() string { return strconv.FormatUint(uint64(s.cfg.ServerID), 10) } -// DDLInfo returns a chan from which can receive DDLInfo -func (s *Syncer) DDLInfo() <-chan *pb.DDLInfo { - s.RLock() - defer s.RUnlock() - return s.ddlInfoCh -} - -// ExecuteDDL executes or skips a hanging-up DDL when in sharding -func (s *Syncer) ExecuteDDL(ctx context.Context, execReq *pb.ExecDDLRequest) (<-chan error, error) { - if len(s.ddlExecInfo.BlockingDDLs()) == 0 { - return nil, terror.ErrSyncerUnitExecWithNoBlockingDDL.Generate() - } - item := newDDLExecItem(execReq) - err := s.ddlExecInfo.Send(ctx, item) - if err != nil { - return nil, err - } - return item.resp, nil -} - // UpdateFromConfig updates config for `From` func (s *Syncer) UpdateFromConfig(cfg *config.SubTaskConfig) error { s.Lock() @@ -2478,3 +2389,13 @@ func (s *Syncer) setSyncCfg() { TimestampStringLocation: s.timezone, } } + +// ShardDDLInfo returns the current pending to handle shard DDL info. +func (s *Syncer) ShardDDLInfo() *pessimism.Info { + return s.pessimist.PendingInfo() +} + +// ShardDDLOperation returns the current pending to handle shard DDL lock operation. +func (s *Syncer) ShardDDLOperation() *pessimism.Operation { + return s.pessimist.PendingOperation() +} diff --git a/syncer/syncer_test.go b/syncer/syncer_test.go index 95adc8f9fb..8bf8f380b6 100644 --- a/syncer/syncer_test.go +++ b/syncer/syncer_test.go @@ -16,7 +16,6 @@ package syncer import ( "context" "database/sql" - "database/sql/driver" "fmt" "strings" "sync" @@ -267,7 +266,7 @@ func (s *testSyncerSuite) TestSelectDB(c *C) { p, err := s.mockParser(db, mock) c.Assert(err, IsNil) - syncer := NewSyncer(s.cfg) + syncer := NewSyncer(s.cfg, nil) syncer.bwList, err = filter.New(syncer.cfg.CaseSensitive, syncer.cfg.BWList) c.Assert(err, IsNil) err = syncer.genRouter() @@ -376,7 +375,7 @@ func (s *testSyncerSuite) TestSelectTable(c *C) { p, err := s.mockParser(db, mock) c.Assert(err, IsNil) - syncer := NewSyncer(s.cfg) + syncer := NewSyncer(s.cfg, nil) syncer.bwList, err = filter.New(syncer.cfg.CaseSensitive, syncer.cfg.BWList) c.Assert(err, IsNil) syncer.genRouter() @@ -448,7 +447,7 @@ func (s *testSyncerSuite) TestIgnoreDB(c *C) { p, err := s.mockParser(db, mock) c.Assert(err, IsNil) - syncer := NewSyncer(s.cfg) + syncer := NewSyncer(s.cfg, nil) syncer.bwList, err = filter.New(syncer.cfg.CaseSensitive, syncer.cfg.BWList) c.Assert(err, IsNil) syncer.genRouter() @@ -541,7 +540,7 @@ func (s *testSyncerSuite) TestIgnoreTable(c *C) { p, err := s.mockParser(db, mock) c.Assert(err, IsNil) - syncer := NewSyncer(s.cfg) + syncer := NewSyncer(s.cfg, nil) syncer.bwList, err = filter.New(syncer.cfg.CaseSensitive, syncer.cfg.BWList) c.Assert(err, IsNil) syncer.genRouter() @@ -671,7 +670,7 @@ func (s *testSyncerSuite) TestSkipDML(c *C) { p, err := s.mockParser(db, mock) c.Assert(err, IsNil) - syncer := NewSyncer(s.cfg) + syncer := NewSyncer(s.cfg, nil) syncer.genRouter() syncer.binlogFilter, err = bf.NewBinlogEvent(false, s.cfg.FilterRules) @@ -945,7 +944,7 @@ func (s *testSyncerSuite) TestGeneratedColumn(c *C) { c.Assert(err, IsNil) } - syncer := NewSyncer(s.cfg) + syncer := NewSyncer(s.cfg, nil) // use upstream dbConn as mock downstream dbConn, err := db.Conn(context.Background()) c.Assert(err, IsNil) @@ -1022,13 +1021,13 @@ func (s *testSyncerSuite) TestGeneratedColumn(c *C) { } func (s *testSyncerSuite) TestcheckpointID(c *C) { - syncer := NewSyncer(s.cfg) + syncer := NewSyncer(s.cfg, nil) checkpointID := syncer.checkpointID() c.Assert(checkpointID, Equals, "101") } func (s *testSyncerSuite) TestExecErrors(c *C) { - syncer := NewSyncer(s.cfg) + syncer := NewSyncer(s.cfg, nil) syncer.appendExecErrors(new(ExecErrorContext)) c.Assert(syncer.execErrors.errors, HasLen, 1) @@ -1039,7 +1038,7 @@ func (s *testSyncerSuite) TestExecErrors(c *C) { func (s *testSyncerSuite) TestCasuality(c *C) { var wg sync.WaitGroup s.cfg.WorkerCount = 1 - syncer := NewSyncer(s.cfg) + syncer := NewSyncer(s.cfg, nil) syncer.jobs = []chan *job{make(chan *job, 1)} wg.Add(1) @@ -1066,298 +1065,7 @@ func (s *testSyncerSuite) TestCasuality(c *C) { wg.Wait() } -func (s *testSyncerSuite) TestSharding(c *C) { - - events := mockBinlogEvents{ - mockBinlogEvent{typ: DBCreate, args: []interface{}{"stest_1"}}, - mockBinlogEvent{typ: TableCreate, args: []interface{}{"stest_1", "create table stest_1.st_1(id int, age int)"}}, - mockBinlogEvent{typ: TableCreate, args: []interface{}{"stest_1", "create table stest_1.st_2(id int, age int)"}}, - } - - testCases := []struct { - testEvents mockBinlogEvents - expectSQLS []struct { - sql string - args []driver.Value - } - }{ - // case 1: - // upstream binlog events: - // insert t1 -> insert t2 -> alter t1 -> insert t2 -> alter t2 -> insert t1(new schema) -> insert t2(new schema) - // downstream expected events: - // insert t(from t1) -> insert t(from t2) -> insert t(from t2) -> alter t(from t2 same as t1) -> insert t1(new schema) -> insert t2(new schema) - { - mockBinlogEvents{ - mockBinlogEvent{typ: Write, args: []interface{}{uint64(8), "stest_1", "st_1", []byte{mysql.MYSQL_TYPE_LONG, mysql.MYSQL_TYPE_LONG}, [][]interface{}{{int32(1), int32(1)}}}}, - mockBinlogEvent{typ: Write, args: []interface{}{uint64(9), "stest_1", "st_2", []byte{mysql.MYSQL_TYPE_LONG, mysql.MYSQL_TYPE_LONG}, [][]interface{}{{int32(2), int32(2)}}}}, - mockBinlogEvent{typ: DDL, args: []interface{}{"stest_1", "ALTER TABLE `stest_1`.`st_1` ADD COLUMN NAME VARCHAR(30)"}}, - mockBinlogEvent{typ: Write, args: []interface{}{uint64(9), "stest_1", "st_2", []byte{mysql.MYSQL_TYPE_LONG, mysql.MYSQL_TYPE_LONG}, [][]interface{}{{int32(4), int32(4)}}}}, - mockBinlogEvent{typ: DDL, args: []interface{}{"stest_1", "ALTER TABLE `stest_1`.`st_2` ADD COLUMN NAME VARCHAR(30)"}}, - mockBinlogEvent{typ: Write, args: []interface{}{uint64(8), "stest_1", "st_1", []byte{mysql.MYSQL_TYPE_LONG, mysql.MYSQL_TYPE_LONG, mysql.MYSQL_TYPE_STRING}, [][]interface{}{{int32(3), int32(3), "test"}}}}, - mockBinlogEvent{typ: Write, args: []interface{}{uint64(9), "stest_1", "st_2", []byte{mysql.MYSQL_TYPE_LONG, mysql.MYSQL_TYPE_LONG, mysql.MYSQL_TYPE_STRING}, [][]interface{}{{int32(6), int32(6), "test"}}}}, - }, - []struct { - sql string - args []driver.Value - }{ - { - "REPLACE INTO", - []driver.Value{1, 1}, - }, - { - "REPLACE INTO", - []driver.Value{2, 2}, - }, - { - "REPLACE INTO", - []driver.Value{4, 4}, - }, - { - "ALTER TABLE", - []driver.Value{}, - }, - { - "REPLACE INTO", - []driver.Value{3, 3, "test"}, - }, - { - "REPLACE INTO", - []driver.Value{6, 6, "test"}, - }, - }, - }, - // case 2: - // upstream binlog events: - // insert t1 -> insert t2 -> alter t1 -> insert t1(new schema) -> insert t2 -> alter t2 -> insert t1(new schema) -> insert t2(new schema) - // downstream expected events: - // insert t(from t1) -> insert t(from t2) -> insert t(from t2) -> alter t(from t2 same as t1) -> insert t1(new schema) -> insert t1(new schema) -> insert t2(new schema) - { - mockBinlogEvents{ - mockBinlogEvent{typ: Write, args: []interface{}{uint64(8), "stest_1", "st_1", []byte{mysql.MYSQL_TYPE_LONG, mysql.MYSQL_TYPE_LONG}, [][]interface{}{{int32(1), int32(1)}}}}, - mockBinlogEvent{typ: Write, args: []interface{}{uint64(9), "stest_1", "st_2", []byte{mysql.MYSQL_TYPE_LONG, mysql.MYSQL_TYPE_LONG}, [][]interface{}{{int32(2), int32(2)}}}}, - mockBinlogEvent{typ: DDL, args: []interface{}{"stest_1", "ALTER TABLE `stest_1`.`st_1` ADD COLUMN NAME VARCHAR(30)"}}, - mockBinlogEvent{typ: Write, args: []interface{}{uint64(8), "stest_1", "st_1", []byte{mysql.MYSQL_TYPE_LONG, mysql.MYSQL_TYPE_LONG, mysql.MYSQL_TYPE_STRING}, [][]interface{}{{int32(3), int32(3), "test"}}}}, - mockBinlogEvent{typ: Write, args: []interface{}{uint64(9), "stest_1", "st_2", []byte{mysql.MYSQL_TYPE_LONG, mysql.MYSQL_TYPE_LONG}, [][]interface{}{{int32(4), int32(4)}}}}, - mockBinlogEvent{typ: DDL, args: []interface{}{"stest_1", "ALTER TABLE `stest_1`.`st_2` ADD COLUMN NAME VARCHAR(30)"}}, - mockBinlogEvent{typ: Write, args: []interface{}{uint64(8), "stest_1", "st_1", []byte{mysql.MYSQL_TYPE_LONG, mysql.MYSQL_TYPE_LONG, mysql.MYSQL_TYPE_STRING}, [][]interface{}{{int32(5), int32(5), "test"}}}}, - mockBinlogEvent{typ: Write, args: []interface{}{uint64(9), "stest_1", "st_2", []byte{mysql.MYSQL_TYPE_LONG, mysql.MYSQL_TYPE_LONG, mysql.MYSQL_TYPE_STRING}, [][]interface{}{{int32(6), int32(6), "test"}}}}, - }, - []struct { - sql string - args []driver.Value - }{ - { - "REPLACE INTO", - []driver.Value{1, 1}, - }, - { - "REPLACE INTO", - []driver.Value{2, 2}, - }, - { - "REPLACE INTO", - []driver.Value{4, 4}, - }, - { - "ALTER TABLE", - []driver.Value{}, - }, - { - "REPLACE INTO", - []driver.Value{3, 3, "test"}, - }, - { - "REPLACE INTO", - []driver.Value{5, 5, "test"}, - }, - { - "REPLACE INTO", - []driver.Value{6, 6, "test"}, - }, - }, - }, - } - - s.cfg.Flavor = "mysql" - s.cfg.BWList = &filter.Rules{ - DoDBs: []string{"stest_1"}, - } - s.cfg.IsSharding = true - s.cfg.RouteRules = []*router.TableRule{ - { - SchemaPattern: "stest_1", - TablePattern: "st_*", - TargetSchema: "stest", - TargetTable: "st", - }, - } - // set batch to 1 is easy to mock - s.cfg.Batch = 1 - s.cfg.WorkerCount = 1 - - for i, _case := range testCases { - s.resetEventsGenerator(c) - createEvents := s.generateEvents(events, c) - - db, mock, err := sqlmock.New() - c.Assert(err, IsNil) - - fromDB, fromMock, err := sqlmock.New() - c.Assert(err, IsNil) - // mock initShardingGroups - fromMock.ExpectQuery("SHOW DATABASES").WillReturnRows(sqlmock.NewRows([]string{"Database"}).AddRow("information_schema").AddRow("mysql").AddRow("sys").AddRow("stest_1")) - fromMock.ExpectQuery("SHOW FULL TABLES").WillReturnRows(sqlmock.NewRows([]string{"Table_in_stest_1", "Table_type"}).AddRow("st_1", "BASE TABLE").AddRow("st_2", "BASE TABLE")) - - checkPointDB, checkPointMock, err := sqlmock.New() - c.Assert(err, IsNil) - // mock checkpoint init - checkPointMock.ExpectBegin() - checkPointMock.ExpectExec(fmt.Sprintf("CREATE SCHEMA IF NOT EXISTS `%s`", s.cfg.MetaSchema)).WillReturnResult(sqlmock.NewResult(1, 1)) - checkPointMock.ExpectCommit() - checkPointMock.ExpectBegin() - checkPointMock.ExpectExec(fmt.Sprintf("CREATE TABLE IF NOT EXISTS `%s`.`%s_syncer_checkpoint`", s.cfg.MetaSchema, s.cfg.Name)).WillReturnResult(sqlmock.NewResult(1, 1)) - checkPointMock.ExpectCommit() - - shardGroupDB, shardGroupMock, err := sqlmock.New() - c.Assert(err, IsNil) - // mock initShardingGroups - shardGroupMock.ExpectBegin() - shardGroupMock.ExpectExec(fmt.Sprintf("CREATE SCHEMA IF NOT EXISTS `%s`", s.cfg.MetaSchema)).WillReturnResult(sqlmock.NewResult(1, 1)) - shardGroupMock.ExpectCommit() - shardGroupMock.ExpectBegin() - shardGroupMock.ExpectExec(fmt.Sprintf("CREATE TABLE IF NOT EXISTS `%s`.`%s_syncer_sharding_meta`", s.cfg.MetaSchema, s.cfg.Name)).WillReturnResult(sqlmock.NewResult(1, 1)) - shardGroupMock.ExpectCommit() - shardGroupMock.ExpectQuery("SELECT").WillReturnRows(sqlmock.NewRows([]string{"unreachable", "unreachable", "unreachable", "unreachable", "unreachable"})) - - // make syncer write to mock baseConn - syncer := NewSyncer(s.cfg) - - ctx := context.Background() - // fromDB mocks upstream dbConn, dbConn mocks downstream dbConn - syncer.fromDB = &UpStreamConn{BaseDB: conn.NewBaseDB(fromDB)} - dbConn, err := db.Conn(ctx) - c.Assert(err, IsNil) - syncer.toDBConns = []*DBConn{{cfg: s.cfg, baseConn: conn.NewBaseConn(dbConn, &retry.FiniteRetryStrategy{})}} - syncer.ddlDBConn = &DBConn{cfg: s.cfg, baseConn: conn.NewBaseConn(dbConn, &retry.FiniteRetryStrategy{})} - - // mock syncer.Init() function, because we need to pass mock dbs to different members' init - syncer.genRouter() - shardGroupDBConn, err := shardGroupDB.Conn(ctx) - c.Assert(err, IsNil) - checkPointDBConn, err := checkPointDB.Conn(ctx) - c.Assert(err, IsNil) - - // mock syncer.shardGroupkeeper.Init() function - syncer.sgk.dbConn = &DBConn{cfg: s.cfg, baseConn: conn.NewBaseConn(shardGroupDBConn, &retry.FiniteRetryStrategy{})} - syncer.sgk.prepare() - syncer.initShardingGroups() - - // mock syncer.checkpoint.Init() function - syncer.checkpoint.(*RemoteCheckPoint).dbConn = &DBConn{cfg: s.cfg, baseConn: conn.NewBaseConn(checkPointDBConn, &retry.FiniteRetryStrategy{})} - syncer.checkpoint.(*RemoteCheckPoint).prepare(tcontext.Background()) - - syncer.reset() - events := append(createEvents, s.generateEvents(_case.testEvents, c)...) - - mockStreamerProducer := &MockStreamProducer{events} - mockStreamer, err := mockStreamerProducer.generateStreamer(mysql.Position{}) - c.Assert(err, IsNil) - syncer.streamerController = &StreamerController{ - streamerProducer: mockStreamerProducer, - streamer: mockStreamer, - } - - fromMock.ExpectQuery("SHOW GLOBAL VARIABLES LIKE"). - WillReturnRows(sqlmock.NewRows([]string{"Variable_name", "Value"}). - AddRow("sql_mode", "ONLY_FULL_GROUP_BY,STRICT_TRANS_TABLES,NO_ZERO_IN_DATE,NO_ZERO_DATE,ERROR_FOR_DIVISION_BY_ZERO,NO_AUTO_CREATE_USER,NO_ENGINE_SUBSTITUTION")) - - // mock checkpoint db after create db table1 table2 - s.mockCheckPointCreate(checkPointMock, "db") - s.mockCheckPointCreate(checkPointMock, "table1") - s.mockCheckPointCreate(checkPointMock, "table2") - - // mock downstream db result - mock.ExpectBegin() - mock.ExpectExec("CREATE DATABASE").WillReturnResult(sqlmock.NewResult(1, 1)) - mock.ExpectCommit() - mock.ExpectBegin() - mock.ExpectExec("CREATE TABLE").WillReturnResult(sqlmock.NewResult(1, 1)) - mock.ExpectCommit() - mock.ExpectBegin() - e := newMysqlErr(1050, "Table exist") - mock.ExpectExec("CREATE TABLE").WillReturnError(e) - mock.ExpectCommit() - - // mock fetching table schema from downstream - // called twice for each upstream schema: once for `db`.`table1`, once for `db`.`table2`. - mock.ExpectQuery("SHOW CREATE TABLE `stest`.`st`"). - WillReturnRows(sqlmock.NewRows([]string{"Table", "Create Table"}).AddRow("-", "create table st(id int, age int)")) - mock.ExpectQuery("SHOW CREATE TABLE `stest`.`st`"). - WillReturnRows(sqlmock.NewRows([]string{"Table", "Create Table"}).AddRow("-", "create table st(id int, age int)")) - - // mock expect sql - for i, expectSQL := range _case.expectSQLS { - mock.ExpectBegin() - if strings.HasPrefix(expectSQL.sql, "ALTER") { - mock.ExpectExec(expectSQL.sql).WillReturnResult(sqlmock.NewResult(1, int64(i)+1)) - mock.ExpectCommit() - s.mockCheckPointFlush(checkPointMock, i) - } else { - // change insert to replace because of safe mode - mock.ExpectExec(expectSQL.sql).WithArgs(expectSQL.args...).WillReturnResult(sqlmock.NewResult(1, 1)) - mock.ExpectCommit() - } - } - ctx, cancel := context.WithCancel(context.Background()) - resultCh := make(chan pb.ProcessResult) - - s.mockCheckPointFlush(checkPointMock, -1) - - go syncer.Process(ctx, resultCh) - - var wg sync.WaitGroup - wg.Add(1) - go func() { - defer wg.Done() - reqMismatch := &DDLExecItem{&pb.ExecDDLRequest{Exec: true, DDLs: []string{"stmt"}}, make(chan error, 1)} - c.Assert(syncer.ddlExecInfo.Send(ctx, reqMismatch), IsNil) - reqMatch := &DDLExecItem{&pb.ExecDDLRequest{Exec: true}, make(chan error, 1)} - c.Assert(syncer.ddlExecInfo.Send(ctx, reqMatch), IsNil) - }() - - select { - case r := <-resultCh: - for _, err := range r.Errors { - c.Errorf("Case %d: Process Err:%s", i, err) - } - c.Assert(len(r.Errors), Equals, 0) - case <-time.After(2 * time.Second): - } - cancel() - wg.Wait() - - syncer.Close() - c.Assert(syncer.isClosed(), IsTrue) - - flushedGP := syncer.checkpoint.FlushedGlobalPoint().Pos - GP := syncer.checkpoint.GlobalPoint().Pos - c.Assert(GP, Equals, flushedGP) - - // check expectations for mock baseConn - if err := mock.ExpectationsWereMet(); err != nil { - c.Errorf("db unfulfilled expectations: %s", err) - } - if err := fromMock.ExpectationsWereMet(); err != nil { - c.Errorf("fromDB unfulfilled expectations: %s", err) - } - if err := checkPointMock.ExpectationsWereMet(); err != nil { - c.Errorf("checkpointDB unfulfilled expectations: %s", err) - } - if err := shardGroupMock.ExpectationsWereMet(); err != nil { - c.Errorf("shardGroupDB unfulfilled expectations: %s", err) - } - } -} +// TODO: add `TestSharding` later. func (s *testSyncerSuite) TestRun(c *C) { // 1. run syncer with column mapping @@ -1400,7 +1108,7 @@ func (s *testSyncerSuite) TestRun(c *C) { s.cfg.MaxRetry = 1 s.cfg.DisableCausality = false - syncer := NewSyncer(s.cfg) + syncer := NewSyncer(s.cfg, nil) syncer.fromDB = &UpStreamConn{BaseDB: conn.NewBaseDB(db)} syncer.toDBConns = []*DBConn{{cfg: s.cfg, baseConn: conn.NewBaseConn(dbConn, &retry.FiniteRetryStrategy{})}, {cfg: s.cfg, baseConn: conn.NewBaseConn(dbConn, &retry.FiniteRetryStrategy{})}} diff --git a/tests/all_mode/run.sh b/tests/all_mode/run.sh index 7a81a9182a..83bc2ad9c2 100755 --- a/tests/all_mode/run.sh +++ b/tests/all_mode/run.sh @@ -50,7 +50,7 @@ function run() { 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 - sleep 5 + sleep 10 echo "after restart dm-worker, task should resume automatically" run_dm_ctl $WORK_DIR "127.0.0.1:$MASTER_PORT" \ "start-task $cur/conf/dm-task.yaml" \ diff --git a/tests/ha/run.sh b/tests/ha/run.sh index 4f0c0c602d..262c1e9163 100755 --- a/tests/ha/run.sh +++ b/tests/ha/run.sh @@ -51,6 +51,7 @@ 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 + sleep 8 echo "wait and check task running" check_http_alive 127.0.0.1:$MASTER_PORT/apis/${API_VERSION}/status/test '"name":"test","stage":"Running"' 10 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 1760ba7278..ae196e94d3 100755 --- a/tests/incremental_mode/run.sh +++ b/tests/incremental_mode/run.sh @@ -32,6 +32,7 @@ function run() { "true" 1 # start a task in `full` mode + echo "start task in full mode" cat $cur/conf/dm-task.yaml > $WORK_DIR/dm-task.yaml sed -i "s/task-mode-placeholder/full/g" $WORK_DIR/dm-task.yaml # avoid cannot unmarshal !!str `binlog-...` into uint32 error @@ -68,6 +69,7 @@ function run() { run_dm_worker $WORK_DIR/worker2 $WORKER2_PORT $WORK_DIR/dm-worker2.toml check_rpc_alive $cur/../bin/check_worker_online 127.0.0.1:$WORKER2_PORT + 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 name1=$(grep "Log: " $WORK_DIR/worker1/dumped_data.$TASK_NAME/metadata|awk -F: '{print $2}'|tr -d ' ') @@ -78,7 +80,7 @@ function run() { sed -i "s/binlog-pos-placeholder-1/$pos1/g" $WORK_DIR/dm-task.yaml sed -i "s/binlog-name-placeholder-2/$name2/g" $WORK_DIR/dm-task.yaml sed -i "s/binlog-pos-placeholder-2/$pos2/g" $WORK_DIR/dm-task.yaml - sleep 5 + sleep 8 dmctl_start_task $WORK_DIR/dm-task.yaml check_sync_diff $WORK_DIR $cur/conf/diff_config.toml diff --git a/tests/load_interrupt/run.sh b/tests/load_interrupt/run.sh index 235157088f..736aa0dee6 100755 --- a/tests/load_interrupt/run.sh +++ b/tests/load_interrupt/run.sh @@ -77,23 +77,21 @@ function run() { check_rpc_alive $cur/../bin/check_worker_online 127.0.0.1:$WORKER1_PORT check_rpc_alive $cur/../bin/check_worker_online 127.0.0.1:$WORKER2_PORT - sleep 10 - echo "start task after restarted dm-worker" - # TODO: skip this now. problem has been added to document - # run_dm_ctl $WORK_DIR "127.0.0.1:$MASTER_PORT" \ - # "query-status -s $SOURCE_ID1,$SOURCE_ID2" \ - # "\"taskName\": \"test\"" 1 \ - # "\"taskStatus\": \"Running\"" 1 - + sleep 8 + echo "check sync diff after restarted dm-worker" check_sync_diff $WORK_DIR $cur/conf/diff_config.toml # LoadExecCreateTableFailed error return twice + sleep 8 err_cnt=`grep LoadExecCreateTableFailed $WORK_DIR/worker1/log/dm-worker.log | wc -l` if [ $err_cnt -ne 2 ]; then echo "error LoadExecCreateTableFailed's count is not 2" exit 2 fi + # strange, TiDB (at least with mockTiKV) needs a long time to see the update of `test_loader_checkpoint`, + # and even later txn may see the older state than the earlier txn. + sleep 8 run_sql "SELECT count(*) from dm_meta.test_loader_checkpoint where cp_schema = '$TEST_NAME' and offset = end_pos" $TIDB_PORT check_contains "count(*): 2" diff --git a/tests/safe_mode/conf/mysql1.toml b/tests/safe_mode/conf/mysql1.toml index b66f6527d3..64f4b035e7 100644 --- a/tests/safe_mode/conf/mysql1.toml +++ b/tests/safe_mode/conf/mysql1.toml @@ -12,8 +12,8 @@ user = "root" password = "" port = 3306 -[tracer] -enable = true -tracer-addr = "127.0.0.1:8264" -batch-size = 1 -checksum = true +#[tracer] +#enable = true +#tracer-addr = "127.0.0.1:8264" +#batch-size = 1 +#checksum = true diff --git a/tests/safe_mode/conf/mysql2.toml b/tests/safe_mode/conf/mysql2.toml index 86ae904c92..d7362dd417 100644 --- a/tests/safe_mode/conf/mysql2.toml +++ b/tests/safe_mode/conf/mysql2.toml @@ -12,8 +12,8 @@ user = "root" password = "" port = 3307 -[tracer] -enable = true -tracer-addr = "127.0.0.1:8264" -batch-size = 1 -checksum = true +#[tracer] +#enable = true +#tracer-addr = "127.0.0.1:8264" +#batch-size = 1 +#checksum = true diff --git a/tests/safe_mode/run.sh b/tests/safe_mode/run.sh index a5718cb86c..030c05c9ef 100755 --- a/tests/safe_mode/run.sh +++ b/tests/safe_mode/run.sh @@ -31,9 +31,6 @@ function run() { "operate-worker create $WORK_DIR/mysql2.toml" \ "true" 1 - run_dm_tracer $WORK_DIR/tracer $TRACER_PORT $cur/conf/dm-tracer.toml - check_port_alive $TRACER_PORT - dmctl_start_task check_sync_diff $WORK_DIR $cur/conf/diff_config.toml @@ -51,9 +48,8 @@ function run() { check_rpc_alive $cur/../bin/check_worker_online 127.0.0.1:$WORKER1_PORT check_rpc_alive $cur/../bin/check_worker_online 127.0.0.1:$WORKER2_PORT - sleep 2 - echo "stat task after set SafeModeInitPhaseSeconds failpoint" - dmctl_start_task + sleep 5 + echo "check sync diff after set SafeModeInitPhaseSeconds failpoint" check_sync_diff $WORK_DIR $cur/conf/diff_config.toml # DM-worker exit when waiting for sharding group synced @@ -67,7 +63,6 @@ function run() { # DM-worker1 is sharding lock owner and exits if [ "$(check_port_return $WORKER1_PORT)" == "0" ]; then echo "DM-worker1 is sharding lock owner and detects it offline" - truncate_trace_events $TRACER_PORT export GO_FAILPOINTS='github.com/pingcap/dm/syncer/SafeModeInitPhaseSeconds=return(0)' 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 @@ -78,7 +73,6 @@ function run() { # DM-worker2 is sharding lock owner and exits if [ "$(check_port_return $WORKER2_PORT)" == "0" ]; then echo "DM-worker2 is sharding lock owner and detects it offline" - truncate_trace_events $TRACER_PORT export GO_FAILPOINTS='github.com/pingcap/dm/syncer/SafeModeInitPhaseSeconds=return(0)' 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 @@ -95,16 +89,9 @@ function run() { exit 1 fi - sleep 2 - echo "start task after restart DDL owner" - task_conf="$cur/conf/dm-task.yaml" - run_dm_ctl $WORK_DIR "127.0.0.1:$MASTER_PORT" \ - "start-task $task_conf" \ - "\"result\": true" 2 \ - "\"worker\": \"127.0.0.1:$OWNER_PORT\"" 1 + sleep 5 + echo "check sync diff after restart DDL owner" check_sync_diff $WORK_DIR $cur/conf/diff_config.toml - - $cur/../bin/check_safe_mode $check_instance_id } cleanup_data safe_mode_target diff --git a/tests/sequence_safe_mode/conf/mysql1.toml b/tests/sequence_safe_mode/conf/mysql1.toml index b66f6527d3..64f4b035e7 100644 --- a/tests/sequence_safe_mode/conf/mysql1.toml +++ b/tests/sequence_safe_mode/conf/mysql1.toml @@ -12,8 +12,8 @@ user = "root" password = "" port = 3306 -[tracer] -enable = true -tracer-addr = "127.0.0.1:8264" -batch-size = 1 -checksum = true +#[tracer] +#enable = true +#tracer-addr = "127.0.0.1:8264" +#batch-size = 1 +#checksum = true diff --git a/tests/sequence_safe_mode/conf/mysql2.toml b/tests/sequence_safe_mode/conf/mysql2.toml index 86ae904c92..d7362dd417 100644 --- a/tests/sequence_safe_mode/conf/mysql2.toml +++ b/tests/sequence_safe_mode/conf/mysql2.toml @@ -12,8 +12,8 @@ user = "root" password = "" port = 3307 -[tracer] -enable = true -tracer-addr = "127.0.0.1:8264" -batch-size = 1 -checksum = true +#[tracer] +#enable = true +#tracer-addr = "127.0.0.1:8264" +#batch-size = 1 +#checksum = true diff --git a/tests/sequence_safe_mode/run.sh b/tests/sequence_safe_mode/run.sh index 970c98f35d..9acba91ccc 100755 --- a/tests/sequence_safe_mode/run.sh +++ b/tests/sequence_safe_mode/run.sh @@ -31,9 +31,6 @@ function run() { "operate-worker create $WORK_DIR/mysql2.toml" \ "true" 1 - run_dm_tracer $WORK_DIR/tracer $TRACER_PORT $cur/conf/dm-tracer.toml - check_port_alive $TRACER_PORT - dmctl_start_task check_sync_diff $WORK_DIR $cur/conf/diff_config.toml @@ -51,8 +48,7 @@ function run() { check_rpc_alive $cur/../bin/check_worker_online 127.0.0.1:$WORKER2_PORT sleep 2 - echo "stat task after reset failpoint" - dmctl_start_task + echo "check sync diff after reset failpoint" check_sync_diff $WORK_DIR $cur/conf/diff_config.toml pkill -hup dm-worker.test 2>/dev/null || true @@ -67,8 +63,6 @@ function run() { check_rpc_alive $cur/../bin/check_worker_online 127.0.0.1:$WORKER2_PORT sleep 2 - echo "stat task after set SequenceShardSyncedExecutionExit failpoint" - dmctl_start_task # DM-worker exit when waiting for sharding group synced run_sql_file $cur/data/db1.increment2.sql $MYSQL_HOST1 $MYSQL_PORT1 @@ -81,7 +75,6 @@ function run() { # DM-worker1 is sharding lock owner and exits if [ "$(check_port_return $WORKER1_PORT)" == "0" ]; then echo "DM-worker1 is sharding lock owner and detects it offline" - truncate_trace_events $TRACER_PORT export GO_FAILPOINTS='github.com/pingcap/dm/syncer/SafeModeInitPhaseSeconds=return(0)' 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 @@ -92,7 +85,6 @@ function run() { # DM-worker2 is sharding lock owner and exits if [ "$(check_port_return $WORKER2_PORT)" == "0" ]; then echo "DM-worker2 is sharding lock owner and detects it offline" - truncate_trace_events $TRACER_PORT export GO_FAILPOINTS='github.com/pingcap/dm/syncer/SafeModeInitPhaseSeconds=return(0)' 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 @@ -109,16 +101,9 @@ function run() { exit 1 fi - sleep 2 - echo "start task after restart DDL owner" - task_conf="$cur/conf/dm-task.yaml" - run_dm_ctl $WORK_DIR "127.0.0.1:$MASTER_PORT" \ - "start-task $task_conf" \ - "\"result\": true" 2 \ - "\"worker\": \"127.0.0.1:$OWNER_PORT\"" 1 + sleep 5 + echo "check sync diff after restart DDL owner" check_sync_diff $WORK_DIR $cur/conf/diff_config.toml - - $cur/../bin/check_safe_mode $check_instance_id } cleanup_data sequence_safe_mode_target diff --git a/tests/sharding/run.sh b/tests/sharding/run.sh index 2fc51673a7..b50e034064 100755 --- a/tests/sharding/run.sh +++ b/tests/sharding/run.sh @@ -45,6 +45,7 @@ function run() { # TODO: check sharding partition id # use sync_diff_inspector to check full dump loader + echo "check sync diff for full dump and load" check_sync_diff $WORK_DIR $cur/conf/diff_config.toml run_sql_file $cur/data/db1.increment.sql $MYSQL_HOST1 $MYSQL_PORT1 @@ -52,6 +53,7 @@ function run() { # TODO: check sharding partition id # use sync_diff_inspector to check data now! + echo "check sync diff for the first increment replication" check_sync_diff $WORK_DIR $cur/conf/diff_config.toml # test create database, create table in sharding mode @@ -60,6 +62,7 @@ function run() { cp $cur/conf/diff_config.toml $WORK_DIR/diff_config.toml printf "\n[[table-config.source-tables]]\ninstance-id = \"source-1\"\nschema = \"sharding2\"\ntable = \"~t.*\"" >> $WORK_DIR/diff_config.toml printf "\n[[table-config.source-tables]]\ninstance-id = \"source-2\"\nschema = \"sharding2\"\ntable = \"~t.*\"" >> $WORK_DIR/diff_config.toml + echo "check sync diff for the second increment replication" check_sync_diff $WORK_DIR $WORK_DIR/diff_config.toml old_checksum=$(checksum) @@ -70,6 +73,7 @@ function run() { cp $cur/conf/diff_config.toml $WORK_DIR/diff_config.toml printf "\n[[table-config.source-tables]]\ninstance-id = \"source-1\"\nschema = \"sharding2\"\ntable = \"~t.*\"" >> $WORK_DIR/diff_config.toml sed -i "s/^# range-placeholder/range = \"uid < 70000\"/g" $WORK_DIR/diff_config.toml + echo "check sync diff for the third increment replication" check_sync_diff $WORK_DIR $WORK_DIR/diff_config.toml new_checksum=$(checksum)