diff --git a/dm/ctl/ctl.go b/dm/ctl/ctl.go index aac8688266..460aa36f33 100644 --- a/dm/ctl/ctl.go +++ b/dm/ctl/ctl.go @@ -67,7 +67,6 @@ func NewRootCmd() *cobra.Command { master.NewSQLInjectCmd(), master.NewShowDDLLocksCmd(), master.NewUnlockDDLLockCmd(), - master.NewBreakDDLLockCmd(), master.NewSwitchRelayMasterCmd(), master.NewPauseRelayCmd(), master.NewResumeRelayCmd(), diff --git a/dm/ctl/master/break_ddl_lock.go b/dm/ctl/master/break_ddl_lock.go deleted file mode 100644 index 4b0fbf5230..0000000000 --- a/dm/ctl/master/break_ddl_lock.go +++ /dev/null @@ -1,104 +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 master - -import ( - "context" - "fmt" - "os" - - "github.com/pingcap/dm/dm/ctl/common" - "github.com/pingcap/dm/dm/pb" - - "github.com/pingcap/errors" - "github.com/spf13/cobra" -) - -// NewBreakDDLLockCmd creates a BreakDDLLock command -func NewBreakDDLLockCmd() *cobra.Command { - cmd := &cobra.Command{ - Use: "break-ddl-lock <-s source ...> [--remove-id] [--exec] [--skip]", - Short: "forcefully break DM-worker's DDL lock", - Run: breakDDLLockFunc, - } - cmd.Flags().StringP("remove-id", "i", "", "DDLLockInfo's ID which need to remove") - cmd.Flags().BoolP("exec", "", false, "whether execute DDL which is blocking") - cmd.Flags().BoolP("skip", "", false, "whether skip DDL which in blocking") - return cmd -} - -// breakDDLLockFunc does break DDL lock -func breakDDLLockFunc(cmd *cobra.Command, _ []string) { - if len(cmd.Flags().Args()) != 1 { - cmd.SetOut(os.Stdout) - cmd.Usage() - return - } - taskName := cmd.Flags().Arg(0) - - sources, err := common.GetSourceArgs(cmd) - if err != nil { - fmt.Println(errors.ErrorStack(err)) - return - } - if len(sources) == 0 { - fmt.Println("must specify at least one source (`-s` / `--source`)") - return - } - - removeLockID, err := cmd.Flags().GetString("remove-id") - if err != nil { - fmt.Println(errors.ErrorStack(err)) - return - } - - exec, err := cmd.Flags().GetBool("exec") - if err != nil { - fmt.Println(errors.ErrorStack(err)) - return - } - - skip, err := cmd.Flags().GetBool("skip") - if err != nil { - fmt.Println(errors.ErrorStack(err)) - return - } - - if len(removeLockID) == 0 && !exec && !skip { - fmt.Println("`remove-id`, `exec`, `skip` must specify at least one") - return - } - - if exec && skip { - fmt.Println("`exec` and `skip` can not specify both at the same time") - return - } - - ctx, cancel := context.WithCancel(context.Background()) - defer cancel() - cli := common.MasterClient() - resp, err := cli.BreakWorkerDDLLock(ctx, &pb.BreakWorkerDDLLockRequest{ - Sources: sources, - Task: taskName, - RemoveLockID: removeLockID, - ExecDDL: exec, - SkipDDL: skip, - }) - if err != nil { - common.PrintLines("can not break DDL lock (in sources %v):\n%s", sources, errors.ErrorStack(err)) - return - } - - common.PrettyPrintResponse(resp) -} diff --git a/dm/ctl/master/unlock_ddl_lock.go b/dm/ctl/master/unlock_ddl_lock.go index 532c42d0f2..cf4c3244ed 100644 --- a/dm/ctl/master/unlock_ddl_lock.go +++ b/dm/ctl/master/unlock_ddl_lock.go @@ -32,7 +32,7 @@ func NewUnlockDDLLockCmd() *cobra.Command { Short: "forcefully unlock DDL lock", Run: unlockDDLLockFunc, } - cmd.Flags().StringP("owner", "o", "", "DM-worker to replace the default owner") + cmd.Flags().StringP("owner", "o", "", "source to replace the default owner") cmd.Flags().BoolP("force-remove", "f", false, "force to remove DDL lock") return cmd } @@ -52,9 +52,9 @@ func unlockDDLLockFunc(cmd *cobra.Command, _ []string) { lockID := cmd.Flags().Arg(0) - sources, err := common.GetSourceArgs(cmd) - if err != nil { - fmt.Println(errors.ErrorStack(err)) + sources, _ := common.GetSourceArgs(cmd) + if len(sources) > 0 { + fmt.Println("shoud not specify any sources") return } @@ -70,11 +70,10 @@ func unlockDDLLockFunc(cmd *cobra.Command, _ []string) { resp, err := cli.UnlockDDLLock(ctx, &pb.UnlockDDLLockRequest{ ID: lockID, ReplaceOwner: owner, - Sources: sources, ForceRemove: forceRemove, }) if err != nil { - common.PrintLines("can not unlock DDL lock %s (in sources %v):\n%s", lockID, sources, errors.ErrorStack(err)) + common.PrintLines("can not unlock DDL lock %s \n%s", lockID, err.Error()) return } diff --git a/dm/master/election.go b/dm/master/election.go index 19a270c8bd..f79dcef38e 100644 --- a/dm/master/election.go +++ b/dm/master/election.go @@ -37,6 +37,7 @@ func (s *Server) electionNotify(ctx context.Context) { // retire from leader if leaderInfo == nil { if s.leader == oneselfLeader { + s.pessimist.Close() s.scheduler.Close() s.Lock() @@ -61,6 +62,11 @@ func (s *Server) electionNotify(ctx context.Context) { log.L().Error("scheduler do not started", zap.Error(err)) } + err = s.pessimist.Start(ctx, s.etcdClient) + if err != nil { + log.L().Error("pessimist do not started", zap.Error(err)) + } + s.Lock() s.leader = oneselfLeader s.closeLeaderClient() diff --git a/dm/master/server.go b/dm/master/server.go index c7229293c2..34540a61dc 100644 --- a/dm/master/server.go +++ b/dm/master/server.go @@ -180,12 +180,6 @@ 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) @@ -226,8 +220,6 @@ func (s *Server) Close() { s.Lock() defer s.Unlock() - s.pessimist.Close() - if s.election != nil { s.election.Close() } @@ -682,6 +674,8 @@ func (s *Server) ShowDDLLocks(ctx context.Context, req *pb.ShowDDLLocksRequest) l.Unsynced = append(l.Unsynced, worker) } } + sort.Strings(l.Synced) + sort.Strings(l.Unsynced) resp.Locks = append(resp.Locks, l) } @@ -695,20 +689,25 @@ func (s *Server) ShowDDLLocks(ctx context.Context, req *pb.ShowDDLLocksRequest) // 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")) - 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")) - return &pb.BreakWorkerDDLLockResponse{ - Result: false, - Msg: "not implement", - }, nil + isLeader, needForward := s.isLeaderAndNeedForward() + if !isLeader { + if needForward { + return s.leaderClient.UnlockDDLLock(ctx, req) + } + return nil, terror.ErrMasterRequestIsNotForwardToLeader + } + + resp := &pb.UnlockDDLLockResponse{ + Result: true, + } + err := s.pessimist.UnlockLock(ctx, req.ID, req.ReplaceOwner, req.ForceRemove) + if err != nil { + resp.Result = false + resp.Msg = terror.Message(err) + } + + return resp, nil } // HandleSQLs implements MasterServer.HandleSQLs diff --git a/dm/master/shardddl/pessimist.go b/dm/master/shardddl/pessimist.go index 46eee1a300..739761f1a0 100644 --- a/dm/master/shardddl/pessimist.go +++ b/dm/master/shardddl/pessimist.go @@ -15,13 +15,22 @@ package shardddl import ( "context" + "sort" "sync" + "time" "go.etcd.io/etcd/clientv3" "go.uber.org/zap" "github.com/pingcap/dm/pkg/log" "github.com/pingcap/dm/pkg/shardddl/pessimism" + "github.com/pingcap/dm/pkg/terror" +) + +var ( + // variables to control the behavior of waiting for the operation to be done for `UnlockLock`. + unlockWaitInterval = time.Second + unlockWaitNum = 10 ) // Pessimist used to coordinate the shard DDL migration in pessimism mode. @@ -146,6 +155,110 @@ func (p *Pessimist) Locks() map[string]*pessimism.Lock { return p.lk.Locks() } +// UnlockLock unlocks a shard DDL lock manually when using `unlock-ddl-lock` command. +// ID: the shard DDL lock ID. +// replaceOwner: the new owner used to replace the original DDL for executing DDL to downstream. +// if the original owner is still exist, we should NOT specify any replaceOwner. +// forceRemove: whether force to remove the DDL lock even fail to unlock it (for the owner). +// if specified forceRemove and then fail to unlock, we may need to use `BreakLock` later. +// NOTE: this function has side effects, if it failed, some status can't revert anymore. +// NOTE: this function should not be called if the lock is still in automatic resolving. +func (p *Pessimist) UnlockLock(ctx context.Context, ID, replaceOwner string, forceRemove bool) error { + // 1. find the lock. + lock := p.lk.FindLock(ID) + if lock == nil { + return terror.ErrMasterLockNotFound.Generate(ID) + } + + // 2. check whether has resolved before (this often should not happen). + if lock.IsResolved() { + err := p.removeLock(lock) + if err != nil { + return err + } + return terror.ErrMasterLockIsResolving.Generatef("the lock %s has been resolved before", ID) + } + + // 3. find out synced & un-synced sources. + ready := lock.Ready() + synced := make([]string, 0, len(ready)) + unsynced := make([]string, 0, len(ready)) + for source, isSynced := range ready { + if isSynced { + synced = append(synced, source) + } else { + unsynced = append(unsynced, source) + } + } + sort.Strings(synced) + sort.Strings(unsynced) + p.logger.Warn("some sources are still not synced before unlock the lock", + zap.Strings("un-synced", unsynced), zap.Strings("synced", synced)) + + // 4. check whether the owner has synced (and it must be synced if using `UnlockLock`). + // if no source synced yet, we should choose to use `BreakLock` instead. + owner := lock.Owner + if replaceOwner != "" { + p.logger.Warn("replace the owner of the lock", zap.String("lock", ID), + zap.String("original owner", owner), zap.String("new owner", replaceOwner)) + owner = replaceOwner + } + if isSynced, ok := ready[owner]; !ok || !isSynced { + return terror.ErrMasterWorkerNotWaitLock.Generatef( + "owner %s is not waiting for a lock, but sources %v are waiting for the lock", owner, synced) + } + + // 5. force to mark the lock as synced. + lock.ForceSynced() + var revertLockSync bool // revert lock's sync status if the operation for the owner is not done. + defer func() { + if revertLockSync { + lock.RevertSynced(unsynced) + p.logger.Warn("revert some sources stage to un-synced", zap.Strings("sources", unsynced)) + } + }() + + // 6. put `exec` operation for the owner, and wait for the owner to be done. + // TODO: `sql-skip`/`sql-replace` supported later. + done, err := p.waitOwnerToBeDone(ctx, lock, owner) + if err != nil { + revertLockSync = true + return err + } else if !done && !forceRemove { // if `forceRemove==true`, we still try to complete following steps. + revertLockSync = true + return terror.ErrMasterOwnerExecDDL.Generatef( + "the owner %s of the lock %s has not done the operation", owner, ID) + } + + // 7. put `skip` operations for other sources, and wait for them to be done. + // NOTE: we don't put operations for un-synced sources, + // because they should be not waiting for these operations. + done, err = p.waitNonOwnerToBeDone(ctx, lock, owner, synced) + if err != nil { + p.logger.Error("the owner has done the exec operation, but fail to wait for some other sources done the skip operation, the lock is still removed", + zap.String("lock", ID), zap.Bool("force remove", forceRemove), zap.String("owner", owner), + zap.Strings("un-synced", unsynced), zap.Strings("synced", synced), zap.Error(err)) + } else if !done { + p.logger.Error("the owner has done the exec operation, but some other sources have not done the skip operation, the lock is still removed", + zap.String("lock", ID), zap.Bool("force remove", forceRemove), zap.String("owner", owner), + zap.Strings("un-synced", unsynced), zap.Strings("synced", synced)) + } + + // 8. remove or clear shard DDL lock and info. + p.lk.RemoveLock(ID) + err2 := p.deleteInfosOps(lock) + + if err != nil && err2 != nil { + return terror.ErrMasterPartWorkerExecDDLFail.AnnotateDelegate( + err, "fail to wait for non-owner sources %v to skip the shard DDL and delete shard DDL infos and operations, %s", unsynced, err2.Error()) + } else if err != nil { + return terror.ErrMasterPartWorkerExecDDLFail.Delegate(err, "fail to wait for non-owner sources to skip the shard DDL") + } else if err2 != nil { + return terror.ErrMasterPartWorkerExecDDLFail.Delegate(err2, "fail to delete shard DDL infos and operations") + } + return nil +} + // 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. @@ -192,7 +305,7 @@ func (p *Pessimist) recoverLocks(ifm map[string]map[string]pessimism.Info, opm m 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) + err := p.handleLock(lock.ID, "") if err != nil { return err } @@ -223,7 +336,7 @@ func (p *Pessimist) handleInfoPut(ctx context.Context, infoCh <-chan pessimism.I } p.logger.Info("the shard DDL lock has synced", zap.String("lock", lockID)) - err = p.handleLock(lockID) + err = p.handleLock(lockID, info.Source) if err != nil { // TODO: add & update metrics. p.logger.Error("fail to handle the shard DDL lock", zap.String("lock", lockID), log.ShortError(err)) @@ -264,12 +377,11 @@ func (p *Pessimist) handleOperationPut(ctx context.Context, opCh <-chan pessimis 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) + err := p.removeLock(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 } @@ -283,7 +395,7 @@ func (p *Pessimist) handleOperationPut(ctx context.Context, opCh <-chan pessimis // 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) + 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)) @@ -293,7 +405,9 @@ func (p *Pessimist) handleOperationPut(ctx context.Context, opCh <-chan pessimis } // handleLock handles a single shard DDL lock. -func (p *Pessimist) handleLock(lockID string) error { +// if source is not empty, it means the function is triggered by an Info with the source, +// this is often called when the source re-PUTed again after an interrupt. +func (p *Pessimist) handleLock(lockID, source string) error { lock := p.lk.FindLock(lockID) if lock == nil { return nil @@ -306,11 +420,10 @@ func (p *Pessimist) handleLock(lockID string) error { 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) + err := p.removeLock(lock) if err != nil { return err } - p.lk.RemoveLock(lock.ID) return nil } @@ -319,7 +432,7 @@ func (p *Pessimist) handleLock(lockID string) error { // 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) + err := p.putOpsForNonOwner(lock, source, true) if err != nil { return err } @@ -327,36 +440,54 @@ func (p *Pessimist) handleLock(lockID string) error { } // put `exec=true` for the owner and skip it if already existing. - return p.putOpForOwner(lock, true) + return p.putOpForOwner(lock, lock.Owner, 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) +func (p *Pessimist) putOpForOwner(lock *pessimism.Lock, owner string, skipDone bool) error { + op := pessimism.NewOperation(lock.ID, lock.Task, 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)) + 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 done", !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)) +func (p *Pessimist) putOpsForNonOwner(lock *pessimism.Lock, onlySource string, skipDone bool) error { + var sources []string + if onlySource != "" { + sources = append(sources, onlySource) + } else { + for source := range lock.Ready() { + if source != lock.Owner { + sources = append(sources, source) + } } } + + ops := make([]pessimism.Operation, 0, len(sources)) + for _, source := range sources { + 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)) + 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 done", !succ), zap.Int64("revision", rev)) + return nil +} + +// removeLock removes the lock in memory and its information in etcd. +func (p *Pessimist) removeLock(lock *pessimism.Lock) error { + // remove all operations for this shard DDL lock. + err := p.deleteOps(lock) + if err != nil { + return err + } + p.lk.RemoveLock(lock.ID) return nil } @@ -376,3 +507,124 @@ func (p *Pessimist) deleteOps(lock *pessimism.Lock) error { p.logger.Info("delete shard DDL lock operations", zap.String("lock", lock.ID), zap.Int64("revision", rev)) return err } + +// deleteInfos DELETEs shard DDL lock infos and operations relative to the lock. +func (p *Pessimist) deleteInfosOps(lock *pessimism.Lock) error { + ready := lock.Ready() + infos := make([]pessimism.Info, 0, len(ready)) + for source := range lock.Ready() { + // NOTE: we rely one the `schema` and `table` not used in `DeleteInfosOperations`. + infos = append(infos, pessimism.NewInfo(lock.Task, source, "", "", lock.DDLs)) + } + 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.DeleteInfosOperations(p.cli, infos, ops) + if err != nil { + return err + } + p.logger.Info("delete shard DDL infos and operations", zap.String("lock", lock.ID), zap.Int64("revision", rev)) + return nil +} + +// waitOwnerToBeDone waits for the owner of the lock to be done for the `exec` operation. +func (p *Pessimist) waitOwnerToBeDone(ctx context.Context, lock *pessimism.Lock, owner string) (bool, error) { + if lock.IsDone(owner) { + p.logger.Info("the owner of the lock has been done before", + zap.String("owner", owner), zap.String("lock", lock.ID)) + return true, nil // done before. + } + + // put the `exec` operation. + err := p.putOpForOwner(lock, owner, true) + if err != nil { + return false, err + } + + // wait for the owner done the operation. + for retryNum := 1; retryNum <= unlockWaitNum; retryNum++ { + select { + case <-ctx.Done(): + return lock.IsDone(owner), ctx.Err() + case <-time.After(unlockWaitInterval): + } + if lock.IsDone(owner) { + break + } else { + p.logger.Info("retry to wait for the owner done the operation", + zap.String("owner", owner), zap.String("lock", lock.ID), zap.Int("retry", retryNum)) + } + } + + return lock.IsDone(owner), nil +} + +// waitNonOwnerToBeDone waits for the non-owner sources of the lock to be done for the `skip` operations. +func (p *Pessimist) waitNonOwnerToBeDone(ctx context.Context, lock *pessimism.Lock, owner string, sources []string) (bool, error) { + // check whether some sources need to wait. + if len(sources) == 0 { + p.logger.Info("no non-owner sources need to wait for the operations", zap.String("lock", lock.ID)) + return true, nil + } + waitSources := make([]string, 0, len(sources)-1) + for _, source := range sources { + if source != owner { + waitSources = append(waitSources, source) + } + } + if len(waitSources) == 0 { + p.logger.Info("no non-owner sources need to wait for the operations", zap.String("lock", lock.ID)) + return true, nil + } + + // check whether already done before. + allDone := func() bool { + for _, source := range waitSources { + if !lock.IsDone(source) { + return false + } + } + return true + } + if allDone() { + p.logger.Info("non-owner sources of the lock have been done before", + zap.String("lock", lock.ID), zap.Strings("sources", waitSources)) + return true, nil + } + + // put `skip` operations. + // NOTE: the auto triggered `putOpsForNonOwner` in `handleOperationPut` by the done operation of the owner + // may put `skip` operations for all non-owner sources, but in order to support `replace owner`, + // we still put `skip` operations for waitSources one more time with `skipDone=true`. + ops := make([]pessimism.Operation, 0, len(waitSources)) + for _, source := range waitSources { + ops = append(ops, pessimism.NewOperation(lock.ID, lock.Task, source, lock.DDLs, false, false)) + } + rev, succ, err := pessimism.PutOperations(p.cli, true, ops...) + if err != nil { + return false, err + } + p.logger.Info("put skip shard DDL lock operations for non-owner", zap.String("lock", lock.ID), zap.Strings("non-owner", waitSources), zap.Bool("already done", !succ), zap.Int64("revision", rev)) + + // wait sources done the operations. + for retryNum := 1; retryNum <= unlockWaitNum; retryNum++ { + var ctxDone bool + select { + case <-ctx.Done(): + ctxDone = true + case <-time.After(unlockWaitInterval): + } + if ctxDone || allDone() { + break + } else { + p.logger.Info("retry to wait for non-owner sources done the operation", + zap.String("lock", lock.ID), zap.Strings("sources", waitSources), zap.Int("retry", retryNum)) + } + } + + return allDone(), nil +} diff --git a/dm/master/shardddl/pessimist_test.go b/dm/master/shardddl/pessimist_test.go index fc16efcb38..4c493478a5 100644 --- a/dm/master/shardddl/pessimist_test.go +++ b/dm/master/shardddl/pessimist_test.go @@ -15,6 +15,8 @@ package shardddl import ( "context" + "fmt" + "sync" "testing" "time" @@ -25,6 +27,7 @@ import ( "github.com/pingcap/dm/dm/common" "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" ) @@ -60,15 +63,15 @@ func (t *testPessimist) TestPessimist(c *C) { var ( watchTimeout = 500 * time.Millisecond - task1 = "task-1" - task2 = "task-2" + task1 = "task-pessimist-1" + task2 = "task-pessimist-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`" + ID1 = fmt.Sprintf("%s-`%s`.`%s`", task1, schema, table) + ID2 = fmt.Sprintf("%s-`%s`.`%s`", task2, schema, table) i11 = pessimism.NewInfo(task1, source1, schema, table, DDLs) i12 = pessimism.NewInfo(task1, source2, schema, table, DDLs) i21 = pessimism.NewInfo(task2, source1, schema, table, DDLs) @@ -106,7 +109,7 @@ func (t *testPessimist) TestPessimist(c *C) { // 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 { + c.Assert(utils.WaitSomething(30, 100*time.Millisecond, func() bool { return len(p.Locks()) == 1 }), IsTrue) c.Assert(p.Locks(), HasKey, ID1) @@ -117,7 +120,7 @@ func (t *testPessimist) TestPessimist(c *C) { // 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 { + c.Assert(utils.WaitSomething(30, 100*time.Millisecond, func() bool { synced, _ = p.Locks()[ID1].IsSynced() return synced }), IsTrue) @@ -125,7 +128,7 @@ func (t *testPessimist) TestPessimist(c *C) { // 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) + pessimism.WatchOperationPut(ctx2, etcdTestCli, task1, source1, rev1+1, opCh) cancel2() close(opCh) c.Assert(len(opCh), Equals, 1) @@ -136,16 +139,17 @@ func (t *testPessimist) TestPessimist(c *C) { // mark exec operation for the owner as `done` (and delete the info). op11c := op11 op11c.Done = true - rev2, err := pessimism.PutOperationDeleteInfo(etcdTestCli, op11c, i11) + done, rev2, err := pessimism.PutOperationDeleteExistInfo(etcdTestCli, op11c, i11) c.Assert(err, IsNil) - c.Assert(utils.WaitSomething(30, 10*time.Millisecond, func() bool { + c.Assert(done, IsTrue) + c.Assert(utils.WaitSomething(30, 100*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) + pessimism.WatchOperationPut(ctx2, etcdTestCli, task1, source2, rev2+1, opCh) cancel2() close(opCh) c.Assert(len(opCh), Equals, 1) @@ -157,9 +161,10 @@ func (t *testPessimist) TestPessimist(c *C) { // the lock should become resolved and deleted. op12c := op12 op12c.Done = true - _, err = pessimism.PutOperationDeleteInfo(etcdTestCli, op12c, i12) + done, _, err = pessimism.PutOperationDeleteExistInfo(etcdTestCli, op12c, i12) c.Assert(err, IsNil) - c.Assert(utils.WaitSomething(30, 10*time.Millisecond, func() bool { + c.Assert(done, IsTrue) + c.Assert(utils.WaitSomething(30, 100*time.Millisecond, func() bool { _, ok := p.Locks()[ID1] return !ok }), IsTrue) @@ -170,7 +175,7 @@ func (t *testPessimist) TestPessimist(c *C) { c.Assert(err, IsNil) _, err = pessimism.PutInfo(etcdTestCli, i22) c.Assert(err, IsNil) - c.Assert(utils.WaitSomething(30, 10*time.Millisecond, func() bool { + c.Assert(utils.WaitSomething(30, 100*time.Millisecond, func() bool { lock := p.Locks()[ID2] if lock == nil { return false @@ -192,7 +197,7 @@ func (t *testPessimist) TestPessimist(c *C) { // 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 { + c.Assert(utils.WaitSomething(30, 100*time.Millisecond, func() bool { synced, _ = p.Locks()[ID2].IsSynced() return synced }), IsTrue) @@ -202,10 +207,9 @@ func (t *testPessimist) TestPessimist(c *C) { ctx2, cancel2 = context.WithTimeout(ctx, watchTimeout) // both source1 and source2 have shard DDL info exist, and neither of them have operation exist. // we must ensure source1 always become the owner of the lock. - pessimism.WatchOperationPut(ctx2, etcdTestCli, task2, source1, rev3, opCh) + pessimism.WatchOperationPut(ctx2, etcdTestCli, task2, source1, rev3+1, 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) @@ -224,8 +228,9 @@ func (t *testPessimist) TestPessimist(c *C) { // mark exec operation for the owner as `done` (and delete the info). op21c := op21 op21c.Done = true - _, err = pessimism.PutOperationDeleteInfo(etcdTestCli, op21c, i21) + done, _, err = pessimism.PutOperationDeleteExistInfo(etcdTestCli, op21c, i21) c.Assert(err, IsNil) + c.Assert(done, IsTrue) c.Assert(utils.WaitSomething(30, 100*time.Millisecond, func() bool { return p.Locks()[ID2].IsDone(source1) }), IsTrue) @@ -243,9 +248,10 @@ func (t *testPessimist) TestPessimist(c *C) { // 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) + done, _, err = pessimism.PutOperationDeleteExistInfo(etcdTestCli, op22c, i22) c.Assert(err, IsNil) - c.Assert(utils.WaitSomething(30, 10*time.Millisecond, func() bool { + c.Assert(done, IsTrue) + c.Assert(utils.WaitSomething(30, 100*time.Millisecond, func() bool { return p.Locks()[ID2].IsDone(source2) }), IsTrue) @@ -264,9 +270,10 @@ func (t *testPessimist) TestPessimist(c *C) { // 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) + done, _, err = pessimism.PutOperationDeleteExistInfo(etcdTestCli, op23c, i23) c.Assert(err, IsNil) - c.Assert(utils.WaitSomething(30, 10*time.Millisecond, func() bool { + c.Assert(done, IsTrue) + c.Assert(utils.WaitSomething(30, 100*time.Millisecond, func() bool { _, ok := p.Locks()[ID2] return !ok }), IsTrue) @@ -279,3 +286,518 @@ func (t *testPessimist) TestPessimist(c *C) { c.Assert(p.Locks(), HasLen, 0) p.Close() // close the Pessimist. } + +func (t *testPessimist) TestSourceReEntrant(c *C) { + // sources (owner or non-owner) may be interrupted and re-run the sequence again. + defer clearTestInfoOperation(c) + + var ( + watchTimeout = 500 * time.Millisecond + task = "task-source-re-entrant" + 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"} + ID = fmt.Sprintf("%s-`%s`.`%s`", task, schema, table) + i11 = pessimism.NewInfo(task, source1, schema, table, DDLs) + i12 = pessimism.NewInfo(task, source2, schema, table, DDLs) + i13 = pessimism.NewInfo(task, source3, schema, table, DDLs) + + sources = func(task string) []string { + switch task { + case task: + 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() + + // 0. start the pessimist. + c.Assert(p.Start(ctx, etcdTestCli), IsNil) + c.Assert(p.Locks(), HasLen, 0) + defer p.Close() + + // 1. PUT i11 and i12, will create a lock but not synced. + _, err := pessimism.PutInfo(etcdTestCli, i11) + c.Assert(err, IsNil) + _, err = pessimism.PutInfo(etcdTestCli, i12) + c.Assert(err, IsNil) + c.Assert(utils.WaitSomething(30, 100*time.Millisecond, func() bool { + lock := p.Locks()[ID] + if lock == nil { + return false + } + _, remain := lock.IsSynced() + return remain == 1 + }), IsTrue) + + // 2. re-PUT i11, to simulate the re-entrant of the owner before the lock become synced. + rev1, err := pessimism.PutInfo(etcdTestCli, i11) + c.Assert(err, IsNil) + + // 3. re-PUT i12, to simulate the re-entrant of the non-owner before the lock become synced. + rev2, err := pessimism.PutInfo(etcdTestCli, i12) + c.Assert(err, IsNil) + + // 4. wait exec operation for the owner become available. + var wg sync.WaitGroup + wg.Add(1) + go func() { + defer wg.Done() + opCh := make(chan pessimism.Operation, 10) + ctx2, cancel2 := context.WithTimeout(ctx, watchTimeout) + pessimism.WatchOperationPut(ctx2, etcdTestCli, task, source1, rev1+1, opCh) + cancel2() + close(opCh) + c.Assert(len(opCh), Equals, 1) + op := <-opCh + c.Assert(op.Exec, IsTrue) + c.Assert(op.Done, IsFalse) + }() + + // 5. put i13, the lock will become synced, then an operation PUT for the owner will be triggered. + _, err = pessimism.PutInfo(etcdTestCli, i13) + c.Assert(err, IsNil) + wg.Wait() + + // 6. re-PUT i11, to simulate the re-entrant of the owner after the lock become synced. + rev1, err = pessimism.PutInfo(etcdTestCli, i11) + c.Assert(err, IsNil) + + // 8. wait exec operation for the owner become available again (with new revision). + opCh := make(chan pessimism.Operation, 10) + ctx2, cancel2 := context.WithTimeout(ctx, watchTimeout) + pessimism.WatchOperationPut(ctx2, etcdTestCli, task, source1, rev1+1, opCh) + cancel2() + close(opCh) + c.Assert(len(opCh), Equals, 1) + op11 := <-opCh + c.Assert(op11.Exec, IsTrue) + c.Assert(op11.Done, IsFalse) + + // 9. wait exec operation for the non-owner become available. + wg.Add(1) + go func() { + defer wg.Done() + opCh = make(chan pessimism.Operation, 10) + ctx2, cancel2 = context.WithTimeout(ctx, watchTimeout) + pessimism.WatchOperationPut(ctx2, etcdTestCli, task, source2, rev2+1, opCh) + cancel2() + close(opCh) + c.Assert(len(opCh), Equals, 1) + op := <-opCh + c.Assert(op.Exec, IsFalse) + c.Assert(op.Done, IsFalse) + }() + + // 10. mark exec operation for the owner as `done` (and delete the info). + op11c := op11 + op11c.Done = true + done, _, err := pessimism.PutOperationDeleteExistInfo(etcdTestCli, op11c, i11) + c.Assert(err, IsNil) + c.Assert(done, IsTrue) + c.Assert(utils.WaitSomething(30, 100*time.Millisecond, func() bool { + return p.Locks()[ID].IsDone(source1) + }), IsTrue) + wg.Wait() + + // 11. re-PUT i12, to simulate the re-entrant of the non-owner after the lock become synced. + rev2, err = pessimism.PutInfo(etcdTestCli, i12) + c.Assert(err, IsNil) + + // 12. wait skip operation for the non-owner become available again (with new revision, without existing done). + opCh = make(chan pessimism.Operation, 10) + ctx2, cancel2 = context.WithTimeout(ctx, watchTimeout) + pessimism.WatchOperationPut(ctx2, etcdTestCli, task, source2, rev2+1, opCh) + cancel2() + close(opCh) + c.Assert(len(opCh), Equals, 1) + op12 := <-opCh + c.Assert(op12.Exec, IsFalse) + c.Assert(op12.Done, IsFalse) + + // 13. mark skip operation for the non-owner as `done` (and delete the info). + op12c := op12 + op12c.Done = true + done, _, err = pessimism.PutOperationDeleteExistInfo(etcdTestCli, op12c, i12) + c.Assert(err, IsNil) + c.Assert(done, IsTrue) + c.Assert(utils.WaitSomething(30, 100*time.Millisecond, func() bool { + return p.Locks()[ID].IsDone(source2) + }), IsTrue) + + // 14. re-PUT i13, to simulate the re-entrant of the owner after the lock become synced. + rev3, err := pessimism.PutInfo(etcdTestCli, i13) + c.Assert(err, IsNil) + + // 15. wait skip operation for the non-owner become available again (with new revision, with existing done). + opCh = make(chan pessimism.Operation, 10) + ctx2, cancel2 = context.WithTimeout(ctx, watchTimeout) + pessimism.WatchOperationPut(ctx2, etcdTestCli, task, source3, rev3+1, opCh) + cancel2() + close(opCh) + c.Assert(len(opCh), Equals, 1) + op13 := <-opCh + c.Assert(op13.Exec, IsFalse) + c.Assert(op13.Done, IsFalse) + + // 16. mark skip operation for the non-owner as `done` (and delete the info). + // the lock should become resolved now. + op13c := op13 + op13c.Done = true + done, _, err = pessimism.PutOperationDeleteExistInfo(etcdTestCli, op13c, i13) + c.Assert(err, IsNil) + c.Assert(done, IsTrue) + c.Assert(utils.WaitSomething(30, 100*time.Millisecond, func() bool { + _, ok := p.Locks()[ID] + return !ok + }), IsTrue) + t.noLockExist(c, p) +} + +func (t *testPessimist) TestUnlockSourceMissBeforeSynced(c *C) { + // some sources may be deleted (miss) before the lock become synced. + defer clearTestInfoOperation(c) + + oriUnlockWaitOwnerInterval := unlockWaitInterval + unlockWaitInterval = 100 * time.Millisecond + defer func() { + unlockWaitInterval = oriUnlockWaitOwnerInterval + }() + + var ( + watchTimeout = 500 * time.Millisecond + task = "task-unlock-source-lack-before-synced" + 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"} + ID = fmt.Sprintf("%s-`%s`.`%s`", task, schema, table) + i11 = pessimism.NewInfo(task, source1, schema, table, DDLs) + i12 = pessimism.NewInfo(task, source2, schema, table, DDLs) + + sources = func(task string) []string { + switch task { + case task: + 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() + + // 0. start the pessimist. + c.Assert(p.Start(ctx, etcdTestCli), IsNil) + c.Assert(p.Locks(), HasLen, 0) + defer p.Close() + + // no lock need to be unlock now. + c.Assert(terror.ErrMasterLockNotFound.Equal(p.UnlockLock(ctx, ID, "", false)), IsTrue) + + // 1. PUT i11 & i12, will create a lock but now synced. + // not PUT info for source3 to simulate the deletion of it. + _, err := pessimism.PutInfo(etcdTestCli, i11) + c.Assert(err, IsNil) + rev1, err := pessimism.PutInfo(etcdTestCli, i12) + c.Assert(err, IsNil) + c.Assert(utils.WaitSomething(30, 100*time.Millisecond, func() bool { + if len(p.Locks()) != 1 { + return false + } + _, remain := p.Locks()[ID].IsSynced() + return remain == 1 + }), IsTrue) + c.Assert(p.Locks(), HasKey, ID) + synced, _ := p.Locks()[ID].IsSynced() + c.Assert(synced, IsFalse) + ready := p.Locks()[ID].Ready() + c.Assert(ready, HasLen, 3) + c.Assert(ready[source1], IsTrue) + c.Assert(ready[source2], IsTrue) + c.Assert(ready[source3], IsFalse) + + // 2. try to unlock the lock manually, but the owner has not done the operation. + // this will put `exec` operation for the done. + c.Assert(terror.ErrMasterOwnerExecDDL.Equal(p.UnlockLock(ctx, ID, "", false)), IsTrue) + + // 3. try to unlock the lock manually, and the owner done the operation. + var wg sync.WaitGroup + wg.Add(2) + go func() { + defer wg.Done() + // put done for the owner. + t.putDoneForSource(ctx, task, source1, i11, true, rev1+1, watchTimeout, c) + }() + go func() { + defer wg.Done() + // put done for the synced `source2`, no need to put done for the un-synced `source3`. + t.putDoneForSource(ctx, task, source2, i12, false, rev1+1, watchTimeout, c) + }() + c.Assert(p.UnlockLock(ctx, ID, "", false), IsNil) + wg.Wait() + + // 4. the lock should be removed now. + t.noLockExist(c, p) +} + +func (t *testPessimist) TestUnlockSourceInterrupt(c *C) { + // operations may be done but not be deleted, and then interrupted. + defer clearTestInfoOperation(c) + + oriUnlockWaitOwnerInterval := unlockWaitInterval + unlockWaitInterval = 100 * time.Millisecond + defer func() { + unlockWaitInterval = oriUnlockWaitOwnerInterval + }() + + var ( + watchTimeout = 500 * time.Millisecond + task = "task-unlock-source-interrupt" + source1 = "mysql-replica-1" + source2 = "mysql-replica-2" + schema, table = "foo", "bar" + DDLs = []string{"ALTER TABLE bar ADD COLUMN c1 INT"} + ID = fmt.Sprintf("%s-`%s`.`%s`", task, schema, table) + i11 = pessimism.NewInfo(task, source1, schema, table, DDLs) + i12 = pessimism.NewInfo(task, source2, schema, table, DDLs) + + sources = func(task string) []string { + switch task { + case task: + return []string{source1, source2} + default: + c.Fatalf("unsupported task %s", task) + } + return []string{} + } + logger = log.L() + p = NewPessimist(&logger, sources) + ) + + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + + // 0. start the pessimist. + c.Assert(p.Start(ctx, etcdTestCli), IsNil) + c.Assert(p.Locks(), HasLen, 0) + defer p.Close() + + // CASE 1: owner interrupted. + // 1. PUT i11 & i12, will create a lock and synced. + rev1, err := pessimism.PutInfo(etcdTestCli, i11) + c.Assert(err, IsNil) + _, err = pessimism.PutInfo(etcdTestCli, i12) + c.Assert(err, IsNil) + c.Assert(utils.WaitSomething(30, 100*time.Millisecond, func() bool { + if len(p.Locks()) != 1 { + return false + } + synced, remain := p.Locks()[ID].IsSynced() + return synced && remain == 0 + }), IsTrue) + c.Assert(p.Locks(), HasKey, ID) + ready := p.Locks()[ID].Ready() + c.Assert(ready, HasLen, 2) + c.Assert(ready[source1], IsTrue) + c.Assert(ready[source2], IsTrue) + + // 2. watch until get not-done operation for the owner. + opCh := make(chan pessimism.Operation, 10) + ctx2, cancel2 := context.WithTimeout(ctx, watchTimeout) + pessimism.WatchOperationPut(ctx2, etcdTestCli, task, "", rev1+1, opCh) + cancel2() + close(opCh) + c.Assert(len(opCh), Equals, 1) + op := <-opCh + c.Assert(op.Source, Equals, source1) + c.Assert(op.Exec, IsTrue) + c.Assert(op.Done, IsFalse) + c.Assert(p.Locks()[ID].IsResolved(), IsFalse) + + // 3. try to unlock the lock, but no `done` marked for the owner. + c.Assert(terror.ErrMasterOwnerExecDDL.Equal(p.UnlockLock(ctx, ID, "", false)), IsTrue) + c.Assert(p.Locks()[ID].IsResolved(), IsFalse) + + // 4. force to remove the lock even no `done` marked for the owner. + c.Assert(p.UnlockLock(ctx, ID, "", true), IsNil) + t.noLockExist(c, p) + + // CASE 2: non-owner interrupted. + // 1. PUT i11 & i12, will create a lock and synced. + rev1, err = pessimism.PutInfo(etcdTestCli, i11) + c.Assert(err, IsNil) + _, err = pessimism.PutInfo(etcdTestCli, i12) + c.Assert(err, IsNil) + c.Assert(utils.WaitSomething(30, 100*time.Millisecond, func() bool { + if len(p.Locks()) != 1 { + return false + } + synced, remain := p.Locks()[ID].IsSynced() + return synced && remain == 0 + }), IsTrue) + c.Assert(p.Locks(), HasKey, ID) + ready = p.Locks()[ID].Ready() + c.Assert(ready, HasLen, 2) + c.Assert(ready[source1], IsTrue) + c.Assert(ready[source2], IsTrue) + + // 2. putDone for the owner. + t.putDoneForSource(ctx, task, source1, i11, true, rev1+1, watchTimeout, c) + c.Assert(utils.WaitSomething(30, 100*time.Millisecond, func() bool { + return p.Locks()[ID].IsDone(source1) + }), IsTrue) + c.Assert(p.Locks()[ID].IsDone(source2), IsFalse) + + // 3. unlock the lock. + c.Assert(p.UnlockLock(ctx, ID, "", false), IsNil) + t.noLockExist(c, p) +} + +func (t *testPessimist) TestUnlockSourceOwnerRemoved(c *C) { + // the owner may be deleted before the lock become synced. + defer clearTestInfoOperation(c) + + oriUnlockWaitOwnerInterval := unlockWaitInterval + unlockWaitInterval = 100 * time.Millisecond + defer func() { + unlockWaitInterval = oriUnlockWaitOwnerInterval + }() + + var ( + watchTimeout = 500 * time.Millisecond + task = "task-unlock-source-owner-removed" + 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"} + ID = fmt.Sprintf("%s-`%s`.`%s`", task, schema, table) + i11 = pessimism.NewInfo(task, source1, schema, table, DDLs) + i12 = pessimism.NewInfo(task, source2, schema, table, DDLs) + + sources = func(task string) []string { + switch task { + case task: + 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() + + // 0. start the pessimist. + c.Assert(p.Start(ctx, etcdTestCli), IsNil) + c.Assert(p.Locks(), HasLen, 0) + defer p.Close() + + // no lock need to be unlock now. + c.Assert(terror.ErrMasterLockNotFound.Equal(p.UnlockLock(ctx, ID, "", false)), IsTrue) + + // 1. PUT i11 & i12, will create a lock but now synced. + _, err := pessimism.PutInfo(etcdTestCli, i11) + c.Assert(err, IsNil) + rev1, err := pessimism.PutInfo(etcdTestCli, i12) + c.Assert(err, IsNil) + c.Assert(utils.WaitSomething(30, 100*time.Millisecond, func() bool { + if len(p.Locks()) != 1 { + return false + } + _, remain := p.Locks()[ID].IsSynced() + return remain == 1 + }), IsTrue) + c.Assert(p.Locks(), HasKey, ID) + synced, _ := p.Locks()[ID].IsSynced() + c.Assert(synced, IsFalse) + ready := p.Locks()[ID].Ready() + c.Assert(ready, HasLen, 3) + c.Assert(ready[source1], IsTrue) + c.Assert(ready[source2], IsTrue) + c.Assert(ready[source3], IsFalse) + + // 2. try to unlock the lock with an un-synced replace owner. + c.Assert(terror.ErrMasterWorkerNotWaitLock.Equal(p.UnlockLock(ctx, ID, source3, false)), IsTrue) + + // 3. try to unlock the lock with a synced replace owner, but the replace owner has not done the operation. + // this will put `exec` operation for the done. + c.Assert(terror.ErrMasterOwnerExecDDL.Equal(p.UnlockLock(ctx, ID, source2, false)), IsTrue) + + // 4. put done for the replace owner then can unlock the lock. + var wg sync.WaitGroup + wg.Add(1) + go func() { + defer wg.Done() + t.putDoneForSource(ctx, task, source2, i11, true, rev1+1, watchTimeout, c) + }() + c.Assert(p.UnlockLock(ctx, ID, source2, false), IsNil) + wg.Wait() + + // 4. the lock should be removed now. + t.noLockExist(c, p) +} + +func (t *testPessimist) putDoneForSource( + ctx context.Context, task, source string, info pessimism.Info, exec bool, + watchRev int64, watchTimeout time.Duration, c *C) { + var ( + wg sync.WaitGroup + opCh = make(chan pessimism.Operation, 10) + ctx2, cancel2 = context.WithTimeout(ctx, watchTimeout) + ) + + wg.Add(2) + go func() { + defer wg.Done() + pessimism.WatchOperationPut(ctx2, etcdTestCli, task, source, watchRev, opCh) + close(opCh) + }() + go func() { + defer func() { + cancel2() + wg.Done() + }() + select { + case <-ctx2.Done(): + c.Fatal("wait for the operation of the source timeout") + case op := <-opCh: + // put `done` after received non-`done`. + c.Assert(op.Exec, Equals, exec) + c.Assert(op.Done, IsFalse) + op.Done = true + done, _, err := pessimism.PutOperationDeleteExistInfo(etcdTestCli, op, info) + c.Assert(err, IsNil) + c.Assert(done, IsTrue) + } + }() + wg.Wait() +} + +func (t *testPessimist) noLockExist(c *C, p *Pessimist) { + c.Assert(p.Locks(), HasLen, 0) + ifm, _, err := pessimism.GetAllInfo(etcdTestCli) + c.Assert(err, IsNil) + c.Assert(ifm, HasLen, 0) + opm, _, err := pessimism.GetAllOperations(etcdTestCli) + c.Assert(err, IsNil) + c.Assert(opm, HasLen, 0) +} diff --git a/dm/master/workerrpc/interface.go b/dm/master/workerrpc/interface.go index 450f3f676a..78efb4e294 100644 --- a/dm/master/workerrpc/interface.go +++ b/dm/master/workerrpc/interface.go @@ -35,8 +35,6 @@ const ( CmdQueryWorkerConfig CmdHandleSubTaskSQLs - CmdExecDDL - CmdBreakDDLLock CmdSwitchRelayMaster CmdOperateRelay @@ -81,8 +79,6 @@ type Response struct { QueryWorkerConfig *pb.QueryWorkerConfigResponse HandleSubTaskSQLs *pb.CommonWorkerResponse - ExecDDL *pb.CommonWorkerResponse - BreakDDLLock *pb.CommonWorkerResponse SwitchRelayMaster *pb.CommonWorkerResponse OperateRelay *pb.OperateRelayResponse diff --git a/dm/pb/dmmaster.pb.go b/dm/pb/dmmaster.pb.go index 18816a03a8..4e72087ee6 100644 --- a/dm/pb/dmmaster.pb.go +++ b/dm/pb/dmmaster.pb.go @@ -1068,14 +1068,11 @@ func (m *ShowDDLLocksResponse) GetLocks() []*DDLLock { // UnlockDDLLockRequest used to unlock (resolve) DDL lock manually // ID: DDL lock ID // replaceOwner: dm-worker used to replace the original DDL lock's owner -// workers: dm-workers need to execute / skip DDL, empty for all dm-workers -// replaceOwner in or not in workers are both fine -// forceRemove: force to remove the DDL lock even unlock fail +// forceRemove: force to remove the DDL lock even fail to execute the DDL for the owner. type UnlockDDLLockRequest struct { - ID string `protobuf:"bytes,1,opt,name=ID,proto3" json:"ID,omitempty"` - ReplaceOwner string `protobuf:"bytes,2,opt,name=replaceOwner,proto3" json:"replaceOwner,omitempty"` - Sources []string `protobuf:"bytes,3,rep,name=sources,proto3" json:"sources,omitempty"` - ForceRemove bool `protobuf:"varint,4,opt,name=forceRemove,proto3" json:"forceRemove,omitempty"` + ID string `protobuf:"bytes,1,opt,name=ID,proto3" json:"ID,omitempty"` + ReplaceOwner string `protobuf:"bytes,2,opt,name=replaceOwner,proto3" json:"replaceOwner,omitempty"` + ForceRemove bool `protobuf:"varint,3,opt,name=forceRemove,proto3" json:"forceRemove,omitempty"` } func (m *UnlockDDLLockRequest) Reset() { *m = UnlockDDLLockRequest{} } @@ -1125,13 +1122,6 @@ func (m *UnlockDDLLockRequest) GetReplaceOwner() string { return "" } -func (m *UnlockDDLLockRequest) GetSources() []string { - if m != nil { - return m.Sources - } - return nil -} - func (m *UnlockDDLLockRequest) GetForceRemove() bool { if m != nil { return m.ForceRemove @@ -1140,9 +1130,8 @@ func (m *UnlockDDLLockRequest) GetForceRemove() bool { } type UnlockDDLLockResponse struct { - Result bool `protobuf:"varint,1,opt,name=result,proto3" json:"result,omitempty"` - Msg string `protobuf:"bytes,2,opt,name=msg,proto3" json:"msg,omitempty"` - Workers []*CommonWorkerResponse `protobuf:"bytes,3,rep,name=workers,proto3" json:"workers,omitempty"` + Result bool `protobuf:"varint,1,opt,name=result,proto3" json:"result,omitempty"` + Msg string `protobuf:"bytes,2,opt,name=msg,proto3" json:"msg,omitempty"` } func (m *UnlockDDLLockResponse) Reset() { *m = UnlockDDLLockResponse{} } @@ -1192,156 +1181,6 @@ func (m *UnlockDDLLockResponse) GetMsg() string { return "" } -func (m *UnlockDDLLockResponse) GetWorkers() []*CommonWorkerResponse { - if m != nil { - return m.Workers - } - return nil -} - -// BreakWorkerDDLLockRequest represents a request for some dm-workers to force to break the DDL lock -// workers: dm-workers need to break 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 BreakWorkerDDLLockRequest struct { - Sources []string `protobuf:"bytes,1,rep,name=sources,proto3" json:"sources,omitempty"` - Task string `protobuf:"bytes,2,opt,name=task,proto3" json:"task,omitempty"` - RemoveLockID string `protobuf:"bytes,3,opt,name=removeLockID,proto3" json:"removeLockID,omitempty"` - ExecDDL bool `protobuf:"varint,4,opt,name=execDDL,proto3" json:"execDDL,omitempty"` - SkipDDL bool `protobuf:"varint,5,opt,name=skipDDL,proto3" json:"skipDDL,omitempty"` -} - -func (m *BreakWorkerDDLLockRequest) Reset() { *m = BreakWorkerDDLLockRequest{} } -func (m *BreakWorkerDDLLockRequest) String() string { return proto.CompactTextString(m) } -func (*BreakWorkerDDLLockRequest) ProtoMessage() {} -func (*BreakWorkerDDLLockRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_f9bef11f2a341f03, []int{19} -} -func (m *BreakWorkerDDLLockRequest) XXX_Unmarshal(b []byte) error { - return m.Unmarshal(b) -} -func (m *BreakWorkerDDLLockRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { - if deterministic { - return xxx_messageInfo_BreakWorkerDDLLockRequest.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 *BreakWorkerDDLLockRequest) XXX_Merge(src proto.Message) { - xxx_messageInfo_BreakWorkerDDLLockRequest.Merge(m, src) -} -func (m *BreakWorkerDDLLockRequest) XXX_Size() int { - return m.Size() -} -func (m *BreakWorkerDDLLockRequest) XXX_DiscardUnknown() { - xxx_messageInfo_BreakWorkerDDLLockRequest.DiscardUnknown(m) -} - -var xxx_messageInfo_BreakWorkerDDLLockRequest proto.InternalMessageInfo - -func (m *BreakWorkerDDLLockRequest) GetSources() []string { - if m != nil { - return m.Sources - } - return nil -} - -func (m *BreakWorkerDDLLockRequest) GetTask() string { - if m != nil { - return m.Task - } - return "" -} - -func (m *BreakWorkerDDLLockRequest) GetRemoveLockID() string { - if m != nil { - return m.RemoveLockID - } - return "" -} - -func (m *BreakWorkerDDLLockRequest) GetExecDDL() bool { - if m != nil { - return m.ExecDDL - } - return false -} - -func (m *BreakWorkerDDLLockRequest) GetSkipDDL() bool { - if m != nil { - return m.SkipDDL - } - return false -} - -type BreakWorkerDDLLockResponse struct { - Result bool `protobuf:"varint,1,opt,name=result,proto3" json:"result,omitempty"` - Msg string `protobuf:"bytes,2,opt,name=msg,proto3" json:"msg,omitempty"` - Sources []*CommonWorkerResponse `protobuf:"bytes,3,rep,name=sources,proto3" json:"sources,omitempty"` -} - -func (m *BreakWorkerDDLLockResponse) Reset() { *m = BreakWorkerDDLLockResponse{} } -func (m *BreakWorkerDDLLockResponse) String() string { return proto.CompactTextString(m) } -func (*BreakWorkerDDLLockResponse) ProtoMessage() {} -func (*BreakWorkerDDLLockResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_f9bef11f2a341f03, []int{20} -} -func (m *BreakWorkerDDLLockResponse) XXX_Unmarshal(b []byte) error { - return m.Unmarshal(b) -} -func (m *BreakWorkerDDLLockResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { - if deterministic { - return xxx_messageInfo_BreakWorkerDDLLockResponse.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 *BreakWorkerDDLLockResponse) XXX_Merge(src proto.Message) { - xxx_messageInfo_BreakWorkerDDLLockResponse.Merge(m, src) -} -func (m *BreakWorkerDDLLockResponse) XXX_Size() int { - return m.Size() -} -func (m *BreakWorkerDDLLockResponse) XXX_DiscardUnknown() { - xxx_messageInfo_BreakWorkerDDLLockResponse.DiscardUnknown(m) -} - -var xxx_messageInfo_BreakWorkerDDLLockResponse proto.InternalMessageInfo - -func (m *BreakWorkerDDLLockResponse) GetResult() bool { - if m != nil { - return m.Result - } - return false -} - -func (m *BreakWorkerDDLLockResponse) GetMsg() string { - if m != nil { - return m.Msg - } - return "" -} - -func (m *BreakWorkerDDLLockResponse) GetSources() []*CommonWorkerResponse { - if m != nil { - return m.Sources - } - return nil -} - // SwitchWorkerRelayMasterRequest represents a request for some dm-workers to switch relay unit's master server // workers: relay unit in these dm-workers need to switch master server type SwitchWorkerRelayMasterRequest struct { @@ -1352,7 +1191,7 @@ func (m *SwitchWorkerRelayMasterRequest) Reset() { *m = SwitchWorkerRela func (m *SwitchWorkerRelayMasterRequest) String() string { return proto.CompactTextString(m) } func (*SwitchWorkerRelayMasterRequest) ProtoMessage() {} func (*SwitchWorkerRelayMasterRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_f9bef11f2a341f03, []int{21} + return fileDescriptor_f9bef11f2a341f03, []int{19} } func (m *SwitchWorkerRelayMasterRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1398,7 +1237,7 @@ func (m *SwitchWorkerRelayMasterResponse) Reset() { *m = SwitchWorkerRel func (m *SwitchWorkerRelayMasterResponse) String() string { return proto.CompactTextString(m) } func (*SwitchWorkerRelayMasterResponse) ProtoMessage() {} func (*SwitchWorkerRelayMasterResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_f9bef11f2a341f03, []int{22} + return fileDescriptor_f9bef11f2a341f03, []int{20} } func (m *SwitchWorkerRelayMasterResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1458,7 +1297,7 @@ func (m *OperateWorkerRelayRequest) Reset() { *m = OperateWorkerRelayReq func (m *OperateWorkerRelayRequest) String() string { return proto.CompactTextString(m) } func (*OperateWorkerRelayRequest) ProtoMessage() {} func (*OperateWorkerRelayRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_f9bef11f2a341f03, []int{23} + return fileDescriptor_f9bef11f2a341f03, []int{21} } func (m *OperateWorkerRelayRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1512,7 +1351,7 @@ func (m *OperateWorkerRelayResponse) Reset() { *m = OperateWorkerRelayRe func (m *OperateWorkerRelayResponse) String() string { return proto.CompactTextString(m) } func (*OperateWorkerRelayResponse) ProtoMessage() {} func (*OperateWorkerRelayResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_f9bef11f2a341f03, []int{24} + return fileDescriptor_f9bef11f2a341f03, []int{22} } func (m *OperateWorkerRelayResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1583,7 +1422,7 @@ func (m *HandleSQLsRequest) Reset() { *m = HandleSQLsRequest{} } func (m *HandleSQLsRequest) String() string { return proto.CompactTextString(m) } func (*HandleSQLsRequest) ProtoMessage() {} func (*HandleSQLsRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_f9bef11f2a341f03, []int{25} + return fileDescriptor_f9bef11f2a341f03, []int{23} } func (m *HandleSQLsRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1671,7 +1510,7 @@ func (m *HandleSQLsResponse) Reset() { *m = HandleSQLsResponse{} } func (m *HandleSQLsResponse) String() string { return proto.CompactTextString(m) } func (*HandleSQLsResponse) ProtoMessage() {} func (*HandleSQLsResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_f9bef11f2a341f03, []int{26} + return fileDescriptor_f9bef11f2a341f03, []int{24} } func (m *HandleSQLsResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1739,7 +1578,7 @@ func (m *PurgeWorkerRelayRequest) Reset() { *m = PurgeWorkerRelayRequest func (m *PurgeWorkerRelayRequest) String() string { return proto.CompactTextString(m) } func (*PurgeWorkerRelayRequest) ProtoMessage() {} func (*PurgeWorkerRelayRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_f9bef11f2a341f03, []int{27} + return fileDescriptor_f9bef11f2a341f03, []int{25} } func (m *PurgeWorkerRelayRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1813,7 +1652,7 @@ func (m *PurgeWorkerRelayResponse) Reset() { *m = PurgeWorkerRelayRespon func (m *PurgeWorkerRelayResponse) String() string { return proto.CompactTextString(m) } func (*PurgeWorkerRelayResponse) ProtoMessage() {} func (*PurgeWorkerRelayResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_f9bef11f2a341f03, []int{28} + return fileDescriptor_f9bef11f2a341f03, []int{26} } func (m *PurgeWorkerRelayResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1871,7 +1710,7 @@ func (m *CheckTaskRequest) Reset() { *m = CheckTaskRequest{} } func (m *CheckTaskRequest) String() string { return proto.CompactTextString(m) } func (*CheckTaskRequest) ProtoMessage() {} func (*CheckTaskRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_f9bef11f2a341f03, []int{29} + return fileDescriptor_f9bef11f2a341f03, []int{27} } func (m *CheckTaskRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1916,7 +1755,7 @@ func (m *CheckTaskResponse) Reset() { *m = CheckTaskResponse{} } func (m *CheckTaskResponse) String() string { return proto.CompactTextString(m) } func (*CheckTaskResponse) ProtoMessage() {} func (*CheckTaskResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_f9bef11f2a341f03, []int{30} + return fileDescriptor_f9bef11f2a341f03, []int{28} } func (m *CheckTaskResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1968,7 +1807,7 @@ func (m *OperateSourceRequest) Reset() { *m = OperateSourceRequest{} } func (m *OperateSourceRequest) String() string { return proto.CompactTextString(m) } func (*OperateSourceRequest) ProtoMessage() {} func (*OperateSourceRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_f9bef11f2a341f03, []int{31} + return fileDescriptor_f9bef11f2a341f03, []int{29} } func (m *OperateSourceRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2021,7 +1860,7 @@ func (m *OperateSourceResponse) Reset() { *m = OperateSourceResponse{} } func (m *OperateSourceResponse) String() string { return proto.CompactTextString(m) } func (*OperateSourceResponse) ProtoMessage() {} func (*OperateSourceResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_f9bef11f2a341f03, []int{32} + return fileDescriptor_f9bef11f2a341f03, []int{30} } func (m *OperateSourceResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2080,7 +1919,7 @@ func (m *RegisterWorkerRequest) Reset() { *m = RegisterWorkerRequest{} } func (m *RegisterWorkerRequest) String() string { return proto.CompactTextString(m) } func (*RegisterWorkerRequest) ProtoMessage() {} func (*RegisterWorkerRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_f9bef11f2a341f03, []int{33} + return fileDescriptor_f9bef11f2a341f03, []int{31} } func (m *RegisterWorkerRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2132,7 +1971,7 @@ func (m *RegisterWorkerResponse) Reset() { *m = RegisterWorkerResponse{} func (m *RegisterWorkerResponse) String() string { return proto.CompactTextString(m) } func (*RegisterWorkerResponse) ProtoMessage() {} func (*RegisterWorkerResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_f9bef11f2a341f03, []int{34} + return fileDescriptor_f9bef11f2a341f03, []int{32} } func (m *RegisterWorkerResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2184,7 +2023,7 @@ func (m *OfflineWorkerRequest) Reset() { *m = OfflineWorkerRequest{} } func (m *OfflineWorkerRequest) String() string { return proto.CompactTextString(m) } func (*OfflineWorkerRequest) ProtoMessage() {} func (*OfflineWorkerRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_f9bef11f2a341f03, []int{35} + return fileDescriptor_f9bef11f2a341f03, []int{33} } func (m *OfflineWorkerRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2236,7 +2075,7 @@ func (m *OfflineWorkerResponse) Reset() { *m = OfflineWorkerResponse{} } func (m *OfflineWorkerResponse) String() string { return proto.CompactTextString(m) } func (*OfflineWorkerResponse) ProtoMessage() {} func (*OfflineWorkerResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_f9bef11f2a341f03, []int{36} + return fileDescriptor_f9bef11f2a341f03, []int{34} } func (m *OfflineWorkerResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2300,8 +2139,6 @@ func init() { proto.RegisterType((*ShowDDLLocksResponse)(nil), "pb.ShowDDLLocksResponse") proto.RegisterType((*UnlockDDLLockRequest)(nil), "pb.UnlockDDLLockRequest") proto.RegisterType((*UnlockDDLLockResponse)(nil), "pb.UnlockDDLLockResponse") - proto.RegisterType((*BreakWorkerDDLLockRequest)(nil), "pb.BreakWorkerDDLLockRequest") - proto.RegisterType((*BreakWorkerDDLLockResponse)(nil), "pb.BreakWorkerDDLLockResponse") proto.RegisterType((*SwitchWorkerRelayMasterRequest)(nil), "pb.SwitchWorkerRelayMasterRequest") proto.RegisterType((*SwitchWorkerRelayMasterResponse)(nil), "pb.SwitchWorkerRelayMasterResponse") proto.RegisterType((*OperateWorkerRelayRequest)(nil), "pb.OperateWorkerRelayRequest") @@ -2323,99 +2160,93 @@ func init() { func init() { proto.RegisterFile("dmmaster.proto", fileDescriptor_f9bef11f2a341f03) } var fileDescriptor_f9bef11f2a341f03 = []byte{ - // 1462 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xbc, 0x58, 0x4d, 0x6f, 0xdb, 0x46, - 0x13, 0x16, 0x29, 0xdb, 0xb1, 0xc6, 0x8e, 0x23, 0x6f, 0x64, 0x89, 0x62, 0x1c, 0xc5, 0x2f, 0xdf, - 0x22, 0x30, 0x72, 0x88, 0x1b, 0xe7, 0x16, 0x20, 0x40, 0x63, 0xcb, 0x41, 0x0d, 0x28, 0xb5, 0x43, - 0x35, 0x28, 0x72, 0x29, 0x40, 0x4b, 0x2b, 0x99, 0x10, 0x45, 0xd2, 0x24, 0x65, 0xc7, 0x2d, 0x8a, - 0x02, 0x3d, 0xf4, 0x52, 0xa0, 0x2d, 0xd0, 0x43, 0xce, 0xfd, 0x37, 0x39, 0x06, 0xe8, 0xa5, 0xc7, - 0x22, 0xe9, 0x0f, 0x29, 0xf6, 0x83, 0xe4, 0xf2, 0x4b, 0x89, 0x5c, 0x54, 0x37, 0xce, 0xcc, 0xee, - 0xcc, 0xb3, 0xb3, 0xb3, 0x3b, 0xcf, 0x12, 0xd6, 0xfa, 0xe3, 0xb1, 0xe1, 0x07, 0xd8, 0xbb, 0xef, - 0x7a, 0x4e, 0xe0, 0x20, 0xd9, 0x3d, 0x51, 0xd7, 0xfa, 0xe3, 0x0b, 0xc7, 0x1b, 0x85, 0x3a, 0x75, - 0x73, 0xe8, 0x38, 0x43, 0x0b, 0xef, 0x18, 0xae, 0xb9, 0x63, 0xd8, 0xb6, 0x13, 0x18, 0x81, 0xe9, - 0xd8, 0x3e, 0xb3, 0x6a, 0x67, 0xd0, 0x7c, 0x66, 0x0e, 0x3d, 0x23, 0xc0, 0x5f, 0xd1, 0x49, 0x3a, - 0xb6, 0x8c, 0x4b, 0x1d, 0x9f, 0x4d, 0xb0, 0x1f, 0xa0, 0x16, 0xc0, 0x9e, 0x69, 0x5b, 0xce, 0xf0, - 0x0b, 0x63, 0x8c, 0x15, 0x69, 0x4b, 0xda, 0xae, 0xe8, 0x82, 0x06, 0x6d, 0x42, 0x85, 0x49, 0xc7, - 0x8e, 0xaf, 0xc8, 0x5b, 0xd2, 0xf6, 0x75, 0x3d, 0x56, 0xa0, 0x3a, 0x2c, 0xf9, 0xce, 0xc4, 0xeb, - 0x61, 0xa5, 0x4c, 0x67, 0x72, 0x49, 0x3b, 0x86, 0xd6, 0x0b, 0xb7, 0x9f, 0x8c, 0xb8, 0xef, 0xd8, - 0x03, 0x73, 0x18, 0xc6, 0xad, 0xc3, 0x52, 0x8f, 0x2a, 0x78, 0x4c, 0x2e, 0x09, 0x1e, 0xe5, 0x84, - 0xc7, 0xcf, 0xa0, 0xda, 0x0d, 0x0c, 0x2f, 0xf8, 0xd2, 0xf0, 0x47, 0xa1, 0x0f, 0x04, 0x0b, 0x81, - 0xe1, 0x8f, 0xb8, 0x07, 0xfa, 0x8d, 0x14, 0xb8, 0xc6, 0x66, 0x10, 0xb4, 0xe5, 0xed, 0x8a, 0x1e, - 0x8a, 0xda, 0x19, 0xac, 0x0b, 0x1e, 0x7c, 0xd7, 0xb1, 0x7d, 0x4c, 0xc2, 0x79, 0xd8, 0x9f, 0x58, - 0x01, 0x75, 0xb2, 0xac, 0x73, 0x09, 0x55, 0xa1, 0x3c, 0xf6, 0x87, 0x1c, 0x03, 0xf9, 0x44, 0xbb, - 0xb1, 0xe3, 0xf2, 0x56, 0x79, 0x7b, 0x65, 0x57, 0xb9, 0xef, 0x9e, 0xdc, 0xdf, 0x77, 0xc6, 0x63, - 0xc7, 0x0e, 0x57, 0xc9, 0x9c, 0xc6, 0x21, 0x1f, 0x42, 0x93, 0xa5, 0xe1, 0x19, 0xdd, 0xc1, 0x8f, - 0xca, 0x80, 0x76, 0x09, 0x6a, 0xde, 0xa4, 0x99, 0x01, 0x3f, 0x48, 0x03, 0x6e, 0x10, 0xc0, 0xcf, - 0x27, 0xd8, 0xbb, 0xec, 0x06, 0x46, 0x30, 0xf1, 0xb3, 0x78, 0xbf, 0x06, 0x74, 0xe4, 0x62, 0x52, - 0x29, 0x62, 0x9a, 0x55, 0x90, 0x1d, 0x97, 0x86, 0x5b, 0xdb, 0x05, 0xe2, 0x83, 0x18, 0x8f, 0x5c, - 0x5d, 0x76, 0x5c, 0xb2, 0x05, 0x36, 0x29, 0x1c, 0x16, 0x97, 0x7e, 0x8b, 0x5b, 0x50, 0x4e, 0x6e, - 0xc1, 0x2f, 0x12, 0xdc, 0x4c, 0x04, 0xe0, 0x8b, 0x9a, 0x16, 0x21, 0x5e, 0xb0, 0x9c, 0xb7, 0xe0, - 0x72, 0xee, 0x0e, 0x2d, 0x7c, 0xec, 0x0e, 0x3d, 0x81, 0x75, 0x96, 0xec, 0xab, 0xd7, 0x95, 0x07, - 0x48, 0x74, 0x31, 0x97, 0xc2, 0x7a, 0x0a, 0x75, 0x61, 0x23, 0x3b, 0xa6, 0x1f, 0x08, 0xd8, 0xed, - 0xf8, 0x24, 0x67, 0x36, 0x24, 0x85, 0xfd, 0x1c, 0x1a, 0x19, 0x3f, 0xf3, 0x28, 0xb4, 0x03, 0xd8, - 0xa0, 0xf6, 0x03, 0xcf, 0x73, 0xbc, 0xab, 0xc3, 0x0f, 0x78, 0x1a, 0x04, 0x37, 0x33, 0xa3, 0xff, - 0x34, 0x8d, 0xbe, 0x1e, 0xa1, 0xa7, 0x6e, 0xb3, 0xe0, 0xf7, 0xe1, 0x66, 0xf7, 0xd4, 0xb9, 0x68, - 0xb7, 0x3b, 0x1d, 0xa7, 0x37, 0xf2, 0xaf, 0x56, 0x35, 0x3f, 0x49, 0x70, 0x8d, 0x7b, 0x40, 0x6b, - 0x20, 0x1f, 0xb6, 0xf9, 0x3c, 0xf9, 0xb0, 0x1d, 0x79, 0x92, 0x05, 0x4f, 0x35, 0x58, 0x74, 0x2e, - 0x6c, 0xec, 0xf1, 0x82, 0x67, 0x02, 0x19, 0xd9, 0x6e, 0x77, 0x58, 0xbd, 0x57, 0x74, 0xfa, 0x4d, - 0x6f, 0xd0, 0x4b, 0xbb, 0x87, 0xfb, 0xca, 0x22, 0xd5, 0x72, 0x09, 0xa9, 0xb0, 0x3c, 0xb1, 0xb9, - 0x65, 0x89, 0x5a, 0x22, 0x59, 0xeb, 0x41, 0x2d, 0xb9, 0xa4, 0x99, 0xd3, 0xf8, 0x3f, 0x58, 0xb4, - 0xc8, 0x54, 0x9e, 0xc4, 0x15, 0x92, 0x44, 0xee, 0x4e, 0x67, 0x16, 0xed, 0x47, 0x09, 0x6a, 0x2f, - 0x6c, 0xf2, 0x1d, 0x1a, 0x78, 0xe6, 0xd2, 0xeb, 0xd7, 0x60, 0xd5, 0xc3, 0xae, 0x65, 0xf4, 0xf0, - 0x11, 0x5d, 0x32, 0x0b, 0x93, 0xd0, 0x15, 0x5f, 0x32, 0x68, 0x0b, 0x56, 0x06, 0x8e, 0xd7, 0xc3, - 0x3a, 0x1e, 0x3b, 0xe7, 0x58, 0x59, 0xa0, 0xc0, 0x45, 0x95, 0x36, 0x81, 0x8d, 0x14, 0x8e, 0xab, - 0x1c, 0x5a, 0xd6, 0x81, 0x3f, 0xe2, 0xd0, 0xf2, 0x81, 0xda, 0xef, 0x12, 0x34, 0xf7, 0x3c, 0x6c, - 0x8c, 0xd8, 0x80, 0x54, 0x12, 0x84, 0x05, 0x49, 0xc9, 0x05, 0xe5, 0x95, 0x03, 0x4d, 0x11, 0x59, - 0x0c, 0x71, 0x71, 0xd8, 0xe6, 0x55, 0x91, 0xd0, 0x11, 0x8f, 0xf8, 0x15, 0xee, 0xb5, 0xdb, 0x1d, - 0x9e, 0x84, 0x50, 0xa4, 0xb1, 0x46, 0xa6, 0x4b, 0x2c, 0x8b, 0xcc, 0xc2, 0x45, 0xed, 0x1b, 0x50, - 0xf3, 0x20, 0xce, 0xe5, 0x52, 0x7b, 0x04, 0xad, 0xee, 0x85, 0x19, 0xf4, 0x4e, 0x05, 0xd2, 0xc0, - 0x7a, 0xe0, 0x07, 0x73, 0xa4, 0x7d, 0x0f, 0x77, 0x0a, 0xe7, 0xce, 0x05, 0xbc, 0x0e, 0x4d, 0xde, - 0xd9, 0x72, 0x48, 0xd6, 0x2d, 0xa1, 0xbf, 0xd1, 0x93, 0x41, 0xad, 0xbc, 0xc1, 0x15, 0xdf, 0x11, - 0xaf, 0x25, 0x50, 0xf3, 0x9c, 0xf2, 0x05, 0x4d, 0xf5, 0xfa, 0xdf, 0xb6, 0xcd, 0x37, 0x12, 0xac, - 0x7f, 0x6e, 0xd8, 0x7d, 0x0b, 0x77, 0x9f, 0x77, 0xfc, 0x69, 0x97, 0x77, 0x93, 0x82, 0x94, 0x29, - 0xc8, 0x0a, 0x71, 0xdc, 0x7d, 0xde, 0x89, 0xb9, 0x83, 0xe1, 0x0d, 0xc3, 0xf3, 0x4b, 0xbf, 0x09, - 0xdd, 0x3c, 0x89, 0xe8, 0xe6, 0x02, 0xf5, 0x13, 0x2b, 0x04, 0x72, 0xb8, 0x28, 0x92, 0x43, 0x42, - 0x62, 0xfd, 0x33, 0xeb, 0xd8, 0x08, 0x02, 0xec, 0xd9, 0xca, 0x12, 0x23, 0xb1, 0xb1, 0x86, 0x5c, - 0x7d, 0xfe, 0xa9, 0xe1, 0xf5, 0x4d, 0x7b, 0xa8, 0x5c, 0xa3, 0xe9, 0x88, 0x64, 0xd2, 0xbe, 0xc5, - 0x95, 0xcc, 0xa5, 0x58, 0x5e, 0x4b, 0xd0, 0x38, 0x9e, 0x78, 0xc3, 0xbc, 0x5a, 0x29, 0xbe, 0x07, - 0x54, 0x58, 0x36, 0x6d, 0xa3, 0x17, 0x98, 0xe7, 0x98, 0x6f, 0x6a, 0x24, 0xd3, 0x3b, 0xc2, 0x1c, - 0x33, 0x1a, 0x5e, 0xd6, 0xe9, 0x37, 0x19, 0x3f, 0x30, 0x2d, 0x4c, 0xb7, 0x84, 0xa5, 0x32, 0x92, - 0x69, 0x26, 0x27, 0x27, 0x6d, 0xd3, 0x8b, 0x32, 0x49, 0x25, 0xed, 0x15, 0x28, 0x59, 0x60, 0x73, - 0xc9, 0xc9, 0x5d, 0xa8, 0xee, 0x9f, 0xe2, 0xde, 0xe8, 0x03, 0x44, 0x4c, 0x7b, 0x0c, 0xeb, 0xc2, - 0xb8, 0x59, 0xa1, 0x69, 0x1d, 0xa8, 0xf1, 0x23, 0xd5, 0xa5, 0x81, 0xc3, 0x50, 0x9b, 0xc2, 0x61, - 0x5a, 0xa5, 0x75, 0x4a, 0xcd, 0xf1, 0x69, 0xe2, 0x5c, 0x5d, 0x4e, 0x70, 0xf5, 0x09, 0x6c, 0xa4, - 0xbc, 0xcd, 0x25, 0x57, 0x07, 0xb0, 0xa1, 0xe3, 0xa1, 0x49, 0xae, 0xb7, 0x70, 0xc8, 0x54, 0xfa, - 0x64, 0xf4, 0xfb, 0x1e, 0xf6, 0x7d, 0x1e, 0x36, 0x14, 0xb5, 0x3d, 0xa8, 0xa7, 0xdd, 0xcc, 0x9c, - 0xcf, 0x36, 0xd4, 0x8e, 0x06, 0x03, 0xcb, 0xb4, 0xf1, 0xbf, 0x41, 0xf2, 0x04, 0x36, 0x52, 0x5e, - 0x66, 0x05, 0x72, 0x4f, 0x87, 0xe5, 0x70, 0xcb, 0xd0, 0x4d, 0xb8, 0x71, 0x68, 0x9f, 0x1b, 0x96, - 0xd9, 0x0f, 0x55, 0xd5, 0x12, 0xba, 0x01, 0x2b, 0xf4, 0xfd, 0xc7, 0x54, 0x55, 0x09, 0x55, 0x61, - 0x95, 0x11, 0x77, 0xae, 0x91, 0xd1, 0x1a, 0x40, 0x37, 0x70, 0x5c, 0x2e, 0x97, 0x77, 0x7f, 0x5e, - 0x85, 0x25, 0xd6, 0x45, 0xd0, 0x4b, 0xa8, 0x44, 0xaf, 0x47, 0x54, 0xa3, 0x05, 0x92, 0x7a, 0x8e, - 0xaa, 0x1b, 0x29, 0x2d, 0x5b, 0x82, 0x76, 0xe7, 0x87, 0x3f, 0xfe, 0xfe, 0x4d, 0x6e, 0x6a, 0x35, - 0xf2, 0x3c, 0xf7, 0x77, 0xce, 0x1f, 0x18, 0x96, 0x7b, 0x6a, 0x3c, 0xd8, 0x21, 0xd5, 0xec, 0x3f, - 0x92, 0xee, 0xa1, 0x01, 0xac, 0x08, 0x8f, 0x22, 0x44, 0xf9, 0x67, 0xf6, 0x19, 0xa6, 0x36, 0x32, - 0x7a, 0x1e, 0xe0, 0x2e, 0x0d, 0xb0, 0xa5, 0xde, 0xca, 0x0b, 0xb0, 0xf3, 0x2d, 0x49, 0xfd, 0x77, - 0x24, 0xce, 0x63, 0x80, 0xf8, 0xa1, 0x82, 0x28, 0xda, 0xcc, 0xdb, 0x47, 0xad, 0xa7, 0xd5, 0x3c, - 0x48, 0x09, 0x59, 0xb0, 0x22, 0x70, 0x7a, 0xa4, 0xa6, 0x48, 0xbe, 0xc0, 0xe2, 0xd5, 0x5b, 0xb9, - 0x36, 0xee, 0xe9, 0x13, 0x0a, 0xb7, 0x85, 0x36, 0x53, 0x70, 0x7d, 0x3a, 0x94, 0xe3, 0x45, 0x07, - 0x00, 0x31, 0x07, 0x47, 0xcd, 0x24, 0x27, 0x17, 0x63, 0xa9, 0x79, 0xa6, 0x08, 0xf4, 0x3e, 0xac, - 0x8a, 0xc4, 0x16, 0xd1, 0x24, 0xe6, 0xb0, 0x77, 0x55, 0xc9, 0x1a, 0x22, 0x27, 0x4f, 0xe1, 0x7a, - 0x82, 0x2f, 0x22, 0x3a, 0x38, 0x8f, 0xca, 0xaa, 0xcd, 0x1c, 0x4b, 0xe4, 0xe7, 0x45, 0xf8, 0x52, - 0x14, 0x5f, 0xf6, 0xe8, 0x76, 0x9c, 0xf1, 0x9c, 0xdf, 0x04, 0x6a, 0xab, 0xc8, 0x1c, 0xb9, 0x7d, - 0x09, 0x8d, 0x82, 0x9f, 0x2d, 0x48, 0x8b, 0x27, 0x17, 0xfd, 0x89, 0x51, 0x0b, 0xef, 0x1b, 0x86, - 0x38, 0x4b, 0x07, 0x19, 0xe2, 0x42, 0x26, 0xcb, 0x10, 0x17, 0xb3, 0x48, 0xad, 0x44, 0x2a, 0x31, - 0xee, 0xb9, 0xac, 0x12, 0x33, 0x6c, 0x82, 0x55, 0x62, 0xb6, 0x35, 0x6b, 0x25, 0xd4, 0x87, 0x46, - 0x01, 0xd9, 0x63, 0x0b, 0x9e, 0xce, 0x22, 0xd5, 0xff, 0x4f, 0x1d, 0x23, 0xa4, 0xb5, 0x9e, 0x25, - 0x5f, 0xf4, 0xe8, 0xdc, 0x16, 0x4e, 0x62, 0xb6, 0x83, 0xb3, 0xf5, 0x17, 0xf3, 0x36, 0xad, 0x84, - 0x8e, 0xa0, 0x9a, 0xee, 0xb2, 0x88, 0x9e, 0x99, 0x02, 0x52, 0xa0, 0x6e, 0xe6, 0x1b, 0x05, 0x87, - 0x28, 0xfb, 0x8b, 0x8f, 0xe1, 0x2c, 0xfc, 0xf5, 0x37, 0x75, 0xe3, 0x1f, 0x41, 0x25, 0xea, 0xb2, - 0xec, 0xba, 0x4b, 0x37, 0x67, 0x76, 0xdd, 0x65, 0x5a, 0x31, 0x3b, 0x2e, 0x89, 0xa6, 0xc8, 0x8e, - 0x4b, 0x5e, 0xd7, 0x65, 0xc7, 0x25, 0xb7, 0x83, 0x6a, 0x25, 0x74, 0x08, 0x6b, 0xc9, 0xf6, 0xc4, - 0xae, 0x81, 0xdc, 0xce, 0xc7, 0xae, 0x81, 0xfc, 0x6e, 0xc6, 0x21, 0x89, 0xfd, 0x85, 0x43, 0xca, - 0x69, 0x5c, 0x1c, 0x52, 0x5e, 0x33, 0xd2, 0x4a, 0x7b, 0xca, 0x9b, 0x77, 0x2d, 0xe9, 0xed, 0xbb, - 0x96, 0xf4, 0xd7, 0xbb, 0x96, 0xf4, 0xeb, 0xfb, 0x56, 0xe9, 0xed, 0xfb, 0x56, 0xe9, 0xcf, 0xf7, - 0xad, 0xd2, 0xc9, 0x12, 0xfd, 0xd7, 0xfa, 0xf0, 0x9f, 0x00, 0x00, 0x00, 0xff, 0xff, 0xa5, 0x52, - 0xbc, 0x72, 0xaf, 0x15, 0x00, 0x00, + // 1375 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xbc, 0x58, 0xcd, 0x6e, 0xdb, 0x46, + 0x10, 0x16, 0xa9, 0x58, 0xb1, 0x46, 0x8e, 0x22, 0x6f, 0xf4, 0xcb, 0x38, 0x8a, 0xcb, 0x16, 0x81, + 0x91, 0x43, 0xdc, 0x38, 0xb7, 0x00, 0x01, 0x6a, 0x5b, 0x0e, 0x6a, 0x40, 0xa9, 0x6d, 0xaa, 0x46, + 0x91, 0x4b, 0x01, 0x5a, 0x5a, 0xc9, 0x84, 0x29, 0x92, 0x26, 0x29, 0xbb, 0x46, 0x51, 0x14, 0xe8, + 0xb5, 0x87, 0x16, 0xe8, 0x21, 0x8f, 0xd2, 0x57, 0xc8, 0x31, 0x40, 0x2f, 0x3d, 0x16, 0x76, 0x1f, + 0xa4, 0xd8, 0x1f, 0x92, 0xcb, 0x1f, 0xa9, 0x91, 0x8b, 0xea, 0xc6, 0x9d, 0xdd, 0xfd, 0xe6, 0xdb, + 0x99, 0xd9, 0x99, 0x59, 0x42, 0x79, 0x30, 0x1e, 0xeb, 0x9e, 0x8f, 0xdd, 0x67, 0x8e, 0x6b, 0xfb, + 0x36, 0x92, 0x9d, 0x13, 0xa5, 0x3c, 0x18, 0x5f, 0xda, 0xee, 0x59, 0x20, 0x53, 0xd6, 0x46, 0xb6, + 0x3d, 0x32, 0xf1, 0xa6, 0xee, 0x18, 0x9b, 0xba, 0x65, 0xd9, 0xbe, 0xee, 0x1b, 0xb6, 0xe5, 0xb1, + 0x59, 0xf5, 0x1c, 0x5a, 0x6f, 0x8c, 0x91, 0xab, 0xfb, 0xf8, 0x1b, 0xba, 0x49, 0xc3, 0xa6, 0x7e, + 0xa5, 0xe1, 0xf3, 0x09, 0xf6, 0x7c, 0xd4, 0x06, 0xd8, 0x31, 0x2c, 0xd3, 0x1e, 0x7d, 0xa5, 0x8f, + 0x71, 0x53, 0x5a, 0x97, 0x36, 0x8a, 0x9a, 0x20, 0x41, 0x6b, 0x50, 0x64, 0xa3, 0x43, 0xdb, 0x6b, + 0xca, 0xeb, 0xd2, 0xc6, 0x3d, 0x2d, 0x12, 0xa0, 0x3a, 0x14, 0x3c, 0x7b, 0xe2, 0xf6, 0x71, 0x33, + 0x4f, 0x77, 0xf2, 0x91, 0x7a, 0x08, 0xed, 0x63, 0x67, 0x10, 0xd7, 0xb8, 0x6b, 0x5b, 0x43, 0x63, + 0x14, 0xe8, 0xad, 0x43, 0xa1, 0x4f, 0x05, 0x5c, 0x27, 0x1f, 0x09, 0x88, 0x72, 0x0c, 0xf1, 0x0b, + 0xa8, 0xf4, 0x7c, 0xdd, 0xf5, 0xbf, 0xd6, 0xbd, 0xb3, 0x00, 0x03, 0xc1, 0x1d, 0x5f, 0xf7, 0xce, + 0x38, 0x02, 0xfd, 0x46, 0x4d, 0xb8, 0xcb, 0x76, 0x10, 0xb6, 0xf9, 0x8d, 0xa2, 0x16, 0x0c, 0xd5, + 0x73, 0x58, 0x15, 0x10, 0x3c, 0xc7, 0xb6, 0x3c, 0x4c, 0xd4, 0xb9, 0xd8, 0x9b, 0x98, 0x3e, 0x05, + 0x59, 0xd6, 0xf8, 0x08, 0x55, 0x20, 0x3f, 0xf6, 0x46, 0x9c, 0x03, 0xf9, 0x44, 0x5b, 0x11, 0x70, + 0x7e, 0x3d, 0xbf, 0x51, 0xda, 0x6a, 0x3e, 0x73, 0x4e, 0x9e, 0xed, 0xda, 0xe3, 0xb1, 0x6d, 0x05, + 0xa7, 0x64, 0xa0, 0x91, 0xca, 0x17, 0xd0, 0x62, 0x66, 0x78, 0x43, 0x3d, 0xf8, 0x51, 0x16, 0x50, + 0xaf, 0x40, 0xc9, 0xda, 0x34, 0x37, 0xe1, 0xe7, 0x49, 0xc2, 0x0d, 0x42, 0xf8, 0x68, 0x82, 0xdd, + 0xab, 0x9e, 0xaf, 0xfb, 0x13, 0x2f, 0xcd, 0xf7, 0x5b, 0x40, 0x07, 0x0e, 0x26, 0x91, 0x22, 0x9a, + 0x59, 0x01, 0xd9, 0x76, 0xa8, 0xba, 0xf2, 0x16, 0x10, 0x0c, 0x32, 0x79, 0xe0, 0x68, 0xb2, 0xed, + 0x10, 0x17, 0x58, 0x24, 0x70, 0x98, 0x5e, 0xfa, 0x2d, 0xba, 0x20, 0x1f, 0x77, 0xc1, 0x2f, 0x12, + 0x3c, 0x88, 0x29, 0xe0, 0x87, 0x9a, 0xa5, 0x21, 0x3a, 0xb0, 0x9c, 0x75, 0xe0, 0x7c, 0xa6, 0x87, + 0xee, 0x7c, 0xac, 0x87, 0xb6, 0x61, 0x95, 0x19, 0xfb, 0xf6, 0x71, 0xe5, 0x02, 0x12, 0x21, 0x16, + 0x12, 0x58, 0xaf, 0xa1, 0x2e, 0x38, 0xb2, 0x6b, 0x78, 0xbe, 0xc0, 0xdd, 0x8a, 0x6e, 0x72, 0xca, + 0x21, 0x09, 0xee, 0x17, 0xd0, 0x48, 0xe1, 0x2c, 0x22, 0xd0, 0xf6, 0xa0, 0x46, 0xe7, 0xf7, 0x5c, + 0xd7, 0x76, 0x6f, 0x4f, 0xdf, 0xe7, 0x66, 0x10, 0x60, 0xe6, 0x66, 0xff, 0x79, 0x92, 0x7d, 0x3d, + 0x64, 0x4f, 0x61, 0xd3, 0xe4, 0x77, 0xe1, 0x41, 0xef, 0xd4, 0xbe, 0xec, 0x74, 0xba, 0x5d, 0xbb, + 0x7f, 0xe6, 0xdd, 0x2e, 0x6a, 0x7e, 0x96, 0xe0, 0x2e, 0x47, 0x40, 0x65, 0x90, 0xf7, 0x3b, 0x7c, + 0x9f, 0xbc, 0xdf, 0x09, 0x91, 0x64, 0x01, 0xa9, 0x0a, 0x4b, 0xf6, 0xa5, 0x85, 0x5d, 0x1e, 0xf0, + 0x6c, 0x40, 0x56, 0x76, 0x3a, 0x5d, 0x16, 0xef, 0x45, 0x8d, 0x7e, 0xd3, 0x0c, 0x7a, 0x65, 0xf5, + 0xf1, 0xa0, 0xb9, 0x44, 0xa5, 0x7c, 0x84, 0x14, 0x58, 0x9e, 0x58, 0x7c, 0xa6, 0x40, 0x67, 0xc2, + 0xb1, 0xda, 0x87, 0x6a, 0xfc, 0x48, 0x73, 0x9b, 0xf1, 0x13, 0x58, 0x32, 0xc9, 0x56, 0x6e, 0xc4, + 0x12, 0x31, 0x22, 0x87, 0xd3, 0xd8, 0x8c, 0x6a, 0x42, 0xf5, 0xd8, 0x22, 0x9f, 0x81, 0x9c, 0x1b, + 0x2e, 0x79, 0x7c, 0x15, 0x56, 0x5c, 0xec, 0x98, 0x7a, 0x1f, 0x1f, 0xd0, 0x13, 0x33, 0x2d, 0x31, + 0x19, 0x5a, 0x87, 0xd2, 0xd0, 0x76, 0xfb, 0x58, 0xc3, 0x63, 0xfb, 0x82, 0x55, 0x9f, 0x65, 0x4d, + 0x14, 0xa9, 0xdb, 0x50, 0x4b, 0x68, 0x9b, 0xf7, 0x4c, 0xea, 0x4b, 0x68, 0xf7, 0x2e, 0x0d, 0xbf, + 0x7f, 0x2a, 0x54, 0x31, 0x96, 0x94, 0x03, 0xea, 0x82, 0x7f, 0xa5, 0xb8, 0x7f, 0x7f, 0x84, 0xc7, + 0x53, 0xf7, 0x2e, 0x24, 0x45, 0x68, 0xd0, 0xe2, 0xa9, 0x36, 0xa3, 0xea, 0x3f, 0x14, 0x12, 0x2e, + 0x75, 0x15, 0x9d, 0xe5, 0x19, 0x77, 0x7a, 0xd0, 0xbe, 0x93, 0x40, 0xc9, 0x02, 0xe5, 0x07, 0x9a, + 0x89, 0xfa, 0xff, 0xe6, 0xf1, 0xf7, 0x12, 0xac, 0x7e, 0xa9, 0x5b, 0x03, 0x13, 0xf7, 0x8e, 0xba, + 0xde, 0xac, 0x6c, 0xd2, 0xa2, 0x24, 0x65, 0x4a, 0xb2, 0x48, 0x80, 0x7b, 0x47, 0xdd, 0xa8, 0x98, + 0xe9, 0xee, 0x28, 0xa8, 0x5a, 0xf4, 0x9b, 0xf4, 0x3f, 0x27, 0x61, 0xff, 0x73, 0x87, 0xe2, 0x44, + 0x02, 0xa1, 0x5b, 0x59, 0x12, 0xbb, 0x15, 0xd2, 0x55, 0x79, 0xe7, 0xe6, 0xa1, 0xee, 0xfb, 0xd8, + 0xb5, 0x9a, 0x05, 0xd6, 0x55, 0x45, 0x12, 0x72, 0x17, 0xbd, 0x53, 0xdd, 0x1d, 0x18, 0xd6, 0xa8, + 0x79, 0x97, 0x9a, 0x23, 0x1c, 0x93, 0x7a, 0x22, 0x9e, 0x64, 0x21, 0xc1, 0xf2, 0x4e, 0x82, 0xc6, + 0xe1, 0xc4, 0x1d, 0x65, 0xc5, 0xca, 0xd4, 0x18, 0x27, 0xa7, 0x30, 0x2c, 0xbd, 0xef, 0x1b, 0x17, + 0x98, 0x3b, 0x35, 0x1c, 0xd3, 0x1c, 0x66, 0x8c, 0xd9, 0xcd, 0xcc, 0x6b, 0xf4, 0x9b, 0xac, 0x1f, + 0x1a, 0x26, 0xa6, 0x2e, 0x61, 0xa6, 0x0c, 0xc7, 0xd4, 0x92, 0x93, 0x93, 0x8e, 0xe1, 0x86, 0x96, + 0xa4, 0x23, 0xf5, 0x3b, 0x68, 0xa6, 0x89, 0x2d, 0xc4, 0x26, 0x4f, 0xa0, 0xb2, 0x7b, 0x8a, 0xfb, + 0x67, 0xff, 0xd2, 0x19, 0xa8, 0xaf, 0x60, 0x55, 0x58, 0x37, 0x77, 0x92, 0xe9, 0x42, 0x95, 0x5f, + 0xa9, 0x1e, 0x55, 0x1c, 0xa8, 0x5a, 0x13, 0x2e, 0xd3, 0x0a, 0x8d, 0x53, 0x3a, 0x1d, 0xdd, 0x26, + 0xde, 0x3c, 0xca, 0xb1, 0xe6, 0x71, 0x02, 0xb5, 0x04, 0xda, 0x42, 0x6c, 0xb5, 0x07, 0x35, 0x0d, + 0x8f, 0x0c, 0x92, 0xde, 0x82, 0x25, 0x33, 0xeb, 0xb9, 0x3e, 0x18, 0xb8, 0xd8, 0xf3, 0xb8, 0xda, + 0x60, 0xa8, 0xee, 0x40, 0x3d, 0x09, 0x33, 0xb7, 0x3d, 0x3b, 0x50, 0x3d, 0x18, 0x0e, 0x4d, 0xc3, + 0xc2, 0xff, 0x85, 0xc9, 0x36, 0xd4, 0x12, 0x28, 0xf3, 0x12, 0x79, 0xaa, 0xc1, 0x72, 0xe0, 0x32, + 0xf4, 0x00, 0xee, 0xef, 0x5b, 0x17, 0xba, 0x69, 0x0c, 0x02, 0x51, 0x25, 0x87, 0xee, 0x43, 0x89, + 0x3e, 0x48, 0x98, 0xa8, 0x22, 0xa1, 0x0a, 0xac, 0xb0, 0x4e, 0x92, 0x4b, 0x64, 0x54, 0x06, 0xe8, + 0xf9, 0xb6, 0xc3, 0xc7, 0xf9, 0xad, 0xdf, 0x4b, 0x50, 0x60, 0x55, 0x04, 0xbd, 0x85, 0x62, 0xf8, + 0x9c, 0x41, 0x55, 0x1a, 0x20, 0x89, 0xf7, 0x91, 0x52, 0x4b, 0x48, 0xd9, 0x11, 0xd4, 0xc7, 0x3f, + 0xfd, 0xf1, 0xf7, 0x6f, 0x72, 0x4b, 0xad, 0x92, 0xf7, 0xa2, 0xb7, 0x79, 0xf1, 0x5c, 0x37, 0x9d, + 0x53, 0xfd, 0xf9, 0x26, 0x89, 0x66, 0xef, 0xa5, 0xf4, 0x14, 0x0d, 0xa1, 0x24, 0x74, 0xe9, 0x88, + 0x36, 0x44, 0xe9, 0x77, 0x81, 0xd2, 0x48, 0xc9, 0xb9, 0x82, 0x27, 0x54, 0xc1, 0xba, 0xf2, 0x30, + 0x4b, 0xc1, 0xe6, 0xf7, 0xc4, 0xf4, 0x3f, 0x10, 0x3d, 0xaf, 0x00, 0xa2, 0xce, 0x19, 0x51, 0xb6, + 0xa9, 0x66, 0x5c, 0xa9, 0x27, 0xc5, 0x5c, 0x49, 0x0e, 0x99, 0x50, 0x12, 0x9a, 0x4c, 0xa4, 0x24, + 0xba, 0x4e, 0xa1, 0xad, 0x54, 0x1e, 0x66, 0xce, 0x71, 0xa4, 0xcf, 0x28, 0xdd, 0x36, 0x5a, 0x4b, + 0xd0, 0xf5, 0xe8, 0x52, 0xce, 0x17, 0xed, 0x01, 0x44, 0x4d, 0x21, 0x6a, 0xc5, 0x9b, 0x44, 0x51, + 0x97, 0x92, 0x35, 0x15, 0x92, 0xde, 0x85, 0x15, 0xb1, 0xd3, 0x42, 0xd4, 0x88, 0x19, 0xed, 0xa4, + 0xd2, 0x4c, 0x4f, 0x84, 0x20, 0xaf, 0xe1, 0x5e, 0xac, 0xb7, 0x41, 0x74, 0x71, 0x56, 0x73, 0xa5, + 0xb4, 0x32, 0x66, 0x42, 0x9c, 0xe3, 0xe0, 0xe9, 0x22, 0x3e, 0x35, 0xd1, 0xa3, 0xc8, 0xe2, 0x19, + 0xef, 0x56, 0xa5, 0x3d, 0x6d, 0x3a, 0x84, 0x7d, 0x0b, 0x8d, 0x29, 0xaf, 0x7f, 0xa4, 0x46, 0x9b, + 0xa7, 0xfd, 0x1a, 0x50, 0xa6, 0xe6, 0x1b, 0x35, 0x47, 0x42, 0x26, 0x2a, 0x8e, 0x2c, 0x64, 0x52, + 0x65, 0x9f, 0x85, 0x4c, 0xba, 0x86, 0xaa, 0x39, 0x34, 0x80, 0xc6, 0x94, 0xae, 0x8c, 0x31, 0x9b, + 0xdd, 0xee, 0x29, 0x9f, 0xce, 0x5c, 0x23, 0x9c, 0xbf, 0x9e, 0xee, 0x92, 0x68, 0x8c, 0x3f, 0x12, + 0xae, 0x4c, 0xba, 0xd4, 0x32, 0xd3, 0x4e, 0x6f, 0xb0, 0xd4, 0x1c, 0x3a, 0x80, 0x4a, 0xb2, 0x1c, + 0x22, 0x1a, 0xdc, 0x53, 0xaa, 0xb7, 0xb2, 0x96, 0x3d, 0x29, 0x00, 0xa2, 0xf4, 0xcf, 0x21, 0xc6, + 0x73, 0xea, 0x4f, 0xa3, 0x99, 0x1e, 0x7a, 0x09, 0xc5, 0xb0, 0x1c, 0xb2, 0xbc, 0x94, 0xac, 0xa2, + 0x2c, 0x2f, 0xa5, 0x6a, 0x26, 0x8b, 0xeb, 0x58, 0xf5, 0x62, 0x71, 0x9d, 0x55, 0x1e, 0x59, 0x5c, + 0x67, 0x96, 0x3a, 0x35, 0x87, 0xf6, 0xa1, 0x1c, 0xaf, 0x23, 0xec, 0xbe, 0x66, 0x96, 0x28, 0x76, + 0x5f, 0xb3, 0xcb, 0x0e, 0xa7, 0x24, 0x16, 0x02, 0x4e, 0x29, 0xa3, 0xc2, 0x70, 0x4a, 0x59, 0x55, + 0x43, 0xcd, 0xed, 0x34, 0xdf, 0x5f, 0xb7, 0xa5, 0x0f, 0xd7, 0x6d, 0xe9, 0xaf, 0xeb, 0xb6, 0xf4, + 0xeb, 0x4d, 0x3b, 0xf7, 0xe1, 0xa6, 0x9d, 0xfb, 0xf3, 0xa6, 0x9d, 0x3b, 0x29, 0xd0, 0xbf, 0x74, + 0x2f, 0xfe, 0x09, 0x00, 0x00, 0xff, 0xff, 0x67, 0xf0, 0x75, 0xc5, 0xe9, 0x13, 0x00, 0x00, } // Reference imports to suppress errors if they are not otherwise used. @@ -2441,9 +2272,6 @@ type MasterClient interface { UnlockDDLLock(ctx context.Context, in *UnlockDDLLockRequest, opts ...grpc.CallOption) (*UnlockDDLLockResponse, error) UpdateMasterConfig(ctx context.Context, in *UpdateMasterConfigRequest, opts ...grpc.CallOption) (*UpdateMasterConfigResponse, error) UpdateWorkerRelayConfig(ctx context.Context, in *UpdateWorkerRelayConfigRequest, opts ...grpc.CallOption) (*CommonWorkerResponse, error) - // BreakDDLLock request some dm-workers to break a DDL lock - // including remove DDLLockInfo and/or execute/skip DDL - BreakWorkerDDLLock(ctx context.Context, in *BreakWorkerDDLLockRequest, opts ...grpc.CallOption) (*BreakWorkerDDLLockResponse, error) HandleSQLs(ctx context.Context, in *HandleSQLsRequest, opts ...grpc.CallOption) (*HandleSQLsResponse, error) // SwitchWorkerRelayMaster requests some dm-workers to switch relay unit's master server SwitchWorkerRelayMaster(ctx context.Context, in *SwitchWorkerRelayMasterRequest, opts ...grpc.CallOption) (*SwitchWorkerRelayMasterResponse, error) @@ -2552,15 +2380,6 @@ func (c *masterClient) UpdateWorkerRelayConfig(ctx context.Context, in *UpdateWo return out, nil } -func (c *masterClient) BreakWorkerDDLLock(ctx context.Context, in *BreakWorkerDDLLockRequest, opts ...grpc.CallOption) (*BreakWorkerDDLLockResponse, error) { - out := new(BreakWorkerDDLLockResponse) - err := c.cc.Invoke(ctx, "/pb.Master/BreakWorkerDDLLock", in, out, opts...) - if err != nil { - return nil, err - } - return out, nil -} - func (c *masterClient) HandleSQLs(ctx context.Context, in *HandleSQLsRequest, opts ...grpc.CallOption) (*HandleSQLsResponse, error) { out := new(HandleSQLsResponse) err := c.cc.Invoke(ctx, "/pb.Master/HandleSQLs", in, out, opts...) @@ -2655,9 +2474,6 @@ type MasterServer interface { UnlockDDLLock(context.Context, *UnlockDDLLockRequest) (*UnlockDDLLockResponse, error) UpdateMasterConfig(context.Context, *UpdateMasterConfigRequest) (*UpdateMasterConfigResponse, error) UpdateWorkerRelayConfig(context.Context, *UpdateWorkerRelayConfigRequest) (*CommonWorkerResponse, error) - // BreakDDLLock request some dm-workers to break a DDL lock - // including remove DDLLockInfo and/or execute/skip DDL - BreakWorkerDDLLock(context.Context, *BreakWorkerDDLLockRequest) (*BreakWorkerDDLLockResponse, error) HandleSQLs(context.Context, *HandleSQLsRequest) (*HandleSQLsResponse, error) // SwitchWorkerRelayMaster requests some dm-workers to switch relay unit's master server SwitchWorkerRelayMaster(context.Context, *SwitchWorkerRelayMasterRequest) (*SwitchWorkerRelayMasterResponse, error) @@ -2708,9 +2524,6 @@ func (*UnimplementedMasterServer) UpdateMasterConfig(ctx context.Context, req *U func (*UnimplementedMasterServer) UpdateWorkerRelayConfig(ctx context.Context, req *UpdateWorkerRelayConfigRequest) (*CommonWorkerResponse, error) { return nil, status.Errorf(codes.Unimplemented, "method UpdateWorkerRelayConfig not implemented") } -func (*UnimplementedMasterServer) BreakWorkerDDLLock(ctx context.Context, req *BreakWorkerDDLLockRequest) (*BreakWorkerDDLLockResponse, error) { - return nil, status.Errorf(codes.Unimplemented, "method BreakWorkerDDLLock not implemented") -} func (*UnimplementedMasterServer) HandleSQLs(ctx context.Context, req *HandleSQLsRequest) (*HandleSQLsResponse, error) { return nil, status.Errorf(codes.Unimplemented, "method HandleSQLs not implemented") } @@ -2905,24 +2718,6 @@ func _Master_UpdateWorkerRelayConfig_Handler(srv interface{}, ctx context.Contex return interceptor(ctx, in, info, handler) } -func _Master_BreakWorkerDDLLock_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { - in := new(BreakWorkerDDLLockRequest) - if err := dec(in); err != nil { - return nil, err - } - if interceptor == nil { - return srv.(MasterServer).BreakWorkerDDLLock(ctx, in) - } - info := &grpc.UnaryServerInfo{ - Server: srv, - FullMethod: "/pb.Master/BreakWorkerDDLLock", - } - handler := func(ctx context.Context, req interface{}) (interface{}, error) { - return srv.(MasterServer).BreakWorkerDDLLock(ctx, req.(*BreakWorkerDDLLockRequest)) - } - return interceptor(ctx, in, info, handler) -} - func _Master_HandleSQLs_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { in := new(HandleSQLsRequest) if err := dec(in); err != nil { @@ -3125,10 +2920,6 @@ var _Master_serviceDesc = grpc.ServiceDesc{ MethodName: "UpdateWorkerRelayConfig", Handler: _Master_UpdateWorkerRelayConfig_Handler, }, - { - MethodName: "BreakWorkerDDLLock", - Handler: _Master_BreakWorkerDDLLock_Handler, - }, { MethodName: "HandleSQLs", Handler: _Master_HandleSQLs_Handler, @@ -4000,16 +3791,7 @@ func (m *UnlockDDLLockRequest) MarshalToSizedBuffer(dAtA []byte) (int, error) { dAtA[i] = 0 } i-- - dAtA[i] = 0x20 - } - if len(m.Sources) > 0 { - for iNdEx := len(m.Sources) - 1; iNdEx >= 0; iNdEx-- { - i -= len(m.Sources[iNdEx]) - copy(dAtA[i:], m.Sources[iNdEx]) - i = encodeVarintDmmaster(dAtA, i, uint64(len(m.Sources[iNdEx]))) - i-- - dAtA[i] = 0x1a - } + dAtA[i] = 0x18 } if len(m.ReplaceOwner) > 0 { i -= len(m.ReplaceOwner) @@ -4048,20 +3830,6 @@ func (m *UnlockDDLLockResponse) MarshalToSizedBuffer(dAtA []byte) (int, error) { _ = i var l int _ = l - if len(m.Workers) > 0 { - for iNdEx := len(m.Workers) - 1; iNdEx >= 0; iNdEx-- { - { - size, err := m.Workers[iNdEx].MarshalToSizedBuffer(dAtA[:i]) - if err != nil { - return 0, err - } - i -= size - i = encodeVarintDmmaster(dAtA, i, uint64(size)) - } - i-- - dAtA[i] = 0x1a - } - } if len(m.Msg) > 0 { i -= len(m.Msg) copy(dAtA[i:], m.Msg) @@ -4082,7 +3850,7 @@ func (m *UnlockDDLLockResponse) MarshalToSizedBuffer(dAtA []byte) (int, error) { return len(dAtA) - i, nil } -func (m *BreakWorkerDDLLockRequest) Marshal() (dAtA []byte, err error) { +func (m *SwitchWorkerRelayMasterRequest) Marshal() (dAtA []byte, err error) { size := m.Size() dAtA = make([]byte, size) n, err := m.MarshalToSizedBuffer(dAtA[:size]) @@ -4092,50 +3860,16 @@ func (m *BreakWorkerDDLLockRequest) Marshal() (dAtA []byte, err error) { return dAtA[:n], nil } -func (m *BreakWorkerDDLLockRequest) MarshalTo(dAtA []byte) (int, error) { +func (m *SwitchWorkerRelayMasterRequest) MarshalTo(dAtA []byte) (int, error) { size := m.Size() return m.MarshalToSizedBuffer(dAtA[:size]) } -func (m *BreakWorkerDDLLockRequest) MarshalToSizedBuffer(dAtA []byte) (int, error) { +func (m *SwitchWorkerRelayMasterRequest) 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] = 0x28 - } - if m.ExecDDL { - i-- - if m.ExecDDL { - dAtA[i] = 1 - } else { - dAtA[i] = 0 - } - i-- - dAtA[i] = 0x20 - } - if len(m.RemoveLockID) > 0 { - i -= len(m.RemoveLockID) - copy(dAtA[i:], m.RemoveLockID) - i = encodeVarintDmmaster(dAtA, i, uint64(len(m.RemoveLockID))) - i-- - dAtA[i] = 0x1a - } - if len(m.Task) > 0 { - i -= len(m.Task) - copy(dAtA[i:], m.Task) - i = encodeVarintDmmaster(dAtA, i, uint64(len(m.Task))) - i-- - dAtA[i] = 0x12 - } if len(m.Sources) > 0 { for iNdEx := len(m.Sources) - 1; iNdEx >= 0; iNdEx-- { i -= len(m.Sources[iNdEx]) @@ -4148,7 +3882,7 @@ func (m *BreakWorkerDDLLockRequest) MarshalToSizedBuffer(dAtA []byte) (int, erro return len(dAtA) - i, nil } -func (m *BreakWorkerDDLLockResponse) Marshal() (dAtA []byte, err error) { +func (m *SwitchWorkerRelayMasterResponse) Marshal() (dAtA []byte, err error) { size := m.Size() dAtA = make([]byte, size) n, err := m.MarshalToSizedBuffer(dAtA[:size]) @@ -4158,12 +3892,12 @@ func (m *BreakWorkerDDLLockResponse) Marshal() (dAtA []byte, err error) { return dAtA[:n], nil } -func (m *BreakWorkerDDLLockResponse) MarshalTo(dAtA []byte) (int, error) { +func (m *SwitchWorkerRelayMasterResponse) MarshalTo(dAtA []byte) (int, error) { size := m.Size() return m.MarshalToSizedBuffer(dAtA[:size]) } -func (m *BreakWorkerDDLLockResponse) MarshalToSizedBuffer(dAtA []byte) (int, error) { +func (m *SwitchWorkerRelayMasterResponse) MarshalToSizedBuffer(dAtA []byte) (int, error) { i := len(dAtA) _ = i var l int @@ -4202,7 +3936,7 @@ func (m *BreakWorkerDDLLockResponse) MarshalToSizedBuffer(dAtA []byte) (int, err return len(dAtA) - i, nil } -func (m *SwitchWorkerRelayMasterRequest) Marshal() (dAtA []byte, err error) { +func (m *OperateWorkerRelayRequest) Marshal() (dAtA []byte, err error) { size := m.Size() dAtA = make([]byte, size) n, err := m.MarshalToSizedBuffer(dAtA[:size]) @@ -4212,12 +3946,12 @@ func (m *SwitchWorkerRelayMasterRequest) Marshal() (dAtA []byte, err error) { return dAtA[:n], nil } -func (m *SwitchWorkerRelayMasterRequest) MarshalTo(dAtA []byte) (int, error) { +func (m *OperateWorkerRelayRequest) MarshalTo(dAtA []byte) (int, error) { size := m.Size() return m.MarshalToSizedBuffer(dAtA[:size]) } -func (m *SwitchWorkerRelayMasterRequest) MarshalToSizedBuffer(dAtA []byte) (int, error) { +func (m *OperateWorkerRelayRequest) MarshalToSizedBuffer(dAtA []byte) (int, error) { i := len(dAtA) _ = i var l int @@ -4228,13 +3962,18 @@ func (m *SwitchWorkerRelayMasterRequest) MarshalToSizedBuffer(dAtA []byte) (int, copy(dAtA[i:], m.Sources[iNdEx]) i = encodeVarintDmmaster(dAtA, i, uint64(len(m.Sources[iNdEx]))) i-- - dAtA[i] = 0xa + dAtA[i] = 0x12 } } + if m.Op != 0 { + i = encodeVarintDmmaster(dAtA, i, uint64(m.Op)) + i-- + dAtA[i] = 0x8 + } return len(dAtA) - i, nil } -func (m *SwitchWorkerRelayMasterResponse) Marshal() (dAtA []byte, err error) { +func (m *OperateWorkerRelayResponse) Marshal() (dAtA []byte, err error) { size := m.Size() dAtA = make([]byte, size) n, err := m.MarshalToSizedBuffer(dAtA[:size]) @@ -4244,12 +3983,12 @@ func (m *SwitchWorkerRelayMasterResponse) Marshal() (dAtA []byte, err error) { return dAtA[:n], nil } -func (m *SwitchWorkerRelayMasterResponse) MarshalTo(dAtA []byte) (int, error) { +func (m *OperateWorkerRelayResponse) MarshalTo(dAtA []byte) (int, error) { size := m.Size() return m.MarshalToSizedBuffer(dAtA[:size]) } -func (m *SwitchWorkerRelayMasterResponse) MarshalToSizedBuffer(dAtA []byte) (int, error) { +func (m *OperateWorkerRelayResponse) MarshalToSizedBuffer(dAtA []byte) (int, error) { i := len(dAtA) _ = i var l int @@ -4265,7 +4004,7 @@ func (m *SwitchWorkerRelayMasterResponse) MarshalToSizedBuffer(dAtA []byte) (int i = encodeVarintDmmaster(dAtA, i, uint64(size)) } i-- - dAtA[i] = 0x1a + dAtA[i] = 0x22 } } if len(m.Msg) > 0 { @@ -4273,7 +4012,7 @@ func (m *SwitchWorkerRelayMasterResponse) MarshalToSizedBuffer(dAtA []byte) (int copy(dAtA[i:], m.Msg) i = encodeVarintDmmaster(dAtA, i, uint64(len(m.Msg))) i-- - dAtA[i] = 0x12 + dAtA[i] = 0x1a } if m.Result { i-- @@ -4283,12 +4022,17 @@ func (m *SwitchWorkerRelayMasterResponse) MarshalToSizedBuffer(dAtA []byte) (int dAtA[i] = 0 } i-- + dAtA[i] = 0x10 + } + if m.Op != 0 { + i = encodeVarintDmmaster(dAtA, i, uint64(m.Op)) + i-- dAtA[i] = 0x8 } return len(dAtA) - i, nil } -func (m *OperateWorkerRelayRequest) Marshal() (dAtA []byte, err error) { +func (m *HandleSQLsRequest) Marshal() (dAtA []byte, err error) { size := m.Size() dAtA = make([]byte, size) n, err := m.MarshalToSizedBuffer(dAtA[:size]) @@ -4298,113 +4042,17 @@ func (m *OperateWorkerRelayRequest) Marshal() (dAtA []byte, err error) { return dAtA[:n], nil } -func (m *OperateWorkerRelayRequest) MarshalTo(dAtA []byte) (int, error) { +func (m *HandleSQLsRequest) MarshalTo(dAtA []byte) (int, error) { size := m.Size() return m.MarshalToSizedBuffer(dAtA[:size]) } -func (m *OperateWorkerRelayRequest) MarshalToSizedBuffer(dAtA []byte) (int, error) { +func (m *HandleSQLsRequest) MarshalToSizedBuffer(dAtA []byte) (int, error) { i := len(dAtA) _ = i var l int _ = l - if len(m.Sources) > 0 { - for iNdEx := len(m.Sources) - 1; iNdEx >= 0; iNdEx-- { - i -= len(m.Sources[iNdEx]) - copy(dAtA[i:], m.Sources[iNdEx]) - i = encodeVarintDmmaster(dAtA, i, uint64(len(m.Sources[iNdEx]))) - i-- - dAtA[i] = 0x12 - } - } - if m.Op != 0 { - i = encodeVarintDmmaster(dAtA, i, uint64(m.Op)) - i-- - dAtA[i] = 0x8 - } - return len(dAtA) - i, nil -} - -func (m *OperateWorkerRelayResponse) 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 *OperateWorkerRelayResponse) MarshalTo(dAtA []byte) (int, error) { - size := m.Size() - return m.MarshalToSizedBuffer(dAtA[:size]) -} - -func (m *OperateWorkerRelayResponse) MarshalToSizedBuffer(dAtA []byte) (int, error) { - i := len(dAtA) - _ = i - var l int - _ = l - if len(m.Sources) > 0 { - for iNdEx := len(m.Sources) - 1; iNdEx >= 0; iNdEx-- { - { - size, err := m.Sources[iNdEx].MarshalToSizedBuffer(dAtA[:i]) - if err != nil { - return 0, err - } - i -= size - i = encodeVarintDmmaster(dAtA, i, uint64(size)) - } - i-- - dAtA[i] = 0x22 - } - } - if len(m.Msg) > 0 { - i -= len(m.Msg) - copy(dAtA[i:], m.Msg) - i = encodeVarintDmmaster(dAtA, i, uint64(len(m.Msg))) - i-- - dAtA[i] = 0x1a - } - if m.Result { - i-- - if m.Result { - dAtA[i] = 1 - } else { - dAtA[i] = 0 - } - i-- - dAtA[i] = 0x10 - } - if m.Op != 0 { - i = encodeVarintDmmaster(dAtA, i, uint64(m.Op)) - i-- - dAtA[i] = 0x8 - } - return len(dAtA) - i, nil -} - -func (m *HandleSQLsRequest) 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 *HandleSQLsRequest) MarshalTo(dAtA []byte) (int, error) { - size := m.Size() - return m.MarshalToSizedBuffer(dAtA[:size]) -} - -func (m *HandleSQLsRequest) MarshalToSizedBuffer(dAtA []byte) (int, error) { - i := len(dAtA) - _ = i - var l int - _ = l - if m.Sharding { + if m.Sharding { i-- if m.Sharding { dAtA[i] = 1 @@ -5329,12 +4977,6 @@ func (m *UnlockDDLLockRequest) Size() (n int) { if l > 0 { n += 1 + l + sovDmmaster(uint64(l)) } - if len(m.Sources) > 0 { - for _, s := range m.Sources { - l = len(s) - n += 1 + l + sovDmmaster(uint64(l)) - } - } if m.ForceRemove { n += 2 } @@ -5354,63 +4996,6 @@ func (m *UnlockDDLLockResponse) Size() (n int) { if l > 0 { n += 1 + l + sovDmmaster(uint64(l)) } - if len(m.Workers) > 0 { - for _, e := range m.Workers { - l = e.Size() - n += 1 + l + sovDmmaster(uint64(l)) - } - } - return n -} - -func (m *BreakWorkerDDLLockRequest) Size() (n int) { - if m == nil { - return 0 - } - var l int - _ = l - if len(m.Sources) > 0 { - for _, s := range m.Sources { - l = len(s) - n += 1 + l + sovDmmaster(uint64(l)) - } - } - l = len(m.Task) - if l > 0 { - n += 1 + l + sovDmmaster(uint64(l)) - } - l = len(m.RemoveLockID) - if l > 0 { - n += 1 + l + sovDmmaster(uint64(l)) - } - if m.ExecDDL { - n += 2 - } - if m.SkipDDL { - n += 2 - } - return n -} - -func (m *BreakWorkerDDLLockResponse) Size() (n int) { - if m == nil { - return 0 - } - var l int - _ = l - if m.Result { - n += 2 - } - l = len(m.Msg) - if l > 0 { - n += 1 + l + sovDmmaster(uint64(l)) - } - if len(m.Sources) > 0 { - for _, e := range m.Sources { - l = e.Size() - n += 1 + l + sovDmmaster(uint64(l)) - } - } return n } @@ -8133,38 +7718,6 @@ func (m *UnlockDDLLockRequest) Unmarshal(dAtA []byte) error { m.ReplaceOwner = string(dAtA[iNdEx:postIndex]) iNdEx = postIndex case 3: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Sources", wireType) - } - var stringLen uint64 - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowDmmaster - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - stringLen |= uint64(b&0x7F) << shift - if b < 0x80 { - break - } - } - intStringLen := int(stringLen) - if intStringLen < 0 { - return ErrInvalidLengthDmmaster - } - postIndex := iNdEx + intStringLen - if postIndex < 0 { - return ErrInvalidLengthDmmaster - } - if postIndex > l { - return io.ErrUnexpectedEOF - } - m.Sources = append(m.Sources, string(dAtA[iNdEx:postIndex])) - iNdEx = postIndex - case 4: if wireType != 0 { return fmt.Errorf("proto: wrong wireType = %d for field ForceRemove", wireType) } @@ -8289,368 +7842,6 @@ func (m *UnlockDDLLockResponse) Unmarshal(dAtA []byte) error { } m.Msg = string(dAtA[iNdEx:postIndex]) iNdEx = postIndex - case 3: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Workers", wireType) - } - var msglen int - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowDmmaster - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - msglen |= int(b&0x7F) << shift - if b < 0x80 { - break - } - } - if msglen < 0 { - return ErrInvalidLengthDmmaster - } - postIndex := iNdEx + msglen - if postIndex < 0 { - return ErrInvalidLengthDmmaster - } - if postIndex > l { - return io.ErrUnexpectedEOF - } - m.Workers = append(m.Workers, &CommonWorkerResponse{}) - if err := m.Workers[len(m.Workers)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { - return err - } - iNdEx = postIndex - default: - iNdEx = preIndex - skippy, err := skipDmmaster(dAtA[iNdEx:]) - if err != nil { - return err - } - if skippy < 0 { - return ErrInvalidLengthDmmaster - } - if (iNdEx + skippy) < 0 { - return ErrInvalidLengthDmmaster - } - if (iNdEx + skippy) > l { - return io.ErrUnexpectedEOF - } - iNdEx += skippy - } - } - - if iNdEx > l { - return io.ErrUnexpectedEOF - } - return nil -} -func (m *BreakWorkerDDLLockRequest) Unmarshal(dAtA []byte) error { - l := len(dAtA) - iNdEx := 0 - for iNdEx < l { - preIndex := iNdEx - var wire uint64 - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowDmmaster - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - wire |= uint64(b&0x7F) << shift - if b < 0x80 { - break - } - } - fieldNum := int32(wire >> 3) - wireType := int(wire & 0x7) - if wireType == 4 { - return fmt.Errorf("proto: BreakWorkerDDLLockRequest: wiretype end group for non-group") - } - if fieldNum <= 0 { - return fmt.Errorf("proto: BreakWorkerDDLLockRequest: illegal tag %d (wire type %d)", fieldNum, wire) - } - switch fieldNum { - case 1: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Sources", wireType) - } - var stringLen uint64 - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowDmmaster - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - stringLen |= uint64(b&0x7F) << shift - if b < 0x80 { - break - } - } - intStringLen := int(stringLen) - if intStringLen < 0 { - return ErrInvalidLengthDmmaster - } - postIndex := iNdEx + intStringLen - if postIndex < 0 { - return ErrInvalidLengthDmmaster - } - if postIndex > l { - return io.ErrUnexpectedEOF - } - m.Sources = append(m.Sources, string(dAtA[iNdEx:postIndex])) - iNdEx = postIndex - case 2: - 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 ErrIntOverflowDmmaster - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - stringLen |= uint64(b&0x7F) << shift - if b < 0x80 { - break - } - } - intStringLen := int(stringLen) - if intStringLen < 0 { - return ErrInvalidLengthDmmaster - } - postIndex := iNdEx + intStringLen - if postIndex < 0 { - return ErrInvalidLengthDmmaster - } - if postIndex > l { - return io.ErrUnexpectedEOF - } - m.Task = string(dAtA[iNdEx:postIndex]) - iNdEx = postIndex - case 3: - 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 ErrIntOverflowDmmaster - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - stringLen |= uint64(b&0x7F) << shift - if b < 0x80 { - break - } - } - intStringLen := int(stringLen) - if intStringLen < 0 { - return ErrInvalidLengthDmmaster - } - postIndex := iNdEx + intStringLen - if postIndex < 0 { - return ErrInvalidLengthDmmaster - } - if postIndex > l { - return io.ErrUnexpectedEOF - } - m.RemoveLockID = string(dAtA[iNdEx:postIndex]) - iNdEx = postIndex - case 4: - 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 ErrIntOverflowDmmaster - } - 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 5: - 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 ErrIntOverflowDmmaster - } - 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 := skipDmmaster(dAtA[iNdEx:]) - if err != nil { - return err - } - if skippy < 0 { - return ErrInvalidLengthDmmaster - } - if (iNdEx + skippy) < 0 { - return ErrInvalidLengthDmmaster - } - if (iNdEx + skippy) > l { - return io.ErrUnexpectedEOF - } - iNdEx += skippy - } - } - - if iNdEx > l { - return io.ErrUnexpectedEOF - } - return nil -} -func (m *BreakWorkerDDLLockResponse) Unmarshal(dAtA []byte) error { - l := len(dAtA) - iNdEx := 0 - for iNdEx < l { - preIndex := iNdEx - var wire uint64 - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowDmmaster - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - wire |= uint64(b&0x7F) << shift - if b < 0x80 { - break - } - } - fieldNum := int32(wire >> 3) - wireType := int(wire & 0x7) - if wireType == 4 { - return fmt.Errorf("proto: BreakWorkerDDLLockResponse: wiretype end group for non-group") - } - if fieldNum <= 0 { - return fmt.Errorf("proto: BreakWorkerDDLLockResponse: illegal tag %d (wire type %d)", fieldNum, wire) - } - switch fieldNum { - case 1: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field Result", wireType) - } - var v int - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowDmmaster - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - v |= int(b&0x7F) << shift - if b < 0x80 { - break - } - } - m.Result = bool(v != 0) - case 2: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Msg", wireType) - } - var stringLen uint64 - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowDmmaster - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - stringLen |= uint64(b&0x7F) << shift - if b < 0x80 { - break - } - } - intStringLen := int(stringLen) - if intStringLen < 0 { - return ErrInvalidLengthDmmaster - } - postIndex := iNdEx + intStringLen - if postIndex < 0 { - return ErrInvalidLengthDmmaster - } - if postIndex > l { - return io.ErrUnexpectedEOF - } - m.Msg = string(dAtA[iNdEx:postIndex]) - iNdEx = postIndex - case 3: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Sources", wireType) - } - var msglen int - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowDmmaster - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - msglen |= int(b&0x7F) << shift - if b < 0x80 { - break - } - } - if msglen < 0 { - return ErrInvalidLengthDmmaster - } - postIndex := iNdEx + msglen - if postIndex < 0 { - return ErrInvalidLengthDmmaster - } - if postIndex > l { - return io.ErrUnexpectedEOF - } - m.Sources = append(m.Sources, &CommonWorkerResponse{}) - if err := m.Sources[len(m.Sources)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { - return err - } - iNdEx = postIndex default: iNdEx = preIndex skippy, err := skipDmmaster(dAtA[iNdEx:]) diff --git a/dm/pbmock/dmmaster.go b/dm/pbmock/dmmaster.go index c181f699dc..1a0bb7823e 100644 --- a/dm/pbmock/dmmaster.go +++ b/dm/pbmock/dmmaster.go @@ -35,26 +35,6 @@ func (m *MockMasterClient) EXPECT() *MockMasterClientMockRecorder { return m.recorder } -// BreakWorkerDDLLock mocks base method -func (m *MockMasterClient) BreakWorkerDDLLock(arg0 context.Context, arg1 *pb.BreakWorkerDDLLockRequest, arg2 ...grpc.CallOption) (*pb.BreakWorkerDDLLockResponse, error) { - m.ctrl.T.Helper() - varargs := []interface{}{arg0, arg1} - for _, a := range arg2 { - varargs = append(varargs, a) - } - ret := m.ctrl.Call(m, "BreakWorkerDDLLock", varargs...) - ret0, _ := ret[0].(*pb.BreakWorkerDDLLockResponse) - ret1, _ := ret[1].(error) - return ret0, ret1 -} - -// BreakWorkerDDLLock indicates an expected call of BreakWorkerDDLLock -func (mr *MockMasterClientMockRecorder) BreakWorkerDDLLock(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, "BreakWorkerDDLLock", reflect.TypeOf((*MockMasterClient)(nil).BreakWorkerDDLLock), varargs...) -} - // CheckTask mocks base method func (m *MockMasterClient) CheckTask(arg0 context.Context, arg1 *pb.CheckTaskRequest, arg2 ...grpc.CallOption) (*pb.CheckTaskResponse, error) { m.ctrl.T.Helper() @@ -438,21 +418,6 @@ func (m *MockMasterServer) EXPECT() *MockMasterServerMockRecorder { return m.recorder } -// BreakWorkerDDLLock mocks base method -func (m *MockMasterServer) BreakWorkerDDLLock(arg0 context.Context, arg1 *pb.BreakWorkerDDLLockRequest) (*pb.BreakWorkerDDLLockResponse, error) { - m.ctrl.T.Helper() - ret := m.ctrl.Call(m, "BreakWorkerDDLLock", arg0, arg1) - ret0, _ := ret[0].(*pb.BreakWorkerDDLLockResponse) - ret1, _ := ret[1].(error) - return ret0, ret1 -} - -// BreakWorkerDDLLock indicates an expected call of BreakWorkerDDLLock -func (mr *MockMasterServerMockRecorder) BreakWorkerDDLLock(arg0, arg1 interface{}) *gomock.Call { - mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "BreakWorkerDDLLock", reflect.TypeOf((*MockMasterServer)(nil).BreakWorkerDDLLock), arg0, arg1) -} - // CheckTask mocks base method func (m *MockMasterServer) CheckTask(arg0 context.Context, arg1 *pb.CheckTaskRequest) (*pb.CheckTaskResponse, error) { m.ctrl.T.Helper() diff --git a/dm/proto/dmmaster.proto b/dm/proto/dmmaster.proto index 1b2553403b..19accbd7eb 100644 --- a/dm/proto/dmmaster.proto +++ b/dm/proto/dmmaster.proto @@ -37,10 +37,6 @@ service Master { rpc UpdateWorkerRelayConfig(UpdateWorkerRelayConfigRequest) returns (CommonWorkerResponse) {} - // BreakDDLLock request some dm-workers to break a DDL lock - // including remove DDLLockInfo and/or execute/skip DDL - rpc BreakWorkerDDLLock (BreakWorkerDDLLockRequest) returns (BreakWorkerDDLLockResponse) {} - rpc HandleSQLs (HandleSQLsRequest) returns (HandleSQLsResponse) {} // SwitchWorkerRelayMaster requests some dm-workers to switch relay unit's master server @@ -189,41 +185,16 @@ message ShowDDLLocksResponse { // UnlockDDLLockRequest used to unlock (resolve) DDL lock manually // ID: DDL lock ID // replaceOwner: dm-worker used to replace the original DDL lock's owner -// workers: dm-workers need to execute / skip DDL, empty for all dm-workers -// replaceOwner in or not in workers are both fine -// forceRemove: force to remove the DDL lock even unlock fail +// forceRemove: force to remove the DDL lock even fail to execute the DDL for the owner. message UnlockDDLLockRequest { string ID = 1; string replaceOwner = 2; - repeated string sources = 3; - bool forceRemove = 4; + bool forceRemove = 3; } message UnlockDDLLockResponse { bool result = 1; string msg = 2; - repeated CommonWorkerResponse workers = 3; -} - -// BreakWorkerDDLLockRequest represents a request for some dm-workers to force to break the DDL lock -// workers: dm-workers need to break 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 BreakWorkerDDLLockRequest { - repeated string sources = 1; - string task = 2; - string removeLockID = 3; - bool execDDL = 4; - bool skipDDL = 5; -} - -message BreakWorkerDDLLockResponse { - bool result = 1; - string msg = 2; - repeated CommonWorkerResponse sources = 3; } // SwitchWorkerRelayMasterRequest represents a request for some dm-workers to switch relay unit's master server diff --git a/pkg/shardddl/pessimism/info.go b/pkg/shardddl/pessimism/info.go index bf1b98dbe1..c64b0a0a56 100644 --- a/pkg/shardddl/pessimism/info.go +++ b/pkg/shardddl/pessimism/info.go @@ -24,6 +24,7 @@ import ( "github.com/pingcap/dm/dm/common" "github.com/pingcap/dm/pkg/etcdutil" "github.com/pingcap/dm/pkg/log" + "go.etcd.io/etcd/clientv3/clientv3util" ) // Info represents the shard DDL information. @@ -165,3 +166,8 @@ func WatchInfoPut(ctx context.Context, cli *clientv3.Client, revision int64, out func deleteInfoOp(info Info) clientv3.Op { return clientv3.OpDelete(common.ShardDDLPessimismInfoKeyAdapter.Encode(info.Task, info.Source)) } + +// infoExistCmp returns a etcd Cmp which indicates the info exists. +func infoExistCmp(info Info) clientv3.Cmp { + return clientv3util.KeyExists(common.ShardDDLPessimismInfoKeyAdapter.Encode(info.Task, info.Source)) +} diff --git a/pkg/shardddl/pessimism/lock.go b/pkg/shardddl/pessimism/lock.go index a4883960b4..23ed71d9b7 100644 --- a/pkg/shardddl/pessimism/lock.go +++ b/pkg/shardddl/pessimism/lock.go @@ -100,6 +100,19 @@ func (l *Lock) ForceSynced() { l.remain = 0 } +// RevertSynced reverts the synced stage of the sources. +func (l *Lock) RevertSynced(sources []string) { + l.mu.Lock() + defer l.mu.Unlock() + + for _, source := range sources { + if synced, ok := l.ready[source]; ok && synced { + l.ready[source] = false + l.remain++ + } + } +} + // IsSynced returns whether the lock has synced. func (l *Lock) IsSynced() (bool, int) { l.mu.RLock() diff --git a/pkg/shardddl/pessimism/lock_test.go b/pkg/shardddl/pessimism/lock_test.go index 9390b44d28..59f1cb7562 100644 --- a/pkg/shardddl/pessimism/lock_test.go +++ b/pkg/shardddl/pessimism/lock_test.go @@ -129,4 +129,13 @@ func (t *testLock) TestLock(c *C) { synced, remain = l3.IsSynced() c.Assert(synced, IsTrue) c.Assert(remain, Equals, 0) + + // revert the synced stage. + l3.RevertSynced([]string{source2}) + synced, remain = l3.IsSynced() + c.Assert(synced, IsFalse) + c.Assert(remain, Equals, 1) + ready := l3.Ready() + c.Assert(ready[source1], IsTrue) + c.Assert(ready[source2], IsFalse) } diff --git a/pkg/shardddl/pessimism/operation.go b/pkg/shardddl/pessimism/operation.go index 12506e283a..0d9ea8b783 100644 --- a/pkg/shardddl/pessimism/operation.go +++ b/pkg/shardddl/pessimism/operation.go @@ -134,18 +134,10 @@ func PutOperations(cli *clientv3.Client, skipDone bool, ops ...Operation) (rev i 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)) + opsDel = append(opsDel, deleteOperationOp(op)) } - - 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 + _, rev, err := etcdutil.DoOpsInOneTxn(cli, opsDel...) + return rev, err } // GetAllOperations gets all DDL lock operation in etcd currently. @@ -224,3 +216,8 @@ func putOperationOp(o Operation) (clientv3.Op, error) { return clientv3.OpPut(key, value), nil } + +// deleteOperationOp returns a DELETE etcd operation for Operation. +func deleteOperationOp(op Operation) clientv3.Op { + return clientv3.OpDelete(common.ShardDDLPessimismOperationKeyAdapter.Encode(op.Task, op.Source)) +} diff --git a/pkg/shardddl/pessimism/ops.go b/pkg/shardddl/pessimism/ops.go index 6eb0e132b6..1bc45d3174 100644 --- a/pkg/shardddl/pessimism/ops.go +++ b/pkg/shardddl/pessimism/ops.go @@ -23,21 +23,38 @@ import ( // TODO(csuzhangxc): assign terror code before merged into the master branch. -// PutOperationDeleteInfo puts an operation and deletes an info in one txn. +// PutOperationDeleteExistInfo puts an operation and deletes an info in one txn, +// if the info exists in etcd before. // This function should often be called by DM-worker. -func PutOperationDeleteInfo(cli *clientv3.Client, op Operation, info Info) (int64, error) { +func PutOperationDeleteExistInfo(cli *clientv3.Client, op Operation, info Info) (done bool, rev int64, err error) { putOp, err := putOperationOp(op) if err != nil { - return 0, nil + return false, 0, nil } delOp := deleteInfoOp(info) + infoCmp := infoExistCmp(info) + ctx, cancel := context.WithTimeout(cli.Ctx(), etcdutil.DefaultRequestTimeout) defer cancel() - resp, err := cli.Txn(ctx).Then(putOp, delOp).Commit() + resp, err := cli.Txn(ctx).If(infoCmp).Then(putOp, delOp).Commit() if err != nil { - return 0, err + return false, 0, err + } + return resp.Succeeded, resp.Header.Revision, nil +} + +// DeleteInfosOperations deletes the shard DDL infos and operations in etcd. +// This function should often be called by DM-master when calling UnlockDDL. +func DeleteInfosOperations(cli *clientv3.Client, infos []Info, ops []Operation) (int64, error) { + opsDel := make([]clientv3.Op, 0, len(infos)+len(ops)) + for _, info := range infos { + opsDel = append(opsDel, deleteInfoOp(info)) + } + for _, op := range ops { + opsDel = append(opsDel, deleteOperationOp(op)) } - return resp.Header.Revision, nil + _, rev, err := etcdutil.DoOpsInOneTxn(cli, opsDel...) + return rev, err } diff --git a/pkg/shardddl/pessimism/ops_test.go b/pkg/shardddl/pessimism/ops_test.go index d0e6ebbbaf..7519175ad2 100644 --- a/pkg/shardddl/pessimism/ops_test.go +++ b/pkg/shardddl/pessimism/ops_test.go @@ -45,8 +45,9 @@ func (t *testForEtcd) TestPutOperationDeleteInfo(c *C) { c.Assert(opm, HasLen, 0) // put operation & delete info. - _, err = PutOperationDeleteInfo(etcdTestCli, op, info) + done, _, err := PutOperationDeleteExistInfo(etcdTestCli, op, info) c.Assert(err, IsNil) + c.Assert(done, IsTrue) // verify no info exit. ifm, _, err = GetAllInfo(etcdTestCli) @@ -59,4 +60,43 @@ func (t *testForEtcd) TestPutOperationDeleteInfo(c *C) { c.Assert(opm, HasLen, 1) c.Assert(opm, HasKey, task) c.Assert(opm[task][source], DeepEquals, op) + + // try put operation & delete info again, failed. + done, _, err = PutOperationDeleteExistInfo(etcdTestCli, op, info) + c.Assert(err, IsNil) + c.Assert(done, IsFalse) + + // PUT info and operation. + _, err = PutInfo(etcdTestCli, info) + c.Assert(err, IsNil) + _, _, err = PutOperations(etcdTestCli, true, op) + 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 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) + + // DELETE info and operation. + _, err = DeleteInfosOperations(etcdTestCli, []Info{info}, []Operation{op}) + c.Assert(err, IsNil) + + // verify no info exit. + ifm, _, err = GetAllInfo(etcdTestCli) + c.Assert(err, IsNil) + c.Assert(ifm, HasLen, 0) + + // verify no operations exist. + opm, _, err = GetAllOperations(etcdTestCli) + c.Assert(err, IsNil) + c.Assert(opm, HasLen, 0) } diff --git a/syncer/shardddl/pessimist.go b/syncer/shardddl/pessimist.go index b89e4fea30..16d516b369 100644 --- a/syncer/shardddl/pessimist.go +++ b/syncer/shardddl/pessimist.go @@ -22,6 +22,7 @@ import ( "github.com/pingcap/dm/pkg/log" "github.com/pingcap/dm/pkg/shardddl/pessimism" + "github.com/pingcap/dm/pkg/terror" ) // Pessimist used to coordinate the shard DDL migration in pessimism mode. @@ -99,9 +100,11 @@ func (p *Pessimist) GetOperation(ctx context.Context, info pessimism.Info, rev i // 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) + done, _, err := pessimism.PutOperationDeleteExistInfo(p.cli, op, info) if err != nil { return err + } else if !done { + return terror.ErrWorkerDDLLockInfoNotFound.Generatef("DDL info for (%s, %s) not found", info.Task, info.Source) } p.mu.Lock() diff --git a/syncer/shardddl/pessimist_test.go b/syncer/shardddl/pessimist_test.go index e2e8e3453b..e0aaab22a4 100644 --- a/syncer/shardddl/pessimist_test.go +++ b/syncer/shardddl/pessimist_test.go @@ -24,6 +24,7 @@ import ( "github.com/pingcap/dm/dm/common" "github.com/pingcap/dm/pkg/log" "github.com/pingcap/dm/pkg/shardddl/pessimism" + "github.com/pingcap/dm/pkg/terror" ) var ( @@ -104,6 +105,7 @@ func (t *testPessimist) TestPessimist(c *C) { // mark the operation as done and delete the info. c.Assert(p.DoneOperationDeleteInfo(op, info), IsNil) + c.Assert(terror.ErrWorkerDDLLockInfoNotFound.Equal(p.DoneOperationDeleteInfo(op, info)), IsTrue) // verify the operation and info. opc := op2 diff --git a/syncer/syncer.go b/syncer/syncer.go index c7986a78b5..636b8b7b79 100644 --- a/syncer/syncer.go +++ b/syncer/syncer.go @@ -1799,7 +1799,7 @@ func (s *Syncer) handleQueryEvent(ev *replication.QueryEvent, ec eventContext) e 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)) - shardOp, err2 := s.pessimist.GetOperation(ec.tctx.Ctx, shardInfo, rev) + shardOp, err2 := s.pessimist.GetOperation(ec.tctx.Ctx, shardInfo, rev+1) shardLockResolving.WithLabelValues(s.cfg.Name).Set(0) if err2 != nil { return err2 diff --git a/tests/dmctl_advance/check_list/break_ddl_lock.sh b/tests/dmctl_advance/check_list/break_ddl_lock.sh deleted file mode 100644 index 43132e4ed0..0000000000 --- a/tests/dmctl_advance/check_list/break_ddl_lock.sh +++ /dev/null @@ -1,31 +0,0 @@ -#!bin/bash - -function break_ddl_lock_wrong_arg() { - run_dm_ctl $WORK_DIR "127.0.0.1:$MASTER_PORT" \ - "break-ddl-lock" \ - "break-ddl-lock <-s source ...> \[--remove-id\] \[--exec\] \[--skip\] \[flags\]" 1 -} - -function break_ddl_lock_without_worker() { - run_dm_ctl $WORK_DIR "127.0.0.1:$MASTER_PORT" \ - "break-ddl-lock test" \ - "must specify at least one source (\`-s\` \/ \`--source\`)" 1 -} - -function break_ddl_lock_shoud_specify_at_least_one() { - run_dm_ctl $WORK_DIR "127.0.0.1:$MASTER_PORT" \ - "break-ddl-lock test -s $SOURCE_ID1" \ - "\`remove-id\`, \`exec\`, \`skip\` must specify at least one" 1 -} - -function break_ddl_lock_exec_skip_conflict() { - run_dm_ctl $WORK_DIR "127.0.0.1:$MASTER_PORT" \ - "break-ddl-lock test -s $SOURCE_ID1 --exec --skip" \ - "\`exec\` and \`skip\` can not specify both at the same time" 1 -} - -function break_ddl_lock_while_master_down() { - run_dm_ctl $WORK_DIR "127.0.0.1:$MASTER_PORT" \ - "break-ddl-lock test -s $SOURCE_ID1 --exec" \ - "can not break DDL lock (in sources \[$SOURCE_ID1\]):" 1 -} diff --git a/tests/dmctl_advance/check_list/unlock_ddl_lock.sh b/tests/dmctl_advance/check_list/unlock_ddl_lock.sh index dc60084a29..52557a50bd 100644 --- a/tests/dmctl_advance/check_list/unlock_ddl_lock.sh +++ b/tests/dmctl_advance/check_list/unlock_ddl_lock.sh @@ -16,5 +16,5 @@ function unlock_ddl_lock_while_master_down() { lock_id="test-\`shard_db\`.\`shard_table\`" run_dm_ctl $WORK_DIR "127.0.0.1:$MASTER_PORT" \ "unlock-ddl-lock $lock_id" \ - "can not unlock DDL lock $lock_id (in sources \[\]):" 1 + "can not unlock DDL lock $lock_id" 1 } diff --git a/tests/dmctl_advance/run.sh b/tests/dmctl_advance/run.sh index ce57a847ad..ef12dac1f8 100755 --- a/tests/dmctl_advance/run.sh +++ b/tests/dmctl_advance/run.sh @@ -12,12 +12,6 @@ SQL_RESULT_FILE="$TEST_DIR/sql_res.$TEST_NAME.txt" # used to coverage wrong usage of dmctl command function usage_and_arg_test() { - break_ddl_lock_wrong_arg - break_ddl_lock_without_worker - break_ddl_lock_shoud_specify_at_least_one - break_ddl_lock_exec_skip_conflict - break_ddl_lock_while_master_down - migrate_relay_wrong_arg migrate_relay_without_worker # todo: no sense to fix diff --git a/tests/dmctl_command/run.sh b/tests/dmctl_command/run.sh index 8ff2505e96..c5938bd0fc 100644 --- a/tests/dmctl_command/run.sh +++ b/tests/dmctl_command/run.sh @@ -13,7 +13,7 @@ function run() { $PWD/bin/dmctl.test DEVEL > $WORK_DIR/help.log help_msg=$(cat $WORK_DIR/help.log) help_msg_cnt=$(echo "${help_msg}" | wc -l |xargs) - if [ "$help_msg_cnt" != 35 ]; then + if [ "$help_msg_cnt" != 34 ]; then echo "dmctl case 1 help failed: $help_msg" echo $help_msg_cnt exit 1 @@ -24,7 +24,7 @@ function run() { $PWD/bin/dmctl.test DEVEL --help > $WORK_DIR/help.log help_msg=$(cat $WORK_DIR/help.log) help_msg_cnt=$(echo "${help_msg}" | wc -l |xargs) - if [ "$help_msg_cnt" != 35 ]; then + if [ "$help_msg_cnt" != 34 ]; then echo "dmctl case 2 help failed: $help_msg" exit 1 fi