diff --git a/_utils/terror_gen/errors_release.txt b/_utils/terror_gen/errors_release.txt index 90deb0b818..b3a9889bb2 100644 --- a/_utils/terror_gen/errors_release.txt +++ b/_utils/terror_gen/errors_release.txt @@ -374,7 +374,9 @@ ErrWorkerGetVersionFromKV,[code=40045:class=dm-worker:scope=internal:level=high] ErrWorkerSaveVersionToKV,[code=40046:class=dm-worker:scope=internal:level=high],"save version %v into levelDB with key %v" ErrWorkerVerAutoDowngrade,[code=40047:class=dm-worker:scope=internal:level=high],"the previous version %s is newer than current %s, automatic downgrade is not supported now, please handle it manually" ErrWorkerStartService,[code=40048:class=dm-worker:scope=internal:level=high],"start server" +ErrWorkerNoStart,[code=40070:class=dm-worker:scope=internal:level=high],"worker has not started" ErrWorkerAlreadyClosed,[code=40049:class=dm-worker:scope=internal:level=high],"worker already closed" +ErrWorkerAlreadyStart,[code=40071:class=dm-worker:scope=internal:level=high],"worker already started" ErrWorkerNotRunningStage,[code=40050:class=dm-worker:scope=internal:level=high],"current stage is not running not valid" ErrWorkerNotPausedStage,[code=40051:class=dm-worker:scope=internal:level=high],"current stage is not paused not valid" ErrWorkerUpdateTaskStage,[code=40052:class=dm-worker:scope=internal:level=high],"can only update task on Paused stage, but current stage is %s" diff --git a/dm/config/checker_config.go b/dm/config/checker_config.go new file mode 100644 index 0000000000..5c2b2c29ba --- /dev/null +++ b/dm/config/checker_config.go @@ -0,0 +1,62 @@ +package config + +import ( + "encoding/json" + "time" +) + +// Backoff related constants +var ( + DefaultCheckInterval = 5 * time.Second + DefaultBackoffRollback = 5 * time.Minute + DefaultBackoffMin = 1 * time.Second + DefaultBackoffMax = 5 * time.Minute + DefaultBackoffJitter = true + DefaultBackoffFactor float64 = 2 +) + +// Duration is used to hold a time.Duration field +type Duration struct { + time.Duration +} + +// MarshalText hacks to satisfy the encoding.TextMarshaler interface +func (d Duration) MarshalText() ([]byte, error) { + return []byte(d.Duration.String()), nil +} + +// UnmarshalText hacks to satisfy the encoding.TextUnmarshaler interface +func (d Duration) UnmarshalText(text []byte) error { + var err error + d.Duration, err = time.ParseDuration(string(text)) + return err +} + +// MarshalJSON hacks to satisfy the json.Marshaler interface +func (d *Duration) MarshalJSON() ([]byte, error) { + return json.Marshal(&struct { + Duration string `json:"Duration"` + }{ + d.Duration.String(), + }) +} + +// CheckerConfig is configuration used for TaskStatusChecker +type CheckerConfig struct { + CheckEnable bool `toml:"check-enable" json:"check-enable"` + BackoffRollback Duration `toml:"backoff-rollback" json:"backoff-rollback"` + BackoffMax Duration `toml:"backoff-max" json:"backoff-max"` + // unexpose config + CheckInterval Duration `json:"-"` + BackoffMin Duration `json:"-"` + BackoffJitter bool `json:"-"` + BackoffFactor float64 `json:"-"` +} + +// Adjust sets default value for field: CheckInterval/BackoffMin/BackoffJitter/BackoffFactor +func (cc *CheckerConfig) Adjust() { + cc.CheckInterval = Duration{Duration: DefaultCheckInterval} + cc.BackoffMin = Duration{Duration: DefaultBackoffMin} + cc.BackoffJitter = DefaultBackoffJitter + cc.BackoffFactor = DefaultBackoffFactor +} diff --git a/dm/config/mysql_config.go b/dm/config/mysql_config.go new file mode 100644 index 0000000000..8adfc51734 --- /dev/null +++ b/dm/config/mysql_config.go @@ -0,0 +1,280 @@ +package config + +import ( + "bytes" + "context" + "database/sql" + "encoding/json" + "github.com/BurntSushi/toml" + "github.com/pingcap/dm/pkg/binlog" + "github.com/pingcap/dm/pkg/gtid" + "github.com/pingcap/dm/pkg/log" + "github.com/pingcap/dm/pkg/terror" + "github.com/pingcap/dm/pkg/tracing" + "github.com/pingcap/dm/pkg/utils" + "github.com/siddontang/go-mysql/mysql" + "math" + "math/rand" + "strings" + "time" +) + +const ( + // dbReadTimeout is readTimeout for DB connection in adjust + dbReadTimeout = "30s" + // dbGetTimeout is timeout for getting some information from DB + dbGetTimeout = 30 * time.Second + + // the default base(min) server id generated by random + defaultBaseServerID = math.MaxUint32 / 10 +) + +// PurgeConfig is the configuration for Purger +type PurgeConfig struct { + Interval int64 `toml:"interval" json:"interval"` // check whether need to purge at this @Interval (seconds) + Expires int64 `toml:"expires" json:"expires"` // if file's modified time is older than @Expires (hours), then it can be purged + RemainSpace int64 `toml:"remain-space" json:"remain-space"` // if remain space in @RelayBaseDir less than @RemainSpace (GB), then it can be purged +} + +// MysqlConfig is the configuration for Worker +type MysqlConfig struct { + EnableGTID bool `toml:"enable-gtid" json:"enable-gtid"` + AutoFixGTID bool `toml:"auto-fix-gtid" json:"auto-fix-gtid"` + RelayDir string `toml:"relay-dir" json:"relay-dir"` + MetaDir string `toml:"meta-dir" json:"meta-dir"` + Flavor string `toml:"flavor" json:"flavor"` + Charset string `toml:"charset" json:"charset"` + + // relay synchronous starting point (if specified) + RelayBinLogName string `toml:"relay-binlog-name" json:"relay-binlog-name"` + RelayBinlogGTID string `toml:"relay-binlog-gtid" json:"relay-binlog-gtid"` + + SourceID string `toml:"source-id" json:"source-id"` + From DBConfig `toml:"from" json:"from"` + + // config items for purger + Purge PurgeConfig `toml:"purge" json:"purge"` + + // config items for task status checker + Checker CheckerConfig `toml:"checker" json:"checker"` + + // config items for tracer + Tracer tracing.Config `toml:"tracer" json:"tracer"` + + // id of the worker on which this task run + ServerID uint32 `toml:"server-id" json:"server-id"` +} + +// NewWorkerConfig creates a new base config for worker. +func NewWorkerConfig() *MysqlConfig { + c := &MysqlConfig{ + RelayDir: "relay-dir", + Purge: PurgeConfig{ + Interval: 60 * 60, + Expires: 0, + RemainSpace: 15, + }, + Checker: CheckerConfig{ + CheckEnable: true, + BackoffRollback: Duration{DefaultBackoffRollback}, + BackoffMax: Duration{DefaultBackoffMax}, + }, + Tracer: tracing.Config{ + Enable: false, + TracerAddr: "", + BatchSize: 20, + Checksum: false, + }, + } + c.adjust() + return c +} + +// Clone clones a config +func (c *MysqlConfig) Clone() *MysqlConfig { + clone := &MysqlConfig{} + *clone = *c + return clone +} + +// Toml returns TOML format representation of config +func (c *MysqlConfig) Toml() (string, error) { + var b bytes.Buffer + + err := toml.NewEncoder(&b).Encode(c) + if err != nil { + log.L().Error("fail to marshal config to toml", log.ShortError(err)) + } + + return b.String(), nil +} + +// Parse parses flag definitions from the argument list. +func (c *MysqlConfig) Parse(content string) error { + // Parse first to get config file. + metaData, err := toml.Decode(content, c) + return c.check(&metaData, err) +} + +func (c *MysqlConfig) String() string { + cfg, err := json.Marshal(c) + if err != nil { + log.L().Error("fail to marshal config to json", log.ShortError(err)) + } + return string(cfg) +} + +func (c *MysqlConfig) adjust() { + c.From.Adjust() + c.Checker.Adjust() +} + +// Verify verifies the config +func (c *MysqlConfig) Verify() error { + if len(c.SourceID) == 0 { + return terror.ErrWorkerNeedSourceID.Generate() + } + if len(c.SourceID) > MaxSourceIDLength { + return terror.ErrWorkerTooLongSourceID.Generate(c.SourceID, MaxSourceIDLength) + } + + var err error + if len(c.RelayBinLogName) > 0 { + if !binlog.VerifyFilename(c.RelayBinLogName) { + return terror.ErrWorkerRelayBinlogName.Generate(c.RelayBinLogName) + } + } + if len(c.RelayBinlogGTID) > 0 { + _, err = gtid.ParserGTID(c.Flavor, c.RelayBinlogGTID) + if err != nil { + return terror.WithClass(terror.Annotatef(err, "relay-binlog-gtid %s", c.RelayBinlogGTID), terror.ClassDMWorker) + } + } + + _, err = c.DecryptPassword() + if err != nil { + return err + } + + return nil +} + +// DecryptPassword returns a decrypted config replica in config +func (c *MysqlConfig) DecryptPassword() (*MysqlConfig, error) { + clone := c.Clone() + var ( + pswdFrom string + err error + ) + if len(clone.From.Password) > 0 { + pswdFrom, err = utils.Decrypt(clone.From.Password) + if err != nil { + return nil, terror.WithClass(err, terror.ClassDMWorker) + } + } + clone.From.Password = pswdFrom + return clone, nil +} + +// GenerateDBConfig creates DBConfig for DB +func (c *MysqlConfig) GenerateDBConfig() (*DBConfig, error) { + // decrypt password + clone, err := c.DecryptPassword() + if err != nil { + return nil, err + } + from := &clone.From + from.RawDBCfg = DefaultRawDBConfig().SetReadTimeout(dbReadTimeout) + return from, nil +} + +// Adjust flavor and serverid of MysqlConfig +func (c *MysqlConfig) Adjust(db *sql.DB) (err error) { + c.From.Adjust() + c.Checker.Adjust() + + if c.Flavor == "" || c.ServerID == 0 { + ctx, cancel := context.WithTimeout(context.Background(), dbGetTimeout) + defer cancel() + + err = c.AdjustFlavor(ctx, db) + if err != nil { + return err + } + + err = c.AdjustServerID(ctx, db) + if err != nil { + return err + } + } + + return nil +} + +// AdjustFlavor adjust Flavor from DB +func (c *MysqlConfig) AdjustFlavor(ctx context.Context, db *sql.DB) (err error) { + if c.Flavor != "" { + switch c.Flavor { + case mysql.MariaDBFlavor, mysql.MySQLFlavor: + return nil + default: + return terror.ErrNotSupportedFlavor.Generate(c.Flavor) + } + } + + c.Flavor, err = utils.GetFlavor(ctx, db) + if ctx.Err() != nil { + err = terror.Annotatef(err, "time cost to get flavor info exceeds %s", dbGetTimeout) + } + return terror.WithScope(err, terror.ScopeUpstream) +} + +// AdjustServerID adjust server id from DB +func (c *MysqlConfig) AdjustServerID(ctx context.Context, db *sql.DB) error { + if c.ServerID != 0 { + return nil + } + + serverIDs, err := utils.GetAllServerID(ctx, db) + if ctx.Err() != nil { + err = terror.Annotatef(err, "time cost to get server-id info exceeds %s", dbGetTimeout) + } + if err != nil { + return terror.WithScope(err, terror.ScopeUpstream) + } + + for i := 0; i < 5; i++ { + randomValue := uint32(rand.Intn(100000)) + randomServerID := defaultBaseServerID + randomValue + if _, ok := serverIDs[randomServerID]; ok { + continue + } + + c.ServerID = randomServerID + return nil + } + + return terror.ErrInvalidServerID.Generatef("can't find a random available server ID") +} + +// LoadFromFile loads config from file. +func (c *MysqlConfig) LoadFromFile(path string) error { + metaData, err := toml.DecodeFile(path, c) + return c.check(&metaData, err) +} + +func (c *MysqlConfig) check(metaData *toml.MetaData, err error) error { + if err != nil { + return terror.ErrWorkerDecodeConfigFromFile.Delegate(err) + } + undecoded := metaData.Undecoded() + if len(undecoded) > 0 && err == nil { + var undecodedItems []string + for _, item := range undecoded { + undecodedItems = append(undecodedItems, item.String()) + } + return terror.ErrWorkerUndecodedItemFromFile.Generate(strings.Join(undecodedItems, ",")) + } + c.adjust() + return nil +} diff --git a/dm/ctl/common/operate_relay.go b/dm/ctl/common/operate_relay.go index 91dd877b1c..1a21645ae0 100644 --- a/dm/ctl/common/operate_relay.go +++ b/dm/ctl/common/operate_relay.go @@ -26,6 +26,6 @@ func OperateRelay(op pb.RelayOp, workers []string) (*pb.OperateWorkerRelayRespon cli := MasterClient() return cli.OperateWorkerRelayTask(ctx, &pb.OperateWorkerRelayRequest{ Op: op, - Workers: workers, + Sources: workers, }) } diff --git a/dm/ctl/common/operate_task.go b/dm/ctl/common/operate_task.go index 361abe2c58..5f1d3355a0 100644 --- a/dm/ctl/common/operate_task.go +++ b/dm/ctl/common/operate_task.go @@ -27,6 +27,6 @@ func OperateTask(op pb.TaskOp, name string, workers []string) (*pb.OperateTaskRe return cli.OperateTask(ctx, &pb.OperateTaskRequest{ Op: op, Name: name, - Workers: workers, + Sources: workers, }) } diff --git a/dm/ctl/ctl.go b/dm/ctl/ctl.go index 483824931f..d0463ff843 100644 --- a/dm/ctl/ctl.go +++ b/dm/ctl/ctl.go @@ -62,7 +62,6 @@ func NewRootCmd() *cobra.Command { master.NewUpdateTaskCmd(), master.NewQueryStatusCmd(), master.NewQueryErrorCmd(), - master.NewRefreshWorkerTasks(), master.NewSQLReplaceCmd(), master.NewSQLSkipCmd(), master.NewSQLInjectCmd(), @@ -76,6 +75,7 @@ func NewRootCmd() *cobra.Command { master.NewUpdateRelayCmd(), master.NewPurgeRelayCmd(), master.NewMigrateRelayCmd(), + master.NewOperateMysqlWorkerCmd(), ) return cmd } diff --git a/dm/ctl/master/break_ddl_lock.go b/dm/ctl/master/break_ddl_lock.go index bd27999de5..8e1874e6ad 100644 --- a/dm/ctl/master/break_ddl_lock.go +++ b/dm/ctl/master/break_ddl_lock.go @@ -89,7 +89,7 @@ func breakDDLLockFunc(cmd *cobra.Command, _ []string) { defer cancel() cli := common.MasterClient() resp, err := cli.BreakWorkerDDLLock(ctx, &pb.BreakWorkerDDLLockRequest{ - Workers: workers, + Sources: workers, Task: taskName, RemoveLockID: removeLockID, ExecDDL: exec, diff --git a/dm/ctl/master/operate_mysql_task.go b/dm/ctl/master/operate_mysql_task.go new file mode 100644 index 0000000000..dfe89889ba --- /dev/null +++ b/dm/ctl/master/operate_mysql_task.go @@ -0,0 +1,77 @@ +// 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" + "github.com/pingcap/dm/checker" + "github.com/pingcap/dm/dm/ctl/common" + "github.com/pingcap/dm/dm/pb" + "github.com/pingcap/errors" + "github.com/spf13/cobra" + "os" +) + +// NewOperateMysqlWorkerCmd creates a OperateMysqlTask command +func NewOperateMysqlWorkerCmd() *cobra.Command { + cmd := &cobra.Command{ + Use: "operate-worker ", + Short: "create/update/stop mysql task", + Run: operateMysqlWorkerFunc, + } + return cmd +} + +func convertCmdType(t string) pb.WorkerOp { + if t == "create" { + return pb.WorkerOp_StartWorker + } else if t == "update" { + return pb.WorkerOp_UpdateConfig + } else { + return pb.WorkerOp_StopWorker + } +} + +// operateMysqlFunc does migrate relay request +func operateMysqlWorkerFunc(cmd *cobra.Command, _ []string) { + if len(cmd.Flags().Args()) != 2 { + cmd.SetOut(os.Stdout) + cmd.Usage() + return + } + + cmdType := cmd.Flags().Arg(0) + configFile := cmd.Flags().Arg(1) + content, err := common.GetFileContent(configFile) + if err != nil { + common.PrintLines("get file content error:\n%v", errors.ErrorStack(err)) + return + } + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + + cli := common.MasterClient() + resp, err := cli.OperateMysqlWorker(ctx, &pb.MysqlTaskRequest{ + Config: string(content), + Op: convertCmdType(cmdType), + }) + if err != nil { + common.PrintLines("can not update task:\n%v", errors.ErrorStack(err)) + return + } + if !common.PrettyPrintResponseWithCheckTask(resp, checker.ErrorMsgHeader) { + common.PrettyPrintResponse(resp) + } + +} diff --git a/dm/ctl/master/purge_relay.go b/dm/ctl/master/purge_relay.go index 5aca31997c..8680a727db 100644 --- a/dm/ctl/master/purge_relay.go +++ b/dm/ctl/master/purge_relay.go @@ -138,7 +138,7 @@ func purgeRelayFunc(cmd *cobra.Command, _ []string) { cli := common.MasterClient() resp, err := cli.PurgeWorkerRelay(ctx, &pb.PurgeWorkerRelayRequest{ - Workers: workers, + Sources: workers, //Inactive: inactive, //Time: time2.Unix(), Filename: filename, diff --git a/dm/ctl/master/query_error.go b/dm/ctl/master/query_error.go index 83e2ad32ca..23a66b6cb4 100644 --- a/dm/ctl/master/query_error.go +++ b/dm/ctl/master/query_error.go @@ -54,7 +54,7 @@ func queryErrorFunc(cmd *cobra.Command, _ []string) { cli := common.MasterClient() resp, err := cli.QueryError(ctx, &pb.QueryErrorListRequest{ Name: taskName, - Workers: workers, + Sources: workers, }) if err != nil { common.PrintLines("dmctl query error failed") diff --git a/dm/ctl/master/query_status.go b/dm/ctl/master/query_status.go index 64e4f7e770..bcfb678296 100644 --- a/dm/ctl/master/query_status.go +++ b/dm/ctl/master/query_status.go @@ -69,7 +69,7 @@ func queryStatusFunc(cmd *cobra.Command, _ []string) { defer cancel() resp, err := cli.QueryStatus(ctx, &pb.QueryStatusListRequest{ Name: taskName, - Workers: workers, + Sources: workers, }) if err != nil { common.PrintLines("can not query %s task's status(in workers %v):\n%s", taskName, workers, errors.ErrorStack(err)) diff --git a/dm/ctl/master/refresh_worker_tasks.go b/dm/ctl/master/refresh_worker_tasks.go deleted file mode 100644 index 5dfb234363..0000000000 --- a/dm/ctl/master/refresh_worker_tasks.go +++ /dev/null @@ -1,55 +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" - "os" - - "github.com/pingcap/dm/dm/ctl/common" - "github.com/pingcap/dm/dm/pb" - - "github.com/pingcap/errors" - "github.com/spf13/cobra" -) - -// NewRefreshWorkerTasks creates a RefreshWorkerTasks command -func NewRefreshWorkerTasks() *cobra.Command { - cmd := &cobra.Command{ - Use: "refresh-worker-tasks", - Short: "refresh worker -> tasks mapper", - Run: refreshWorkerTasksFunc, - } - return cmd -} - -// refreshWorkerTasksFunc does refresh workerTasks request -func refreshWorkerTasksFunc(cmd *cobra.Command, _ []string) { - if len(cmd.Flags().Args()) > 0 { - cmd.SetOut(os.Stdout) - cmd.Usage() - return - } - - ctx, cancel := context.WithCancel(context.Background()) - defer cancel() - cli := common.MasterClient() - resp, err := cli.RefreshWorkerTasks(ctx, &pb.RefreshWorkerTasksRequest{}) - if err != nil { - common.PrintLines("can not refresh workerTasks:\n%v", errors.ErrorStack(err)) - return - } - - common.PrettyPrintResponse(resp) -} diff --git a/dm/ctl/master/show_ddl_locks.go b/dm/ctl/master/show_ddl_locks.go index a01eddc76f..f4a30ddb75 100644 --- a/dm/ctl/master/show_ddl_locks.go +++ b/dm/ctl/master/show_ddl_locks.go @@ -54,7 +54,7 @@ func showDDLLocksFunc(cmd *cobra.Command, _ []string) { cli := common.MasterClient() resp, err := cli.ShowDDLLocks(ctx, &pb.ShowDDLLocksRequest{ Task: taskName, - Workers: workers, + Sources: workers, }) if err != nil { common.PrintLines("can not show DDL locks for task %s and workers %v:\n%s", taskName, workers, errors.ErrorStack(err)) diff --git a/dm/ctl/master/start_task.go b/dm/ctl/master/start_task.go index b072eaff81..0ba36ed155 100644 --- a/dm/ctl/master/start_task.go +++ b/dm/ctl/master/start_task.go @@ -61,7 +61,7 @@ func startTaskFunc(cmd *cobra.Command, _ []string) { cli := common.MasterClient() resp, err := cli.StartTask(ctx, &pb.StartTaskRequest{ Task: string(content), - Workers: workers, + Sources: workers, }) if err != nil { common.PrintLines("can not start task:\n%v", errors.ErrorStack(err)) diff --git a/dm/ctl/master/switch_relay_master.go b/dm/ctl/master/switch_relay_master.go index bbdd6b1305..d2b471f3b5 100644 --- a/dm/ctl/master/switch_relay_master.go +++ b/dm/ctl/master/switch_relay_master.go @@ -57,7 +57,7 @@ func switchRelayMasterFunc(cmd *cobra.Command, _ []string) { defer cancel() cli := common.MasterClient() resp, err := cli.SwitchWorkerRelayMaster(ctx, &pb.SwitchWorkerRelayMasterRequest{ - Workers: workers, + Sources: workers, }) if err != nil { common.PrintLines("can not switch relay's master server (in workers %v):\n%s", workers, errors.ErrorStack(err)) diff --git a/dm/ctl/master/unlock_ddl_lock.go b/dm/ctl/master/unlock_ddl_lock.go index 948f8c2bc4..3b36aa3a69 100644 --- a/dm/ctl/master/unlock_ddl_lock.go +++ b/dm/ctl/master/unlock_ddl_lock.go @@ -70,7 +70,7 @@ func unlockDDLLockFunc(cmd *cobra.Command, _ []string) { resp, err := cli.UnlockDDLLock(ctx, &pb.UnlockDDLLockRequest{ ID: lockID, ReplaceOwner: owner, - Workers: workers, + Sources: workers, ForceRemove: forceRemove, }) if err != nil { diff --git a/dm/ctl/master/update_task.go b/dm/ctl/master/update_task.go index e0fd88db28..a49b383a8e 100644 --- a/dm/ctl/master/update_task.go +++ b/dm/ctl/master/update_task.go @@ -61,7 +61,7 @@ func updateTaskFunc(cmd *cobra.Command, _ []string) { cli := common.MasterClient() resp, err := cli.UpdateTask(ctx, &pb.UpdateTaskRequest{ Task: string(content), - Workers: workers, + Sources: workers, }) if err != nil { common.PrintLines("can not update task:\n%v", errors.ErrorStack(err)) diff --git a/dm/master/config.go b/dm/master/config.go index 9f72189ec0..2b888c9732 100644 --- a/dm/master/config.go +++ b/dm/master/config.go @@ -50,6 +50,7 @@ var SampleConfigFile string // NewConfig creates a config for dm-master func NewConfig() *Config { cfg := &Config{} + cfg.Debug = false cfg.FlagSet = flag.NewFlagSet("dm-master", flag.ContinueOnError) fs := cfg.FlagSet @@ -117,7 +118,8 @@ type Config struct { AdvertisePeerUrls string `toml:"advertise-peer-urls" json:"advertise-peer-urls"` InitialCluster string `toml:"initial-cluster" json:"initial-cluster"` InitialClusterState string `toml:"initial-cluster-state" json:"initial-cluster-state"` - Join string `toml:"join" json:"join"` // cluster's client address (endpoints), not peer address + Join string `toml:"join" json:"join"` // cluster's client address (endpoints), not peer address + Debug bool `toml:"debug" json:"debug"` // only use for test printVersion bool printSampleConfig bool diff --git a/dm/master/coordinator/coordinator.go b/dm/master/coordinator/coordinator.go index 063a89babb..f5dbf76c4d 100644 --- a/dm/master/coordinator/coordinator.go +++ b/dm/master/coordinator/coordinator.go @@ -15,6 +15,7 @@ package coordinator import ( "context" + "github.com/pingcap/dm/dm/config" "strings" "sync" @@ -36,15 +37,29 @@ type Coordinator struct { workers map[string]*Worker // upstream(source-id) -> worker upstreams map[string]*Worker + + // upstream(address) -> config + configs map[string]config.MysqlConfig + + // pending create taks (sourceid) --> address + pendingtask map[string]string } // NewCoordinator returns a coordinate. func NewCoordinator() *Coordinator { return &Coordinator{ - workers: make(map[string]*Worker), + workers: make(map[string]*Worker), + configs: make(map[string]config.MysqlConfig), + pendingtask: make(map[string]string), + upstreams: make(map[string]*Worker), } } +// Init would recover infomation from etcd +func (c *Coordinator) Init(etcdClient *clientv3.Client) { + // TODO: recover upstreams and configs and workers +} + // AddWorker add the dm-worker to the coordinate. func (c *Coordinator) AddWorker(name string, address string) { c.mu.Lock() @@ -52,6 +67,47 @@ func (c *Coordinator) AddWorker(name string, address string) { c.workers[address] = NewWorker(name, address) } +// HandleStartedWorker change worker status when mysql task started +func (c *Coordinator) HandleStartedWorker(w *Worker, cfg *config.MysqlConfig, succ bool) { + c.mu.Lock() + defer c.mu.Unlock() + if succ { + c.upstreams[cfg.SourceID] = w + c.configs[w.Address()] = *cfg + } else { + w.SetStatus(WorkerFree) + } + delete(c.pendingtask, cfg.SourceID) +} + +// HandleStoppedWorker change worker status when mysql task stopped +func (c *Coordinator) HandleStoppedWorker(w *Worker, cfg *config.MysqlConfig) { + c.mu.Lock() + defer c.mu.Unlock() + delete(c.upstreams, cfg.SourceID) + delete(c.configs, w.Address()) + w.SetStatus(WorkerFree) +} + +// GetFreeWorkerForSource get the free worker to create mysql delay task, and add it to pending task +// to avoid create a task in two worker +func (c *Coordinator) GetFreeWorkerForSource(source string) (*Worker, string) { + c.mu.RLock() + defer c.mu.RUnlock() + if addr, ok := c.pendingtask[source]; ok { + return nil, addr + } + for _, w := range c.workers { + if w.status.Load() == WorkerFree { + // we bound worker to avoid another task trying to get it + w.status.Store(WorkerBound) + c.pendingtask[source] = w.Address() + return w, "" + } + } + return nil, "" +} + // GetWorkerByAddress gets the worker through address. func (c *Coordinator) GetWorkerByAddress(address string) *Worker { c.mu.RLock() @@ -83,8 +139,33 @@ func (c *Coordinator) GetAllWorkers() map[string]*Worker { return c.workers } +// GetRunningMysqlSource gets all souce which is running. +func (c *Coordinator) GetRunningMysqlSource() map[string]string { + c.mu.RLock() + defer c.mu.RUnlock() + res := make(map[string]string) + for source, w := range c.upstreams { + if w.State() == WorkerBound { + res[source] = w.address + } + } + return res +} + // GetWorkerBySourceID gets the worker through source id. func (c *Coordinator) GetWorkerBySourceID(source string) *Worker { + c.mu.RLock() + defer c.mu.RUnlock() + return c.upstreams[source] +} + +// GetConfigBySourceID gets db config through source id. +func (c *Coordinator) GetConfigBySourceID(source string) *config.MysqlConfig { + c.mu.RLock() + defer c.mu.RUnlock() + if cfg, ok := c.configs[source]; ok { + return &cfg + } return nil } @@ -124,7 +205,9 @@ func (c *Coordinator) ObserveWorkers(ctx context.Context, client *clientv3.Clien c.mu.Lock() if w, ok := c.workers[addr]; ok && name == w.Name() { log.L().Info("worker became online, state: free", zap.String("name", w.Name()), zap.String("address", w.Address())) - w.setStatus(WorkerFree) + w.SetStatus(WorkerFree) + } else { + // TODO: how to deal with unregister worker } c.mu.Unlock() @@ -136,7 +219,10 @@ func (c *Coordinator) ObserveWorkers(ctx context.Context, client *clientv3.Clien c.mu.Lock() if w, ok := c.workers[addr]; ok && name == w.Name() { log.L().Info("worker became offline, state: closed", zap.String("name", w.Name()), zap.String("address", w.Address())) - w.setStatus(WorkerClosed) + w.SetStatus(WorkerClosed) + cfg := c.configs[addr] + delete(c.upstreams, cfg.SourceID) + delete(c.configs, addr) } c.mu.Unlock() } diff --git a/dm/master/coordinator/worker.go b/dm/master/coordinator/worker.go index 30dcb68236..e0be8f95e8 100644 --- a/dm/master/coordinator/worker.go +++ b/dm/master/coordinator/worker.go @@ -14,8 +14,11 @@ package coordinator import ( + "context" "fmt" + "github.com/pingcap/dm/dm/pb" "sync/atomic" + "time" "github.com/pingcap/dm/dm/master/workerrpc" ) @@ -81,6 +84,33 @@ func (w *Worker) State() WorkerState { return w.status.Load().(WorkerState) } -func (w *Worker) setStatus(s WorkerState) { +// SetStatus change the status of worker +func (w *Worker) SetStatus(s WorkerState) { w.status.Store(s) } + +// OperateMysqlTask in a idle worker +func (w *Worker) OperateMysqlTask(ctx context.Context, req *pb.MysqlTaskRequest, d time.Duration) (*pb.MysqlTaskResponse, error) { + ownerReq := &workerrpc.Request{ + Type: workerrpc.CmdOperateMysqlTask, + MysqlTask: req, + } + cli, err := w.GetClient() + if err != nil { + return nil, err + } + resp, err := cli.SendRequest(ctx, ownerReq, d) + if err != nil { + return nil, err + } + return resp.MysqlTask, err +} + +// SendRequest by client +func (w *Worker) SendRequest(ctx context.Context, req *workerrpc.Request, d time.Duration) (*workerrpc.Response, error) { + cli, err := w.GetClient() + if err != nil { + return nil, err + } + return cli.SendRequest(ctx, req, d) +} diff --git a/dm/master/server.go b/dm/master/server.go index ad5ea4c556..3f6241ea93 100644 --- a/dm/master/server.go +++ b/dm/master/server.go @@ -16,6 +16,7 @@ package master import ( "context" "fmt" + "github.com/pingcap/dm/pkg/conn" "io" "net/http" "path" @@ -80,7 +81,7 @@ type Server struct { workerClients map[string]workerrpc.Client // task-name -> worker-list - taskWorkers map[string][]string + taskSources map[string][]string // DDL lock keeper lockKeeper *LockKeeper @@ -103,7 +104,7 @@ func NewServer(cfg *Config) *Server { cfg: cfg, coordinator: coordinator.NewCoordinator(), workerClients: make(map[string]workerrpc.Client), - taskWorkers: make(map[string][]string), + taskSources: make(map[string][]string), lockKeeper: NewLockKeeper(), sqlOperatorHolder: operator.NewHolder(), idGen: tracing.NewIDGen(), @@ -135,15 +136,6 @@ func (s *Server) Start(ctx context.Context) (err error) { return } - // create clients to DM-workers - // for _, workerAddr := range s.cfg.DeployMap { - // s.workerClients[workerAddr], err = workerrpc.NewGRPCClient(workerAddr) - // if err != nil { - // return - // } - // } - // - // get an HTTP to gRPC API handler. apiHandler, err := getHTTPAPIHandler(ctx, s.cfg.MasterAddr) if err != nil { return @@ -200,6 +192,7 @@ func (s *Server) Start(ctx context.Context) (err error) { }() s.bgFunWg.Add(1) + s.coordinator.Init(s.etcdClient) go func() { defer s.bgFunWg.Done() s.coordinator.ObserveWorkers(ctx, s.etcdClient) @@ -295,6 +288,7 @@ func (s *Server) RegisterWorker(ctx context.Context, req *pb.RegisterWorkerReque return respWorker, nil } } + fmt.Println("=======Register a worker") s.coordinator.AddWorker(req.Name, req.Address) log.L().Info("register worker successfully", zap.String("name", req.Name), zap.String("address", req.Address)) respWorker := &pb.RegisterWorkerResponse{ @@ -316,59 +310,42 @@ func (s *Server) StartTask(ctx context.Context, req *pb.StartTaskRequest) (*pb.S } log.L().Info("", zap.String("task name", cfg.Name), zap.Stringer("task", cfg), zap.String("request", "StartTask")) - workerRespCh := make(chan *pb.CommonWorkerResponse, len(stCfgs)+len(req.Workers)) - if len(req.Workers) > 0 { - // specify only start task on partial dm-workers - workerCfg := make(map[string]*config.SubTaskConfig) - for _, stCfg := range stCfgs { - worker, ok := s.cfg.DeployMap[stCfg.SourceID] - if ok { - workerCfg[worker] = stCfg - } - } - stCfgs = make([]*config.SubTaskConfig, 0, len(req.Workers)) - for _, worker := range req.Workers { - if stCfg, ok := workerCfg[worker]; ok { - stCfgs = append(stCfgs, stCfg) - } else { - workerRespCh <- errorCommonWorkerResponse("worker not found in task's config or deployment config", worker) - } - } - } - - validWorkerCh := make(chan string, len(stCfgs)) + workerRespCh := make(chan *pb.CommonWorkerResponse, len(stCfgs)) var wg sync.WaitGroup + subSourceIDs := make([]string, 0, len(stCfgs)) for _, stCfg := range stCfgs { + subSourceIDs = append(subSourceIDs, stCfg.SourceID) wg.Add(1) go s.ap.Emit(ctx, 0, func(args ...interface{}) { defer wg.Done() - cli, worker, stCfgToml, _, err := s.taskConfigArgsExtractor(args...) + cfg, _ := args[0].(*config.SubTaskConfig) + worker, stCfgToml, _, err := s.taskConfigArgsExtractor(cfg) if err != nil { - workerRespCh <- errorCommonWorkerResponse(err.Error(), worker) + workerRespCh <- errorCommonWorkerResponse(err.Error(), cfg.SourceID) return } - validWorkerCh <- worker request := &workerrpc.Request{ Type: workerrpc.CmdStartSubTask, StartSubTask: &pb.StartSubTaskRequest{Task: stCfgToml}, } - resp, err := cli.SendRequest(ctx, request, s.cfg.RPCTimeout) + resp, err := worker.SendRequest(ctx, request, s.cfg.RPCTimeout) if err != nil { resp = &workerrpc.Response{ Type: workerrpc.CmdStartSubTask, - StartSubTask: errorCommonWorkerResponse(err.Error(), worker), + StartSubTask: errorCommonWorkerResponse(err.Error(), cfg.SourceID), } } - resp.StartSubTask.Worker = worker + resp.StartSubTask.Worker = cfg.SourceID workerRespCh <- resp.StartSubTask }, func(args ...interface{}) { defer wg.Done() - _, worker, _, _, err := s.taskConfigArgsExtractor(args...) + cfg, _ := args[0].(*config.SubTaskConfig) + worker, _, _, err := s.taskConfigArgsExtractor(cfg) if err != nil { - workerRespCh <- errorCommonWorkerResponse(err.Error(), worker) + workerRespCh <- errorCommonWorkerResponse(err.Error(), cfg.SourceID) return } - workerRespCh <- errorCommonWorkerResponse(terror.ErrMasterNoEmitToken.Generate(worker).Error(), worker) + workerRespCh <- errorCommonWorkerResponse(terror.ErrMasterNoEmitToken.Generate(worker.Address()).Error(), cfg.SourceID) }, stCfg) } wg.Wait() @@ -389,13 +366,7 @@ func (s *Server) StartTask(ctx context.Context, req *pb.StartTaskRequest) (*pb.S } // record task -> workers map - validWorkers := make([]string, 0, len(validWorkerCh)) - for len(validWorkerCh) > 0 { - worker := <-validWorkerCh - validWorkers = append(validWorkers, worker) - } - replace := len(req.Workers) == 0 // a fresh start - s.addTaskWorkers(cfg.Name, validWorkers, replace) + s.taskSources[cfg.Name] = subSourceIDs return &pb.StartTaskResponse{ Result: true, @@ -412,15 +383,8 @@ func (s *Server) OperateTask(ctx context.Context, req *pb.OperateTaskRequest) (* Result: false, } - workers := s.getTaskWorkers(req.Name) - if len(workers) == 0 { - resp.Msg = fmt.Sprintf("task %s has no workers or not exist, can try `refresh-worker-tasks` cmd first", req.Name) - return resp, nil - } - if len(req.Workers) > 0 { - workers = req.Workers // specify only do operation on partial dm-workers - } - workerRespCh := make(chan *pb.OperateSubTaskResponse, len(workers)) + sources := s.getTaskResources(req.Name) + workerRespCh := make(chan *pb.OperateSubTaskResponse, len(sources)) handleErr := func(err error, worker string) { log.L().Error("response error", zap.Error(err)) @@ -442,16 +406,18 @@ func (s *Server) OperateTask(ctx context.Context, req *pb.OperateTaskRequest) (* } var wg sync.WaitGroup - for _, worker := range workers { + for _, source := range sources { wg.Add(1) go s.ap.Emit(ctx, 0, func(args ...interface{}) { defer wg.Done() - cli, worker1, err := s.workerArgsExtractor(args...) - if err != nil { - handleErr(err, worker1) + sourceID, _ := args[0].(string) + worker1 := s.coordinator.GetWorkerBySourceID(sourceID) + if worker1 == nil { + err := terror.ErrMasterWorkerArgsExtractor.Generatef("%s relevant worker-client not found", sourceID) + handleErr(err, sourceID) return } - resp, err := cli.SendRequest(ctx, subReq, s.cfg.RPCTimeout) + resp, err := worker1.SendRequest(ctx, subReq, s.cfg.RPCTimeout) if err != nil { resp = &workerrpc.Response{ Type: workerrpc.CmdOperateSubTask, @@ -462,38 +428,30 @@ func (s *Server) OperateTask(ctx context.Context, req *pb.OperateTaskRequest) (* }, } } - resp.OperateSubTask.Worker = worker1 + resp.OperateSubTask.Worker = sourceID workerRespCh <- resp.OperateSubTask }, func(args ...interface{}) { defer wg.Done() - _, worker1, err := s.workerArgsExtractor(args...) - if err != nil { - handleErr(err, worker1) - return - } - handleErr(terror.ErrMasterNoEmitToken.Generate(worker1), worker1) - }, worker) + sourceID, _ := args[0].(string) + handleErr(terror.ErrMasterNoEmitToken.Generate(sourceID), sourceID) + }, source) } wg.Wait() - validWorkers := make([]string, 0, len(workers)) - workerRespMap := make(map[string]*pb.OperateSubTaskResponse, len(workers)) + workerRespMap := make(map[string]*pb.OperateSubTaskResponse, len(sources)) for len(workerRespCh) > 0 { workerResp := <-workerRespCh workerRespMap[workerResp.Worker] = workerResp - if len(workerResp.Msg) == 0 { // no error occurred - validWorkers = append(validWorkers, workerResp.Worker) - } } - workerResps := make([]*pb.OperateSubTaskResponse, 0, len(workers)) - for _, worker := range workers { + workerResps := make([]*pb.OperateSubTaskResponse, 0, len(sources)) + for _, worker := range sources { workerResps = append(workerResps, workerRespMap[worker]) } if req.Op == pb.TaskOp_Stop { // remove (partial / all) workers for a task - s.removeTaskWorkers(req.Name, validWorkers) + s.removeTaskWorkers(req.Name, sources) } resp.Result = true @@ -515,24 +473,21 @@ func (s *Server) UpdateTask(ctx context.Context, req *pb.UpdateTaskRequest) (*pb } log.L().Info("update task", zap.String("task name", cfg.Name), zap.Stringer("task", cfg)) - workerRespCh := make(chan *pb.CommonWorkerResponse, len(stCfgs)+len(req.Workers)) - if len(req.Workers) > 0 { + workerRespCh := make(chan *pb.CommonWorkerResponse, len(stCfgs)+len(req.Sources)) + if len(req.Sources) > 0 { // specify only update task on partial dm-workers // filter sub-task-configs through user specified workers // if worker not exist, an error message will return workerCfg := make(map[string]*config.SubTaskConfig) for _, stCfg := range stCfgs { - worker, ok := s.cfg.DeployMap[stCfg.SourceID] - if ok { - workerCfg[worker] = stCfg - } // only record existed workers + workerCfg[stCfg.SourceID] = stCfg } - stCfgs = make([]*config.SubTaskConfig, 0, len(req.Workers)) - for _, worker := range req.Workers { - if stCfg, ok := workerCfg[worker]; ok { - stCfgs = append(stCfgs, stCfg) + stCfgs = make([]*config.SubTaskConfig, 0, len(req.Sources)) + for _, source := range req.Sources { + if sourceCfg, ok := workerCfg[source]; ok { + stCfgs = append(stCfgs, sourceCfg) } else { - workerRespCh <- errorCommonWorkerResponse("worker not found in task's config or deployment config", worker) + workerRespCh <- errorCommonWorkerResponse("worker not found in task's config or deployment config", source) } } } @@ -542,32 +497,34 @@ func (s *Server) UpdateTask(ctx context.Context, req *pb.UpdateTaskRequest) (*pb wg.Add(1) go s.ap.Emit(ctx, 0, func(args ...interface{}) { defer wg.Done() - cli, worker, stCfgToml, _, err := s.taskConfigArgsExtractor(args...) + cfg, _ := args[0].(*config.SubTaskConfig) + worker, stCfgToml, _, err := s.taskConfigArgsExtractor(cfg) if err != nil { - workerRespCh <- errorCommonWorkerResponse(err.Error(), worker) + workerRespCh <- errorCommonWorkerResponse(err.Error(), cfg.SourceID) return } request := &workerrpc.Request{ Type: workerrpc.CmdUpdateSubTask, UpdateSubTask: &pb.UpdateSubTaskRequest{Task: stCfgToml}, } - resp, err := cli.SendRequest(ctx, request, s.cfg.RPCTimeout) + resp, err := worker.SendRequest(ctx, request, s.cfg.RPCTimeout) if err != nil { resp = &workerrpc.Response{ Type: workerrpc.CmdUpdateSubTask, - UpdateSubTask: errorCommonWorkerResponse(err.Error(), worker), + UpdateSubTask: errorCommonWorkerResponse(err.Error(), cfg.SourceID), } } - resp.UpdateSubTask.Worker = worker + resp.UpdateSubTask.Worker = cfg.SourceID workerRespCh <- resp.UpdateSubTask }, func(args ...interface{}) { defer wg.Done() - _, worker, _, _, err := s.taskConfigArgsExtractor(args...) + cfg, _ := args[0].(*config.SubTaskConfig) + worker, _, _, err := s.taskConfigArgsExtractor(cfg) if err != nil { - workerRespCh <- errorCommonWorkerResponse(err.Error(), worker) + workerRespCh <- errorCommonWorkerResponse(err.Error(), cfg.SourceID) return } - workerRespCh <- errorCommonWorkerResponse(terror.ErrMasterNoEmitToken.Generate(worker).Error(), worker) + workerRespCh <- errorCommonWorkerResponse(terror.ErrMasterNoEmitToken.Generate(worker.Address()).Error(), cfg.SourceID) }, stCfg) } wg.Wait() @@ -593,38 +550,37 @@ func (s *Server) UpdateTask(ctx context.Context, req *pb.UpdateTaskRequest) (*pb } type hasWokers interface { - GetWorkers() []string + GetSources() []string GetName() string } func extractWorkers(s *Server, req hasWokers) ([]string, error) { workers := make([]string, 0, len(s.coordinator.GetAllWorkers())) - if len(req.GetWorkers()) > 0 { + if len(req.GetSources()) > 0 { // query specified dm-workers - invalidWorkers := make([]string, 0, len(req.GetWorkers())) - for _, workerAddress := range req.GetWorkers() { - w := s.coordinator.GetWorkerByAddress(workerAddress) + invalidWorkers := make([]string, 0, len(req.GetSources())) + for _, source := range req.GetSources() { + w := s.coordinator.GetWorkerBySourceID(source) if w == nil || w.State() == coordinator.WorkerClosed { - invalidWorkers = append(invalidWorkers, workerAddress) + invalidWorkers = append(invalidWorkers, source) } } if len(invalidWorkers) > 0 { return nil, errors.Errorf("%s relevant worker-client not found", strings.Join(invalidWorkers, ", ")) } - workers = req.GetWorkers() + workers = req.GetSources() } else if len(req.GetName()) > 0 { // query specified task's workers - workers = s.getTaskWorkers(req.GetName()) + workers = s.getTaskResources(req.GetName()) if len(workers) == 0 { return nil, errors.Errorf("task %s has no workers or not exist, can try `refresh-worker-tasks` cmd first", req.GetName()) } } else { // query all workers log.L().Info("get workers") - for _, worker := range s.coordinator.GetAllWorkers() { - log.L().Info("debug", zap.Stringer("worker", worker)) - workers = append(workers, worker.String()) + for source := range s.coordinator.GetRunningMysqlSource() { + workers = append(workers, source) } } return workers, nil @@ -633,24 +589,24 @@ func extractWorkers(s *Server, req hasWokers) ([]string, error) { // QueryStatus implements MasterServer.QueryStatus func (s *Server) QueryStatus(ctx context.Context, req *pb.QueryStatusListRequest) (*pb.QueryStatusListResponse, error) { log.L().Info("", zap.Stringer("payload", req), zap.String("request", "QueryStatus")) - workers, err := extractWorkers(s, req) + sources, err := extractWorkers(s, req) if err != nil { return &pb.QueryStatusListResponse{ Result: false, Msg: err.Error(), }, nil } - workerRespCh := s.getStatusFromWorkers(ctx, workers, req.Name) + workerRespCh := s.getStatusFromWorkers(ctx, sources, req.Name) - workerRespMap := make(map[string]*pb.QueryStatusResponse, len(workers)) + workerRespMap := make(map[string]*pb.QueryStatusResponse, len(sources)) for len(workerRespCh) > 0 { workerResp := <-workerRespCh workerRespMap[workerResp.Worker] = workerResp } - sort.Strings(workers) - workerResps := make([]*pb.QueryStatusResponse, 0, len(workers)) - for _, worker := range workers { + sort.Strings(sources) + workerResps := make([]*pb.QueryStatusResponse, 0, len(sources)) + for _, worker := range sources { workerResps = append(workerResps, workerRespMap[worker]) } resp := &pb.QueryStatusListResponse{ @@ -664,7 +620,7 @@ func (s *Server) QueryStatus(ctx context.Context, req *pb.QueryStatusListRequest func (s *Server) QueryError(ctx context.Context, req *pb.QueryErrorListRequest) (*pb.QueryErrorListResponse, error) { log.L().Info("", zap.Stringer("payload", req), zap.String("request", "QueryError")) - workers, err := extractWorkers(s, req) + sources, err := extractWorkers(s, req) if err != nil { return &pb.QueryErrorListResponse{ Result: false, @@ -672,17 +628,17 @@ func (s *Server) QueryError(ctx context.Context, req *pb.QueryErrorListRequest) }, nil } - workerRespCh := s.getErrorFromWorkers(ctx, workers, req.Name) + workerRespCh := s.getErrorFromWorkers(ctx, sources, req.Name) - workerRespMap := make(map[string]*pb.QueryErrorResponse, len(workers)) + workerRespMap := make(map[string]*pb.QueryErrorResponse, len(sources)) for len(workerRespCh) > 0 { workerResp := <-workerRespCh workerRespMap[workerResp.Worker] = workerResp } - sort.Strings(workers) - workerResps := make([]*pb.QueryErrorResponse, 0, len(workers)) - for _, worker := range workers { + sort.Strings(sources) + workerResps := make([]*pb.QueryErrorResponse, 0, len(sources)) + for _, worker := range sources { workerResps = append(workerResps, workerRespMap[worker]) } resp := &pb.QueryErrorListResponse{ @@ -707,8 +663,8 @@ func (s *Server) ShowDDLLocks(ctx context.Context, req *pb.ShowDDLLocksRequest) continue // specify task and mismatch } ready := lock.Ready() - if len(req.Workers) > 0 { - for _, worker := range req.Workers { + if len(req.Sources) > 0 { + for _, worker := range req.Sources { if _, ok := ready[worker]; ok { goto FOUND } @@ -744,7 +700,7 @@ func (s *Server) ShowDDLLocks(ctx context.Context, req *pb.ShowDDLLocksRequest) func (s *Server) UnlockDDLLock(ctx context.Context, req *pb.UnlockDDLLockRequest) (*pb.UnlockDDLLockResponse, error) { log.L().Info("", zap.String("lock ID", req.ID), zap.Stringer("payload", req), zap.String("request", "UnlockDDLLock")) - workerResps, err := s.resolveDDLLock(ctx, req.ID, req.ReplaceOwner, req.Workers) + workerResps, err := s.resolveDDLLock(ctx, req.ID, req.ReplaceOwner, req.Sources) resp := &pb.UnlockDDLLockResponse{ Result: true, Workers: workerResps, @@ -779,15 +735,15 @@ func (s *Server) BreakWorkerDDLLock(ctx context.Context, req *pb.BreakWorkerDDLL }, } - workerRespCh := make(chan *pb.CommonWorkerResponse, len(req.Workers)) + workerRespCh := make(chan *pb.CommonWorkerResponse, len(req.Sources)) var wg sync.WaitGroup - for _, address := range req.Workers { + for _, source := range req.Sources { wg.Add(1) - go func(addr string) { + go func(sourceID string) { defer wg.Done() - worker := s.coordinator.GetWorkerByAddress(addr) + worker := s.coordinator.GetWorkerBySourceID(sourceID) if worker == nil || worker.State() == coordinator.WorkerClosed { - workerRespCh <- errorCommonWorkerResponse(fmt.Sprintf("worker %s relevant worker-client not found", worker), worker.Address()) + workerRespCh <- errorCommonWorkerResponse(fmt.Sprintf("worker %s relevant worker-client not found", sourceID), sourceID) return } cli, err := worker.GetClient() @@ -797,25 +753,25 @@ func (s *Server) BreakWorkerDDLLock(ctx context.Context, req *pb.BreakWorkerDDLL resp, err := cli.SendRequest(ctx, request, s.cfg.RPCTimeout) workerResp := &pb.CommonWorkerResponse{} if err != nil { - workerResp = errorCommonWorkerResponse(errors.ErrorStack(err), "") + workerResp = errorCommonWorkerResponse(errors.ErrorStack(err), sourceID) } else { workerResp = resp.BreakDDLLock } - workerResp.Worker = worker.Address() + workerResp.Worker = sourceID workerRespCh <- workerResp - }(address) + }(source) } wg.Wait() - workerRespMap := make(map[string]*pb.CommonWorkerResponse, len(req.Workers)) + workerRespMap := make(map[string]*pb.CommonWorkerResponse, len(req.Sources)) for len(workerRespCh) > 0 { workerResp := <-workerRespCh workerRespMap[workerResp.Worker] = workerResp } - sort.Strings(req.Workers) - workerResps := make([]*pb.CommonWorkerResponse, 0, len(req.Workers)) - for _, worker := range req.Workers { + sort.Strings(req.Sources) + workerResps := make([]*pb.CommonWorkerResponse, 0, len(req.Sources)) + for _, worker := range req.Sources { workerResps = append(workerResps, workerRespMap[worker]) } @@ -866,7 +822,7 @@ func (s *Server) HandleSQLs(ctx context.Context, req *pb.HandleSQLsRequest) (*pb SqlPattern: req.SqlPattern, }, } - cli := s.coordinator.GetWorkerClientByAddress(req.Worker) + cli := s.coordinator.GetWorkerBySourceID(req.Worker) if cli == nil { resp.Msg = fmt.Sprintf("worker %s client not found in %v", req.Worker, s.coordinator.GetAllWorkers()) return resp, nil @@ -874,7 +830,7 @@ func (s *Server) HandleSQLs(ctx context.Context, req *pb.HandleSQLsRequest) (*pb response, err := cli.SendRequest(ctx, subReq, s.cfg.RPCTimeout) workerResp := &pb.CommonWorkerResponse{} if err != nil { - workerResp = errorCommonWorkerResponse(errors.ErrorStack(err), "") + workerResp = errorCommonWorkerResponse(errors.ErrorStack(err), req.Worker) } else { workerResp = response.HandleSubTaskSQLs } @@ -897,39 +853,39 @@ func (s *Server) PurgeWorkerRelay(ctx context.Context, req *pb.PurgeWorkerRelayR }, } - workerRespCh := make(chan *pb.CommonWorkerResponse, len(req.Workers)) + workerRespCh := make(chan *pb.CommonWorkerResponse, len(req.Sources)) var wg sync.WaitGroup - for _, worker := range req.Workers { + for _, source := range req.Sources { wg.Add(1) - go func(worker string) { + go func(source string) { defer wg.Done() - cli := s.coordinator.GetWorkerClientByAddress(worker) + cli := s.coordinator.GetWorkerBySourceID(source) if cli == nil { - workerRespCh <- errorCommonWorkerResponse(fmt.Sprintf("worker %s relevant worker-client not found", worker), worker) + workerRespCh <- errorCommonWorkerResponse(fmt.Sprintf("worker %s relevant worker-client not found", source), source) return } resp, err := cli.SendRequest(ctx, workerReq, s.cfg.RPCTimeout) workerResp := &pb.CommonWorkerResponse{} if err != nil { - workerResp = errorCommonWorkerResponse(errors.ErrorStack(err), "") + workerResp = errorCommonWorkerResponse(errors.ErrorStack(err), source) } else { workerResp = resp.PurgeRelay } - workerResp.Worker = worker + workerResp.Worker = source workerRespCh <- workerResp - }(worker) + }(source) } wg.Wait() - workerRespMap := make(map[string]*pb.CommonWorkerResponse, len(req.Workers)) + workerRespMap := make(map[string]*pb.CommonWorkerResponse, len(req.Sources)) for len(workerRespCh) > 0 { workerResp := <-workerRespCh workerRespMap[workerResp.Worker] = workerResp } - sort.Strings(req.Workers) - workerResps := make([]*pb.CommonWorkerResponse, 0, len(req.Workers)) - for _, worker := range req.Workers { + sort.Strings(req.Sources) + workerResps := make([]*pb.CommonWorkerResponse, 0, len(req.Sources)) + for _, worker := range req.Sources { workerResps = append(workerResps, workerRespMap[worker]) } @@ -943,7 +899,7 @@ func (s *Server) PurgeWorkerRelay(ctx context.Context, req *pb.PurgeWorkerRelayR func (s *Server) SwitchWorkerRelayMaster(ctx context.Context, req *pb.SwitchWorkerRelayMasterRequest) (*pb.SwitchWorkerRelayMasterResponse, error) { log.L().Info("", zap.Stringer("payload", req), zap.String("request", "SwitchWorkerRelayMaster")) - workerRespCh := make(chan *pb.CommonWorkerResponse, len(req.Workers)) + workerRespCh := make(chan *pb.CommonWorkerResponse, len(req.Sources)) handleErr := func(err error, worker string) { log.L().Error("response error", zap.Error(err)) @@ -952,13 +908,15 @@ func (s *Server) SwitchWorkerRelayMaster(ctx context.Context, req *pb.SwitchWork } var wg sync.WaitGroup - for _, worker := range req.Workers { + for _, source := range req.Sources { wg.Add(1) go s.ap.Emit(ctx, 0, func(args ...interface{}) { defer wg.Done() - cli, worker1, err := s.workerArgsExtractor(args...) - if err != nil { - handleErr(err, worker1) + sourceID, _ := args[0].(string) + cli := s.coordinator.GetWorkerBySourceID(sourceID) + if cli == nil { + err := terror.ErrMasterWorkerArgsExtractor.Generatef("%s relevant worker-client not found", sourceID) + handleErr(err, sourceID) return } request := &workerrpc.Request{ @@ -968,33 +926,29 @@ func (s *Server) SwitchWorkerRelayMaster(ctx context.Context, req *pb.SwitchWork resp, err := cli.SendRequest(ctx, request, s.cfg.RPCTimeout) workerResp := &pb.CommonWorkerResponse{} if err != nil { - workerResp = errorCommonWorkerResponse(errors.ErrorStack(err), "") + workerResp = errorCommonWorkerResponse(errors.ErrorStack(err), sourceID) } else { workerResp = resp.SwitchRelayMaster } - workerResp.Worker = worker1 + workerResp.Worker = sourceID workerRespCh <- workerResp }, func(args ...interface{}) { defer wg.Done() - _, worker1, err := s.workerArgsExtractor(args...) - if err != nil { - handleErr(err, worker1) - return - } - workerRespCh <- errorCommonWorkerResponse(terror.ErrMasterNoEmitToken.Generate(worker1).Error(), worker1) - }, worker) + sourceID, _ := args[0].(string) + workerRespCh <- errorCommonWorkerResponse(terror.ErrMasterNoEmitToken.Generate(sourceID).Error(), sourceID) + }, source) } wg.Wait() - workerRespMap := make(map[string]*pb.CommonWorkerResponse, len(req.Workers)) + workerRespMap := make(map[string]*pb.CommonWorkerResponse, len(req.Sources)) for len(workerRespCh) > 0 { workerResp := <-workerRespCh workerRespMap[workerResp.Worker] = workerResp } - sort.Strings(req.Workers) - workerResps := make([]*pb.CommonWorkerResponse, 0, len(req.Workers)) - for _, worker := range req.Workers { + sort.Strings(req.Sources) + workerResps := make([]*pb.CommonWorkerResponse, 0, len(req.Sources)) + for _, worker := range req.Sources { workerResps = append(workerResps, workerRespMap[worker]) } @@ -1012,19 +966,19 @@ func (s *Server) OperateWorkerRelayTask(ctx context.Context, req *pb.OperateWork Type: workerrpc.CmdOperateRelay, OperateRelay: &pb.OperateRelayRequest{Op: req.Op}, } - workerRespCh := make(chan *pb.OperateRelayResponse, len(req.Workers)) + workerRespCh := make(chan *pb.OperateRelayResponse, len(req.Sources)) var wg sync.WaitGroup - for _, worker := range req.Workers { + for _, source := range req.Sources { wg.Add(1) - go func(worker string) { + go func(source string) { defer wg.Done() - cli := s.coordinator.GetWorkerClientByAddress(worker) + cli := s.coordinator.GetWorkerBySourceID(source) if cli == nil { workerResp := &pb.OperateRelayResponse{ Op: req.Op, Result: false, - Worker: worker, - Msg: fmt.Sprintf("%s relevant worker-client not found", worker), + Worker: source, + Msg: fmt.Sprintf("%s relevant worker-client not found", source), } workerRespCh <- workerResp return @@ -1040,21 +994,21 @@ func (s *Server) OperateWorkerRelayTask(ctx context.Context, req *pb.OperateWork workerResp = resp.OperateRelay } workerResp.Op = req.Op - workerResp.Worker = worker + workerResp.Worker = source workerRespCh <- workerResp - }(worker) + }(source) } wg.Wait() - workerRespMap := make(map[string]*pb.OperateRelayResponse, len(req.Workers)) + workerRespMap := make(map[string]*pb.OperateRelayResponse, len(req.Sources)) for len(workerRespCh) > 0 { workerResp := <-workerRespCh workerRespMap[workerResp.Worker] = workerResp } - sort.Strings(req.Workers) - workerResps := make([]*pb.OperateRelayResponse, 0, len(req.Workers)) - for _, worker := range req.Workers { + sort.Strings(req.Sources) + workerResps := make([]*pb.OperateRelayResponse, 0, len(req.Sources)) + for _, worker := range req.Sources { workerResps = append(workerResps, workerRespMap[worker]) } @@ -1064,71 +1018,6 @@ func (s *Server) OperateWorkerRelayTask(ctx context.Context, req *pb.OperateWork }, nil } -// RefreshWorkerTasks implements MasterServer.RefreshWorkerTasks -func (s *Server) RefreshWorkerTasks(ctx context.Context, req *pb.RefreshWorkerTasksRequest) (*pb.RefreshWorkerTasksResponse, error) { - log.L().Info("", zap.Stringer("payload", req), zap.String("request", "RefreshWorkerTasks")) - - taskWorkers, workerMsgMap := s.fetchTaskWorkers(ctx) - // always update task workers mapping in memory - s.replaceTaskWorkers(taskWorkers) - log.L().Info("refresh workers of task", zap.Reflect("workers of task", taskWorkers), zap.String("request", "RefreshWorkerTasks")) - - workers := make([]string, 0, len(workerMsgMap)) - for worker := range workerMsgMap { - workers = append(workers, worker) - } - sort.Strings(workers) - - workerMsgs := make([]*pb.RefreshWorkerTasksMsg, 0, len(workers)) - for _, worker := range workers { - workerMsgs = append(workerMsgs, &pb.RefreshWorkerTasksMsg{ - Worker: worker, - Msg: workerMsgMap[worker], - }) - } - return &pb.RefreshWorkerTasksResponse{ - Result: true, - Workers: workerMsgs, - }, nil -} - -// addTaskWorkers adds a task-workers pair -// replace indicates whether replace old workers -func (s *Server) addTaskWorkers(task string, workers []string, replace bool) { - if len(workers) == 0 { - return - } - - valid := make([]string, 0, len(workers)) - for _, worker := range workers { - if w := s.coordinator.GetWorkerByAddress(worker); w != nil { - valid = append(valid, worker) - } - } - - s.Lock() - defer s.Unlock() - if !replace { - // merge with old workers - old, ok := s.taskWorkers[task] - if ok { - exist := make(map[string]struct{}) - for _, worker := range valid { - exist[worker] = struct{}{} - } - for _, worker := range old { - if _, ok := exist[worker]; !ok { - valid = append(valid, worker) - } - } - } - } - - sort.Strings(valid) - s.taskWorkers[task] = valid - log.L().Info("update workers of task", zap.String("task", task), zap.Strings("workers", valid)) -} - // replaceTaskWorkers replaces the whole task-workers mapper func (s *Server) replaceTaskWorkers(taskWorkers map[string][]string) { for task := range taskWorkers { @@ -1136,7 +1025,7 @@ func (s *Server) replaceTaskWorkers(taskWorkers map[string][]string) { } s.Lock() defer s.Unlock() - s.taskWorkers = taskWorkers + s.taskSources = taskWorkers } // removeTaskWorkers remove (partial / all) workers for a task @@ -1148,30 +1037,30 @@ func (s *Server) removeTaskWorkers(task string, workers []string) { s.Lock() defer s.Unlock() - if _, ok := s.taskWorkers[task]; !ok { + if _, ok := s.taskSources[task]; !ok { log.L().Warn("not found workers", zap.String("task", task)) return } - remain := make([]string, 0, len(s.taskWorkers[task])) - for _, worker := range s.taskWorkers[task] { + remain := make([]string, 0, len(s.taskSources[task])) + for _, worker := range s.taskSources[task] { if _, ok := toRemove[worker]; !ok { remain = append(remain, worker) } } if len(remain) == 0 { - delete(s.taskWorkers, task) + delete(s.taskSources, task) log.L().Info("remove task from taskWorker", zap.String("task", task)) } else { - s.taskWorkers[task] = remain + s.taskSources[task] = remain log.L().Info("update workers of task", zap.String("task", task), zap.Strings("reamin workers", remain)) } } -// getTaskWorkers gets workers relevant to specified task -func (s *Server) getTaskWorkers(task string) []string { +// getTaskResources gets workers relevant to specified task +func (s *Server) getTaskResources(task string) []string { s.Lock() defer s.Unlock() - workers, ok := s.taskWorkers[task] + workers, ok := s.taskSources[task] if !ok { return []string{} } @@ -1215,9 +1104,11 @@ func (s *Server) getStatusFromWorkers(ctx context.Context, workers []string, tas wg.Add(1) go s.ap.Emit(ctx, 0, func(args ...interface{}) { defer wg.Done() - cli, worker1, err := s.workerArgsExtractor(args...) - if err != nil { - handleErr(err, worker1) + sourceID, _ := args[0].(string) + cli := s.coordinator.GetWorkerBySourceID(sourceID) + if cli == nil { + err := terror.ErrMasterWorkerArgsExtractor.Generatef("%s relevant worker-client not found", sourceID) + handleErr(err, sourceID) return } resp, err := cli.SendRequest(ctx, workerReq, s.cfg.RPCTimeout) @@ -1230,16 +1121,12 @@ func (s *Server) getStatusFromWorkers(ctx context.Context, workers []string, tas } else { workerStatus = resp.QueryStatus } - workerStatus.Worker = worker1 + workerStatus.Worker = sourceID workerRespCh <- workerStatus }, func(args ...interface{}) { defer wg.Done() - _, worker1, err := s.workerArgsExtractor(args...) - if err != nil { - handleErr(err, worker1) - return - } - handleErr(terror.ErrMasterNoEmitToken.Generate(worker1), worker1) + sourceID, _ := args[0].(string) + handleErr(terror.ErrMasterNoEmitToken.Generate(sourceID), sourceID) }, worker) } wg.Wait() @@ -1270,11 +1157,14 @@ func (s *Server) getErrorFromWorkers(ctx context.Context, workers []string, task wg.Add(1) go s.ap.Emit(ctx, 0, func(args ...interface{}) { defer wg.Done() - cli, worker1, err := s.workerArgsExtractor(args...) - if err != nil { - handleErr(err, worker1) + sourceID, _ := args[0].(string) + cli := s.coordinator.GetWorkerBySourceID(sourceID) + if cli == nil { + err := terror.ErrMasterWorkerArgsExtractor.Generatef("%s relevant worker-client not found", sourceID) + handleErr(err, sourceID) return } + resp, err := cli.SendRequest(ctx, workerReq, s.cfg.RPCTimeout) workerError := &pb.QueryErrorResponse{} if err != nil { @@ -1285,23 +1175,19 @@ func (s *Server) getErrorFromWorkers(ctx context.Context, workers []string, task } else { workerError = resp.QueryError } - workerError.Worker = worker1 + workerError.Worker = sourceID workerRespCh <- workerError }, func(args ...interface{}) { defer wg.Done() - _, worker1, err := s.workerArgsExtractor(args...) - if err != nil { - handleErr(err, worker1) - return - } - handleErr(terror.ErrMasterNoEmitToken.Generate(worker1), worker1) + sourceID, _ := args[0].(string) + handleErr(terror.ErrMasterNoEmitToken.Generate(sourceID), sourceID) }, worker) } wg.Wait() return workerRespCh } -// updateTaskWorkers fetches task-workers mapper from dm-workers and update s.taskWorkers +// updateTaskWorkers fetches task-workers mapper from dm-workers and update s.taskSources func (s *Server) updateTaskWorkers(ctx context.Context) { taskWorkers, _ := s.fetchTaskWorkers(ctx) if len(taskWorkers) == 0 { @@ -1353,7 +1239,7 @@ func (s *Server) fetchTaskWorkers(ctx context.Context) (map[string][]string, map // return true means match, false means mismatch. func (s *Server) checkTaskAndWorkerMatch(taskname string, targetWorker string) bool { // find worker - workers := s.getTaskWorkers(taskname) + workers := s.getTaskResources(taskname) if len(workers) == 0 { return false } @@ -1371,7 +1257,7 @@ func (s *Server) fetchWorkerDDLInfo(ctx context.Context) { var wg sync.WaitGroup request := &workerrpc.Request{Type: workerrpc.CmdFetchDDLInfo} - for address, w := range s.coordinator.GetAllWorkers() { + for _, w := range s.coordinator.GetWorkersByStatus(coordinator.WorkerBound) { client, err := w.GetClient() if err != nil { log.L().Error("cannot get worker client", zap.String("name", w.Name())) @@ -1421,7 +1307,7 @@ func (s *Server) fetchWorkerDDLInfo(ctx context.Context) { } log.L().Info("receive ddl info", zap.Stringer("ddl info", in), zap.String("worker", worker)) - workers := s.getTaskWorkers(in.Task) + workers := s.getTaskResources(in.Task) if len(workers) == 0 { // should happen only when starting and before updateTaskWorkers return log.L().Error("try to sync shard DDL, but with no workers", zap.String("task", in.Task)) @@ -1481,7 +1367,7 @@ func (s *Server) fetchWorkerDDLInfo(ctx context.Context) { } } - }(address, client) + }(w.Address(), client) } wg.Wait() @@ -1565,7 +1451,7 @@ func (s *Server) resolveDDLLock(ctx context.Context, lockID string, replaceOwner resp, err := cli.SendRequest(ctx, ownerReq, ownerTimeout) ownerResp := &pb.CommonWorkerResponse{} if err != nil { - ownerResp = errorCommonWorkerResponse(errors.ErrorStack(err), "") + ownerResp = errorCommonWorkerResponse(errors.ErrorStack(err), owner) } else { ownerResp = resp.ExecDDL } @@ -1687,89 +1573,12 @@ func (s *Server) UpdateMasterConfig(ctx context.Context, req *pb.UpdateMasterCon }, nil } log.L().Info("update dm-master config", zap.Stringer("config", cfg), zap.String("request", "UpdateMasterConfig")) - - // TODO: fix me - // delete worker - invalidWorkers := s.coordinator.GetWorkersByStatus(coordinator.WorkerClosed) - wokerList := make([]string, 0, len(invalidWorkers)) - for _, worker := range invalidWorkers { - wokerList = append(wokerList, worker.Address()) - DDLreq := &pb.ShowDDLLocksRequest{ - Task: "", - Workers: wokerList, - } - resp, err2 := s.ShowDDLLocks(ctx, DDLreq) - if err2 != nil { - s.Unlock() - return &pb.UpdateMasterConfigResponse{ - Result: false, - Msg: fmt.Sprintf("Failed to get DDL lock Info from %s, detail: ", worker.Address()) + errors.ErrorStack(err2), - }, nil - } - if len(resp.Locks) != 0 { - err = terror.ErrMasterWorkerExistDDLLock.Generate(worker.Address()) - s.Unlock() - return &pb.UpdateMasterConfigResponse{ - Result: false, - Msg: errors.ErrorStack(err), - }, nil - } - } - - for i := 0; i < len(wokerList); i++ { - delete(s.workerClients, wokerList[i]) - } - - // TODO: fix me - // // add new worker - // for _, workerAddr := range cfg.DeployMap { - // if _, ok := s.workerClients[workerAddr]; !ok { - // cli, err2 := workerrpc.NewGRPCClient(workerAddr) - // if err2 != nil { - // s.Unlock() - // return &pb.UpdateMasterConfigResponse{ - // Result: false, - // Msg: fmt.Sprintf("Failed to add woker %s, detail: ", workerAddr) + errors.ErrorStack(err2), - // }, nil - // } - // s.workerClients[workerAddr] = cli - // } - // } - - // update log configure. not supported now - /*log.SetLevelByString(strings.ToLower(cfg.LogLevel)) - if len(cfg.LogFile) > 0 { - log.SetOutputByName(cfg.LogFile) - }*/ - s.cfg = cfg log.L().Info("update dm-master config file success", zap.String("request", "UpdateMasterConfig")) s.Unlock() - - workers := make([]string, 0, len(s.workerClients)) - for worker := range s.workerClients { - workers = append(workers, worker) - } - - workerRespCh := s.getStatusFromWorkers(ctx, workers, "") - log.L().Info("checking every dm-worker status...", zap.String("request", "UpdateMasterConfig")) - - workerRespMap := make(map[string]*pb.QueryStatusResponse, len(workers)) - for len(workerRespCh) > 0 { - workerResp := <-workerRespCh - workerRespMap[workerResp.Worker] = workerResp - } - - sort.Strings(workers) - workerResps := make([]*pb.QueryStatusResponse, 0, len(workers)) - for _, worker := range workers { - workerResps = append(workerResps, workerRespMap[worker]) - } - return &pb.UpdateMasterConfigResponse{ - Result: true, - Msg: "", - Workers: workerResps, + Result: true, + Msg: "", }, nil } @@ -1778,7 +1587,7 @@ func (s *Server) UpdateWorkerRelayConfig(ctx context.Context, req *pb.UpdateWork log.L().Info("", zap.Stringer("payload", req), zap.String("request", "UpdateWorkerRelayConfig")) worker := req.Worker content := req.Config - cli := s.coordinator.GetWorkerClientByAddress(worker) + cli := s.coordinator.GetWorkerBySourceID(worker) if cli == nil { return errorCommonWorkerResponse(fmt.Sprintf("worker %s relevant worker-client not found", worker), worker), nil } @@ -1796,109 +1605,14 @@ func (s *Server) UpdateWorkerRelayConfig(ctx context.Context, req *pb.UpdateWork } // TODO: refine the call stack of this API, query worker configs that we needed only -func (s *Server) getWorkerConfigs(ctx context.Context, workerIDs []string) (map[string]config.DBConfig, error) { - var ( - wg sync.WaitGroup - workerMutex sync.Mutex - workerCfgs = make(map[string]config.DBConfig) - errCh = make(chan error, len(s.coordinator.GetAllWorkers())) - err error - ) - handleErr := func(err2 error) bool { - if err2 != nil { - log.L().Error("response error", zap.Error(err2)) - } - errCh <- err2 - return false - } - - argsExtractor := func(args ...interface{}) (string, workerrpc.Client, bool) { - if len(args) != 2 { - return "", nil, handleErr(terror.ErrMasterGetWorkerCfgExtractor.Generatef("fail to call emit to fetch worker config, miss some arguments %v", args)) - } - - worker, ok := args[0].(string) - if !ok { - return "", nil, handleErr(terror.ErrMasterGetWorkerCfgExtractor.Generatef("fail to call emit to fetch worker config, can't get id from args[0], arguments %v", args)) - } - - client, ok := args[1].(*workerrpc.GRPCClient) - if !ok { - return "", nil, handleErr(terror.ErrMasterGetWorkerCfgExtractor.Generatef("fail to call emit to fetch config of worker %s, can't get worker client from args[1], arguments %v", worker, args)) - } - - return worker, client, true - } - - request := &workerrpc.Request{ - Type: workerrpc.CmdQueryWorkerConfig, - QueryWorkerConfig: &pb.QueryWorkerConfigRequest{}, - } - for _, worker := range workerIDs { - client := s.coordinator.GetWorkerClientByAddress(worker) - if client == nil { - continue // outer caller can handle the lack of the config +func (s *Server) getWorkerConfigs(ctx context.Context, workers []*config.MySQLInstance) map[string]config.DBConfig { + cfgs := make(map[string]config.DBConfig) + for _, w := range workers { + if cfg := s.coordinator.GetConfigBySourceID(w.SourceID); cfg != nil { + cfgs[w.SourceID] = cfg.From } - wg.Add(1) - go s.ap.Emit(ctx, 0, func(args ...interface{}) { - defer wg.Done() - - worker1, client1, ok := argsExtractor(args...) - if !ok { - return - } - - response, err1 := client1.SendRequest(ctx, request, s.cfg.RPCTimeout) - if err1 != nil { - handleErr(terror.Annotatef(err1, "fetch config of worker %s", worker1)) - return - } - - resp := response.QueryWorkerConfig - if !resp.Result { - handleErr(terror.ErrMasterQueryWorkerConfig.Generatef("fail to query config from worker %s, message %s", worker1, resp.Msg)) - return - } - - if len(resp.Content) == 0 { - handleErr(terror.ErrMasterQueryWorkerConfig.Generatef("fail to query config from worker %s, config is empty", worker1)) - return - } - - if len(resp.SourceID) == 0 { - handleErr(terror.ErrMasterQueryWorkerConfig.Generatef("fail to query config from worker %s, source ID is empty, it should be set in worker config", worker1)) - return - } - - dbCfg := &config.DBConfig{} - err2 := dbCfg.Decode(resp.Content) - if err2 != nil { - handleErr(terror.WithClass(terror.Annotatef(err2, "unmarshal worker %s config, resp: %s", worker1, resp.Content), terror.ClassDMMaster)) - return - } - - workerMutex.Lock() - workerCfgs[resp.SourceID] = *dbCfg - workerMutex.Unlock() - - }, func(args ...interface{}) { - defer wg.Done() - - worker1, _, ok := argsExtractor(args...) - if !ok { - return - } - handleErr(terror.ErrMasterNoEmitToken.Generate(worker1)) - }, worker, client) - } - - wg.Wait() - - if len(errCh) > 0 { - err = <-errCh } - - return workerCfgs, err + return cfgs } // MigrateWorkerRelay migrates dm-woker relay unit @@ -1907,7 +1621,7 @@ func (s *Server) MigrateWorkerRelay(ctx context.Context, req *pb.MigrateWorkerRe worker := req.Worker binlogPos := req.BinlogPos binlogName := req.BinlogName - cli := s.coordinator.GetWorkerClientByAddress(worker) + cli := s.coordinator.GetWorkerBySourceID(worker) if cli == nil { return errorCommonWorkerResponse(fmt.Sprintf("worker %s relevant worker-client not found", worker), worker), nil } @@ -1941,6 +1655,98 @@ func (s *Server) CheckTask(ctx context.Context, req *pb.CheckTaskRequest) (*pb.C }, nil } +func makeMysqlTaskResponse(err error) (*pb.MysqlTaskResponse, error) { + return &pb.MysqlTaskResponse{ + Result: false, + Msg: errors.ErrorStack(err), + }, nil +} + +// OperateMysqlWorker will create or update a Worker +func (s *Server) OperateMysqlWorker(ctx context.Context, req *pb.MysqlTaskRequest) (*pb.MysqlTaskResponse, error) { + cfg := config.NewWorkerConfig() + if err := cfg.Parse(req.Config); err != nil { + return makeMysqlTaskResponse(err) + } + + dbConfig, err := cfg.GenerateDBConfig() + if err != nil { + return makeMysqlTaskResponse(err) + } + fromDB, err := conn.DefaultDBProvider.Apply(*dbConfig) + if err != nil { + return makeMysqlTaskResponse(err) + } + if err := cfg.Adjust(fromDB.DB); err != nil { + return makeMysqlTaskResponse(err) + } + var resp *pb.MysqlTaskResponse + if req.Op == pb.WorkerOp_StartWorker { + w := s.coordinator.GetWorkerBySourceID(cfg.SourceID) + if w != nil { + return &pb.MysqlTaskResponse{ + Result: false, + Msg: "Create worker failed. worker has been started", + }, nil + } + w, addr := s.coordinator.GetFreeWorkerForSource(cfg.SourceID) + if w == nil { + if addr != "" { + return &pb.MysqlTaskResponse{ + Result: false, + Msg: fmt.Sprintf("Create worker failed. the same source has been started in worker: %v", addr), + }, nil + } + return &pb.MysqlTaskResponse{ + Result: false, + Msg: "Create worker failed. no free worker could start mysql task", + }, nil + } + resp, err = w.OperateMysqlTask(ctx, req, s.cfg.RPCTimeout) + if err != nil { + s.coordinator.HandleStartedWorker(w, cfg, false) + return &pb.MysqlTaskResponse{ + Result: false, + Msg: errors.ErrorStack(err), + }, nil + } + s.coordinator.HandleStartedWorker(w, cfg, true) + } else if req.Op == pb.WorkerOp_UpdateConfig { + w := s.coordinator.GetWorkerBySourceID(cfg.SourceID) + if w == nil { + return &pb.MysqlTaskResponse{ + Result: false, + Msg: "Update worker config failed. worker has not been started", + }, nil + } + if resp, err = w.OperateMysqlTask(ctx, req, s.cfg.RPCTimeout); err != nil { + return &pb.MysqlTaskResponse{ + Result: false, + Msg: errors.ErrorStack(err), + }, nil + } + } else { + w := s.coordinator.GetWorkerBySourceID(cfg.SourceID) + if w == nil { + return &pb.MysqlTaskResponse{ + Result: false, + Msg: "Stop worker failed. worker has not been started", + }, nil + } + if resp, err = w.OperateMysqlTask(ctx, req, s.cfg.RPCTimeout); err != nil { + return &pb.MysqlTaskResponse{ + Result: false, + Msg: errors.ErrorStack(err), + }, nil + } + s.coordinator.HandleStoppedWorker(w, cfg) + } + return &pb.MysqlTaskResponse{ + Result: resp.Result, + Msg: resp.Msg, + }, nil +} + func (s *Server) generateSubTask(ctx context.Context, task string) (*config.TaskConfig, []*config.SubTaskConfig, error) { cfg := config.NewTaskConfig() err := cfg.Decode(task) @@ -1948,17 +1754,7 @@ func (s *Server) generateSubTask(ctx context.Context, task string) (*config.Task return nil, nil, terror.WithClass(err, terror.ClassDMMaster) } - // get workerID from deploy map by sourceID, refactor this when dynamic add/remove worker supported. - workerIDs := make([]string, 0, len(cfg.MySQLInstances)) - for _, inst := range cfg.MySQLInstances { - workerID, ok := s.cfg.DeployMap[inst.SourceID] - if !ok { - return nil, nil, terror.ErrMasterTaskConfigExtractor.Generatef("%s relevant worker not found", inst.SourceID) - } - workerIDs = append(workerIDs, workerID) - } - - sourceCfgs, err := s.getWorkerConfigs(ctx, workerIDs) + sourceCfgs := s.getWorkerConfigs(ctx, cfg.MySQLInstances) if err != nil { return nil, nil, err } @@ -1982,50 +1778,32 @@ var ( // taskConfigArgsExtractor extracts SubTaskConfig from args and returns its relevant // grpc client, worker id (host:port), subtask config in toml, task name and error -func (s *Server) taskConfigArgsExtractor(args ...interface{}) (workerrpc.Client, string, string, string, error) { +func (s *Server) taskConfigArgsExtractor(cfg *config.SubTaskConfig) (*coordinator.Worker, string, string, error) { handleErr := func(err error) error { log.L().Error("response", zap.Error(err)) return err } - if len(args) != 1 { - return nil, "", "", "", handleErr(terror.ErrMasterTaskConfigExtractor.Generatef("miss task config %v", args)) - } - - cfg, ok := args[0].(*config.SubTaskConfig) - if !ok { - return nil, "", "", "", handleErr(terror.ErrMasterTaskConfigExtractor.Generatef("args[0] is not SubTaskConfig: %v", args[0])) - } - - worker, ok1 := s.cfg.DeployMap[cfg.SourceID] - cli := s.coordinator.GetWorkerClientByAddress(worker) - if !ok1 || cli == nil { - return nil, "", "", "", handleErr(terror.ErrMasterTaskConfigExtractor.Generatef("%s relevant worker-client not found", worker)) + worker := s.coordinator.GetWorkerBySourceID(cfg.SourceID) + if worker == nil { + return nil, "", "", handleErr(terror.ErrMasterTaskConfigExtractor.Generatef("%s relevant worker-client not found", worker)) } cfgToml, err := cfg.Toml() if err != nil { - return nil, "", "", "", handleErr(err) + return nil, "", "", handleErr(err) } - return cli, worker, cfgToml, cfg.Name, nil + return worker, cfgToml, cfg.Name, nil } // workerArgsExtractor extracts worker from args and returns its relevant // grpc client, worker id (host:port) and error -func (s *Server) workerArgsExtractor(args ...interface{}) (workerrpc.Client, string, error) { - if len(args) != 1 { - return nil, "", terror.ErrMasterWorkerArgsExtractor.Generatef("miss worker id %v", args) - } - worker, ok := args[0].(string) - if !ok { - return nil, "", terror.ErrMasterWorkerArgsExtractor.Generatef("invalid argument, args[0] is not valid worker id: %v", args[0]) - } - log.L().Info("Debug get worker", zap.String("worker", worker)) - cli := s.coordinator.GetWorkerClientByAddress(worker) +func (s *Server) workerArgsExtractor(source string) (*coordinator.Worker, error) { + log.L().Info("Debug get worker", zap.String("source-id", source)) + cli := s.coordinator.GetWorkerBySourceID(source) if cli == nil { - return nil, worker, terror.ErrMasterWorkerArgsExtractor.Generatef("%s relevant worker-client not found", worker) + return nil, terror.ErrMasterWorkerArgsExtractor.Generatef("%s relevant worker-client not found", source) } - - return cli, worker, nil + return cli, nil } diff --git a/dm/master/server_test.go b/dm/master/server_test.go index 33b88b69a5..65759fb085 100644 --- a/dm/master/server_test.go +++ b/dm/master/server_test.go @@ -286,14 +286,14 @@ func (t *testMaster) TestQueryStatus(c *check.C) { server.workerClients[workerAddr] = newMockRPCClient(mockWorkerClient) } resp, err = server.QueryStatus(context.Background(), &pb.QueryStatusListRequest{ - Workers: workers, + Sources: workers, }) c.Assert(err, check.IsNil) c.Assert(resp.Result, check.IsTrue) // query with invalid dm-worker[s] resp, err = server.QueryStatus(context.Background(), &pb.QueryStatusListRequest{ - Workers: []string{"invalid-worker1", "invalid-worker2"}, + Sources: []string{"invalid-worker1", "invalid-worker2"}, }) c.Assert(err, check.IsNil) c.Assert(resp.Result, check.IsFalse) @@ -352,7 +352,7 @@ func (t *testMaster) TestShowDDLLocks(c *check.C) { // test query with task name resp, err = server.ShowDDLLocks(context.Background(), &pb.ShowDDLLocksRequest{ Task: "testA", - Workers: workers, + Sources: workers, }) c.Assert(err, check.IsNil) c.Assert(resp.Result, check.IsTrue) @@ -361,7 +361,7 @@ func (t *testMaster) TestShowDDLLocks(c *check.C) { // test specify a mismatch worker resp, err = server.ShowDDLLocks(context.Background(), &pb.ShowDDLLocksRequest{ - Workers: []string{"invalid-worker"}, + Sources: []string{"invalid-worker"}, }) c.Assert(err, check.IsNil) c.Assert(resp.Result, check.IsTrue) @@ -436,7 +436,7 @@ func (t *testMaster) TestStartTask(c *check.C) { testMockStartTask(c, server, ctrl, workerCfg, true) resp, err = server.StartTask(context.Background(), &pb.StartTaskRequest{ Task: taskConfig, - Workers: workers, + Sources: workers, }) c.Assert(err, check.IsNil) c.Assert(resp.Result, check.IsTrue) @@ -446,7 +446,7 @@ func (t *testMaster) TestStartTask(c *check.C) { testMockWorkerConfig(c, server, ctrl, "", true) resp, err = server.StartTask(context.Background(), &pb.StartTaskRequest{ Task: taskConfig, - Workers: []string{invalidWorker}, + Sources: []string{invalidWorker}, }) c.Assert(err, check.IsNil) c.Assert(resp.Result, check.IsTrue) @@ -478,7 +478,7 @@ func (t *testMaster) TestStartTask(c *check.C) { testMockWorkerConfig(c, server, ctrl, "", true) resp, err = server.StartTask(context.Background(), &pb.StartTaskRequest{ Task: taskConfig, - Workers: workers, + Sources: workers, }) c.Assert(err, check.IsNil) c.Assert(resp.Result, check.IsFalse) @@ -515,14 +515,14 @@ func (t *testMaster) TestQueryError(c *check.C) { server.workerClients[workerAddr] = newMockRPCClient(mockWorkerClient) } resp, err = server.QueryError(context.Background(), &pb.QueryErrorListRequest{ - Workers: workers, + Sources: workers, }) c.Assert(err, check.IsNil) c.Assert(resp.Result, check.IsTrue) // query with invalid dm-worker[s] resp, err = server.QueryError(context.Background(), &pb.QueryErrorListRequest{ - Workers: []string{"invalid-worker1", "invalid-worker2"}, + Sources: []string{"invalid-worker1", "invalid-worker2"}, }) c.Assert(err, check.IsNil) c.Assert(resp.Result, check.IsFalse) @@ -563,7 +563,7 @@ func (t *testMaster) TestOperateTask(c *check.C) { c.Assert(resp.Msg, check.Equals, fmt.Sprintf("task %s has no workers or not exist, can try `refresh-worker-tasks` cmd first", taskName)) // test operate-task while worker clients not found - server.taskWorkers[taskName] = workers + server.taskSources[taskName] = workers resp, err = server.OperateTask(context.Background(), &pb.OperateTaskRequest{ Op: pauseOp, Name: taskName, @@ -620,7 +620,7 @@ func (t *testMaster) TestOperateTask(c *check.C) { resp, err = server.OperateTask(context.Background(), &pb.OperateTaskRequest{ Op: pb.TaskOp_Pause, Name: taskName, - Workers: workers, + Sources: workers, }) c.Assert(err, check.IsNil) c.Assert(resp.Result, check.IsTrue) @@ -647,17 +647,17 @@ func (t *testMaster) TestOperateTask(c *check.C) { resp, err = server.OperateTask(context.Background(), &pb.OperateTaskRequest{ Op: pb.TaskOp_Stop, Name: taskName, - Workers: []string{workers[0]}, + Sources: []string{workers[0]}, }) c.Assert(err, check.IsNil) c.Assert(resp.Result, check.IsTrue) c.Assert(resp.Workers, check.HasLen, 1) c.Assert(resp.Workers[0].Result, check.IsTrue) - c.Assert(server.taskWorkers, check.HasKey, taskName) - c.Assert(server.taskWorkers[taskName], check.DeepEquals, []string{workers[1]}) + c.Assert(server.taskSources, check.HasKey, taskName) + c.Assert(server.taskSources[taskName], check.DeepEquals, []string{workers[1]}) // test stop task successfully, remove all workers - server.taskWorkers[taskName] = workers + server.taskSources[taskName] = workers for _, deploy := range server.cfg.Deploy { mockWorkerClient := pbmock.NewMockWorkerClient(ctrl) mockWorkerClient.EXPECT().OperateSubTask( @@ -684,7 +684,7 @@ func (t *testMaster) TestOperateTask(c *check.C) { c.Assert(subtaskResp.Op, check.Equals, pb.TaskOp_Stop) c.Assert(subtaskResp.Result, check.IsTrue) } - c.Assert(len(server.taskWorkers), check.Equals, 0) + c.Assert(len(server.taskSources), check.Equals, 0) } func (t *testMaster) TestUpdateTask(c *check.C) { @@ -765,7 +765,7 @@ func (t *testMaster) TestUpdateTask(c *check.C) { mockUpdateTask(true) resp, err = server.UpdateTask(context.Background(), &pb.UpdateTaskRequest{ Task: taskConfig, - Workers: workers, + Sources: workers, }) c.Assert(err, check.IsNil) c.Assert(resp.Result, check.IsTrue) @@ -775,7 +775,7 @@ func (t *testMaster) TestUpdateTask(c *check.C) { testMockWorkerConfig(c, server, ctrl, "", true) resp, err = server.UpdateTask(context.Background(), &pb.UpdateTaskRequest{ Task: taskConfig, - Workers: []string{invalidWorker}, + Sources: []string{invalidWorker}, }) c.Assert(err, check.IsNil) c.Assert(resp.Result, check.IsTrue) @@ -978,7 +978,7 @@ func (t *testMaster) TestBreakWorkerDDLLock(c *check.C) { // test BreakWorkerDDLLock with invalid dm-worker[s] resp, err := server.BreakWorkerDDLLock(context.Background(), &pb.BreakWorkerDDLLockRequest{ Task: task, - Workers: []string{"invalid-worker1", "invalid-worker2"}, + Sources: []string{"invalid-worker1", "invalid-worker2"}, RemoveLockID: lockID, SkipDDL: true, }) @@ -994,7 +994,7 @@ func (t *testMaster) TestBreakWorkerDDLLock(c *check.C) { mockBreakDDLLock(true) resp, err = server.BreakWorkerDDLLock(context.Background(), &pb.BreakWorkerDDLLockRequest{ Task: task, - Workers: workers, + Sources: workers, RemoveLockID: lockID, SkipDDL: true, }) @@ -1009,7 +1009,7 @@ func (t *testMaster) TestBreakWorkerDDLLock(c *check.C) { mockBreakDDLLock(false) resp, err = server.BreakWorkerDDLLock(context.Background(), &pb.BreakWorkerDDLLockRequest{ Task: task, - Workers: workers, + Sources: workers, RemoveLockID: lockID, SkipDDL: true, }) @@ -1022,78 +1022,6 @@ func (t *testMaster) TestBreakWorkerDDLLock(c *check.C) { } } -func (t *testMaster) TestRefreshWorkerTasks(c *check.C) { - ctrl := gomock.NewController(c) - defer ctrl.Finish() - - server := testDefaultMasterServer(c) - - workers := make([]string, 0, len(server.cfg.DeployMap)) - - // mock query status, each worker has two valid tasks - for _, deploy := range server.cfg.Deploy { - workers = append(workers, deploy.Worker) - mockWorkerClient := pbmock.NewMockWorkerClient(ctrl) - mockWorkerClient.EXPECT().QueryStatus( - gomock.Any(), - &pb.QueryStatusRequest{}, - ).Return(&pb.QueryStatusResponse{ - Result: true, - Worker: deploy.Worker, - SubTaskStatus: []*pb.SubTaskStatus{ - { - Stage: pb.Stage_Running, - Name: "test", - }, - { - Stage: pb.Stage_Running, - Name: "test2", - }, - { - Stage: pb.Stage_InvalidStage, // this will be ignored - Name: "test3", - }, - }, - }, nil) - server.workerClients[deploy.Worker] = newMockRPCClient(mockWorkerClient) - } - - // test RefreshWorkerTasks, with two running tasks for each workers - resp, err := server.RefreshWorkerTasks(context.Background(), &pb.RefreshWorkerTasksRequest{}) - c.Assert(err, check.IsNil) - c.Assert(resp.Result, check.IsTrue) - c.Assert(len(server.taskWorkers), check.Equals, 2) - for _, taskName := range []string{"test", "test2"} { - c.Assert(server.taskWorkers, check.HasKey, taskName) - c.Assert(server.taskWorkers[taskName], check.DeepEquals, workers) - } - - // mock query status, each worker has no task - for _, deploy := range server.cfg.Deploy { - workers = append(workers, deploy.Worker) - mockWorkerClient := pbmock.NewMockWorkerClient(ctrl) - mockWorkerClient.EXPECT().QueryStatus( - gomock.Any(), - &pb.QueryStatusRequest{}, - ).Return(&pb.QueryStatusResponse{ - Result: true, - Worker: deploy.Worker, - Msg: msgNoSubTask, - }, nil) - server.workerClients[deploy.Worker] = newMockRPCClient(mockWorkerClient) - } - - // test RefreshWorkerTasks, with no started tasks - resp, err = server.RefreshWorkerTasks(context.Background(), &pb.RefreshWorkerTasksRequest{}) - c.Assert(err, check.IsNil) - c.Assert(resp.Result, check.IsTrue) - c.Assert(len(server.taskWorkers), check.Equals, 0) - c.Assert(resp.Workers, check.HasLen, 2) - for _, w := range resp.Workers { - c.Assert(w.Msg, check.Matches, msgNoSubTaskReg) - } -} - func (t *testMaster) TestPurgeWorkerRelay(c *check.C) { ctrl := gomock.NewController(c) defer ctrl.Finish() @@ -1141,7 +1069,7 @@ func (t *testMaster) TestPurgeWorkerRelay(c *check.C) { // test PurgeWorkerRelay with invalid dm-worker[s] resp, err := server.PurgeWorkerRelay(context.Background(), &pb.PurgeWorkerRelayRequest{ - Workers: []string{"invalid-worker1", "invalid-worker2"}, + Sources: []string{"invalid-worker1", "invalid-worker2"}, Time: now, Filename: filename, }) @@ -1156,7 +1084,7 @@ func (t *testMaster) TestPurgeWorkerRelay(c *check.C) { // test PurgeWorkerRelay successfully mockPurgeRelay(true) resp, err = server.PurgeWorkerRelay(context.Background(), &pb.PurgeWorkerRelayRequest{ - Workers: workers, + Sources: workers, Time: now, Filename: filename, }) @@ -1170,7 +1098,7 @@ func (t *testMaster) TestPurgeWorkerRelay(c *check.C) { // test PurgeWorkerRelay with error response mockPurgeRelay(false) resp, err = server.PurgeWorkerRelay(context.Background(), &pb.PurgeWorkerRelayRequest{ - Workers: workers, + Sources: workers, Time: now, Filename: filename, }) @@ -1223,7 +1151,7 @@ func (t *testMaster) TestSwitchWorkerRelayMaster(c *check.C) { // test SwitchWorkerRelayMaster with invalid dm-worker[s] resp, err := server.SwitchWorkerRelayMaster(context.Background(), &pb.SwitchWorkerRelayMasterRequest{ - Workers: []string{"invalid-worker1", "invalid-worker2"}, + Sources: []string{"invalid-worker1", "invalid-worker2"}, }) c.Assert(err, check.IsNil) c.Assert(resp.Result, check.IsTrue) @@ -1236,7 +1164,7 @@ func (t *testMaster) TestSwitchWorkerRelayMaster(c *check.C) { // test SwitchWorkerRelayMaster successfully mockSwitchRelayMaster(true) resp, err = server.SwitchWorkerRelayMaster(context.Background(), &pb.SwitchWorkerRelayMasterRequest{ - Workers: workers, + Sources: workers, }) c.Assert(err, check.IsNil) c.Assert(resp.Result, check.IsTrue) @@ -1248,7 +1176,7 @@ func (t *testMaster) TestSwitchWorkerRelayMaster(c *check.C) { // test SwitchWorkerRelayMaster with error response mockSwitchRelayMaster(false) resp, err = server.SwitchWorkerRelayMaster(context.Background(), &pb.SwitchWorkerRelayMasterRequest{ - Workers: workers, + Sources: workers, }) c.Assert(err, check.IsNil) c.Assert(resp.Result, check.IsTrue) @@ -1300,7 +1228,7 @@ func (t *testMaster) TestOperateWorkerRelayTask(c *check.C) { // test OperateWorkerRelayTask with invalid dm-worker[s] resp, err := server.OperateWorkerRelayTask(context.Background(), &pb.OperateWorkerRelayRequest{ - Workers: []string{"invalid-worker1", "invalid-worker2"}, + Sources: []string{"invalid-worker1", "invalid-worker2"}, Op: pb.RelayOp_PauseRelay, }) c.Assert(err, check.IsNil) @@ -1314,7 +1242,7 @@ func (t *testMaster) TestOperateWorkerRelayTask(c *check.C) { // test OperateWorkerRelayTask successfully mockOperateRelay(true) resp, err = server.OperateWorkerRelayTask(context.Background(), &pb.OperateWorkerRelayRequest{ - Workers: workers, + Sources: workers, Op: pb.RelayOp_PauseRelay, }) c.Assert(err, check.IsNil) @@ -1328,7 +1256,7 @@ func (t *testMaster) TestOperateWorkerRelayTask(c *check.C) { // test OperateWorkerRelayTask with error response mockOperateRelay(false) resp, err = server.OperateWorkerRelayTask(context.Background(), &pb.OperateWorkerRelayRequest{ - Workers: workers, + Sources: workers, Op: pb.RelayOp_PauseRelay, }) c.Assert(err, check.IsNil) @@ -1349,7 +1277,7 @@ func (t *testMaster) TestFetchWorkerDDLInfo(c *check.C) { for _, deploy := range server.cfg.Deploy { workers = append(workers, deploy.Worker) } - server.taskWorkers = map[string][]string{"test": workers} + server.taskSources = map[string][]string{"test": workers} var ( task = "test" schema = "test_db" diff --git a/dm/master/workerrpc/interface.go b/dm/master/workerrpc/interface.go index ee127f2f13..6df3346106 100644 --- a/dm/master/workerrpc/interface.go +++ b/dm/master/workerrpc/interface.go @@ -45,6 +45,7 @@ const ( CmdMigrateRelay CmdFetchDDLInfo + CmdOperateMysqlTask ) // Request wraps all dm-worker rpc requests. @@ -68,6 +69,7 @@ type Request struct { PurgeRelay *pb.PurgeRelayRequest UpdateRelay *pb.UpdateRelayRequest MigrateRelay *pb.MigrateRelayRequest + MysqlTask *pb.MysqlTaskRequest } // Response wraps all dm-worker rpc responses. @@ -91,6 +93,7 @@ type Response struct { PurgeRelay *pb.CommonWorkerResponse UpdateRelay *pb.CommonWorkerResponse MigrateRelay *pb.CommonWorkerResponse + MysqlTask *pb.MysqlTaskResponse FetchDDLInfo pb.Worker_FetchDDLInfoClient } diff --git a/dm/master/workerrpc/rawgrpc.go b/dm/master/workerrpc/rawgrpc.go index 3be5552e80..ccd232028c 100644 --- a/dm/master/workerrpc/rawgrpc.go +++ b/dm/master/workerrpc/rawgrpc.go @@ -115,6 +115,8 @@ func callRPC(ctx context.Context, client pb.WorkerClient, req *Request) (*Respon resp.MigrateRelay, err = client.MigrateRelay(ctx, req.MigrateRelay) case CmdFetchDDLInfo: resp.FetchDDLInfo, err = client.FetchDDLInfo(ctx) + case CmdOperateMysqlTask: + resp.MysqlTask, err = client.OperateMysqlTask(ctx, req.MysqlTask) default: return nil, terror.ErrMasterGRPCInvalidReqType.Generate(req.Type) } diff --git a/dm/pb/dmmaster.pb.go b/dm/pb/dmmaster.pb.go index 1bf52b19a3..4a62fa5090 100644 --- a/dm/pb/dmmaster.pb.go +++ b/dm/pb/dmmaster.pb.go @@ -4,19 +4,17 @@ package pb import ( - "fmt" - - proto "github.com/gogo/protobuf/proto" - + context "context" + fmt "fmt" + io "io" math "math" + math_bits "math/bits" + proto "github.com/gogo/protobuf/proto" _ "google.golang.org/genproto/googleapis/api/annotations" - - context "golang.org/x/net/context" - grpc "google.golang.org/grpc" - - io "io" + codes "google.golang.org/grpc/codes" + status "google.golang.org/grpc/status" ) // Reference imports to suppress errors if they are not otherwise used. @@ -28,7 +26,7 @@ var _ = math.Inf // is compatible with the proto package it is being compiled against. // A compilation error at this line likely means your copy of the // proto package needs to be updated. -const _ = proto.GoGoProtoPackageIsVersion2 // please upgrade the proto package +const _ = proto.GoGoProtoPackageIsVersion3 // please upgrade the proto package type MigrateWorkerRelayRequest struct { BinlogName string `protobuf:"bytes,1,opt,name=BinlogName,proto3" json:"BinlogName,omitempty"` @@ -40,7 +38,7 @@ func (m *MigrateWorkerRelayRequest) Reset() { *m = MigrateWorkerRelayReq func (m *MigrateWorkerRelayRequest) String() string { return proto.CompactTextString(m) } func (*MigrateWorkerRelayRequest) ProtoMessage() {} func (*MigrateWorkerRelayRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_dmmaster_632e41001d9f6d87, []int{0} + return fileDescriptor_f9bef11f2a341f03, []int{0} } func (m *MigrateWorkerRelayRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -50,15 +48,15 @@ func (m *MigrateWorkerRelayRequest) XXX_Marshal(b []byte, deterministic bool) ([ return xxx_messageInfo_MigrateWorkerRelayRequest.Marshal(b, m, deterministic) } else { b = b[:cap(b)] - n, err := m.MarshalTo(b) + n, err := m.MarshalToSizedBuffer(b) if err != nil { return nil, err } return b[:n], nil } } -func (dst *MigrateWorkerRelayRequest) XXX_Merge(src proto.Message) { - xxx_messageInfo_MigrateWorkerRelayRequest.Merge(dst, src) +func (m *MigrateWorkerRelayRequest) XXX_Merge(src proto.Message) { + xxx_messageInfo_MigrateWorkerRelayRequest.Merge(m, src) } func (m *MigrateWorkerRelayRequest) XXX_Size() int { return m.Size() @@ -99,7 +97,7 @@ func (m *UpdateWorkerRelayConfigRequest) Reset() { *m = UpdateWorkerRela func (m *UpdateWorkerRelayConfigRequest) String() string { return proto.CompactTextString(m) } func (*UpdateWorkerRelayConfigRequest) ProtoMessage() {} func (*UpdateWorkerRelayConfigRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_dmmaster_632e41001d9f6d87, []int{1} + return fileDescriptor_f9bef11f2a341f03, []int{1} } func (m *UpdateWorkerRelayConfigRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -109,15 +107,15 @@ func (m *UpdateWorkerRelayConfigRequest) XXX_Marshal(b []byte, deterministic boo return xxx_messageInfo_UpdateWorkerRelayConfigRequest.Marshal(b, m, deterministic) } else { b = b[:cap(b)] - n, err := m.MarshalTo(b) + n, err := m.MarshalToSizedBuffer(b) if err != nil { return nil, err } return b[:n], nil } } -func (dst *UpdateWorkerRelayConfigRequest) XXX_Merge(src proto.Message) { - xxx_messageInfo_UpdateWorkerRelayConfigRequest.Merge(dst, src) +func (m *UpdateWorkerRelayConfigRequest) XXX_Merge(src proto.Message) { + xxx_messageInfo_UpdateWorkerRelayConfigRequest.Merge(m, src) } func (m *UpdateWorkerRelayConfigRequest) XXX_Size() int { return m.Size() @@ -144,14 +142,14 @@ func (m *UpdateWorkerRelayConfigRequest) GetWorker() string { type StartTaskRequest struct { Task string `protobuf:"bytes,1,opt,name=task,proto3" json:"task,omitempty"` - Workers []string `protobuf:"bytes,2,rep,name=workers" json:"workers,omitempty"` + Sources []string `protobuf:"bytes,2,rep,name=sources,proto3" json:"sources,omitempty"` } func (m *StartTaskRequest) Reset() { *m = StartTaskRequest{} } func (m *StartTaskRequest) String() string { return proto.CompactTextString(m) } func (*StartTaskRequest) ProtoMessage() {} func (*StartTaskRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_dmmaster_632e41001d9f6d87, []int{2} + return fileDescriptor_f9bef11f2a341f03, []int{2} } func (m *StartTaskRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -161,15 +159,15 @@ func (m *StartTaskRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, er return xxx_messageInfo_StartTaskRequest.Marshal(b, m, deterministic) } else { b = b[:cap(b)] - n, err := m.MarshalTo(b) + n, err := m.MarshalToSizedBuffer(b) if err != nil { return nil, err } return b[:n], nil } } -func (dst *StartTaskRequest) XXX_Merge(src proto.Message) { - xxx_messageInfo_StartTaskRequest.Merge(dst, src) +func (m *StartTaskRequest) XXX_Merge(src proto.Message) { + xxx_messageInfo_StartTaskRequest.Merge(m, src) } func (m *StartTaskRequest) XXX_Size() int { return m.Size() @@ -187,9 +185,9 @@ func (m *StartTaskRequest) GetTask() string { return "" } -func (m *StartTaskRequest) GetWorkers() []string { +func (m *StartTaskRequest) GetSources() []string { if m != nil { - return m.Workers + return m.Sources } return nil } @@ -197,14 +195,14 @@ func (m *StartTaskRequest) GetWorkers() []string { type StartTaskResponse 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" json:"workers,omitempty"` + Workers []*CommonWorkerResponse `protobuf:"bytes,3,rep,name=workers,proto3" json:"workers,omitempty"` } func (m *StartTaskResponse) Reset() { *m = StartTaskResponse{} } func (m *StartTaskResponse) String() string { return proto.CompactTextString(m) } func (*StartTaskResponse) ProtoMessage() {} func (*StartTaskResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_dmmaster_632e41001d9f6d87, []int{3} + return fileDescriptor_f9bef11f2a341f03, []int{3} } func (m *StartTaskResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -214,15 +212,15 @@ func (m *StartTaskResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, e return xxx_messageInfo_StartTaskResponse.Marshal(b, m, deterministic) } else { b = b[:cap(b)] - n, err := m.MarshalTo(b) + n, err := m.MarshalToSizedBuffer(b) if err != nil { return nil, err } return b[:n], nil } } -func (dst *StartTaskResponse) XXX_Merge(src proto.Message) { - xxx_messageInfo_StartTaskResponse.Merge(dst, src) +func (m *StartTaskResponse) XXX_Merge(src proto.Message) { + xxx_messageInfo_StartTaskResponse.Merge(m, src) } func (m *StartTaskResponse) XXX_Size() int { return m.Size() @@ -262,7 +260,7 @@ func (m *UpdateMasterConfigRequest) Reset() { *m = UpdateMasterConfigReq func (m *UpdateMasterConfigRequest) String() string { return proto.CompactTextString(m) } func (*UpdateMasterConfigRequest) ProtoMessage() {} func (*UpdateMasterConfigRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_dmmaster_632e41001d9f6d87, []int{4} + return fileDescriptor_f9bef11f2a341f03, []int{4} } func (m *UpdateMasterConfigRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -272,15 +270,15 @@ func (m *UpdateMasterConfigRequest) XXX_Marshal(b []byte, deterministic bool) ([ return xxx_messageInfo_UpdateMasterConfigRequest.Marshal(b, m, deterministic) } else { b = b[:cap(b)] - n, err := m.MarshalTo(b) + n, err := m.MarshalToSizedBuffer(b) if err != nil { return nil, err } return b[:n], nil } } -func (dst *UpdateMasterConfigRequest) XXX_Merge(src proto.Message) { - xxx_messageInfo_UpdateMasterConfigRequest.Merge(dst, src) +func (m *UpdateMasterConfigRequest) XXX_Merge(src proto.Message) { + xxx_messageInfo_UpdateMasterConfigRequest.Merge(m, src) } func (m *UpdateMasterConfigRequest) XXX_Size() int { return m.Size() @@ -301,14 +299,14 @@ func (m *UpdateMasterConfigRequest) GetConfig() string { type UpdateMasterConfigResponse 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 []*QueryStatusResponse `protobuf:"bytes,3,rep,name=workers" json:"workers,omitempty"` + Workers []*QueryStatusResponse `protobuf:"bytes,3,rep,name=workers,proto3" json:"workers,omitempty"` } func (m *UpdateMasterConfigResponse) Reset() { *m = UpdateMasterConfigResponse{} } func (m *UpdateMasterConfigResponse) String() string { return proto.CompactTextString(m) } func (*UpdateMasterConfigResponse) ProtoMessage() {} func (*UpdateMasterConfigResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_dmmaster_632e41001d9f6d87, []int{5} + return fileDescriptor_f9bef11f2a341f03, []int{5} } func (m *UpdateMasterConfigResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -318,15 +316,15 @@ func (m *UpdateMasterConfigResponse) XXX_Marshal(b []byte, deterministic bool) ( return xxx_messageInfo_UpdateMasterConfigResponse.Marshal(b, m, deterministic) } else { b = b[:cap(b)] - n, err := m.MarshalTo(b) + n, err := m.MarshalToSizedBuffer(b) if err != nil { return nil, err } return b[:n], nil } } -func (dst *UpdateMasterConfigResponse) XXX_Merge(src proto.Message) { - xxx_messageInfo_UpdateMasterConfigResponse.Merge(dst, src) +func (m *UpdateMasterConfigResponse) XXX_Merge(src proto.Message) { + xxx_messageInfo_UpdateMasterConfigResponse.Merge(m, src) } func (m *UpdateMasterConfigResponse) XXX_Size() int { return m.Size() @@ -361,14 +359,14 @@ func (m *UpdateMasterConfigResponse) GetWorkers() []*QueryStatusResponse { type OperateTaskRequest struct { Op TaskOp `protobuf:"varint,1,opt,name=op,proto3,enum=pb.TaskOp" json:"op,omitempty"` Name string `protobuf:"bytes,2,opt,name=name,proto3" json:"name,omitempty"` - Workers []string `protobuf:"bytes,3,rep,name=workers" json:"workers,omitempty"` + Sources []string `protobuf:"bytes,3,rep,name=sources,proto3" json:"sources,omitempty"` } func (m *OperateTaskRequest) Reset() { *m = OperateTaskRequest{} } func (m *OperateTaskRequest) String() string { return proto.CompactTextString(m) } func (*OperateTaskRequest) ProtoMessage() {} func (*OperateTaskRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_dmmaster_632e41001d9f6d87, []int{6} + return fileDescriptor_f9bef11f2a341f03, []int{6} } func (m *OperateTaskRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -378,15 +376,15 @@ func (m *OperateTaskRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, return xxx_messageInfo_OperateTaskRequest.Marshal(b, m, deterministic) } else { b = b[:cap(b)] - n, err := m.MarshalTo(b) + n, err := m.MarshalToSizedBuffer(b) if err != nil { return nil, err } return b[:n], nil } } -func (dst *OperateTaskRequest) XXX_Merge(src proto.Message) { - xxx_messageInfo_OperateTaskRequest.Merge(dst, src) +func (m *OperateTaskRequest) XXX_Merge(src proto.Message) { + xxx_messageInfo_OperateTaskRequest.Merge(m, src) } func (m *OperateTaskRequest) XXX_Size() int { return m.Size() @@ -411,9 +409,9 @@ func (m *OperateTaskRequest) GetName() string { return "" } -func (m *OperateTaskRequest) GetWorkers() []string { +func (m *OperateTaskRequest) GetSources() []string { if m != nil { - return m.Workers + return m.Sources } return nil } @@ -422,14 +420,14 @@ type OperateTaskResponse struct { Op TaskOp `protobuf:"varint,1,opt,name=op,proto3,enum=pb.TaskOp" json:"op,omitempty"` Result bool `protobuf:"varint,2,opt,name=result,proto3" json:"result,omitempty"` Msg string `protobuf:"bytes,3,opt,name=msg,proto3" json:"msg,omitempty"` - Workers []*OperateSubTaskResponse `protobuf:"bytes,4,rep,name=workers" json:"workers,omitempty"` + Workers []*OperateSubTaskResponse `protobuf:"bytes,4,rep,name=workers,proto3" json:"workers,omitempty"` } func (m *OperateTaskResponse) Reset() { *m = OperateTaskResponse{} } func (m *OperateTaskResponse) String() string { return proto.CompactTextString(m) } func (*OperateTaskResponse) ProtoMessage() {} func (*OperateTaskResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_dmmaster_632e41001d9f6d87, []int{7} + return fileDescriptor_f9bef11f2a341f03, []int{7} } func (m *OperateTaskResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -439,15 +437,15 @@ func (m *OperateTaskResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, return xxx_messageInfo_OperateTaskResponse.Marshal(b, m, deterministic) } else { b = b[:cap(b)] - n, err := m.MarshalTo(b) + n, err := m.MarshalToSizedBuffer(b) if err != nil { return nil, err } return b[:n], nil } } -func (dst *OperateTaskResponse) XXX_Merge(src proto.Message) { - xxx_messageInfo_OperateTaskResponse.Merge(dst, src) +func (m *OperateTaskResponse) XXX_Merge(src proto.Message) { + xxx_messageInfo_OperateTaskResponse.Merge(m, src) } func (m *OperateTaskResponse) XXX_Size() int { return m.Size() @@ -493,14 +491,14 @@ func (m *OperateTaskResponse) GetWorkers() []*OperateSubTaskResponse { // workers need to do update, empty for all workers in processing the task type UpdateTaskRequest struct { Task string `protobuf:"bytes,1,opt,name=task,proto3" json:"task,omitempty"` - Workers []string `protobuf:"bytes,2,rep,name=workers" json:"workers,omitempty"` + Sources []string `protobuf:"bytes,2,rep,name=sources,proto3" json:"sources,omitempty"` } func (m *UpdateTaskRequest) Reset() { *m = UpdateTaskRequest{} } func (m *UpdateTaskRequest) String() string { return proto.CompactTextString(m) } func (*UpdateTaskRequest) ProtoMessage() {} func (*UpdateTaskRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_dmmaster_632e41001d9f6d87, []int{8} + return fileDescriptor_f9bef11f2a341f03, []int{8} } func (m *UpdateTaskRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -510,15 +508,15 @@ func (m *UpdateTaskRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, e return xxx_messageInfo_UpdateTaskRequest.Marshal(b, m, deterministic) } else { b = b[:cap(b)] - n, err := m.MarshalTo(b) + n, err := m.MarshalToSizedBuffer(b) if err != nil { return nil, err } return b[:n], nil } } -func (dst *UpdateTaskRequest) XXX_Merge(src proto.Message) { - xxx_messageInfo_UpdateTaskRequest.Merge(dst, src) +func (m *UpdateTaskRequest) XXX_Merge(src proto.Message) { + xxx_messageInfo_UpdateTaskRequest.Merge(m, src) } func (m *UpdateTaskRequest) XXX_Size() int { return m.Size() @@ -536,9 +534,9 @@ func (m *UpdateTaskRequest) GetTask() string { return "" } -func (m *UpdateTaskRequest) GetWorkers() []string { +func (m *UpdateTaskRequest) GetSources() []string { if m != nil { - return m.Workers + return m.Sources } return nil } @@ -546,14 +544,14 @@ func (m *UpdateTaskRequest) GetWorkers() []string { type UpdateTaskResponse 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" json:"workers,omitempty"` + Workers []*CommonWorkerResponse `protobuf:"bytes,3,rep,name=workers,proto3" json:"workers,omitempty"` } func (m *UpdateTaskResponse) Reset() { *m = UpdateTaskResponse{} } func (m *UpdateTaskResponse) String() string { return proto.CompactTextString(m) } func (*UpdateTaskResponse) ProtoMessage() {} func (*UpdateTaskResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_dmmaster_632e41001d9f6d87, []int{9} + return fileDescriptor_f9bef11f2a341f03, []int{9} } func (m *UpdateTaskResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -563,15 +561,15 @@ func (m *UpdateTaskResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, return xxx_messageInfo_UpdateTaskResponse.Marshal(b, m, deterministic) } else { b = b[:cap(b)] - n, err := m.MarshalTo(b) + n, err := m.MarshalToSizedBuffer(b) if err != nil { return nil, err } return b[:n], nil } } -func (dst *UpdateTaskResponse) XXX_Merge(src proto.Message) { - xxx_messageInfo_UpdateTaskResponse.Merge(dst, src) +func (m *UpdateTaskResponse) XXX_Merge(src proto.Message) { + xxx_messageInfo_UpdateTaskResponse.Merge(m, src) } func (m *UpdateTaskResponse) XXX_Size() int { return m.Size() @@ -605,14 +603,14 @@ func (m *UpdateTaskResponse) GetWorkers() []*CommonWorkerResponse { type QueryStatusListRequest struct { Name string `protobuf:"bytes,1,opt,name=name,proto3" json:"name,omitempty"` - Workers []string `protobuf:"bytes,2,rep,name=workers" json:"workers,omitempty"` + Sources []string `protobuf:"bytes,2,rep,name=sources,proto3" json:"sources,omitempty"` } func (m *QueryStatusListRequest) Reset() { *m = QueryStatusListRequest{} } func (m *QueryStatusListRequest) String() string { return proto.CompactTextString(m) } func (*QueryStatusListRequest) ProtoMessage() {} func (*QueryStatusListRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_dmmaster_632e41001d9f6d87, []int{10} + return fileDescriptor_f9bef11f2a341f03, []int{10} } func (m *QueryStatusListRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -622,15 +620,15 @@ func (m *QueryStatusListRequest) XXX_Marshal(b []byte, deterministic bool) ([]by return xxx_messageInfo_QueryStatusListRequest.Marshal(b, m, deterministic) } else { b = b[:cap(b)] - n, err := m.MarshalTo(b) + n, err := m.MarshalToSizedBuffer(b) if err != nil { return nil, err } return b[:n], nil } } -func (dst *QueryStatusListRequest) XXX_Merge(src proto.Message) { - xxx_messageInfo_QueryStatusListRequest.Merge(dst, src) +func (m *QueryStatusListRequest) XXX_Merge(src proto.Message) { + xxx_messageInfo_QueryStatusListRequest.Merge(m, src) } func (m *QueryStatusListRequest) XXX_Size() int { return m.Size() @@ -648,9 +646,9 @@ func (m *QueryStatusListRequest) GetName() string { return "" } -func (m *QueryStatusListRequest) GetWorkers() []string { +func (m *QueryStatusListRequest) GetSources() []string { if m != nil { - return m.Workers + return m.Sources } return nil } @@ -658,14 +656,14 @@ func (m *QueryStatusListRequest) GetWorkers() []string { type QueryStatusListResponse 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 []*QueryStatusResponse `protobuf:"bytes,3,rep,name=workers" json:"workers,omitempty"` + Workers []*QueryStatusResponse `protobuf:"bytes,3,rep,name=workers,proto3" json:"workers,omitempty"` } func (m *QueryStatusListResponse) Reset() { *m = QueryStatusListResponse{} } func (m *QueryStatusListResponse) String() string { return proto.CompactTextString(m) } func (*QueryStatusListResponse) ProtoMessage() {} func (*QueryStatusListResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_dmmaster_632e41001d9f6d87, []int{11} + return fileDescriptor_f9bef11f2a341f03, []int{11} } func (m *QueryStatusListResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -675,15 +673,15 @@ func (m *QueryStatusListResponse) XXX_Marshal(b []byte, deterministic bool) ([]b return xxx_messageInfo_QueryStatusListResponse.Marshal(b, m, deterministic) } else { b = b[:cap(b)] - n, err := m.MarshalTo(b) + n, err := m.MarshalToSizedBuffer(b) if err != nil { return nil, err } return b[:n], nil } } -func (dst *QueryStatusListResponse) XXX_Merge(src proto.Message) { - xxx_messageInfo_QueryStatusListResponse.Merge(dst, src) +func (m *QueryStatusListResponse) XXX_Merge(src proto.Message) { + xxx_messageInfo_QueryStatusListResponse.Merge(m, src) } func (m *QueryStatusListResponse) XXX_Size() int { return m.Size() @@ -717,14 +715,14 @@ func (m *QueryStatusListResponse) GetWorkers() []*QueryStatusResponse { type QueryErrorListRequest struct { Name string `protobuf:"bytes,1,opt,name=name,proto3" json:"name,omitempty"` - Workers []string `protobuf:"bytes,2,rep,name=workers" json:"workers,omitempty"` + Sources []string `protobuf:"bytes,2,rep,name=sources,proto3" json:"sources,omitempty"` } func (m *QueryErrorListRequest) Reset() { *m = QueryErrorListRequest{} } func (m *QueryErrorListRequest) String() string { return proto.CompactTextString(m) } func (*QueryErrorListRequest) ProtoMessage() {} func (*QueryErrorListRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_dmmaster_632e41001d9f6d87, []int{12} + return fileDescriptor_f9bef11f2a341f03, []int{12} } func (m *QueryErrorListRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -734,15 +732,15 @@ func (m *QueryErrorListRequest) XXX_Marshal(b []byte, deterministic bool) ([]byt return xxx_messageInfo_QueryErrorListRequest.Marshal(b, m, deterministic) } else { b = b[:cap(b)] - n, err := m.MarshalTo(b) + n, err := m.MarshalToSizedBuffer(b) if err != nil { return nil, err } return b[:n], nil } } -func (dst *QueryErrorListRequest) XXX_Merge(src proto.Message) { - xxx_messageInfo_QueryErrorListRequest.Merge(dst, src) +func (m *QueryErrorListRequest) XXX_Merge(src proto.Message) { + xxx_messageInfo_QueryErrorListRequest.Merge(m, src) } func (m *QueryErrorListRequest) XXX_Size() int { return m.Size() @@ -760,9 +758,9 @@ func (m *QueryErrorListRequest) GetName() string { return "" } -func (m *QueryErrorListRequest) GetWorkers() []string { +func (m *QueryErrorListRequest) GetSources() []string { if m != nil { - return m.Workers + return m.Sources } return nil } @@ -770,14 +768,14 @@ func (m *QueryErrorListRequest) GetWorkers() []string { type QueryErrorListResponse 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 []*QueryErrorResponse `protobuf:"bytes,3,rep,name=workers" json:"workers,omitempty"` + Workers []*QueryErrorResponse `protobuf:"bytes,3,rep,name=workers,proto3" json:"workers,omitempty"` } func (m *QueryErrorListResponse) Reset() { *m = QueryErrorListResponse{} } func (m *QueryErrorListResponse) String() string { return proto.CompactTextString(m) } func (*QueryErrorListResponse) ProtoMessage() {} func (*QueryErrorListResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_dmmaster_632e41001d9f6d87, []int{13} + return fileDescriptor_f9bef11f2a341f03, []int{13} } func (m *QueryErrorListResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -787,15 +785,15 @@ func (m *QueryErrorListResponse) XXX_Marshal(b []byte, deterministic bool) ([]by return xxx_messageInfo_QueryErrorListResponse.Marshal(b, m, deterministic) } else { b = b[:cap(b)] - n, err := m.MarshalTo(b) + n, err := m.MarshalToSizedBuffer(b) if err != nil { return nil, err } return b[:n], nil } } -func (dst *QueryErrorListResponse) XXX_Merge(src proto.Message) { - xxx_messageInfo_QueryErrorListResponse.Merge(dst, src) +func (m *QueryErrorListResponse) XXX_Merge(src proto.Message) { + xxx_messageInfo_QueryErrorListResponse.Merge(m, src) } func (m *QueryErrorListResponse) XXX_Size() int { return m.Size() @@ -834,14 +832,14 @@ func (m *QueryErrorListResponse) GetWorkers() []*QueryErrorResponse { // if specify task and workers both, and workers not doing the task , it will return empty DDL locks type ShowDDLLocksRequest struct { Task string `protobuf:"bytes,1,opt,name=task,proto3" json:"task,omitempty"` - Workers []string `protobuf:"bytes,2,rep,name=workers" json:"workers,omitempty"` + Sources []string `protobuf:"bytes,2,rep,name=sources,proto3" json:"sources,omitempty"` } func (m *ShowDDLLocksRequest) Reset() { *m = ShowDDLLocksRequest{} } func (m *ShowDDLLocksRequest) String() string { return proto.CompactTextString(m) } func (*ShowDDLLocksRequest) ProtoMessage() {} func (*ShowDDLLocksRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_dmmaster_632e41001d9f6d87, []int{14} + return fileDescriptor_f9bef11f2a341f03, []int{14} } func (m *ShowDDLLocksRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -851,15 +849,15 @@ func (m *ShowDDLLocksRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, return xxx_messageInfo_ShowDDLLocksRequest.Marshal(b, m, deterministic) } else { b = b[:cap(b)] - n, err := m.MarshalTo(b) + n, err := m.MarshalToSizedBuffer(b) if err != nil { return nil, err } return b[:n], nil } } -func (dst *ShowDDLLocksRequest) XXX_Merge(src proto.Message) { - xxx_messageInfo_ShowDDLLocksRequest.Merge(dst, src) +func (m *ShowDDLLocksRequest) XXX_Merge(src proto.Message) { + xxx_messageInfo_ShowDDLLocksRequest.Merge(m, src) } func (m *ShowDDLLocksRequest) XXX_Size() int { return m.Size() @@ -877,9 +875,9 @@ func (m *ShowDDLLocksRequest) GetTask() string { return "" } -func (m *ShowDDLLocksRequest) GetWorkers() []string { +func (m *ShowDDLLocksRequest) GetSources() []string { if m != nil { - return m.Workers + return m.Sources } return nil } @@ -896,16 +894,16 @@ type DDLLock struct { ID string `protobuf:"bytes,1,opt,name=ID,proto3" json:"ID,omitempty"` Task string `protobuf:"bytes,2,opt,name=task,proto3" json:"task,omitempty"` Owner string `protobuf:"bytes,3,opt,name=owner,proto3" json:"owner,omitempty"` - DDLs []string `protobuf:"bytes,4,rep,name=DDLs" json:"DDLs,omitempty"` - Synced []string `protobuf:"bytes,5,rep,name=synced" json:"synced,omitempty"` - Unsynced []string `protobuf:"bytes,6,rep,name=unsynced" json:"unsynced,omitempty"` + DDLs []string `protobuf:"bytes,4,rep,name=DDLs,proto3" json:"DDLs,omitempty"` + Synced []string `protobuf:"bytes,5,rep,name=synced,proto3" json:"synced,omitempty"` + Unsynced []string `protobuf:"bytes,6,rep,name=unsynced,proto3" json:"unsynced,omitempty"` } func (m *DDLLock) Reset() { *m = DDLLock{} } func (m *DDLLock) String() string { return proto.CompactTextString(m) } func (*DDLLock) ProtoMessage() {} func (*DDLLock) Descriptor() ([]byte, []int) { - return fileDescriptor_dmmaster_632e41001d9f6d87, []int{15} + return fileDescriptor_f9bef11f2a341f03, []int{15} } func (m *DDLLock) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -915,15 +913,15 @@ func (m *DDLLock) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { return xxx_messageInfo_DDLLock.Marshal(b, m, deterministic) } else { b = b[:cap(b)] - n, err := m.MarshalTo(b) + n, err := m.MarshalToSizedBuffer(b) if err != nil { return nil, err } return b[:n], nil } } -func (dst *DDLLock) XXX_Merge(src proto.Message) { - xxx_messageInfo_DDLLock.Merge(dst, src) +func (m *DDLLock) XXX_Merge(src proto.Message) { + xxx_messageInfo_DDLLock.Merge(m, src) } func (m *DDLLock) XXX_Size() int { return m.Size() @@ -979,14 +977,14 @@ func (m *DDLLock) GetUnsynced() []string { type ShowDDLLocksResponse 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"` - Locks []*DDLLock `protobuf:"bytes,3,rep,name=locks" json:"locks,omitempty"` + Locks []*DDLLock `protobuf:"bytes,3,rep,name=locks,proto3" json:"locks,omitempty"` } func (m *ShowDDLLocksResponse) Reset() { *m = ShowDDLLocksResponse{} } func (m *ShowDDLLocksResponse) String() string { return proto.CompactTextString(m) } func (*ShowDDLLocksResponse) ProtoMessage() {} func (*ShowDDLLocksResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_dmmaster_632e41001d9f6d87, []int{16} + return fileDescriptor_f9bef11f2a341f03, []int{16} } func (m *ShowDDLLocksResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -996,15 +994,15 @@ func (m *ShowDDLLocksResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte return xxx_messageInfo_ShowDDLLocksResponse.Marshal(b, m, deterministic) } else { b = b[:cap(b)] - n, err := m.MarshalTo(b) + n, err := m.MarshalToSizedBuffer(b) if err != nil { return nil, err } return b[:n], nil } } -func (dst *ShowDDLLocksResponse) XXX_Merge(src proto.Message) { - xxx_messageInfo_ShowDDLLocksResponse.Merge(dst, src) +func (m *ShowDDLLocksResponse) XXX_Merge(src proto.Message) { + xxx_messageInfo_ShowDDLLocksResponse.Merge(m, src) } func (m *ShowDDLLocksResponse) XXX_Size() int { return m.Size() @@ -1045,7 +1043,7 @@ func (m *ShowDDLLocksResponse) GetLocks() []*DDLLock { 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"` - Workers []string `protobuf:"bytes,3,rep,name=workers" json:"workers,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"` } @@ -1053,7 +1051,7 @@ func (m *UnlockDDLLockRequest) Reset() { *m = UnlockDDLLockRequest{} } func (m *UnlockDDLLockRequest) String() string { return proto.CompactTextString(m) } func (*UnlockDDLLockRequest) ProtoMessage() {} func (*UnlockDDLLockRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_dmmaster_632e41001d9f6d87, []int{17} + return fileDescriptor_f9bef11f2a341f03, []int{17} } func (m *UnlockDDLLockRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1063,15 +1061,15 @@ func (m *UnlockDDLLockRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte return xxx_messageInfo_UnlockDDLLockRequest.Marshal(b, m, deterministic) } else { b = b[:cap(b)] - n, err := m.MarshalTo(b) + n, err := m.MarshalToSizedBuffer(b) if err != nil { return nil, err } return b[:n], nil } } -func (dst *UnlockDDLLockRequest) XXX_Merge(src proto.Message) { - xxx_messageInfo_UnlockDDLLockRequest.Merge(dst, src) +func (m *UnlockDDLLockRequest) XXX_Merge(src proto.Message) { + xxx_messageInfo_UnlockDDLLockRequest.Merge(m, src) } func (m *UnlockDDLLockRequest) XXX_Size() int { return m.Size() @@ -1096,9 +1094,9 @@ func (m *UnlockDDLLockRequest) GetReplaceOwner() string { return "" } -func (m *UnlockDDLLockRequest) GetWorkers() []string { +func (m *UnlockDDLLockRequest) GetSources() []string { if m != nil { - return m.Workers + return m.Sources } return nil } @@ -1113,14 +1111,14 @@ 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" json:"workers,omitempty"` + Workers []*CommonWorkerResponse `protobuf:"bytes,3,rep,name=workers,proto3" json:"workers,omitempty"` } func (m *UnlockDDLLockResponse) Reset() { *m = UnlockDDLLockResponse{} } func (m *UnlockDDLLockResponse) String() string { return proto.CompactTextString(m) } func (*UnlockDDLLockResponse) ProtoMessage() {} func (*UnlockDDLLockResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_dmmaster_632e41001d9f6d87, []int{18} + return fileDescriptor_f9bef11f2a341f03, []int{18} } func (m *UnlockDDLLockResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1130,15 +1128,15 @@ func (m *UnlockDDLLockResponse) XXX_Marshal(b []byte, deterministic bool) ([]byt return xxx_messageInfo_UnlockDDLLockResponse.Marshal(b, m, deterministic) } else { b = b[:cap(b)] - n, err := m.MarshalTo(b) + n, err := m.MarshalToSizedBuffer(b) if err != nil { return nil, err } return b[:n], nil } } -func (dst *UnlockDDLLockResponse) XXX_Merge(src proto.Message) { - xxx_messageInfo_UnlockDDLLockResponse.Merge(dst, src) +func (m *UnlockDDLLockResponse) XXX_Merge(src proto.Message) { + xxx_messageInfo_UnlockDDLLockResponse.Merge(m, src) } func (m *UnlockDDLLockResponse) XXX_Size() int { return m.Size() @@ -1178,7 +1176,7 @@ func (m *UnlockDDLLockResponse) GetWorkers() []*CommonWorkerResponse { // skipDDL: skip DDL which is blocking // execDDL and skipDDL can not specify both at the same time, but can specify neither type BreakWorkerDDLLockRequest struct { - Workers []string `protobuf:"bytes,1,rep,name=workers" json:"workers,omitempty"` + 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"` @@ -1189,7 +1187,7 @@ func (m *BreakWorkerDDLLockRequest) Reset() { *m = BreakWorkerDDLLockReq func (m *BreakWorkerDDLLockRequest) String() string { return proto.CompactTextString(m) } func (*BreakWorkerDDLLockRequest) ProtoMessage() {} func (*BreakWorkerDDLLockRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_dmmaster_632e41001d9f6d87, []int{19} + return fileDescriptor_f9bef11f2a341f03, []int{19} } func (m *BreakWorkerDDLLockRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1199,15 +1197,15 @@ func (m *BreakWorkerDDLLockRequest) XXX_Marshal(b []byte, deterministic bool) ([ return xxx_messageInfo_BreakWorkerDDLLockRequest.Marshal(b, m, deterministic) } else { b = b[:cap(b)] - n, err := m.MarshalTo(b) + n, err := m.MarshalToSizedBuffer(b) if err != nil { return nil, err } return b[:n], nil } } -func (dst *BreakWorkerDDLLockRequest) XXX_Merge(src proto.Message) { - xxx_messageInfo_BreakWorkerDDLLockRequest.Merge(dst, src) +func (m *BreakWorkerDDLLockRequest) XXX_Merge(src proto.Message) { + xxx_messageInfo_BreakWorkerDDLLockRequest.Merge(m, src) } func (m *BreakWorkerDDLLockRequest) XXX_Size() int { return m.Size() @@ -1218,9 +1216,9 @@ func (m *BreakWorkerDDLLockRequest) XXX_DiscardUnknown() { var xxx_messageInfo_BreakWorkerDDLLockRequest proto.InternalMessageInfo -func (m *BreakWorkerDDLLockRequest) GetWorkers() []string { +func (m *BreakWorkerDDLLockRequest) GetSources() []string { if m != nil { - return m.Workers + return m.Sources } return nil } @@ -1256,14 +1254,14 @@ func (m *BreakWorkerDDLLockRequest) GetSkipDDL() bool { 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"` - Workers []*CommonWorkerResponse `protobuf:"bytes,3,rep,name=workers" json:"workers,omitempty"` + Workers []*CommonWorkerResponse `protobuf:"bytes,3,rep,name=workers,proto3" json:"workers,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_dmmaster_632e41001d9f6d87, []int{20} + return fileDescriptor_f9bef11f2a341f03, []int{20} } func (m *BreakWorkerDDLLockResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1273,15 +1271,15 @@ func (m *BreakWorkerDDLLockResponse) XXX_Marshal(b []byte, deterministic bool) ( return xxx_messageInfo_BreakWorkerDDLLockResponse.Marshal(b, m, deterministic) } else { b = b[:cap(b)] - n, err := m.MarshalTo(b) + n, err := m.MarshalToSizedBuffer(b) if err != nil { return nil, err } return b[:n], nil } } -func (dst *BreakWorkerDDLLockResponse) XXX_Merge(src proto.Message) { - xxx_messageInfo_BreakWorkerDDLLockResponse.Merge(dst, src) +func (m *BreakWorkerDDLLockResponse) XXX_Merge(src proto.Message) { + xxx_messageInfo_BreakWorkerDDLLockResponse.Merge(m, src) } func (m *BreakWorkerDDLLockResponse) XXX_Size() int { return m.Size() @@ -1316,14 +1314,14 @@ func (m *BreakWorkerDDLLockResponse) GetWorkers() []*CommonWorkerResponse { // 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 { - Workers []string `protobuf:"bytes,1,rep,name=workers" json:"workers,omitempty"` + Sources []string `protobuf:"bytes,1,rep,name=sources,proto3" json:"sources,omitempty"` } func (m *SwitchWorkerRelayMasterRequest) Reset() { *m = SwitchWorkerRelayMasterRequest{} } func (m *SwitchWorkerRelayMasterRequest) String() string { return proto.CompactTextString(m) } func (*SwitchWorkerRelayMasterRequest) ProtoMessage() {} func (*SwitchWorkerRelayMasterRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_dmmaster_632e41001d9f6d87, []int{21} + return fileDescriptor_f9bef11f2a341f03, []int{21} } func (m *SwitchWorkerRelayMasterRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1333,15 +1331,15 @@ func (m *SwitchWorkerRelayMasterRequest) XXX_Marshal(b []byte, deterministic boo return xxx_messageInfo_SwitchWorkerRelayMasterRequest.Marshal(b, m, deterministic) } else { b = b[:cap(b)] - n, err := m.MarshalTo(b) + n, err := m.MarshalToSizedBuffer(b) if err != nil { return nil, err } return b[:n], nil } } -func (dst *SwitchWorkerRelayMasterRequest) XXX_Merge(src proto.Message) { - xxx_messageInfo_SwitchWorkerRelayMasterRequest.Merge(dst, src) +func (m *SwitchWorkerRelayMasterRequest) XXX_Merge(src proto.Message) { + xxx_messageInfo_SwitchWorkerRelayMasterRequest.Merge(m, src) } func (m *SwitchWorkerRelayMasterRequest) XXX_Size() int { return m.Size() @@ -1352,9 +1350,9 @@ func (m *SwitchWorkerRelayMasterRequest) XXX_DiscardUnknown() { var xxx_messageInfo_SwitchWorkerRelayMasterRequest proto.InternalMessageInfo -func (m *SwitchWorkerRelayMasterRequest) GetWorkers() []string { +func (m *SwitchWorkerRelayMasterRequest) GetSources() []string { if m != nil { - return m.Workers + return m.Sources } return nil } @@ -1362,14 +1360,14 @@ func (m *SwitchWorkerRelayMasterRequest) GetWorkers() []string { type SwitchWorkerRelayMasterResponse 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" json:"workers,omitempty"` + Workers []*CommonWorkerResponse `protobuf:"bytes,3,rep,name=workers,proto3" json:"workers,omitempty"` } func (m *SwitchWorkerRelayMasterResponse) Reset() { *m = SwitchWorkerRelayMasterResponse{} } func (m *SwitchWorkerRelayMasterResponse) String() string { return proto.CompactTextString(m) } func (*SwitchWorkerRelayMasterResponse) ProtoMessage() {} func (*SwitchWorkerRelayMasterResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_dmmaster_632e41001d9f6d87, []int{22} + return fileDescriptor_f9bef11f2a341f03, []int{22} } func (m *SwitchWorkerRelayMasterResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1379,15 +1377,15 @@ func (m *SwitchWorkerRelayMasterResponse) XXX_Marshal(b []byte, deterministic bo return xxx_messageInfo_SwitchWorkerRelayMasterResponse.Marshal(b, m, deterministic) } else { b = b[:cap(b)] - n, err := m.MarshalTo(b) + n, err := m.MarshalToSizedBuffer(b) if err != nil { return nil, err } return b[:n], nil } } -func (dst *SwitchWorkerRelayMasterResponse) XXX_Merge(src proto.Message) { - xxx_messageInfo_SwitchWorkerRelayMasterResponse.Merge(dst, src) +func (m *SwitchWorkerRelayMasterResponse) XXX_Merge(src proto.Message) { + xxx_messageInfo_SwitchWorkerRelayMasterResponse.Merge(m, src) } func (m *SwitchWorkerRelayMasterResponse) XXX_Size() int { return m.Size() @@ -1422,14 +1420,14 @@ func (m *SwitchWorkerRelayMasterResponse) GetWorkers() []*CommonWorkerResponse { // OperateWorkerRelayRequest represents a request for some dm-workers to operate relay unit type OperateWorkerRelayRequest struct { Op RelayOp `protobuf:"varint,1,opt,name=op,proto3,enum=pb.RelayOp" json:"op,omitempty"` - Workers []string `protobuf:"bytes,2,rep,name=workers" json:"workers,omitempty"` + Sources []string `protobuf:"bytes,2,rep,name=sources,proto3" json:"sources,omitempty"` } func (m *OperateWorkerRelayRequest) Reset() { *m = OperateWorkerRelayRequest{} } func (m *OperateWorkerRelayRequest) String() string { return proto.CompactTextString(m) } func (*OperateWorkerRelayRequest) ProtoMessage() {} func (*OperateWorkerRelayRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_dmmaster_632e41001d9f6d87, []int{23} + return fileDescriptor_f9bef11f2a341f03, []int{23} } func (m *OperateWorkerRelayRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1439,15 +1437,15 @@ func (m *OperateWorkerRelayRequest) XXX_Marshal(b []byte, deterministic bool) ([ return xxx_messageInfo_OperateWorkerRelayRequest.Marshal(b, m, deterministic) } else { b = b[:cap(b)] - n, err := m.MarshalTo(b) + n, err := m.MarshalToSizedBuffer(b) if err != nil { return nil, err } return b[:n], nil } } -func (dst *OperateWorkerRelayRequest) XXX_Merge(src proto.Message) { - xxx_messageInfo_OperateWorkerRelayRequest.Merge(dst, src) +func (m *OperateWorkerRelayRequest) XXX_Merge(src proto.Message) { + xxx_messageInfo_OperateWorkerRelayRequest.Merge(m, src) } func (m *OperateWorkerRelayRequest) XXX_Size() int { return m.Size() @@ -1465,9 +1463,9 @@ func (m *OperateWorkerRelayRequest) GetOp() RelayOp { return RelayOp_InvalidRelayOp } -func (m *OperateWorkerRelayRequest) GetWorkers() []string { +func (m *OperateWorkerRelayRequest) GetSources() []string { if m != nil { - return m.Workers + return m.Sources } return nil } @@ -1476,14 +1474,14 @@ type OperateWorkerRelayResponse struct { Op RelayOp `protobuf:"varint,1,opt,name=op,proto3,enum=pb.RelayOp" json:"op,omitempty"` Result bool `protobuf:"varint,2,opt,name=result,proto3" json:"result,omitempty"` Msg string `protobuf:"bytes,3,opt,name=msg,proto3" json:"msg,omitempty"` - Workers []*OperateRelayResponse `protobuf:"bytes,4,rep,name=workers" json:"workers,omitempty"` + Workers []*OperateRelayResponse `protobuf:"bytes,4,rep,name=workers,proto3" json:"workers,omitempty"` } func (m *OperateWorkerRelayResponse) Reset() { *m = OperateWorkerRelayResponse{} } func (m *OperateWorkerRelayResponse) String() string { return proto.CompactTextString(m) } func (*OperateWorkerRelayResponse) ProtoMessage() {} func (*OperateWorkerRelayResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_dmmaster_632e41001d9f6d87, []int{24} + return fileDescriptor_f9bef11f2a341f03, []int{24} } func (m *OperateWorkerRelayResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1493,15 +1491,15 @@ func (m *OperateWorkerRelayResponse) XXX_Marshal(b []byte, deterministic bool) ( return xxx_messageInfo_OperateWorkerRelayResponse.Marshal(b, m, deterministic) } else { b = b[:cap(b)] - n, err := m.MarshalTo(b) + n, err := m.MarshalToSizedBuffer(b) if err != nil { return nil, err } return b[:n], nil } } -func (dst *OperateWorkerRelayResponse) XXX_Merge(src proto.Message) { - xxx_messageInfo_OperateWorkerRelayResponse.Merge(dst, src) +func (m *OperateWorkerRelayResponse) XXX_Merge(src proto.Message) { + xxx_messageInfo_OperateWorkerRelayResponse.Merge(m, src) } func (m *OperateWorkerRelayResponse) XXX_Size() int { return m.Size() @@ -1540,150 +1538,10 @@ func (m *OperateWorkerRelayResponse) GetWorkers() []*OperateRelayResponse { return nil } -type RefreshWorkerTasksRequest struct { -} - -func (m *RefreshWorkerTasksRequest) Reset() { *m = RefreshWorkerTasksRequest{} } -func (m *RefreshWorkerTasksRequest) String() string { return proto.CompactTextString(m) } -func (*RefreshWorkerTasksRequest) ProtoMessage() {} -func (*RefreshWorkerTasksRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_dmmaster_632e41001d9f6d87, []int{25} -} -func (m *RefreshWorkerTasksRequest) XXX_Unmarshal(b []byte) error { - return m.Unmarshal(b) -} -func (m *RefreshWorkerTasksRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { - if deterministic { - return xxx_messageInfo_RefreshWorkerTasksRequest.Marshal(b, m, deterministic) - } else { - b = b[:cap(b)] - n, err := m.MarshalTo(b) - if err != nil { - return nil, err - } - return b[:n], nil - } -} -func (dst *RefreshWorkerTasksRequest) XXX_Merge(src proto.Message) { - xxx_messageInfo_RefreshWorkerTasksRequest.Merge(dst, src) -} -func (m *RefreshWorkerTasksRequest) XXX_Size() int { - return m.Size() -} -func (m *RefreshWorkerTasksRequest) XXX_DiscardUnknown() { - xxx_messageInfo_RefreshWorkerTasksRequest.DiscardUnknown(m) -} - -var xxx_messageInfo_RefreshWorkerTasksRequest proto.InternalMessageInfo - -type RefreshWorkerTasksMsg struct { - Worker string `protobuf:"bytes,1,opt,name=worker,proto3" json:"worker,omitempty"` - Msg string `protobuf:"bytes,2,opt,name=msg,proto3" json:"msg,omitempty"` -} - -func (m *RefreshWorkerTasksMsg) Reset() { *m = RefreshWorkerTasksMsg{} } -func (m *RefreshWorkerTasksMsg) String() string { return proto.CompactTextString(m) } -func (*RefreshWorkerTasksMsg) ProtoMessage() {} -func (*RefreshWorkerTasksMsg) Descriptor() ([]byte, []int) { - return fileDescriptor_dmmaster_632e41001d9f6d87, []int{26} -} -func (m *RefreshWorkerTasksMsg) XXX_Unmarshal(b []byte) error { - return m.Unmarshal(b) -} -func (m *RefreshWorkerTasksMsg) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { - if deterministic { - return xxx_messageInfo_RefreshWorkerTasksMsg.Marshal(b, m, deterministic) - } else { - b = b[:cap(b)] - n, err := m.MarshalTo(b) - if err != nil { - return nil, err - } - return b[:n], nil - } -} -func (dst *RefreshWorkerTasksMsg) XXX_Merge(src proto.Message) { - xxx_messageInfo_RefreshWorkerTasksMsg.Merge(dst, src) -} -func (m *RefreshWorkerTasksMsg) XXX_Size() int { - return m.Size() -} -func (m *RefreshWorkerTasksMsg) XXX_DiscardUnknown() { - xxx_messageInfo_RefreshWorkerTasksMsg.DiscardUnknown(m) -} - -var xxx_messageInfo_RefreshWorkerTasksMsg proto.InternalMessageInfo - -func (m *RefreshWorkerTasksMsg) GetWorker() string { - if m != nil { - return m.Worker - } - return "" -} - -func (m *RefreshWorkerTasksMsg) GetMsg() string { - if m != nil { - return m.Msg - } - return "" -} - -type RefreshWorkerTasksResponse struct { - Result bool `protobuf:"varint,1,opt,name=result,proto3" json:"result,omitempty"` - Workers []*RefreshWorkerTasksMsg `protobuf:"bytes,2,rep,name=workers" json:"workers,omitempty"` -} - -func (m *RefreshWorkerTasksResponse) Reset() { *m = RefreshWorkerTasksResponse{} } -func (m *RefreshWorkerTasksResponse) String() string { return proto.CompactTextString(m) } -func (*RefreshWorkerTasksResponse) ProtoMessage() {} -func (*RefreshWorkerTasksResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_dmmaster_632e41001d9f6d87, []int{27} -} -func (m *RefreshWorkerTasksResponse) XXX_Unmarshal(b []byte) error { - return m.Unmarshal(b) -} -func (m *RefreshWorkerTasksResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { - if deterministic { - return xxx_messageInfo_RefreshWorkerTasksResponse.Marshal(b, m, deterministic) - } else { - b = b[:cap(b)] - n, err := m.MarshalTo(b) - if err != nil { - return nil, err - } - return b[:n], nil - } -} -func (dst *RefreshWorkerTasksResponse) XXX_Merge(src proto.Message) { - xxx_messageInfo_RefreshWorkerTasksResponse.Merge(dst, src) -} -func (m *RefreshWorkerTasksResponse) XXX_Size() int { - return m.Size() -} -func (m *RefreshWorkerTasksResponse) XXX_DiscardUnknown() { - xxx_messageInfo_RefreshWorkerTasksResponse.DiscardUnknown(m) -} - -var xxx_messageInfo_RefreshWorkerTasksResponse proto.InternalMessageInfo - -func (m *RefreshWorkerTasksResponse) GetResult() bool { - if m != nil { - return m.Result - } - return false -} - -func (m *RefreshWorkerTasksResponse) GetWorkers() []*RefreshWorkerTasksMsg { - if m != nil { - return m.Workers - } - return nil -} - type HandleSQLsRequest struct { Name string `protobuf:"bytes,1,opt,name=name,proto3" json:"name,omitempty"` Op SQLOp `protobuf:"varint,2,opt,name=op,proto3,enum=pb.SQLOp" json:"op,omitempty"` - Args []string `protobuf:"bytes,3,rep,name=args" json:"args,omitempty"` + Args []string `protobuf:"bytes,3,rep,name=args,proto3" json:"args,omitempty"` BinlogPos string `protobuf:"bytes,4,opt,name=binlogPos,proto3" json:"binlogPos,omitempty"` Worker string `protobuf:"bytes,5,opt,name=worker,proto3" json:"worker,omitempty"` SqlPattern string `protobuf:"bytes,6,opt,name=sqlPattern,proto3" json:"sqlPattern,omitempty"` @@ -1694,7 +1552,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_dmmaster_632e41001d9f6d87, []int{28} + return fileDescriptor_f9bef11f2a341f03, []int{25} } func (m *HandleSQLsRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1704,15 +1562,15 @@ func (m *HandleSQLsRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, e return xxx_messageInfo_HandleSQLsRequest.Marshal(b, m, deterministic) } else { b = b[:cap(b)] - n, err := m.MarshalTo(b) + n, err := m.MarshalToSizedBuffer(b) if err != nil { return nil, err } return b[:n], nil } } -func (dst *HandleSQLsRequest) XXX_Merge(src proto.Message) { - xxx_messageInfo_HandleSQLsRequest.Merge(dst, src) +func (m *HandleSQLsRequest) XXX_Merge(src proto.Message) { + xxx_messageInfo_HandleSQLsRequest.Merge(m, src) } func (m *HandleSQLsRequest) XXX_Size() int { return m.Size() @@ -1775,14 +1633,14 @@ func (m *HandleSQLsRequest) GetSharding() bool { type HandleSQLsResponse 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" json:"workers,omitempty"` + Workers []*CommonWorkerResponse `protobuf:"bytes,3,rep,name=workers,proto3" json:"workers,omitempty"` } 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_dmmaster_632e41001d9f6d87, []int{29} + return fileDescriptor_f9bef11f2a341f03, []int{26} } func (m *HandleSQLsResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1792,15 +1650,15 @@ func (m *HandleSQLsResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, return xxx_messageInfo_HandleSQLsResponse.Marshal(b, m, deterministic) } else { b = b[:cap(b)] - n, err := m.MarshalTo(b) + n, err := m.MarshalToSizedBuffer(b) if err != nil { return nil, err } return b[:n], nil } } -func (dst *HandleSQLsResponse) XXX_Merge(src proto.Message) { - xxx_messageInfo_HandleSQLsResponse.Merge(dst, src) +func (m *HandleSQLsResponse) XXX_Merge(src proto.Message) { + xxx_messageInfo_HandleSQLsResponse.Merge(m, src) } func (m *HandleSQLsResponse) XXX_Size() int { return m.Size() @@ -1839,7 +1697,7 @@ func (m *HandleSQLsResponse) GetWorkers() []*CommonWorkerResponse { // filename: whether purge relay log files before this filename // subDir: specify relay sub directory for @filename type PurgeWorkerRelayRequest struct { - Workers []string `protobuf:"bytes,1,rep,name=workers" json:"workers,omitempty"` + Sources []string `protobuf:"bytes,1,rep,name=sources,proto3" json:"sources,omitempty"` Inactive bool `protobuf:"varint,2,opt,name=inactive,proto3" json:"inactive,omitempty"` Time int64 `protobuf:"varint,3,opt,name=time,proto3" json:"time,omitempty"` Filename string `protobuf:"bytes,4,opt,name=filename,proto3" json:"filename,omitempty"` @@ -1850,7 +1708,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_dmmaster_632e41001d9f6d87, []int{30} + return fileDescriptor_f9bef11f2a341f03, []int{27} } func (m *PurgeWorkerRelayRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1860,15 +1718,15 @@ func (m *PurgeWorkerRelayRequest) XXX_Marshal(b []byte, deterministic bool) ([]b return xxx_messageInfo_PurgeWorkerRelayRequest.Marshal(b, m, deterministic) } else { b = b[:cap(b)] - n, err := m.MarshalTo(b) + n, err := m.MarshalToSizedBuffer(b) if err != nil { return nil, err } return b[:n], nil } } -func (dst *PurgeWorkerRelayRequest) XXX_Merge(src proto.Message) { - xxx_messageInfo_PurgeWorkerRelayRequest.Merge(dst, src) +func (m *PurgeWorkerRelayRequest) XXX_Merge(src proto.Message) { + xxx_messageInfo_PurgeWorkerRelayRequest.Merge(m, src) } func (m *PurgeWorkerRelayRequest) XXX_Size() int { return m.Size() @@ -1879,9 +1737,9 @@ func (m *PurgeWorkerRelayRequest) XXX_DiscardUnknown() { var xxx_messageInfo_PurgeWorkerRelayRequest proto.InternalMessageInfo -func (m *PurgeWorkerRelayRequest) GetWorkers() []string { +func (m *PurgeWorkerRelayRequest) GetSources() []string { if m != nil { - return m.Workers + return m.Sources } return nil } @@ -1917,14 +1775,14 @@ func (m *PurgeWorkerRelayRequest) GetSubDir() string { type PurgeWorkerRelayResponse 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" json:"workers,omitempty"` + Workers []*CommonWorkerResponse `protobuf:"bytes,3,rep,name=workers,proto3" json:"workers,omitempty"` } func (m *PurgeWorkerRelayResponse) Reset() { *m = PurgeWorkerRelayResponse{} } func (m *PurgeWorkerRelayResponse) String() string { return proto.CompactTextString(m) } func (*PurgeWorkerRelayResponse) ProtoMessage() {} func (*PurgeWorkerRelayResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_dmmaster_632e41001d9f6d87, []int{31} + return fileDescriptor_f9bef11f2a341f03, []int{28} } func (m *PurgeWorkerRelayResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1934,15 +1792,15 @@ func (m *PurgeWorkerRelayResponse) XXX_Marshal(b []byte, deterministic bool) ([] return xxx_messageInfo_PurgeWorkerRelayResponse.Marshal(b, m, deterministic) } else { b = b[:cap(b)] - n, err := m.MarshalTo(b) + n, err := m.MarshalToSizedBuffer(b) if err != nil { return nil, err } return b[:n], nil } } -func (dst *PurgeWorkerRelayResponse) XXX_Merge(src proto.Message) { - xxx_messageInfo_PurgeWorkerRelayResponse.Merge(dst, src) +func (m *PurgeWorkerRelayResponse) XXX_Merge(src proto.Message) { + xxx_messageInfo_PurgeWorkerRelayResponse.Merge(m, src) } func (m *PurgeWorkerRelayResponse) XXX_Size() int { return m.Size() @@ -1982,7 +1840,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_dmmaster_632e41001d9f6d87, []int{32} + return fileDescriptor_f9bef11f2a341f03, []int{29} } func (m *CheckTaskRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1992,15 +1850,15 @@ func (m *CheckTaskRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, er return xxx_messageInfo_CheckTaskRequest.Marshal(b, m, deterministic) } else { b = b[:cap(b)] - n, err := m.MarshalTo(b) + n, err := m.MarshalToSizedBuffer(b) if err != nil { return nil, err } return b[:n], nil } } -func (dst *CheckTaskRequest) XXX_Merge(src proto.Message) { - xxx_messageInfo_CheckTaskRequest.Merge(dst, src) +func (m *CheckTaskRequest) XXX_Merge(src proto.Message) { + xxx_messageInfo_CheckTaskRequest.Merge(m, src) } func (m *CheckTaskRequest) XXX_Size() int { return m.Size() @@ -2027,7 +1885,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_dmmaster_632e41001d9f6d87, []int{33} + return fileDescriptor_f9bef11f2a341f03, []int{30} } func (m *CheckTaskResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2037,15 +1895,15 @@ func (m *CheckTaskResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, e return xxx_messageInfo_CheckTaskResponse.Marshal(b, m, deterministic) } else { b = b[:cap(b)] - n, err := m.MarshalTo(b) + n, err := m.MarshalToSizedBuffer(b) if err != nil { return nil, err } return b[:n], nil } } -func (dst *CheckTaskResponse) XXX_Merge(src proto.Message) { - xxx_messageInfo_CheckTaskResponse.Merge(dst, src) +func (m *CheckTaskResponse) XXX_Merge(src proto.Message) { + xxx_messageInfo_CheckTaskResponse.Merge(m, src) } func (m *CheckTaskResponse) XXX_Size() int { return m.Size() @@ -2079,7 +1937,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_dmmaster_632e41001d9f6d87, []int{34} + return fileDescriptor_f9bef11f2a341f03, []int{31} } func (m *RegisterWorkerRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2089,15 +1947,15 @@ func (m *RegisterWorkerRequest) XXX_Marshal(b []byte, deterministic bool) ([]byt return xxx_messageInfo_RegisterWorkerRequest.Marshal(b, m, deterministic) } else { b = b[:cap(b)] - n, err := m.MarshalTo(b) + n, err := m.MarshalToSizedBuffer(b) if err != nil { return nil, err } return b[:n], nil } } -func (dst *RegisterWorkerRequest) XXX_Merge(src proto.Message) { - xxx_messageInfo_RegisterWorkerRequest.Merge(dst, src) +func (m *RegisterWorkerRequest) XXX_Merge(src proto.Message) { + xxx_messageInfo_RegisterWorkerRequest.Merge(m, src) } func (m *RegisterWorkerRequest) XXX_Size() int { return m.Size() @@ -2131,7 +1989,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_dmmaster_632e41001d9f6d87, []int{35} + return fileDescriptor_f9bef11f2a341f03, []int{32} } func (m *RegisterWorkerResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2141,15 +1999,15 @@ func (m *RegisterWorkerResponse) XXX_Marshal(b []byte, deterministic bool) ([]by return xxx_messageInfo_RegisterWorkerResponse.Marshal(b, m, deterministic) } else { b = b[:cap(b)] - n, err := m.MarshalTo(b) + n, err := m.MarshalToSizedBuffer(b) if err != nil { return nil, err } return b[:n], nil } } -func (dst *RegisterWorkerResponse) XXX_Merge(src proto.Message) { - xxx_messageInfo_RegisterWorkerResponse.Merge(dst, src) +func (m *RegisterWorkerResponse) XXX_Merge(src proto.Message) { + xxx_messageInfo_RegisterWorkerResponse.Merge(m, src) } func (m *RegisterWorkerResponse) XXX_Size() int { return m.Size() @@ -2200,9 +2058,6 @@ func init() { proto.RegisterType((*SwitchWorkerRelayMasterResponse)(nil), "pb.SwitchWorkerRelayMasterResponse") proto.RegisterType((*OperateWorkerRelayRequest)(nil), "pb.OperateWorkerRelayRequest") proto.RegisterType((*OperateWorkerRelayResponse)(nil), "pb.OperateWorkerRelayResponse") - proto.RegisterType((*RefreshWorkerTasksRequest)(nil), "pb.RefreshWorkerTasksRequest") - proto.RegisterType((*RefreshWorkerTasksMsg)(nil), "pb.RefreshWorkerTasksMsg") - proto.RegisterType((*RefreshWorkerTasksResponse)(nil), "pb.RefreshWorkerTasksResponse") proto.RegisterType((*HandleSQLsRequest)(nil), "pb.HandleSQLsRequest") proto.RegisterType((*HandleSQLsResponse)(nil), "pb.HandleSQLsResponse") proto.RegisterType((*PurgeWorkerRelayRequest)(nil), "pb.PurgeWorkerRelayRequest") @@ -2213,6 +2068,98 @@ func init() { proto.RegisterType((*RegisterWorkerResponse)(nil), "pb.RegisterWorkerResponse") } +func init() { proto.RegisterFile("dmmaster.proto", fileDescriptor_f9bef11f2a341f03) } + +var fileDescriptor_f9bef11f2a341f03 = []byte{ + // 1372 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xbc, 0x58, 0xbf, 0x6f, 0xdb, 0x46, + 0x14, 0x16, 0x25, 0x5b, 0x8e, 0x9e, 0x12, 0xc3, 0xbe, 0x38, 0x12, 0xc5, 0x38, 0x8a, 0xcb, 0x16, + 0x81, 0xd1, 0x21, 0x6e, 0x9c, 0x4e, 0x01, 0x02, 0xb4, 0xb6, 0x12, 0xd4, 0x80, 0x52, 0x3b, 0x54, + 0x8d, 0x22, 0x4b, 0x01, 0x4a, 0x3a, 0xcb, 0x84, 0x28, 0x92, 0xe6, 0x51, 0x76, 0xdc, 0xa2, 0x28, + 0xd0, 0xa1, 0x4b, 0x97, 0x16, 0x1d, 0x32, 0xf7, 0x6f, 0xe9, 0x92, 0x31, 0x40, 0x97, 0x8e, 0x85, + 0xdd, 0x3f, 0xa4, 0xb8, 0x1f, 0x24, 0x8f, 0xbf, 0x94, 0xd8, 0x83, 0x36, 0xde, 0x7b, 0xe4, 0xf7, + 0xbe, 0xf7, 0xee, 0xee, 0xdd, 0x77, 0x84, 0xe5, 0xe1, 0x64, 0x62, 0x92, 0x00, 0xfb, 0x0f, 0x3d, + 0xdf, 0x0d, 0x5c, 0x54, 0xf6, 0xfa, 0xda, 0xf2, 0x70, 0x72, 0xe6, 0xfa, 0xe3, 0xd0, 0xa6, 0xad, + 0x8f, 0x5c, 0x77, 0x64, 0xe3, 0x2d, 0xd3, 0xb3, 0xb6, 0x4c, 0xc7, 0x71, 0x03, 0x33, 0xb0, 0x5c, + 0x87, 0x70, 0xaf, 0x7e, 0x02, 0xad, 0x17, 0xd6, 0xc8, 0x37, 0x03, 0xfc, 0x2d, 0xfb, 0xc8, 0xc0, + 0xb6, 0x79, 0x6e, 0xe0, 0x93, 0x29, 0x26, 0x01, 0x6a, 0x03, 0xec, 0x58, 0x8e, 0xed, 0x8e, 0xbe, + 0x36, 0x27, 0x58, 0x55, 0x36, 0x94, 0xcd, 0x9a, 0x21, 0x59, 0xd0, 0x3a, 0xd4, 0xf8, 0xe8, 0xc0, + 0x25, 0x6a, 0x79, 0x43, 0xd9, 0xbc, 0x65, 0xc4, 0x06, 0xd4, 0x80, 0x2a, 0x27, 0xa2, 0x56, 0xd8, + 0x97, 0x62, 0xa4, 0x1f, 0x40, 0xfb, 0xd0, 0x1b, 0x26, 0x23, 0xee, 0xba, 0xce, 0x91, 0x35, 0x0a, + 0xe3, 0x36, 0xa0, 0x3a, 0x60, 0x06, 0x11, 0x53, 0x8c, 0x24, 0xc4, 0x72, 0x02, 0xf1, 0x0b, 0x58, + 0xe9, 0x05, 0xa6, 0x1f, 0x7c, 0x63, 0x92, 0x71, 0x88, 0x81, 0x60, 0x21, 0x30, 0xc9, 0x58, 0x20, + 0xb0, 0x67, 0xa4, 0xc2, 0x12, 0x71, 0xa7, 0xfe, 0x00, 0x53, 0xb6, 0x95, 0xcd, 0x9a, 0x11, 0x0e, + 0xf5, 0x13, 0x58, 0x95, 0x10, 0x88, 0xe7, 0x3a, 0x04, 0xd3, 0x70, 0x3e, 0x26, 0x53, 0x3b, 0x60, + 0x20, 0x37, 0x0c, 0x31, 0x42, 0x2b, 0x50, 0x99, 0x90, 0x91, 0xe0, 0x40, 0x1f, 0xd1, 0x36, 0x2c, + 0x71, 0x2a, 0x44, 0xad, 0x6c, 0x54, 0x36, 0xeb, 0xdb, 0xea, 0x43, 0xaf, 0xff, 0x70, 0xd7, 0x9d, + 0x4c, 0x5c, 0x27, 0xcc, 0x92, 0x83, 0x1a, 0xe1, 0x8b, 0xfa, 0x63, 0x68, 0xf1, 0x32, 0xbc, 0x60, + 0x33, 0xf8, 0x41, 0x15, 0xd0, 0xcf, 0x41, 0xcb, 0xfb, 0xe8, 0xca, 0x84, 0x1f, 0xa5, 0x09, 0x37, + 0x29, 0xe1, 0x97, 0x53, 0xec, 0x9f, 0xf7, 0x02, 0x33, 0x98, 0x92, 0x2c, 0xdf, 0xef, 0x00, 0xed, + 0x7b, 0x98, 0xae, 0x14, 0xb9, 0xcc, 0x1a, 0x94, 0x5d, 0x8f, 0x85, 0x5b, 0xde, 0x06, 0x8a, 0x41, + 0x9d, 0xfb, 0x9e, 0x51, 0x76, 0x3d, 0x3a, 0x05, 0x0e, 0x5d, 0x38, 0x3c, 0x2e, 0x7b, 0x96, 0xa7, + 0xa0, 0x92, 0x9c, 0x82, 0xdf, 0x15, 0xb8, 0x9d, 0x08, 0x20, 0x92, 0x9a, 0x15, 0x21, 0x4e, 0xb8, + 0x9c, 0x97, 0x70, 0x25, 0x4e, 0xf8, 0xf3, 0x38, 0xe1, 0x05, 0x96, 0xb0, 0x46, 0xa1, 0x44, 0xbc, + 0xde, 0xb4, 0x2f, 0x87, 0x8c, 0x73, 0xfe, 0x12, 0x56, 0x79, 0xb9, 0xaf, 0xbf, 0xb2, 0x7c, 0x40, + 0x32, 0xc4, 0x5c, 0x96, 0xd6, 0x73, 0x68, 0x48, 0x53, 0xd9, 0xb5, 0x48, 0x20, 0x71, 0x77, 0xe2, + 0xbd, 0x9c, 0x99, 0x92, 0x14, 0xf7, 0x53, 0x68, 0x66, 0x70, 0xe6, 0xb1, 0xd4, 0x9e, 0xc1, 0x1d, + 0xe6, 0x7f, 0xe6, 0xfb, 0xae, 0x7f, 0x7d, 0xfa, 0x81, 0x28, 0x83, 0x04, 0x73, 0x65, 0xf6, 0x9f, + 0xa5, 0xd9, 0x37, 0x22, 0xf6, 0x0c, 0x36, 0x4b, 0x7e, 0x17, 0x6e, 0xf7, 0x8e, 0xdd, 0xb3, 0x4e, + 0xa7, 0xdb, 0x75, 0x07, 0x63, 0x72, 0xbd, 0x55, 0xf3, 0xab, 0x02, 0x4b, 0x02, 0x01, 0x2d, 0x43, + 0x79, 0xaf, 0x23, 0xbe, 0x2b, 0xef, 0x75, 0x22, 0xa4, 0xb2, 0x84, 0xb4, 0x06, 0x8b, 0xee, 0x99, + 0x13, 0xb5, 0x5a, 0x3e, 0xa0, 0x6f, 0x76, 0x3a, 0x5d, 0xbe, 0xe2, 0x6b, 0x06, 0x7b, 0xa6, 0xa9, + 0x93, 0x73, 0x67, 0x80, 0x87, 0xea, 0x22, 0xb3, 0x8a, 0x11, 0xd2, 0xe0, 0xc6, 0xd4, 0x11, 0x9e, + 0x2a, 0xf3, 0x44, 0x63, 0x7d, 0x00, 0x6b, 0xc9, 0x94, 0xae, 0x5c, 0xc6, 0x8f, 0x60, 0xd1, 0xa6, + 0x9f, 0x8a, 0x22, 0xd6, 0x69, 0x11, 0x05, 0x9c, 0xc1, 0x3d, 0xfa, 0x2f, 0x0a, 0xac, 0x1d, 0x3a, + 0xf4, 0x39, 0x74, 0x88, 0xca, 0xa5, 0xf3, 0xd7, 0xe1, 0xa6, 0x8f, 0x3d, 0xdb, 0x1c, 0xe0, 0x7d, + 0x96, 0x32, 0x0f, 0x93, 0xb0, 0x15, 0xb7, 0x19, 0xb4, 0x01, 0xf5, 0x23, 0xd7, 0x1f, 0x60, 0x03, + 0x4f, 0xdc, 0x53, 0xac, 0x2e, 0x30, 0xe2, 0xb2, 0x49, 0x9f, 0xc2, 0x9d, 0x14, 0x8f, 0xb9, 0x6c, + 0xda, 0x3f, 0x15, 0x68, 0xed, 0xf8, 0xd8, 0x1c, 0xf3, 0x17, 0x52, 0x45, 0x90, 0x12, 0x52, 0x92, + 0x09, 0xe5, 0x2d, 0x07, 0x56, 0x22, 0x9a, 0x0c, 0x85, 0xd8, 0xeb, 0x88, 0x55, 0x91, 0xb0, 0x51, + 0x44, 0xfc, 0x1a, 0x0f, 0x3a, 0x9d, 0xae, 0x28, 0x42, 0x38, 0x64, 0xb1, 0xc6, 0x96, 0x47, 0x3d, + 0x8b, 0xdc, 0x23, 0x86, 0xfa, 0xf7, 0xa0, 0xe5, 0x51, 0x9c, 0x4b, 0x7d, 0x9e, 0x40, 0xbb, 0x77, + 0x66, 0x05, 0x83, 0x63, 0x49, 0x36, 0xf0, 0x53, 0xf0, 0xbd, 0x35, 0xd2, 0x7f, 0x82, 0xfb, 0x85, + 0xdf, 0xce, 0x85, 0xbc, 0x01, 0x2d, 0x71, 0xd6, 0xe4, 0xc8, 0xac, 0xbb, 0xd2, 0x09, 0xc7, 0x76, + 0x06, 0xf3, 0x8a, 0x23, 0xae, 0xb8, 0x47, 0xbc, 0x51, 0x40, 0xcb, 0x03, 0x15, 0x09, 0xcd, 0x44, + 0xfd, 0xf0, 0x83, 0x73, 0x3b, 0x7d, 0x70, 0xaa, 0xd2, 0xc1, 0x99, 0x88, 0x18, 0x67, 0xfb, 0x56, + 0x81, 0xd5, 0xaf, 0x4c, 0x67, 0x68, 0xe3, 0xde, 0xcb, 0x2e, 0x99, 0xd5, 0xbc, 0x5b, 0x8c, 0x64, + 0x99, 0x91, 0xac, 0x51, 0xe0, 0xde, 0xcb, 0x6e, 0xac, 0x1e, 0x4c, 0x7f, 0x14, 0xee, 0x5f, 0xf6, + 0x4c, 0x05, 0x67, 0x3f, 0x12, 0x9c, 0x0b, 0x0c, 0x27, 0x36, 0x48, 0xf2, 0x70, 0x51, 0x96, 0x87, + 0x54, 0xc6, 0x92, 0x13, 0xfb, 0xc0, 0x0c, 0x02, 0xec, 0x3b, 0x6a, 0x95, 0xcb, 0xd8, 0xd8, 0x42, + 0x5b, 0x1f, 0x39, 0x36, 0xfd, 0xa1, 0xe5, 0x8c, 0xd4, 0x25, 0x56, 0x8e, 0x68, 0x4c, 0x8f, 0x6f, + 0x39, 0x93, 0xb9, 0x2c, 0x96, 0x37, 0x0a, 0x34, 0x0f, 0xa6, 0xfe, 0x28, 0x6f, 0xad, 0x14, 0xf7, + 0x01, 0x0d, 0x6e, 0x58, 0x8e, 0x39, 0x08, 0xac, 0x53, 0x2c, 0x26, 0x35, 0x1a, 0xb3, 0x1e, 0x61, + 0x4d, 0x30, 0x9b, 0xd7, 0x8a, 0xc1, 0x9e, 0xe9, 0xfb, 0x47, 0x96, 0x8d, 0xd9, 0x94, 0xf0, 0x52, + 0x46, 0x63, 0x76, 0x48, 0x4c, 0xfb, 0x1d, 0x2b, 0xaa, 0x24, 0x1f, 0xe9, 0xaf, 0x41, 0xcd, 0x12, + 0x9b, 0x4b, 0x4d, 0x1e, 0xc0, 0xca, 0xee, 0x31, 0x1e, 0x8c, 0xdf, 0x23, 0xc4, 0xf4, 0xa7, 0xb0, + 0x2a, 0xbd, 0x77, 0x55, 0x6a, 0x54, 0x79, 0x18, 0x78, 0x64, 0xd1, 0xce, 0x10, 0x32, 0x99, 0xa9, + 0x3c, 0xcc, 0xe1, 0xd0, 0xc7, 0x84, 0x08, 0x88, 0x70, 0xa8, 0xef, 0x40, 0x23, 0x0d, 0x73, 0x55, + 0x2a, 0xdb, 0x7f, 0xd5, 0xa1, 0xca, 0x7b, 0x14, 0x7a, 0x05, 0xb5, 0xe8, 0x76, 0x82, 0xd6, 0xd8, + 0x36, 0x49, 0x5d, 0x77, 0xb4, 0x3b, 0x29, 0x2b, 0x0f, 0xa7, 0xdf, 0xff, 0xf9, 0xef, 0xff, 0xfe, + 0x28, 0xb7, 0xf4, 0x35, 0x7a, 0xfd, 0x23, 0x5b, 0xa7, 0x8f, 0x4c, 0xdb, 0x3b, 0x36, 0x1f, 0x6d, + 0xd1, 0x5a, 0x91, 0x27, 0xca, 0xa7, 0xe8, 0x08, 0xea, 0x92, 0xe8, 0x46, 0x0d, 0x69, 0x73, 0xcb, + 0xf0, 0xcd, 0x8c, 0x5d, 0x04, 0x78, 0xc0, 0x02, 0x6c, 0x68, 0x77, 0xf3, 0x02, 0x6c, 0xfd, 0x40, + 0xeb, 0xf4, 0x23, 0x8d, 0xf3, 0x14, 0x20, 0x96, 0xc1, 0x88, 0xb1, 0xcd, 0x28, 0x6b, 0xad, 0x91, + 0x36, 0x8b, 0x20, 0x25, 0x64, 0x43, 0x5d, 0x52, 0x8c, 0x48, 0x4b, 0x49, 0x48, 0x49, 0x23, 0x6a, + 0x77, 0x73, 0x7d, 0x02, 0xe9, 0x13, 0x46, 0xb7, 0x8d, 0xd6, 0x53, 0x74, 0x09, 0x7b, 0x55, 0xf0, + 0x45, 0xcf, 0x00, 0x62, 0x85, 0x87, 0x5a, 0x49, 0xc5, 0x27, 0xc7, 0xd2, 0xf2, 0x5c, 0x11, 0xe9, + 0x5d, 0xb8, 0x29, 0xcb, 0x26, 0xc4, 0x8a, 0x98, 0xa3, 0x0d, 0x35, 0x35, 0xeb, 0x88, 0x40, 0x9e, + 0xc3, 0xad, 0x84, 0x1a, 0x41, 0xec, 0xe5, 0x3c, 0xa1, 0xa4, 0xb5, 0x72, 0x3c, 0x11, 0xce, 0x61, + 0x78, 0x0f, 0x91, 0x6f, 0x8e, 0xe8, 0x5e, 0x5c, 0xf1, 0x9c, 0x6b, 0xa8, 0xd6, 0x2e, 0x72, 0x47, + 0xb0, 0xaf, 0xa0, 0x59, 0x70, 0x99, 0x47, 0x7a, 0xfc, 0x71, 0xd1, 0x4d, 0x5f, 0x2b, 0xdc, 0xf9, + 0x9c, 0x71, 0x56, 0x6c, 0x70, 0xc6, 0x85, 0x3a, 0x89, 0x33, 0x2e, 0xd6, 0x28, 0x7a, 0x89, 0xae, + 0xc4, 0xb8, 0xa3, 0xf3, 0x95, 0x98, 0x39, 0xab, 0xf8, 0x4a, 0xcc, 0x36, 0x7e, 0xbd, 0x84, 0x86, + 0xd0, 0x2c, 0x90, 0x12, 0x3c, 0xe1, 0xd9, 0x1a, 0x45, 0xfb, 0x78, 0xe6, 0x3b, 0x52, 0x59, 0x1b, + 0xd9, 0xa3, 0x9d, 0x6d, 0x9d, 0x7b, 0xd2, 0x4e, 0xcc, 0x9e, 0x0f, 0x3c, 0xff, 0x62, 0x55, 0xa0, + 0x97, 0xd0, 0x3e, 0xac, 0xa4, 0x7b, 0x38, 0x62, 0x7b, 0xa6, 0xe0, 0xc8, 0xd1, 0xd6, 0xf3, 0x9d, + 0x12, 0x20, 0xca, 0xfe, 0x42, 0xe2, 0x3c, 0x0b, 0x7f, 0x2d, 0xcd, 0x9c, 0xf8, 0x27, 0x50, 0x8b, + 0x7a, 0x38, 0x6f, 0x77, 0xe9, 0xd6, 0xcf, 0xdb, 0x5d, 0xa6, 0xd1, 0xb3, 0x3d, 0x17, 0xfe, 0xa5, + 0x78, 0x71, 0x4e, 0x4e, 0x6c, 0x8e, 0xcd, 0x41, 0x98, 0x21, 0x03, 0x22, 0x59, 0x23, 0x90, 0x3d, + 0x58, 0x4e, 0xb6, 0x6f, 0xde, 0x03, 0x72, 0x4f, 0x06, 0xde, 0x03, 0xf2, 0xbb, 0xbd, 0x5e, 0xda, + 0x51, 0xdf, 0x5e, 0xb4, 0x95, 0x77, 0x17, 0x6d, 0xe5, 0xdf, 0x8b, 0xb6, 0xf2, 0xdb, 0x65, 0xbb, + 0xf4, 0xee, 0xb2, 0x5d, 0xfa, 0xe7, 0xb2, 0x5d, 0xea, 0x57, 0xd9, 0x0f, 0xb8, 0xc7, 0xff, 0x07, + 0x00, 0x00, 0xff, 0xff, 0xcf, 0xf2, 0xa1, 0x64, 0xc4, 0x13, 0x00, 0x00, +} + // Reference imports to suppress errors if they are not otherwise used. var _ context.Context var _ grpc.ClientConn @@ -2246,13 +2193,12 @@ type MasterClient interface { OperateWorkerRelayTask(ctx context.Context, in *OperateWorkerRelayRequest, opts ...grpc.CallOption) (*OperateWorkerRelayResponse, error) // PurgeWorkerRelay purges relay log files for some dm-workers PurgeWorkerRelay(ctx context.Context, in *PurgeWorkerRelayRequest, opts ...grpc.CallOption) (*PurgeWorkerRelayResponse, error) - // used by dmctl, to force refresh the task -> workers mapper - // it should be used rarely only when task -> workers mapper corrupted - RefreshWorkerTasks(ctx context.Context, in *RefreshWorkerTasksRequest, opts ...grpc.CallOption) (*RefreshWorkerTasksResponse, error) // MigrateRelay request migrate old dm-woker to a new one. MigrateWorkerRelay(ctx context.Context, in *MigrateWorkerRelayRequest, opts ...grpc.CallOption) (*CommonWorkerResponse, error) // CheckTask checks legality of task configuration CheckTask(ctx context.Context, in *CheckTaskRequest, opts ...grpc.CallOption) (*CheckTaskResponse, error) + // Operate mysql-worker for server + OperateMysqlWorker(ctx context.Context, in *MysqlTaskRequest, opts ...grpc.CallOption) (*MysqlTaskResponse, error) // RegisterWorker register the dm-workers. RegisterWorker(ctx context.Context, in *RegisterWorkerRequest, opts ...grpc.CallOption) (*RegisterWorkerResponse, error) } @@ -2391,15 +2337,6 @@ func (c *masterClient) PurgeWorkerRelay(ctx context.Context, in *PurgeWorkerRela return out, nil } -func (c *masterClient) RefreshWorkerTasks(ctx context.Context, in *RefreshWorkerTasksRequest, opts ...grpc.CallOption) (*RefreshWorkerTasksResponse, error) { - out := new(RefreshWorkerTasksResponse) - err := c.cc.Invoke(ctx, "/pb.Master/RefreshWorkerTasks", in, out, opts...) - if err != nil { - return nil, err - } - return out, nil -} - func (c *masterClient) MigrateWorkerRelay(ctx context.Context, in *MigrateWorkerRelayRequest, opts ...grpc.CallOption) (*CommonWorkerResponse, error) { out := new(CommonWorkerResponse) err := c.cc.Invoke(ctx, "/pb.Master/MigrateWorkerRelay", in, out, opts...) @@ -2418,6 +2355,15 @@ func (c *masterClient) CheckTask(ctx context.Context, in *CheckTaskRequest, opts return out, nil } +func (c *masterClient) OperateMysqlWorker(ctx context.Context, in *MysqlTaskRequest, opts ...grpc.CallOption) (*MysqlTaskResponse, error) { + out := new(MysqlTaskResponse) + err := c.cc.Invoke(ctx, "/pb.Master/OperateMysqlWorker", in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + func (c *masterClient) RegisterWorker(ctx context.Context, in *RegisterWorkerRequest, opts ...grpc.CallOption) (*RegisterWorkerResponse, error) { out := new(RegisterWorkerResponse) err := c.cc.Invoke(ctx, "/pb.Master/RegisterWorker", in, out, opts...) @@ -2450,17 +2396,75 @@ type MasterServer interface { OperateWorkerRelayTask(context.Context, *OperateWorkerRelayRequest) (*OperateWorkerRelayResponse, error) // PurgeWorkerRelay purges relay log files for some dm-workers PurgeWorkerRelay(context.Context, *PurgeWorkerRelayRequest) (*PurgeWorkerRelayResponse, error) - // used by dmctl, to force refresh the task -> workers mapper - // it should be used rarely only when task -> workers mapper corrupted - RefreshWorkerTasks(context.Context, *RefreshWorkerTasksRequest) (*RefreshWorkerTasksResponse, error) // MigrateRelay request migrate old dm-woker to a new one. MigrateWorkerRelay(context.Context, *MigrateWorkerRelayRequest) (*CommonWorkerResponse, error) // CheckTask checks legality of task configuration CheckTask(context.Context, *CheckTaskRequest) (*CheckTaskResponse, error) + // Operate mysql-worker for server + OperateMysqlWorker(context.Context, *MysqlTaskRequest) (*MysqlTaskResponse, error) // RegisterWorker register the dm-workers. RegisterWorker(context.Context, *RegisterWorkerRequest) (*RegisterWorkerResponse, error) } +// UnimplementedMasterServer can be embedded to have forward compatible implementations. +type UnimplementedMasterServer struct { +} + +func (*UnimplementedMasterServer) StartTask(ctx context.Context, req *StartTaskRequest) (*StartTaskResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method StartTask not implemented") +} +func (*UnimplementedMasterServer) OperateTask(ctx context.Context, req *OperateTaskRequest) (*OperateTaskResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method OperateTask not implemented") +} +func (*UnimplementedMasterServer) UpdateTask(ctx context.Context, req *UpdateTaskRequest) (*UpdateTaskResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method UpdateTask not implemented") +} +func (*UnimplementedMasterServer) QueryStatus(ctx context.Context, req *QueryStatusListRequest) (*QueryStatusListResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method QueryStatus not implemented") +} +func (*UnimplementedMasterServer) QueryError(ctx context.Context, req *QueryErrorListRequest) (*QueryErrorListResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method QueryError not implemented") +} +func (*UnimplementedMasterServer) ShowDDLLocks(ctx context.Context, req *ShowDDLLocksRequest) (*ShowDDLLocksResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method ShowDDLLocks not implemented") +} +func (*UnimplementedMasterServer) UnlockDDLLock(ctx context.Context, req *UnlockDDLLockRequest) (*UnlockDDLLockResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method UnlockDDLLock not implemented") +} +func (*UnimplementedMasterServer) UpdateMasterConfig(ctx context.Context, req *UpdateMasterConfigRequest) (*UpdateMasterConfigResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method UpdateMasterConfig not implemented") +} +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") +} +func (*UnimplementedMasterServer) SwitchWorkerRelayMaster(ctx context.Context, req *SwitchWorkerRelayMasterRequest) (*SwitchWorkerRelayMasterResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method SwitchWorkerRelayMaster not implemented") +} +func (*UnimplementedMasterServer) OperateWorkerRelayTask(ctx context.Context, req *OperateWorkerRelayRequest) (*OperateWorkerRelayResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method OperateWorkerRelayTask not implemented") +} +func (*UnimplementedMasterServer) PurgeWorkerRelay(ctx context.Context, req *PurgeWorkerRelayRequest) (*PurgeWorkerRelayResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method PurgeWorkerRelay not implemented") +} +func (*UnimplementedMasterServer) MigrateWorkerRelay(ctx context.Context, req *MigrateWorkerRelayRequest) (*CommonWorkerResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method MigrateWorkerRelay not implemented") +} +func (*UnimplementedMasterServer) CheckTask(ctx context.Context, req *CheckTaskRequest) (*CheckTaskResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method CheckTask not implemented") +} +func (*UnimplementedMasterServer) OperateMysqlWorker(ctx context.Context, req *MysqlTaskRequest) (*MysqlTaskResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method OperateMysqlWorker not implemented") +} +func (*UnimplementedMasterServer) RegisterWorker(ctx context.Context, req *RegisterWorkerRequest) (*RegisterWorkerResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method RegisterWorker not implemented") +} + func RegisterMasterServer(s *grpc.Server, srv MasterServer) { s.RegisterService(&_Master_serviceDesc, srv) } @@ -2717,56 +2721,56 @@ func _Master_PurgeWorkerRelay_Handler(srv interface{}, ctx context.Context, dec return interceptor(ctx, in, info, handler) } -func _Master_RefreshWorkerTasks_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { - in := new(RefreshWorkerTasksRequest) +func _Master_MigrateWorkerRelay_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(MigrateWorkerRelayRequest) if err := dec(in); err != nil { return nil, err } if interceptor == nil { - return srv.(MasterServer).RefreshWorkerTasks(ctx, in) + return srv.(MasterServer).MigrateWorkerRelay(ctx, in) } info := &grpc.UnaryServerInfo{ Server: srv, - FullMethod: "/pb.Master/RefreshWorkerTasks", + FullMethod: "/pb.Master/MigrateWorkerRelay", } handler := func(ctx context.Context, req interface{}) (interface{}, error) { - return srv.(MasterServer).RefreshWorkerTasks(ctx, req.(*RefreshWorkerTasksRequest)) + return srv.(MasterServer).MigrateWorkerRelay(ctx, req.(*MigrateWorkerRelayRequest)) } return interceptor(ctx, in, info, handler) } -func _Master_MigrateWorkerRelay_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { - in := new(MigrateWorkerRelayRequest) +func _Master_CheckTask_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(CheckTaskRequest) if err := dec(in); err != nil { return nil, err } if interceptor == nil { - return srv.(MasterServer).MigrateWorkerRelay(ctx, in) + return srv.(MasterServer).CheckTask(ctx, in) } info := &grpc.UnaryServerInfo{ Server: srv, - FullMethod: "/pb.Master/MigrateWorkerRelay", + FullMethod: "/pb.Master/CheckTask", } handler := func(ctx context.Context, req interface{}) (interface{}, error) { - return srv.(MasterServer).MigrateWorkerRelay(ctx, req.(*MigrateWorkerRelayRequest)) + return srv.(MasterServer).CheckTask(ctx, req.(*CheckTaskRequest)) } return interceptor(ctx, in, info, handler) } -func _Master_CheckTask_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { - in := new(CheckTaskRequest) +func _Master_OperateMysqlWorker_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(MysqlTaskRequest) if err := dec(in); err != nil { return nil, err } if interceptor == nil { - return srv.(MasterServer).CheckTask(ctx, in) + return srv.(MasterServer).OperateMysqlWorker(ctx, in) } info := &grpc.UnaryServerInfo{ Server: srv, - FullMethod: "/pb.Master/CheckTask", + FullMethod: "/pb.Master/OperateMysqlWorker", } handler := func(ctx context.Context, req interface{}) (interface{}, error) { - return srv.(MasterServer).CheckTask(ctx, req.(*CheckTaskRequest)) + return srv.(MasterServer).OperateMysqlWorker(ctx, req.(*MysqlTaskRequest)) } return interceptor(ctx, in, info, handler) } @@ -2849,10 +2853,6 @@ var _Master_serviceDesc = grpc.ServiceDesc{ MethodName: "PurgeWorkerRelay", Handler: _Master_PurgeWorkerRelay_Handler, }, - { - MethodName: "RefreshWorkerTasks", - Handler: _Master_RefreshWorkerTasks_Handler, - }, { MethodName: "MigrateWorkerRelay", Handler: _Master_MigrateWorkerRelay_Handler, @@ -2861,6 +2861,10 @@ var _Master_serviceDesc = grpc.ServiceDesc{ MethodName: "CheckTask", Handler: _Master_CheckTask_Handler, }, + { + MethodName: "OperateMysqlWorker", + Handler: _Master_OperateMysqlWorker_Handler, + }, { MethodName: "RegisterWorker", Handler: _Master_RegisterWorker_Handler, @@ -2873,7 +2877,7 @@ var _Master_serviceDesc = grpc.ServiceDesc{ func (m *MigrateWorkerRelayRequest) Marshal() (dAtA []byte, err error) { size := m.Size() dAtA = make([]byte, size) - n, err := m.MarshalTo(dAtA) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) if err != nil { return nil, err } @@ -2881,34 +2885,41 @@ func (m *MigrateWorkerRelayRequest) Marshal() (dAtA []byte, err error) { } func (m *MigrateWorkerRelayRequest) MarshalTo(dAtA []byte) (int, error) { - var i int + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *MigrateWorkerRelayRequest) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) _ = i var l int _ = l - if len(m.BinlogName) > 0 { - dAtA[i] = 0xa - i++ - i = encodeVarintDmmaster(dAtA, i, uint64(len(m.BinlogName))) - i += copy(dAtA[i:], m.BinlogName) + if len(m.Worker) > 0 { + i -= len(m.Worker) + copy(dAtA[i:], m.Worker) + i = encodeVarintDmmaster(dAtA, i, uint64(len(m.Worker))) + i-- + dAtA[i] = 0x1a } if m.BinlogPos != 0 { - dAtA[i] = 0x10 - i++ i = encodeVarintDmmaster(dAtA, i, uint64(m.BinlogPos)) + i-- + dAtA[i] = 0x10 } - if len(m.Worker) > 0 { - dAtA[i] = 0x1a - i++ - i = encodeVarintDmmaster(dAtA, i, uint64(len(m.Worker))) - i += copy(dAtA[i:], m.Worker) + if len(m.BinlogName) > 0 { + i -= len(m.BinlogName) + copy(dAtA[i:], m.BinlogName) + i = encodeVarintDmmaster(dAtA, i, uint64(len(m.BinlogName))) + i-- + dAtA[i] = 0xa } - return i, nil + return len(dAtA) - i, nil } func (m *UpdateWorkerRelayConfigRequest) Marshal() (dAtA []byte, err error) { size := m.Size() dAtA = make([]byte, size) - n, err := m.MarshalTo(dAtA) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) if err != nil { return nil, err } @@ -2916,29 +2927,36 @@ func (m *UpdateWorkerRelayConfigRequest) Marshal() (dAtA []byte, err error) { } func (m *UpdateWorkerRelayConfigRequest) MarshalTo(dAtA []byte) (int, error) { - var i int + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *UpdateWorkerRelayConfigRequest) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) _ = i var l int _ = l - if len(m.Config) > 0 { - dAtA[i] = 0xa - i++ - i = encodeVarintDmmaster(dAtA, i, uint64(len(m.Config))) - i += copy(dAtA[i:], m.Config) - } if len(m.Worker) > 0 { - dAtA[i] = 0x12 - i++ + i -= len(m.Worker) + copy(dAtA[i:], m.Worker) i = encodeVarintDmmaster(dAtA, i, uint64(len(m.Worker))) - i += copy(dAtA[i:], m.Worker) + i-- + dAtA[i] = 0x12 + } + if len(m.Config) > 0 { + i -= len(m.Config) + copy(dAtA[i:], m.Config) + i = encodeVarintDmmaster(dAtA, i, uint64(len(m.Config))) + i-- + dAtA[i] = 0xa } - return i, nil + return len(dAtA) - i, nil } func (m *StartTaskRequest) Marshal() (dAtA []byte, err error) { size := m.Size() dAtA = make([]byte, size) - n, err := m.MarshalTo(dAtA) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) if err != nil { return nil, err } @@ -2946,38 +2964,38 @@ func (m *StartTaskRequest) Marshal() (dAtA []byte, err error) { } func (m *StartTaskRequest) MarshalTo(dAtA []byte) (int, error) { - var i int + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *StartTaskRequest) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) _ = i var l int _ = l - if len(m.Task) > 0 { - dAtA[i] = 0xa - i++ - i = encodeVarintDmmaster(dAtA, i, uint64(len(m.Task))) - i += copy(dAtA[i:], m.Task) - } - if len(m.Workers) > 0 { - for _, s := range m.Workers { + 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 - i++ - l = len(s) - for l >= 1<<7 { - dAtA[i] = uint8(uint64(l)&0x7f | 0x80) - l >>= 7 - i++ - } - dAtA[i] = uint8(l) - i++ - i += copy(dAtA[i:], s) } } - return i, nil + 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] = 0xa + } + return len(dAtA) - i, nil } func (m *StartTaskResponse) Marshal() (dAtA []byte, err error) { size := m.Size() dAtA = make([]byte, size) - n, err := m.MarshalTo(dAtA) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) if err != nil { return nil, err } @@ -2985,45 +3003,53 @@ func (m *StartTaskResponse) Marshal() (dAtA []byte, err error) { } func (m *StartTaskResponse) MarshalTo(dAtA []byte) (int, error) { - var i int + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *StartTaskResponse) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) _ = i var l int _ = l - if m.Result { - dAtA[i] = 0x8 - i++ - if m.Result { - dAtA[i] = 1 - } else { - dAtA[i] = 0 + 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 } - i++ } if len(m.Msg) > 0 { - dAtA[i] = 0x12 - i++ + i -= len(m.Msg) + copy(dAtA[i:], m.Msg) i = encodeVarintDmmaster(dAtA, i, uint64(len(m.Msg))) - i += copy(dAtA[i:], m.Msg) + i-- + dAtA[i] = 0x12 } - if len(m.Workers) > 0 { - for _, msg := range m.Workers { - dAtA[i] = 0x1a - i++ - i = encodeVarintDmmaster(dAtA, i, uint64(msg.Size())) - n, err := msg.MarshalTo(dAtA[i:]) - if err != nil { - return 0, err - } - i += n + if m.Result { + i-- + if m.Result { + dAtA[i] = 1 + } else { + dAtA[i] = 0 } + i-- + dAtA[i] = 0x8 } - return i, nil + return len(dAtA) - i, nil } func (m *UpdateMasterConfigRequest) Marshal() (dAtA []byte, err error) { size := m.Size() dAtA = make([]byte, size) - n, err := m.MarshalTo(dAtA) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) if err != nil { return nil, err } @@ -3031,23 +3057,29 @@ func (m *UpdateMasterConfigRequest) Marshal() (dAtA []byte, err error) { } func (m *UpdateMasterConfigRequest) MarshalTo(dAtA []byte) (int, error) { - var i int + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *UpdateMasterConfigRequest) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) _ = i var l int _ = l if len(m.Config) > 0 { - dAtA[i] = 0xa - i++ + i -= len(m.Config) + copy(dAtA[i:], m.Config) i = encodeVarintDmmaster(dAtA, i, uint64(len(m.Config))) - i += copy(dAtA[i:], m.Config) + i-- + dAtA[i] = 0xa } - return i, nil + return len(dAtA) - i, nil } func (m *UpdateMasterConfigResponse) Marshal() (dAtA []byte, err error) { size := m.Size() dAtA = make([]byte, size) - n, err := m.MarshalTo(dAtA) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) if err != nil { return nil, err } @@ -3055,45 +3087,53 @@ func (m *UpdateMasterConfigResponse) Marshal() (dAtA []byte, err error) { } func (m *UpdateMasterConfigResponse) MarshalTo(dAtA []byte) (int, error) { - var i int + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *UpdateMasterConfigResponse) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) _ = 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) + i = encodeVarintDmmaster(dAtA, i, uint64(len(m.Msg))) + i-- + dAtA[i] = 0x12 + } if m.Result { - dAtA[i] = 0x8 - i++ + i-- if m.Result { dAtA[i] = 1 } else { dAtA[i] = 0 } - i++ - } - if len(m.Msg) > 0 { - dAtA[i] = 0x12 - i++ - i = encodeVarintDmmaster(dAtA, i, uint64(len(m.Msg))) - i += copy(dAtA[i:], m.Msg) - } - if len(m.Workers) > 0 { - for _, msg := range m.Workers { - dAtA[i] = 0x1a - i++ - i = encodeVarintDmmaster(dAtA, i, uint64(msg.Size())) - n, err := msg.MarshalTo(dAtA[i:]) - if err != nil { - return 0, err - } - i += n - } + i-- + dAtA[i] = 0x8 } - return i, nil + return len(dAtA) - i, nil } func (m *OperateTaskRequest) Marshal() (dAtA []byte, err error) { size := m.Size() dAtA = make([]byte, size) - n, err := m.MarshalTo(dAtA) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) if err != nil { return nil, err } @@ -3101,43 +3141,43 @@ func (m *OperateTaskRequest) Marshal() (dAtA []byte, err error) { } func (m *OperateTaskRequest) MarshalTo(dAtA []byte) (int, error) { - var i int + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *OperateTaskRequest) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) _ = i var l int _ = l - if m.Op != 0 { - dAtA[i] = 0x8 - i++ - i = encodeVarintDmmaster(dAtA, i, uint64(m.Op)) + 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 + } } if len(m.Name) > 0 { - dAtA[i] = 0x12 - i++ + i -= len(m.Name) + copy(dAtA[i:], m.Name) i = encodeVarintDmmaster(dAtA, i, uint64(len(m.Name))) - i += copy(dAtA[i:], m.Name) + i-- + dAtA[i] = 0x12 } - if len(m.Workers) > 0 { - for _, s := range m.Workers { - dAtA[i] = 0x1a - i++ - l = len(s) - for l >= 1<<7 { - dAtA[i] = uint8(uint64(l)&0x7f | 0x80) - l >>= 7 - i++ - } - dAtA[i] = uint8(l) - i++ - i += copy(dAtA[i:], s) - } + if m.Op != 0 { + i = encodeVarintDmmaster(dAtA, i, uint64(m.Op)) + i-- + dAtA[i] = 0x8 } - return i, nil + return len(dAtA) - i, nil } func (m *OperateTaskResponse) Marshal() (dAtA []byte, err error) { size := m.Size() dAtA = make([]byte, size) - n, err := m.MarshalTo(dAtA) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) if err != nil { return nil, err } @@ -3145,50 +3185,58 @@ func (m *OperateTaskResponse) Marshal() (dAtA []byte, err error) { } func (m *OperateTaskResponse) MarshalTo(dAtA []byte) (int, error) { - var i int + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *OperateTaskResponse) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) _ = i var l int _ = l - if m.Op != 0 { - dAtA[i] = 0x8 - i++ - i = encodeVarintDmmaster(dAtA, i, uint64(m.Op)) + 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] = 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 { - dAtA[i] = 0x10 - i++ + i-- if m.Result { dAtA[i] = 1 } else { dAtA[i] = 0 } - i++ - } - if len(m.Msg) > 0 { - dAtA[i] = 0x1a - i++ - i = encodeVarintDmmaster(dAtA, i, uint64(len(m.Msg))) - i += copy(dAtA[i:], m.Msg) + i-- + dAtA[i] = 0x10 } - if len(m.Workers) > 0 { - for _, msg := range m.Workers { - dAtA[i] = 0x22 - i++ - i = encodeVarintDmmaster(dAtA, i, uint64(msg.Size())) - n, err := msg.MarshalTo(dAtA[i:]) - if err != nil { - return 0, err - } - i += n - } + if m.Op != 0 { + i = encodeVarintDmmaster(dAtA, i, uint64(m.Op)) + i-- + dAtA[i] = 0x8 } - return i, nil + return len(dAtA) - i, nil } func (m *UpdateTaskRequest) Marshal() (dAtA []byte, err error) { size := m.Size() dAtA = make([]byte, size) - n, err := m.MarshalTo(dAtA) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) if err != nil { return nil, err } @@ -3196,38 +3244,38 @@ func (m *UpdateTaskRequest) Marshal() (dAtA []byte, err error) { } func (m *UpdateTaskRequest) MarshalTo(dAtA []byte) (int, error) { - var i int + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *UpdateTaskRequest) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) _ = i var l int _ = l - if len(m.Task) > 0 { - dAtA[i] = 0xa - i++ - i = encodeVarintDmmaster(dAtA, i, uint64(len(m.Task))) - i += copy(dAtA[i:], m.Task) - } - if len(m.Workers) > 0 { - for _, s := range m.Workers { + 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 - i++ - l = len(s) - for l >= 1<<7 { - dAtA[i] = uint8(uint64(l)&0x7f | 0x80) - l >>= 7 - i++ - } - dAtA[i] = uint8(l) - i++ - i += copy(dAtA[i:], s) } } - return i, nil + 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] = 0xa + } + return len(dAtA) - i, nil } func (m *UpdateTaskResponse) Marshal() (dAtA []byte, err error) { size := m.Size() dAtA = make([]byte, size) - n, err := m.MarshalTo(dAtA) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) if err != nil { return nil, err } @@ -3235,45 +3283,53 @@ func (m *UpdateTaskResponse) Marshal() (dAtA []byte, err error) { } func (m *UpdateTaskResponse) MarshalTo(dAtA []byte) (int, error) { - var i int + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *UpdateTaskResponse) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) _ = i var l int _ = l - if m.Result { - dAtA[i] = 0x8 - i++ - if m.Result { - dAtA[i] = 1 - } else { - dAtA[i] = 0 + 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 } - i++ } if len(m.Msg) > 0 { - dAtA[i] = 0x12 - i++ + i -= len(m.Msg) + copy(dAtA[i:], m.Msg) i = encodeVarintDmmaster(dAtA, i, uint64(len(m.Msg))) - i += copy(dAtA[i:], m.Msg) + i-- + dAtA[i] = 0x12 } - if len(m.Workers) > 0 { - for _, msg := range m.Workers { - dAtA[i] = 0x1a - i++ - i = encodeVarintDmmaster(dAtA, i, uint64(msg.Size())) - n, err := msg.MarshalTo(dAtA[i:]) - if err != nil { - return 0, err - } - i += n + if m.Result { + i-- + if m.Result { + dAtA[i] = 1 + } else { + dAtA[i] = 0 } + i-- + dAtA[i] = 0x8 } - return i, nil + return len(dAtA) - i, nil } func (m *QueryStatusListRequest) Marshal() (dAtA []byte, err error) { size := m.Size() dAtA = make([]byte, size) - n, err := m.MarshalTo(dAtA) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) if err != nil { return nil, err } @@ -3281,38 +3337,38 @@ func (m *QueryStatusListRequest) Marshal() (dAtA []byte, err error) { } func (m *QueryStatusListRequest) MarshalTo(dAtA []byte) (int, error) { - var i int + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *QueryStatusListRequest) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) _ = i var l int _ = l - if len(m.Name) > 0 { - dAtA[i] = 0xa - i++ - i = encodeVarintDmmaster(dAtA, i, uint64(len(m.Name))) - i += copy(dAtA[i:], m.Name) - } - if len(m.Workers) > 0 { - for _, s := range m.Workers { + 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 - i++ - l = len(s) - for l >= 1<<7 { - dAtA[i] = uint8(uint64(l)&0x7f | 0x80) - l >>= 7 - i++ - } - dAtA[i] = uint8(l) - i++ - i += copy(dAtA[i:], s) } } - return i, nil + if len(m.Name) > 0 { + i -= len(m.Name) + copy(dAtA[i:], m.Name) + i = encodeVarintDmmaster(dAtA, i, uint64(len(m.Name))) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil } func (m *QueryStatusListResponse) Marshal() (dAtA []byte, err error) { size := m.Size() dAtA = make([]byte, size) - n, err := m.MarshalTo(dAtA) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) if err != nil { return nil, err } @@ -3320,45 +3376,53 @@ func (m *QueryStatusListResponse) Marshal() (dAtA []byte, err error) { } func (m *QueryStatusListResponse) MarshalTo(dAtA []byte) (int, error) { - var i int + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *QueryStatusListResponse) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) _ = i var l int _ = l - if m.Result { - dAtA[i] = 0x8 - i++ - if m.Result { - dAtA[i] = 1 - } else { - dAtA[i] = 0 + 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 } - i++ } if len(m.Msg) > 0 { - dAtA[i] = 0x12 - i++ + i -= len(m.Msg) + copy(dAtA[i:], m.Msg) i = encodeVarintDmmaster(dAtA, i, uint64(len(m.Msg))) - i += copy(dAtA[i:], m.Msg) + i-- + dAtA[i] = 0x12 } - if len(m.Workers) > 0 { - for _, msg := range m.Workers { - dAtA[i] = 0x1a - i++ - i = encodeVarintDmmaster(dAtA, i, uint64(msg.Size())) - n, err := msg.MarshalTo(dAtA[i:]) - if err != nil { - return 0, err - } - i += n + if m.Result { + i-- + if m.Result { + dAtA[i] = 1 + } else { + dAtA[i] = 0 } + i-- + dAtA[i] = 0x8 } - return i, nil + return len(dAtA) - i, nil } func (m *QueryErrorListRequest) Marshal() (dAtA []byte, err error) { size := m.Size() dAtA = make([]byte, size) - n, err := m.MarshalTo(dAtA) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) if err != nil { return nil, err } @@ -3366,38 +3430,38 @@ func (m *QueryErrorListRequest) Marshal() (dAtA []byte, err error) { } func (m *QueryErrorListRequest) MarshalTo(dAtA []byte) (int, error) { - var i int + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *QueryErrorListRequest) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) _ = i var l int _ = l - if len(m.Name) > 0 { - dAtA[i] = 0xa - i++ - i = encodeVarintDmmaster(dAtA, i, uint64(len(m.Name))) - i += copy(dAtA[i:], m.Name) - } - if len(m.Workers) > 0 { - for _, s := range m.Workers { + 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 - i++ - l = len(s) - for l >= 1<<7 { - dAtA[i] = uint8(uint64(l)&0x7f | 0x80) - l >>= 7 - i++ - } - dAtA[i] = uint8(l) - i++ - i += copy(dAtA[i:], s) } } - return i, nil + if len(m.Name) > 0 { + i -= len(m.Name) + copy(dAtA[i:], m.Name) + i = encodeVarintDmmaster(dAtA, i, uint64(len(m.Name))) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil } func (m *QueryErrorListResponse) Marshal() (dAtA []byte, err error) { size := m.Size() dAtA = make([]byte, size) - n, err := m.MarshalTo(dAtA) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) if err != nil { return nil, err } @@ -3405,45 +3469,53 @@ func (m *QueryErrorListResponse) Marshal() (dAtA []byte, err error) { } func (m *QueryErrorListResponse) MarshalTo(dAtA []byte) (int, error) { - var i int + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *QueryErrorListResponse) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) _ = i var l int _ = l - if m.Result { - dAtA[i] = 0x8 - i++ - if m.Result { - dAtA[i] = 1 - } else { - dAtA[i] = 0 + 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 } - i++ } if len(m.Msg) > 0 { - dAtA[i] = 0x12 - i++ + i -= len(m.Msg) + copy(dAtA[i:], m.Msg) i = encodeVarintDmmaster(dAtA, i, uint64(len(m.Msg))) - i += copy(dAtA[i:], m.Msg) + i-- + dAtA[i] = 0x12 } - if len(m.Workers) > 0 { - for _, msg := range m.Workers { - dAtA[i] = 0x1a - i++ - i = encodeVarintDmmaster(dAtA, i, uint64(msg.Size())) - n, err := msg.MarshalTo(dAtA[i:]) - if err != nil { - return 0, err - } - i += n + if m.Result { + i-- + if m.Result { + dAtA[i] = 1 + } else { + dAtA[i] = 0 } + i-- + dAtA[i] = 0x8 } - return i, nil + return len(dAtA) - i, nil } func (m *ShowDDLLocksRequest) Marshal() (dAtA []byte, err error) { size := m.Size() dAtA = make([]byte, size) - n, err := m.MarshalTo(dAtA) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) if err != nil { return nil, err } @@ -3451,38 +3523,38 @@ func (m *ShowDDLLocksRequest) Marshal() (dAtA []byte, err error) { } func (m *ShowDDLLocksRequest) MarshalTo(dAtA []byte) (int, error) { - var i int + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *ShowDDLLocksRequest) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) _ = i var l int _ = l - if len(m.Task) > 0 { - dAtA[i] = 0xa - i++ - i = encodeVarintDmmaster(dAtA, i, uint64(len(m.Task))) - i += copy(dAtA[i:], m.Task) - } - if len(m.Workers) > 0 { - for _, s := range m.Workers { + 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 - i++ - l = len(s) - for l >= 1<<7 { - dAtA[i] = uint8(uint64(l)&0x7f | 0x80) - l >>= 7 - i++ - } - dAtA[i] = uint8(l) - i++ - i += copy(dAtA[i:], s) } } - return i, nil + 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] = 0xa + } + return len(dAtA) - i, nil } func (m *DDLLock) Marshal() (dAtA []byte, err error) { size := m.Size() dAtA = make([]byte, size) - n, err := m.MarshalTo(dAtA) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) if err != nil { return nil, err } @@ -3490,80 +3562,70 @@ func (m *DDLLock) Marshal() (dAtA []byte, err error) { } func (m *DDLLock) MarshalTo(dAtA []byte) (int, error) { - var i int + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *DDLLock) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) _ = i var l int _ = l - if len(m.ID) > 0 { - dAtA[i] = 0xa - i++ - i = encodeVarintDmmaster(dAtA, i, uint64(len(m.ID))) - i += copy(dAtA[i:], m.ID) - } - if len(m.Task) > 0 { - dAtA[i] = 0x12 - i++ - i = encodeVarintDmmaster(dAtA, i, uint64(len(m.Task))) - i += copy(dAtA[i:], m.Task) - } - if len(m.Owner) > 0 { - dAtA[i] = 0x1a - i++ - i = encodeVarintDmmaster(dAtA, i, uint64(len(m.Owner))) - i += copy(dAtA[i:], m.Owner) - } - if len(m.DDLs) > 0 { - for _, s := range m.DDLs { - dAtA[i] = 0x22 - i++ - l = len(s) - for l >= 1<<7 { - dAtA[i] = uint8(uint64(l)&0x7f | 0x80) - l >>= 7 - i++ - } - dAtA[i] = uint8(l) - i++ - i += copy(dAtA[i:], s) + if len(m.Unsynced) > 0 { + for iNdEx := len(m.Unsynced) - 1; iNdEx >= 0; iNdEx-- { + i -= len(m.Unsynced[iNdEx]) + copy(dAtA[i:], m.Unsynced[iNdEx]) + i = encodeVarintDmmaster(dAtA, i, uint64(len(m.Unsynced[iNdEx]))) + i-- + dAtA[i] = 0x32 } } if len(m.Synced) > 0 { - for _, s := range m.Synced { + for iNdEx := len(m.Synced) - 1; iNdEx >= 0; iNdEx-- { + i -= len(m.Synced[iNdEx]) + copy(dAtA[i:], m.Synced[iNdEx]) + i = encodeVarintDmmaster(dAtA, i, uint64(len(m.Synced[iNdEx]))) + i-- dAtA[i] = 0x2a - i++ - l = len(s) - for l >= 1<<7 { - dAtA[i] = uint8(uint64(l)&0x7f | 0x80) - l >>= 7 - i++ - } - dAtA[i] = uint8(l) - i++ - i += copy(dAtA[i:], s) } } - if len(m.Unsynced) > 0 { - for _, s := range m.Unsynced { - dAtA[i] = 0x32 - i++ - l = len(s) - for l >= 1<<7 { - dAtA[i] = uint8(uint64(l)&0x7f | 0x80) - l >>= 7 - i++ - } - dAtA[i] = uint8(l) - i++ - i += copy(dAtA[i:], s) + if len(m.DDLs) > 0 { + for iNdEx := len(m.DDLs) - 1; iNdEx >= 0; iNdEx-- { + i -= len(m.DDLs[iNdEx]) + copy(dAtA[i:], m.DDLs[iNdEx]) + i = encodeVarintDmmaster(dAtA, i, uint64(len(m.DDLs[iNdEx]))) + i-- + dAtA[i] = 0x22 } } - return i, nil + if len(m.Owner) > 0 { + i -= len(m.Owner) + copy(dAtA[i:], m.Owner) + i = encodeVarintDmmaster(dAtA, i, uint64(len(m.Owner))) + 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.ID) > 0 { + i -= len(m.ID) + copy(dAtA[i:], m.ID) + i = encodeVarintDmmaster(dAtA, i, uint64(len(m.ID))) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil } func (m *ShowDDLLocksResponse) Marshal() (dAtA []byte, err error) { size := m.Size() dAtA = make([]byte, size) - n, err := m.MarshalTo(dAtA) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) if err != nil { return nil, err } @@ -3571,45 +3633,53 @@ func (m *ShowDDLLocksResponse) Marshal() (dAtA []byte, err error) { } func (m *ShowDDLLocksResponse) MarshalTo(dAtA []byte) (int, error) { - var i int + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *ShowDDLLocksResponse) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) _ = i var l int _ = l - if m.Result { - dAtA[i] = 0x8 - i++ - if m.Result { - dAtA[i] = 1 - } else { - dAtA[i] = 0 + if len(m.Locks) > 0 { + for iNdEx := len(m.Locks) - 1; iNdEx >= 0; iNdEx-- { + { + size, err := m.Locks[iNdEx].MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintDmmaster(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x1a } - i++ } if len(m.Msg) > 0 { - dAtA[i] = 0x12 - i++ + i -= len(m.Msg) + copy(dAtA[i:], m.Msg) i = encodeVarintDmmaster(dAtA, i, uint64(len(m.Msg))) - i += copy(dAtA[i:], m.Msg) + i-- + dAtA[i] = 0x12 } - if len(m.Locks) > 0 { - for _, msg := range m.Locks { - dAtA[i] = 0x1a - i++ - i = encodeVarintDmmaster(dAtA, i, uint64(msg.Size())) - n, err := msg.MarshalTo(dAtA[i:]) - if err != nil { - return 0, err - } - i += n + if m.Result { + i-- + if m.Result { + dAtA[i] = 1 + } else { + dAtA[i] = 0 } + i-- + dAtA[i] = 0x8 } - return i, nil + return len(dAtA) - i, nil } func (m *UnlockDDLLockRequest) Marshal() (dAtA []byte, err error) { size := m.Size() dAtA = make([]byte, size) - n, err := m.MarshalTo(dAtA) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) if err != nil { return nil, err } @@ -3617,54 +3687,55 @@ func (m *UnlockDDLLockRequest) Marshal() (dAtA []byte, err error) { } func (m *UnlockDDLLockRequest) MarshalTo(dAtA []byte) (int, error) { - var i int + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *UnlockDDLLockRequest) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) _ = i var l int _ = l - if len(m.ID) > 0 { - dAtA[i] = 0xa - i++ - i = encodeVarintDmmaster(dAtA, i, uint64(len(m.ID))) - i += copy(dAtA[i:], m.ID) - } - if len(m.ReplaceOwner) > 0 { - dAtA[i] = 0x12 - i++ - i = encodeVarintDmmaster(dAtA, i, uint64(len(m.ReplaceOwner))) - i += copy(dAtA[i:], m.ReplaceOwner) - } - if len(m.Workers) > 0 { - for _, s := range m.Workers { - dAtA[i] = 0x1a - i++ - l = len(s) - for l >= 1<<7 { - dAtA[i] = uint8(uint64(l)&0x7f | 0x80) - l >>= 7 - i++ - } - dAtA[i] = uint8(l) - i++ - i += copy(dAtA[i:], s) - } - } if m.ForceRemove { - dAtA[i] = 0x20 - i++ + i-- if m.ForceRemove { dAtA[i] = 1 } else { dAtA[i] = 0 } - i++ + 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 + } + } + if len(m.ReplaceOwner) > 0 { + i -= len(m.ReplaceOwner) + copy(dAtA[i:], m.ReplaceOwner) + i = encodeVarintDmmaster(dAtA, i, uint64(len(m.ReplaceOwner))) + i-- + dAtA[i] = 0x12 } - return i, nil + if len(m.ID) > 0 { + i -= len(m.ID) + copy(dAtA[i:], m.ID) + i = encodeVarintDmmaster(dAtA, i, uint64(len(m.ID))) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil } func (m *UnlockDDLLockResponse) Marshal() (dAtA []byte, err error) { size := m.Size() dAtA = make([]byte, size) - n, err := m.MarshalTo(dAtA) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) if err != nil { return nil, err } @@ -3672,45 +3743,53 @@ func (m *UnlockDDLLockResponse) Marshal() (dAtA []byte, err error) { } func (m *UnlockDDLLockResponse) MarshalTo(dAtA []byte) (int, error) { - var i int + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *UnlockDDLLockResponse) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) _ = i var l int _ = l - if m.Result { - dAtA[i] = 0x8 - i++ - if m.Result { - dAtA[i] = 1 - } else { - dAtA[i] = 0 + 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 } - i++ } if len(m.Msg) > 0 { - dAtA[i] = 0x12 - i++ + i -= len(m.Msg) + copy(dAtA[i:], m.Msg) i = encodeVarintDmmaster(dAtA, i, uint64(len(m.Msg))) - i += copy(dAtA[i:], m.Msg) + i-- + dAtA[i] = 0x12 } - if len(m.Workers) > 0 { - for _, msg := range m.Workers { - dAtA[i] = 0x1a - i++ - i = encodeVarintDmmaster(dAtA, i, uint64(msg.Size())) - n, err := msg.MarshalTo(dAtA[i:]) - if err != nil { - return 0, err - } - i += n + if m.Result { + i-- + if m.Result { + dAtA[i] = 1 + } else { + dAtA[i] = 0 } + i-- + dAtA[i] = 0x8 } - return i, nil + return len(dAtA) - i, nil } func (m *BreakWorkerDDLLockRequest) Marshal() (dAtA []byte, err error) { size := m.Size() dAtA = make([]byte, size) - n, err := m.MarshalTo(dAtA) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) if err != nil { return nil, err } @@ -3718,64 +3797,65 @@ func (m *BreakWorkerDDLLockRequest) Marshal() (dAtA []byte, err error) { } func (m *BreakWorkerDDLLockRequest) MarshalTo(dAtA []byte) (int, error) { - var i int + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *BreakWorkerDDLLockRequest) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) _ = i var l int _ = l - if len(m.Workers) > 0 { - for _, s := range m.Workers { - dAtA[i] = 0xa - i++ - l = len(s) - for l >= 1<<7 { - dAtA[i] = uint8(uint64(l)&0x7f | 0x80) - l >>= 7 - i++ - } - dAtA[i] = uint8(l) - i++ - i += copy(dAtA[i:], s) + if m.SkipDDL { + i-- + if m.SkipDDL { + dAtA[i] = 1 + } else { + dAtA[i] = 0 } - } - if len(m.Task) > 0 { - dAtA[i] = 0x12 - i++ - i = encodeVarintDmmaster(dAtA, i, uint64(len(m.Task))) - i += copy(dAtA[i:], m.Task) - } - if len(m.RemoveLockID) > 0 { - dAtA[i] = 0x1a - i++ - i = encodeVarintDmmaster(dAtA, i, uint64(len(m.RemoveLockID))) - i += copy(dAtA[i:], m.RemoveLockID) + i-- + dAtA[i] = 0x28 } if m.ExecDDL { - dAtA[i] = 0x20 - i++ + i-- if m.ExecDDL { dAtA[i] = 1 } else { dAtA[i] = 0 } - i++ + i-- + dAtA[i] = 0x20 } - if m.SkipDDL { - dAtA[i] = 0x28 - i++ - if m.SkipDDL { - dAtA[i] = 1 - } else { - dAtA[i] = 0 + 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]) + copy(dAtA[i:], m.Sources[iNdEx]) + i = encodeVarintDmmaster(dAtA, i, uint64(len(m.Sources[iNdEx]))) + i-- + dAtA[i] = 0xa } - i++ } - return i, nil + return len(dAtA) - i, nil } func (m *BreakWorkerDDLLockResponse) Marshal() (dAtA []byte, err error) { size := m.Size() dAtA = make([]byte, size) - n, err := m.MarshalTo(dAtA) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) if err != nil { return nil, err } @@ -3783,45 +3863,53 @@ func (m *BreakWorkerDDLLockResponse) Marshal() (dAtA []byte, err error) { } func (m *BreakWorkerDDLLockResponse) MarshalTo(dAtA []byte) (int, error) { - var i int + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *BreakWorkerDDLLockResponse) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) _ = i var l int _ = l - if m.Result { - dAtA[i] = 0x8 - i++ - if m.Result { - dAtA[i] = 1 - } else { - dAtA[i] = 0 + 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 } - i++ } if len(m.Msg) > 0 { - dAtA[i] = 0x12 - i++ + i -= len(m.Msg) + copy(dAtA[i:], m.Msg) i = encodeVarintDmmaster(dAtA, i, uint64(len(m.Msg))) - i += copy(dAtA[i:], m.Msg) + i-- + dAtA[i] = 0x12 } - if len(m.Workers) > 0 { - for _, msg := range m.Workers { - dAtA[i] = 0x1a - i++ - i = encodeVarintDmmaster(dAtA, i, uint64(msg.Size())) - n, err := msg.MarshalTo(dAtA[i:]) - if err != nil { - return 0, err - } - i += n + if m.Result { + i-- + if m.Result { + dAtA[i] = 1 + } else { + dAtA[i] = 0 } + i-- + dAtA[i] = 0x8 } - return i, nil + return len(dAtA) - i, nil } func (m *SwitchWorkerRelayMasterRequest) Marshal() (dAtA []byte, err error) { size := m.Size() dAtA = make([]byte, size) - n, err := m.MarshalTo(dAtA) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) if err != nil { return nil, err } @@ -3829,32 +3917,31 @@ func (m *SwitchWorkerRelayMasterRequest) Marshal() (dAtA []byte, err error) { } func (m *SwitchWorkerRelayMasterRequest) MarshalTo(dAtA []byte) (int, error) { - var i int + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *SwitchWorkerRelayMasterRequest) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) _ = i var l int _ = l - if len(m.Workers) > 0 { - for _, s := range m.Workers { + 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] = 0xa - i++ - l = len(s) - for l >= 1<<7 { - dAtA[i] = uint8(uint64(l)&0x7f | 0x80) - l >>= 7 - i++ - } - dAtA[i] = uint8(l) - i++ - i += copy(dAtA[i:], s) } } - return i, nil + return len(dAtA) - i, nil } func (m *SwitchWorkerRelayMasterResponse) Marshal() (dAtA []byte, err error) { size := m.Size() dAtA = make([]byte, size) - n, err := m.MarshalTo(dAtA) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) if err != nil { return nil, err } @@ -3862,222 +3949,149 @@ func (m *SwitchWorkerRelayMasterResponse) Marshal() (dAtA []byte, err error) { } func (m *SwitchWorkerRelayMasterResponse) MarshalTo(dAtA []byte) (int, error) { - var i int - _ = i - var l int - _ = l - if m.Result { - dAtA[i] = 0x8 - i++ - if m.Result { - dAtA[i] = 1 - } else { - dAtA[i] = 0 - } - i++ - } - if len(m.Msg) > 0 { - dAtA[i] = 0x12 - i++ - i = encodeVarintDmmaster(dAtA, i, uint64(len(m.Msg))) - i += copy(dAtA[i:], m.Msg) - } - if len(m.Workers) > 0 { - for _, msg := range m.Workers { - dAtA[i] = 0x1a - i++ - i = encodeVarintDmmaster(dAtA, i, uint64(msg.Size())) - n, err := msg.MarshalTo(dAtA[i:]) - if err != nil { - return 0, err - } - i += n - } - } - return i, nil -} - -func (m *OperateWorkerRelayRequest) Marshal() (dAtA []byte, err error) { size := m.Size() - dAtA = make([]byte, size) - n, err := m.MarshalTo(dAtA) - if err != nil { - return nil, err - } - return dAtA[:n], nil + return m.MarshalToSizedBuffer(dAtA[:size]) } -func (m *OperateWorkerRelayRequest) MarshalTo(dAtA []byte) (int, error) { - var i int +func (m *SwitchWorkerRelayMasterResponse) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) _ = i var l int _ = l - if m.Op != 0 { - dAtA[i] = 0x8 - i++ - i = encodeVarintDmmaster(dAtA, i, uint64(m.Op)) - } if len(m.Workers) > 0 { - for _, s := range m.Workers { - dAtA[i] = 0x12 - i++ - l = len(s) - for l >= 1<<7 { - dAtA[i] = uint8(uint64(l)&0x7f | 0x80) - l >>= 7 - i++ + 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)) } - dAtA[i] = uint8(l) - i++ - i += copy(dAtA[i:], s) + i-- + dAtA[i] = 0x1a } } - return i, nil -} - -func (m *OperateWorkerRelayResponse) Marshal() (dAtA []byte, err error) { - size := m.Size() - dAtA = make([]byte, size) - n, err := m.MarshalTo(dAtA) - if err != nil { - return nil, err - } - return dAtA[:n], nil -} - -func (m *OperateWorkerRelayResponse) MarshalTo(dAtA []byte) (int, error) { - var i int - _ = i - var l int - _ = l - if m.Op != 0 { - dAtA[i] = 0x8 - i++ - i = encodeVarintDmmaster(dAtA, i, uint64(m.Op)) + if len(m.Msg) > 0 { + i -= len(m.Msg) + copy(dAtA[i:], m.Msg) + i = encodeVarintDmmaster(dAtA, i, uint64(len(m.Msg))) + i-- + dAtA[i] = 0x12 } if m.Result { - dAtA[i] = 0x10 - i++ + i-- if m.Result { dAtA[i] = 1 } else { dAtA[i] = 0 } - i++ - } - if len(m.Msg) > 0 { - dAtA[i] = 0x1a - i++ - i = encodeVarintDmmaster(dAtA, i, uint64(len(m.Msg))) - i += copy(dAtA[i:], m.Msg) - } - if len(m.Workers) > 0 { - for _, msg := range m.Workers { - dAtA[i] = 0x22 - i++ - i = encodeVarintDmmaster(dAtA, i, uint64(msg.Size())) - n, err := msg.MarshalTo(dAtA[i:]) - if err != nil { - return 0, err - } - i += n - } + i-- + dAtA[i] = 0x8 } - return i, nil + return len(dAtA) - i, nil } -func (m *RefreshWorkerTasksRequest) Marshal() (dAtA []byte, err error) { +func (m *OperateWorkerRelayRequest) Marshal() (dAtA []byte, err error) { size := m.Size() dAtA = make([]byte, size) - n, err := m.MarshalTo(dAtA) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) if err != nil { return nil, err } return dAtA[:n], nil } -func (m *RefreshWorkerTasksRequest) MarshalTo(dAtA []byte) (int, error) { - var i int - _ = i - var l int - _ = l - return i, nil -} - -func (m *RefreshWorkerTasksMsg) Marshal() (dAtA []byte, err error) { +func (m *OperateWorkerRelayRequest) MarshalTo(dAtA []byte) (int, error) { size := m.Size() - dAtA = make([]byte, size) - n, err := m.MarshalTo(dAtA) - if err != nil { - return nil, err - } - return dAtA[:n], nil + return m.MarshalToSizedBuffer(dAtA[:size]) } -func (m *RefreshWorkerTasksMsg) MarshalTo(dAtA []byte) (int, error) { - var i int +func (m *OperateWorkerRelayRequest) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) _ = i var l int _ = l - if len(m.Worker) > 0 { - dAtA[i] = 0xa - i++ - i = encodeVarintDmmaster(dAtA, i, uint64(len(m.Worker))) - i += copy(dAtA[i:], m.Worker) + 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 len(m.Msg) > 0 { - dAtA[i] = 0x12 - i++ - i = encodeVarintDmmaster(dAtA, i, uint64(len(m.Msg))) - i += copy(dAtA[i:], m.Msg) + if m.Op != 0 { + i = encodeVarintDmmaster(dAtA, i, uint64(m.Op)) + i-- + dAtA[i] = 0x8 } - return i, nil + return len(dAtA) - i, nil } -func (m *RefreshWorkerTasksResponse) Marshal() (dAtA []byte, err error) { +func (m *OperateWorkerRelayResponse) Marshal() (dAtA []byte, err error) { size := m.Size() dAtA = make([]byte, size) - n, err := m.MarshalTo(dAtA) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) if err != nil { return nil, err } return dAtA[:n], nil } -func (m *RefreshWorkerTasksResponse) MarshalTo(dAtA []byte) (int, error) { - var i int +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.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] = 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 { - dAtA[i] = 0x8 - i++ + i-- if m.Result { dAtA[i] = 1 } else { dAtA[i] = 0 } - i++ + i-- + dAtA[i] = 0x10 } - if len(m.Workers) > 0 { - for _, msg := range m.Workers { - dAtA[i] = 0x12 - i++ - i = encodeVarintDmmaster(dAtA, i, uint64(msg.Size())) - n, err := msg.MarshalTo(dAtA[i:]) - if err != nil { - return 0, err - } - i += n - } + if m.Op != 0 { + i = encodeVarintDmmaster(dAtA, i, uint64(m.Op)) + i-- + dAtA[i] = 0x8 } - return i, nil + return len(dAtA) - i, nil } func (m *HandleSQLsRequest) Marshal() (dAtA []byte, err error) { size := m.Size() dAtA = make([]byte, size) - n, err := m.MarshalTo(dAtA) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) if err != nil { return nil, err } @@ -4085,71 +4099,74 @@ func (m *HandleSQLsRequest) Marshal() (dAtA []byte, err error) { } func (m *HandleSQLsRequest) MarshalTo(dAtA []byte) (int, error) { - var i int + 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 len(m.Name) > 0 { - dAtA[i] = 0xa - i++ - i = encodeVarintDmmaster(dAtA, i, uint64(len(m.Name))) - i += copy(dAtA[i:], m.Name) - } - if m.Op != 0 { - dAtA[i] = 0x10 - i++ - i = encodeVarintDmmaster(dAtA, i, uint64(m.Op)) - } - if len(m.Args) > 0 { - for _, s := range m.Args { - dAtA[i] = 0x1a - i++ - l = len(s) - for l >= 1<<7 { - dAtA[i] = uint8(uint64(l)&0x7f | 0x80) - l >>= 7 - i++ - } - dAtA[i] = uint8(l) - i++ - i += copy(dAtA[i:], s) + if m.Sharding { + i-- + if m.Sharding { + dAtA[i] = 1 + } else { + dAtA[i] = 0 } + i-- + dAtA[i] = 0x38 } - if len(m.BinlogPos) > 0 { - dAtA[i] = 0x22 - i++ - i = encodeVarintDmmaster(dAtA, i, uint64(len(m.BinlogPos))) - i += copy(dAtA[i:], m.BinlogPos) + if len(m.SqlPattern) > 0 { + i -= len(m.SqlPattern) + copy(dAtA[i:], m.SqlPattern) + i = encodeVarintDmmaster(dAtA, i, uint64(len(m.SqlPattern))) + i-- + dAtA[i] = 0x32 } if len(m.Worker) > 0 { - dAtA[i] = 0x2a - i++ + i -= len(m.Worker) + copy(dAtA[i:], m.Worker) i = encodeVarintDmmaster(dAtA, i, uint64(len(m.Worker))) - i += copy(dAtA[i:], m.Worker) + i-- + dAtA[i] = 0x2a } - if len(m.SqlPattern) > 0 { - dAtA[i] = 0x32 - i++ - i = encodeVarintDmmaster(dAtA, i, uint64(len(m.SqlPattern))) - i += copy(dAtA[i:], m.SqlPattern) + if len(m.BinlogPos) > 0 { + i -= len(m.BinlogPos) + copy(dAtA[i:], m.BinlogPos) + i = encodeVarintDmmaster(dAtA, i, uint64(len(m.BinlogPos))) + i-- + dAtA[i] = 0x22 } - if m.Sharding { - dAtA[i] = 0x38 - i++ - if m.Sharding { - dAtA[i] = 1 - } else { - dAtA[i] = 0 + if len(m.Args) > 0 { + for iNdEx := len(m.Args) - 1; iNdEx >= 0; iNdEx-- { + i -= len(m.Args[iNdEx]) + copy(dAtA[i:], m.Args[iNdEx]) + i = encodeVarintDmmaster(dAtA, i, uint64(len(m.Args[iNdEx]))) + i-- + dAtA[i] = 0x1a } - i++ } - return i, nil + if m.Op != 0 { + i = encodeVarintDmmaster(dAtA, i, uint64(m.Op)) + i-- + dAtA[i] = 0x10 + } + if len(m.Name) > 0 { + i -= len(m.Name) + copy(dAtA[i:], m.Name) + i = encodeVarintDmmaster(dAtA, i, uint64(len(m.Name))) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil } func (m *HandleSQLsResponse) Marshal() (dAtA []byte, err error) { size := m.Size() dAtA = make([]byte, size) - n, err := m.MarshalTo(dAtA) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) if err != nil { return nil, err } @@ -4157,45 +4174,53 @@ func (m *HandleSQLsResponse) Marshal() (dAtA []byte, err error) { } func (m *HandleSQLsResponse) MarshalTo(dAtA []byte) (int, error) { - var i int + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *HandleSQLsResponse) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) _ = i var l int _ = l - if m.Result { - dAtA[i] = 0x8 - i++ - if m.Result { - dAtA[i] = 1 - } else { - dAtA[i] = 0 + 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 } - i++ } if len(m.Msg) > 0 { - dAtA[i] = 0x12 - i++ + i -= len(m.Msg) + copy(dAtA[i:], m.Msg) i = encodeVarintDmmaster(dAtA, i, uint64(len(m.Msg))) - i += copy(dAtA[i:], m.Msg) + i-- + dAtA[i] = 0x12 } - if len(m.Workers) > 0 { - for _, msg := range m.Workers { - dAtA[i] = 0x1a - i++ - i = encodeVarintDmmaster(dAtA, i, uint64(msg.Size())) - n, err := msg.MarshalTo(dAtA[i:]) - if err != nil { - return 0, err - } - i += n + if m.Result { + i-- + if m.Result { + dAtA[i] = 1 + } else { + dAtA[i] = 0 } + i-- + dAtA[i] = 0x8 } - return i, nil + return len(dAtA) - i, nil } func (m *PurgeWorkerRelayRequest) Marshal() (dAtA []byte, err error) { size := m.Size() dAtA = make([]byte, size) - n, err := m.MarshalTo(dAtA) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) if err != nil { return nil, err } @@ -4203,59 +4228,60 @@ func (m *PurgeWorkerRelayRequest) Marshal() (dAtA []byte, err error) { } func (m *PurgeWorkerRelayRequest) MarshalTo(dAtA []byte) (int, error) { - var i int + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *PurgeWorkerRelayRequest) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) _ = i var l int _ = l - if len(m.Workers) > 0 { - for _, s := range m.Workers { - dAtA[i] = 0xa - i++ - l = len(s) - for l >= 1<<7 { - dAtA[i] = uint8(uint64(l)&0x7f | 0x80) - l >>= 7 - i++ - } - dAtA[i] = uint8(l) - i++ - i += copy(dAtA[i:], s) - } + if len(m.SubDir) > 0 { + i -= len(m.SubDir) + copy(dAtA[i:], m.SubDir) + i = encodeVarintDmmaster(dAtA, i, uint64(len(m.SubDir))) + i-- + dAtA[i] = 0x2a + } + if len(m.Filename) > 0 { + i -= len(m.Filename) + copy(dAtA[i:], m.Filename) + i = encodeVarintDmmaster(dAtA, i, uint64(len(m.Filename))) + i-- + dAtA[i] = 0x22 + } + if m.Time != 0 { + i = encodeVarintDmmaster(dAtA, i, uint64(m.Time)) + i-- + dAtA[i] = 0x18 } if m.Inactive { - dAtA[i] = 0x10 - i++ + i-- if m.Inactive { dAtA[i] = 1 } else { dAtA[i] = 0 } - i++ - } - if m.Time != 0 { - dAtA[i] = 0x18 - i++ - i = encodeVarintDmmaster(dAtA, i, uint64(m.Time)) - } - if len(m.Filename) > 0 { - dAtA[i] = 0x22 - i++ - i = encodeVarintDmmaster(dAtA, i, uint64(len(m.Filename))) - i += copy(dAtA[i:], m.Filename) + i-- + dAtA[i] = 0x10 } - if len(m.SubDir) > 0 { - dAtA[i] = 0x2a - i++ - i = encodeVarintDmmaster(dAtA, i, uint64(len(m.SubDir))) - i += copy(dAtA[i:], m.SubDir) + 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] = 0xa + } } - return i, nil + return len(dAtA) - i, nil } func (m *PurgeWorkerRelayResponse) Marshal() (dAtA []byte, err error) { size := m.Size() dAtA = make([]byte, size) - n, err := m.MarshalTo(dAtA) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) if err != nil { return nil, err } @@ -4263,45 +4289,53 @@ func (m *PurgeWorkerRelayResponse) Marshal() (dAtA []byte, err error) { } func (m *PurgeWorkerRelayResponse) MarshalTo(dAtA []byte) (int, error) { - var i int + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *PurgeWorkerRelayResponse) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) _ = i var l int _ = l - if m.Result { - dAtA[i] = 0x8 - i++ - if m.Result { - dAtA[i] = 1 - } else { - dAtA[i] = 0 + 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 } - i++ } if len(m.Msg) > 0 { - dAtA[i] = 0x12 - i++ + i -= len(m.Msg) + copy(dAtA[i:], m.Msg) i = encodeVarintDmmaster(dAtA, i, uint64(len(m.Msg))) - i += copy(dAtA[i:], m.Msg) + i-- + dAtA[i] = 0x12 } - if len(m.Workers) > 0 { - for _, msg := range m.Workers { - dAtA[i] = 0x1a - i++ - i = encodeVarintDmmaster(dAtA, i, uint64(msg.Size())) - n, err := msg.MarshalTo(dAtA[i:]) - if err != nil { - return 0, err - } - i += n + if m.Result { + i-- + if m.Result { + dAtA[i] = 1 + } else { + dAtA[i] = 0 } + i-- + dAtA[i] = 0x8 } - return i, nil + return len(dAtA) - i, nil } func (m *CheckTaskRequest) Marshal() (dAtA []byte, err error) { size := m.Size() dAtA = make([]byte, size) - n, err := m.MarshalTo(dAtA) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) if err != nil { return nil, err } @@ -4309,23 +4343,29 @@ func (m *CheckTaskRequest) Marshal() (dAtA []byte, err error) { } func (m *CheckTaskRequest) MarshalTo(dAtA []byte) (int, error) { - var i int + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *CheckTaskRequest) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) _ = i var l int _ = l if len(m.Task) > 0 { - dAtA[i] = 0xa - i++ + i -= len(m.Task) + copy(dAtA[i:], m.Task) i = encodeVarintDmmaster(dAtA, i, uint64(len(m.Task))) - i += copy(dAtA[i:], m.Task) + i-- + dAtA[i] = 0xa } - return i, nil + return len(dAtA) - i, nil } func (m *CheckTaskResponse) Marshal() (dAtA []byte, err error) { size := m.Size() dAtA = make([]byte, size) - n, err := m.MarshalTo(dAtA) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) if err != nil { return nil, err } @@ -4333,33 +4373,39 @@ func (m *CheckTaskResponse) Marshal() (dAtA []byte, err error) { } func (m *CheckTaskResponse) MarshalTo(dAtA []byte) (int, error) { - var i int + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *CheckTaskResponse) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) _ = i var l int _ = l + if len(m.Msg) > 0 { + i -= len(m.Msg) + copy(dAtA[i:], m.Msg) + i = encodeVarintDmmaster(dAtA, i, uint64(len(m.Msg))) + i-- + dAtA[i] = 0x12 + } if m.Result { - dAtA[i] = 0x8 - i++ + i-- if m.Result { dAtA[i] = 1 } else { dAtA[i] = 0 } - i++ - } - if len(m.Msg) > 0 { - dAtA[i] = 0x12 - i++ - i = encodeVarintDmmaster(dAtA, i, uint64(len(m.Msg))) - i += copy(dAtA[i:], m.Msg) + i-- + dAtA[i] = 0x8 } - return i, nil + return len(dAtA) - i, nil } func (m *RegisterWorkerRequest) Marshal() (dAtA []byte, err error) { size := m.Size() dAtA = make([]byte, size) - n, err := m.MarshalTo(dAtA) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) if err != nil { return nil, err } @@ -4367,29 +4413,36 @@ func (m *RegisterWorkerRequest) Marshal() (dAtA []byte, err error) { } func (m *RegisterWorkerRequest) MarshalTo(dAtA []byte) (int, error) { - var i int + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *RegisterWorkerRequest) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) _ = i var l int _ = l - if len(m.Name) > 0 { - dAtA[i] = 0xa - i++ - i = encodeVarintDmmaster(dAtA, i, uint64(len(m.Name))) - i += copy(dAtA[i:], m.Name) - } if len(m.Address) > 0 { - dAtA[i] = 0x12 - i++ + i -= len(m.Address) + copy(dAtA[i:], m.Address) i = encodeVarintDmmaster(dAtA, i, uint64(len(m.Address))) - i += copy(dAtA[i:], m.Address) + i-- + dAtA[i] = 0x12 + } + if len(m.Name) > 0 { + i -= len(m.Name) + copy(dAtA[i:], m.Name) + i = encodeVarintDmmaster(dAtA, i, uint64(len(m.Name))) + i-- + dAtA[i] = 0xa } - return i, nil + return len(dAtA) - i, nil } func (m *RegisterWorkerResponse) Marshal() (dAtA []byte, err error) { size := m.Size() dAtA = make([]byte, size) - n, err := m.MarshalTo(dAtA) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) if err != nil { return nil, err } @@ -4397,37 +4450,45 @@ func (m *RegisterWorkerResponse) Marshal() (dAtA []byte, err error) { } func (m *RegisterWorkerResponse) MarshalTo(dAtA []byte) (int, error) { - var i int + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *RegisterWorkerResponse) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) _ = i var l int _ = l + if len(m.Msg) > 0 { + i -= len(m.Msg) + copy(dAtA[i:], m.Msg) + i = encodeVarintDmmaster(dAtA, i, uint64(len(m.Msg))) + i-- + dAtA[i] = 0x12 + } if m.Result { - dAtA[i] = 0x8 - i++ + i-- if m.Result { dAtA[i] = 1 } else { dAtA[i] = 0 } - i++ - } - if len(m.Msg) > 0 { - dAtA[i] = 0x12 - i++ - i = encodeVarintDmmaster(dAtA, i, uint64(len(m.Msg))) - i += copy(dAtA[i:], m.Msg) + i-- + dAtA[i] = 0x8 } - return i, nil + return len(dAtA) - i, nil } func encodeVarintDmmaster(dAtA []byte, offset int, v uint64) int { + offset -= sovDmmaster(v) + base := offset for v >= 1<<7 { dAtA[offset] = uint8(v&0x7f | 0x80) v >>= 7 offset++ } dAtA[offset] = uint8(v) - return offset + 1 + return base } func (m *MigrateWorkerRelayRequest) Size() (n int) { if m == nil { @@ -4476,8 +4537,8 @@ func (m *StartTaskRequest) Size() (n int) { if l > 0 { n += 1 + l + sovDmmaster(uint64(l)) } - if len(m.Workers) > 0 { - for _, s := range m.Workers { + if len(m.Sources) > 0 { + for _, s := range m.Sources { l = len(s) n += 1 + l + sovDmmaster(uint64(l)) } @@ -4555,8 +4616,8 @@ func (m *OperateTaskRequest) Size() (n int) { if l > 0 { n += 1 + l + sovDmmaster(uint64(l)) } - if len(m.Workers) > 0 { - for _, s := range m.Workers { + if len(m.Sources) > 0 { + for _, s := range m.Sources { l = len(s) n += 1 + l + sovDmmaster(uint64(l)) } @@ -4599,8 +4660,8 @@ func (m *UpdateTaskRequest) Size() (n int) { if l > 0 { n += 1 + l + sovDmmaster(uint64(l)) } - if len(m.Workers) > 0 { - for _, s := range m.Workers { + if len(m.Sources) > 0 { + for _, s := range m.Sources { l = len(s) n += 1 + l + sovDmmaster(uint64(l)) } @@ -4640,8 +4701,8 @@ func (m *QueryStatusListRequest) Size() (n int) { if l > 0 { n += 1 + l + sovDmmaster(uint64(l)) } - if len(m.Workers) > 0 { - for _, s := range m.Workers { + if len(m.Sources) > 0 { + for _, s := range m.Sources { l = len(s) n += 1 + l + sovDmmaster(uint64(l)) } @@ -4681,8 +4742,8 @@ func (m *QueryErrorListRequest) Size() (n int) { if l > 0 { n += 1 + l + sovDmmaster(uint64(l)) } - if len(m.Workers) > 0 { - for _, s := range m.Workers { + if len(m.Sources) > 0 { + for _, s := range m.Sources { l = len(s) n += 1 + l + sovDmmaster(uint64(l)) } @@ -4722,8 +4783,8 @@ func (m *ShowDDLLocksRequest) Size() (n int) { if l > 0 { n += 1 + l + sovDmmaster(uint64(l)) } - if len(m.Workers) > 0 { - for _, s := range m.Workers { + if len(m.Sources) > 0 { + for _, s := range m.Sources { l = len(s) n += 1 + l + sovDmmaster(uint64(l)) } @@ -4806,8 +4867,8 @@ func (m *UnlockDDLLockRequest) Size() (n int) { if l > 0 { n += 1 + l + sovDmmaster(uint64(l)) } - if len(m.Workers) > 0 { - for _, s := range m.Workers { + if len(m.Sources) > 0 { + for _, s := range m.Sources { l = len(s) n += 1 + l + sovDmmaster(uint64(l)) } @@ -4846,8 +4907,8 @@ func (m *BreakWorkerDDLLockRequest) Size() (n int) { } var l int _ = l - if len(m.Workers) > 0 { - for _, s := range m.Workers { + if len(m.Sources) > 0 { + for _, s := range m.Sources { l = len(s) n += 1 + l + sovDmmaster(uint64(l)) } @@ -4895,50 +4956,10 @@ func (m *SwitchWorkerRelayMasterRequest) Size() (n int) { if m == nil { return 0 } - var l int - _ = l - if len(m.Workers) > 0 { - for _, s := range m.Workers { - l = len(s) - n += 1 + l + sovDmmaster(uint64(l)) - } - } - return n -} - -func (m *SwitchWorkerRelayMasterResponse) 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.Workers) > 0 { - for _, e := range m.Workers { - l = e.Size() - n += 1 + l + sovDmmaster(uint64(l)) - } - } - return n -} - -func (m *OperateWorkerRelayRequest) Size() (n int) { - if m == nil { - return 0 - } - var l int - _ = l - if m.Op != 0 { - n += 1 + sovDmmaster(uint64(m.Op)) - } - if len(m.Workers) > 0 { - for _, s := range m.Workers { + var l int + _ = l + if len(m.Sources) > 0 { + for _, s := range m.Sources { l = len(s) n += 1 + l + sovDmmaster(uint64(l)) } @@ -4946,15 +4967,12 @@ func (m *OperateWorkerRelayRequest) Size() (n int) { return n } -func (m *OperateWorkerRelayResponse) Size() (n int) { +func (m *SwitchWorkerRelayMasterResponse) Size() (n int) { if m == nil { return 0 } var l int _ = l - if m.Op != 0 { - n += 1 + sovDmmaster(uint64(m.Op)) - } if m.Result { n += 2 } @@ -4971,41 +4989,40 @@ func (m *OperateWorkerRelayResponse) Size() (n int) { return n } -func (m *RefreshWorkerTasksRequest) Size() (n int) { - if m == nil { - return 0 - } - var l int - _ = l - return n -} - -func (m *RefreshWorkerTasksMsg) Size() (n int) { +func (m *OperateWorkerRelayRequest) Size() (n int) { if m == nil { return 0 } var l int _ = l - l = len(m.Worker) - if l > 0 { - n += 1 + l + sovDmmaster(uint64(l)) + if m.Op != 0 { + n += 1 + sovDmmaster(uint64(m.Op)) } - l = len(m.Msg) - 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)) + } } return n } -func (m *RefreshWorkerTasksResponse) Size() (n int) { +func (m *OperateWorkerRelayResponse) Size() (n int) { if m == nil { return 0 } var l int _ = l + if m.Op != 0 { + n += 1 + sovDmmaster(uint64(m.Op)) + } if m.Result { n += 2 } + l = len(m.Msg) + if l > 0 { + n += 1 + l + sovDmmaster(uint64(l)) + } if len(m.Workers) > 0 { for _, e := range m.Workers { l = e.Size() @@ -5080,8 +5097,8 @@ func (m *PurgeWorkerRelayRequest) Size() (n int) { } var l int _ = l - if len(m.Workers) > 0 { - for _, s := range m.Workers { + if len(m.Sources) > 0 { + for _, s := range m.Sources { l = len(s) n += 1 + l + sovDmmaster(uint64(l)) } @@ -5188,14 +5205,7 @@ func (m *RegisterWorkerResponse) Size() (n int) { } func sovDmmaster(x uint64) (n int) { - for { - n++ - x >>= 7 - if x == 0 { - break - } - } - return n + return (math_bits.Len64(x|1) + 6) / 7 } func sozDmmaster(x uint64) (n int) { return sovDmmaster(uint64((x << 1) ^ uint64((int64(x) >> 63)))) @@ -5215,7 +5225,7 @@ func (m *MigrateWorkerRelayRequest) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - wire |= (uint64(b) & 0x7F) << shift + wire |= uint64(b&0x7F) << shift if b < 0x80 { break } @@ -5243,7 +5253,7 @@ func (m *MigrateWorkerRelayRequest) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= (uint64(b) & 0x7F) << shift + stringLen |= uint64(b&0x7F) << shift if b < 0x80 { break } @@ -5253,6 +5263,9 @@ func (m *MigrateWorkerRelayRequest) Unmarshal(dAtA []byte) error { return ErrInvalidLengthDmmaster } postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthDmmaster + } if postIndex > l { return io.ErrUnexpectedEOF } @@ -5272,7 +5285,7 @@ func (m *MigrateWorkerRelayRequest) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - m.BinlogPos |= (uint32(b) & 0x7F) << shift + m.BinlogPos |= uint32(b&0x7F) << shift if b < 0x80 { break } @@ -5291,7 +5304,7 @@ func (m *MigrateWorkerRelayRequest) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= (uint64(b) & 0x7F) << shift + stringLen |= uint64(b&0x7F) << shift if b < 0x80 { break } @@ -5301,6 +5314,9 @@ func (m *MigrateWorkerRelayRequest) Unmarshal(dAtA []byte) error { return ErrInvalidLengthDmmaster } postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthDmmaster + } if postIndex > l { return io.ErrUnexpectedEOF } @@ -5315,6 +5331,9 @@ func (m *MigrateWorkerRelayRequest) Unmarshal(dAtA []byte) error { if skippy < 0 { return ErrInvalidLengthDmmaster } + if (iNdEx + skippy) < 0 { + return ErrInvalidLengthDmmaster + } if (iNdEx + skippy) > l { return io.ErrUnexpectedEOF } @@ -5342,7 +5361,7 @@ func (m *UpdateWorkerRelayConfigRequest) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - wire |= (uint64(b) & 0x7F) << shift + wire |= uint64(b&0x7F) << shift if b < 0x80 { break } @@ -5370,7 +5389,7 @@ func (m *UpdateWorkerRelayConfigRequest) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= (uint64(b) & 0x7F) << shift + stringLen |= uint64(b&0x7F) << shift if b < 0x80 { break } @@ -5380,6 +5399,9 @@ func (m *UpdateWorkerRelayConfigRequest) Unmarshal(dAtA []byte) error { return ErrInvalidLengthDmmaster } postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthDmmaster + } if postIndex > l { return io.ErrUnexpectedEOF } @@ -5399,7 +5421,7 @@ func (m *UpdateWorkerRelayConfigRequest) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= (uint64(b) & 0x7F) << shift + stringLen |= uint64(b&0x7F) << shift if b < 0x80 { break } @@ -5409,6 +5431,9 @@ func (m *UpdateWorkerRelayConfigRequest) Unmarshal(dAtA []byte) error { return ErrInvalidLengthDmmaster } postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthDmmaster + } if postIndex > l { return io.ErrUnexpectedEOF } @@ -5423,6 +5448,9 @@ func (m *UpdateWorkerRelayConfigRequest) Unmarshal(dAtA []byte) error { if skippy < 0 { return ErrInvalidLengthDmmaster } + if (iNdEx + skippy) < 0 { + return ErrInvalidLengthDmmaster + } if (iNdEx + skippy) > l { return io.ErrUnexpectedEOF } @@ -5450,7 +5478,7 @@ func (m *StartTaskRequest) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - wire |= (uint64(b) & 0x7F) << shift + wire |= uint64(b&0x7F) << shift if b < 0x80 { break } @@ -5478,7 +5506,7 @@ func (m *StartTaskRequest) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= (uint64(b) & 0x7F) << shift + stringLen |= uint64(b&0x7F) << shift if b < 0x80 { break } @@ -5488,113 +5516,17 @@ func (m *StartTaskRequest) Unmarshal(dAtA []byte) error { return ErrInvalidLengthDmmaster } postIndex := iNdEx + intStringLen - if postIndex > l { - return io.ErrUnexpectedEOF - } - m.Task = string(dAtA[iNdEx:postIndex]) - iNdEx = postIndex - case 2: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Workers", 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 { + if postIndex < 0 { return ErrInvalidLengthDmmaster } - postIndex := iNdEx + intStringLen if postIndex > l { return io.ErrUnexpectedEOF } - m.Workers = append(m.Workers, string(dAtA[iNdEx:postIndex])) + m.Task = string(dAtA[iNdEx:postIndex]) iNdEx = postIndex - default: - iNdEx = preIndex - skippy, err := skipDmmaster(dAtA[iNdEx:]) - if err != nil { - return err - } - if skippy < 0 { - return ErrInvalidLengthDmmaster - } - if (iNdEx + skippy) > l { - return io.ErrUnexpectedEOF - } - iNdEx += skippy - } - } - - if iNdEx > l { - return io.ErrUnexpectedEOF - } - return nil -} -func (m *StartTaskResponse) 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: StartTaskResponse: wiretype end group for non-group") - } - if fieldNum <= 0 { - return fmt.Errorf("proto: StartTaskResponse: 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) + return fmt.Errorf("proto: wrong wireType = %d for field Sources", wireType) } var stringLen uint64 for shift := uint(0); ; shift += 7 { @@ -5606,7 +5538,7 @@ func (m *StartTaskResponse) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= (uint64(b) & 0x7F) << shift + stringLen |= uint64(b&0x7F) << shift if b < 0x80 { break } @@ -5616,41 +5548,13 @@ func (m *StartTaskResponse) Unmarshal(dAtA []byte) error { return ErrInvalidLengthDmmaster } postIndex := iNdEx + intStringLen - 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 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 { + if postIndex < 0 { return ErrInvalidLengthDmmaster } - postIndex := iNdEx + msglen 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 - } + m.Sources = append(m.Sources, string(dAtA[iNdEx:postIndex])) iNdEx = postIndex default: iNdEx = preIndex @@ -5661,83 +5565,7 @@ func (m *StartTaskResponse) Unmarshal(dAtA []byte) error { if skippy < 0 { return ErrInvalidLengthDmmaster } - if (iNdEx + skippy) > l { - return io.ErrUnexpectedEOF - } - iNdEx += skippy - } - } - - if iNdEx > l { - return io.ErrUnexpectedEOF - } - return nil -} -func (m *UpdateMasterConfigRequest) 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: UpdateMasterConfigRequest: wiretype end group for non-group") - } - if fieldNum <= 0 { - return fmt.Errorf("proto: UpdateMasterConfigRequest: illegal tag %d (wire type %d)", fieldNum, wire) - } - switch fieldNum { - case 1: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Config", 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 > l { - return io.ErrUnexpectedEOF - } - m.Config = string(dAtA[iNdEx:postIndex]) - iNdEx = postIndex - default: - iNdEx = preIndex - skippy, err := skipDmmaster(dAtA[iNdEx:]) - if err != nil { - return err - } - if skippy < 0 { + if (iNdEx + skippy) < 0 { return ErrInvalidLengthDmmaster } if (iNdEx + skippy) > l { @@ -5752,7 +5580,7 @@ func (m *UpdateMasterConfigRequest) Unmarshal(dAtA []byte) error { } return nil } -func (m *UpdateMasterConfigResponse) Unmarshal(dAtA []byte) error { +func (m *StartTaskResponse) Unmarshal(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -5767,7 +5595,7 @@ func (m *UpdateMasterConfigResponse) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - wire |= (uint64(b) & 0x7F) << shift + wire |= uint64(b&0x7F) << shift if b < 0x80 { break } @@ -5775,10 +5603,10 @@ func (m *UpdateMasterConfigResponse) Unmarshal(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: UpdateMasterConfigResponse: wiretype end group for non-group") + return fmt.Errorf("proto: StartTaskResponse: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: UpdateMasterConfigResponse: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: StartTaskResponse: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: @@ -5795,7 +5623,7 @@ func (m *UpdateMasterConfigResponse) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - v |= (int(b) & 0x7F) << shift + v |= int(b&0x7F) << shift if b < 0x80 { break } @@ -5815,7 +5643,7 @@ func (m *UpdateMasterConfigResponse) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= (uint64(b) & 0x7F) << shift + stringLen |= uint64(b&0x7F) << shift if b < 0x80 { break } @@ -5825,6 +5653,9 @@ func (m *UpdateMasterConfigResponse) Unmarshal(dAtA []byte) error { return ErrInvalidLengthDmmaster } postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthDmmaster + } if postIndex > l { return io.ErrUnexpectedEOF } @@ -5844,7 +5675,7 @@ func (m *UpdateMasterConfigResponse) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - msglen |= (int(b) & 0x7F) << shift + msglen |= int(b&0x7F) << shift if b < 0x80 { break } @@ -5853,10 +5684,13 @@ func (m *UpdateMasterConfigResponse) Unmarshal(dAtA []byte) error { return ErrInvalidLengthDmmaster } postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthDmmaster + } if postIndex > l { return io.ErrUnexpectedEOF } - m.Workers = append(m.Workers, &QueryStatusResponse{}) + m.Workers = append(m.Workers, &CommonWorkerResponse{}) if err := m.Workers[len(m.Workers)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -5870,6 +5704,9 @@ func (m *UpdateMasterConfigResponse) Unmarshal(dAtA []byte) error { if skippy < 0 { return ErrInvalidLengthDmmaster } + if (iNdEx + skippy) < 0 { + return ErrInvalidLengthDmmaster + } if (iNdEx + skippy) > l { return io.ErrUnexpectedEOF } @@ -5882,7 +5719,7 @@ func (m *UpdateMasterConfigResponse) Unmarshal(dAtA []byte) error { } return nil } -func (m *OperateTaskRequest) Unmarshal(dAtA []byte) error { +func (m *UpdateMasterConfigRequest) Unmarshal(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -5897,7 +5734,7 @@ func (m *OperateTaskRequest) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - wire |= (uint64(b) & 0x7F) << shift + wire |= uint64(b&0x7F) << shift if b < 0x80 { break } @@ -5905,34 +5742,15 @@ func (m *OperateTaskRequest) Unmarshal(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: OperateTaskRequest: wiretype end group for non-group") + return fmt.Errorf("proto: UpdateMasterConfigRequest: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: OperateTaskRequest: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: UpdateMasterConfigRequest: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field Op", wireType) - } - m.Op = 0 - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowDmmaster - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - m.Op |= (TaskOp(b) & 0x7F) << shift - if b < 0x80 { - break - } - } - case 2: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Name", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Config", wireType) } var stringLen uint64 for shift := uint(0); ; shift += 7 { @@ -5944,7 +5762,7 @@ func (m *OperateTaskRequest) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= (uint64(b) & 0x7F) << shift + stringLen |= uint64(b&0x7F) << shift if b < 0x80 { break } @@ -5954,39 +5772,13 @@ func (m *OperateTaskRequest) Unmarshal(dAtA []byte) error { return ErrInvalidLengthDmmaster } postIndex := iNdEx + intStringLen - if postIndex > l { - return io.ErrUnexpectedEOF - } - m.Name = string(dAtA[iNdEx:postIndex]) - iNdEx = postIndex - case 3: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Workers", 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 { + if postIndex < 0 { return ErrInvalidLengthDmmaster } - postIndex := iNdEx + intStringLen if postIndex > l { return io.ErrUnexpectedEOF } - m.Workers = append(m.Workers, string(dAtA[iNdEx:postIndex])) + m.Config = string(dAtA[iNdEx:postIndex]) iNdEx = postIndex default: iNdEx = preIndex @@ -5997,6 +5789,9 @@ func (m *OperateTaskRequest) Unmarshal(dAtA []byte) error { if skippy < 0 { return ErrInvalidLengthDmmaster } + if (iNdEx + skippy) < 0 { + return ErrInvalidLengthDmmaster + } if (iNdEx + skippy) > l { return io.ErrUnexpectedEOF } @@ -6009,7 +5804,7 @@ func (m *OperateTaskRequest) Unmarshal(dAtA []byte) error { } return nil } -func (m *OperateTaskResponse) Unmarshal(dAtA []byte) error { +func (m *UpdateMasterConfigResponse) Unmarshal(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -6024,7 +5819,7 @@ func (m *OperateTaskResponse) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - wire |= (uint64(b) & 0x7F) << shift + wire |= uint64(b&0x7F) << shift if b < 0x80 { break } @@ -6032,32 +5827,13 @@ func (m *OperateTaskResponse) Unmarshal(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: OperateTaskResponse: wiretype end group for non-group") + return fmt.Errorf("proto: UpdateMasterConfigResponse: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: OperateTaskResponse: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: UpdateMasterConfigResponse: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field Op", wireType) - } - m.Op = 0 - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowDmmaster - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - m.Op |= (TaskOp(b) & 0x7F) << shift - if b < 0x80 { - break - } - } - case 2: if wireType != 0 { return fmt.Errorf("proto: wrong wireType = %d for field Result", wireType) } @@ -6071,13 +5847,13 @@ func (m *OperateTaskResponse) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - v |= (int(b) & 0x7F) << shift + v |= int(b&0x7F) << shift if b < 0x80 { break } } m.Result = bool(v != 0) - case 3: + case 2: if wireType != 2 { return fmt.Errorf("proto: wrong wireType = %d for field Msg", wireType) } @@ -6091,7 +5867,7 @@ func (m *OperateTaskResponse) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= (uint64(b) & 0x7F) << shift + stringLen |= uint64(b&0x7F) << shift if b < 0x80 { break } @@ -6101,12 +5877,15 @@ func (m *OperateTaskResponse) Unmarshal(dAtA []byte) error { 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 4: + case 3: if wireType != 2 { return fmt.Errorf("proto: wrong wireType = %d for field Workers", wireType) } @@ -6120,7 +5899,7 @@ func (m *OperateTaskResponse) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - msglen |= (int(b) & 0x7F) << shift + msglen |= int(b&0x7F) << shift if b < 0x80 { break } @@ -6129,10 +5908,13 @@ func (m *OperateTaskResponse) Unmarshal(dAtA []byte) error { return ErrInvalidLengthDmmaster } postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthDmmaster + } if postIndex > l { return io.ErrUnexpectedEOF } - m.Workers = append(m.Workers, &OperateSubTaskResponse{}) + m.Workers = append(m.Workers, &QueryStatusResponse{}) if err := m.Workers[len(m.Workers)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -6146,6 +5928,9 @@ func (m *OperateTaskResponse) Unmarshal(dAtA []byte) error { if skippy < 0 { return ErrInvalidLengthDmmaster } + if (iNdEx + skippy) < 0 { + return ErrInvalidLengthDmmaster + } if (iNdEx + skippy) > l { return io.ErrUnexpectedEOF } @@ -6158,7 +5943,7 @@ func (m *OperateTaskResponse) Unmarshal(dAtA []byte) error { } return nil } -func (m *UpdateTaskRequest) Unmarshal(dAtA []byte) error { +func (m *OperateTaskRequest) Unmarshal(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -6173,7 +5958,7 @@ func (m *UpdateTaskRequest) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - wire |= (uint64(b) & 0x7F) << shift + wire |= uint64(b&0x7F) << shift if b < 0x80 { break } @@ -6181,15 +5966,34 @@ func (m *UpdateTaskRequest) Unmarshal(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: UpdateTaskRequest: wiretype end group for non-group") + return fmt.Errorf("proto: OperateTaskRequest: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: UpdateTaskRequest: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: OperateTaskRequest: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Op", wireType) + } + m.Op = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowDmmaster + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.Op |= TaskOp(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 2: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Task", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Name", wireType) } var stringLen uint64 for shift := uint(0); ; shift += 7 { @@ -6201,7 +6005,7 @@ func (m *UpdateTaskRequest) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= (uint64(b) & 0x7F) << shift + stringLen |= uint64(b&0x7F) << shift if b < 0x80 { break } @@ -6211,14 +6015,17 @@ func (m *UpdateTaskRequest) Unmarshal(dAtA []byte) error { return ErrInvalidLengthDmmaster } postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthDmmaster + } if postIndex > l { return io.ErrUnexpectedEOF } - m.Task = string(dAtA[iNdEx:postIndex]) + m.Name = string(dAtA[iNdEx:postIndex]) iNdEx = postIndex - case 2: + case 3: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Workers", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Sources", wireType) } var stringLen uint64 for shift := uint(0); ; shift += 7 { @@ -6230,7 +6037,7 @@ func (m *UpdateTaskRequest) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= (uint64(b) & 0x7F) << shift + stringLen |= uint64(b&0x7F) << shift if b < 0x80 { break } @@ -6240,10 +6047,13 @@ func (m *UpdateTaskRequest) Unmarshal(dAtA []byte) error { return ErrInvalidLengthDmmaster } postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthDmmaster + } if postIndex > l { return io.ErrUnexpectedEOF } - m.Workers = append(m.Workers, string(dAtA[iNdEx:postIndex])) + m.Sources = append(m.Sources, string(dAtA[iNdEx:postIndex])) iNdEx = postIndex default: iNdEx = preIndex @@ -6254,6 +6064,9 @@ func (m *UpdateTaskRequest) Unmarshal(dAtA []byte) error { if skippy < 0 { return ErrInvalidLengthDmmaster } + if (iNdEx + skippy) < 0 { + return ErrInvalidLengthDmmaster + } if (iNdEx + skippy) > l { return io.ErrUnexpectedEOF } @@ -6266,7 +6079,7 @@ func (m *UpdateTaskRequest) Unmarshal(dAtA []byte) error { } return nil } -func (m *UpdateTaskResponse) Unmarshal(dAtA []byte) error { +func (m *OperateTaskResponse) Unmarshal(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -6281,7 +6094,7 @@ func (m *UpdateTaskResponse) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - wire |= (uint64(b) & 0x7F) << shift + wire |= uint64(b&0x7F) << shift if b < 0x80 { break } @@ -6289,13 +6102,32 @@ func (m *UpdateTaskResponse) Unmarshal(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: UpdateTaskResponse: wiretype end group for non-group") + return fmt.Errorf("proto: OperateTaskResponse: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: UpdateTaskResponse: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: OperateTaskResponse: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Op", wireType) + } + m.Op = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowDmmaster + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.Op |= TaskOp(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 2: if wireType != 0 { return fmt.Errorf("proto: wrong wireType = %d for field Result", wireType) } @@ -6309,13 +6141,13 @@ func (m *UpdateTaskResponse) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - v |= (int(b) & 0x7F) << shift + v |= int(b&0x7F) << shift if b < 0x80 { break } } m.Result = bool(v != 0) - case 2: + case 3: if wireType != 2 { return fmt.Errorf("proto: wrong wireType = %d for field Msg", wireType) } @@ -6329,7 +6161,7 @@ func (m *UpdateTaskResponse) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= (uint64(b) & 0x7F) << shift + stringLen |= uint64(b&0x7F) << shift if b < 0x80 { break } @@ -6339,12 +6171,15 @@ func (m *UpdateTaskResponse) Unmarshal(dAtA []byte) error { 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: + case 4: if wireType != 2 { return fmt.Errorf("proto: wrong wireType = %d for field Workers", wireType) } @@ -6358,7 +6193,7 @@ func (m *UpdateTaskResponse) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - msglen |= (int(b) & 0x7F) << shift + msglen |= int(b&0x7F) << shift if b < 0x80 { break } @@ -6367,10 +6202,13 @@ func (m *UpdateTaskResponse) Unmarshal(dAtA []byte) error { return ErrInvalidLengthDmmaster } postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthDmmaster + } if postIndex > l { return io.ErrUnexpectedEOF } - m.Workers = append(m.Workers, &CommonWorkerResponse{}) + m.Workers = append(m.Workers, &OperateSubTaskResponse{}) if err := m.Workers[len(m.Workers)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -6384,6 +6222,9 @@ func (m *UpdateTaskResponse) Unmarshal(dAtA []byte) error { if skippy < 0 { return ErrInvalidLengthDmmaster } + if (iNdEx + skippy) < 0 { + return ErrInvalidLengthDmmaster + } if (iNdEx + skippy) > l { return io.ErrUnexpectedEOF } @@ -6396,7 +6237,7 @@ func (m *UpdateTaskResponse) Unmarshal(dAtA []byte) error { } return nil } -func (m *QueryStatusListRequest) Unmarshal(dAtA []byte) error { +func (m *UpdateTaskRequest) Unmarshal(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -6411,7 +6252,7 @@ func (m *QueryStatusListRequest) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - wire |= (uint64(b) & 0x7F) << shift + wire |= uint64(b&0x7F) << shift if b < 0x80 { break } @@ -6419,15 +6260,15 @@ func (m *QueryStatusListRequest) Unmarshal(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: QueryStatusListRequest: wiretype end group for non-group") + return fmt.Errorf("proto: UpdateTaskRequest: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: QueryStatusListRequest: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: UpdateTaskRequest: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Name", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Task", wireType) } var stringLen uint64 for shift := uint(0); ; shift += 7 { @@ -6439,7 +6280,7 @@ func (m *QueryStatusListRequest) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= (uint64(b) & 0x7F) << shift + stringLen |= uint64(b&0x7F) << shift if b < 0x80 { break } @@ -6449,14 +6290,17 @@ func (m *QueryStatusListRequest) Unmarshal(dAtA []byte) error { return ErrInvalidLengthDmmaster } postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthDmmaster + } if postIndex > l { return io.ErrUnexpectedEOF } - m.Name = string(dAtA[iNdEx:postIndex]) + m.Task = string(dAtA[iNdEx:postIndex]) iNdEx = postIndex case 2: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Workers", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Sources", wireType) } var stringLen uint64 for shift := uint(0); ; shift += 7 { @@ -6468,7 +6312,7 @@ func (m *QueryStatusListRequest) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= (uint64(b) & 0x7F) << shift + stringLen |= uint64(b&0x7F) << shift if b < 0x80 { break } @@ -6478,10 +6322,13 @@ func (m *QueryStatusListRequest) Unmarshal(dAtA []byte) error { return ErrInvalidLengthDmmaster } postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthDmmaster + } if postIndex > l { return io.ErrUnexpectedEOF } - m.Workers = append(m.Workers, string(dAtA[iNdEx:postIndex])) + m.Sources = append(m.Sources, string(dAtA[iNdEx:postIndex])) iNdEx = postIndex default: iNdEx = preIndex @@ -6492,6 +6339,9 @@ func (m *QueryStatusListRequest) Unmarshal(dAtA []byte) error { if skippy < 0 { return ErrInvalidLengthDmmaster } + if (iNdEx + skippy) < 0 { + return ErrInvalidLengthDmmaster + } if (iNdEx + skippy) > l { return io.ErrUnexpectedEOF } @@ -6504,7 +6354,7 @@ func (m *QueryStatusListRequest) Unmarshal(dAtA []byte) error { } return nil } -func (m *QueryStatusListResponse) Unmarshal(dAtA []byte) error { +func (m *UpdateTaskResponse) Unmarshal(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -6519,7 +6369,7 @@ func (m *QueryStatusListResponse) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - wire |= (uint64(b) & 0x7F) << shift + wire |= uint64(b&0x7F) << shift if b < 0x80 { break } @@ -6527,10 +6377,10 @@ func (m *QueryStatusListResponse) Unmarshal(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: QueryStatusListResponse: wiretype end group for non-group") + return fmt.Errorf("proto: UpdateTaskResponse: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: QueryStatusListResponse: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: UpdateTaskResponse: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: @@ -6547,7 +6397,7 @@ func (m *QueryStatusListResponse) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - v |= (int(b) & 0x7F) << shift + v |= int(b&0x7F) << shift if b < 0x80 { break } @@ -6567,7 +6417,7 @@ func (m *QueryStatusListResponse) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= (uint64(b) & 0x7F) << shift + stringLen |= uint64(b&0x7F) << shift if b < 0x80 { break } @@ -6577,6 +6427,9 @@ func (m *QueryStatusListResponse) Unmarshal(dAtA []byte) error { return ErrInvalidLengthDmmaster } postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthDmmaster + } if postIndex > l { return io.ErrUnexpectedEOF } @@ -6596,7 +6449,7 @@ func (m *QueryStatusListResponse) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - msglen |= (int(b) & 0x7F) << shift + msglen |= int(b&0x7F) << shift if b < 0x80 { break } @@ -6605,10 +6458,13 @@ func (m *QueryStatusListResponse) Unmarshal(dAtA []byte) error { return ErrInvalidLengthDmmaster } postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthDmmaster + } if postIndex > l { return io.ErrUnexpectedEOF } - m.Workers = append(m.Workers, &QueryStatusResponse{}) + m.Workers = append(m.Workers, &CommonWorkerResponse{}) if err := m.Workers[len(m.Workers)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -6622,6 +6478,9 @@ func (m *QueryStatusListResponse) Unmarshal(dAtA []byte) error { if skippy < 0 { return ErrInvalidLengthDmmaster } + if (iNdEx + skippy) < 0 { + return ErrInvalidLengthDmmaster + } if (iNdEx + skippy) > l { return io.ErrUnexpectedEOF } @@ -6634,7 +6493,7 @@ func (m *QueryStatusListResponse) Unmarshal(dAtA []byte) error { } return nil } -func (m *QueryErrorListRequest) Unmarshal(dAtA []byte) error { +func (m *QueryStatusListRequest) Unmarshal(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -6649,7 +6508,7 @@ func (m *QueryErrorListRequest) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - wire |= (uint64(b) & 0x7F) << shift + wire |= uint64(b&0x7F) << shift if b < 0x80 { break } @@ -6657,10 +6516,10 @@ func (m *QueryErrorListRequest) Unmarshal(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: QueryErrorListRequest: wiretype end group for non-group") + return fmt.Errorf("proto: QueryStatusListRequest: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: QueryErrorListRequest: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: QueryStatusListRequest: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: @@ -6677,7 +6536,7 @@ func (m *QueryErrorListRequest) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= (uint64(b) & 0x7F) << shift + stringLen |= uint64(b&0x7F) << shift if b < 0x80 { break } @@ -6687,6 +6546,9 @@ func (m *QueryErrorListRequest) Unmarshal(dAtA []byte) error { return ErrInvalidLengthDmmaster } postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthDmmaster + } if postIndex > l { return io.ErrUnexpectedEOF } @@ -6694,7 +6556,7 @@ func (m *QueryErrorListRequest) Unmarshal(dAtA []byte) error { iNdEx = postIndex case 2: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Workers", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Sources", wireType) } var stringLen uint64 for shift := uint(0); ; shift += 7 { @@ -6706,7 +6568,7 @@ func (m *QueryErrorListRequest) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= (uint64(b) & 0x7F) << shift + stringLen |= uint64(b&0x7F) << shift if b < 0x80 { break } @@ -6716,10 +6578,13 @@ func (m *QueryErrorListRequest) Unmarshal(dAtA []byte) error { return ErrInvalidLengthDmmaster } postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthDmmaster + } if postIndex > l { return io.ErrUnexpectedEOF } - m.Workers = append(m.Workers, string(dAtA[iNdEx:postIndex])) + m.Sources = append(m.Sources, string(dAtA[iNdEx:postIndex])) iNdEx = postIndex default: iNdEx = preIndex @@ -6730,6 +6595,9 @@ func (m *QueryErrorListRequest) Unmarshal(dAtA []byte) error { if skippy < 0 { return ErrInvalidLengthDmmaster } + if (iNdEx + skippy) < 0 { + return ErrInvalidLengthDmmaster + } if (iNdEx + skippy) > l { return io.ErrUnexpectedEOF } @@ -6742,7 +6610,7 @@ func (m *QueryErrorListRequest) Unmarshal(dAtA []byte) error { } return nil } -func (m *QueryErrorListResponse) Unmarshal(dAtA []byte) error { +func (m *QueryStatusListResponse) Unmarshal(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -6757,7 +6625,7 @@ func (m *QueryErrorListResponse) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - wire |= (uint64(b) & 0x7F) << shift + wire |= uint64(b&0x7F) << shift if b < 0x80 { break } @@ -6765,10 +6633,10 @@ func (m *QueryErrorListResponse) Unmarshal(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: QueryErrorListResponse: wiretype end group for non-group") + return fmt.Errorf("proto: QueryStatusListResponse: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: QueryErrorListResponse: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: QueryStatusListResponse: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: @@ -6785,7 +6653,7 @@ func (m *QueryErrorListResponse) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - v |= (int(b) & 0x7F) << shift + v |= int(b&0x7F) << shift if b < 0x80 { break } @@ -6805,7 +6673,7 @@ func (m *QueryErrorListResponse) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= (uint64(b) & 0x7F) << shift + stringLen |= uint64(b&0x7F) << shift if b < 0x80 { break } @@ -6815,6 +6683,9 @@ func (m *QueryErrorListResponse) Unmarshal(dAtA []byte) error { return ErrInvalidLengthDmmaster } postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthDmmaster + } if postIndex > l { return io.ErrUnexpectedEOF } @@ -6834,7 +6705,7 @@ func (m *QueryErrorListResponse) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - msglen |= (int(b) & 0x7F) << shift + msglen |= int(b&0x7F) << shift if b < 0x80 { break } @@ -6843,10 +6714,13 @@ func (m *QueryErrorListResponse) Unmarshal(dAtA []byte) error { return ErrInvalidLengthDmmaster } postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthDmmaster + } if postIndex > l { return io.ErrUnexpectedEOF } - m.Workers = append(m.Workers, &QueryErrorResponse{}) + m.Workers = append(m.Workers, &QueryStatusResponse{}) if err := m.Workers[len(m.Workers)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -6860,6 +6734,9 @@ func (m *QueryErrorListResponse) Unmarshal(dAtA []byte) error { if skippy < 0 { return ErrInvalidLengthDmmaster } + if (iNdEx + skippy) < 0 { + return ErrInvalidLengthDmmaster + } if (iNdEx + skippy) > l { return io.ErrUnexpectedEOF } @@ -6872,7 +6749,7 @@ func (m *QueryErrorListResponse) Unmarshal(dAtA []byte) error { } return nil } -func (m *ShowDDLLocksRequest) Unmarshal(dAtA []byte) error { +func (m *QueryErrorListRequest) Unmarshal(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -6887,7 +6764,7 @@ func (m *ShowDDLLocksRequest) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - wire |= (uint64(b) & 0x7F) << shift + wire |= uint64(b&0x7F) << shift if b < 0x80 { break } @@ -6895,15 +6772,15 @@ func (m *ShowDDLLocksRequest) Unmarshal(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: ShowDDLLocksRequest: wiretype end group for non-group") + return fmt.Errorf("proto: QueryErrorListRequest: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: ShowDDLLocksRequest: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: QueryErrorListRequest: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Task", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Name", wireType) } var stringLen uint64 for shift := uint(0); ; shift += 7 { @@ -6915,7 +6792,7 @@ func (m *ShowDDLLocksRequest) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= (uint64(b) & 0x7F) << shift + stringLen |= uint64(b&0x7F) << shift if b < 0x80 { break } @@ -6925,14 +6802,17 @@ func (m *ShowDDLLocksRequest) Unmarshal(dAtA []byte) error { return ErrInvalidLengthDmmaster } postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthDmmaster + } if postIndex > l { return io.ErrUnexpectedEOF } - m.Task = string(dAtA[iNdEx:postIndex]) + m.Name = string(dAtA[iNdEx:postIndex]) iNdEx = postIndex case 2: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Workers", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Sources", wireType) } var stringLen uint64 for shift := uint(0); ; shift += 7 { @@ -6944,7 +6824,7 @@ func (m *ShowDDLLocksRequest) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= (uint64(b) & 0x7F) << shift + stringLen |= uint64(b&0x7F) << shift if b < 0x80 { break } @@ -6954,10 +6834,13 @@ func (m *ShowDDLLocksRequest) Unmarshal(dAtA []byte) error { return ErrInvalidLengthDmmaster } postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthDmmaster + } if postIndex > l { return io.ErrUnexpectedEOF } - m.Workers = append(m.Workers, string(dAtA[iNdEx:postIndex])) + m.Sources = append(m.Sources, string(dAtA[iNdEx:postIndex])) iNdEx = postIndex default: iNdEx = preIndex @@ -6968,6 +6851,9 @@ func (m *ShowDDLLocksRequest) Unmarshal(dAtA []byte) error { if skippy < 0 { return ErrInvalidLengthDmmaster } + if (iNdEx + skippy) < 0 { + return ErrInvalidLengthDmmaster + } if (iNdEx + skippy) > l { return io.ErrUnexpectedEOF } @@ -6980,7 +6866,7 @@ func (m *ShowDDLLocksRequest) Unmarshal(dAtA []byte) error { } return nil } -func (m *DDLLock) Unmarshal(dAtA []byte) error { +func (m *QueryErrorListResponse) Unmarshal(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -6995,7 +6881,7 @@ func (m *DDLLock) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - wire |= (uint64(b) & 0x7F) << shift + wire |= uint64(b&0x7F) << shift if b < 0x80 { break } @@ -7003,17 +6889,17 @@ func (m *DDLLock) Unmarshal(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: DDLLock: wiretype end group for non-group") + return fmt.Errorf("proto: QueryErrorListResponse: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: DDLLock: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: QueryErrorListResponse: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field ID", wireType) + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Result", wireType) } - var stringLen uint64 + var v int for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowDmmaster @@ -7023,24 +6909,15 @@ func (m *DDLLock) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= (uint64(b) & 0x7F) << shift + v |= int(b&0x7F) << shift if b < 0x80 { break } } - intStringLen := int(stringLen) - if intStringLen < 0 { - return ErrInvalidLengthDmmaster - } - postIndex := iNdEx + intStringLen - if postIndex > l { - return io.ErrUnexpectedEOF - } - m.ID = string(dAtA[iNdEx:postIndex]) - iNdEx = postIndex + m.Result = bool(v != 0) case 2: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Task", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Msg", wireType) } var stringLen uint64 for shift := uint(0); ; shift += 7 { @@ -7052,7 +6929,7 @@ func (m *DDLLock) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= (uint64(b) & 0x7F) << shift + stringLen |= uint64(b&0x7F) << shift if b < 0x80 { break } @@ -7062,16 +6939,19 @@ func (m *DDLLock) Unmarshal(dAtA []byte) error { return ErrInvalidLengthDmmaster } postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthDmmaster + } if postIndex > l { return io.ErrUnexpectedEOF } - m.Task = string(dAtA[iNdEx:postIndex]) + m.Msg = string(dAtA[iNdEx:postIndex]) iNdEx = postIndex case 3: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Owner", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Workers", wireType) } - var stringLen uint64 + var msglen int for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowDmmaster @@ -7081,53 +6961,82 @@ func (m *DDLLock) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= (uint64(b) & 0x7F) << shift + msglen |= int(b&0x7F) << shift if b < 0x80 { break } } - intStringLen := int(stringLen) - if intStringLen < 0 { + if msglen < 0 { + return ErrInvalidLengthDmmaster + } + postIndex := iNdEx + msglen + if postIndex < 0 { return ErrInvalidLengthDmmaster } - postIndex := iNdEx + intStringLen if postIndex > l { return io.ErrUnexpectedEOF } - m.Owner = string(dAtA[iNdEx:postIndex]) + m.Workers = append(m.Workers, &QueryErrorResponse{}) + if err := m.Workers[len(m.Workers)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } iNdEx = postIndex - case 4: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field DDLs", wireType) + default: + iNdEx = preIndex + skippy, err := skipDmmaster(dAtA[iNdEx:]) + if err != nil { + return err } - 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 - } + if skippy < 0 { + return ErrInvalidLengthDmmaster } - intStringLen := int(stringLen) - if intStringLen < 0 { + if (iNdEx + skippy) < 0 { return ErrInvalidLengthDmmaster } - postIndex := iNdEx + intStringLen - if postIndex > l { + if (iNdEx + skippy) > l { return io.ErrUnexpectedEOF } - m.DDLs = append(m.DDLs, string(dAtA[iNdEx:postIndex])) - iNdEx = postIndex - case 5: + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *ShowDDLLocksRequest) 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: ShowDDLLocksRequest: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ShowDDLLocksRequest: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Synced", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Task", wireType) } var stringLen uint64 for shift := uint(0); ; shift += 7 { @@ -7139,7 +7048,7 @@ func (m *DDLLock) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= (uint64(b) & 0x7F) << shift + stringLen |= uint64(b&0x7F) << shift if b < 0x80 { break } @@ -7149,14 +7058,17 @@ func (m *DDLLock) Unmarshal(dAtA []byte) error { return ErrInvalidLengthDmmaster } postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthDmmaster + } if postIndex > l { return io.ErrUnexpectedEOF } - m.Synced = append(m.Synced, string(dAtA[iNdEx:postIndex])) + m.Task = string(dAtA[iNdEx:postIndex]) iNdEx = postIndex - case 6: + case 2: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Unsynced", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Sources", wireType) } var stringLen uint64 for shift := uint(0); ; shift += 7 { @@ -7168,7 +7080,7 @@ func (m *DDLLock) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= (uint64(b) & 0x7F) << shift + stringLen |= uint64(b&0x7F) << shift if b < 0x80 { break } @@ -7178,10 +7090,13 @@ func (m *DDLLock) Unmarshal(dAtA []byte) error { return ErrInvalidLengthDmmaster } postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthDmmaster + } if postIndex > l { return io.ErrUnexpectedEOF } - m.Unsynced = append(m.Unsynced, string(dAtA[iNdEx:postIndex])) + m.Sources = append(m.Sources, string(dAtA[iNdEx:postIndex])) iNdEx = postIndex default: iNdEx = preIndex @@ -7192,6 +7107,9 @@ func (m *DDLLock) Unmarshal(dAtA []byte) error { if skippy < 0 { return ErrInvalidLengthDmmaster } + if (iNdEx + skippy) < 0 { + return ErrInvalidLengthDmmaster + } if (iNdEx + skippy) > l { return io.ErrUnexpectedEOF } @@ -7204,7 +7122,7 @@ func (m *DDLLock) Unmarshal(dAtA []byte) error { } return nil } -func (m *ShowDDLLocksResponse) Unmarshal(dAtA []byte) error { +func (m *DDLLock) Unmarshal(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -7219,7 +7137,7 @@ func (m *ShowDDLLocksResponse) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - wire |= (uint64(b) & 0x7F) << shift + wire |= uint64(b&0x7F) << shift if b < 0x80 { break } @@ -7227,35 +7145,15 @@ func (m *ShowDDLLocksResponse) Unmarshal(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: ShowDDLLocksResponse: wiretype end group for non-group") + return fmt.Errorf("proto: DDLLock: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: ShowDDLLocksResponse: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: DDLLock: 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) + return fmt.Errorf("proto: wrong wireType = %d for field ID", wireType) } var stringLen uint64 for shift := uint(0); ; shift += 7 { @@ -7267,7 +7165,7 @@ func (m *ShowDDLLocksResponse) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= (uint64(b) & 0x7F) << shift + stringLen |= uint64(b&0x7F) << shift if b < 0x80 { break } @@ -7277,16 +7175,19 @@ func (m *ShowDDLLocksResponse) Unmarshal(dAtA []byte) error { return ErrInvalidLengthDmmaster } postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthDmmaster + } if postIndex > l { return io.ErrUnexpectedEOF } - m.Msg = string(dAtA[iNdEx:postIndex]) + m.ID = string(dAtA[iNdEx:postIndex]) iNdEx = postIndex - case 3: + case 2: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Locks", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Task", wireType) } - var msglen int + var stringLen uint64 for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowDmmaster @@ -7296,76 +7197,27 @@ func (m *ShowDDLLocksResponse) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - msglen |= (int(b) & 0x7F) << shift + stringLen |= uint64(b&0x7F) << shift if b < 0x80 { break } } - if msglen < 0 { + intStringLen := int(stringLen) + if intStringLen < 0 { return ErrInvalidLengthDmmaster } - postIndex := iNdEx + msglen - if postIndex > l { - return io.ErrUnexpectedEOF - } - m.Locks = append(m.Locks, &DDLLock{}) - if err := m.Locks[len(m.Locks)-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 { + postIndex := iNdEx + intStringLen + if postIndex < 0 { return ErrInvalidLengthDmmaster } - if (iNdEx + skippy) > l { - return io.ErrUnexpectedEOF - } - iNdEx += skippy - } - } - - if iNdEx > l { - return io.ErrUnexpectedEOF - } - return nil -} -func (m *UnlockDDLLockRequest) 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 { + if postIndex > 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: UnlockDDLLockRequest: wiretype end group for non-group") - } - if fieldNum <= 0 { - return fmt.Errorf("proto: UnlockDDLLockRequest: illegal tag %d (wire type %d)", fieldNum, wire) - } - switch fieldNum { - case 1: + m.Task = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 3: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field ID", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Owner", wireType) } var stringLen uint64 for shift := uint(0); ; shift += 7 { @@ -7377,7 +7229,7 @@ func (m *UnlockDDLLockRequest) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= (uint64(b) & 0x7F) << shift + stringLen |= uint64(b&0x7F) << shift if b < 0x80 { break } @@ -7387,14 +7239,17 @@ func (m *UnlockDDLLockRequest) Unmarshal(dAtA []byte) error { return ErrInvalidLengthDmmaster } postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthDmmaster + } if postIndex > l { return io.ErrUnexpectedEOF } - m.ID = string(dAtA[iNdEx:postIndex]) + m.Owner = string(dAtA[iNdEx:postIndex]) iNdEx = postIndex - case 2: + case 4: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field ReplaceOwner", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field DDLs", wireType) } var stringLen uint64 for shift := uint(0); ; shift += 7 { @@ -7406,7 +7261,7 @@ func (m *UnlockDDLLockRequest) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= (uint64(b) & 0x7F) << shift + stringLen |= uint64(b&0x7F) << shift if b < 0x80 { break } @@ -7416,14 +7271,17 @@ func (m *UnlockDDLLockRequest) Unmarshal(dAtA []byte) error { return ErrInvalidLengthDmmaster } postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthDmmaster + } if postIndex > l { return io.ErrUnexpectedEOF } - m.ReplaceOwner = string(dAtA[iNdEx:postIndex]) + m.DDLs = append(m.DDLs, string(dAtA[iNdEx:postIndex])) iNdEx = postIndex - case 3: + case 5: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Workers", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Synced", wireType) } var stringLen uint64 for shift := uint(0); ; shift += 7 { @@ -7435,7 +7293,7 @@ func (m *UnlockDDLLockRequest) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= (uint64(b) & 0x7F) << shift + stringLen |= uint64(b&0x7F) << shift if b < 0x80 { break } @@ -7445,16 +7303,19 @@ func (m *UnlockDDLLockRequest) Unmarshal(dAtA []byte) error { return ErrInvalidLengthDmmaster } postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthDmmaster + } if postIndex > l { return io.ErrUnexpectedEOF } - m.Workers = append(m.Workers, string(dAtA[iNdEx:postIndex])) + m.Synced = append(m.Synced, string(dAtA[iNdEx:postIndex])) iNdEx = postIndex - case 4: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field ForceRemove", wireType) + case 6: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Unsynced", wireType) } - var v int + var stringLen uint64 for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowDmmaster @@ -7464,12 +7325,24 @@ func (m *UnlockDDLLockRequest) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - v |= (int(b) & 0x7F) << shift + stringLen |= uint64(b&0x7F) << shift if b < 0x80 { break } } - m.ForceRemove = bool(v != 0) + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLengthDmmaster + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthDmmaster + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Unsynced = append(m.Unsynced, string(dAtA[iNdEx:postIndex])) + iNdEx = postIndex default: iNdEx = preIndex skippy, err := skipDmmaster(dAtA[iNdEx:]) @@ -7479,6 +7352,9 @@ func (m *UnlockDDLLockRequest) Unmarshal(dAtA []byte) error { if skippy < 0 { return ErrInvalidLengthDmmaster } + if (iNdEx + skippy) < 0 { + return ErrInvalidLengthDmmaster + } if (iNdEx + skippy) > l { return io.ErrUnexpectedEOF } @@ -7491,7 +7367,7 @@ func (m *UnlockDDLLockRequest) Unmarshal(dAtA []byte) error { } return nil } -func (m *UnlockDDLLockResponse) Unmarshal(dAtA []byte) error { +func (m *ShowDDLLocksResponse) Unmarshal(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -7506,7 +7382,7 @@ func (m *UnlockDDLLockResponse) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - wire |= (uint64(b) & 0x7F) << shift + wire |= uint64(b&0x7F) << shift if b < 0x80 { break } @@ -7514,10 +7390,10 @@ func (m *UnlockDDLLockResponse) Unmarshal(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: UnlockDDLLockResponse: wiretype end group for non-group") + return fmt.Errorf("proto: ShowDDLLocksResponse: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: UnlockDDLLockResponse: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: ShowDDLLocksResponse: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: @@ -7534,7 +7410,7 @@ func (m *UnlockDDLLockResponse) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - v |= (int(b) & 0x7F) << shift + v |= int(b&0x7F) << shift if b < 0x80 { break } @@ -7554,7 +7430,7 @@ func (m *UnlockDDLLockResponse) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= (uint64(b) & 0x7F) << shift + stringLen |= uint64(b&0x7F) << shift if b < 0x80 { break } @@ -7564,6 +7440,9 @@ func (m *UnlockDDLLockResponse) Unmarshal(dAtA []byte) error { return ErrInvalidLengthDmmaster } postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthDmmaster + } if postIndex > l { return io.ErrUnexpectedEOF } @@ -7571,7 +7450,7 @@ func (m *UnlockDDLLockResponse) Unmarshal(dAtA []byte) error { iNdEx = postIndex case 3: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Workers", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Locks", wireType) } var msglen int for shift := uint(0); ; shift += 7 { @@ -7583,7 +7462,7 @@ func (m *UnlockDDLLockResponse) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - msglen |= (int(b) & 0x7F) << shift + msglen |= int(b&0x7F) << shift if b < 0x80 { break } @@ -7592,11 +7471,14 @@ func (m *UnlockDDLLockResponse) Unmarshal(dAtA []byte) error { 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 { + m.Locks = append(m.Locks, &DDLLock{}) + if err := m.Locks[len(m.Locks)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { return err } iNdEx = postIndex @@ -7609,6 +7491,9 @@ func (m *UnlockDDLLockResponse) Unmarshal(dAtA []byte) error { if skippy < 0 { return ErrInvalidLengthDmmaster } + if (iNdEx + skippy) < 0 { + return ErrInvalidLengthDmmaster + } if (iNdEx + skippy) > l { return io.ErrUnexpectedEOF } @@ -7621,7 +7506,7 @@ func (m *UnlockDDLLockResponse) Unmarshal(dAtA []byte) error { } return nil } -func (m *BreakWorkerDDLLockRequest) Unmarshal(dAtA []byte) error { +func (m *UnlockDDLLockRequest) Unmarshal(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -7636,7 +7521,7 @@ func (m *BreakWorkerDDLLockRequest) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - wire |= (uint64(b) & 0x7F) << shift + wire |= uint64(b&0x7F) << shift if b < 0x80 { break } @@ -7644,15 +7529,15 @@ func (m *BreakWorkerDDLLockRequest) Unmarshal(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: BreakWorkerDDLLockRequest: wiretype end group for non-group") + return fmt.Errorf("proto: UnlockDDLLockRequest: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: BreakWorkerDDLLockRequest: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: UnlockDDLLockRequest: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Workers", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field ID", wireType) } var stringLen uint64 for shift := uint(0); ; shift += 7 { @@ -7664,7 +7549,7 @@ func (m *BreakWorkerDDLLockRequest) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= (uint64(b) & 0x7F) << shift + stringLen |= uint64(b&0x7F) << shift if b < 0x80 { break } @@ -7674,14 +7559,17 @@ func (m *BreakWorkerDDLLockRequest) Unmarshal(dAtA []byte) error { return ErrInvalidLengthDmmaster } postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthDmmaster + } if postIndex > l { return io.ErrUnexpectedEOF } - m.Workers = append(m.Workers, string(dAtA[iNdEx:postIndex])) + m.ID = string(dAtA[iNdEx:postIndex]) iNdEx = postIndex case 2: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Task", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field ReplaceOwner", wireType) } var stringLen uint64 for shift := uint(0); ; shift += 7 { @@ -7693,7 +7581,7 @@ func (m *BreakWorkerDDLLockRequest) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= (uint64(b) & 0x7F) << shift + stringLen |= uint64(b&0x7F) << shift if b < 0x80 { break } @@ -7703,14 +7591,17 @@ func (m *BreakWorkerDDLLockRequest) Unmarshal(dAtA []byte) error { return ErrInvalidLengthDmmaster } postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthDmmaster + } if postIndex > l { return io.ErrUnexpectedEOF } - m.Task = string(dAtA[iNdEx:postIndex]) + m.ReplaceOwner = string(dAtA[iNdEx:postIndex]) iNdEx = postIndex case 3: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field RemoveLockID", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Sources", wireType) } var stringLen uint64 for shift := uint(0); ; shift += 7 { @@ -7722,7 +7613,7 @@ func (m *BreakWorkerDDLLockRequest) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= (uint64(b) & 0x7F) << shift + stringLen |= uint64(b&0x7F) << shift if b < 0x80 { break } @@ -7732,34 +7623,17 @@ func (m *BreakWorkerDDLLockRequest) Unmarshal(dAtA []byte) error { return ErrInvalidLengthDmmaster } postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthDmmaster + } if postIndex > l { return io.ErrUnexpectedEOF } - m.RemoveLockID = string(dAtA[iNdEx:postIndex]) + 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 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) + return fmt.Errorf("proto: wrong wireType = %d for field ForceRemove", wireType) } var v int for shift := uint(0); ; shift += 7 { @@ -7771,12 +7645,12 @@ func (m *BreakWorkerDDLLockRequest) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - v |= (int(b) & 0x7F) << shift + v |= int(b&0x7F) << shift if b < 0x80 { break } } - m.SkipDDL = bool(v != 0) + m.ForceRemove = bool(v != 0) default: iNdEx = preIndex skippy, err := skipDmmaster(dAtA[iNdEx:]) @@ -7786,6 +7660,9 @@ func (m *BreakWorkerDDLLockRequest) Unmarshal(dAtA []byte) error { if skippy < 0 { return ErrInvalidLengthDmmaster } + if (iNdEx + skippy) < 0 { + return ErrInvalidLengthDmmaster + } if (iNdEx + skippy) > l { return io.ErrUnexpectedEOF } @@ -7798,7 +7675,7 @@ func (m *BreakWorkerDDLLockRequest) Unmarshal(dAtA []byte) error { } return nil } -func (m *BreakWorkerDDLLockResponse) Unmarshal(dAtA []byte) error { +func (m *UnlockDDLLockResponse) Unmarshal(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -7813,7 +7690,7 @@ func (m *BreakWorkerDDLLockResponse) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - wire |= (uint64(b) & 0x7F) << shift + wire |= uint64(b&0x7F) << shift if b < 0x80 { break } @@ -7821,10 +7698,10 @@ func (m *BreakWorkerDDLLockResponse) Unmarshal(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: BreakWorkerDDLLockResponse: wiretype end group for non-group") + return fmt.Errorf("proto: UnlockDDLLockResponse: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: BreakWorkerDDLLockResponse: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: UnlockDDLLockResponse: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: @@ -7841,7 +7718,7 @@ func (m *BreakWorkerDDLLockResponse) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - v |= (int(b) & 0x7F) << shift + v |= int(b&0x7F) << shift if b < 0x80 { break } @@ -7861,7 +7738,7 @@ func (m *BreakWorkerDDLLockResponse) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= (uint64(b) & 0x7F) << shift + stringLen |= uint64(b&0x7F) << shift if b < 0x80 { break } @@ -7871,6 +7748,9 @@ func (m *BreakWorkerDDLLockResponse) Unmarshal(dAtA []byte) error { return ErrInvalidLengthDmmaster } postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthDmmaster + } if postIndex > l { return io.ErrUnexpectedEOF } @@ -7890,7 +7770,7 @@ func (m *BreakWorkerDDLLockResponse) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - msglen |= (int(b) & 0x7F) << shift + msglen |= int(b&0x7F) << shift if b < 0x80 { break } @@ -7899,6 +7779,9 @@ func (m *BreakWorkerDDLLockResponse) Unmarshal(dAtA []byte) error { return ErrInvalidLengthDmmaster } postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthDmmaster + } if postIndex > l { return io.ErrUnexpectedEOF } @@ -7916,6 +7799,9 @@ func (m *BreakWorkerDDLLockResponse) Unmarshal(dAtA []byte) error { if skippy < 0 { return ErrInvalidLengthDmmaster } + if (iNdEx + skippy) < 0 { + return ErrInvalidLengthDmmaster + } if (iNdEx + skippy) > l { return io.ErrUnexpectedEOF } @@ -7928,7 +7814,7 @@ func (m *BreakWorkerDDLLockResponse) Unmarshal(dAtA []byte) error { } return nil } -func (m *SwitchWorkerRelayMasterRequest) Unmarshal(dAtA []byte) error { +func (m *BreakWorkerDDLLockRequest) Unmarshal(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -7943,7 +7829,7 @@ func (m *SwitchWorkerRelayMasterRequest) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - wire |= (uint64(b) & 0x7F) << shift + wire |= uint64(b&0x7F) << shift if b < 0x80 { break } @@ -7951,15 +7837,79 @@ func (m *SwitchWorkerRelayMasterRequest) Unmarshal(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: SwitchWorkerRelayMasterRequest: wiretype end group for non-group") + return fmt.Errorf("proto: BreakWorkerDDLLockRequest: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: SwitchWorkerRelayMasterRequest: illegal tag %d (wire type %d)", fieldNum, wire) + 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 Workers", wireType) + 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 { @@ -7971,7 +7921,7 @@ func (m *SwitchWorkerRelayMasterRequest) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= (uint64(b) & 0x7F) << shift + stringLen |= uint64(b&0x7F) << shift if b < 0x80 { break } @@ -7981,11 +7931,54 @@ func (m *SwitchWorkerRelayMasterRequest) Unmarshal(dAtA []byte) error { return ErrInvalidLengthDmmaster } postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthDmmaster + } if postIndex > l { return io.ErrUnexpectedEOF } - m.Workers = append(m.Workers, string(dAtA[iNdEx:postIndex])) + 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:]) @@ -7995,6 +7988,9 @@ func (m *SwitchWorkerRelayMasterRequest) Unmarshal(dAtA []byte) error { if skippy < 0 { return ErrInvalidLengthDmmaster } + if (iNdEx + skippy) < 0 { + return ErrInvalidLengthDmmaster + } if (iNdEx + skippy) > l { return io.ErrUnexpectedEOF } @@ -8007,7 +8003,7 @@ func (m *SwitchWorkerRelayMasterRequest) Unmarshal(dAtA []byte) error { } return nil } -func (m *SwitchWorkerRelayMasterResponse) Unmarshal(dAtA []byte) error { +func (m *BreakWorkerDDLLockResponse) Unmarshal(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -8022,7 +8018,7 @@ func (m *SwitchWorkerRelayMasterResponse) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - wire |= (uint64(b) & 0x7F) << shift + wire |= uint64(b&0x7F) << shift if b < 0x80 { break } @@ -8030,10 +8026,10 @@ func (m *SwitchWorkerRelayMasterResponse) Unmarshal(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: SwitchWorkerRelayMasterResponse: wiretype end group for non-group") + return fmt.Errorf("proto: BreakWorkerDDLLockResponse: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: SwitchWorkerRelayMasterResponse: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: BreakWorkerDDLLockResponse: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: @@ -8050,7 +8046,7 @@ func (m *SwitchWorkerRelayMasterResponse) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - v |= (int(b) & 0x7F) << shift + v |= int(b&0x7F) << shift if b < 0x80 { break } @@ -8070,7 +8066,7 @@ func (m *SwitchWorkerRelayMasterResponse) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= (uint64(b) & 0x7F) << shift + stringLen |= uint64(b&0x7F) << shift if b < 0x80 { break } @@ -8080,6 +8076,9 @@ func (m *SwitchWorkerRelayMasterResponse) Unmarshal(dAtA []byte) error { return ErrInvalidLengthDmmaster } postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthDmmaster + } if postIndex > l { return io.ErrUnexpectedEOF } @@ -8099,7 +8098,7 @@ func (m *SwitchWorkerRelayMasterResponse) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - msglen |= (int(b) & 0x7F) << shift + msglen |= int(b&0x7F) << shift if b < 0x80 { break } @@ -8108,6 +8107,9 @@ func (m *SwitchWorkerRelayMasterResponse) Unmarshal(dAtA []byte) error { return ErrInvalidLengthDmmaster } postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthDmmaster + } if postIndex > l { return io.ErrUnexpectedEOF } @@ -8125,6 +8127,9 @@ func (m *SwitchWorkerRelayMasterResponse) Unmarshal(dAtA []byte) error { if skippy < 0 { return ErrInvalidLengthDmmaster } + if (iNdEx + skippy) < 0 { + return ErrInvalidLengthDmmaster + } if (iNdEx + skippy) > l { return io.ErrUnexpectedEOF } @@ -8137,7 +8142,7 @@ func (m *SwitchWorkerRelayMasterResponse) Unmarshal(dAtA []byte) error { } return nil } -func (m *OperateWorkerRelayRequest) Unmarshal(dAtA []byte) error { +func (m *SwitchWorkerRelayMasterRequest) Unmarshal(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -8152,7 +8157,7 @@ func (m *OperateWorkerRelayRequest) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - wire |= (uint64(b) & 0x7F) << shift + wire |= uint64(b&0x7F) << shift if b < 0x80 { break } @@ -8160,34 +8165,15 @@ func (m *OperateWorkerRelayRequest) Unmarshal(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: OperateWorkerRelayRequest: wiretype end group for non-group") - } - if fieldNum <= 0 { - return fmt.Errorf("proto: OperateWorkerRelayRequest: illegal tag %d (wire type %d)", fieldNum, wire) - } - switch fieldNum { - case 1: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field Op", wireType) - } - m.Op = 0 - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowDmmaster - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - m.Op |= (RelayOp(b) & 0x7F) << shift - if b < 0x80 { - break - } - } - case 2: + return fmt.Errorf("proto: SwitchWorkerRelayMasterRequest: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: SwitchWorkerRelayMasterRequest: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Workers", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Sources", wireType) } var stringLen uint64 for shift := uint(0); ; shift += 7 { @@ -8199,7 +8185,7 @@ func (m *OperateWorkerRelayRequest) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= (uint64(b) & 0x7F) << shift + stringLen |= uint64(b&0x7F) << shift if b < 0x80 { break } @@ -8209,10 +8195,13 @@ func (m *OperateWorkerRelayRequest) Unmarshal(dAtA []byte) error { return ErrInvalidLengthDmmaster } postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthDmmaster + } if postIndex > l { return io.ErrUnexpectedEOF } - m.Workers = append(m.Workers, string(dAtA[iNdEx:postIndex])) + m.Sources = append(m.Sources, string(dAtA[iNdEx:postIndex])) iNdEx = postIndex default: iNdEx = preIndex @@ -8223,6 +8212,9 @@ func (m *OperateWorkerRelayRequest) Unmarshal(dAtA []byte) error { if skippy < 0 { return ErrInvalidLengthDmmaster } + if (iNdEx + skippy) < 0 { + return ErrInvalidLengthDmmaster + } if (iNdEx + skippy) > l { return io.ErrUnexpectedEOF } @@ -8235,7 +8227,7 @@ func (m *OperateWorkerRelayRequest) Unmarshal(dAtA []byte) error { } return nil } -func (m *OperateWorkerRelayResponse) Unmarshal(dAtA []byte) error { +func (m *SwitchWorkerRelayMasterResponse) Unmarshal(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -8250,7 +8242,7 @@ func (m *OperateWorkerRelayResponse) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - wire |= (uint64(b) & 0x7F) << shift + wire |= uint64(b&0x7F) << shift if b < 0x80 { break } @@ -8258,32 +8250,13 @@ func (m *OperateWorkerRelayResponse) Unmarshal(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: OperateWorkerRelayResponse: wiretype end group for non-group") + return fmt.Errorf("proto: SwitchWorkerRelayMasterResponse: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: OperateWorkerRelayResponse: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: SwitchWorkerRelayMasterResponse: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field Op", wireType) - } - m.Op = 0 - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowDmmaster - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - m.Op |= (RelayOp(b) & 0x7F) << shift - if b < 0x80 { - break - } - } - case 2: if wireType != 0 { return fmt.Errorf("proto: wrong wireType = %d for field Result", wireType) } @@ -8297,13 +8270,13 @@ func (m *OperateWorkerRelayResponse) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - v |= (int(b) & 0x7F) << shift + v |= int(b&0x7F) << shift if b < 0x80 { break } } m.Result = bool(v != 0) - case 3: + case 2: if wireType != 2 { return fmt.Errorf("proto: wrong wireType = %d for field Msg", wireType) } @@ -8317,7 +8290,7 @@ func (m *OperateWorkerRelayResponse) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= (uint64(b) & 0x7F) << shift + stringLen |= uint64(b&0x7F) << shift if b < 0x80 { break } @@ -8327,12 +8300,15 @@ func (m *OperateWorkerRelayResponse) Unmarshal(dAtA []byte) error { 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 4: + case 3: if wireType != 2 { return fmt.Errorf("proto: wrong wireType = %d for field Workers", wireType) } @@ -8346,7 +8322,7 @@ func (m *OperateWorkerRelayResponse) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - msglen |= (int(b) & 0x7F) << shift + msglen |= int(b&0x7F) << shift if b < 0x80 { break } @@ -8355,10 +8331,13 @@ func (m *OperateWorkerRelayResponse) Unmarshal(dAtA []byte) error { return ErrInvalidLengthDmmaster } postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthDmmaster + } if postIndex > l { return io.ErrUnexpectedEOF } - m.Workers = append(m.Workers, &OperateRelayResponse{}) + m.Workers = append(m.Workers, &CommonWorkerResponse{}) if err := m.Workers[len(m.Workers)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -8372,54 +8351,7 @@ func (m *OperateWorkerRelayResponse) Unmarshal(dAtA []byte) error { if skippy < 0 { return ErrInvalidLengthDmmaster } - if (iNdEx + skippy) > l { - return io.ErrUnexpectedEOF - } - iNdEx += skippy - } - } - - if iNdEx > l { - return io.ErrUnexpectedEOF - } - return nil -} -func (m *RefreshWorkerTasksRequest) 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: RefreshWorkerTasksRequest: wiretype end group for non-group") - } - if fieldNum <= 0 { - return fmt.Errorf("proto: RefreshWorkerTasksRequest: illegal tag %d (wire type %d)", fieldNum, wire) - } - switch fieldNum { - default: - iNdEx = preIndex - skippy, err := skipDmmaster(dAtA[iNdEx:]) - if err != nil { - return err - } - if skippy < 0 { + if (iNdEx + skippy) < 0 { return ErrInvalidLengthDmmaster } if (iNdEx + skippy) > l { @@ -8434,7 +8366,7 @@ func (m *RefreshWorkerTasksRequest) Unmarshal(dAtA []byte) error { } return nil } -func (m *RefreshWorkerTasksMsg) Unmarshal(dAtA []byte) error { +func (m *OperateWorkerRelayRequest) Unmarshal(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -8449,7 +8381,7 @@ func (m *RefreshWorkerTasksMsg) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - wire |= (uint64(b) & 0x7F) << shift + wire |= uint64(b&0x7F) << shift if b < 0x80 { break } @@ -8457,17 +8389,17 @@ func (m *RefreshWorkerTasksMsg) Unmarshal(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: RefreshWorkerTasksMsg: wiretype end group for non-group") + return fmt.Errorf("proto: OperateWorkerRelayRequest: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: RefreshWorkerTasksMsg: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: OperateWorkerRelayRequest: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Worker", wireType) + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Op", wireType) } - var stringLen uint64 + m.Op = 0 for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowDmmaster @@ -8477,24 +8409,14 @@ func (m *RefreshWorkerTasksMsg) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= (uint64(b) & 0x7F) << shift + m.Op |= RelayOp(b&0x7F) << shift if b < 0x80 { break } } - intStringLen := int(stringLen) - if intStringLen < 0 { - return ErrInvalidLengthDmmaster - } - postIndex := iNdEx + intStringLen - if postIndex > l { - return io.ErrUnexpectedEOF - } - m.Worker = string(dAtA[iNdEx:postIndex]) - iNdEx = postIndex case 2: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Msg", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Sources", wireType) } var stringLen uint64 for shift := uint(0); ; shift += 7 { @@ -8506,7 +8428,7 @@ func (m *RefreshWorkerTasksMsg) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= (uint64(b) & 0x7F) << shift + stringLen |= uint64(b&0x7F) << shift if b < 0x80 { break } @@ -8516,10 +8438,13 @@ func (m *RefreshWorkerTasksMsg) Unmarshal(dAtA []byte) error { return ErrInvalidLengthDmmaster } postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthDmmaster + } if postIndex > l { return io.ErrUnexpectedEOF } - m.Msg = string(dAtA[iNdEx:postIndex]) + m.Sources = append(m.Sources, string(dAtA[iNdEx:postIndex])) iNdEx = postIndex default: iNdEx = preIndex @@ -8530,6 +8455,9 @@ func (m *RefreshWorkerTasksMsg) Unmarshal(dAtA []byte) error { if skippy < 0 { return ErrInvalidLengthDmmaster } + if (iNdEx + skippy) < 0 { + return ErrInvalidLengthDmmaster + } if (iNdEx + skippy) > l { return io.ErrUnexpectedEOF } @@ -8542,7 +8470,7 @@ func (m *RefreshWorkerTasksMsg) Unmarshal(dAtA []byte) error { } return nil } -func (m *RefreshWorkerTasksResponse) Unmarshal(dAtA []byte) error { +func (m *OperateWorkerRelayResponse) Unmarshal(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -8557,7 +8485,7 @@ func (m *RefreshWorkerTasksResponse) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - wire |= (uint64(b) & 0x7F) << shift + wire |= uint64(b&0x7F) << shift if b < 0x80 { break } @@ -8565,13 +8493,32 @@ func (m *RefreshWorkerTasksResponse) Unmarshal(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: RefreshWorkerTasksResponse: wiretype end group for non-group") + return fmt.Errorf("proto: OperateWorkerRelayResponse: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: RefreshWorkerTasksResponse: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: OperateWorkerRelayResponse: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Op", wireType) + } + m.Op = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowDmmaster + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.Op |= RelayOp(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 2: if wireType != 0 { return fmt.Errorf("proto: wrong wireType = %d for field Result", wireType) } @@ -8585,13 +8532,45 @@ func (m *RefreshWorkerTasksResponse) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - v |= (int(b) & 0x7F) << shift + v |= int(b&0x7F) << shift if b < 0x80 { break } } m.Result = bool(v != 0) - case 2: + case 3: + 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 4: if wireType != 2 { return fmt.Errorf("proto: wrong wireType = %d for field Workers", wireType) } @@ -8605,7 +8584,7 @@ func (m *RefreshWorkerTasksResponse) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - msglen |= (int(b) & 0x7F) << shift + msglen |= int(b&0x7F) << shift if b < 0x80 { break } @@ -8614,10 +8593,13 @@ func (m *RefreshWorkerTasksResponse) Unmarshal(dAtA []byte) error { return ErrInvalidLengthDmmaster } postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthDmmaster + } if postIndex > l { return io.ErrUnexpectedEOF } - m.Workers = append(m.Workers, &RefreshWorkerTasksMsg{}) + m.Workers = append(m.Workers, &OperateRelayResponse{}) if err := m.Workers[len(m.Workers)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -8631,6 +8613,9 @@ func (m *RefreshWorkerTasksResponse) Unmarshal(dAtA []byte) error { if skippy < 0 { return ErrInvalidLengthDmmaster } + if (iNdEx + skippy) < 0 { + return ErrInvalidLengthDmmaster + } if (iNdEx + skippy) > l { return io.ErrUnexpectedEOF } @@ -8658,7 +8643,7 @@ func (m *HandleSQLsRequest) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - wire |= (uint64(b) & 0x7F) << shift + wire |= uint64(b&0x7F) << shift if b < 0x80 { break } @@ -8686,7 +8671,7 @@ func (m *HandleSQLsRequest) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= (uint64(b) & 0x7F) << shift + stringLen |= uint64(b&0x7F) << shift if b < 0x80 { break } @@ -8696,6 +8681,9 @@ func (m *HandleSQLsRequest) Unmarshal(dAtA []byte) error { return ErrInvalidLengthDmmaster } postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthDmmaster + } if postIndex > l { return io.ErrUnexpectedEOF } @@ -8715,7 +8703,7 @@ func (m *HandleSQLsRequest) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - m.Op |= (SQLOp(b) & 0x7F) << shift + m.Op |= SQLOp(b&0x7F) << shift if b < 0x80 { break } @@ -8734,7 +8722,7 @@ func (m *HandleSQLsRequest) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= (uint64(b) & 0x7F) << shift + stringLen |= uint64(b&0x7F) << shift if b < 0x80 { break } @@ -8744,6 +8732,9 @@ func (m *HandleSQLsRequest) Unmarshal(dAtA []byte) error { return ErrInvalidLengthDmmaster } postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthDmmaster + } if postIndex > l { return io.ErrUnexpectedEOF } @@ -8763,7 +8754,7 @@ func (m *HandleSQLsRequest) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= (uint64(b) & 0x7F) << shift + stringLen |= uint64(b&0x7F) << shift if b < 0x80 { break } @@ -8773,6 +8764,9 @@ func (m *HandleSQLsRequest) Unmarshal(dAtA []byte) error { return ErrInvalidLengthDmmaster } postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthDmmaster + } if postIndex > l { return io.ErrUnexpectedEOF } @@ -8792,7 +8786,7 @@ func (m *HandleSQLsRequest) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= (uint64(b) & 0x7F) << shift + stringLen |= uint64(b&0x7F) << shift if b < 0x80 { break } @@ -8802,6 +8796,9 @@ func (m *HandleSQLsRequest) Unmarshal(dAtA []byte) error { return ErrInvalidLengthDmmaster } postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthDmmaster + } if postIndex > l { return io.ErrUnexpectedEOF } @@ -8821,7 +8818,7 @@ func (m *HandleSQLsRequest) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= (uint64(b) & 0x7F) << shift + stringLen |= uint64(b&0x7F) << shift if b < 0x80 { break } @@ -8831,6 +8828,9 @@ func (m *HandleSQLsRequest) Unmarshal(dAtA []byte) error { return ErrInvalidLengthDmmaster } postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthDmmaster + } if postIndex > l { return io.ErrUnexpectedEOF } @@ -8850,7 +8850,7 @@ func (m *HandleSQLsRequest) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - v |= (int(b) & 0x7F) << shift + v |= int(b&0x7F) << shift if b < 0x80 { break } @@ -8865,6 +8865,9 @@ func (m *HandleSQLsRequest) Unmarshal(dAtA []byte) error { if skippy < 0 { return ErrInvalidLengthDmmaster } + if (iNdEx + skippy) < 0 { + return ErrInvalidLengthDmmaster + } if (iNdEx + skippy) > l { return io.ErrUnexpectedEOF } @@ -8892,7 +8895,7 @@ func (m *HandleSQLsResponse) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - wire |= (uint64(b) & 0x7F) << shift + wire |= uint64(b&0x7F) << shift if b < 0x80 { break } @@ -8920,7 +8923,7 @@ func (m *HandleSQLsResponse) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - v |= (int(b) & 0x7F) << shift + v |= int(b&0x7F) << shift if b < 0x80 { break } @@ -8940,7 +8943,7 @@ func (m *HandleSQLsResponse) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= (uint64(b) & 0x7F) << shift + stringLen |= uint64(b&0x7F) << shift if b < 0x80 { break } @@ -8950,6 +8953,9 @@ func (m *HandleSQLsResponse) Unmarshal(dAtA []byte) error { return ErrInvalidLengthDmmaster } postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthDmmaster + } if postIndex > l { return io.ErrUnexpectedEOF } @@ -8969,7 +8975,7 @@ func (m *HandleSQLsResponse) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - msglen |= (int(b) & 0x7F) << shift + msglen |= int(b&0x7F) << shift if b < 0x80 { break } @@ -8978,6 +8984,9 @@ func (m *HandleSQLsResponse) Unmarshal(dAtA []byte) error { return ErrInvalidLengthDmmaster } postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthDmmaster + } if postIndex > l { return io.ErrUnexpectedEOF } @@ -8995,6 +9004,9 @@ func (m *HandleSQLsResponse) Unmarshal(dAtA []byte) error { if skippy < 0 { return ErrInvalidLengthDmmaster } + if (iNdEx + skippy) < 0 { + return ErrInvalidLengthDmmaster + } if (iNdEx + skippy) > l { return io.ErrUnexpectedEOF } @@ -9022,7 +9034,7 @@ func (m *PurgeWorkerRelayRequest) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - wire |= (uint64(b) & 0x7F) << shift + wire |= uint64(b&0x7F) << shift if b < 0x80 { break } @@ -9038,7 +9050,7 @@ func (m *PurgeWorkerRelayRequest) Unmarshal(dAtA []byte) error { switch fieldNum { case 1: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Workers", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Sources", wireType) } var stringLen uint64 for shift := uint(0); ; shift += 7 { @@ -9050,7 +9062,7 @@ func (m *PurgeWorkerRelayRequest) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= (uint64(b) & 0x7F) << shift + stringLen |= uint64(b&0x7F) << shift if b < 0x80 { break } @@ -9060,10 +9072,13 @@ func (m *PurgeWorkerRelayRequest) Unmarshal(dAtA []byte) error { return ErrInvalidLengthDmmaster } postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthDmmaster + } if postIndex > l { return io.ErrUnexpectedEOF } - m.Workers = append(m.Workers, string(dAtA[iNdEx:postIndex])) + m.Sources = append(m.Sources, string(dAtA[iNdEx:postIndex])) iNdEx = postIndex case 2: if wireType != 0 { @@ -9079,7 +9094,7 @@ func (m *PurgeWorkerRelayRequest) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - v |= (int(b) & 0x7F) << shift + v |= int(b&0x7F) << shift if b < 0x80 { break } @@ -9099,7 +9114,7 @@ func (m *PurgeWorkerRelayRequest) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - m.Time |= (int64(b) & 0x7F) << shift + m.Time |= int64(b&0x7F) << shift if b < 0x80 { break } @@ -9118,7 +9133,7 @@ func (m *PurgeWorkerRelayRequest) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= (uint64(b) & 0x7F) << shift + stringLen |= uint64(b&0x7F) << shift if b < 0x80 { break } @@ -9128,6 +9143,9 @@ func (m *PurgeWorkerRelayRequest) Unmarshal(dAtA []byte) error { return ErrInvalidLengthDmmaster } postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthDmmaster + } if postIndex > l { return io.ErrUnexpectedEOF } @@ -9147,7 +9165,7 @@ func (m *PurgeWorkerRelayRequest) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= (uint64(b) & 0x7F) << shift + stringLen |= uint64(b&0x7F) << shift if b < 0x80 { break } @@ -9157,6 +9175,9 @@ func (m *PurgeWorkerRelayRequest) Unmarshal(dAtA []byte) error { return ErrInvalidLengthDmmaster } postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthDmmaster + } if postIndex > l { return io.ErrUnexpectedEOF } @@ -9171,6 +9192,9 @@ func (m *PurgeWorkerRelayRequest) Unmarshal(dAtA []byte) error { if skippy < 0 { return ErrInvalidLengthDmmaster } + if (iNdEx + skippy) < 0 { + return ErrInvalidLengthDmmaster + } if (iNdEx + skippy) > l { return io.ErrUnexpectedEOF } @@ -9198,7 +9222,7 @@ func (m *PurgeWorkerRelayResponse) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - wire |= (uint64(b) & 0x7F) << shift + wire |= uint64(b&0x7F) << shift if b < 0x80 { break } @@ -9226,7 +9250,7 @@ func (m *PurgeWorkerRelayResponse) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - v |= (int(b) & 0x7F) << shift + v |= int(b&0x7F) << shift if b < 0x80 { break } @@ -9246,7 +9270,7 @@ func (m *PurgeWorkerRelayResponse) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= (uint64(b) & 0x7F) << shift + stringLen |= uint64(b&0x7F) << shift if b < 0x80 { break } @@ -9256,6 +9280,9 @@ func (m *PurgeWorkerRelayResponse) Unmarshal(dAtA []byte) error { return ErrInvalidLengthDmmaster } postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthDmmaster + } if postIndex > l { return io.ErrUnexpectedEOF } @@ -9275,7 +9302,7 @@ func (m *PurgeWorkerRelayResponse) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - msglen |= (int(b) & 0x7F) << shift + msglen |= int(b&0x7F) << shift if b < 0x80 { break } @@ -9284,6 +9311,9 @@ func (m *PurgeWorkerRelayResponse) Unmarshal(dAtA []byte) error { return ErrInvalidLengthDmmaster } postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthDmmaster + } if postIndex > l { return io.ErrUnexpectedEOF } @@ -9301,6 +9331,9 @@ func (m *PurgeWorkerRelayResponse) Unmarshal(dAtA []byte) error { if skippy < 0 { return ErrInvalidLengthDmmaster } + if (iNdEx + skippy) < 0 { + return ErrInvalidLengthDmmaster + } if (iNdEx + skippy) > l { return io.ErrUnexpectedEOF } @@ -9328,7 +9361,7 @@ func (m *CheckTaskRequest) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - wire |= (uint64(b) & 0x7F) << shift + wire |= uint64(b&0x7F) << shift if b < 0x80 { break } @@ -9356,7 +9389,7 @@ func (m *CheckTaskRequest) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= (uint64(b) & 0x7F) << shift + stringLen |= uint64(b&0x7F) << shift if b < 0x80 { break } @@ -9366,6 +9399,9 @@ func (m *CheckTaskRequest) Unmarshal(dAtA []byte) error { return ErrInvalidLengthDmmaster } postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthDmmaster + } if postIndex > l { return io.ErrUnexpectedEOF } @@ -9380,6 +9416,9 @@ func (m *CheckTaskRequest) Unmarshal(dAtA []byte) error { if skippy < 0 { return ErrInvalidLengthDmmaster } + if (iNdEx + skippy) < 0 { + return ErrInvalidLengthDmmaster + } if (iNdEx + skippy) > l { return io.ErrUnexpectedEOF } @@ -9407,7 +9446,7 @@ func (m *CheckTaskResponse) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - wire |= (uint64(b) & 0x7F) << shift + wire |= uint64(b&0x7F) << shift if b < 0x80 { break } @@ -9435,7 +9474,7 @@ func (m *CheckTaskResponse) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - v |= (int(b) & 0x7F) << shift + v |= int(b&0x7F) << shift if b < 0x80 { break } @@ -9455,7 +9494,7 @@ func (m *CheckTaskResponse) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= (uint64(b) & 0x7F) << shift + stringLen |= uint64(b&0x7F) << shift if b < 0x80 { break } @@ -9465,6 +9504,9 @@ func (m *CheckTaskResponse) Unmarshal(dAtA []byte) error { return ErrInvalidLengthDmmaster } postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthDmmaster + } if postIndex > l { return io.ErrUnexpectedEOF } @@ -9479,6 +9521,9 @@ func (m *CheckTaskResponse) Unmarshal(dAtA []byte) error { if skippy < 0 { return ErrInvalidLengthDmmaster } + if (iNdEx + skippy) < 0 { + return ErrInvalidLengthDmmaster + } if (iNdEx + skippy) > l { return io.ErrUnexpectedEOF } @@ -9506,7 +9551,7 @@ func (m *RegisterWorkerRequest) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - wire |= (uint64(b) & 0x7F) << shift + wire |= uint64(b&0x7F) << shift if b < 0x80 { break } @@ -9534,7 +9579,7 @@ func (m *RegisterWorkerRequest) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= (uint64(b) & 0x7F) << shift + stringLen |= uint64(b&0x7F) << shift if b < 0x80 { break } @@ -9544,6 +9589,9 @@ func (m *RegisterWorkerRequest) Unmarshal(dAtA []byte) error { return ErrInvalidLengthDmmaster } postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthDmmaster + } if postIndex > l { return io.ErrUnexpectedEOF } @@ -9563,7 +9611,7 @@ func (m *RegisterWorkerRequest) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= (uint64(b) & 0x7F) << shift + stringLen |= uint64(b&0x7F) << shift if b < 0x80 { break } @@ -9573,6 +9621,9 @@ func (m *RegisterWorkerRequest) Unmarshal(dAtA []byte) error { return ErrInvalidLengthDmmaster } postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthDmmaster + } if postIndex > l { return io.ErrUnexpectedEOF } @@ -9587,6 +9638,9 @@ func (m *RegisterWorkerRequest) Unmarshal(dAtA []byte) error { if skippy < 0 { return ErrInvalidLengthDmmaster } + if (iNdEx + skippy) < 0 { + return ErrInvalidLengthDmmaster + } if (iNdEx + skippy) > l { return io.ErrUnexpectedEOF } @@ -9614,7 +9668,7 @@ func (m *RegisterWorkerResponse) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - wire |= (uint64(b) & 0x7F) << shift + wire |= uint64(b&0x7F) << shift if b < 0x80 { break } @@ -9642,7 +9696,7 @@ func (m *RegisterWorkerResponse) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - v |= (int(b) & 0x7F) << shift + v |= int(b&0x7F) << shift if b < 0x80 { break } @@ -9662,7 +9716,7 @@ func (m *RegisterWorkerResponse) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= (uint64(b) & 0x7F) << shift + stringLen |= uint64(b&0x7F) << shift if b < 0x80 { break } @@ -9672,6 +9726,9 @@ func (m *RegisterWorkerResponse) Unmarshal(dAtA []byte) error { return ErrInvalidLengthDmmaster } postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthDmmaster + } if postIndex > l { return io.ErrUnexpectedEOF } @@ -9686,6 +9743,9 @@ func (m *RegisterWorkerResponse) Unmarshal(dAtA []byte) error { if skippy < 0 { return ErrInvalidLengthDmmaster } + if (iNdEx + skippy) < 0 { + return ErrInvalidLengthDmmaster + } if (iNdEx + skippy) > l { return io.ErrUnexpectedEOF } @@ -9701,6 +9761,7 @@ func (m *RegisterWorkerResponse) Unmarshal(dAtA []byte) error { func skipDmmaster(dAtA []byte) (n int, err error) { l := len(dAtA) iNdEx := 0 + depth := 0 for iNdEx < l { var wire uint64 for shift := uint(0); ; shift += 7 { @@ -9732,10 +9793,8 @@ func skipDmmaster(dAtA []byte) (n int, err error) { break } } - return iNdEx, nil case 1: iNdEx += 8 - return iNdEx, nil case 2: var length int for shift := uint(0); ; shift += 7 { @@ -9752,148 +9811,34 @@ func skipDmmaster(dAtA []byte) (n int, err error) { break } } - iNdEx += length if length < 0 { return 0, ErrInvalidLengthDmmaster } - return iNdEx, nil + iNdEx += length case 3: - for { - var innerWire uint64 - var start int = iNdEx - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return 0, ErrIntOverflowDmmaster - } - if iNdEx >= l { - return 0, io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - innerWire |= (uint64(b) & 0x7F) << shift - if b < 0x80 { - break - } - } - innerWireType := int(innerWire & 0x7) - if innerWireType == 4 { - break - } - next, err := skipDmmaster(dAtA[start:]) - if err != nil { - return 0, err - } - iNdEx = start + next - } - return iNdEx, nil + depth++ case 4: - return iNdEx, nil + if depth == 0 { + return 0, ErrUnexpectedEndOfGroupDmmaster + } + depth-- case 5: iNdEx += 4 - return iNdEx, nil default: return 0, fmt.Errorf("proto: illegal wireType %d", wireType) } + if iNdEx < 0 { + return 0, ErrInvalidLengthDmmaster + } + if depth == 0 { + return iNdEx, nil + } } - panic("unreachable") + return 0, io.ErrUnexpectedEOF } var ( - ErrInvalidLengthDmmaster = fmt.Errorf("proto: negative length found during unmarshaling") - ErrIntOverflowDmmaster = fmt.Errorf("proto: integer overflow") + ErrInvalidLengthDmmaster = fmt.Errorf("proto: negative length found during unmarshaling") + ErrIntOverflowDmmaster = fmt.Errorf("proto: integer overflow") + ErrUnexpectedEndOfGroupDmmaster = fmt.Errorf("proto: unexpected end of group") ) - -func init() { proto.RegisterFile("dmmaster.proto", fileDescriptor_dmmaster_632e41001d9f6d87) } - -var fileDescriptor_dmmaster_632e41001d9f6d87 = []byte{ - // 1411 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xbc, 0x58, 0x4d, 0x6f, 0xdb, 0x46, - 0x13, 0x16, 0x25, 0x5b, 0x89, 0x46, 0x89, 0x61, 0x6f, 0x6c, 0x89, 0xa2, 0x1d, 0xc6, 0x2f, 0xdf, - 0x22, 0x30, 0x7a, 0x88, 0x1b, 0xa5, 0xa7, 0x00, 0x01, 0x1a, 0x5b, 0x09, 0x6a, 0x40, 0xa9, 0x1d, - 0xaa, 0x41, 0x91, 0x4b, 0x01, 0x4a, 0x5a, 0xcb, 0x84, 0x24, 0x92, 0xe6, 0x52, 0x76, 0xdc, 0xa2, - 0x28, 0xd0, 0x43, 0x2f, 0xbd, 0xb4, 0xe8, 0x21, 0xe7, 0xfe, 0x9b, 0x1c, 0x03, 0x14, 0x05, 0x7a, - 0x2c, 0x92, 0xfe, 0x90, 0x62, 0x3f, 0x48, 0x2e, 0xbf, 0x94, 0xd8, 0x07, 0xdd, 0xb8, 0x3b, 0xe4, - 0x33, 0xcf, 0xcc, 0x0e, 0x67, 0x1e, 0x12, 0x56, 0x86, 0xd3, 0xa9, 0x45, 0x02, 0xec, 0xdf, 0xf3, - 0x7c, 0x37, 0x70, 0x51, 0xd9, 0xeb, 0x6b, 0x2b, 0xc3, 0xe9, 0xb9, 0xeb, 0x8f, 0xc3, 0x3d, 0x6d, - 0x6b, 0xe4, 0xba, 0xa3, 0x09, 0xde, 0xb5, 0x3c, 0x7b, 0xd7, 0x72, 0x1c, 0x37, 0xb0, 0x02, 0xdb, - 0x75, 0x08, 0xb7, 0x1a, 0xa7, 0xd0, 0x7a, 0x66, 0x8f, 0x7c, 0x2b, 0xc0, 0xdf, 0xb0, 0x87, 0x4c, - 0x3c, 0xb1, 0x2e, 0x4c, 0x7c, 0x3a, 0xc3, 0x24, 0x40, 0x3a, 0xc0, 0x9e, 0xed, 0x4c, 0xdc, 0xd1, - 0x57, 0xd6, 0x14, 0xab, 0xca, 0xb6, 0xb2, 0x53, 0x33, 0xa5, 0x1d, 0xb4, 0x05, 0x35, 0xbe, 0x3a, - 0x72, 0x89, 0x5a, 0xde, 0x56, 0x76, 0x6e, 0x9a, 0xf1, 0x06, 0x6a, 0x40, 0x95, 0x13, 0x51, 0x2b, - 0xec, 0x49, 0xb1, 0x32, 0x8e, 0x40, 0x7f, 0xe1, 0x0d, 0x93, 0x1e, 0xf7, 0x5d, 0xe7, 0xd8, 0x1e, - 0x85, 0x7e, 0x1b, 0x50, 0x1d, 0xb0, 0x0d, 0xe1, 0x53, 0xac, 0x24, 0xc4, 0x72, 0x02, 0xf1, 0x0b, - 0x58, 0xed, 0x05, 0x96, 0x1f, 0x7c, 0x6d, 0x91, 0x71, 0x88, 0x81, 0x60, 0x29, 0xb0, 0xc8, 0x58, - 0x20, 0xb0, 0x6b, 0xa4, 0xc2, 0x35, 0xfe, 0x04, 0x65, 0x5b, 0xd9, 0xa9, 0x99, 0xe1, 0xd2, 0x38, - 0x85, 0x35, 0x09, 0x81, 0x78, 0xae, 0x43, 0x30, 0x75, 0xe7, 0x63, 0x32, 0x9b, 0x04, 0x0c, 0xe4, - 0xba, 0x29, 0x56, 0x68, 0x15, 0x2a, 0x53, 0x32, 0x12, 0x1c, 0xe8, 0x25, 0x6a, 0xc7, 0xc0, 0x95, - 0xed, 0xca, 0x4e, 0xbd, 0xad, 0xde, 0xf3, 0xfa, 0xf7, 0xf6, 0xdd, 0xe9, 0xd4, 0x75, 0xc2, 0x28, - 0x39, 0x68, 0xec, 0xf2, 0x01, 0xb4, 0x78, 0x1a, 0x9e, 0xb1, 0x13, 0xfc, 0xa8, 0x0c, 0x18, 0x17, - 0xa0, 0xe5, 0x3d, 0x74, 0x69, 0xc2, 0xf7, 0xd3, 0x84, 0x9b, 0x94, 0xf0, 0xf3, 0x19, 0xf6, 0x2f, - 0x7a, 0x81, 0x15, 0xcc, 0x48, 0x96, 0xef, 0xb7, 0x80, 0x0e, 0x3d, 0x4c, 0x2b, 0x45, 0x4e, 0xb3, - 0x06, 0x65, 0xd7, 0x63, 0xee, 0x56, 0xda, 0x40, 0x31, 0xa8, 0xf1, 0xd0, 0x33, 0xcb, 0xae, 0x47, - 0x8f, 0xc0, 0xa1, 0x85, 0xc3, 0xfd, 0xb2, 0x6b, 0xf9, 0x08, 0x2a, 0xc9, 0x23, 0xf8, 0x4d, 0x81, - 0x5b, 0x09, 0x07, 0x22, 0xa8, 0x79, 0x1e, 0xe2, 0x80, 0xcb, 0x79, 0x01, 0x57, 0xe2, 0x80, 0x3f, - 0x8f, 0xfd, 0x2e, 0xb1, 0x80, 0x35, 0x0a, 0x25, 0xfc, 0xf5, 0x66, 0x7d, 0xd9, 0x65, 0xcc, 0xe9, - 0x31, 0xac, 0xf1, 0x74, 0x5f, 0xbd, 0xb2, 0x7c, 0x40, 0x32, 0xc4, 0x42, 0x4a, 0xeb, 0x29, 0x34, - 0xa4, 0xa3, 0xec, 0xda, 0x24, 0x90, 0xb8, 0x3b, 0xf1, 0xbb, 0x9c, 0x39, 0x92, 0x14, 0xf7, 0x33, - 0x68, 0x66, 0x70, 0x16, 0x51, 0x6a, 0x4f, 0x60, 0x83, 0xd9, 0x9f, 0xf8, 0xbe, 0xeb, 0x5f, 0x9d, - 0x7e, 0x20, 0xd2, 0x20, 0xc1, 0x5c, 0x9a, 0xfd, 0x67, 0x69, 0xf6, 0x8d, 0x88, 0x3d, 0x83, 0xcd, - 0x92, 0xdf, 0x87, 0x5b, 0xbd, 0x13, 0xf7, 0xbc, 0xd3, 0xe9, 0x76, 0xdd, 0xc1, 0x98, 0x5c, 0xad, - 0x6a, 0x7e, 0x51, 0xe0, 0x9a, 0x40, 0x40, 0x2b, 0x50, 0x3e, 0xe8, 0x88, 0xe7, 0xca, 0x07, 0x9d, - 0x08, 0xa9, 0x2c, 0x21, 0xad, 0xc3, 0xb2, 0x7b, 0xee, 0x44, 0xad, 0x96, 0x2f, 0xe8, 0x9d, 0x9d, - 0x4e, 0x97, 0x57, 0x7c, 0xcd, 0x64, 0xd7, 0x34, 0x74, 0x72, 0xe1, 0x0c, 0xf0, 0x50, 0x5d, 0x66, - 0xbb, 0x62, 0x85, 0x34, 0xb8, 0x3e, 0x73, 0x84, 0xa5, 0xca, 0x2c, 0xd1, 0xda, 0x18, 0xc0, 0x7a, - 0x32, 0xa4, 0x4b, 0xa7, 0xf1, 0x7f, 0xb0, 0x3c, 0xa1, 0x8f, 0x8a, 0x24, 0xd6, 0x69, 0x12, 0x05, - 0x9c, 0xc9, 0x2d, 0xc6, 0xcf, 0x0a, 0xac, 0xbf, 0x70, 0xe8, 0x75, 0x68, 0x10, 0x99, 0x4b, 0xc7, - 0x6f, 0xc0, 0x0d, 0x1f, 0x7b, 0x13, 0x6b, 0x80, 0x0f, 0x59, 0xc8, 0xdc, 0x4d, 0x62, 0xaf, 0xb8, - 0xcd, 0xa0, 0x6d, 0xa8, 0x1f, 0xbb, 0xfe, 0x00, 0x9b, 0x78, 0xea, 0x9e, 0x61, 0x75, 0x89, 0x11, - 0x97, 0xb7, 0x8c, 0x19, 0x6c, 0xa4, 0x78, 0x2c, 0xe4, 0xa5, 0xfd, 0x43, 0x81, 0xd6, 0x9e, 0x8f, - 0xad, 0x31, 0xbf, 0x21, 0x95, 0x04, 0x29, 0x20, 0x25, 0x19, 0x50, 0x5e, 0x39, 0xb0, 0x14, 0xd1, - 0x60, 0x28, 0xc4, 0x41, 0x47, 0x54, 0x45, 0x62, 0x8f, 0x22, 0xe2, 0x57, 0x78, 0xd0, 0xe9, 0x74, - 0x45, 0x12, 0xc2, 0x25, 0xb5, 0x90, 0xb1, 0xed, 0x51, 0xcb, 0x32, 0xb7, 0x88, 0xa5, 0xf1, 0x1d, - 0x68, 0x79, 0x14, 0x17, 0x92, 0x9f, 0x87, 0xa0, 0xf7, 0xce, 0xed, 0x60, 0x70, 0x22, 0xc9, 0x06, - 0x3e, 0x05, 0x3f, 0x98, 0x23, 0xe3, 0x47, 0xb8, 0x53, 0xf8, 0xec, 0x42, 0xc8, 0x9b, 0xd0, 0x12, - 0xb3, 0x26, 0x47, 0x66, 0x6d, 0x4a, 0x13, 0x8e, 0xbd, 0x19, 0xcc, 0x2a, 0x46, 0x5c, 0x71, 0x8f, - 0x78, 0xad, 0x80, 0x96, 0x07, 0x2a, 0x02, 0x9a, 0x8b, 0xfa, 0xf1, 0x83, 0xb3, 0x9d, 0x1e, 0x9c, - 0xaa, 0x34, 0x38, 0x13, 0x1e, 0x63, 0x66, 0x9b, 0xd0, 0x32, 0xf1, 0xb1, 0x8f, 0x89, 0xc8, 0x37, - 0x1d, 0x7d, 0x61, 0x23, 0x34, 0x1e, 0xc3, 0x46, 0xd6, 0xf8, 0x8c, 0xc8, 0xea, 0x4e, 0x91, 0xd5, - 0x5d, 0xf6, 0x04, 0x0c, 0x1b, 0xb4, 0x3c, 0xfc, 0x0f, 0x9c, 0xe4, 0x83, 0x64, 0x26, 0xeb, 0xed, - 0x16, 0xcf, 0x4a, 0x0e, 0x97, 0x38, 0x94, 0x37, 0x0a, 0xac, 0x7d, 0x69, 0x39, 0xc3, 0x09, 0xee, - 0x3d, 0xef, 0x92, 0x79, 0x73, 0xa8, 0xc5, 0xf2, 0x5d, 0x66, 0xf9, 0xae, 0x51, 0xe4, 0xde, 0xf3, - 0x6e, 0x2c, 0x84, 0x2c, 0x7f, 0x14, 0xb6, 0x22, 0x76, 0x4d, 0xb5, 0x73, 0x3f, 0xd2, 0xce, 0x4b, - 0x0c, 0x27, 0xde, 0x90, 0x72, 0xb1, 0x9c, 0xc8, 0x85, 0x0e, 0x40, 0x4e, 0x27, 0x47, 0x56, 0x10, - 0x60, 0xdf, 0x51, 0xab, 0x5c, 0x91, 0xc7, 0x3b, 0xb4, 0x8b, 0x93, 0x13, 0xcb, 0x1f, 0xda, 0xce, - 0x48, 0xbd, 0xc6, 0xa2, 0x8f, 0xd6, 0x54, 0x89, 0xc8, 0x91, 0x2c, 0xa4, 0xee, 0x5f, 0x2b, 0xd0, - 0x3c, 0x9a, 0xf9, 0xa3, 0xbc, 0xb2, 0x2f, 0x6e, 0x69, 0x1a, 0x5c, 0xb7, 0x1d, 0x6b, 0x10, 0xd8, - 0x67, 0x58, 0xd4, 0x67, 0xb4, 0x66, 0xed, 0xce, 0x9e, 0x62, 0x56, 0xa2, 0x15, 0x93, 0x5d, 0xd3, - 0xfb, 0x8f, 0xed, 0x09, 0x66, 0x47, 0xc2, 0x53, 0x19, 0xad, 0xd9, 0xbc, 0x9b, 0xf5, 0x3b, 0x76, - 0x94, 0x49, 0xbe, 0x32, 0x5e, 0x81, 0x9a, 0x25, 0xb6, 0x90, 0x9c, 0xdc, 0x85, 0xd5, 0xfd, 0x13, - 0x3c, 0x18, 0x7f, 0x40, 0x53, 0x1a, 0x8f, 0x60, 0x4d, 0xba, 0xef, 0xb2, 0xd4, 0xa8, 0x88, 0x32, - 0xf1, 0xc8, 0xa6, 0x4d, 0x2e, 0x64, 0x32, 0x57, 0x44, 0x59, 0xc3, 0xa1, 0x8f, 0x09, 0x11, 0x10, - 0xe1, 0xd2, 0xd8, 0x83, 0x46, 0x1a, 0xe6, 0xb2, 0x54, 0xda, 0x7f, 0xd5, 0xa1, 0xca, 0xdb, 0x2d, - 0x7a, 0x09, 0xb5, 0xe8, 0x43, 0x0b, 0xad, 0xb3, 0xd7, 0x24, 0xf5, 0xe5, 0xa6, 0x6d, 0xa4, 0x76, - 0xb9, 0x3b, 0xe3, 0xce, 0x4f, 0x7f, 0xfe, 0xfb, 0x7b, 0xb9, 0x65, 0xac, 0xd3, 0x2f, 0x59, 0xb2, - 0x7b, 0x76, 0xdf, 0x9a, 0x78, 0x27, 0xd6, 0xfd, 0x5d, 0x9a, 0x2b, 0xf2, 0x50, 0xf9, 0x14, 0x1d, - 0x43, 0x5d, 0xfa, 0x7e, 0x40, 0x0d, 0xa9, 0x4f, 0xc9, 0xf0, 0xcd, 0xcc, 0xbe, 0x70, 0x70, 0x97, - 0x39, 0xd8, 0xd6, 0x36, 0xf3, 0x1c, 0xec, 0x7e, 0x4f, 0xf3, 0xf4, 0x03, 0xf5, 0xf3, 0x08, 0x20, - 0x56, 0xf4, 0x88, 0xb1, 0xcd, 0x7c, 0x24, 0x68, 0x8d, 0xf4, 0xb6, 0x70, 0x52, 0x42, 0x13, 0xa8, - 0x4b, 0xe2, 0x17, 0x69, 0x29, 0x35, 0x2c, 0xc9, 0x5d, 0x6d, 0x33, 0xd7, 0x26, 0x90, 0x3e, 0x61, - 0x74, 0x75, 0xb4, 0x95, 0xa2, 0x4b, 0xd8, 0xad, 0x82, 0x2f, 0x7a, 0x02, 0x10, 0x8b, 0x55, 0xd4, - 0x4a, 0x8a, 0x57, 0xd9, 0x97, 0x96, 0x67, 0x8a, 0x48, 0xef, 0xc3, 0x0d, 0x59, 0x01, 0x22, 0x96, - 0xc4, 0x1c, 0x99, 0xab, 0xa9, 0x59, 0x43, 0x04, 0xf2, 0x14, 0x6e, 0x26, 0x84, 0x15, 0x62, 0x37, - 0xe7, 0x69, 0x3e, 0xad, 0x95, 0x63, 0x89, 0x70, 0x5e, 0x84, 0x9f, 0x54, 0xf2, 0x47, 0x30, 0xba, - 0x1d, 0x67, 0x3c, 0xe7, 0x8b, 0x5a, 0xd3, 0x8b, 0xcc, 0x11, 0xec, 0x4b, 0x68, 0x16, 0xfc, 0x97, - 0x40, 0x46, 0xfc, 0x70, 0xd1, 0x4f, 0x0b, 0xad, 0xf0, 0xcd, 0xe7, 0x8c, 0xb3, 0xba, 0x89, 0x33, - 0x2e, 0x94, 0x7c, 0x9c, 0x71, 0xb1, 0xdc, 0x32, 0x4a, 0xb4, 0x12, 0xe3, 0x8e, 0xce, 0x2b, 0x31, - 0x33, 0xab, 0x78, 0x25, 0x66, 0x1b, 0xbf, 0x51, 0x42, 0x43, 0x68, 0x16, 0xa8, 0x22, 0x1e, 0xf0, - 0x7c, 0xb9, 0xa5, 0xfd, 0x7f, 0xee, 0x3d, 0x52, 0x5a, 0x1b, 0x59, 0x95, 0xc2, 0x5e, 0x9d, 0xdb, - 0xd2, 0x9b, 0x98, 0x9d, 0x0f, 0x3c, 0xfe, 0x62, 0x81, 0x63, 0x94, 0xd0, 0x21, 0xac, 0xa6, 0x7b, - 0x38, 0x62, 0xef, 0x4c, 0xc1, 0xc8, 0xd1, 0xb6, 0xf2, 0x8d, 0xf2, 0x39, 0x65, 0xf5, 0x00, 0xe7, - 0x59, 0x28, 0x68, 0x38, 0xcf, 0x62, 0x3d, 0xc2, 0x78, 0xa2, 0xec, 0x4f, 0x36, 0x0e, 0x5b, 0xf8, - 0xf3, 0x6d, 0x6e, 0x3d, 0x3d, 0x84, 0x5a, 0x34, 0x1a, 0x78, 0x17, 0x4d, 0x4f, 0x14, 0xde, 0x45, - 0x33, 0xf3, 0xc3, 0x28, 0xa1, 0x03, 0x58, 0x49, 0x36, 0x74, 0x24, 0x74, 0x50, 0xce, 0xac, 0xe0, - 0x5d, 0x21, 0xbf, 0xff, 0x1b, 0xa5, 0x3d, 0xf5, 0xcd, 0x3b, 0x5d, 0x79, 0xfb, 0x4e, 0x57, 0xfe, - 0x79, 0xa7, 0x2b, 0xbf, 0xbe, 0xd7, 0x4b, 0x6f, 0xdf, 0xeb, 0xa5, 0xbf, 0xdf, 0xeb, 0xa5, 0x7e, - 0x95, 0xfd, 0x5d, 0x7c, 0xf0, 0x5f, 0x00, 0x00, 0x00, 0xff, 0xff, 0x33, 0x19, 0x56, 0x3e, 0xa1, - 0x14, 0x00, 0x00, -} diff --git a/dm/pb/dmmaster.pb.gw.go b/dm/pb/dmmaster.pb.gw.go index 953fb4ef8a..b3c4e66074 100644 --- a/dm/pb/dmmaster.pb.gw.go +++ b/dm/pb/dmmaster.pb.gw.go @@ -13,6 +13,7 @@ import ( "io" "net/http" + "github.com/golang/protobuf/descriptor" "github.com/golang/protobuf/proto" "github.com/grpc-ecosystem/grpc-gateway/runtime" "github.com/grpc-ecosystem/grpc-gateway/utilities" @@ -22,11 +23,13 @@ import ( "google.golang.org/grpc/status" ) +// Suppress "imported and not used" errors var _ codes.Code var _ io.Reader var _ status.Status var _ = runtime.String var _ = utilities.NewDoubleArray +var _ = descriptor.ForMessage func request_Master_StartTask_0(ctx context.Context, marshaler runtime.Marshaler, client MasterClient, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { var protoReq StartTaskRequest @@ -45,6 +48,23 @@ func request_Master_StartTask_0(ctx context.Context, marshaler runtime.Marshaler } +func local_request_Master_StartTask_0(ctx context.Context, marshaler runtime.Marshaler, server MasterServer, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq StartTaskRequest + var metadata runtime.ServerMetadata + + newReader, berr := utilities.IOReaderFactory(req.Body) + if berr != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", berr) + } + if err := marshaler.NewDecoder(newReader()).Decode(&protoReq); err != nil && err != io.EOF { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + + msg, err := server.StartTask(ctx, &protoReq) + return msg, metadata, err + +} + func request_Master_OperateTask_0(ctx context.Context, marshaler runtime.Marshaler, client MasterClient, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { var protoReq OperateTaskRequest var metadata runtime.ServerMetadata @@ -80,6 +100,41 @@ func request_Master_OperateTask_0(ctx context.Context, marshaler runtime.Marshal } +func local_request_Master_OperateTask_0(ctx context.Context, marshaler runtime.Marshaler, server MasterServer, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq OperateTaskRequest + var metadata runtime.ServerMetadata + + newReader, berr := utilities.IOReaderFactory(req.Body) + if berr != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", berr) + } + if err := marshaler.NewDecoder(newReader()).Decode(&protoReq); err != nil && err != io.EOF { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + + var ( + val string + ok bool + err error + _ = err + ) + + val, ok = pathParams["name"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "name") + } + + protoReq.Name, err = runtime.String(val) + + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "name", err) + } + + msg, err := server.OperateTask(ctx, &protoReq) + return msg, metadata, err + +} + var ( filter_Master_QueryStatus_0 = &utilities.DoubleArray{Encoding: map[string]int{"name": 0}, Base: []int{1, 1, 0}, Check: []int{0, 1, 2}} ) @@ -118,6 +173,105 @@ func request_Master_QueryStatus_0(ctx context.Context, marshaler runtime.Marshal } +func local_request_Master_QueryStatus_0(ctx context.Context, marshaler runtime.Marshaler, server MasterServer, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq QueryStatusListRequest + var metadata runtime.ServerMetadata + + var ( + val string + ok bool + err error + _ = err + ) + + val, ok = pathParams["name"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "name") + } + + protoReq.Name, err = runtime.String(val) + + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "name", err) + } + + if err := runtime.PopulateQueryParameters(&protoReq, req.URL.Query(), filter_Master_QueryStatus_0); err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + + msg, err := server.QueryStatus(ctx, &protoReq) + return msg, metadata, err + +} + +// RegisterMasterHandlerServer registers the http handlers for service Master to "mux". +// UnaryRPC :call MasterServer directly. +// StreamingRPC :currently unsupported pending https://github.com/grpc/grpc-go/issues/906. +func RegisterMasterHandlerServer(ctx context.Context, mux *runtime.ServeMux, server MasterServer) error { + + mux.Handle("POST", pattern_Master_StartTask_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + rctx, err := runtime.AnnotateIncomingContext(ctx, mux, req) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := local_request_Master_StartTask_0(rctx, inboundMarshaler, server, req, pathParams) + ctx = runtime.NewServerMetadataContext(ctx, md) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + + forward_Master_StartTask_0(ctx, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("PUT", pattern_Master_OperateTask_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + rctx, err := runtime.AnnotateIncomingContext(ctx, mux, req) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := local_request_Master_OperateTask_0(rctx, inboundMarshaler, server, req, pathParams) + ctx = runtime.NewServerMetadataContext(ctx, md) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + + forward_Master_OperateTask_0(ctx, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("GET", pattern_Master_QueryStatus_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + rctx, err := runtime.AnnotateIncomingContext(ctx, mux, req) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := local_request_Master_QueryStatus_0(rctx, inboundMarshaler, server, req, pathParams) + ctx = runtime.NewServerMetadataContext(ctx, md) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + + forward_Master_QueryStatus_0(ctx, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + return nil +} + // RegisterMasterHandlerFromEndpoint is same as RegisterMasterHandler but // automatically dials to "endpoint" and closes the connection when "ctx" gets done. func RegisterMasterHandlerFromEndpoint(ctx context.Context, mux *runtime.ServeMux, endpoint string, opts []grpc.DialOption) (err error) { diff --git a/dm/pb/dmworker.pb.go b/dm/pb/dmworker.pb.go index d7e0425e0f..e385dc6429 100644 --- a/dm/pb/dmworker.pb.go +++ b/dm/pb/dmworker.pb.go @@ -4,17 +4,16 @@ package pb import ( - "fmt" - - proto "github.com/gogo/protobuf/proto" - + context "context" + fmt "fmt" + io "io" math "math" + math_bits "math/bits" - context "golang.org/x/net/context" - + proto "github.com/gogo/protobuf/proto" grpc "google.golang.org/grpc" - - io "io" + codes "google.golang.org/grpc/codes" + status "google.golang.org/grpc/status" ) // Reference imports to suppress errors if they are not otherwise used. @@ -26,7 +25,7 @@ var _ = math.Inf // is compatible with the proto package it is being compiled against. // A compilation error at this line likely means your copy of the // proto package needs to be updated. -const _ = proto.GoGoProtoPackageIsVersion2 // please upgrade the proto package +const _ = proto.GoGoProtoPackageIsVersion3 // please upgrade the proto package type TaskOp int32 @@ -49,6 +48,7 @@ var TaskOp_name = map[int32]string{ 5: "Update", 6: "AutoResume", } + var TaskOp_value = map[string]int32{ "InvalidOp": 0, "Stop": 1, @@ -62,8 +62,9 @@ var TaskOp_value = map[string]int32{ func (x TaskOp) String() string { return proto.EnumName(TaskOp_name, int32(x)) } + func (TaskOp) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_dmworker_84482ac67274de9a, []int{0} + return fileDescriptor_51a1b9e17fd67b10, []int{0} } type SQLOp int32 @@ -79,6 +80,7 @@ var SQLOp_name = map[int32]string{ 1: "REPLACE", 2: "INJECT", } + var SQLOp_value = map[string]int32{ "SKIP": 0, "REPLACE": 1, @@ -88,8 +90,9 @@ var SQLOp_value = map[string]int32{ func (x SQLOp) String() string { return proto.EnumName(SQLOp_name, int32(x)) } + func (SQLOp) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_dmworker_84482ac67274de9a, []int{1} + return fileDescriptor_51a1b9e17fd67b10, []int{1} } // Stage represents current stage for a (sub) task @@ -133,6 +136,7 @@ var Stage_name = map[int32]string{ 4: "Stopped", 5: "Finished", } + var Stage_value = map[string]int32{ "InvalidStage": 0, "New": 1, @@ -145,8 +149,9 @@ var Stage_value = map[string]int32{ func (x Stage) String() string { return proto.EnumName(Stage_name, int32(x)) } + func (Stage) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_dmworker_84482ac67274de9a, []int{2} + return fileDescriptor_51a1b9e17fd67b10, []int{2} } // UnitType represents the dm unit's type @@ -169,6 +174,7 @@ var UnitType_name = map[int32]string{ 4: "Sync", 100: "Relay", } + var UnitType_value = map[string]int32{ "InvalidUnit": 0, "Check": 1, @@ -181,8 +187,9 @@ var UnitType_value = map[string]int32{ func (x UnitType) String() string { return proto.EnumName(UnitType_name, int32(x)) } + func (UnitType) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_dmworker_84482ac67274de9a, []int{3} + return fileDescriptor_51a1b9e17fd67b10, []int{3} } // ErrorType represents type of error produced by a dm unit @@ -200,6 +207,7 @@ var ErrorType_name = map[int32]string{ 1: "ExecSQL", 2: "CheckFailed", } + var ErrorType_value = map[string]int32{ "UnknownError": 0, "ExecSQL": 1, @@ -209,8 +217,9 @@ var ErrorType_value = map[string]int32{ func (x ErrorType) String() string { return proto.EnumName(ErrorType_name, int32(x)) } + func (ErrorType) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_dmworker_84482ac67274de9a, []int{4} + return fileDescriptor_51a1b9e17fd67b10, []int{4} } // RelayOp differs from TaskOp @@ -229,6 +238,7 @@ var RelayOp_name = map[int32]string{ 2: "PauseRelay", 3: "ResumeRelay", } + var RelayOp_value = map[string]int32{ "InvalidRelayOp": 0, "StopRelay": 1, @@ -239,8 +249,37 @@ var RelayOp_value = map[string]int32{ func (x RelayOp) String() string { return proto.EnumName(RelayOp_name, int32(x)) } + func (RelayOp) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_dmworker_84482ac67274de9a, []int{5} + return fileDescriptor_51a1b9e17fd67b10, []int{5} +} + +type WorkerOp int32 + +const ( + WorkerOp_StartWorker WorkerOp = 0 + WorkerOp_UpdateConfig WorkerOp = 1 + WorkerOp_StopWorker WorkerOp = 2 +) + +var WorkerOp_name = map[int32]string{ + 0: "StartWorker", + 1: "UpdateConfig", + 2: "StopWorker", +} + +var WorkerOp_value = map[string]int32{ + "StartWorker": 0, + "UpdateConfig": 1, + "StopWorker": 2, +} + +func (x WorkerOp) String() string { + return proto.EnumName(WorkerOp_name, int32(x)) +} + +func (WorkerOp) EnumDescriptor() ([]byte, []int) { + return fileDescriptor_51a1b9e17fd67b10, []int{6} } type StartSubTaskRequest struct { @@ -251,7 +290,7 @@ func (m *StartSubTaskRequest) Reset() { *m = StartSubTaskRequest{} } func (m *StartSubTaskRequest) String() string { return proto.CompactTextString(m) } func (*StartSubTaskRequest) ProtoMessage() {} func (*StartSubTaskRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_dmworker_84482ac67274de9a, []int{0} + return fileDescriptor_51a1b9e17fd67b10, []int{0} } func (m *StartSubTaskRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -261,15 +300,15 @@ func (m *StartSubTaskRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, return xxx_messageInfo_StartSubTaskRequest.Marshal(b, m, deterministic) } else { b = b[:cap(b)] - n, err := m.MarshalTo(b) + n, err := m.MarshalToSizedBuffer(b) if err != nil { return nil, err } return b[:n], nil } } -func (dst *StartSubTaskRequest) XXX_Merge(src proto.Message) { - xxx_messageInfo_StartSubTaskRequest.Merge(dst, src) +func (m *StartSubTaskRequest) XXX_Merge(src proto.Message) { + xxx_messageInfo_StartSubTaskRequest.Merge(m, src) } func (m *StartSubTaskRequest) XXX_Size() int { return m.Size() @@ -295,7 +334,7 @@ func (m *UpdateRelayRequest) Reset() { *m = UpdateRelayRequest{} } func (m *UpdateRelayRequest) String() string { return proto.CompactTextString(m) } func (*UpdateRelayRequest) ProtoMessage() {} func (*UpdateRelayRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_dmworker_84482ac67274de9a, []int{1} + return fileDescriptor_51a1b9e17fd67b10, []int{1} } func (m *UpdateRelayRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -305,15 +344,15 @@ func (m *UpdateRelayRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, return xxx_messageInfo_UpdateRelayRequest.Marshal(b, m, deterministic) } else { b = b[:cap(b)] - n, err := m.MarshalTo(b) + n, err := m.MarshalToSizedBuffer(b) if err != nil { return nil, err } return b[:n], nil } } -func (dst *UpdateRelayRequest) XXX_Merge(src proto.Message) { - xxx_messageInfo_UpdateRelayRequest.Merge(dst, src) +func (m *UpdateRelayRequest) XXX_Merge(src proto.Message) { + xxx_messageInfo_UpdateRelayRequest.Merge(m, src) } func (m *UpdateRelayRequest) XXX_Size() int { return m.Size() @@ -340,7 +379,7 @@ func (m *MigrateRelayRequest) Reset() { *m = MigrateRelayRequest{} } func (m *MigrateRelayRequest) String() string { return proto.CompactTextString(m) } func (*MigrateRelayRequest) ProtoMessage() {} func (*MigrateRelayRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_dmworker_84482ac67274de9a, []int{2} + return fileDescriptor_51a1b9e17fd67b10, []int{2} } func (m *MigrateRelayRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -350,15 +389,15 @@ func (m *MigrateRelayRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, return xxx_messageInfo_MigrateRelayRequest.Marshal(b, m, deterministic) } else { b = b[:cap(b)] - n, err := m.MarshalTo(b) + n, err := m.MarshalToSizedBuffer(b) if err != nil { return nil, err } return b[:n], nil } } -func (dst *MigrateRelayRequest) XXX_Merge(src proto.Message) { - xxx_messageInfo_MigrateRelayRequest.Merge(dst, src) +func (m *MigrateRelayRequest) XXX_Merge(src proto.Message) { + xxx_messageInfo_MigrateRelayRequest.Merge(m, src) } func (m *MigrateRelayRequest) XXX_Size() int { return m.Size() @@ -392,7 +431,7 @@ func (m *OperateSubTaskRequest) Reset() { *m = OperateSubTaskRequest{} } func (m *OperateSubTaskRequest) String() string { return proto.CompactTextString(m) } func (*OperateSubTaskRequest) ProtoMessage() {} func (*OperateSubTaskRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_dmworker_84482ac67274de9a, []int{3} + return fileDescriptor_51a1b9e17fd67b10, []int{3} } func (m *OperateSubTaskRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -402,15 +441,15 @@ func (m *OperateSubTaskRequest) XXX_Marshal(b []byte, deterministic bool) ([]byt return xxx_messageInfo_OperateSubTaskRequest.Marshal(b, m, deterministic) } else { b = b[:cap(b)] - n, err := m.MarshalTo(b) + n, err := m.MarshalToSizedBuffer(b) if err != nil { return nil, err } return b[:n], nil } } -func (dst *OperateSubTaskRequest) XXX_Merge(src proto.Message) { - xxx_messageInfo_OperateSubTaskRequest.Merge(dst, src) +func (m *OperateSubTaskRequest) XXX_Merge(src proto.Message) { + xxx_messageInfo_OperateSubTaskRequest.Merge(m, src) } func (m *OperateSubTaskRequest) XXX_Size() int { return m.Size() @@ -446,7 +485,7 @@ func (m *OperateSubTaskResponse) Reset() { *m = OperateSubTaskResponse{} func (m *OperateSubTaskResponse) String() string { return proto.CompactTextString(m) } func (*OperateSubTaskResponse) ProtoMessage() {} func (*OperateSubTaskResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_dmworker_84482ac67274de9a, []int{4} + return fileDescriptor_51a1b9e17fd67b10, []int{4} } func (m *OperateSubTaskResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -456,15 +495,15 @@ func (m *OperateSubTaskResponse) XXX_Marshal(b []byte, deterministic bool) ([]by return xxx_messageInfo_OperateSubTaskResponse.Marshal(b, m, deterministic) } else { b = b[:cap(b)] - n, err := m.MarshalTo(b) + n, err := m.MarshalToSizedBuffer(b) if err != nil { return nil, err } return b[:n], nil } } -func (dst *OperateSubTaskResponse) XXX_Merge(src proto.Message) { - xxx_messageInfo_OperateSubTaskResponse.Merge(dst, src) +func (m *OperateSubTaskResponse) XXX_Merge(src proto.Message) { + xxx_messageInfo_OperateSubTaskResponse.Merge(m, src) } func (m *OperateSubTaskResponse) XXX_Size() int { return m.Size() @@ -511,7 +550,7 @@ func (m *UpdateSubTaskRequest) Reset() { *m = UpdateSubTaskRequest{} } func (m *UpdateSubTaskRequest) String() string { return proto.CompactTextString(m) } func (*UpdateSubTaskRequest) ProtoMessage() {} func (*UpdateSubTaskRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_dmworker_84482ac67274de9a, []int{5} + return fileDescriptor_51a1b9e17fd67b10, []int{5} } func (m *UpdateSubTaskRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -521,15 +560,15 @@ func (m *UpdateSubTaskRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte return xxx_messageInfo_UpdateSubTaskRequest.Marshal(b, m, deterministic) } else { b = b[:cap(b)] - n, err := m.MarshalTo(b) + n, err := m.MarshalToSizedBuffer(b) if err != nil { return nil, err } return b[:n], nil } } -func (dst *UpdateSubTaskRequest) XXX_Merge(src proto.Message) { - xxx_messageInfo_UpdateSubTaskRequest.Merge(dst, src) +func (m *UpdateSubTaskRequest) XXX_Merge(src proto.Message) { + xxx_messageInfo_UpdateSubTaskRequest.Merge(m, src) } func (m *UpdateSubTaskRequest) XXX_Size() int { return m.Size() @@ -555,7 +594,7 @@ func (m *QueryStatusRequest) Reset() { *m = QueryStatusRequest{} } func (m *QueryStatusRequest) String() string { return proto.CompactTextString(m) } func (*QueryStatusRequest) ProtoMessage() {} func (*QueryStatusRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_dmworker_84482ac67274de9a, []int{6} + return fileDescriptor_51a1b9e17fd67b10, []int{6} } func (m *QueryStatusRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -565,15 +604,15 @@ func (m *QueryStatusRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, return xxx_messageInfo_QueryStatusRequest.Marshal(b, m, deterministic) } else { b = b[:cap(b)] - n, err := m.MarshalTo(b) + n, err := m.MarshalToSizedBuffer(b) if err != nil { return nil, err } return b[:n], nil } } -func (dst *QueryStatusRequest) XXX_Merge(src proto.Message) { - xxx_messageInfo_QueryStatusRequest.Merge(dst, src) +func (m *QueryStatusRequest) XXX_Merge(src proto.Message) { + xxx_messageInfo_QueryStatusRequest.Merge(m, src) } func (m *QueryStatusRequest) XXX_Size() int { return m.Size() @@ -599,7 +638,7 @@ func (m *QueryErrorRequest) Reset() { *m = QueryErrorRequest{} } func (m *QueryErrorRequest) String() string { return proto.CompactTextString(m) } func (*QueryErrorRequest) ProtoMessage() {} func (*QueryErrorRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_dmworker_84482ac67274de9a, []int{7} + return fileDescriptor_51a1b9e17fd67b10, []int{7} } func (m *QueryErrorRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -609,15 +648,15 @@ func (m *QueryErrorRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, e return xxx_messageInfo_QueryErrorRequest.Marshal(b, m, deterministic) } else { b = b[:cap(b)] - n, err := m.MarshalTo(b) + n, err := m.MarshalToSizedBuffer(b) if err != nil { return nil, err } return b[:n], nil } } -func (dst *QueryErrorRequest) XXX_Merge(src proto.Message) { - xxx_messageInfo_QueryErrorRequest.Merge(dst, src) +func (m *QueryErrorRequest) XXX_Merge(src proto.Message) { + xxx_messageInfo_QueryErrorRequest.Merge(m, src) } func (m *QueryErrorRequest) XXX_Size() int { return m.Size() @@ -639,7 +678,7 @@ func (m *QueryErrorRequest) GetName() string { type HandleSubTaskSQLsRequest struct { Name string `protobuf:"bytes,1,opt,name=name,proto3" json:"name,omitempty"` Op SQLOp `protobuf:"varint,2,opt,name=op,proto3,enum=pb.SQLOp" json:"op,omitempty"` - Args []string `protobuf:"bytes,3,rep,name=args" json:"args,omitempty"` + Args []string `protobuf:"bytes,3,rep,name=args,proto3" json:"args,omitempty"` BinlogPos string `protobuf:"bytes,4,opt,name=binlogPos,proto3" json:"binlogPos,omitempty"` SqlPattern string `protobuf:"bytes,5,opt,name=sqlPattern,proto3" json:"sqlPattern,omitempty"` } @@ -648,7 +687,7 @@ func (m *HandleSubTaskSQLsRequest) Reset() { *m = HandleSubTaskSQLsReque func (m *HandleSubTaskSQLsRequest) String() string { return proto.CompactTextString(m) } func (*HandleSubTaskSQLsRequest) ProtoMessage() {} func (*HandleSubTaskSQLsRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_dmworker_84482ac67274de9a, []int{8} + return fileDescriptor_51a1b9e17fd67b10, []int{8} } func (m *HandleSubTaskSQLsRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -658,15 +697,15 @@ func (m *HandleSubTaskSQLsRequest) XXX_Marshal(b []byte, deterministic bool) ([] return xxx_messageInfo_HandleSubTaskSQLsRequest.Marshal(b, m, deterministic) } else { b = b[:cap(b)] - n, err := m.MarshalTo(b) + n, err := m.MarshalToSizedBuffer(b) if err != nil { return nil, err } return b[:n], nil } } -func (dst *HandleSubTaskSQLsRequest) XXX_Merge(src proto.Message) { - xxx_messageInfo_HandleSubTaskSQLsRequest.Merge(dst, src) +func (m *HandleSubTaskSQLsRequest) XXX_Merge(src proto.Message) { + xxx_messageInfo_HandleSubTaskSQLsRequest.Merge(m, src) } func (m *HandleSubTaskSQLsRequest) XXX_Size() int { return m.Size() @@ -722,7 +761,7 @@ func (m *CommonWorkerResponse) Reset() { *m = CommonWorkerResponse{} } func (m *CommonWorkerResponse) String() string { return proto.CompactTextString(m) } func (*CommonWorkerResponse) ProtoMessage() {} func (*CommonWorkerResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_dmworker_84482ac67274de9a, []int{9} + return fileDescriptor_51a1b9e17fd67b10, []int{9} } func (m *CommonWorkerResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -732,15 +771,15 @@ func (m *CommonWorkerResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte return xxx_messageInfo_CommonWorkerResponse.Marshal(b, m, deterministic) } else { b = b[:cap(b)] - n, err := m.MarshalTo(b) + n, err := m.MarshalToSizedBuffer(b) if err != nil { return nil, err } return b[:n], nil } } -func (dst *CommonWorkerResponse) XXX_Merge(src proto.Message) { - xxx_messageInfo_CommonWorkerResponse.Merge(dst, src) +func (m *CommonWorkerResponse) XXX_Merge(src proto.Message) { + xxx_messageInfo_CommonWorkerResponse.Merge(m, src) } func (m *CommonWorkerResponse) XXX_Size() int { return m.Size() @@ -778,8 +817,8 @@ type QueryStatusResponse struct { Result bool `protobuf:"varint,1,opt,name=result,proto3" json:"result,omitempty"` Worker string `protobuf:"bytes,2,opt,name=worker,proto3" json:"worker,omitempty"` Msg string `protobuf:"bytes,3,opt,name=msg,proto3" json:"msg,omitempty"` - SubTaskStatus []*SubTaskStatus `protobuf:"bytes,4,rep,name=subTaskStatus" json:"subTaskStatus,omitempty"` - RelayStatus *RelayStatus `protobuf:"bytes,5,opt,name=relayStatus" json:"relayStatus,omitempty"` + SubTaskStatus []*SubTaskStatus `protobuf:"bytes,4,rep,name=subTaskStatus,proto3" json:"subTaskStatus,omitempty"` + RelayStatus *RelayStatus `protobuf:"bytes,5,opt,name=relayStatus,proto3" json:"relayStatus,omitempty"` SourceID string `protobuf:"bytes,6,opt,name=sourceID,proto3" json:"sourceID,omitempty"` } @@ -787,7 +826,7 @@ func (m *QueryStatusResponse) Reset() { *m = QueryStatusResponse{} } func (m *QueryStatusResponse) String() string { return proto.CompactTextString(m) } func (*QueryStatusResponse) ProtoMessage() {} func (*QueryStatusResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_dmworker_84482ac67274de9a, []int{10} + return fileDescriptor_51a1b9e17fd67b10, []int{10} } func (m *QueryStatusResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -797,15 +836,15 @@ func (m *QueryStatusResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, return xxx_messageInfo_QueryStatusResponse.Marshal(b, m, deterministic) } else { b = b[:cap(b)] - n, err := m.MarshalTo(b) + n, err := m.MarshalToSizedBuffer(b) if err != nil { return nil, err } return b[:n], nil } } -func (dst *QueryStatusResponse) XXX_Merge(src proto.Message) { - xxx_messageInfo_QueryStatusResponse.Merge(dst, src) +func (m *QueryStatusResponse) XXX_Merge(src proto.Message) { + xxx_messageInfo_QueryStatusResponse.Merge(m, src) } func (m *QueryStatusResponse) XXX_Size() int { return m.Size() @@ -863,15 +902,15 @@ type QueryErrorResponse struct { Result bool `protobuf:"varint,1,opt,name=result,proto3" json:"result,omitempty"` Worker string `protobuf:"bytes,2,opt,name=worker,proto3" json:"worker,omitempty"` Msg string `protobuf:"bytes,3,opt,name=msg,proto3" json:"msg,omitempty"` - SubTaskError []*SubTaskError `protobuf:"bytes,4,rep,name=subTaskError" json:"subTaskError,omitempty"` - RelayError *RelayError `protobuf:"bytes,5,opt,name=RelayError" json:"RelayError,omitempty"` + SubTaskError []*SubTaskError `protobuf:"bytes,4,rep,name=subTaskError,proto3" json:"subTaskError,omitempty"` + RelayError *RelayError `protobuf:"bytes,5,opt,name=RelayError,proto3" json:"RelayError,omitempty"` } func (m *QueryErrorResponse) Reset() { *m = QueryErrorResponse{} } func (m *QueryErrorResponse) String() string { return proto.CompactTextString(m) } func (*QueryErrorResponse) ProtoMessage() {} func (*QueryErrorResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_dmworker_84482ac67274de9a, []int{11} + return fileDescriptor_51a1b9e17fd67b10, []int{11} } func (m *QueryErrorResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -881,15 +920,15 @@ func (m *QueryErrorResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, return xxx_messageInfo_QueryErrorResponse.Marshal(b, m, deterministic) } else { b = b[:cap(b)] - n, err := m.MarshalTo(b) + n, err := m.MarshalToSizedBuffer(b) if err != nil { return nil, err } return b[:n], nil } } -func (dst *QueryErrorResponse) XXX_Merge(src proto.Message) { - xxx_messageInfo_QueryErrorResponse.Merge(dst, src) +func (m *QueryErrorResponse) XXX_Merge(src proto.Message) { + xxx_messageInfo_QueryErrorResponse.Merge(m, src) } func (m *QueryErrorResponse) XXX_Size() int { return m.Size() @@ -950,7 +989,7 @@ func (m *CheckStatus) Reset() { *m = CheckStatus{} } func (m *CheckStatus) String() string { return proto.CompactTextString(m) } func (*CheckStatus) ProtoMessage() {} func (*CheckStatus) Descriptor() ([]byte, []int) { - return fileDescriptor_dmworker_84482ac67274de9a, []int{12} + return fileDescriptor_51a1b9e17fd67b10, []int{12} } func (m *CheckStatus) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -960,15 +999,15 @@ func (m *CheckStatus) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) return xxx_messageInfo_CheckStatus.Marshal(b, m, deterministic) } else { b = b[:cap(b)] - n, err := m.MarshalTo(b) + n, err := m.MarshalToSizedBuffer(b) if err != nil { return nil, err } return b[:n], nil } } -func (dst *CheckStatus) XXX_Merge(src proto.Message) { - xxx_messageInfo_CheckStatus.Merge(dst, src) +func (m *CheckStatus) XXX_Merge(src proto.Message) { + xxx_messageInfo_CheckStatus.Merge(m, src) } func (m *CheckStatus) XXX_Size() int { return m.Size() @@ -1030,7 +1069,7 @@ func (m *DumpStatus) Reset() { *m = DumpStatus{} } func (m *DumpStatus) String() string { return proto.CompactTextString(m) } func (*DumpStatus) ProtoMessage() {} func (*DumpStatus) Descriptor() ([]byte, []int) { - return fileDescriptor_dmworker_84482ac67274de9a, []int{13} + return fileDescriptor_51a1b9e17fd67b10, []int{13} } func (m *DumpStatus) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1040,15 +1079,15 @@ func (m *DumpStatus) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { return xxx_messageInfo_DumpStatus.Marshal(b, m, deterministic) } else { b = b[:cap(b)] - n, err := m.MarshalTo(b) + n, err := m.MarshalToSizedBuffer(b) if err != nil { return nil, err } return b[:n], nil } } -func (dst *DumpStatus) XXX_Merge(src proto.Message) { - xxx_messageInfo_DumpStatus.Merge(dst, src) +func (m *DumpStatus) XXX_Merge(src proto.Message) { + xxx_messageInfo_DumpStatus.Merge(m, src) } func (m *DumpStatus) XXX_Size() int { return m.Size() @@ -1071,7 +1110,7 @@ func (m *LoadStatus) Reset() { *m = LoadStatus{} } func (m *LoadStatus) String() string { return proto.CompactTextString(m) } func (*LoadStatus) ProtoMessage() {} func (*LoadStatus) Descriptor() ([]byte, []int) { - return fileDescriptor_dmworker_84482ac67274de9a, []int{14} + return fileDescriptor_51a1b9e17fd67b10, []int{14} } func (m *LoadStatus) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1081,15 +1120,15 @@ func (m *LoadStatus) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { return xxx_messageInfo_LoadStatus.Marshal(b, m, deterministic) } else { b = b[:cap(b)] - n, err := m.MarshalTo(b) + n, err := m.MarshalToSizedBuffer(b) if err != nil { return nil, err } return b[:n], nil } } -func (dst *LoadStatus) XXX_Merge(src proto.Message) { - xxx_messageInfo_LoadStatus.Merge(dst, src) +func (m *LoadStatus) XXX_Merge(src proto.Message) { + xxx_messageInfo_LoadStatus.Merge(m, src) } func (m *LoadStatus) XXX_Size() int { return m.Size() @@ -1136,17 +1175,17 @@ func (m *LoadStatus) GetMetaBinlog() string { // unsynced: unsynced source tables type ShardingGroup struct { Target string `protobuf:"bytes,1,opt,name=target,proto3" json:"target,omitempty"` - DDLs []string `protobuf:"bytes,2,rep,name=DDLs" json:"DDLs,omitempty"` + DDLs []string `protobuf:"bytes,2,rep,name=DDLs,proto3" json:"DDLs,omitempty"` FirstPos string `protobuf:"bytes,3,opt,name=firstPos,proto3" json:"firstPos,omitempty"` - Synced []string `protobuf:"bytes,4,rep,name=synced" json:"synced,omitempty"` - Unsynced []string `protobuf:"bytes,5,rep,name=unsynced" json:"unsynced,omitempty"` + Synced []string `protobuf:"bytes,4,rep,name=synced,proto3" json:"synced,omitempty"` + Unsynced []string `protobuf:"bytes,5,rep,name=unsynced,proto3" json:"unsynced,omitempty"` } func (m *ShardingGroup) Reset() { *m = ShardingGroup{} } func (m *ShardingGroup) String() string { return proto.CompactTextString(m) } func (*ShardingGroup) ProtoMessage() {} func (*ShardingGroup) Descriptor() ([]byte, []int) { - return fileDescriptor_dmworker_84482ac67274de9a, []int{15} + return fileDescriptor_51a1b9e17fd67b10, []int{15} } func (m *ShardingGroup) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1156,15 +1195,15 @@ func (m *ShardingGroup) XXX_Marshal(b []byte, deterministic bool) ([]byte, error return xxx_messageInfo_ShardingGroup.Marshal(b, m, deterministic) } else { b = b[:cap(b)] - n, err := m.MarshalTo(b) + n, err := m.MarshalToSizedBuffer(b) if err != nil { return nil, err } return b[:n], nil } } -func (dst *ShardingGroup) XXX_Merge(src proto.Message) { - xxx_messageInfo_ShardingGroup.Merge(dst, src) +func (m *ShardingGroup) XXX_Merge(src proto.Message) { + xxx_messageInfo_ShardingGroup.Merge(m, src) } func (m *ShardingGroup) XXX_Size() int { return m.Size() @@ -1219,8 +1258,8 @@ type SyncStatus struct { MasterBinlogGtid string `protobuf:"bytes,5,opt,name=masterBinlogGtid,proto3" json:"masterBinlogGtid,omitempty"` SyncerBinlog string `protobuf:"bytes,6,opt,name=syncerBinlog,proto3" json:"syncerBinlog,omitempty"` SyncerBinlogGtid string `protobuf:"bytes,7,opt,name=syncerBinlogGtid,proto3" json:"syncerBinlogGtid,omitempty"` - BlockingDDLs []string `protobuf:"bytes,8,rep,name=blockingDDLs" json:"blockingDDLs,omitempty"` - UnresolvedGroups []*ShardingGroup `protobuf:"bytes,9,rep,name=unresolvedGroups" json:"unresolvedGroups,omitempty"` + BlockingDDLs []string `protobuf:"bytes,8,rep,name=blockingDDLs,proto3" json:"blockingDDLs,omitempty"` + UnresolvedGroups []*ShardingGroup `protobuf:"bytes,9,rep,name=unresolvedGroups,proto3" json:"unresolvedGroups,omitempty"` Synced bool `protobuf:"varint,10,opt,name=synced,proto3" json:"synced,omitempty"` } @@ -1228,7 +1267,7 @@ func (m *SyncStatus) Reset() { *m = SyncStatus{} } func (m *SyncStatus) String() string { return proto.CompactTextString(m) } func (*SyncStatus) ProtoMessage() {} func (*SyncStatus) Descriptor() ([]byte, []int) { - return fileDescriptor_dmworker_84482ac67274de9a, []int{16} + return fileDescriptor_51a1b9e17fd67b10, []int{16} } func (m *SyncStatus) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1238,15 +1277,15 @@ func (m *SyncStatus) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { return xxx_messageInfo_SyncStatus.Marshal(b, m, deterministic) } else { b = b[:cap(b)] - n, err := m.MarshalTo(b) + n, err := m.MarshalToSizedBuffer(b) if err != nil { return nil, err } return b[:n], nil } } -func (dst *SyncStatus) XXX_Merge(src proto.Message) { - xxx_messageInfo_SyncStatus.Merge(dst, src) +func (m *SyncStatus) XXX_Merge(src proto.Message) { + xxx_messageInfo_SyncStatus.Merge(m, src) } func (m *SyncStatus) XXX_Size() int { return m.Size() @@ -1336,14 +1375,14 @@ type RelayStatus struct { RelayBinlogGtid string `protobuf:"bytes,5,opt,name=relayBinlogGtid,proto3" json:"relayBinlogGtid,omitempty"` RelayCatchUpMaster bool `protobuf:"varint,6,opt,name=relayCatchUpMaster,proto3" json:"relayCatchUpMaster,omitempty"` Stage Stage `protobuf:"varint,7,opt,name=stage,proto3,enum=pb.Stage" json:"stage,omitempty"` - Result *ProcessResult `protobuf:"bytes,8,opt,name=result" json:"result,omitempty"` + Result *ProcessResult `protobuf:"bytes,8,opt,name=result,proto3" json:"result,omitempty"` } func (m *RelayStatus) Reset() { *m = RelayStatus{} } func (m *RelayStatus) String() string { return proto.CompactTextString(m) } func (*RelayStatus) ProtoMessage() {} func (*RelayStatus) Descriptor() ([]byte, []int) { - return fileDescriptor_dmworker_84482ac67274de9a, []int{17} + return fileDescriptor_51a1b9e17fd67b10, []int{17} } func (m *RelayStatus) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1353,15 +1392,15 @@ func (m *RelayStatus) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) return xxx_messageInfo_RelayStatus.Marshal(b, m, deterministic) } else { b = b[:cap(b)] - n, err := m.MarshalTo(b) + n, err := m.MarshalToSizedBuffer(b) if err != nil { return nil, err } return b[:n], nil } } -func (dst *RelayStatus) XXX_Merge(src proto.Message) { - xxx_messageInfo_RelayStatus.Merge(dst, src) +func (m *RelayStatus) XXX_Merge(src proto.Message) { + xxx_messageInfo_RelayStatus.Merge(m, src) } func (m *RelayStatus) XXX_Size() int { return m.Size() @@ -1442,7 +1481,7 @@ type SubTaskStatus struct { Name string `protobuf:"bytes,1,opt,name=name,proto3" json:"name,omitempty"` Stage Stage `protobuf:"varint,2,opt,name=stage,proto3,enum=pb.Stage" json:"stage,omitempty"` Unit UnitType `protobuf:"varint,3,opt,name=unit,proto3,enum=pb.UnitType" json:"unit,omitempty"` - Result *ProcessResult `protobuf:"bytes,4,opt,name=result" json:"result,omitempty"` + Result *ProcessResult `protobuf:"bytes,4,opt,name=result,proto3" json:"result,omitempty"` UnresolvedDDLLockID string `protobuf:"bytes,5,opt,name=unresolvedDDLLockID,proto3" json:"unresolvedDDLLockID,omitempty"` // Types that are valid to be assigned to Status: // *SubTaskStatus_Msg @@ -1457,7 +1496,7 @@ func (m *SubTaskStatus) Reset() { *m = SubTaskStatus{} } func (m *SubTaskStatus) String() string { return proto.CompactTextString(m) } func (*SubTaskStatus) ProtoMessage() {} func (*SubTaskStatus) Descriptor() ([]byte, []int) { - return fileDescriptor_dmworker_84482ac67274de9a, []int{18} + return fileDescriptor_51a1b9e17fd67b10, []int{18} } func (m *SubTaskStatus) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1467,15 +1506,15 @@ func (m *SubTaskStatus) XXX_Marshal(b []byte, deterministic bool) ([]byte, error return xxx_messageInfo_SubTaskStatus.Marshal(b, m, deterministic) } else { b = b[:cap(b)] - n, err := m.MarshalTo(b) + n, err := m.MarshalToSizedBuffer(b) if err != nil { return nil, err } return b[:n], nil } } -func (dst *SubTaskStatus) XXX_Merge(src proto.Message) { - xxx_messageInfo_SubTaskStatus.Merge(dst, src) +func (m *SubTaskStatus) XXX_Merge(src proto.Message) { + xxx_messageInfo_SubTaskStatus.Merge(m, src) } func (m *SubTaskStatus) XXX_Size() int { return m.Size() @@ -1493,19 +1532,19 @@ type isSubTaskStatus_Status interface { } type SubTaskStatus_Msg struct { - Msg string `protobuf:"bytes,6,opt,name=msg,proto3,oneof"` + Msg string `protobuf:"bytes,6,opt,name=msg,proto3,oneof" json:"msg,omitempty"` } type SubTaskStatus_Check struct { - Check *CheckStatus `protobuf:"bytes,7,opt,name=check,oneof"` + Check *CheckStatus `protobuf:"bytes,7,opt,name=check,proto3,oneof" json:"check,omitempty"` } type SubTaskStatus_Dump struct { - Dump *DumpStatus `protobuf:"bytes,8,opt,name=dump,oneof"` + Dump *DumpStatus `protobuf:"bytes,8,opt,name=dump,proto3,oneof" json:"dump,omitempty"` } type SubTaskStatus_Load struct { - Load *LoadStatus `protobuf:"bytes,9,opt,name=load,oneof"` + Load *LoadStatus `protobuf:"bytes,9,opt,name=load,proto3,oneof" json:"load,omitempty"` } type SubTaskStatus_Sync struct { - Sync *SyncStatus `protobuf:"bytes,10,opt,name=sync,oneof"` + Sync *SyncStatus `protobuf:"bytes,10,opt,name=sync,proto3,oneof" json:"sync,omitempty"` } func (*SubTaskStatus_Msg) isSubTaskStatus_Status() {} @@ -1591,9 +1630,9 @@ func (m *SubTaskStatus) GetSync() *SyncStatus { return nil } -// XXX_OneofFuncs is for the internal use of the proto package. -func (*SubTaskStatus) XXX_OneofFuncs() (func(msg proto.Message, b *proto.Buffer) error, func(msg proto.Message, tag, wire int, b *proto.Buffer) (bool, error), func(msg proto.Message) (n int), []interface{}) { - return _SubTaskStatus_OneofMarshaler, _SubTaskStatus_OneofUnmarshaler, _SubTaskStatus_OneofSizer, []interface{}{ +// XXX_OneofWrappers is for the internal use of the proto package. +func (*SubTaskStatus) XXX_OneofWrappers() []interface{} { + return []interface{}{ (*SubTaskStatus_Msg)(nil), (*SubTaskStatus_Check)(nil), (*SubTaskStatus_Dump)(nil), @@ -1602,132 +1641,16 @@ func (*SubTaskStatus) XXX_OneofFuncs() (func(msg proto.Message, b *proto.Buffer) } } -func _SubTaskStatus_OneofMarshaler(msg proto.Message, b *proto.Buffer) error { - m := msg.(*SubTaskStatus) - // status - switch x := m.Status.(type) { - case *SubTaskStatus_Msg: - _ = b.EncodeVarint(6<<3 | proto.WireBytes) - _ = b.EncodeStringBytes(x.Msg) - case *SubTaskStatus_Check: - _ = b.EncodeVarint(7<<3 | proto.WireBytes) - if err := b.EncodeMessage(x.Check); err != nil { - return err - } - case *SubTaskStatus_Dump: - _ = b.EncodeVarint(8<<3 | proto.WireBytes) - if err := b.EncodeMessage(x.Dump); err != nil { - return err - } - case *SubTaskStatus_Load: - _ = b.EncodeVarint(9<<3 | proto.WireBytes) - if err := b.EncodeMessage(x.Load); err != nil { - return err - } - case *SubTaskStatus_Sync: - _ = b.EncodeVarint(10<<3 | proto.WireBytes) - if err := b.EncodeMessage(x.Sync); err != nil { - return err - } - case nil: - default: - return fmt.Errorf("SubTaskStatus.Status has unexpected type %T", x) - } - return nil -} - -func _SubTaskStatus_OneofUnmarshaler(msg proto.Message, tag, wire int, b *proto.Buffer) (bool, error) { - m := msg.(*SubTaskStatus) - switch tag { - case 6: // status.msg - if wire != proto.WireBytes { - return true, proto.ErrInternalBadWireType - } - x, err := b.DecodeStringBytes() - m.Status = &SubTaskStatus_Msg{x} - return true, err - case 7: // status.check - if wire != proto.WireBytes { - return true, proto.ErrInternalBadWireType - } - msg := new(CheckStatus) - err := b.DecodeMessage(msg) - m.Status = &SubTaskStatus_Check{msg} - return true, err - case 8: // status.dump - if wire != proto.WireBytes { - return true, proto.ErrInternalBadWireType - } - msg := new(DumpStatus) - err := b.DecodeMessage(msg) - m.Status = &SubTaskStatus_Dump{msg} - return true, err - case 9: // status.load - if wire != proto.WireBytes { - return true, proto.ErrInternalBadWireType - } - msg := new(LoadStatus) - err := b.DecodeMessage(msg) - m.Status = &SubTaskStatus_Load{msg} - return true, err - case 10: // status.sync - if wire != proto.WireBytes { - return true, proto.ErrInternalBadWireType - } - msg := new(SyncStatus) - err := b.DecodeMessage(msg) - m.Status = &SubTaskStatus_Sync{msg} - return true, err - default: - return false, nil - } -} - -func _SubTaskStatus_OneofSizer(msg proto.Message) (n int) { - m := msg.(*SubTaskStatus) - // status - switch x := m.Status.(type) { - case *SubTaskStatus_Msg: - n += 1 // tag and wire - n += proto.SizeVarint(uint64(len(x.Msg))) - n += len(x.Msg) - case *SubTaskStatus_Check: - s := proto.Size(x.Check) - n += 1 // tag and wire - n += proto.SizeVarint(uint64(s)) - n += s - case *SubTaskStatus_Dump: - s := proto.Size(x.Dump) - n += 1 // tag and wire - n += proto.SizeVarint(uint64(s)) - n += s - case *SubTaskStatus_Load: - s := proto.Size(x.Load) - n += 1 // tag and wire - n += proto.SizeVarint(uint64(s)) - n += s - case *SubTaskStatus_Sync: - s := proto.Size(x.Sync) - n += 1 // tag and wire - n += proto.SizeVarint(uint64(s)) - n += s - case nil: - default: - panic(fmt.Sprintf("proto: unexpected type %T in oneof", x)) - } - return n -} - // SubTaskStatusList used for internal jsonpb marshal type SubTaskStatusList struct { - Status []*SubTaskStatus `protobuf:"bytes,1,rep,name=status" json:"status,omitempty"` + Status []*SubTaskStatus `protobuf:"bytes,1,rep,name=status,proto3" json:"status,omitempty"` } func (m *SubTaskStatusList) Reset() { *m = SubTaskStatusList{} } func (m *SubTaskStatusList) String() string { return proto.CompactTextString(m) } func (*SubTaskStatusList) ProtoMessage() {} func (*SubTaskStatusList) Descriptor() ([]byte, []int) { - return fileDescriptor_dmworker_84482ac67274de9a, []int{19} + return fileDescriptor_51a1b9e17fd67b10, []int{19} } func (m *SubTaskStatusList) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1737,15 +1660,15 @@ func (m *SubTaskStatusList) XXX_Marshal(b []byte, deterministic bool) ([]byte, e return xxx_messageInfo_SubTaskStatusList.Marshal(b, m, deterministic) } else { b = b[:cap(b)] - n, err := m.MarshalTo(b) + n, err := m.MarshalToSizedBuffer(b) if err != nil { return nil, err } return b[:n], nil } } -func (dst *SubTaskStatusList) XXX_Merge(src proto.Message) { - xxx_messageInfo_SubTaskStatusList.Merge(dst, src) +func (m *SubTaskStatusList) XXX_Merge(src proto.Message) { + xxx_messageInfo_SubTaskStatusList.Merge(m, src) } func (m *SubTaskStatusList) XXX_Size() int { return m.Size() @@ -1773,7 +1696,7 @@ func (m *CheckError) Reset() { *m = CheckError{} } func (m *CheckError) String() string { return proto.CompactTextString(m) } func (*CheckError) ProtoMessage() {} func (*CheckError) Descriptor() ([]byte, []int) { - return fileDescriptor_dmworker_84482ac67274de9a, []int{20} + return fileDescriptor_51a1b9e17fd67b10, []int{20} } func (m *CheckError) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1783,15 +1706,15 @@ func (m *CheckError) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { return xxx_messageInfo_CheckError.Marshal(b, m, deterministic) } else { b = b[:cap(b)] - n, err := m.MarshalTo(b) + n, err := m.MarshalToSizedBuffer(b) if err != nil { return nil, err } return b[:n], nil } } -func (dst *CheckError) XXX_Merge(src proto.Message) { - xxx_messageInfo_CheckError.Merge(dst, src) +func (m *CheckError) XXX_Merge(src proto.Message) { + xxx_messageInfo_CheckError.Merge(m, src) } func (m *CheckError) XXX_Size() int { return m.Size() @@ -1819,7 +1742,7 @@ func (m *DumpError) Reset() { *m = DumpError{} } func (m *DumpError) String() string { return proto.CompactTextString(m) } func (*DumpError) ProtoMessage() {} func (*DumpError) Descriptor() ([]byte, []int) { - return fileDescriptor_dmworker_84482ac67274de9a, []int{21} + return fileDescriptor_51a1b9e17fd67b10, []int{21} } func (m *DumpError) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1829,15 +1752,15 @@ func (m *DumpError) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { return xxx_messageInfo_DumpError.Marshal(b, m, deterministic) } else { b = b[:cap(b)] - n, err := m.MarshalTo(b) + n, err := m.MarshalToSizedBuffer(b) if err != nil { return nil, err } return b[:n], nil } } -func (dst *DumpError) XXX_Merge(src proto.Message) { - xxx_messageInfo_DumpError.Merge(dst, src) +func (m *DumpError) XXX_Merge(src proto.Message) { + xxx_messageInfo_DumpError.Merge(m, src) } func (m *DumpError) XXX_Size() int { return m.Size() @@ -1864,7 +1787,7 @@ func (m *LoadError) Reset() { *m = LoadError{} } func (m *LoadError) String() string { return proto.CompactTextString(m) } func (*LoadError) ProtoMessage() {} func (*LoadError) Descriptor() ([]byte, []int) { - return fileDescriptor_dmworker_84482ac67274de9a, []int{22} + return fileDescriptor_51a1b9e17fd67b10, []int{22} } func (m *LoadError) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1874,15 +1797,15 @@ func (m *LoadError) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { return xxx_messageInfo_LoadError.Marshal(b, m, deterministic) } else { b = b[:cap(b)] - n, err := m.MarshalTo(b) + n, err := m.MarshalToSizedBuffer(b) if err != nil { return nil, err } return b[:n], nil } } -func (dst *LoadError) XXX_Merge(src proto.Message) { - xxx_messageInfo_LoadError.Merge(dst, src) +func (m *LoadError) XXX_Merge(src proto.Message) { + xxx_messageInfo_LoadError.Merge(m, src) } func (m *LoadError) XXX_Size() int { return m.Size() @@ -1911,7 +1834,7 @@ func (m *SyncSQLError) Reset() { *m = SyncSQLError{} } func (m *SyncSQLError) String() string { return proto.CompactTextString(m) } func (*SyncSQLError) ProtoMessage() {} func (*SyncSQLError) Descriptor() ([]byte, []int) { - return fileDescriptor_dmworker_84482ac67274de9a, []int{23} + return fileDescriptor_51a1b9e17fd67b10, []int{23} } func (m *SyncSQLError) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1921,15 +1844,15 @@ func (m *SyncSQLError) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) return xxx_messageInfo_SyncSQLError.Marshal(b, m, deterministic) } else { b = b[:cap(b)] - n, err := m.MarshalTo(b) + n, err := m.MarshalToSizedBuffer(b) if err != nil { return nil, err } return b[:n], nil } } -func (dst *SyncSQLError) XXX_Merge(src proto.Message) { - xxx_messageInfo_SyncSQLError.Merge(dst, src) +func (m *SyncSQLError) XXX_Merge(src proto.Message) { + xxx_messageInfo_SyncSQLError.Merge(m, src) } func (m *SyncSQLError) XXX_Size() int { return m.Size() @@ -1963,14 +1886,14 @@ func (m *SyncSQLError) GetErrorSQL() string { // SyncError represents error list for sync unit type SyncError struct { - Errors []*SyncSQLError `protobuf:"bytes,1,rep,name=errors" json:"errors,omitempty"` + Errors []*SyncSQLError `protobuf:"bytes,1,rep,name=errors,proto3" json:"errors,omitempty"` } func (m *SyncError) Reset() { *m = SyncError{} } func (m *SyncError) String() string { return proto.CompactTextString(m) } func (*SyncError) ProtoMessage() {} func (*SyncError) Descriptor() ([]byte, []int) { - return fileDescriptor_dmworker_84482ac67274de9a, []int{24} + return fileDescriptor_51a1b9e17fd67b10, []int{24} } func (m *SyncError) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1980,15 +1903,15 @@ func (m *SyncError) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { return xxx_messageInfo_SyncError.Marshal(b, m, deterministic) } else { b = b[:cap(b)] - n, err := m.MarshalTo(b) + n, err := m.MarshalToSizedBuffer(b) if err != nil { return nil, err } return b[:n], nil } } -func (dst *SyncError) XXX_Merge(src proto.Message) { - xxx_messageInfo_SyncError.Merge(dst, src) +func (m *SyncError) XXX_Merge(src proto.Message) { + xxx_messageInfo_SyncError.Merge(m, src) } func (m *SyncError) XXX_Size() int { return m.Size() @@ -2015,7 +1938,7 @@ func (m *RelayError) Reset() { *m = RelayError{} } func (m *RelayError) String() string { return proto.CompactTextString(m) } func (*RelayError) ProtoMessage() {} func (*RelayError) Descriptor() ([]byte, []int) { - return fileDescriptor_dmworker_84482ac67274de9a, []int{25} + return fileDescriptor_51a1b9e17fd67b10, []int{25} } func (m *RelayError) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2025,15 +1948,15 @@ func (m *RelayError) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { return xxx_messageInfo_RelayError.Marshal(b, m, deterministic) } else { b = b[:cap(b)] - n, err := m.MarshalTo(b) + n, err := m.MarshalToSizedBuffer(b) if err != nil { return nil, err } return b[:n], nil } } -func (dst *RelayError) XXX_Merge(src proto.Message) { - xxx_messageInfo_RelayError.Merge(dst, src) +func (m *RelayError) XXX_Merge(src proto.Message) { + xxx_messageInfo_RelayError.Merge(m, src) } func (m *RelayError) XXX_Size() int { return m.Size() @@ -2074,7 +1997,7 @@ func (m *SubTaskError) Reset() { *m = SubTaskError{} } func (m *SubTaskError) String() string { return proto.CompactTextString(m) } func (*SubTaskError) ProtoMessage() {} func (*SubTaskError) Descriptor() ([]byte, []int) { - return fileDescriptor_dmworker_84482ac67274de9a, []int{26} + return fileDescriptor_51a1b9e17fd67b10, []int{26} } func (m *SubTaskError) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2084,15 +2007,15 @@ func (m *SubTaskError) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) return xxx_messageInfo_SubTaskError.Marshal(b, m, deterministic) } else { b = b[:cap(b)] - n, err := m.MarshalTo(b) + n, err := m.MarshalToSizedBuffer(b) if err != nil { return nil, err } return b[:n], nil } } -func (dst *SubTaskError) XXX_Merge(src proto.Message) { - xxx_messageInfo_SubTaskError.Merge(dst, src) +func (m *SubTaskError) XXX_Merge(src proto.Message) { + xxx_messageInfo_SubTaskError.Merge(m, src) } func (m *SubTaskError) XXX_Size() int { return m.Size() @@ -2110,19 +2033,19 @@ type isSubTaskError_Error interface { } type SubTaskError_Msg struct { - Msg string `protobuf:"bytes,4,opt,name=msg,proto3,oneof"` + Msg string `protobuf:"bytes,4,opt,name=msg,proto3,oneof" json:"msg,omitempty"` } type SubTaskError_Check struct { - Check *CheckError `protobuf:"bytes,5,opt,name=check,oneof"` + Check *CheckError `protobuf:"bytes,5,opt,name=check,proto3,oneof" json:"check,omitempty"` } type SubTaskError_Dump struct { - Dump *DumpError `protobuf:"bytes,6,opt,name=dump,oneof"` + Dump *DumpError `protobuf:"bytes,6,opt,name=dump,proto3,oneof" json:"dump,omitempty"` } type SubTaskError_Load struct { - Load *LoadError `protobuf:"bytes,7,opt,name=load,oneof"` + Load *LoadError `protobuf:"bytes,7,opt,name=load,proto3,oneof" json:"load,omitempty"` } type SubTaskError_Sync struct { - Sync *SyncError `protobuf:"bytes,8,opt,name=sync,oneof"` + Sync *SyncError `protobuf:"bytes,8,opt,name=sync,proto3,oneof" json:"sync,omitempty"` } func (*SubTaskError_Msg) isSubTaskError_Error() {} @@ -2194,9 +2117,9 @@ func (m *SubTaskError) GetSync() *SyncError { return nil } -// XXX_OneofFuncs is for the internal use of the proto package. -func (*SubTaskError) XXX_OneofFuncs() (func(msg proto.Message, b *proto.Buffer) error, func(msg proto.Message, tag, wire int, b *proto.Buffer) (bool, error), func(msg proto.Message) (n int), []interface{}) { - return _SubTaskError_OneofMarshaler, _SubTaskError_OneofUnmarshaler, _SubTaskError_OneofSizer, []interface{}{ +// XXX_OneofWrappers is for the internal use of the proto package. +func (*SubTaskError) XXX_OneofWrappers() []interface{} { + return []interface{}{ (*SubTaskError_Msg)(nil), (*SubTaskError_Check)(nil), (*SubTaskError_Dump)(nil), @@ -2205,132 +2128,16 @@ func (*SubTaskError) XXX_OneofFuncs() (func(msg proto.Message, b *proto.Buffer) } } -func _SubTaskError_OneofMarshaler(msg proto.Message, b *proto.Buffer) error { - m := msg.(*SubTaskError) - // error - switch x := m.Error.(type) { - case *SubTaskError_Msg: - _ = b.EncodeVarint(4<<3 | proto.WireBytes) - _ = b.EncodeStringBytes(x.Msg) - case *SubTaskError_Check: - _ = b.EncodeVarint(5<<3 | proto.WireBytes) - if err := b.EncodeMessage(x.Check); err != nil { - return err - } - case *SubTaskError_Dump: - _ = b.EncodeVarint(6<<3 | proto.WireBytes) - if err := b.EncodeMessage(x.Dump); err != nil { - return err - } - case *SubTaskError_Load: - _ = b.EncodeVarint(7<<3 | proto.WireBytes) - if err := b.EncodeMessage(x.Load); err != nil { - return err - } - case *SubTaskError_Sync: - _ = b.EncodeVarint(8<<3 | proto.WireBytes) - if err := b.EncodeMessage(x.Sync); err != nil { - return err - } - case nil: - default: - return fmt.Errorf("SubTaskError.Error has unexpected type %T", x) - } - return nil -} - -func _SubTaskError_OneofUnmarshaler(msg proto.Message, tag, wire int, b *proto.Buffer) (bool, error) { - m := msg.(*SubTaskError) - switch tag { - case 4: // error.msg - if wire != proto.WireBytes { - return true, proto.ErrInternalBadWireType - } - x, err := b.DecodeStringBytes() - m.Error = &SubTaskError_Msg{x} - return true, err - case 5: // error.check - if wire != proto.WireBytes { - return true, proto.ErrInternalBadWireType - } - msg := new(CheckError) - err := b.DecodeMessage(msg) - m.Error = &SubTaskError_Check{msg} - return true, err - case 6: // error.dump - if wire != proto.WireBytes { - return true, proto.ErrInternalBadWireType - } - msg := new(DumpError) - err := b.DecodeMessage(msg) - m.Error = &SubTaskError_Dump{msg} - return true, err - case 7: // error.load - if wire != proto.WireBytes { - return true, proto.ErrInternalBadWireType - } - msg := new(LoadError) - err := b.DecodeMessage(msg) - m.Error = &SubTaskError_Load{msg} - return true, err - case 8: // error.sync - if wire != proto.WireBytes { - return true, proto.ErrInternalBadWireType - } - msg := new(SyncError) - err := b.DecodeMessage(msg) - m.Error = &SubTaskError_Sync{msg} - return true, err - default: - return false, nil - } -} - -func _SubTaskError_OneofSizer(msg proto.Message) (n int) { - m := msg.(*SubTaskError) - // error - switch x := m.Error.(type) { - case *SubTaskError_Msg: - n += 1 // tag and wire - n += proto.SizeVarint(uint64(len(x.Msg))) - n += len(x.Msg) - case *SubTaskError_Check: - s := proto.Size(x.Check) - n += 1 // tag and wire - n += proto.SizeVarint(uint64(s)) - n += s - case *SubTaskError_Dump: - s := proto.Size(x.Dump) - n += 1 // tag and wire - n += proto.SizeVarint(uint64(s)) - n += s - case *SubTaskError_Load: - s := proto.Size(x.Load) - n += 1 // tag and wire - n += proto.SizeVarint(uint64(s)) - n += s - case *SubTaskError_Sync: - s := proto.Size(x.Sync) - n += 1 // tag and wire - n += proto.SizeVarint(uint64(s)) - n += s - case nil: - default: - panic(fmt.Sprintf("proto: unexpected type %T in oneof", x)) - } - return n -} - // SubTaskErrorList used for internal jsonpb marshal type SubTaskErrorList struct { - Error []*SubTaskError `protobuf:"bytes,1,rep,name=error" json:"error,omitempty"` + Error []*SubTaskError `protobuf:"bytes,1,rep,name=error,proto3" json:"error,omitempty"` } func (m *SubTaskErrorList) Reset() { *m = SubTaskErrorList{} } func (m *SubTaskErrorList) String() string { return proto.CompactTextString(m) } func (*SubTaskErrorList) ProtoMessage() {} func (*SubTaskErrorList) Descriptor() ([]byte, []int) { - return fileDescriptor_dmworker_84482ac67274de9a, []int{27} + return fileDescriptor_51a1b9e17fd67b10, []int{27} } func (m *SubTaskErrorList) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2340,15 +2147,15 @@ func (m *SubTaskErrorList) XXX_Marshal(b []byte, deterministic bool) ([]byte, er return xxx_messageInfo_SubTaskErrorList.Marshal(b, m, deterministic) } else { b = b[:cap(b)] - n, err := m.MarshalTo(b) + n, err := m.MarshalToSizedBuffer(b) if err != nil { return nil, err } return b[:n], nil } } -func (dst *SubTaskErrorList) XXX_Merge(src proto.Message) { - xxx_messageInfo_SubTaskErrorList.Merge(dst, src) +func (m *SubTaskErrorList) XXX_Merge(src proto.Message) { + xxx_messageInfo_SubTaskErrorList.Merge(m, src) } func (m *SubTaskErrorList) XXX_Size() int { return m.Size() @@ -2372,7 +2179,7 @@ func (m *SubTaskErrorList) GetError() []*SubTaskError { // errors: includes all (potential) errors occured when processing type ProcessResult struct { IsCanceled bool `protobuf:"varint,1,opt,name=isCanceled,proto3" json:"isCanceled,omitempty"` - Errors []*ProcessError `protobuf:"bytes,2,rep,name=errors" json:"errors,omitempty"` + Errors []*ProcessError `protobuf:"bytes,2,rep,name=errors,proto3" json:"errors,omitempty"` Detail []byte `protobuf:"bytes,3,opt,name=detail,proto3" json:"detail,omitempty"` } @@ -2380,7 +2187,7 @@ func (m *ProcessResult) Reset() { *m = ProcessResult{} } func (m *ProcessResult) String() string { return proto.CompactTextString(m) } func (*ProcessResult) ProtoMessage() {} func (*ProcessResult) Descriptor() ([]byte, []int) { - return fileDescriptor_dmworker_84482ac67274de9a, []int{28} + return fileDescriptor_51a1b9e17fd67b10, []int{28} } func (m *ProcessResult) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2390,15 +2197,15 @@ func (m *ProcessResult) XXX_Marshal(b []byte, deterministic bool) ([]byte, error return xxx_messageInfo_ProcessResult.Marshal(b, m, deterministic) } else { b = b[:cap(b)] - n, err := m.MarshalTo(b) + n, err := m.MarshalToSizedBuffer(b) if err != nil { return nil, err } return b[:n], nil } } -func (dst *ProcessResult) XXX_Merge(src proto.Message) { - xxx_messageInfo_ProcessResult.Merge(dst, src) +func (m *ProcessResult) XXX_Merge(src proto.Message) { + xxx_messageInfo_ProcessResult.Merge(m, src) } func (m *ProcessResult) XXX_Size() int { return m.Size() @@ -2446,7 +2253,7 @@ func (m *TError) Reset() { *m = TError{} } func (m *TError) String() string { return proto.CompactTextString(m) } func (*TError) ProtoMessage() {} func (*TError) Descriptor() ([]byte, []int) { - return fileDescriptor_dmworker_84482ac67274de9a, []int{29} + return fileDescriptor_51a1b9e17fd67b10, []int{29} } func (m *TError) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2456,15 +2263,15 @@ func (m *TError) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { return xxx_messageInfo_TError.Marshal(b, m, deterministic) } else { b = b[:cap(b)] - n, err := m.MarshalTo(b) + n, err := m.MarshalToSizedBuffer(b) if err != nil { return nil, err } return b[:n], nil } } -func (dst *TError) XXX_Merge(src proto.Message) { - xxx_messageInfo_TError.Merge(dst, src) +func (m *TError) XXX_Merge(src proto.Message) { + xxx_messageInfo_TError.Merge(m, src) } func (m *TError) XXX_Size() int { return m.Size() @@ -2522,14 +2329,14 @@ func (m *TError) GetRawCause() string { type ProcessError struct { Type ErrorType `protobuf:"varint,1,opt,name=Type,proto3,enum=pb.ErrorType" json:"Type,omitempty"` Msg string `protobuf:"bytes,2,opt,name=msg,proto3" json:"msg,omitempty"` - Error *TError `protobuf:"bytes,3,opt,name=error" json:"error,omitempty"` + Error *TError `protobuf:"bytes,3,opt,name=error,proto3" json:"error,omitempty"` } func (m *ProcessError) Reset() { *m = ProcessError{} } func (m *ProcessError) String() string { return proto.CompactTextString(m) } func (*ProcessError) ProtoMessage() {} func (*ProcessError) Descriptor() ([]byte, []int) { - return fileDescriptor_dmworker_84482ac67274de9a, []int{30} + return fileDescriptor_51a1b9e17fd67b10, []int{30} } func (m *ProcessError) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2539,15 +2346,15 @@ func (m *ProcessError) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) return xxx_messageInfo_ProcessError.Marshal(b, m, deterministic) } else { b = b[:cap(b)] - n, err := m.MarshalTo(b) + n, err := m.MarshalToSizedBuffer(b) if err != nil { return nil, err } return b[:n], nil } } -func (dst *ProcessError) XXX_Merge(src proto.Message) { - xxx_messageInfo_ProcessError.Merge(dst, src) +func (m *ProcessError) XXX_Merge(src proto.Message) { + xxx_messageInfo_ProcessError.Merge(m, src) } func (m *ProcessError) XXX_Size() int { return m.Size() @@ -2587,14 +2394,14 @@ type DDLInfo struct { Task string `protobuf:"bytes,1,opt,name=task,proto3" json:"task,omitempty"` Schema string `protobuf:"bytes,2,opt,name=schema,proto3" json:"schema,omitempty"` Table string `protobuf:"bytes,3,opt,name=table,proto3" json:"table,omitempty"` - DDLs []string `protobuf:"bytes,4,rep,name=DDLs" json:"DDLs,omitempty"` + DDLs []string `protobuf:"bytes,4,rep,name=DDLs,proto3" json:"DDLs,omitempty"` } func (m *DDLInfo) Reset() { *m = DDLInfo{} } func (m *DDLInfo) String() string { return proto.CompactTextString(m) } func (*DDLInfo) ProtoMessage() {} func (*DDLInfo) Descriptor() ([]byte, []int) { - return fileDescriptor_dmworker_84482ac67274de9a, []int{31} + return fileDescriptor_51a1b9e17fd67b10, []int{31} } func (m *DDLInfo) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2604,15 +2411,15 @@ func (m *DDLInfo) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { return xxx_messageInfo_DDLInfo.Marshal(b, m, deterministic) } else { b = b[:cap(b)] - n, err := m.MarshalTo(b) + n, err := m.MarshalToSizedBuffer(b) if err != nil { return nil, err } return b[:n], nil } } -func (dst *DDLInfo) XXX_Merge(src proto.Message) { - xxx_messageInfo_DDLInfo.Merge(dst, src) +func (m *DDLInfo) XXX_Merge(src proto.Message) { + xxx_messageInfo_DDLInfo.Merge(m, src) } func (m *DDLInfo) XXX_Size() int { return m.Size() @@ -2663,7 +2470,7 @@ func (m *DDLLockInfo) Reset() { *m = DDLLockInfo{} } func (m *DDLLockInfo) String() string { return proto.CompactTextString(m) } func (*DDLLockInfo) ProtoMessage() {} func (*DDLLockInfo) Descriptor() ([]byte, []int) { - return fileDescriptor_dmworker_84482ac67274de9a, []int{32} + return fileDescriptor_51a1b9e17fd67b10, []int{32} } func (m *DDLLockInfo) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2673,15 +2480,15 @@ func (m *DDLLockInfo) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) return xxx_messageInfo_DDLLockInfo.Marshal(b, m, deterministic) } else { b = b[:cap(b)] - n, err := m.MarshalTo(b) + n, err := m.MarshalToSizedBuffer(b) if err != nil { return nil, err } return b[:n], nil } } -func (dst *DDLLockInfo) XXX_Merge(src proto.Message) { - xxx_messageInfo_DDLLockInfo.Merge(dst, src) +func (m *DDLLockInfo) XXX_Merge(src proto.Message) { + xxx_messageInfo_DDLLockInfo.Merge(m, src) } func (m *DDLLockInfo) XXX_Size() int { return m.Size() @@ -2712,14 +2519,14 @@ type ExecDDLRequest struct { LockID string `protobuf:"bytes,2,opt,name=lockID,proto3" json:"lockID,omitempty"` Exec bool `protobuf:"varint,3,opt,name=exec,proto3" json:"exec,omitempty"` TraceGID string `protobuf:"bytes,4,opt,name=traceGID,proto3" json:"traceGID,omitempty"` - DDLs []string `protobuf:"bytes,5,rep,name=DDLs" json:"DDLs,omitempty"` + DDLs []string `protobuf:"bytes,5,rep,name=DDLs,proto3" json:"DDLs,omitempty"` } func (m *ExecDDLRequest) Reset() { *m = ExecDDLRequest{} } func (m *ExecDDLRequest) String() string { return proto.CompactTextString(m) } func (*ExecDDLRequest) ProtoMessage() {} func (*ExecDDLRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_dmworker_84482ac67274de9a, []int{33} + return fileDescriptor_51a1b9e17fd67b10, []int{33} } func (m *ExecDDLRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2729,15 +2536,15 @@ func (m *ExecDDLRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, erro return xxx_messageInfo_ExecDDLRequest.Marshal(b, m, deterministic) } else { b = b[:cap(b)] - n, err := m.MarshalTo(b) + n, err := m.MarshalToSizedBuffer(b) if err != nil { return nil, err } return b[:n], nil } } -func (dst *ExecDDLRequest) XXX_Merge(src proto.Message) { - xxx_messageInfo_ExecDDLRequest.Merge(dst, src) +func (m *ExecDDLRequest) XXX_Merge(src proto.Message) { + xxx_messageInfo_ExecDDLRequest.Merge(m, src) } func (m *ExecDDLRequest) XXX_Size() int { return m.Size() @@ -2800,7 +2607,7 @@ func (m *BreakDDLLockRequest) Reset() { *m = BreakDDLLockRequest{} } func (m *BreakDDLLockRequest) String() string { return proto.CompactTextString(m) } func (*BreakDDLLockRequest) ProtoMessage() {} func (*BreakDDLLockRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_dmworker_84482ac67274de9a, []int{34} + return fileDescriptor_51a1b9e17fd67b10, []int{34} } func (m *BreakDDLLockRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2810,15 +2617,15 @@ func (m *BreakDDLLockRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, return xxx_messageInfo_BreakDDLLockRequest.Marshal(b, m, deterministic) } else { b = b[:cap(b)] - n, err := m.MarshalTo(b) + n, err := m.MarshalToSizedBuffer(b) if err != nil { return nil, err } return b[:n], nil } } -func (dst *BreakDDLLockRequest) XXX_Merge(src proto.Message) { - xxx_messageInfo_BreakDDLLockRequest.Merge(dst, src) +func (m *BreakDDLLockRequest) XXX_Merge(src proto.Message) { + xxx_messageInfo_BreakDDLLockRequest.Merge(m, src) } func (m *BreakDDLLockRequest) XXX_Size() int { return m.Size() @@ -2865,7 +2672,7 @@ func (m *SwitchRelayMasterRequest) Reset() { *m = SwitchRelayMasterReque func (m *SwitchRelayMasterRequest) String() string { return proto.CompactTextString(m) } func (*SwitchRelayMasterRequest) ProtoMessage() {} func (*SwitchRelayMasterRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_dmworker_84482ac67274de9a, []int{35} + return fileDescriptor_51a1b9e17fd67b10, []int{35} } func (m *SwitchRelayMasterRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2875,15 +2682,15 @@ func (m *SwitchRelayMasterRequest) XXX_Marshal(b []byte, deterministic bool) ([] return xxx_messageInfo_SwitchRelayMasterRequest.Marshal(b, m, deterministic) } else { b = b[:cap(b)] - n, err := m.MarshalTo(b) + n, err := m.MarshalToSizedBuffer(b) if err != nil { return nil, err } return b[:n], nil } } -func (dst *SwitchRelayMasterRequest) XXX_Merge(src proto.Message) { - xxx_messageInfo_SwitchRelayMasterRequest.Merge(dst, src) +func (m *SwitchRelayMasterRequest) XXX_Merge(src proto.Message) { + xxx_messageInfo_SwitchRelayMasterRequest.Merge(m, src) } func (m *SwitchRelayMasterRequest) XXX_Size() int { return m.Size() @@ -2903,7 +2710,7 @@ func (m *OperateRelayRequest) Reset() { *m = OperateRelayRequest{} } func (m *OperateRelayRequest) String() string { return proto.CompactTextString(m) } func (*OperateRelayRequest) ProtoMessage() {} func (*OperateRelayRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_dmworker_84482ac67274de9a, []int{36} + return fileDescriptor_51a1b9e17fd67b10, []int{36} } func (m *OperateRelayRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2913,15 +2720,15 @@ func (m *OperateRelayRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, return xxx_messageInfo_OperateRelayRequest.Marshal(b, m, deterministic) } else { b = b[:cap(b)] - n, err := m.MarshalTo(b) + n, err := m.MarshalToSizedBuffer(b) if err != nil { return nil, err } return b[:n], nil } } -func (dst *OperateRelayRequest) XXX_Merge(src proto.Message) { - xxx_messageInfo_OperateRelayRequest.Merge(dst, src) +func (m *OperateRelayRequest) XXX_Merge(src proto.Message) { + xxx_messageInfo_OperateRelayRequest.Merge(m, src) } func (m *OperateRelayRequest) XXX_Size() int { return m.Size() @@ -2950,7 +2757,7 @@ func (m *OperateRelayResponse) Reset() { *m = OperateRelayResponse{} } func (m *OperateRelayResponse) String() string { return proto.CompactTextString(m) } func (*OperateRelayResponse) ProtoMessage() {} func (*OperateRelayResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_dmworker_84482ac67274de9a, []int{37} + return fileDescriptor_51a1b9e17fd67b10, []int{37} } func (m *OperateRelayResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2960,15 +2767,15 @@ func (m *OperateRelayResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte return xxx_messageInfo_OperateRelayResponse.Marshal(b, m, deterministic) } else { b = b[:cap(b)] - n, err := m.MarshalTo(b) + n, err := m.MarshalToSizedBuffer(b) if err != nil { return nil, err } return b[:n], nil } } -func (dst *OperateRelayResponse) XXX_Merge(src proto.Message) { - xxx_messageInfo_OperateRelayResponse.Merge(dst, src) +func (m *OperateRelayResponse) XXX_Merge(src proto.Message) { + xxx_messageInfo_OperateRelayResponse.Merge(m, src) } func (m *OperateRelayResponse) XXX_Size() int { return m.Size() @@ -3023,7 +2830,7 @@ func (m *PurgeRelayRequest) Reset() { *m = PurgeRelayRequest{} } func (m *PurgeRelayRequest) String() string { return proto.CompactTextString(m) } func (*PurgeRelayRequest) ProtoMessage() {} func (*PurgeRelayRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_dmworker_84482ac67274de9a, []int{38} + return fileDescriptor_51a1b9e17fd67b10, []int{38} } func (m *PurgeRelayRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3033,15 +2840,15 @@ func (m *PurgeRelayRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, e return xxx_messageInfo_PurgeRelayRequest.Marshal(b, m, deterministic) } else { b = b[:cap(b)] - n, err := m.MarshalTo(b) + n, err := m.MarshalToSizedBuffer(b) if err != nil { return nil, err } return b[:n], nil } } -func (dst *PurgeRelayRequest) XXX_Merge(src proto.Message) { - xxx_messageInfo_PurgeRelayRequest.Merge(dst, src) +func (m *PurgeRelayRequest) XXX_Merge(src proto.Message) { + xxx_messageInfo_PurgeRelayRequest.Merge(m, src) } func (m *PurgeRelayRequest) XXX_Size() int { return m.Size() @@ -3087,7 +2894,7 @@ func (m *QueryWorkerConfigRequest) Reset() { *m = QueryWorkerConfigReque func (m *QueryWorkerConfigRequest) String() string { return proto.CompactTextString(m) } func (*QueryWorkerConfigRequest) ProtoMessage() {} func (*QueryWorkerConfigRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_dmworker_84482ac67274de9a, []int{39} + return fileDescriptor_51a1b9e17fd67b10, []int{39} } func (m *QueryWorkerConfigRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3097,15 +2904,15 @@ func (m *QueryWorkerConfigRequest) XXX_Marshal(b []byte, deterministic bool) ([] return xxx_messageInfo_QueryWorkerConfigRequest.Marshal(b, m, deterministic) } else { b = b[:cap(b)] - n, err := m.MarshalTo(b) + n, err := m.MarshalToSizedBuffer(b) if err != nil { return nil, err } return b[:n], nil } } -func (dst *QueryWorkerConfigRequest) XXX_Merge(src proto.Message) { - xxx_messageInfo_QueryWorkerConfigRequest.Merge(dst, src) +func (m *QueryWorkerConfigRequest) XXX_Merge(src proto.Message) { + xxx_messageInfo_QueryWorkerConfigRequest.Merge(m, src) } func (m *QueryWorkerConfigRequest) XXX_Size() int { return m.Size() @@ -3128,7 +2935,7 @@ func (m *QueryWorkerConfigResponse) Reset() { *m = QueryWorkerConfigResp func (m *QueryWorkerConfigResponse) String() string { return proto.CompactTextString(m) } func (*QueryWorkerConfigResponse) ProtoMessage() {} func (*QueryWorkerConfigResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_dmworker_84482ac67274de9a, []int{40} + return fileDescriptor_51a1b9e17fd67b10, []int{40} } func (m *QueryWorkerConfigResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3138,15 +2945,15 @@ func (m *QueryWorkerConfigResponse) XXX_Marshal(b []byte, deterministic bool) ([ return xxx_messageInfo_QueryWorkerConfigResponse.Marshal(b, m, deterministic) } else { b = b[:cap(b)] - n, err := m.MarshalTo(b) + n, err := m.MarshalToSizedBuffer(b) if err != nil { return nil, err } return b[:n], nil } } -func (dst *QueryWorkerConfigResponse) XXX_Merge(src proto.Message) { - xxx_messageInfo_QueryWorkerConfigResponse.Merge(dst, src) +func (m *QueryWorkerConfigResponse) XXX_Merge(src proto.Message) { + xxx_messageInfo_QueryWorkerConfigResponse.Merge(m, src) } func (m *QueryWorkerConfigResponse) XXX_Size() int { return m.Size() @@ -3192,7 +2999,118 @@ func (m *QueryWorkerConfigResponse) GetContent() string { return "" } +type MysqlTaskRequest struct { + Op WorkerOp `protobuf:"varint,1,opt,name=op,proto3,enum=pb.WorkerOp" json:"op,omitempty"` + Config string `protobuf:"bytes,2,opt,name=config,proto3" json:"config,omitempty"` +} + +func (m *MysqlTaskRequest) Reset() { *m = MysqlTaskRequest{} } +func (m *MysqlTaskRequest) String() string { return proto.CompactTextString(m) } +func (*MysqlTaskRequest) ProtoMessage() {} +func (*MysqlTaskRequest) Descriptor() ([]byte, []int) { + return fileDescriptor_51a1b9e17fd67b10, []int{41} +} +func (m *MysqlTaskRequest) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *MysqlTaskRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_MysqlTaskRequest.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 *MysqlTaskRequest) XXX_Merge(src proto.Message) { + xxx_messageInfo_MysqlTaskRequest.Merge(m, src) +} +func (m *MysqlTaskRequest) XXX_Size() int { + return m.Size() +} +func (m *MysqlTaskRequest) XXX_DiscardUnknown() { + xxx_messageInfo_MysqlTaskRequest.DiscardUnknown(m) +} + +var xxx_messageInfo_MysqlTaskRequest proto.InternalMessageInfo + +func (m *MysqlTaskRequest) GetOp() WorkerOp { + if m != nil { + return m.Op + } + return WorkerOp_StartWorker +} + +func (m *MysqlTaskRequest) GetConfig() string { + if m != nil { + return m.Config + } + return "" +} + +type MysqlTaskResponse struct { + Result bool `protobuf:"varint,1,opt,name=result,proto3" json:"result,omitempty"` + Msg string `protobuf:"bytes,2,opt,name=msg,proto3" json:"msg,omitempty"` +} + +func (m *MysqlTaskResponse) Reset() { *m = MysqlTaskResponse{} } +func (m *MysqlTaskResponse) String() string { return proto.CompactTextString(m) } +func (*MysqlTaskResponse) ProtoMessage() {} +func (*MysqlTaskResponse) Descriptor() ([]byte, []int) { + return fileDescriptor_51a1b9e17fd67b10, []int{42} +} +func (m *MysqlTaskResponse) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *MysqlTaskResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_MysqlTaskResponse.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 *MysqlTaskResponse) XXX_Merge(src proto.Message) { + xxx_messageInfo_MysqlTaskResponse.Merge(m, src) +} +func (m *MysqlTaskResponse) XXX_Size() int { + return m.Size() +} +func (m *MysqlTaskResponse) XXX_DiscardUnknown() { + xxx_messageInfo_MysqlTaskResponse.DiscardUnknown(m) +} + +var xxx_messageInfo_MysqlTaskResponse proto.InternalMessageInfo + +func (m *MysqlTaskResponse) GetResult() bool { + if m != nil { + return m.Result + } + return false +} + +func (m *MysqlTaskResponse) GetMsg() string { + if m != nil { + return m.Msg + } + return "" +} + func init() { + proto.RegisterEnum("pb.TaskOp", TaskOp_name, TaskOp_value) + proto.RegisterEnum("pb.SQLOp", SQLOp_name, SQLOp_value) + proto.RegisterEnum("pb.Stage", Stage_name, Stage_value) + proto.RegisterEnum("pb.UnitType", UnitType_name, UnitType_value) + proto.RegisterEnum("pb.ErrorType", ErrorType_name, ErrorType_value) + proto.RegisterEnum("pb.RelayOp", RelayOp_name, RelayOp_value) + proto.RegisterEnum("pb.WorkerOp", WorkerOp_name, WorkerOp_value) proto.RegisterType((*StartSubTaskRequest)(nil), "pb.StartSubTaskRequest") proto.RegisterType((*UpdateRelayRequest)(nil), "pb.UpdateRelayRequest") proto.RegisterType((*MigrateRelayRequest)(nil), "pb.MigrateRelayRequest") @@ -3234,12 +3152,157 @@ func init() { proto.RegisterType((*PurgeRelayRequest)(nil), "pb.PurgeRelayRequest") proto.RegisterType((*QueryWorkerConfigRequest)(nil), "pb.QueryWorkerConfigRequest") proto.RegisterType((*QueryWorkerConfigResponse)(nil), "pb.QueryWorkerConfigResponse") - proto.RegisterEnum("pb.TaskOp", TaskOp_name, TaskOp_value) - proto.RegisterEnum("pb.SQLOp", SQLOp_name, SQLOp_value) - proto.RegisterEnum("pb.Stage", Stage_name, Stage_value) - proto.RegisterEnum("pb.UnitType", UnitType_name, UnitType_value) - proto.RegisterEnum("pb.ErrorType", ErrorType_name, ErrorType_value) - proto.RegisterEnum("pb.RelayOp", RelayOp_name, RelayOp_value) + proto.RegisterType((*MysqlTaskRequest)(nil), "pb.MysqlTaskRequest") + proto.RegisterType((*MysqlTaskResponse)(nil), "pb.MysqlTaskResponse") +} + +func init() { proto.RegisterFile("dmworker.proto", fileDescriptor_51a1b9e17fd67b10) } + +var fileDescriptor_51a1b9e17fd67b10 = []byte{ + // 2287 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xb4, 0x59, 0xcd, 0x73, 0x1b, 0x49, + 0x15, 0xd7, 0x8c, 0x3e, 0x2c, 0x3f, 0xc9, 0xce, 0xb8, 0xed, 0x64, 0x15, 0x91, 0x35, 0x66, 0xb2, + 0xb5, 0xeb, 0xd5, 0xc1, 0xb5, 0x6b, 0xa0, 0xa8, 0x02, 0x02, 0x24, 0x92, 0x93, 0x08, 0xe4, 0xc4, + 0x1e, 0x39, 0xc0, 0x6d, 0x6b, 0x3c, 0x6a, 0xcb, 0x53, 0x1e, 0xcd, 0x4c, 0xe6, 0xc3, 0x5e, 0xdf, + 0xa0, 0x38, 0x52, 0x45, 0x51, 0xc5, 0x89, 0xe2, 0xcc, 0x8d, 0x3f, 0x81, 0x3b, 0x1c, 0xf7, 0x08, + 0x27, 0xa8, 0xe4, 0xdf, 0xe0, 0x40, 0xbd, 0xd7, 0x3d, 0x33, 0x3d, 0xb6, 0xa4, 0xec, 0x21, 0x7b, + 0x51, 0xcd, 0xfb, 0xe8, 0xd7, 0xaf, 0x7f, 0xef, 0xf5, 0x7b, 0xdd, 0x2d, 0x58, 0x9f, 0xcc, 0xae, + 0x82, 0xe8, 0x82, 0x47, 0x7b, 0x61, 0x14, 0x24, 0x01, 0xd3, 0xc3, 0x53, 0xf3, 0x53, 0xd8, 0x1c, + 0x27, 0x76, 0x94, 0x8c, 0xd3, 0xd3, 0x13, 0x3b, 0xbe, 0xb0, 0xf8, 0xeb, 0x94, 0xc7, 0x09, 0x63, + 0x50, 0x4b, 0xec, 0xf8, 0xa2, 0xa3, 0xed, 0x68, 0xbb, 0xab, 0x16, 0x7d, 0x9b, 0x7b, 0xc0, 0x5e, + 0x85, 0x13, 0x3b, 0xe1, 0x16, 0xf7, 0xec, 0xeb, 0x4c, 0xb3, 0x03, 0x2b, 0x4e, 0xe0, 0x27, 0xdc, + 0x4f, 0xa4, 0x72, 0x46, 0x9a, 0x63, 0xd8, 0x3c, 0x74, 0xa7, 0xd1, 0xcd, 0x01, 0xdb, 0x00, 0x4f, + 0x5c, 0xdf, 0x0b, 0xa6, 0x2f, 0xec, 0x19, 0x97, 0x63, 0x14, 0x0e, 0x7b, 0x00, 0xab, 0x82, 0x3a, + 0x0a, 0xe2, 0x8e, 0xbe, 0xa3, 0xed, 0xae, 0x59, 0x05, 0xc3, 0x7c, 0x06, 0x77, 0x5f, 0x86, 0x1c, + 0x8d, 0xde, 0xf0, 0xb8, 0x0b, 0x7a, 0x10, 0x92, 0xb9, 0xf5, 0x7d, 0xd8, 0x0b, 0x4f, 0xf7, 0x50, + 0xf8, 0x32, 0xb4, 0xf4, 0x20, 0xc4, 0xd5, 0xf8, 0x38, 0x99, 0x2e, 0x56, 0x83, 0xdf, 0xe6, 0x25, + 0xdc, 0xbb, 0x69, 0x28, 0x0e, 0x03, 0x3f, 0xe6, 0x4b, 0x2d, 0xdd, 0x83, 0x46, 0xc4, 0xe3, 0xd4, + 0x4b, 0xc8, 0x56, 0xd3, 0x92, 0x14, 0xf2, 0x05, 0xb4, 0x9d, 0x2a, 0xcd, 0x21, 0x29, 0x66, 0x40, + 0x75, 0x16, 0x4f, 0x3b, 0x35, 0x62, 0xe2, 0xa7, 0xd9, 0x83, 0x2d, 0x81, 0xe2, 0xd7, 0x40, 0x7c, + 0x17, 0xd8, 0x71, 0xca, 0xa3, 0xeb, 0x71, 0x62, 0x27, 0x69, 0xac, 0x68, 0xfa, 0x05, 0x74, 0x62, + 0x35, 0x9f, 0xc0, 0x06, 0x69, 0x1e, 0x44, 0x51, 0x10, 0x2d, 0x53, 0xfc, 0x8b, 0x06, 0x9d, 0xe7, + 0xb6, 0x3f, 0xf1, 0xb2, 0xf9, 0xc7, 0xc7, 0xa3, 0x65, 0x96, 0xd9, 0x7d, 0x42, 0x43, 0x27, 0x34, + 0x56, 0x11, 0x8d, 0xf1, 0xf1, 0xa8, 0x80, 0xd5, 0x8e, 0xa6, 0x71, 0xa7, 0xba, 0x53, 0x45, 0x75, + 0xfc, 0xc6, 0xe8, 0x9d, 0xe6, 0xd1, 0x13, 0xcb, 0x2e, 0x18, 0x18, 0xfb, 0xf8, 0xb5, 0x77, 0x64, + 0x27, 0x09, 0x8f, 0xfc, 0x4e, 0x5d, 0xc4, 0xbe, 0xe0, 0x98, 0xbf, 0x86, 0xad, 0x7e, 0x30, 0x9b, + 0x05, 0xfe, 0xaf, 0x08, 0xbe, 0x3c, 0x24, 0x05, 0xec, 0xda, 0x02, 0xd8, 0xf5, 0x79, 0xb0, 0x57, + 0x0b, 0xd8, 0xff, 0xa3, 0xc1, 0x66, 0x09, 0xcb, 0xf7, 0x65, 0x99, 0xfd, 0x00, 0xd6, 0x62, 0x09, + 0x25, 0x99, 0xee, 0xd4, 0x76, 0xaa, 0xbb, 0xad, 0xfd, 0x0d, 0xc2, 0x4a, 0x15, 0x58, 0x65, 0x3d, + 0xf6, 0x39, 0xb4, 0x22, 0xdc, 0x18, 0x72, 0x18, 0xa2, 0xd1, 0xda, 0xbf, 0x83, 0xc3, 0xac, 0x82, + 0x6d, 0xa9, 0x3a, 0xac, 0x0b, 0xcd, 0x38, 0x48, 0x23, 0x87, 0x0f, 0x07, 0x9d, 0x06, 0xb9, 0x90, + 0xd3, 0xe6, 0xdf, 0x35, 0x99, 0x2d, 0x32, 0x07, 0xde, 0xdb, 0x02, 0xbf, 0x07, 0x6d, 0xe9, 0x38, + 0x59, 0x96, 0xeb, 0x33, 0x94, 0xf5, 0x89, 0x19, 0x4b, 0x5a, 0x6c, 0x0f, 0x80, 0x96, 0x21, 0xc6, + 0x88, 0xc5, 0xad, 0xe7, 0x8b, 0x13, 0x23, 0x14, 0x0d, 0xf3, 0xaf, 0x1a, 0xb4, 0xfa, 0xe7, 0xdc, + 0xc9, 0xd0, 0xb9, 0x07, 0x8d, 0xd0, 0x8e, 0x63, 0x3e, 0xc9, 0xfc, 0x16, 0x14, 0xdb, 0x82, 0x7a, + 0x12, 0x24, 0xb6, 0x47, 0x6e, 0xd7, 0x2d, 0x41, 0x50, 0x62, 0xa5, 0x8e, 0xc3, 0xe3, 0xf8, 0x2c, + 0xf5, 0xc8, 0xf9, 0xba, 0xa5, 0x70, 0xd0, 0xda, 0x99, 0xed, 0x7a, 0x7c, 0x42, 0x39, 0x59, 0xb7, + 0x24, 0x85, 0xd5, 0xeb, 0xca, 0x8e, 0x7c, 0xd7, 0x9f, 0x92, 0x8b, 0x75, 0x2b, 0x23, 0x71, 0xc4, + 0x84, 0x27, 0xb6, 0xeb, 0x11, 0xd0, 0x6d, 0x4b, 0x52, 0x66, 0x1b, 0x60, 0x90, 0xce, 0x42, 0xe1, + 0xa5, 0xf9, 0x07, 0x0d, 0x60, 0x14, 0xd8, 0x13, 0xe9, 0xf4, 0x47, 0xb0, 0x76, 0xe6, 0xfa, 0x6e, + 0x7c, 0xce, 0x27, 0x4f, 0xae, 0x13, 0x1e, 0x93, 0xef, 0x55, 0xab, 0xcc, 0x44, 0x67, 0xc9, 0x6b, + 0xa1, 0xa2, 0x93, 0x8a, 0xc2, 0xc1, 0x28, 0x87, 0x51, 0x30, 0x8d, 0x78, 0x1c, 0xcb, 0x38, 0xe4, + 0x34, 0x8e, 0x9d, 0xf1, 0xc4, 0x16, 0x05, 0x51, 0x6e, 0x30, 0x85, 0x63, 0xfe, 0x5e, 0x83, 0xb5, + 0xf1, 0xb9, 0x1d, 0x4d, 0x5c, 0x7f, 0xfa, 0x2c, 0x0a, 0x52, 0x2a, 0x59, 0x89, 0x1d, 0x4d, 0x79, + 0x56, 0x9f, 0x25, 0x85, 0xbb, 0x77, 0x30, 0x18, 0xe1, 0xfc, 0xb4, 0x7b, 0xf1, 0x1b, 0x67, 0x3e, + 0x73, 0xa3, 0x38, 0xc1, 0xcd, 0x2b, 0x67, 0xce, 0x68, 0xb4, 0x13, 0x5f, 0xfb, 0x0e, 0x41, 0x88, + 0x23, 0x24, 0x85, 0x63, 0x52, 0x5f, 0x4a, 0xea, 0x24, 0xc9, 0x69, 0xf3, 0x77, 0x55, 0x80, 0xf1, + 0xb5, 0xef, 0x48, 0x78, 0x76, 0xa0, 0x45, 0xcb, 0x3c, 0xb8, 0xe4, 0x7e, 0x92, 0x81, 0xa3, 0xb2, + 0xd0, 0x18, 0x91, 0x27, 0x61, 0x06, 0x4c, 0x4e, 0x63, 0x69, 0x89, 0xb8, 0xc3, 0xfd, 0x04, 0x85, + 0x55, 0x12, 0x16, 0x0c, 0x66, 0x42, 0x7b, 0x66, 0xc7, 0x09, 0x8f, 0x4a, 0xd0, 0x94, 0x78, 0xac, + 0x07, 0x86, 0x4a, 0x3f, 0x4b, 0xdc, 0x89, 0x2c, 0x42, 0xb7, 0xf8, 0x68, 0x8f, 0x16, 0x91, 0xd9, + 0x13, 0xdb, 0xad, 0xc4, 0x43, 0x7b, 0x2a, 0x4d, 0xf6, 0x56, 0x84, 0xbd, 0x9b, 0x7c, 0xb4, 0x77, + 0xea, 0x05, 0xce, 0x85, 0xeb, 0x4f, 0x09, 0xf6, 0x26, 0x41, 0x55, 0xe2, 0xb1, 0x47, 0x60, 0xa4, + 0x7e, 0xc4, 0xe3, 0xc0, 0xbb, 0xe4, 0x13, 0x8a, 0x5e, 0xdc, 0x59, 0x55, 0xaa, 0x89, 0x1a, 0x57, + 0xeb, 0x96, 0xaa, 0x12, 0x21, 0x10, 0x5b, 0x46, 0x46, 0xe1, 0x1f, 0x3a, 0xb4, 0x94, 0x92, 0x72, + 0x0b, 0x2a, 0xed, 0x6b, 0x42, 0xa5, 0x2f, 0x80, 0x6a, 0x27, 0x2b, 0x64, 0xe9, 0xe9, 0xc0, 0xcd, + 0x3a, 0xa0, 0xca, 0xca, 0x35, 0x4a, 0xb1, 0x51, 0x59, 0x6c, 0x17, 0xee, 0x28, 0xa4, 0x12, 0x99, + 0x9b, 0x6c, 0xb6, 0x07, 0x8c, 0x58, 0x7d, 0x3b, 0x71, 0xce, 0x5f, 0x85, 0x87, 0xe4, 0x0d, 0x85, + 0xa7, 0x69, 0xcd, 0x91, 0xb0, 0x6f, 0x43, 0x3d, 0x4e, 0xec, 0x29, 0xa7, 0xc8, 0x64, 0x3d, 0x0c, + 0x19, 0x96, 0xe0, 0xb3, 0x4f, 0xf3, 0x0a, 0xd9, 0xa4, 0x2a, 0x45, 0x58, 0x1f, 0x45, 0x01, 0xd6, + 0x0e, 0x8b, 0x04, 0x59, 0xd1, 0x34, 0xff, 0xa7, 0xc3, 0x5a, 0xa9, 0xa6, 0xcf, 0x6d, 0x99, 0xf9, + 0x8c, 0xfa, 0x82, 0x19, 0x77, 0xa0, 0x96, 0xfa, 0x6e, 0x42, 0x48, 0xad, 0xef, 0xb7, 0x51, 0xfe, + 0xca, 0x77, 0x93, 0x93, 0xeb, 0x90, 0x5b, 0x24, 0x51, 0x7c, 0xaa, 0xbd, 0xc3, 0x27, 0xf6, 0x19, + 0x6c, 0x16, 0x99, 0x30, 0x18, 0x8c, 0x46, 0x81, 0x73, 0x31, 0x1c, 0x48, 0xf4, 0xe6, 0x89, 0x18, + 0x13, 0x25, 0x9e, 0x32, 0xfa, 0x79, 0x45, 0x14, 0xf9, 0x4f, 0xa0, 0xee, 0x60, 0xf5, 0x25, 0x94, + 0x64, 0x1b, 0x52, 0xca, 0xf1, 0xf3, 0x8a, 0x25, 0xe4, 0xec, 0x23, 0xa8, 0x4d, 0xd2, 0x59, 0x28, + 0xb1, 0xa2, 0x8a, 0x5e, 0xd4, 0xc3, 0xe7, 0x15, 0x8b, 0xa4, 0xa8, 0xe5, 0x05, 0xf6, 0xa4, 0xb3, + 0x5a, 0x68, 0x15, 0x65, 0x12, 0xb5, 0x50, 0x8a, 0x5a, 0x98, 0xa2, 0x94, 0xae, 0x52, 0xab, 0xa8, + 0x16, 0xa8, 0x85, 0xd2, 0x27, 0x4d, 0x68, 0xc4, 0xa2, 0xda, 0xfe, 0x04, 0x36, 0x4a, 0xe8, 0x8f, + 0xdc, 0x98, 0xa0, 0x12, 0xe2, 0x8e, 0xb6, 0xa8, 0xf1, 0x66, 0xe3, 0xb7, 0x01, 0x68, 0x4d, 0xa2, + 0x43, 0xc9, 0x4e, 0xa7, 0x15, 0x87, 0x84, 0x0f, 0x61, 0x15, 0xd7, 0xb2, 0x44, 0x8c, 0x8b, 0x58, + 0x24, 0x0e, 0xa1, 0x4d, 0xde, 0x1f, 0x8f, 0x16, 0x68, 0xb0, 0x7d, 0xd8, 0x12, 0x7d, 0x27, 0x3f, + 0xcf, 0xba, 0x89, 0x1b, 0xf8, 0x72, 0x63, 0xcd, 0x95, 0x61, 0x45, 0xe4, 0x68, 0x6e, 0x7c, 0x3c, + 0xca, 0x4a, 0x72, 0x46, 0x9b, 0xdf, 0x87, 0x55, 0x9c, 0x51, 0x4c, 0xb7, 0x0b, 0x0d, 0x12, 0x64, + 0x38, 0x18, 0x39, 0x9c, 0xd2, 0x21, 0x4b, 0xca, 0x11, 0x86, 0xa2, 0xf1, 0xce, 0x59, 0xc8, 0x9f, + 0x75, 0x68, 0xab, 0x9d, 0xfd, 0x9b, 0x4a, 0x72, 0xa6, 0x1c, 0x8e, 0xb3, 0x3c, 0xfc, 0x38, 0xcb, + 0x43, 0xe5, 0xc4, 0x50, 0xc4, 0xac, 0x48, 0xc3, 0x87, 0x32, 0x0d, 0x1b, 0xa4, 0xb6, 0x96, 0xa5, + 0x61, 0xa6, 0x25, 0xb2, 0xf0, 0xa1, 0xcc, 0xc2, 0x95, 0x42, 0x29, 0x0f, 0x60, 0x9e, 0x84, 0x0f, + 0x65, 0x12, 0x36, 0x0b, 0xa5, 0x1c, 0xd4, 0x3c, 0x07, 0x57, 0xa0, 0x4e, 0xe0, 0x99, 0x3f, 0x04, + 0x43, 0x85, 0x86, 0x32, 0xf0, 0x63, 0x29, 0x2c, 0x01, 0xaf, 0x9e, 0x8c, 0xe4, 0xd8, 0xd7, 0xb0, + 0x56, 0xda, 0xc2, 0xd8, 0xcc, 0xdd, 0xb8, 0x6f, 0xfb, 0x0e, 0xf7, 0xf2, 0x73, 0x8e, 0xc2, 0x51, + 0x42, 0xaa, 0x17, 0x96, 0xa5, 0x89, 0x52, 0x48, 0x95, 0xd3, 0x4a, 0xb5, 0x74, 0x5a, 0xf9, 0x9b, + 0x06, 0x8d, 0x13, 0x11, 0xc4, 0x0e, 0xac, 0x1c, 0x44, 0x51, 0x3f, 0x98, 0x88, 0x38, 0xd6, 0xad, + 0x8c, 0xc4, 0x14, 0xc3, 0x4f, 0xcf, 0x8e, 0x63, 0x79, 0xaa, 0xca, 0x69, 0x29, 0x1b, 0x3b, 0x41, + 0xc8, 0xe5, 0xb1, 0x2a, 0xa7, 0xa5, 0x6c, 0xc4, 0x2f, 0xb9, 0x27, 0x8f, 0x55, 0x39, 0x8d, 0xb3, + 0x1d, 0xf2, 0x38, 0xc6, 0x04, 0x11, 0x95, 0x28, 0x23, 0x71, 0x94, 0x65, 0x5f, 0xf5, 0xed, 0x34, + 0xe6, 0xd9, 0x19, 0x36, 0xa3, 0x4d, 0x0e, 0x6d, 0x75, 0x79, 0xec, 0x3b, 0x50, 0xc3, 0x7c, 0x91, + 0x97, 0x31, 0x8a, 0x0d, 0x09, 0x44, 0x12, 0xe1, 0x6f, 0x96, 0xbe, 0x7a, 0xb1, 0xcb, 0x76, 0xb2, + 0x70, 0x54, 0x29, 0xa2, 0xe2, 0x0a, 0x57, 0x0a, 0xc4, 0x17, 0xb0, 0x32, 0x18, 0x8c, 0x86, 0xfe, + 0x59, 0x30, 0xef, 0xda, 0x45, 0x7d, 0xd4, 0x39, 0xe7, 0x33, 0x3b, 0x3b, 0x1a, 0x0b, 0x8a, 0x8e, + 0x9e, 0xf6, 0xa9, 0xc7, 0xe5, 0x3e, 0x14, 0x44, 0x7e, 0x8e, 0xaa, 0x15, 0xe7, 0x28, 0xf3, 0x73, + 0x68, 0x65, 0xe5, 0x76, 0xd1, 0x24, 0xeb, 0xa0, 0x0f, 0x07, 0x72, 0x02, 0x7d, 0x38, 0x30, 0x7f, + 0xa3, 0xc1, 0xfa, 0xc1, 0x97, 0xdc, 0x19, 0x0c, 0x46, 0x4b, 0xae, 0x84, 0xe8, 0x9b, 0x27, 0x0a, + 0xbc, 0xf4, 0xcd, 0xcb, 0x6a, 0x7a, 0x8d, 0x7f, 0xc9, 0x1d, 0x72, 0xad, 0x69, 0xd1, 0x37, 0x1d, + 0xa6, 0x22, 0xdb, 0xe1, 0xcf, 0x86, 0x03, 0xd9, 0x72, 0x73, 0x3a, 0xf7, 0xba, 0xae, 0x78, 0xfd, + 0x5b, 0x0d, 0x36, 0x9f, 0x44, 0xdc, 0xbe, 0x90, 0xbe, 0x2f, 0xf3, 0xc3, 0x84, 0x76, 0xc4, 0x67, + 0xc1, 0x25, 0x1f, 0xa9, 0xde, 0x94, 0x78, 0x98, 0x03, 0x5c, 0xac, 0x48, 0xba, 0x95, 0x91, 0x28, + 0x89, 0x2f, 0xdc, 0x10, 0x25, 0x35, 0x21, 0x91, 0xa4, 0xd9, 0x85, 0xce, 0xf8, 0xca, 0x4d, 0x9c, + 0x73, 0xaa, 0x50, 0xa2, 0x85, 0x4b, 0x3f, 0xcc, 0x7d, 0xd8, 0x94, 0x57, 0xf6, 0xd2, 0x83, 0xc2, + 0xb7, 0x94, 0xfb, 0x7a, 0x2b, 0xbf, 0x61, 0x88, 0x3b, 0xaa, 0x99, 0xc2, 0x56, 0x79, 0x8c, 0xbc, + 0x16, 0x2d, 0x1b, 0xf4, 0x1e, 0x6e, 0xf9, 0x57, 0xb0, 0x71, 0x94, 0x46, 0xd3, 0xb2, 0xa3, 0x5d, + 0x68, 0xba, 0xbe, 0xed, 0x24, 0xee, 0x25, 0x97, 0x9b, 0x3d, 0xa7, 0x09, 0x63, 0x57, 0x3e, 0x51, + 0x54, 0x2d, 0xfa, 0x16, 0xa7, 0x71, 0x8f, 0x53, 0xe9, 0xcd, 0x4f, 0xe3, 0x82, 0xa6, 0x1c, 0x15, + 0xc7, 0xad, 0x9a, 0xcc, 0x51, 0xa2, 0x10, 0x3f, 0xba, 0x04, 0x8a, 0x0b, 0x74, 0x3f, 0xf0, 0xcf, + 0xdc, 0x69, 0x86, 0xdf, 0x9f, 0x34, 0xb8, 0x3f, 0x47, 0xf8, 0xde, 0x2e, 0x8a, 0xea, 0xed, 0xb4, + 0x56, 0xbe, 0x9d, 0xaa, 0xcf, 0x44, 0xf5, 0xf2, 0x33, 0xd1, 0x73, 0x30, 0x0e, 0xaf, 0xe3, 0xd7, + 0x9e, 0xfa, 0x18, 0xf2, 0x40, 0x89, 0x0e, 0x75, 0x0e, 0xe1, 0x71, 0x11, 0x1e, 0x87, 0x7c, 0xcf, + 0x3c, 0x12, 0x94, 0xf9, 0x08, 0x36, 0x14, 0x4b, 0xef, 0x58, 0xd6, 0xad, 0xba, 0xd1, 0xfb, 0x02, + 0x1a, 0xe2, 0xa5, 0x87, 0xad, 0xc1, 0xea, 0xd0, 0xbf, 0xb4, 0x3d, 0x77, 0xf2, 0x32, 0x34, 0x2a, + 0xac, 0x09, 0xb5, 0x71, 0x12, 0x84, 0x86, 0xc6, 0x56, 0xa1, 0x7e, 0x84, 0x85, 0xca, 0xd0, 0x19, + 0x40, 0x03, 0xab, 0xf8, 0x8c, 0x1b, 0x55, 0x64, 0xd3, 0x23, 0x9a, 0x51, 0x43, 0xb6, 0x78, 0xde, + 0x31, 0xea, 0x6c, 0x1d, 0xe0, 0x71, 0x9a, 0x04, 0x52, 0xad, 0xd1, 0xeb, 0x41, 0x9d, 0x1e, 0x4f, + 0xc8, 0xe0, 0x2f, 0x86, 0x47, 0x46, 0x85, 0xb5, 0x60, 0xc5, 0x3a, 0x38, 0x1a, 0x3d, 0xee, 0x1f, + 0x18, 0x1a, 0x8e, 0x1d, 0xbe, 0xf8, 0xf9, 0x41, 0xff, 0xc4, 0xd0, 0x7b, 0xbf, 0x24, 0x93, 0x53, + 0xac, 0x6f, 0x6d, 0xe9, 0x0b, 0xd1, 0x46, 0x85, 0xad, 0x40, 0xf5, 0x05, 0xbf, 0x32, 0x34, 0x1a, + 0x9c, 0xfa, 0x78, 0x5b, 0x15, 0xfe, 0x90, 0x6b, 0x13, 0xa3, 0x8a, 0x02, 0x74, 0x38, 0xe4, 0x13, + 0xa3, 0xc6, 0xda, 0xd0, 0x7c, 0x2a, 0xaf, 0x9f, 0x46, 0xbd, 0xf7, 0x12, 0x9a, 0x59, 0x17, 0x66, + 0x77, 0xa0, 0x25, 0x4d, 0x23, 0xcb, 0xa8, 0xe0, 0x3a, 0xa8, 0xd7, 0x1a, 0x1a, 0xba, 0x88, 0xfd, + 0xd4, 0xd0, 0xf1, 0x0b, 0x9b, 0xa6, 0x51, 0x25, 0xb7, 0xaf, 0x7d, 0xc7, 0xa8, 0xa1, 0x22, 0x65, + 0xb6, 0x31, 0xe9, 0xfd, 0x08, 0x56, 0xf3, 0x92, 0x8c, 0xce, 0xbe, 0xf2, 0x2f, 0xfc, 0xe0, 0xca, + 0x27, 0x9e, 0x58, 0x20, 0x56, 0xb5, 0xf1, 0xf1, 0xc8, 0xd0, 0x70, 0x42, 0xb2, 0xff, 0x94, 0x0e, + 0x3a, 0x86, 0xde, 0x3b, 0x84, 0x15, 0xb9, 0xef, 0x18, 0x83, 0x75, 0xe9, 0x8c, 0xe4, 0x18, 0x15, + 0x8c, 0x03, 0xae, 0x43, 0x4c, 0xa5, 0x21, 0x9e, 0xb4, 0x44, 0x41, 0xeb, 0x68, 0x4e, 0x60, 0x2b, + 0x18, 0xd5, 0xde, 0x23, 0x68, 0x66, 0x89, 0x82, 0x42, 0x8a, 0x89, 0x60, 0x18, 0x15, 0xf2, 0x8d, + 0x22, 0x23, 0xf2, 0x5e, 0xd8, 0x43, 0xf3, 0x52, 0x43, 0xdf, 0xff, 0x77, 0x13, 0x1a, 0x82, 0x60, + 0x7d, 0x68, 0xab, 0xcf, 0xa2, 0xec, 0x03, 0x79, 0xbc, 0xb9, 0xf9, 0x50, 0xda, 0xed, 0xd0, 0x01, + 0x65, 0xce, 0x9b, 0x95, 0x59, 0x61, 0x43, 0x58, 0x2f, 0x3f, 0x31, 0xb2, 0xfb, 0xa8, 0x3d, 0xf7, + 0xfd, 0xb2, 0xdb, 0x9d, 0x27, 0xca, 0x4d, 0x1d, 0xc0, 0x5a, 0xe9, 0xd5, 0x90, 0xd1, 0xbc, 0xf3, + 0x1e, 0x12, 0x97, 0x7a, 0xf4, 0x33, 0x68, 0x29, 0x8f, 0x60, 0xec, 0x1e, 0xaa, 0xde, 0x7e, 0x61, + 0xec, 0x7e, 0x70, 0x8b, 0x9f, 0x5b, 0x78, 0x04, 0x50, 0x3c, 0x32, 0xb1, 0xbb, 0xb9, 0xa2, 0xfa, + 0xf0, 0xd8, 0xbd, 0x77, 0x93, 0x9d, 0x0f, 0x7f, 0x0a, 0x20, 0x5f, 0x1f, 0x8f, 0x47, 0x31, 0x7b, + 0x80, 0x7a, 0x8b, 0x5e, 0x23, 0x97, 0x2e, 0x64, 0x1f, 0xda, 0x4f, 0x79, 0xe2, 0x9c, 0x67, 0x6d, + 0x9c, 0xee, 0x2b, 0x4a, 0xcb, 0xed, 0xb6, 0x24, 0x03, 0x09, 0xb3, 0xb2, 0xab, 0x7d, 0xa6, 0xb1, + 0x1f, 0x03, 0x60, 0x2a, 0xa6, 0x09, 0xc7, 0x16, 0xc4, 0xe8, 0x30, 0x51, 0x6a, 0xb8, 0x4b, 0x67, + 0xec, 0x43, 0x5b, 0xed, 0x8d, 0x22, 0x23, 0xe6, 0x74, 0xcb, 0xa5, 0x46, 0x0e, 0x61, 0xe3, 0x56, + 0x77, 0x13, 0x28, 0x2c, 0x6a, 0x7a, 0xef, 0xf2, 0x49, 0x6d, 0x6e, 0xc2, 0xa7, 0x39, 0x2d, 0x52, + 0x18, 0x99, 0xd7, 0x07, 0xcd, 0x0a, 0xfb, 0x29, 0x40, 0xd1, 0xaa, 0x44, 0x44, 0x6f, 0xb5, 0xae, + 0xa5, 0x5e, 0x3c, 0x83, 0x0d, 0xe5, 0x7f, 0x01, 0xb1, 0xbb, 0x44, 0x6a, 0xdd, 0xfe, 0xbb, 0x60, + 0xa9, 0x21, 0x4b, 0x3e, 0x62, 0xab, 0xed, 0x49, 0xa0, 0xb3, 0xa8, 0xa5, 0x75, 0x3f, 0x5c, 0x20, + 0x55, 0x21, 0x52, 0xff, 0x84, 0x10, 0x10, 0xcd, 0xf9, 0x5b, 0x62, 0xa9, 0x63, 0x8f, 0xc1, 0x90, + 0xe0, 0xe5, 0xfd, 0x85, 0x6d, 0x91, 0xa1, 0x1b, 0x8d, 0xab, 0x7b, 0xf7, 0x06, 0x37, 0x33, 0xf1, + 0xa4, 0xf3, 0xcf, 0x37, 0xdb, 0xda, 0x57, 0x6f, 0xb6, 0xb5, 0xff, 0xbe, 0xd9, 0xd6, 0xfe, 0xf8, + 0x76, 0xbb, 0xf2, 0xd5, 0xdb, 0xed, 0xca, 0xbf, 0xde, 0x6e, 0x57, 0x4e, 0x1b, 0xf4, 0x67, 0xcc, + 0x77, 0xff, 0x1f, 0x00, 0x00, 0xff, 0xff, 0x25, 0x43, 0x3b, 0x06, 0x9e, 0x19, 0x00, 0x00, } // Reference imports to suppress errors if they are not otherwise used. @@ -3279,6 +3342,7 @@ type WorkerClient interface { UpdateRelayConfig(ctx context.Context, in *UpdateRelayRequest, opts ...grpc.CallOption) (*CommonWorkerResponse, error) QueryWorkerConfig(ctx context.Context, in *QueryWorkerConfigRequest, opts ...grpc.CallOption) (*QueryWorkerConfigResponse, error) MigrateRelay(ctx context.Context, in *MigrateRelayRequest, opts ...grpc.CallOption) (*CommonWorkerResponse, error) + OperateMysqlTask(ctx context.Context, in *MysqlTaskRequest, opts ...grpc.CallOption) (*MysqlTaskResponse, error) } type workerClient struct { @@ -3446,6 +3510,15 @@ func (c *workerClient) MigrateRelay(ctx context.Context, in *MigrateRelayRequest return out, nil } +func (c *workerClient) OperateMysqlTask(ctx context.Context, in *MysqlTaskRequest, opts ...grpc.CallOption) (*MysqlTaskResponse, error) { + out := new(MysqlTaskResponse) + err := c.cc.Invoke(ctx, "/pb.Worker/OperateMysqlTask", in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + // WorkerServer is the server API for Worker service. type WorkerServer interface { StartSubTask(context.Context, *StartSubTaskRequest) (*CommonWorkerResponse, error) @@ -3473,6 +3546,60 @@ type WorkerServer interface { UpdateRelayConfig(context.Context, *UpdateRelayRequest) (*CommonWorkerResponse, error) QueryWorkerConfig(context.Context, *QueryWorkerConfigRequest) (*QueryWorkerConfigResponse, error) MigrateRelay(context.Context, *MigrateRelayRequest) (*CommonWorkerResponse, error) + OperateMysqlTask(context.Context, *MysqlTaskRequest) (*MysqlTaskResponse, error) +} + +// UnimplementedWorkerServer can be embedded to have forward compatible implementations. +type UnimplementedWorkerServer struct { +} + +func (*UnimplementedWorkerServer) StartSubTask(ctx context.Context, req *StartSubTaskRequest) (*CommonWorkerResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method StartSubTask not implemented") +} +func (*UnimplementedWorkerServer) OperateSubTask(ctx context.Context, req *OperateSubTaskRequest) (*OperateSubTaskResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method OperateSubTask not implemented") +} +func (*UnimplementedWorkerServer) UpdateSubTask(ctx context.Context, req *UpdateSubTaskRequest) (*CommonWorkerResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method UpdateSubTask not implemented") +} +func (*UnimplementedWorkerServer) QueryStatus(ctx context.Context, req *QueryStatusRequest) (*QueryStatusResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method QueryStatus not implemented") +} +func (*UnimplementedWorkerServer) QueryError(ctx context.Context, req *QueryErrorRequest) (*QueryErrorResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method QueryError not implemented") +} +func (*UnimplementedWorkerServer) HandleSQLs(ctx context.Context, req *HandleSubTaskSQLsRequest) (*CommonWorkerResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method HandleSQLs not implemented") +} +func (*UnimplementedWorkerServer) FetchDDLInfo(srv Worker_FetchDDLInfoServer) error { + return status.Errorf(codes.Unimplemented, "method FetchDDLInfo not implemented") +} +func (*UnimplementedWorkerServer) ExecuteDDL(ctx context.Context, req *ExecDDLRequest) (*CommonWorkerResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method ExecuteDDL not implemented") +} +func (*UnimplementedWorkerServer) BreakDDLLock(ctx context.Context, req *BreakDDLLockRequest) (*CommonWorkerResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method BreakDDLLock not implemented") +} +func (*UnimplementedWorkerServer) SwitchRelayMaster(ctx context.Context, req *SwitchRelayMasterRequest) (*CommonWorkerResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method SwitchRelayMaster not implemented") +} +func (*UnimplementedWorkerServer) OperateRelay(ctx context.Context, req *OperateRelayRequest) (*OperateRelayResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method OperateRelay not implemented") +} +func (*UnimplementedWorkerServer) PurgeRelay(ctx context.Context, req *PurgeRelayRequest) (*CommonWorkerResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method PurgeRelay not implemented") +} +func (*UnimplementedWorkerServer) UpdateRelayConfig(ctx context.Context, req *UpdateRelayRequest) (*CommonWorkerResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method UpdateRelayConfig not implemented") +} +func (*UnimplementedWorkerServer) QueryWorkerConfig(ctx context.Context, req *QueryWorkerConfigRequest) (*QueryWorkerConfigResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method QueryWorkerConfig not implemented") +} +func (*UnimplementedWorkerServer) MigrateRelay(ctx context.Context, req *MigrateRelayRequest) (*CommonWorkerResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method MigrateRelay not implemented") +} +func (*UnimplementedWorkerServer) OperateMysqlTask(ctx context.Context, req *MysqlTaskRequest) (*MysqlTaskResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method OperateMysqlTask not implemented") } func RegisterWorkerServer(s *grpc.Server, srv WorkerServer) { @@ -3757,6 +3884,24 @@ func _Worker_MigrateRelay_Handler(srv interface{}, ctx context.Context, dec func return interceptor(ctx, in, info, handler) } +func _Worker_OperateMysqlTask_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(MysqlTaskRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(WorkerServer).OperateMysqlTask(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: "/pb.Worker/OperateMysqlTask", + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(WorkerServer).OperateMysqlTask(ctx, req.(*MysqlTaskRequest)) + } + return interceptor(ctx, in, info, handler) +} + var _Worker_serviceDesc = grpc.ServiceDesc{ ServiceName: "pb.Worker", HandlerType: (*WorkerServer)(nil), @@ -3817,6 +3962,10 @@ var _Worker_serviceDesc = grpc.ServiceDesc{ MethodName: "MigrateRelay", Handler: _Worker_MigrateRelay_Handler, }, + { + MethodName: "OperateMysqlTask", + Handler: _Worker_OperateMysqlTask_Handler, + }, }, Streams: []grpc.StreamDesc{ { @@ -3832,7 +3981,7 @@ var _Worker_serviceDesc = grpc.ServiceDesc{ func (m *StartSubTaskRequest) Marshal() (dAtA []byte, err error) { size := m.Size() dAtA = make([]byte, size) - n, err := m.MarshalTo(dAtA) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) if err != nil { return nil, err } @@ -3840,23 +3989,29 @@ func (m *StartSubTaskRequest) Marshal() (dAtA []byte, err error) { } func (m *StartSubTaskRequest) MarshalTo(dAtA []byte) (int, error) { - var i int + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *StartSubTaskRequest) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) _ = i var l int _ = l if len(m.Task) > 0 { - dAtA[i] = 0xa - i++ + i -= len(m.Task) + copy(dAtA[i:], m.Task) i = encodeVarintDmworker(dAtA, i, uint64(len(m.Task))) - i += copy(dAtA[i:], m.Task) + i-- + dAtA[i] = 0xa } - return i, nil + return len(dAtA) - i, nil } func (m *UpdateRelayRequest) Marshal() (dAtA []byte, err error) { size := m.Size() dAtA = make([]byte, size) - n, err := m.MarshalTo(dAtA) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) if err != nil { return nil, err } @@ -3864,23 +4019,29 @@ func (m *UpdateRelayRequest) Marshal() (dAtA []byte, err error) { } func (m *UpdateRelayRequest) MarshalTo(dAtA []byte) (int, error) { - var i int + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *UpdateRelayRequest) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) _ = i var l int _ = l if len(m.Content) > 0 { - dAtA[i] = 0xa - i++ + i -= len(m.Content) + copy(dAtA[i:], m.Content) i = encodeVarintDmworker(dAtA, i, uint64(len(m.Content))) - i += copy(dAtA[i:], m.Content) + i-- + dAtA[i] = 0xa } - return i, nil + return len(dAtA) - i, nil } func (m *MigrateRelayRequest) Marshal() (dAtA []byte, err error) { size := m.Size() dAtA = make([]byte, size) - n, err := m.MarshalTo(dAtA) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) if err != nil { return nil, err } @@ -3888,28 +4049,34 @@ func (m *MigrateRelayRequest) Marshal() (dAtA []byte, err error) { } func (m *MigrateRelayRequest) MarshalTo(dAtA []byte) (int, error) { - var i int + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *MigrateRelayRequest) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) _ = i var l int _ = l - if len(m.BinlogName) > 0 { - dAtA[i] = 0xa - i++ - i = encodeVarintDmworker(dAtA, i, uint64(len(m.BinlogName))) - i += copy(dAtA[i:], m.BinlogName) - } if m.BinlogPos != 0 { - dAtA[i] = 0x10 - i++ i = encodeVarintDmworker(dAtA, i, uint64(m.BinlogPos)) + i-- + dAtA[i] = 0x10 + } + if len(m.BinlogName) > 0 { + i -= len(m.BinlogName) + copy(dAtA[i:], m.BinlogName) + i = encodeVarintDmworker(dAtA, i, uint64(len(m.BinlogName))) + i-- + dAtA[i] = 0xa } - return i, nil + return len(dAtA) - i, nil } func (m *OperateSubTaskRequest) Marshal() (dAtA []byte, err error) { size := m.Size() dAtA = make([]byte, size) - n, err := m.MarshalTo(dAtA) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) if err != nil { return nil, err } @@ -3917,28 +4084,34 @@ func (m *OperateSubTaskRequest) Marshal() (dAtA []byte, err error) { } func (m *OperateSubTaskRequest) MarshalTo(dAtA []byte) (int, error) { - var i int + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *OperateSubTaskRequest) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) _ = i var l int _ = l - if m.Op != 0 { - dAtA[i] = 0x8 - i++ - i = encodeVarintDmworker(dAtA, i, uint64(m.Op)) - } if len(m.Name) > 0 { - dAtA[i] = 0x12 - i++ + i -= len(m.Name) + copy(dAtA[i:], m.Name) i = encodeVarintDmworker(dAtA, i, uint64(len(m.Name))) - i += copy(dAtA[i:], m.Name) + i-- + dAtA[i] = 0x12 + } + if m.Op != 0 { + i = encodeVarintDmworker(dAtA, i, uint64(m.Op)) + i-- + dAtA[i] = 0x8 } - return i, nil + return len(dAtA) - i, nil } func (m *OperateSubTaskResponse) Marshal() (dAtA []byte, err error) { size := m.Size() dAtA = make([]byte, size) - n, err := m.MarshalTo(dAtA) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) if err != nil { return nil, err } @@ -3946,44 +4119,51 @@ func (m *OperateSubTaskResponse) Marshal() (dAtA []byte, err error) { } func (m *OperateSubTaskResponse) MarshalTo(dAtA []byte) (int, error) { - var i int + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *OperateSubTaskResponse) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) _ = i var l int _ = l - if m.Op != 0 { - dAtA[i] = 0x8 - i++ - i = encodeVarintDmworker(dAtA, i, uint64(m.Op)) + if len(m.Msg) > 0 { + i -= len(m.Msg) + copy(dAtA[i:], m.Msg) + i = encodeVarintDmworker(dAtA, i, uint64(len(m.Msg))) + i-- + dAtA[i] = 0x22 + } + if len(m.Worker) > 0 { + i -= len(m.Worker) + copy(dAtA[i:], m.Worker) + i = encodeVarintDmworker(dAtA, i, uint64(len(m.Worker))) + i-- + dAtA[i] = 0x1a } if m.Result { - dAtA[i] = 0x10 - i++ + i-- if m.Result { dAtA[i] = 1 } else { dAtA[i] = 0 } - i++ - } - if len(m.Worker) > 0 { - dAtA[i] = 0x1a - i++ - i = encodeVarintDmworker(dAtA, i, uint64(len(m.Worker))) - i += copy(dAtA[i:], m.Worker) + i-- + dAtA[i] = 0x10 } - if len(m.Msg) > 0 { - dAtA[i] = 0x22 - i++ - i = encodeVarintDmworker(dAtA, i, uint64(len(m.Msg))) - i += copy(dAtA[i:], m.Msg) + if m.Op != 0 { + i = encodeVarintDmworker(dAtA, i, uint64(m.Op)) + i-- + dAtA[i] = 0x8 } - return i, nil + return len(dAtA) - i, nil } func (m *UpdateSubTaskRequest) Marshal() (dAtA []byte, err error) { size := m.Size() dAtA = make([]byte, size) - n, err := m.MarshalTo(dAtA) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) if err != nil { return nil, err } @@ -3991,23 +4171,29 @@ func (m *UpdateSubTaskRequest) Marshal() (dAtA []byte, err error) { } func (m *UpdateSubTaskRequest) MarshalTo(dAtA []byte) (int, error) { - var i int + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *UpdateSubTaskRequest) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) _ = i var l int _ = l if len(m.Task) > 0 { - dAtA[i] = 0xa - i++ + i -= len(m.Task) + copy(dAtA[i:], m.Task) i = encodeVarintDmworker(dAtA, i, uint64(len(m.Task))) - i += copy(dAtA[i:], m.Task) + i-- + dAtA[i] = 0xa } - return i, nil + return len(dAtA) - i, nil } func (m *QueryStatusRequest) Marshal() (dAtA []byte, err error) { size := m.Size() dAtA = make([]byte, size) - n, err := m.MarshalTo(dAtA) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) if err != nil { return nil, err } @@ -4015,23 +4201,29 @@ func (m *QueryStatusRequest) Marshal() (dAtA []byte, err error) { } func (m *QueryStatusRequest) MarshalTo(dAtA []byte) (int, error) { - var i int + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *QueryStatusRequest) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) _ = i var l int _ = l if len(m.Name) > 0 { - dAtA[i] = 0xa - i++ + i -= len(m.Name) + copy(dAtA[i:], m.Name) i = encodeVarintDmworker(dAtA, i, uint64(len(m.Name))) - i += copy(dAtA[i:], m.Name) + i-- + dAtA[i] = 0xa } - return i, nil + return len(dAtA) - i, nil } func (m *QueryErrorRequest) Marshal() (dAtA []byte, err error) { size := m.Size() dAtA = make([]byte, size) - n, err := m.MarshalTo(dAtA) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) if err != nil { return nil, err } @@ -4039,23 +4231,29 @@ func (m *QueryErrorRequest) Marshal() (dAtA []byte, err error) { } func (m *QueryErrorRequest) MarshalTo(dAtA []byte) (int, error) { - var i int + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *QueryErrorRequest) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) _ = i var l int _ = l if len(m.Name) > 0 { - dAtA[i] = 0xa - i++ + i -= len(m.Name) + copy(dAtA[i:], m.Name) i = encodeVarintDmworker(dAtA, i, uint64(len(m.Name))) - i += copy(dAtA[i:], m.Name) + i-- + dAtA[i] = 0xa } - return i, nil + return len(dAtA) - i, nil } func (m *HandleSubTaskSQLsRequest) Marshal() (dAtA []byte, err error) { size := m.Size() dAtA = make([]byte, size) - n, err := m.MarshalTo(dAtA) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) if err != nil { return nil, err } @@ -4063,55 +4261,57 @@ func (m *HandleSubTaskSQLsRequest) Marshal() (dAtA []byte, err error) { } func (m *HandleSubTaskSQLsRequest) MarshalTo(dAtA []byte) (int, error) { - var i int + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *HandleSubTaskSQLsRequest) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) _ = i var l int _ = l - if len(m.Name) > 0 { - dAtA[i] = 0xa - i++ - i = encodeVarintDmworker(dAtA, i, uint64(len(m.Name))) - i += copy(dAtA[i:], m.Name) + if len(m.SqlPattern) > 0 { + i -= len(m.SqlPattern) + copy(dAtA[i:], m.SqlPattern) + i = encodeVarintDmworker(dAtA, i, uint64(len(m.SqlPattern))) + i-- + dAtA[i] = 0x2a } - if m.Op != 0 { - dAtA[i] = 0x10 - i++ - i = encodeVarintDmworker(dAtA, i, uint64(m.Op)) + if len(m.BinlogPos) > 0 { + i -= len(m.BinlogPos) + copy(dAtA[i:], m.BinlogPos) + i = encodeVarintDmworker(dAtA, i, uint64(len(m.BinlogPos))) + i-- + dAtA[i] = 0x22 } if len(m.Args) > 0 { - for _, s := range m.Args { + for iNdEx := len(m.Args) - 1; iNdEx >= 0; iNdEx-- { + i -= len(m.Args[iNdEx]) + copy(dAtA[i:], m.Args[iNdEx]) + i = encodeVarintDmworker(dAtA, i, uint64(len(m.Args[iNdEx]))) + i-- dAtA[i] = 0x1a - i++ - l = len(s) - for l >= 1<<7 { - dAtA[i] = uint8(uint64(l)&0x7f | 0x80) - l >>= 7 - i++ - } - dAtA[i] = uint8(l) - i++ - i += copy(dAtA[i:], s) } } - if len(m.BinlogPos) > 0 { - dAtA[i] = 0x22 - i++ - i = encodeVarintDmworker(dAtA, i, uint64(len(m.BinlogPos))) - i += copy(dAtA[i:], m.BinlogPos) + if m.Op != 0 { + i = encodeVarintDmworker(dAtA, i, uint64(m.Op)) + i-- + dAtA[i] = 0x10 } - if len(m.SqlPattern) > 0 { - dAtA[i] = 0x2a - i++ - i = encodeVarintDmworker(dAtA, i, uint64(len(m.SqlPattern))) - i += copy(dAtA[i:], m.SqlPattern) + if len(m.Name) > 0 { + i -= len(m.Name) + copy(dAtA[i:], m.Name) + i = encodeVarintDmworker(dAtA, i, uint64(len(m.Name))) + i-- + dAtA[i] = 0xa } - return i, nil + return len(dAtA) - i, nil } func (m *CommonWorkerResponse) Marshal() (dAtA []byte, err error) { size := m.Size() dAtA = make([]byte, size) - n, err := m.MarshalTo(dAtA) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) if err != nil { return nil, err } @@ -4119,39 +4319,46 @@ func (m *CommonWorkerResponse) Marshal() (dAtA []byte, err error) { } func (m *CommonWorkerResponse) MarshalTo(dAtA []byte) (int, error) { - var i int + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *CommonWorkerResponse) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) _ = i var l int _ = l + if len(m.Msg) > 0 { + i -= len(m.Msg) + copy(dAtA[i:], m.Msg) + i = encodeVarintDmworker(dAtA, i, uint64(len(m.Msg))) + i-- + dAtA[i] = 0x1a + } + if len(m.Worker) > 0 { + i -= len(m.Worker) + copy(dAtA[i:], m.Worker) + i = encodeVarintDmworker(dAtA, i, uint64(len(m.Worker))) + i-- + dAtA[i] = 0x12 + } if m.Result { - dAtA[i] = 0x8 - i++ + i-- if m.Result { dAtA[i] = 1 } else { dAtA[i] = 0 } - i++ - } - if len(m.Worker) > 0 { - dAtA[i] = 0x12 - i++ - i = encodeVarintDmworker(dAtA, i, uint64(len(m.Worker))) - i += copy(dAtA[i:], m.Worker) - } - if len(m.Msg) > 0 { - dAtA[i] = 0x1a - i++ - i = encodeVarintDmworker(dAtA, i, uint64(len(m.Msg))) - i += copy(dAtA[i:], m.Msg) + i-- + dAtA[i] = 0x8 } - return i, nil + return len(dAtA) - i, nil } func (m *QueryStatusResponse) Marshal() (dAtA []byte, err error) { size := m.Size() dAtA = make([]byte, size) - n, err := m.MarshalTo(dAtA) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) if err != nil { return nil, err } @@ -4159,67 +4366,79 @@ func (m *QueryStatusResponse) Marshal() (dAtA []byte, err error) { } func (m *QueryStatusResponse) MarshalTo(dAtA []byte) (int, error) { - var i int + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *QueryStatusResponse) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) _ = i var l int _ = l - if m.Result { - dAtA[i] = 0x8 - i++ - if m.Result { - dAtA[i] = 1 - } else { - dAtA[i] = 0 - } - i++ - } - if len(m.Worker) > 0 { - dAtA[i] = 0x12 - i++ - i = encodeVarintDmworker(dAtA, i, uint64(len(m.Worker))) - i += copy(dAtA[i:], m.Worker) - } - if len(m.Msg) > 0 { - dAtA[i] = 0x1a - i++ - i = encodeVarintDmworker(dAtA, i, uint64(len(m.Msg))) - i += copy(dAtA[i:], m.Msg) + if len(m.SourceID) > 0 { + i -= len(m.SourceID) + copy(dAtA[i:], m.SourceID) + i = encodeVarintDmworker(dAtA, i, uint64(len(m.SourceID))) + i-- + dAtA[i] = 0x32 } - if len(m.SubTaskStatus) > 0 { - for _, msg := range m.SubTaskStatus { - dAtA[i] = 0x22 - i++ - i = encodeVarintDmworker(dAtA, i, uint64(msg.Size())) - n, err := msg.MarshalTo(dAtA[i:]) + if m.RelayStatus != nil { + { + size, err := m.RelayStatus.MarshalToSizedBuffer(dAtA[:i]) if err != nil { return 0, err } - i += n + i -= size + i = encodeVarintDmworker(dAtA, i, uint64(size)) } - } - if m.RelayStatus != nil { + i-- dAtA[i] = 0x2a - i++ - i = encodeVarintDmworker(dAtA, i, uint64(m.RelayStatus.Size())) - n1, err := m.RelayStatus.MarshalTo(dAtA[i:]) - if err != nil { - return 0, err + } + if len(m.SubTaskStatus) > 0 { + for iNdEx := len(m.SubTaskStatus) - 1; iNdEx >= 0; iNdEx-- { + { + size, err := m.SubTaskStatus[iNdEx].MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintDmworker(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x22 } - i += n1 } - if len(m.SourceID) > 0 { - dAtA[i] = 0x32 - i++ - i = encodeVarintDmworker(dAtA, i, uint64(len(m.SourceID))) - i += copy(dAtA[i:], m.SourceID) + if len(m.Msg) > 0 { + i -= len(m.Msg) + copy(dAtA[i:], m.Msg) + i = encodeVarintDmworker(dAtA, i, uint64(len(m.Msg))) + i-- + dAtA[i] = 0x1a } - return i, nil + if len(m.Worker) > 0 { + i -= len(m.Worker) + copy(dAtA[i:], m.Worker) + i = encodeVarintDmworker(dAtA, i, uint64(len(m.Worker))) + i-- + dAtA[i] = 0x12 + } + if m.Result { + i-- + if m.Result { + dAtA[i] = 1 + } else { + dAtA[i] = 0 + } + i-- + dAtA[i] = 0x8 + } + return len(dAtA) - i, nil } func (m *QueryErrorResponse) Marshal() (dAtA []byte, err error) { size := m.Size() dAtA = make([]byte, size) - n, err := m.MarshalTo(dAtA) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) if err != nil { return nil, err } @@ -4227,61 +4446,72 @@ func (m *QueryErrorResponse) Marshal() (dAtA []byte, err error) { } func (m *QueryErrorResponse) MarshalTo(dAtA []byte) (int, error) { - var i int + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *QueryErrorResponse) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) _ = i var l int _ = l - if m.Result { - dAtA[i] = 0x8 - i++ - if m.Result { - dAtA[i] = 1 - } else { - dAtA[i] = 0 + if m.RelayError != nil { + { + size, err := m.RelayError.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintDmworker(dAtA, i, uint64(size)) } - i++ + i-- + dAtA[i] = 0x2a } - if len(m.Worker) > 0 { - dAtA[i] = 0x12 - i++ - i = encodeVarintDmworker(dAtA, i, uint64(len(m.Worker))) - i += copy(dAtA[i:], m.Worker) + if len(m.SubTaskError) > 0 { + for iNdEx := len(m.SubTaskError) - 1; iNdEx >= 0; iNdEx-- { + { + size, err := m.SubTaskError[iNdEx].MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintDmworker(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x22 + } } if len(m.Msg) > 0 { - dAtA[i] = 0x1a - i++ + i -= len(m.Msg) + copy(dAtA[i:], m.Msg) i = encodeVarintDmworker(dAtA, i, uint64(len(m.Msg))) - i += copy(dAtA[i:], m.Msg) + i-- + dAtA[i] = 0x1a } - if len(m.SubTaskError) > 0 { - for _, msg := range m.SubTaskError { - dAtA[i] = 0x22 - i++ - i = encodeVarintDmworker(dAtA, i, uint64(msg.Size())) - n, err := msg.MarshalTo(dAtA[i:]) - if err != nil { - return 0, err - } - i += n - } + if len(m.Worker) > 0 { + i -= len(m.Worker) + copy(dAtA[i:], m.Worker) + i = encodeVarintDmworker(dAtA, i, uint64(len(m.Worker))) + i-- + dAtA[i] = 0x12 } - if m.RelayError != nil { - dAtA[i] = 0x2a - i++ - i = encodeVarintDmworker(dAtA, i, uint64(m.RelayError.Size())) - n2, err := m.RelayError.MarshalTo(dAtA[i:]) - if err != nil { - return 0, err + if m.Result { + i-- + if m.Result { + dAtA[i] = 1 + } else { + dAtA[i] = 0 } - i += n2 + i-- + dAtA[i] = 0x8 } - return i, nil + return len(dAtA) - i, nil } func (m *CheckStatus) Marshal() (dAtA []byte, err error) { size := m.Size() dAtA = make([]byte, size) - n, err := m.MarshalTo(dAtA) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) if err != nil { return nil, err } @@ -4289,53 +4519,59 @@ func (m *CheckStatus) Marshal() (dAtA []byte, err error) { } func (m *CheckStatus) MarshalTo(dAtA []byte) (int, error) { - var i int + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *CheckStatus) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) _ = i var l int _ = l - if m.Passed { - dAtA[i] = 0x8 - i++ - if m.Passed { - dAtA[i] = 1 - } else { - dAtA[i] = 0 - } - i++ - } - if m.Total != 0 { - dAtA[i] = 0x10 - i++ - i = encodeVarintDmworker(dAtA, i, uint64(m.Total)) + if len(m.Detail) > 0 { + i -= len(m.Detail) + copy(dAtA[i:], m.Detail) + i = encodeVarintDmworker(dAtA, i, uint64(len(m.Detail))) + i-- + dAtA[i] = 0x32 } - if m.Successful != 0 { - dAtA[i] = 0x18 - i++ - i = encodeVarintDmworker(dAtA, i, uint64(m.Successful)) + if m.Warning != 0 { + i = encodeVarintDmworker(dAtA, i, uint64(m.Warning)) + i-- + dAtA[i] = 0x28 } if m.Failed != 0 { - dAtA[i] = 0x20 - i++ i = encodeVarintDmworker(dAtA, i, uint64(m.Failed)) + i-- + dAtA[i] = 0x20 } - if m.Warning != 0 { - dAtA[i] = 0x28 - i++ - i = encodeVarintDmworker(dAtA, i, uint64(m.Warning)) + if m.Successful != 0 { + i = encodeVarintDmworker(dAtA, i, uint64(m.Successful)) + i-- + dAtA[i] = 0x18 } - if len(m.Detail) > 0 { - dAtA[i] = 0x32 - i++ - i = encodeVarintDmworker(dAtA, i, uint64(len(m.Detail))) - i += copy(dAtA[i:], m.Detail) + if m.Total != 0 { + i = encodeVarintDmworker(dAtA, i, uint64(m.Total)) + i-- + dAtA[i] = 0x10 } - return i, nil + if m.Passed { + i-- + if m.Passed { + dAtA[i] = 1 + } else { + dAtA[i] = 0 + } + i-- + dAtA[i] = 0x8 + } + return len(dAtA) - i, nil } func (m *DumpStatus) Marshal() (dAtA []byte, err error) { size := m.Size() dAtA = make([]byte, size) - n, err := m.MarshalTo(dAtA) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) if err != nil { return nil, err } @@ -4343,17 +4579,22 @@ func (m *DumpStatus) Marshal() (dAtA []byte, err error) { } func (m *DumpStatus) MarshalTo(dAtA []byte) (int, error) { - var i int + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *DumpStatus) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) _ = i var l int _ = l - return i, nil + return len(dAtA) - i, nil } func (m *LoadStatus) Marshal() (dAtA []byte, err error) { size := m.Size() dAtA = make([]byte, size) - n, err := m.MarshalTo(dAtA) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) if err != nil { return nil, err } @@ -4361,39 +4602,46 @@ func (m *LoadStatus) Marshal() (dAtA []byte, err error) { } func (m *LoadStatus) MarshalTo(dAtA []byte) (int, error) { - var i int + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *LoadStatus) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) _ = i var l int _ = l - if m.FinishedBytes != 0 { - dAtA[i] = 0x8 - i++ - i = encodeVarintDmworker(dAtA, i, uint64(m.FinishedBytes)) - } - if m.TotalBytes != 0 { - dAtA[i] = 0x10 - i++ - i = encodeVarintDmworker(dAtA, i, uint64(m.TotalBytes)) + if len(m.MetaBinlog) > 0 { + i -= len(m.MetaBinlog) + copy(dAtA[i:], m.MetaBinlog) + i = encodeVarintDmworker(dAtA, i, uint64(len(m.MetaBinlog))) + i-- + dAtA[i] = 0x22 } if len(m.Progress) > 0 { - dAtA[i] = 0x1a - i++ + i -= len(m.Progress) + copy(dAtA[i:], m.Progress) i = encodeVarintDmworker(dAtA, i, uint64(len(m.Progress))) - i += copy(dAtA[i:], m.Progress) + i-- + dAtA[i] = 0x1a } - if len(m.MetaBinlog) > 0 { - dAtA[i] = 0x22 - i++ - i = encodeVarintDmworker(dAtA, i, uint64(len(m.MetaBinlog))) - i += copy(dAtA[i:], m.MetaBinlog) + if m.TotalBytes != 0 { + i = encodeVarintDmworker(dAtA, i, uint64(m.TotalBytes)) + i-- + dAtA[i] = 0x10 + } + if m.FinishedBytes != 0 { + i = encodeVarintDmworker(dAtA, i, uint64(m.FinishedBytes)) + i-- + dAtA[i] = 0x8 } - return i, nil + return len(dAtA) - i, nil } func (m *ShardingGroup) Marshal() (dAtA []byte, err error) { size := m.Size() dAtA = make([]byte, size) - n, err := m.MarshalTo(dAtA) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) if err != nil { return nil, err } @@ -4401,74 +4649,63 @@ func (m *ShardingGroup) Marshal() (dAtA []byte, err error) { } func (m *ShardingGroup) MarshalTo(dAtA []byte) (int, error) { - var i int + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *ShardingGroup) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) _ = i var l int _ = l - if len(m.Target) > 0 { - dAtA[i] = 0xa - i++ - i = encodeVarintDmworker(dAtA, i, uint64(len(m.Target))) - i += copy(dAtA[i:], m.Target) + if len(m.Unsynced) > 0 { + for iNdEx := len(m.Unsynced) - 1; iNdEx >= 0; iNdEx-- { + i -= len(m.Unsynced[iNdEx]) + copy(dAtA[i:], m.Unsynced[iNdEx]) + i = encodeVarintDmworker(dAtA, i, uint64(len(m.Unsynced[iNdEx]))) + i-- + dAtA[i] = 0x2a + } } - if len(m.DDLs) > 0 { - for _, s := range m.DDLs { - dAtA[i] = 0x12 - i++ - l = len(s) - for l >= 1<<7 { - dAtA[i] = uint8(uint64(l)&0x7f | 0x80) - l >>= 7 - i++ - } - dAtA[i] = uint8(l) - i++ - i += copy(dAtA[i:], s) + if len(m.Synced) > 0 { + for iNdEx := len(m.Synced) - 1; iNdEx >= 0; iNdEx-- { + i -= len(m.Synced[iNdEx]) + copy(dAtA[i:], m.Synced[iNdEx]) + i = encodeVarintDmworker(dAtA, i, uint64(len(m.Synced[iNdEx]))) + i-- + dAtA[i] = 0x22 } } if len(m.FirstPos) > 0 { - dAtA[i] = 0x1a - i++ + i -= len(m.FirstPos) + copy(dAtA[i:], m.FirstPos) i = encodeVarintDmworker(dAtA, i, uint64(len(m.FirstPos))) - i += copy(dAtA[i:], m.FirstPos) + i-- + dAtA[i] = 0x1a } - if len(m.Synced) > 0 { - for _, s := range m.Synced { - dAtA[i] = 0x22 - i++ - l = len(s) - for l >= 1<<7 { - dAtA[i] = uint8(uint64(l)&0x7f | 0x80) - l >>= 7 - i++ - } - dAtA[i] = uint8(l) - i++ - i += copy(dAtA[i:], s) + if len(m.DDLs) > 0 { + for iNdEx := len(m.DDLs) - 1; iNdEx >= 0; iNdEx-- { + i -= len(m.DDLs[iNdEx]) + copy(dAtA[i:], m.DDLs[iNdEx]) + i = encodeVarintDmworker(dAtA, i, uint64(len(m.DDLs[iNdEx]))) + i-- + dAtA[i] = 0x12 } } - if len(m.Unsynced) > 0 { - for _, s := range m.Unsynced { - dAtA[i] = 0x2a - i++ - l = len(s) - for l >= 1<<7 { - dAtA[i] = uint8(uint64(l)&0x7f | 0x80) - l >>= 7 - i++ - } - dAtA[i] = uint8(l) - i++ - i += copy(dAtA[i:], s) - } + if len(m.Target) > 0 { + i -= len(m.Target) + copy(dAtA[i:], m.Target) + i = encodeVarintDmworker(dAtA, i, uint64(len(m.Target))) + i-- + dAtA[i] = 0xa } - return i, nil + return len(dAtA) - i, nil } func (m *SyncStatus) Marshal() (dAtA []byte, err error) { size := m.Size() dAtA = make([]byte, size) - n, err := m.MarshalTo(dAtA) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) if err != nil { return nil, err } @@ -4476,93 +4713,98 @@ func (m *SyncStatus) Marshal() (dAtA []byte, err error) { } func (m *SyncStatus) MarshalTo(dAtA []byte) (int, error) { - var i int + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *SyncStatus) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) _ = i var l int _ = l - if m.TotalEvents != 0 { - dAtA[i] = 0x8 - i++ - i = encodeVarintDmworker(dAtA, i, uint64(m.TotalEvents)) - } - if m.TotalTps != 0 { - dAtA[i] = 0x10 - i++ - i = encodeVarintDmworker(dAtA, i, uint64(m.TotalTps)) + if m.Synced { + i-- + if m.Synced { + dAtA[i] = 1 + } else { + dAtA[i] = 0 + } + i-- + dAtA[i] = 0x50 } - if m.RecentTps != 0 { - dAtA[i] = 0x18 - i++ - i = encodeVarintDmworker(dAtA, i, uint64(m.RecentTps)) + if len(m.UnresolvedGroups) > 0 { + for iNdEx := len(m.UnresolvedGroups) - 1; iNdEx >= 0; iNdEx-- { + { + size, err := m.UnresolvedGroups[iNdEx].MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintDmworker(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x4a + } } - if len(m.MasterBinlog) > 0 { - dAtA[i] = 0x22 - i++ - i = encodeVarintDmworker(dAtA, i, uint64(len(m.MasterBinlog))) - i += copy(dAtA[i:], m.MasterBinlog) + if len(m.BlockingDDLs) > 0 { + for iNdEx := len(m.BlockingDDLs) - 1; iNdEx >= 0; iNdEx-- { + i -= len(m.BlockingDDLs[iNdEx]) + copy(dAtA[i:], m.BlockingDDLs[iNdEx]) + i = encodeVarintDmworker(dAtA, i, uint64(len(m.BlockingDDLs[iNdEx]))) + i-- + dAtA[i] = 0x42 + } } - if len(m.MasterBinlogGtid) > 0 { - dAtA[i] = 0x2a - i++ - i = encodeVarintDmworker(dAtA, i, uint64(len(m.MasterBinlogGtid))) - i += copy(dAtA[i:], m.MasterBinlogGtid) + if len(m.SyncerBinlogGtid) > 0 { + i -= len(m.SyncerBinlogGtid) + copy(dAtA[i:], m.SyncerBinlogGtid) + i = encodeVarintDmworker(dAtA, i, uint64(len(m.SyncerBinlogGtid))) + i-- + dAtA[i] = 0x3a } if len(m.SyncerBinlog) > 0 { - dAtA[i] = 0x32 - i++ + i -= len(m.SyncerBinlog) + copy(dAtA[i:], m.SyncerBinlog) i = encodeVarintDmworker(dAtA, i, uint64(len(m.SyncerBinlog))) - i += copy(dAtA[i:], m.SyncerBinlog) + i-- + dAtA[i] = 0x32 } - if len(m.SyncerBinlogGtid) > 0 { - dAtA[i] = 0x3a - i++ - i = encodeVarintDmworker(dAtA, i, uint64(len(m.SyncerBinlogGtid))) - i += copy(dAtA[i:], m.SyncerBinlogGtid) + if len(m.MasterBinlogGtid) > 0 { + i -= len(m.MasterBinlogGtid) + copy(dAtA[i:], m.MasterBinlogGtid) + i = encodeVarintDmworker(dAtA, i, uint64(len(m.MasterBinlogGtid))) + i-- + dAtA[i] = 0x2a } - if len(m.BlockingDDLs) > 0 { - for _, s := range m.BlockingDDLs { - dAtA[i] = 0x42 - i++ - l = len(s) - for l >= 1<<7 { - dAtA[i] = uint8(uint64(l)&0x7f | 0x80) - l >>= 7 - i++ - } - dAtA[i] = uint8(l) - i++ - i += copy(dAtA[i:], s) - } + if len(m.MasterBinlog) > 0 { + i -= len(m.MasterBinlog) + copy(dAtA[i:], m.MasterBinlog) + i = encodeVarintDmworker(dAtA, i, uint64(len(m.MasterBinlog))) + i-- + dAtA[i] = 0x22 } - if len(m.UnresolvedGroups) > 0 { - for _, msg := range m.UnresolvedGroups { - dAtA[i] = 0x4a - i++ - i = encodeVarintDmworker(dAtA, i, uint64(msg.Size())) - n, err := msg.MarshalTo(dAtA[i:]) - if err != nil { - return 0, err - } - i += n - } + if m.RecentTps != 0 { + i = encodeVarintDmworker(dAtA, i, uint64(m.RecentTps)) + i-- + dAtA[i] = 0x18 } - if m.Synced { - dAtA[i] = 0x50 - i++ - if m.Synced { - dAtA[i] = 1 - } else { - dAtA[i] = 0 - } - i++ + if m.TotalTps != 0 { + i = encodeVarintDmworker(dAtA, i, uint64(m.TotalTps)) + i-- + dAtA[i] = 0x10 + } + if m.TotalEvents != 0 { + i = encodeVarintDmworker(dAtA, i, uint64(m.TotalEvents)) + i-- + dAtA[i] = 0x8 } - return i, nil + return len(dAtA) - i, nil } func (m *RelayStatus) Marshal() (dAtA []byte, err error) { size := m.Size() dAtA = make([]byte, size) - n, err := m.MarshalTo(dAtA) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) if err != nil { return nil, err } @@ -4570,72 +4812,84 @@ func (m *RelayStatus) Marshal() (dAtA []byte, err error) { } func (m *RelayStatus) MarshalTo(dAtA []byte) (int, error) { - var i int + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *RelayStatus) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) _ = i var l int _ = l - if len(m.MasterBinlog) > 0 { - dAtA[i] = 0xa - i++ - i = encodeVarintDmworker(dAtA, i, uint64(len(m.MasterBinlog))) - i += copy(dAtA[i:], m.MasterBinlog) - } - if len(m.MasterBinlogGtid) > 0 { - dAtA[i] = 0x12 - i++ - i = encodeVarintDmworker(dAtA, i, uint64(len(m.MasterBinlogGtid))) - i += copy(dAtA[i:], m.MasterBinlogGtid) - } - if len(m.RelaySubDir) > 0 { - dAtA[i] = 0x1a - i++ - i = encodeVarintDmworker(dAtA, i, uint64(len(m.RelaySubDir))) - i += copy(dAtA[i:], m.RelaySubDir) - } - if len(m.RelayBinlog) > 0 { - dAtA[i] = 0x22 - i++ - i = encodeVarintDmworker(dAtA, i, uint64(len(m.RelayBinlog))) - i += copy(dAtA[i:], m.RelayBinlog) + if m.Result != nil { + { + size, err := m.Result.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintDmworker(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x42 } - if len(m.RelayBinlogGtid) > 0 { - dAtA[i] = 0x2a - i++ - i = encodeVarintDmworker(dAtA, i, uint64(len(m.RelayBinlogGtid))) - i += copy(dAtA[i:], m.RelayBinlogGtid) + if m.Stage != 0 { + i = encodeVarintDmworker(dAtA, i, uint64(m.Stage)) + i-- + dAtA[i] = 0x38 } if m.RelayCatchUpMaster { - dAtA[i] = 0x30 - i++ + i-- if m.RelayCatchUpMaster { dAtA[i] = 1 } else { dAtA[i] = 0 } - i++ + i-- + dAtA[i] = 0x30 } - if m.Stage != 0 { - dAtA[i] = 0x38 - i++ - i = encodeVarintDmworker(dAtA, i, uint64(m.Stage)) + if len(m.RelayBinlogGtid) > 0 { + i -= len(m.RelayBinlogGtid) + copy(dAtA[i:], m.RelayBinlogGtid) + i = encodeVarintDmworker(dAtA, i, uint64(len(m.RelayBinlogGtid))) + i-- + dAtA[i] = 0x2a } - if m.Result != nil { - dAtA[i] = 0x42 - i++ - i = encodeVarintDmworker(dAtA, i, uint64(m.Result.Size())) - n3, err := m.Result.MarshalTo(dAtA[i:]) - if err != nil { - return 0, err - } - i += n3 + if len(m.RelayBinlog) > 0 { + i -= len(m.RelayBinlog) + copy(dAtA[i:], m.RelayBinlog) + i = encodeVarintDmworker(dAtA, i, uint64(len(m.RelayBinlog))) + i-- + dAtA[i] = 0x22 } - return i, nil + if len(m.RelaySubDir) > 0 { + i -= len(m.RelaySubDir) + copy(dAtA[i:], m.RelaySubDir) + i = encodeVarintDmworker(dAtA, i, uint64(len(m.RelaySubDir))) + i-- + dAtA[i] = 0x1a + } + if len(m.MasterBinlogGtid) > 0 { + i -= len(m.MasterBinlogGtid) + copy(dAtA[i:], m.MasterBinlogGtid) + i = encodeVarintDmworker(dAtA, i, uint64(len(m.MasterBinlogGtid))) + i-- + dAtA[i] = 0x12 + } + if len(m.MasterBinlog) > 0 { + i -= len(m.MasterBinlog) + copy(dAtA[i:], m.MasterBinlog) + i = encodeVarintDmworker(dAtA, i, uint64(len(m.MasterBinlog))) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil } func (m *SubTaskStatus) Marshal() (dAtA []byte, err error) { size := m.Size() dAtA = make([]byte, size) - n, err := m.MarshalTo(dAtA) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) if err != nil { return nil, err } @@ -4643,120 +4897,165 @@ func (m *SubTaskStatus) Marshal() (dAtA []byte, err error) { } func (m *SubTaskStatus) MarshalTo(dAtA []byte) (int, error) { - var i int + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *SubTaskStatus) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) _ = i var l int _ = l - if len(m.Name) > 0 { - dAtA[i] = 0xa - i++ - i = encodeVarintDmworker(dAtA, i, uint64(len(m.Name))) - i += copy(dAtA[i:], m.Name) - } - if m.Stage != 0 { - dAtA[i] = 0x10 - i++ - i = encodeVarintDmworker(dAtA, i, uint64(m.Stage)) - } - if m.Unit != 0 { - dAtA[i] = 0x18 - i++ - i = encodeVarintDmworker(dAtA, i, uint64(m.Unit)) - } - if m.Result != nil { - dAtA[i] = 0x22 - i++ - i = encodeVarintDmworker(dAtA, i, uint64(m.Result.Size())) - n4, err := m.Result.MarshalTo(dAtA[i:]) - if err != nil { - return 0, err + if m.Status != nil { + { + size := m.Status.Size() + i -= size + if _, err := m.Status.MarshalTo(dAtA[i:]); err != nil { + return 0, err + } } - i += n4 } if len(m.UnresolvedDDLLockID) > 0 { - dAtA[i] = 0x2a - i++ + i -= len(m.UnresolvedDDLLockID) + copy(dAtA[i:], m.UnresolvedDDLLockID) i = encodeVarintDmworker(dAtA, i, uint64(len(m.UnresolvedDDLLockID))) - i += copy(dAtA[i:], m.UnresolvedDDLLockID) + i-- + dAtA[i] = 0x2a } - if m.Status != nil { - nn5, err := m.Status.MarshalTo(dAtA[i:]) - if err != nil { - return 0, err + if m.Result != nil { + { + size, err := m.Result.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintDmworker(dAtA, i, uint64(size)) } - i += nn5 + i-- + dAtA[i] = 0x22 } - return i, nil + if m.Unit != 0 { + i = encodeVarintDmworker(dAtA, i, uint64(m.Unit)) + i-- + dAtA[i] = 0x18 + } + if m.Stage != 0 { + i = encodeVarintDmworker(dAtA, i, uint64(m.Stage)) + i-- + dAtA[i] = 0x10 + } + if len(m.Name) > 0 { + i -= len(m.Name) + copy(dAtA[i:], m.Name) + i = encodeVarintDmworker(dAtA, i, uint64(len(m.Name))) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil } func (m *SubTaskStatus_Msg) MarshalTo(dAtA []byte) (int, error) { - i := 0 - dAtA[i] = 0x32 - i++ + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *SubTaskStatus_Msg) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + i -= len(m.Msg) + copy(dAtA[i:], m.Msg) i = encodeVarintDmworker(dAtA, i, uint64(len(m.Msg))) - i += copy(dAtA[i:], m.Msg) - return i, nil + i-- + dAtA[i] = 0x32 + return len(dAtA) - i, nil } func (m *SubTaskStatus_Check) MarshalTo(dAtA []byte) (int, error) { - i := 0 + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *SubTaskStatus_Check) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) if m.Check != nil { - dAtA[i] = 0x3a - i++ - i = encodeVarintDmworker(dAtA, i, uint64(m.Check.Size())) - n6, err := m.Check.MarshalTo(dAtA[i:]) - if err != nil { - return 0, err + { + size, err := m.Check.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintDmworker(dAtA, i, uint64(size)) } - i += n6 + i-- + dAtA[i] = 0x3a } - return i, nil + return len(dAtA) - i, nil } func (m *SubTaskStatus_Dump) MarshalTo(dAtA []byte) (int, error) { - i := 0 + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *SubTaskStatus_Dump) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) if m.Dump != nil { - dAtA[i] = 0x42 - i++ - i = encodeVarintDmworker(dAtA, i, uint64(m.Dump.Size())) - n7, err := m.Dump.MarshalTo(dAtA[i:]) - if err != nil { - return 0, err + { + size, err := m.Dump.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintDmworker(dAtA, i, uint64(size)) } - i += n7 + i-- + dAtA[i] = 0x42 } - return i, nil + return len(dAtA) - i, nil } func (m *SubTaskStatus_Load) MarshalTo(dAtA []byte) (int, error) { - i := 0 + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *SubTaskStatus_Load) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) if m.Load != nil { - dAtA[i] = 0x4a - i++ - i = encodeVarintDmworker(dAtA, i, uint64(m.Load.Size())) - n8, err := m.Load.MarshalTo(dAtA[i:]) - if err != nil { - return 0, err + { + size, err := m.Load.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintDmworker(dAtA, i, uint64(size)) } - i += n8 + i-- + dAtA[i] = 0x4a } - return i, nil + return len(dAtA) - i, nil } func (m *SubTaskStatus_Sync) MarshalTo(dAtA []byte) (int, error) { - i := 0 + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *SubTaskStatus_Sync) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) if m.Sync != nil { - dAtA[i] = 0x52 - i++ - i = encodeVarintDmworker(dAtA, i, uint64(m.Sync.Size())) - n9, err := m.Sync.MarshalTo(dAtA[i:]) - if err != nil { - return 0, err + { + size, err := m.Sync.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintDmworker(dAtA, i, uint64(size)) } - i += n9 + i-- + dAtA[i] = 0x52 } - return i, nil + return len(dAtA) - i, nil } func (m *SubTaskStatusList) Marshal() (dAtA []byte, err error) { size := m.Size() dAtA = make([]byte, size) - n, err := m.MarshalTo(dAtA) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) if err != nil { return nil, err } @@ -4764,29 +5063,36 @@ func (m *SubTaskStatusList) Marshal() (dAtA []byte, err error) { } func (m *SubTaskStatusList) MarshalTo(dAtA []byte) (int, error) { - var i int + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *SubTaskStatusList) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) _ = i var l int _ = l if len(m.Status) > 0 { - for _, msg := range m.Status { - dAtA[i] = 0xa - i++ - i = encodeVarintDmworker(dAtA, i, uint64(msg.Size())) - n, err := msg.MarshalTo(dAtA[i:]) - if err != nil { - return 0, err + for iNdEx := len(m.Status) - 1; iNdEx >= 0; iNdEx-- { + { + size, err := m.Status[iNdEx].MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintDmworker(dAtA, i, uint64(size)) } - i += n + i-- + dAtA[i] = 0xa } } - return i, nil + return len(dAtA) - i, nil } func (m *CheckError) Marshal() (dAtA []byte, err error) { size := m.Size() dAtA = make([]byte, size) - n, err := m.MarshalTo(dAtA) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) if err != nil { return nil, err } @@ -4794,23 +5100,29 @@ func (m *CheckError) Marshal() (dAtA []byte, err error) { } func (m *CheckError) MarshalTo(dAtA []byte) (int, error) { - var i int + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *CheckError) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) _ = i var l int _ = l if len(m.Msg) > 0 { - dAtA[i] = 0xa - i++ + i -= len(m.Msg) + copy(dAtA[i:], m.Msg) i = encodeVarintDmworker(dAtA, i, uint64(len(m.Msg))) - i += copy(dAtA[i:], m.Msg) + i-- + dAtA[i] = 0xa } - return i, nil + return len(dAtA) - i, nil } func (m *DumpError) Marshal() (dAtA []byte, err error) { size := m.Size() dAtA = make([]byte, size) - n, err := m.MarshalTo(dAtA) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) if err != nil { return nil, err } @@ -4818,23 +5130,29 @@ func (m *DumpError) Marshal() (dAtA []byte, err error) { } func (m *DumpError) MarshalTo(dAtA []byte) (int, error) { - var i int + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *DumpError) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) _ = i var l int _ = l if len(m.Msg) > 0 { - dAtA[i] = 0xa - i++ + i -= len(m.Msg) + copy(dAtA[i:], m.Msg) i = encodeVarintDmworker(dAtA, i, uint64(len(m.Msg))) - i += copy(dAtA[i:], m.Msg) + i-- + dAtA[i] = 0xa } - return i, nil + return len(dAtA) - i, nil } func (m *LoadError) Marshal() (dAtA []byte, err error) { size := m.Size() dAtA = make([]byte, size) - n, err := m.MarshalTo(dAtA) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) if err != nil { return nil, err } @@ -4842,23 +5160,29 @@ func (m *LoadError) Marshal() (dAtA []byte, err error) { } func (m *LoadError) MarshalTo(dAtA []byte) (int, error) { - var i int + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *LoadError) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) _ = i var l int _ = l if len(m.Msg) > 0 { - dAtA[i] = 0xa - i++ + i -= len(m.Msg) + copy(dAtA[i:], m.Msg) i = encodeVarintDmworker(dAtA, i, uint64(len(m.Msg))) - i += copy(dAtA[i:], m.Msg) + i-- + dAtA[i] = 0xa } - return i, nil + return len(dAtA) - i, nil } func (m *SyncSQLError) Marshal() (dAtA []byte, err error) { size := m.Size() dAtA = make([]byte, size) - n, err := m.MarshalTo(dAtA) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) if err != nil { return nil, err } @@ -4866,35 +5190,43 @@ func (m *SyncSQLError) Marshal() (dAtA []byte, err error) { } func (m *SyncSQLError) MarshalTo(dAtA []byte) (int, error) { - var i int + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *SyncSQLError) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) _ = i var l int _ = l - if len(m.Msg) > 0 { - dAtA[i] = 0xa - i++ - i = encodeVarintDmworker(dAtA, i, uint64(len(m.Msg))) - i += copy(dAtA[i:], m.Msg) + if len(m.ErrorSQL) > 0 { + i -= len(m.ErrorSQL) + copy(dAtA[i:], m.ErrorSQL) + i = encodeVarintDmworker(dAtA, i, uint64(len(m.ErrorSQL))) + i-- + dAtA[i] = 0x1a } if len(m.FailedBinlogPosition) > 0 { - dAtA[i] = 0x12 - i++ + i -= len(m.FailedBinlogPosition) + copy(dAtA[i:], m.FailedBinlogPosition) i = encodeVarintDmworker(dAtA, i, uint64(len(m.FailedBinlogPosition))) - i += copy(dAtA[i:], m.FailedBinlogPosition) + i-- + dAtA[i] = 0x12 } - if len(m.ErrorSQL) > 0 { - dAtA[i] = 0x1a - i++ - i = encodeVarintDmworker(dAtA, i, uint64(len(m.ErrorSQL))) - i += copy(dAtA[i:], m.ErrorSQL) + if len(m.Msg) > 0 { + i -= len(m.Msg) + copy(dAtA[i:], m.Msg) + i = encodeVarintDmworker(dAtA, i, uint64(len(m.Msg))) + i-- + dAtA[i] = 0xa } - return i, nil + return len(dAtA) - i, nil } func (m *SyncError) Marshal() (dAtA []byte, err error) { size := m.Size() dAtA = make([]byte, size) - n, err := m.MarshalTo(dAtA) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) if err != nil { return nil, err } @@ -4902,29 +5234,36 @@ func (m *SyncError) Marshal() (dAtA []byte, err error) { } func (m *SyncError) MarshalTo(dAtA []byte) (int, error) { - var i int + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *SyncError) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) _ = i var l int _ = l if len(m.Errors) > 0 { - for _, msg := range m.Errors { - dAtA[i] = 0xa - i++ - i = encodeVarintDmworker(dAtA, i, uint64(msg.Size())) - n, err := msg.MarshalTo(dAtA[i:]) - if err != nil { - return 0, err + for iNdEx := len(m.Errors) - 1; iNdEx >= 0; iNdEx-- { + { + size, err := m.Errors[iNdEx].MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintDmworker(dAtA, i, uint64(size)) } - i += n + i-- + dAtA[i] = 0xa } } - return i, nil + return len(dAtA) - i, nil } func (m *RelayError) Marshal() (dAtA []byte, err error) { size := m.Size() dAtA = make([]byte, size) - n, err := m.MarshalTo(dAtA) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) if err != nil { return nil, err } @@ -4932,23 +5271,29 @@ func (m *RelayError) Marshal() (dAtA []byte, err error) { } func (m *RelayError) MarshalTo(dAtA []byte) (int, error) { - var i int + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *RelayError) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) _ = i var l int _ = l if len(m.Msg) > 0 { - dAtA[i] = 0xa - i++ + i -= len(m.Msg) + copy(dAtA[i:], m.Msg) i = encodeVarintDmworker(dAtA, i, uint64(len(m.Msg))) - i += copy(dAtA[i:], m.Msg) + i-- + dAtA[i] = 0xa } - return i, nil + return len(dAtA) - i, nil } func (m *SubTaskError) Marshal() (dAtA []byte, err error) { size := m.Size() dAtA = make([]byte, size) - n, err := m.MarshalTo(dAtA) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) if err != nil { return nil, err } @@ -4956,104 +5301,146 @@ func (m *SubTaskError) Marshal() (dAtA []byte, err error) { } func (m *SubTaskError) MarshalTo(dAtA []byte) (int, error) { - var i int + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *SubTaskError) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) _ = i var l int _ = l - if len(m.Name) > 0 { - dAtA[i] = 0xa - i++ - i = encodeVarintDmworker(dAtA, i, uint64(len(m.Name))) - i += copy(dAtA[i:], m.Name) - } - if m.Stage != 0 { - dAtA[i] = 0x10 - i++ - i = encodeVarintDmworker(dAtA, i, uint64(m.Stage)) + if m.Error != nil { + { + size := m.Error.Size() + i -= size + if _, err := m.Error.MarshalTo(dAtA[i:]); err != nil { + return 0, err + } + } } if m.Unit != 0 { - dAtA[i] = 0x18 - i++ i = encodeVarintDmworker(dAtA, i, uint64(m.Unit)) + i-- + dAtA[i] = 0x18 } - if m.Error != nil { - nn10, err := m.Error.MarshalTo(dAtA[i:]) - if err != nil { - return 0, err - } - i += nn10 + if m.Stage != 0 { + i = encodeVarintDmworker(dAtA, i, uint64(m.Stage)) + i-- + dAtA[i] = 0x10 + } + if len(m.Name) > 0 { + i -= len(m.Name) + copy(dAtA[i:], m.Name) + i = encodeVarintDmworker(dAtA, i, uint64(len(m.Name))) + i-- + dAtA[i] = 0xa } - return i, nil + return len(dAtA) - i, nil } func (m *SubTaskError_Msg) MarshalTo(dAtA []byte) (int, error) { - i := 0 - dAtA[i] = 0x22 - i++ + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *SubTaskError_Msg) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + i -= len(m.Msg) + copy(dAtA[i:], m.Msg) i = encodeVarintDmworker(dAtA, i, uint64(len(m.Msg))) - i += copy(dAtA[i:], m.Msg) - return i, nil + i-- + dAtA[i] = 0x22 + return len(dAtA) - i, nil } func (m *SubTaskError_Check) MarshalTo(dAtA []byte) (int, error) { - i := 0 + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *SubTaskError_Check) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) if m.Check != nil { - dAtA[i] = 0x2a - i++ - i = encodeVarintDmworker(dAtA, i, uint64(m.Check.Size())) - n11, err := m.Check.MarshalTo(dAtA[i:]) - if err != nil { - return 0, err + { + size, err := m.Check.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintDmworker(dAtA, i, uint64(size)) } - i += n11 + i-- + dAtA[i] = 0x2a } - return i, nil + return len(dAtA) - i, nil } func (m *SubTaskError_Dump) MarshalTo(dAtA []byte) (int, error) { - i := 0 + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *SubTaskError_Dump) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) if m.Dump != nil { - dAtA[i] = 0x32 - i++ - i = encodeVarintDmworker(dAtA, i, uint64(m.Dump.Size())) - n12, err := m.Dump.MarshalTo(dAtA[i:]) - if err != nil { - return 0, err + { + size, err := m.Dump.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintDmworker(dAtA, i, uint64(size)) } - i += n12 + i-- + dAtA[i] = 0x32 } - return i, nil + return len(dAtA) - i, nil } func (m *SubTaskError_Load) MarshalTo(dAtA []byte) (int, error) { - i := 0 + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *SubTaskError_Load) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) if m.Load != nil { - dAtA[i] = 0x3a - i++ - i = encodeVarintDmworker(dAtA, i, uint64(m.Load.Size())) - n13, err := m.Load.MarshalTo(dAtA[i:]) - if err != nil { - return 0, err + { + size, err := m.Load.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintDmworker(dAtA, i, uint64(size)) } - i += n13 + i-- + dAtA[i] = 0x3a } - return i, nil + return len(dAtA) - i, nil } func (m *SubTaskError_Sync) MarshalTo(dAtA []byte) (int, error) { - i := 0 + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *SubTaskError_Sync) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) if m.Sync != nil { - dAtA[i] = 0x42 - i++ - i = encodeVarintDmworker(dAtA, i, uint64(m.Sync.Size())) - n14, err := m.Sync.MarshalTo(dAtA[i:]) - if err != nil { - return 0, err + { + size, err := m.Sync.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintDmworker(dAtA, i, uint64(size)) } - i += n14 + i-- + dAtA[i] = 0x42 } - return i, nil + return len(dAtA) - i, nil } func (m *SubTaskErrorList) Marshal() (dAtA []byte, err error) { size := m.Size() dAtA = make([]byte, size) - n, err := m.MarshalTo(dAtA) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) if err != nil { return nil, err } @@ -5061,29 +5448,36 @@ func (m *SubTaskErrorList) Marshal() (dAtA []byte, err error) { } func (m *SubTaskErrorList) MarshalTo(dAtA []byte) (int, error) { - var i int + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *SubTaskErrorList) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) _ = i var l int _ = l if len(m.Error) > 0 { - for _, msg := range m.Error { - dAtA[i] = 0xa - i++ - i = encodeVarintDmworker(dAtA, i, uint64(msg.Size())) - n, err := msg.MarshalTo(dAtA[i:]) - if err != nil { - return 0, err + for iNdEx := len(m.Error) - 1; iNdEx >= 0; iNdEx-- { + { + size, err := m.Error[iNdEx].MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintDmworker(dAtA, i, uint64(size)) } - i += n + i-- + dAtA[i] = 0xa } } - return i, nil + return len(dAtA) - i, nil } func (m *ProcessResult) Marshal() (dAtA []byte, err error) { size := m.Size() dAtA = make([]byte, size) - n, err := m.MarshalTo(dAtA) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) if err != nil { return nil, err } @@ -5091,45 +5485,53 @@ func (m *ProcessResult) Marshal() (dAtA []byte, err error) { } func (m *ProcessResult) MarshalTo(dAtA []byte) (int, error) { - var i int + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *ProcessResult) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) _ = i var l int _ = l + if len(m.Detail) > 0 { + i -= len(m.Detail) + copy(dAtA[i:], m.Detail) + i = encodeVarintDmworker(dAtA, i, uint64(len(m.Detail))) + i-- + dAtA[i] = 0x1a + } + if len(m.Errors) > 0 { + for iNdEx := len(m.Errors) - 1; iNdEx >= 0; iNdEx-- { + { + size, err := m.Errors[iNdEx].MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintDmworker(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x12 + } + } if m.IsCanceled { - dAtA[i] = 0x8 - i++ + i-- if m.IsCanceled { dAtA[i] = 1 } else { dAtA[i] = 0 } - i++ - } - if len(m.Errors) > 0 { - for _, msg := range m.Errors { - dAtA[i] = 0x12 - i++ - i = encodeVarintDmworker(dAtA, i, uint64(msg.Size())) - n, err := msg.MarshalTo(dAtA[i:]) - if err != nil { - return 0, err - } - i += n - } - } - if len(m.Detail) > 0 { - dAtA[i] = 0x1a - i++ - i = encodeVarintDmworker(dAtA, i, uint64(len(m.Detail))) - i += copy(dAtA[i:], m.Detail) + i-- + dAtA[i] = 0x8 } - return i, nil + return len(dAtA) - i, nil } func (m *TError) Marshal() (dAtA []byte, err error) { size := m.Size() dAtA = make([]byte, size) - n, err := m.MarshalTo(dAtA) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) if err != nil { return nil, err } @@ -5137,49 +5539,56 @@ func (m *TError) Marshal() (dAtA []byte, err error) { } func (m *TError) MarshalTo(dAtA []byte) (int, error) { - var i int + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *TError) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) _ = i var l int _ = l - if m.ErrCode != 0 { - dAtA[i] = 0x8 - i++ - i = encodeVarintDmworker(dAtA, i, uint64(m.ErrCode)) - } - if m.ErrClass != 0 { - dAtA[i] = 0x10 - i++ - i = encodeVarintDmworker(dAtA, i, uint64(m.ErrClass)) + if len(m.RawCause) > 0 { + i -= len(m.RawCause) + copy(dAtA[i:], m.RawCause) + i = encodeVarintDmworker(dAtA, i, uint64(len(m.RawCause))) + i-- + dAtA[i] = 0x32 } - if m.ErrScope != 0 { - dAtA[i] = 0x18 - i++ - i = encodeVarintDmworker(dAtA, i, uint64(m.ErrScope)) + if len(m.Message) > 0 { + i -= len(m.Message) + copy(dAtA[i:], m.Message) + i = encodeVarintDmworker(dAtA, i, uint64(len(m.Message))) + i-- + dAtA[i] = 0x2a } if m.ErrLevel != 0 { - dAtA[i] = 0x20 - i++ i = encodeVarintDmworker(dAtA, i, uint64(m.ErrLevel)) + i-- + dAtA[i] = 0x20 } - if len(m.Message) > 0 { - dAtA[i] = 0x2a - i++ - i = encodeVarintDmworker(dAtA, i, uint64(len(m.Message))) - i += copy(dAtA[i:], m.Message) + if m.ErrScope != 0 { + i = encodeVarintDmworker(dAtA, i, uint64(m.ErrScope)) + i-- + dAtA[i] = 0x18 } - if len(m.RawCause) > 0 { - dAtA[i] = 0x32 - i++ - i = encodeVarintDmworker(dAtA, i, uint64(len(m.RawCause))) - i += copy(dAtA[i:], m.RawCause) + if m.ErrClass != 0 { + i = encodeVarintDmworker(dAtA, i, uint64(m.ErrClass)) + i-- + dAtA[i] = 0x10 + } + if m.ErrCode != 0 { + i = encodeVarintDmworker(dAtA, i, uint64(m.ErrCode)) + i-- + dAtA[i] = 0x8 } - return i, nil + return len(dAtA) - i, nil } func (m *ProcessError) Marshal() (dAtA []byte, err error) { size := m.Size() dAtA = make([]byte, size) - n, err := m.MarshalTo(dAtA) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) if err != nil { return nil, err } @@ -5187,38 +5596,46 @@ func (m *ProcessError) Marshal() (dAtA []byte, err error) { } func (m *ProcessError) MarshalTo(dAtA []byte) (int, error) { - var i int + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *ProcessError) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) _ = i var l int _ = l - if m.Type != 0 { - dAtA[i] = 0x8 - i++ - i = encodeVarintDmworker(dAtA, i, uint64(m.Type)) + if m.Error != nil { + { + size, err := m.Error.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintDmworker(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x1a } if len(m.Msg) > 0 { - dAtA[i] = 0x12 - i++ + i -= len(m.Msg) + copy(dAtA[i:], m.Msg) i = encodeVarintDmworker(dAtA, i, uint64(len(m.Msg))) - i += copy(dAtA[i:], m.Msg) + i-- + dAtA[i] = 0x12 } - if m.Error != nil { - dAtA[i] = 0x1a - i++ - i = encodeVarintDmworker(dAtA, i, uint64(m.Error.Size())) - n15, err := m.Error.MarshalTo(dAtA[i:]) - if err != nil { - return 0, err - } - i += n15 + if m.Type != 0 { + i = encodeVarintDmworker(dAtA, i, uint64(m.Type)) + i-- + dAtA[i] = 0x8 } - return i, nil + return len(dAtA) - i, nil } func (m *DDLInfo) Marshal() (dAtA []byte, err error) { size := m.Size() dAtA = make([]byte, size) - n, err := m.MarshalTo(dAtA) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) if err != nil { return nil, err } @@ -5226,50 +5643,52 @@ func (m *DDLInfo) Marshal() (dAtA []byte, err error) { } func (m *DDLInfo) MarshalTo(dAtA []byte) (int, error) { - var i int + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *DDLInfo) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) _ = i var l int _ = l - if len(m.Task) > 0 { - dAtA[i] = 0xa - i++ - i = encodeVarintDmworker(dAtA, i, uint64(len(m.Task))) - i += copy(dAtA[i:], m.Task) - } - if len(m.Schema) > 0 { - dAtA[i] = 0x12 - i++ - i = encodeVarintDmworker(dAtA, i, uint64(len(m.Schema))) - i += copy(dAtA[i:], m.Schema) + if len(m.DDLs) > 0 { + for iNdEx := len(m.DDLs) - 1; iNdEx >= 0; iNdEx-- { + i -= len(m.DDLs[iNdEx]) + copy(dAtA[i:], m.DDLs[iNdEx]) + i = encodeVarintDmworker(dAtA, i, uint64(len(m.DDLs[iNdEx]))) + i-- + dAtA[i] = 0x22 + } } if len(m.Table) > 0 { - dAtA[i] = 0x1a - i++ + i -= len(m.Table) + copy(dAtA[i:], m.Table) i = encodeVarintDmworker(dAtA, i, uint64(len(m.Table))) - i += copy(dAtA[i:], m.Table) + i-- + dAtA[i] = 0x1a } - if len(m.DDLs) > 0 { - for _, s := range m.DDLs { - dAtA[i] = 0x22 - i++ - l = len(s) - for l >= 1<<7 { - dAtA[i] = uint8(uint64(l)&0x7f | 0x80) - l >>= 7 - i++ - } - dAtA[i] = uint8(l) - i++ - i += copy(dAtA[i:], s) - } + if len(m.Schema) > 0 { + i -= len(m.Schema) + copy(dAtA[i:], m.Schema) + i = encodeVarintDmworker(dAtA, i, uint64(len(m.Schema))) + i-- + dAtA[i] = 0x12 + } + if len(m.Task) > 0 { + i -= len(m.Task) + copy(dAtA[i:], m.Task) + i = encodeVarintDmworker(dAtA, i, uint64(len(m.Task))) + i-- + dAtA[i] = 0xa } - return i, nil + return len(dAtA) - i, nil } func (m *DDLLockInfo) Marshal() (dAtA []byte, err error) { size := m.Size() dAtA = make([]byte, size) - n, err := m.MarshalTo(dAtA) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) if err != nil { return nil, err } @@ -5277,29 +5696,36 @@ func (m *DDLLockInfo) Marshal() (dAtA []byte, err error) { } func (m *DDLLockInfo) MarshalTo(dAtA []byte) (int, error) { - var i int + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *DDLLockInfo) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) _ = i var l int _ = l - if len(m.Task) > 0 { - dAtA[i] = 0xa - i++ - i = encodeVarintDmworker(dAtA, i, uint64(len(m.Task))) - i += copy(dAtA[i:], m.Task) - } if len(m.ID) > 0 { - dAtA[i] = 0x12 - i++ + i -= len(m.ID) + copy(dAtA[i:], m.ID) i = encodeVarintDmworker(dAtA, i, uint64(len(m.ID))) - i += copy(dAtA[i:], m.ID) + i-- + dAtA[i] = 0x12 + } + if len(m.Task) > 0 { + i -= len(m.Task) + copy(dAtA[i:], m.Task) + i = encodeVarintDmworker(dAtA, i, uint64(len(m.Task))) + i-- + dAtA[i] = 0xa } - return i, nil + return len(dAtA) - i, nil } func (m *ExecDDLRequest) Marshal() (dAtA []byte, err error) { size := m.Size() dAtA = make([]byte, size) - n, err := m.MarshalTo(dAtA) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) if err != nil { return nil, err } @@ -5307,60 +5733,62 @@ func (m *ExecDDLRequest) Marshal() (dAtA []byte, err error) { } func (m *ExecDDLRequest) MarshalTo(dAtA []byte) (int, error) { - var i int + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *ExecDDLRequest) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) _ = i var l int _ = l - if len(m.Task) > 0 { - dAtA[i] = 0xa - i++ - i = encodeVarintDmworker(dAtA, i, uint64(len(m.Task))) - i += copy(dAtA[i:], m.Task) + if len(m.DDLs) > 0 { + for iNdEx := len(m.DDLs) - 1; iNdEx >= 0; iNdEx-- { + i -= len(m.DDLs[iNdEx]) + copy(dAtA[i:], m.DDLs[iNdEx]) + i = encodeVarintDmworker(dAtA, i, uint64(len(m.DDLs[iNdEx]))) + i-- + dAtA[i] = 0x2a + } } - if len(m.LockID) > 0 { - dAtA[i] = 0x12 - i++ - i = encodeVarintDmworker(dAtA, i, uint64(len(m.LockID))) - i += copy(dAtA[i:], m.LockID) + if len(m.TraceGID) > 0 { + i -= len(m.TraceGID) + copy(dAtA[i:], m.TraceGID) + i = encodeVarintDmworker(dAtA, i, uint64(len(m.TraceGID))) + i-- + dAtA[i] = 0x22 } if m.Exec { - dAtA[i] = 0x18 - i++ + i-- if m.Exec { dAtA[i] = 1 } else { dAtA[i] = 0 } - i++ + i-- + dAtA[i] = 0x18 } - if len(m.TraceGID) > 0 { - dAtA[i] = 0x22 - i++ - i = encodeVarintDmworker(dAtA, i, uint64(len(m.TraceGID))) - i += copy(dAtA[i:], m.TraceGID) + if len(m.LockID) > 0 { + i -= len(m.LockID) + copy(dAtA[i:], m.LockID) + i = encodeVarintDmworker(dAtA, i, uint64(len(m.LockID))) + i-- + dAtA[i] = 0x12 } - if len(m.DDLs) > 0 { - for _, s := range m.DDLs { - dAtA[i] = 0x2a - i++ - l = len(s) - for l >= 1<<7 { - dAtA[i] = uint8(uint64(l)&0x7f | 0x80) - l >>= 7 - i++ - } - dAtA[i] = uint8(l) - i++ - i += copy(dAtA[i:], s) - } + if len(m.Task) > 0 { + i -= len(m.Task) + copy(dAtA[i:], m.Task) + i = encodeVarintDmworker(dAtA, i, uint64(len(m.Task))) + i-- + dAtA[i] = 0xa } - return i, nil + return len(dAtA) - i, nil } func (m *BreakDDLLockRequest) Marshal() (dAtA []byte, err error) { size := m.Size() dAtA = make([]byte, size) - n, err := m.MarshalTo(dAtA) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) if err != nil { return nil, err } @@ -5368,49 +5796,56 @@ func (m *BreakDDLLockRequest) Marshal() (dAtA []byte, err error) { } func (m *BreakDDLLockRequest) MarshalTo(dAtA []byte) (int, error) { - var i int + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *BreakDDLLockRequest) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) _ = i var l int _ = l - if len(m.Task) > 0 { - dAtA[i] = 0xa - i++ - i = encodeVarintDmworker(dAtA, i, uint64(len(m.Task))) - i += copy(dAtA[i:], m.Task) - } - if len(m.RemoveLockID) > 0 { - dAtA[i] = 0x12 - i++ - i = encodeVarintDmworker(dAtA, i, uint64(len(m.RemoveLockID))) - i += copy(dAtA[i:], m.RemoveLockID) - } - if m.ExecDDL { - dAtA[i] = 0x18 - i++ - if m.ExecDDL { + if m.SkipDDL { + i-- + if m.SkipDDL { dAtA[i] = 1 } else { dAtA[i] = 0 } - i++ - } - if m.SkipDDL { + i-- dAtA[i] = 0x20 - i++ - if m.SkipDDL { + } + if m.ExecDDL { + i-- + if m.ExecDDL { dAtA[i] = 1 } else { dAtA[i] = 0 } - i++ + i-- + dAtA[i] = 0x18 + } + if len(m.RemoveLockID) > 0 { + i -= len(m.RemoveLockID) + copy(dAtA[i:], m.RemoveLockID) + i = encodeVarintDmworker(dAtA, i, uint64(len(m.RemoveLockID))) + i-- + dAtA[i] = 0x12 + } + if len(m.Task) > 0 { + i -= len(m.Task) + copy(dAtA[i:], m.Task) + i = encodeVarintDmworker(dAtA, i, uint64(len(m.Task))) + i-- + dAtA[i] = 0xa } - return i, nil + return len(dAtA) - i, nil } func (m *SwitchRelayMasterRequest) Marshal() (dAtA []byte, err error) { size := m.Size() dAtA = make([]byte, size) - n, err := m.MarshalTo(dAtA) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) if err != nil { return nil, err } @@ -5418,17 +5853,22 @@ func (m *SwitchRelayMasterRequest) Marshal() (dAtA []byte, err error) { } func (m *SwitchRelayMasterRequest) MarshalTo(dAtA []byte) (int, error) { - var i int + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *SwitchRelayMasterRequest) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) _ = i var l int _ = l - return i, nil + return len(dAtA) - i, nil } func (m *OperateRelayRequest) Marshal() (dAtA []byte, err error) { size := m.Size() dAtA = make([]byte, size) - n, err := m.MarshalTo(dAtA) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) if err != nil { return nil, err } @@ -5436,22 +5876,27 @@ func (m *OperateRelayRequest) Marshal() (dAtA []byte, err error) { } func (m *OperateRelayRequest) MarshalTo(dAtA []byte) (int, error) { - var i int + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *OperateRelayRequest) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) _ = i var l int _ = l if m.Op != 0 { - dAtA[i] = 0x8 - i++ i = encodeVarintDmworker(dAtA, i, uint64(m.Op)) + i-- + dAtA[i] = 0x8 } - return i, nil + return len(dAtA) - i, nil } func (m *OperateRelayResponse) Marshal() (dAtA []byte, err error) { size := m.Size() dAtA = make([]byte, size) - n, err := m.MarshalTo(dAtA) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) if err != nil { return nil, err } @@ -5459,44 +5904,51 @@ func (m *OperateRelayResponse) Marshal() (dAtA []byte, err error) { } func (m *OperateRelayResponse) MarshalTo(dAtA []byte) (int, error) { - var i int + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *OperateRelayResponse) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) _ = i var l int _ = l - if m.Op != 0 { - dAtA[i] = 0x8 - i++ - i = encodeVarintDmworker(dAtA, i, uint64(m.Op)) + if len(m.Msg) > 0 { + i -= len(m.Msg) + copy(dAtA[i:], m.Msg) + i = encodeVarintDmworker(dAtA, i, uint64(len(m.Msg))) + i-- + dAtA[i] = 0x22 + } + if len(m.Worker) > 0 { + i -= len(m.Worker) + copy(dAtA[i:], m.Worker) + i = encodeVarintDmworker(dAtA, i, uint64(len(m.Worker))) + i-- + dAtA[i] = 0x1a } if m.Result { - dAtA[i] = 0x10 - i++ + i-- if m.Result { dAtA[i] = 1 } else { dAtA[i] = 0 } - i++ - } - if len(m.Worker) > 0 { - dAtA[i] = 0x1a - i++ - i = encodeVarintDmworker(dAtA, i, uint64(len(m.Worker))) - i += copy(dAtA[i:], m.Worker) + i-- + dAtA[i] = 0x10 } - if len(m.Msg) > 0 { - dAtA[i] = 0x22 - i++ - i = encodeVarintDmworker(dAtA, i, uint64(len(m.Msg))) - i += copy(dAtA[i:], m.Msg) + if m.Op != 0 { + i = encodeVarintDmworker(dAtA, i, uint64(m.Op)) + i-- + dAtA[i] = 0x8 } - return i, nil + return len(dAtA) - i, nil } func (m *PurgeRelayRequest) Marshal() (dAtA []byte, err error) { size := m.Size() dAtA = make([]byte, size) - n, err := m.MarshalTo(dAtA) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) if err != nil { return nil, err } @@ -5504,44 +5956,51 @@ func (m *PurgeRelayRequest) Marshal() (dAtA []byte, err error) { } func (m *PurgeRelayRequest) MarshalTo(dAtA []byte) (int, error) { - var i int + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *PurgeRelayRequest) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) _ = i var l int _ = l + if len(m.SubDir) > 0 { + i -= len(m.SubDir) + copy(dAtA[i:], m.SubDir) + i = encodeVarintDmworker(dAtA, i, uint64(len(m.SubDir))) + i-- + dAtA[i] = 0x22 + } + if len(m.Filename) > 0 { + i -= len(m.Filename) + copy(dAtA[i:], m.Filename) + i = encodeVarintDmworker(dAtA, i, uint64(len(m.Filename))) + i-- + dAtA[i] = 0x1a + } + if m.Time != 0 { + i = encodeVarintDmworker(dAtA, i, uint64(m.Time)) + i-- + dAtA[i] = 0x10 + } if m.Inactive { - dAtA[i] = 0x8 - i++ + i-- if m.Inactive { dAtA[i] = 1 } else { dAtA[i] = 0 } - i++ - } - if m.Time != 0 { - dAtA[i] = 0x10 - i++ - i = encodeVarintDmworker(dAtA, i, uint64(m.Time)) - } - if len(m.Filename) > 0 { - dAtA[i] = 0x1a - i++ - i = encodeVarintDmworker(dAtA, i, uint64(len(m.Filename))) - i += copy(dAtA[i:], m.Filename) - } - if len(m.SubDir) > 0 { - dAtA[i] = 0x22 - i++ - i = encodeVarintDmworker(dAtA, i, uint64(len(m.SubDir))) - i += copy(dAtA[i:], m.SubDir) + i-- + dAtA[i] = 0x8 } - return i, nil + return len(dAtA) - i, nil } func (m *QueryWorkerConfigRequest) Marshal() (dAtA []byte, err error) { size := m.Size() dAtA = make([]byte, size) - n, err := m.MarshalTo(dAtA) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) if err != nil { return nil, err } @@ -5549,17 +6008,22 @@ func (m *QueryWorkerConfigRequest) Marshal() (dAtA []byte, err error) { } func (m *QueryWorkerConfigRequest) MarshalTo(dAtA []byte) (int, error) { - var i int + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *QueryWorkerConfigRequest) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) _ = i var l int _ = l - return i, nil + return len(dAtA) - i, nil } func (m *QueryWorkerConfigResponse) Marshal() (dAtA []byte, err error) { size := m.Size() dAtA = make([]byte, size) - n, err := m.MarshalTo(dAtA) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) if err != nil { return nil, err } @@ -5567,55 +6031,141 @@ func (m *QueryWorkerConfigResponse) Marshal() (dAtA []byte, err error) { } func (m *QueryWorkerConfigResponse) MarshalTo(dAtA []byte) (int, error) { - var i int + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *QueryWorkerConfigResponse) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) _ = i var l int _ = l + if len(m.Content) > 0 { + i -= len(m.Content) + copy(dAtA[i:], m.Content) + i = encodeVarintDmworker(dAtA, i, uint64(len(m.Content))) + i-- + dAtA[i] = 0x2a + } + if len(m.SourceID) > 0 { + i -= len(m.SourceID) + copy(dAtA[i:], m.SourceID) + i = encodeVarintDmworker(dAtA, i, uint64(len(m.SourceID))) + i-- + dAtA[i] = 0x22 + } + if len(m.Msg) > 0 { + i -= len(m.Msg) + copy(dAtA[i:], m.Msg) + i = encodeVarintDmworker(dAtA, i, uint64(len(m.Msg))) + i-- + dAtA[i] = 0x1a + } + if len(m.Worker) > 0 { + i -= len(m.Worker) + copy(dAtA[i:], m.Worker) + i = encodeVarintDmworker(dAtA, i, uint64(len(m.Worker))) + i-- + dAtA[i] = 0x12 + } if m.Result { - dAtA[i] = 0x8 - i++ + i-- if m.Result { dAtA[i] = 1 } else { dAtA[i] = 0 } - i++ + i-- + dAtA[i] = 0x8 } - if len(m.Worker) > 0 { + return len(dAtA) - i, nil +} + +func (m *MysqlTaskRequest) 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 *MysqlTaskRequest) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *MysqlTaskRequest) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + _ = i + var l int + _ = l + if len(m.Config) > 0 { + i -= len(m.Config) + copy(dAtA[i:], m.Config) + i = encodeVarintDmworker(dAtA, i, uint64(len(m.Config))) + i-- dAtA[i] = 0x12 - i++ - i = encodeVarintDmworker(dAtA, i, uint64(len(m.Worker))) - i += copy(dAtA[i:], m.Worker) } + if m.Op != 0 { + i = encodeVarintDmworker(dAtA, i, uint64(m.Op)) + i-- + dAtA[i] = 0x8 + } + return len(dAtA) - i, nil +} + +func (m *MysqlTaskResponse) 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 *MysqlTaskResponse) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *MysqlTaskResponse) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + _ = i + var l int + _ = l if len(m.Msg) > 0 { - dAtA[i] = 0x1a - i++ + i -= len(m.Msg) + copy(dAtA[i:], m.Msg) i = encodeVarintDmworker(dAtA, i, uint64(len(m.Msg))) - i += copy(dAtA[i:], m.Msg) - } - if len(m.SourceID) > 0 { - dAtA[i] = 0x22 - i++ - i = encodeVarintDmworker(dAtA, i, uint64(len(m.SourceID))) - i += copy(dAtA[i:], m.SourceID) + i-- + dAtA[i] = 0x12 } - if len(m.Content) > 0 { - dAtA[i] = 0x2a - i++ - i = encodeVarintDmworker(dAtA, i, uint64(len(m.Content))) - i += copy(dAtA[i:], m.Content) + if m.Result { + i-- + if m.Result { + dAtA[i] = 1 + } else { + dAtA[i] = 0 + } + i-- + dAtA[i] = 0x8 } - return i, nil + return len(dAtA) - i, nil } func encodeVarintDmworker(dAtA []byte, offset int, v uint64) int { + offset -= sovDmworker(v) + base := offset for v >= 1<<7 { dAtA[offset] = uint8(v&0x7f | 0x80) v >>= 7 offset++ } dAtA[offset] = uint8(v) - return offset + 1 + return base } func (m *StartSubTaskRequest) Size() (n int) { if m == nil { @@ -6592,16 +7142,41 @@ func (m *QueryWorkerConfigResponse) Size() (n int) { return n } -func sovDmworker(x uint64) (n int) { - for { - n++ - x >>= 7 - if x == 0 { - break - } +func (m *MysqlTaskRequest) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Op != 0 { + n += 1 + sovDmworker(uint64(m.Op)) + } + l = len(m.Config) + if l > 0 { + n += 1 + l + sovDmworker(uint64(l)) + } + return n +} + +func (m *MysqlTaskResponse) 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 + sovDmworker(uint64(l)) } return n } + +func sovDmworker(x uint64) (n int) { + return (math_bits.Len64(x|1) + 6) / 7 +} func sozDmworker(x uint64) (n int) { return sovDmworker(uint64((x << 1) ^ uint64((int64(x) >> 63)))) } @@ -6620,7 +7195,7 @@ func (m *StartSubTaskRequest) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - wire |= (uint64(b) & 0x7F) << shift + wire |= uint64(b&0x7F) << shift if b < 0x80 { break } @@ -6648,7 +7223,7 @@ func (m *StartSubTaskRequest) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= (uint64(b) & 0x7F) << shift + stringLen |= uint64(b&0x7F) << shift if b < 0x80 { break } @@ -6658,6 +7233,9 @@ func (m *StartSubTaskRequest) Unmarshal(dAtA []byte) error { return ErrInvalidLengthDmworker } postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthDmworker + } if postIndex > l { return io.ErrUnexpectedEOF } @@ -6672,6 +7250,9 @@ func (m *StartSubTaskRequest) Unmarshal(dAtA []byte) error { if skippy < 0 { return ErrInvalidLengthDmworker } + if (iNdEx + skippy) < 0 { + return ErrInvalidLengthDmworker + } if (iNdEx + skippy) > l { return io.ErrUnexpectedEOF } @@ -6699,7 +7280,7 @@ func (m *UpdateRelayRequest) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - wire |= (uint64(b) & 0x7F) << shift + wire |= uint64(b&0x7F) << shift if b < 0x80 { break } @@ -6727,7 +7308,7 @@ func (m *UpdateRelayRequest) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= (uint64(b) & 0x7F) << shift + stringLen |= uint64(b&0x7F) << shift if b < 0x80 { break } @@ -6737,6 +7318,9 @@ func (m *UpdateRelayRequest) Unmarshal(dAtA []byte) error { return ErrInvalidLengthDmworker } postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthDmworker + } if postIndex > l { return io.ErrUnexpectedEOF } @@ -6751,6 +7335,9 @@ func (m *UpdateRelayRequest) Unmarshal(dAtA []byte) error { if skippy < 0 { return ErrInvalidLengthDmworker } + if (iNdEx + skippy) < 0 { + return ErrInvalidLengthDmworker + } if (iNdEx + skippy) > l { return io.ErrUnexpectedEOF } @@ -6778,7 +7365,7 @@ func (m *MigrateRelayRequest) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - wire |= (uint64(b) & 0x7F) << shift + wire |= uint64(b&0x7F) << shift if b < 0x80 { break } @@ -6806,7 +7393,7 @@ func (m *MigrateRelayRequest) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= (uint64(b) & 0x7F) << shift + stringLen |= uint64(b&0x7F) << shift if b < 0x80 { break } @@ -6816,6 +7403,9 @@ func (m *MigrateRelayRequest) Unmarshal(dAtA []byte) error { return ErrInvalidLengthDmworker } postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthDmworker + } if postIndex > l { return io.ErrUnexpectedEOF } @@ -6835,7 +7425,7 @@ func (m *MigrateRelayRequest) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - m.BinlogPos |= (uint32(b) & 0x7F) << shift + m.BinlogPos |= uint32(b&0x7F) << shift if b < 0x80 { break } @@ -6849,6 +7439,9 @@ func (m *MigrateRelayRequest) Unmarshal(dAtA []byte) error { if skippy < 0 { return ErrInvalidLengthDmworker } + if (iNdEx + skippy) < 0 { + return ErrInvalidLengthDmworker + } if (iNdEx + skippy) > l { return io.ErrUnexpectedEOF } @@ -6876,7 +7469,7 @@ func (m *OperateSubTaskRequest) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - wire |= (uint64(b) & 0x7F) << shift + wire |= uint64(b&0x7F) << shift if b < 0x80 { break } @@ -6904,7 +7497,7 @@ func (m *OperateSubTaskRequest) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - m.Op |= (TaskOp(b) & 0x7F) << shift + m.Op |= TaskOp(b&0x7F) << shift if b < 0x80 { break } @@ -6923,7 +7516,7 @@ func (m *OperateSubTaskRequest) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= (uint64(b) & 0x7F) << shift + stringLen |= uint64(b&0x7F) << shift if b < 0x80 { break } @@ -6933,6 +7526,9 @@ func (m *OperateSubTaskRequest) Unmarshal(dAtA []byte) error { return ErrInvalidLengthDmworker } postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthDmworker + } if postIndex > l { return io.ErrUnexpectedEOF } @@ -6947,6 +7543,9 @@ func (m *OperateSubTaskRequest) Unmarshal(dAtA []byte) error { if skippy < 0 { return ErrInvalidLengthDmworker } + if (iNdEx + skippy) < 0 { + return ErrInvalidLengthDmworker + } if (iNdEx + skippy) > l { return io.ErrUnexpectedEOF } @@ -6974,7 +7573,7 @@ func (m *OperateSubTaskResponse) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - wire |= (uint64(b) & 0x7F) << shift + wire |= uint64(b&0x7F) << shift if b < 0x80 { break } @@ -7002,7 +7601,7 @@ func (m *OperateSubTaskResponse) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - m.Op |= (TaskOp(b) & 0x7F) << shift + m.Op |= TaskOp(b&0x7F) << shift if b < 0x80 { break } @@ -7021,7 +7620,7 @@ func (m *OperateSubTaskResponse) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - v |= (int(b) & 0x7F) << shift + v |= int(b&0x7F) << shift if b < 0x80 { break } @@ -7041,7 +7640,7 @@ func (m *OperateSubTaskResponse) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= (uint64(b) & 0x7F) << shift + stringLen |= uint64(b&0x7F) << shift if b < 0x80 { break } @@ -7051,6 +7650,9 @@ func (m *OperateSubTaskResponse) Unmarshal(dAtA []byte) error { return ErrInvalidLengthDmworker } postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthDmworker + } if postIndex > l { return io.ErrUnexpectedEOF } @@ -7070,7 +7672,7 @@ func (m *OperateSubTaskResponse) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= (uint64(b) & 0x7F) << shift + stringLen |= uint64(b&0x7F) << shift if b < 0x80 { break } @@ -7080,6 +7682,9 @@ func (m *OperateSubTaskResponse) Unmarshal(dAtA []byte) error { return ErrInvalidLengthDmworker } postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthDmworker + } if postIndex > l { return io.ErrUnexpectedEOF } @@ -7094,6 +7699,9 @@ func (m *OperateSubTaskResponse) Unmarshal(dAtA []byte) error { if skippy < 0 { return ErrInvalidLengthDmworker } + if (iNdEx + skippy) < 0 { + return ErrInvalidLengthDmworker + } if (iNdEx + skippy) > l { return io.ErrUnexpectedEOF } @@ -7121,7 +7729,7 @@ func (m *UpdateSubTaskRequest) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - wire |= (uint64(b) & 0x7F) << shift + wire |= uint64(b&0x7F) << shift if b < 0x80 { break } @@ -7149,7 +7757,7 @@ func (m *UpdateSubTaskRequest) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= (uint64(b) & 0x7F) << shift + stringLen |= uint64(b&0x7F) << shift if b < 0x80 { break } @@ -7159,6 +7767,9 @@ func (m *UpdateSubTaskRequest) Unmarshal(dAtA []byte) error { return ErrInvalidLengthDmworker } postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthDmworker + } if postIndex > l { return io.ErrUnexpectedEOF } @@ -7173,6 +7784,9 @@ func (m *UpdateSubTaskRequest) Unmarshal(dAtA []byte) error { if skippy < 0 { return ErrInvalidLengthDmworker } + if (iNdEx + skippy) < 0 { + return ErrInvalidLengthDmworker + } if (iNdEx + skippy) > l { return io.ErrUnexpectedEOF } @@ -7200,7 +7814,7 @@ func (m *QueryStatusRequest) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - wire |= (uint64(b) & 0x7F) << shift + wire |= uint64(b&0x7F) << shift if b < 0x80 { break } @@ -7228,7 +7842,7 @@ func (m *QueryStatusRequest) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= (uint64(b) & 0x7F) << shift + stringLen |= uint64(b&0x7F) << shift if b < 0x80 { break } @@ -7238,6 +7852,9 @@ func (m *QueryStatusRequest) Unmarshal(dAtA []byte) error { return ErrInvalidLengthDmworker } postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthDmworker + } if postIndex > l { return io.ErrUnexpectedEOF } @@ -7252,6 +7869,9 @@ func (m *QueryStatusRequest) Unmarshal(dAtA []byte) error { if skippy < 0 { return ErrInvalidLengthDmworker } + if (iNdEx + skippy) < 0 { + return ErrInvalidLengthDmworker + } if (iNdEx + skippy) > l { return io.ErrUnexpectedEOF } @@ -7279,7 +7899,7 @@ func (m *QueryErrorRequest) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - wire |= (uint64(b) & 0x7F) << shift + wire |= uint64(b&0x7F) << shift if b < 0x80 { break } @@ -7307,7 +7927,7 @@ func (m *QueryErrorRequest) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= (uint64(b) & 0x7F) << shift + stringLen |= uint64(b&0x7F) << shift if b < 0x80 { break } @@ -7317,6 +7937,9 @@ func (m *QueryErrorRequest) Unmarshal(dAtA []byte) error { return ErrInvalidLengthDmworker } postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthDmworker + } if postIndex > l { return io.ErrUnexpectedEOF } @@ -7331,6 +7954,9 @@ func (m *QueryErrorRequest) Unmarshal(dAtA []byte) error { if skippy < 0 { return ErrInvalidLengthDmworker } + if (iNdEx + skippy) < 0 { + return ErrInvalidLengthDmworker + } if (iNdEx + skippy) > l { return io.ErrUnexpectedEOF } @@ -7358,7 +7984,7 @@ func (m *HandleSubTaskSQLsRequest) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - wire |= (uint64(b) & 0x7F) << shift + wire |= uint64(b&0x7F) << shift if b < 0x80 { break } @@ -7386,7 +8012,7 @@ func (m *HandleSubTaskSQLsRequest) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= (uint64(b) & 0x7F) << shift + stringLen |= uint64(b&0x7F) << shift if b < 0x80 { break } @@ -7396,6 +8022,9 @@ func (m *HandleSubTaskSQLsRequest) Unmarshal(dAtA []byte) error { return ErrInvalidLengthDmworker } postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthDmworker + } if postIndex > l { return io.ErrUnexpectedEOF } @@ -7415,7 +8044,7 @@ func (m *HandleSubTaskSQLsRequest) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - m.Op |= (SQLOp(b) & 0x7F) << shift + m.Op |= SQLOp(b&0x7F) << shift if b < 0x80 { break } @@ -7434,7 +8063,7 @@ func (m *HandleSubTaskSQLsRequest) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= (uint64(b) & 0x7F) << shift + stringLen |= uint64(b&0x7F) << shift if b < 0x80 { break } @@ -7444,6 +8073,9 @@ func (m *HandleSubTaskSQLsRequest) Unmarshal(dAtA []byte) error { return ErrInvalidLengthDmworker } postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthDmworker + } if postIndex > l { return io.ErrUnexpectedEOF } @@ -7463,7 +8095,7 @@ func (m *HandleSubTaskSQLsRequest) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= (uint64(b) & 0x7F) << shift + stringLen |= uint64(b&0x7F) << shift if b < 0x80 { break } @@ -7473,6 +8105,9 @@ func (m *HandleSubTaskSQLsRequest) Unmarshal(dAtA []byte) error { return ErrInvalidLengthDmworker } postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthDmworker + } if postIndex > l { return io.ErrUnexpectedEOF } @@ -7492,7 +8127,7 @@ func (m *HandleSubTaskSQLsRequest) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= (uint64(b) & 0x7F) << shift + stringLen |= uint64(b&0x7F) << shift if b < 0x80 { break } @@ -7502,6 +8137,9 @@ func (m *HandleSubTaskSQLsRequest) Unmarshal(dAtA []byte) error { return ErrInvalidLengthDmworker } postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthDmworker + } if postIndex > l { return io.ErrUnexpectedEOF } @@ -7516,6 +8154,9 @@ func (m *HandleSubTaskSQLsRequest) Unmarshal(dAtA []byte) error { if skippy < 0 { return ErrInvalidLengthDmworker } + if (iNdEx + skippy) < 0 { + return ErrInvalidLengthDmworker + } if (iNdEx + skippy) > l { return io.ErrUnexpectedEOF } @@ -7543,7 +8184,7 @@ func (m *CommonWorkerResponse) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - wire |= (uint64(b) & 0x7F) << shift + wire |= uint64(b&0x7F) << shift if b < 0x80 { break } @@ -7571,7 +8212,7 @@ func (m *CommonWorkerResponse) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - v |= (int(b) & 0x7F) << shift + v |= int(b&0x7F) << shift if b < 0x80 { break } @@ -7591,7 +8232,7 @@ func (m *CommonWorkerResponse) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= (uint64(b) & 0x7F) << shift + stringLen |= uint64(b&0x7F) << shift if b < 0x80 { break } @@ -7601,6 +8242,9 @@ func (m *CommonWorkerResponse) Unmarshal(dAtA []byte) error { return ErrInvalidLengthDmworker } postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthDmworker + } if postIndex > l { return io.ErrUnexpectedEOF } @@ -7620,7 +8264,7 @@ func (m *CommonWorkerResponse) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= (uint64(b) & 0x7F) << shift + stringLen |= uint64(b&0x7F) << shift if b < 0x80 { break } @@ -7630,6 +8274,9 @@ func (m *CommonWorkerResponse) Unmarshal(dAtA []byte) error { return ErrInvalidLengthDmworker } postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthDmworker + } if postIndex > l { return io.ErrUnexpectedEOF } @@ -7644,6 +8291,9 @@ func (m *CommonWorkerResponse) Unmarshal(dAtA []byte) error { if skippy < 0 { return ErrInvalidLengthDmworker } + if (iNdEx + skippy) < 0 { + return ErrInvalidLengthDmworker + } if (iNdEx + skippy) > l { return io.ErrUnexpectedEOF } @@ -7671,7 +8321,7 @@ func (m *QueryStatusResponse) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - wire |= (uint64(b) & 0x7F) << shift + wire |= uint64(b&0x7F) << shift if b < 0x80 { break } @@ -7699,7 +8349,7 @@ func (m *QueryStatusResponse) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - v |= (int(b) & 0x7F) << shift + v |= int(b&0x7F) << shift if b < 0x80 { break } @@ -7719,7 +8369,7 @@ func (m *QueryStatusResponse) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= (uint64(b) & 0x7F) << shift + stringLen |= uint64(b&0x7F) << shift if b < 0x80 { break } @@ -7729,6 +8379,9 @@ func (m *QueryStatusResponse) Unmarshal(dAtA []byte) error { return ErrInvalidLengthDmworker } postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthDmworker + } if postIndex > l { return io.ErrUnexpectedEOF } @@ -7748,7 +8401,7 @@ func (m *QueryStatusResponse) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= (uint64(b) & 0x7F) << shift + stringLen |= uint64(b&0x7F) << shift if b < 0x80 { break } @@ -7758,6 +8411,9 @@ func (m *QueryStatusResponse) Unmarshal(dAtA []byte) error { return ErrInvalidLengthDmworker } postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthDmworker + } if postIndex > l { return io.ErrUnexpectedEOF } @@ -7777,7 +8433,7 @@ func (m *QueryStatusResponse) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - msglen |= (int(b) & 0x7F) << shift + msglen |= int(b&0x7F) << shift if b < 0x80 { break } @@ -7786,6 +8442,9 @@ func (m *QueryStatusResponse) Unmarshal(dAtA []byte) error { return ErrInvalidLengthDmworker } postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthDmworker + } if postIndex > l { return io.ErrUnexpectedEOF } @@ -7808,7 +8467,7 @@ func (m *QueryStatusResponse) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - msglen |= (int(b) & 0x7F) << shift + msglen |= int(b&0x7F) << shift if b < 0x80 { break } @@ -7817,6 +8476,9 @@ func (m *QueryStatusResponse) Unmarshal(dAtA []byte) error { return ErrInvalidLengthDmworker } postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthDmworker + } if postIndex > l { return io.ErrUnexpectedEOF } @@ -7841,7 +8503,7 @@ func (m *QueryStatusResponse) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= (uint64(b) & 0x7F) << shift + stringLen |= uint64(b&0x7F) << shift if b < 0x80 { break } @@ -7851,6 +8513,9 @@ func (m *QueryStatusResponse) Unmarshal(dAtA []byte) error { return ErrInvalidLengthDmworker } postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthDmworker + } if postIndex > l { return io.ErrUnexpectedEOF } @@ -7865,6 +8530,9 @@ func (m *QueryStatusResponse) Unmarshal(dAtA []byte) error { if skippy < 0 { return ErrInvalidLengthDmworker } + if (iNdEx + skippy) < 0 { + return ErrInvalidLengthDmworker + } if (iNdEx + skippy) > l { return io.ErrUnexpectedEOF } @@ -7892,7 +8560,7 @@ func (m *QueryErrorResponse) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - wire |= (uint64(b) & 0x7F) << shift + wire |= uint64(b&0x7F) << shift if b < 0x80 { break } @@ -7920,7 +8588,7 @@ func (m *QueryErrorResponse) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - v |= (int(b) & 0x7F) << shift + v |= int(b&0x7F) << shift if b < 0x80 { break } @@ -7940,7 +8608,7 @@ func (m *QueryErrorResponse) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= (uint64(b) & 0x7F) << shift + stringLen |= uint64(b&0x7F) << shift if b < 0x80 { break } @@ -7950,6 +8618,9 @@ func (m *QueryErrorResponse) Unmarshal(dAtA []byte) error { return ErrInvalidLengthDmworker } postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthDmworker + } if postIndex > l { return io.ErrUnexpectedEOF } @@ -7969,7 +8640,7 @@ func (m *QueryErrorResponse) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= (uint64(b) & 0x7F) << shift + stringLen |= uint64(b&0x7F) << shift if b < 0x80 { break } @@ -7979,6 +8650,9 @@ func (m *QueryErrorResponse) Unmarshal(dAtA []byte) error { return ErrInvalidLengthDmworker } postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthDmworker + } if postIndex > l { return io.ErrUnexpectedEOF } @@ -7998,7 +8672,7 @@ func (m *QueryErrorResponse) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - msglen |= (int(b) & 0x7F) << shift + msglen |= int(b&0x7F) << shift if b < 0x80 { break } @@ -8007,6 +8681,9 @@ func (m *QueryErrorResponse) Unmarshal(dAtA []byte) error { return ErrInvalidLengthDmworker } postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthDmworker + } if postIndex > l { return io.ErrUnexpectedEOF } @@ -8029,7 +8706,7 @@ func (m *QueryErrorResponse) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - msglen |= (int(b) & 0x7F) << shift + msglen |= int(b&0x7F) << shift if b < 0x80 { break } @@ -8038,6 +8715,9 @@ func (m *QueryErrorResponse) Unmarshal(dAtA []byte) error { return ErrInvalidLengthDmworker } postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthDmworker + } if postIndex > l { return io.ErrUnexpectedEOF } @@ -8057,6 +8737,9 @@ func (m *QueryErrorResponse) Unmarshal(dAtA []byte) error { if skippy < 0 { return ErrInvalidLengthDmworker } + if (iNdEx + skippy) < 0 { + return ErrInvalidLengthDmworker + } if (iNdEx + skippy) > l { return io.ErrUnexpectedEOF } @@ -8084,7 +8767,7 @@ func (m *CheckStatus) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - wire |= (uint64(b) & 0x7F) << shift + wire |= uint64(b&0x7F) << shift if b < 0x80 { break } @@ -8112,7 +8795,7 @@ func (m *CheckStatus) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - v |= (int(b) & 0x7F) << shift + v |= int(b&0x7F) << shift if b < 0x80 { break } @@ -8132,7 +8815,7 @@ func (m *CheckStatus) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - m.Total |= (int32(b) & 0x7F) << shift + m.Total |= int32(b&0x7F) << shift if b < 0x80 { break } @@ -8151,7 +8834,7 @@ func (m *CheckStatus) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - m.Successful |= (int32(b) & 0x7F) << shift + m.Successful |= int32(b&0x7F) << shift if b < 0x80 { break } @@ -8170,7 +8853,7 @@ func (m *CheckStatus) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - m.Failed |= (int32(b) & 0x7F) << shift + m.Failed |= int32(b&0x7F) << shift if b < 0x80 { break } @@ -8189,7 +8872,7 @@ func (m *CheckStatus) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - m.Warning |= (int32(b) & 0x7F) << shift + m.Warning |= int32(b&0x7F) << shift if b < 0x80 { break } @@ -8208,7 +8891,7 @@ func (m *CheckStatus) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - byteLen |= (int(b) & 0x7F) << shift + byteLen |= int(b&0x7F) << shift if b < 0x80 { break } @@ -8217,6 +8900,9 @@ func (m *CheckStatus) Unmarshal(dAtA []byte) error { return ErrInvalidLengthDmworker } postIndex := iNdEx + byteLen + if postIndex < 0 { + return ErrInvalidLengthDmworker + } if postIndex > l { return io.ErrUnexpectedEOF } @@ -8234,6 +8920,9 @@ func (m *CheckStatus) Unmarshal(dAtA []byte) error { if skippy < 0 { return ErrInvalidLengthDmworker } + if (iNdEx + skippy) < 0 { + return ErrInvalidLengthDmworker + } if (iNdEx + skippy) > l { return io.ErrUnexpectedEOF } @@ -8261,7 +8950,7 @@ func (m *DumpStatus) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - wire |= (uint64(b) & 0x7F) << shift + wire |= uint64(b&0x7F) << shift if b < 0x80 { break } @@ -8284,6 +8973,9 @@ func (m *DumpStatus) Unmarshal(dAtA []byte) error { if skippy < 0 { return ErrInvalidLengthDmworker } + if (iNdEx + skippy) < 0 { + return ErrInvalidLengthDmworker + } if (iNdEx + skippy) > l { return io.ErrUnexpectedEOF } @@ -8311,7 +9003,7 @@ func (m *LoadStatus) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - wire |= (uint64(b) & 0x7F) << shift + wire |= uint64(b&0x7F) << shift if b < 0x80 { break } @@ -8339,7 +9031,7 @@ func (m *LoadStatus) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - m.FinishedBytes |= (int64(b) & 0x7F) << shift + m.FinishedBytes |= int64(b&0x7F) << shift if b < 0x80 { break } @@ -8358,7 +9050,7 @@ func (m *LoadStatus) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - m.TotalBytes |= (int64(b) & 0x7F) << shift + m.TotalBytes |= int64(b&0x7F) << shift if b < 0x80 { break } @@ -8377,7 +9069,7 @@ func (m *LoadStatus) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= (uint64(b) & 0x7F) << shift + stringLen |= uint64(b&0x7F) << shift if b < 0x80 { break } @@ -8387,6 +9079,9 @@ func (m *LoadStatus) Unmarshal(dAtA []byte) error { return ErrInvalidLengthDmworker } postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthDmworker + } if postIndex > l { return io.ErrUnexpectedEOF } @@ -8406,7 +9101,7 @@ func (m *LoadStatus) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= (uint64(b) & 0x7F) << shift + stringLen |= uint64(b&0x7F) << shift if b < 0x80 { break } @@ -8416,6 +9111,9 @@ func (m *LoadStatus) Unmarshal(dAtA []byte) error { return ErrInvalidLengthDmworker } postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthDmworker + } if postIndex > l { return io.ErrUnexpectedEOF } @@ -8430,6 +9128,9 @@ func (m *LoadStatus) Unmarshal(dAtA []byte) error { if skippy < 0 { return ErrInvalidLengthDmworker } + if (iNdEx + skippy) < 0 { + return ErrInvalidLengthDmworker + } if (iNdEx + skippy) > l { return io.ErrUnexpectedEOF } @@ -8457,7 +9158,7 @@ func (m *ShardingGroup) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - wire |= (uint64(b) & 0x7F) << shift + wire |= uint64(b&0x7F) << shift if b < 0x80 { break } @@ -8485,7 +9186,7 @@ func (m *ShardingGroup) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= (uint64(b) & 0x7F) << shift + stringLen |= uint64(b&0x7F) << shift if b < 0x80 { break } @@ -8495,6 +9196,9 @@ func (m *ShardingGroup) Unmarshal(dAtA []byte) error { return ErrInvalidLengthDmworker } postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthDmworker + } if postIndex > l { return io.ErrUnexpectedEOF } @@ -8514,7 +9218,7 @@ func (m *ShardingGroup) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= (uint64(b) & 0x7F) << shift + stringLen |= uint64(b&0x7F) << shift if b < 0x80 { break } @@ -8524,6 +9228,9 @@ func (m *ShardingGroup) Unmarshal(dAtA []byte) error { return ErrInvalidLengthDmworker } postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthDmworker + } if postIndex > l { return io.ErrUnexpectedEOF } @@ -8543,7 +9250,7 @@ func (m *ShardingGroup) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= (uint64(b) & 0x7F) << shift + stringLen |= uint64(b&0x7F) << shift if b < 0x80 { break } @@ -8553,6 +9260,9 @@ func (m *ShardingGroup) Unmarshal(dAtA []byte) error { return ErrInvalidLengthDmworker } postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthDmworker + } if postIndex > l { return io.ErrUnexpectedEOF } @@ -8572,7 +9282,7 @@ func (m *ShardingGroup) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= (uint64(b) & 0x7F) << shift + stringLen |= uint64(b&0x7F) << shift if b < 0x80 { break } @@ -8582,6 +9292,9 @@ func (m *ShardingGroup) Unmarshal(dAtA []byte) error { return ErrInvalidLengthDmworker } postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthDmworker + } if postIndex > l { return io.ErrUnexpectedEOF } @@ -8601,7 +9314,7 @@ func (m *ShardingGroup) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= (uint64(b) & 0x7F) << shift + stringLen |= uint64(b&0x7F) << shift if b < 0x80 { break } @@ -8611,6 +9324,9 @@ func (m *ShardingGroup) Unmarshal(dAtA []byte) error { return ErrInvalidLengthDmworker } postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthDmworker + } if postIndex > l { return io.ErrUnexpectedEOF } @@ -8625,6 +9341,9 @@ func (m *ShardingGroup) Unmarshal(dAtA []byte) error { if skippy < 0 { return ErrInvalidLengthDmworker } + if (iNdEx + skippy) < 0 { + return ErrInvalidLengthDmworker + } if (iNdEx + skippy) > l { return io.ErrUnexpectedEOF } @@ -8652,7 +9371,7 @@ func (m *SyncStatus) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - wire |= (uint64(b) & 0x7F) << shift + wire |= uint64(b&0x7F) << shift if b < 0x80 { break } @@ -8680,7 +9399,7 @@ func (m *SyncStatus) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - m.TotalEvents |= (int64(b) & 0x7F) << shift + m.TotalEvents |= int64(b&0x7F) << shift if b < 0x80 { break } @@ -8699,7 +9418,7 @@ func (m *SyncStatus) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - m.TotalTps |= (int64(b) & 0x7F) << shift + m.TotalTps |= int64(b&0x7F) << shift if b < 0x80 { break } @@ -8718,7 +9437,7 @@ func (m *SyncStatus) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - m.RecentTps |= (int64(b) & 0x7F) << shift + m.RecentTps |= int64(b&0x7F) << shift if b < 0x80 { break } @@ -8737,7 +9456,7 @@ func (m *SyncStatus) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= (uint64(b) & 0x7F) << shift + stringLen |= uint64(b&0x7F) << shift if b < 0x80 { break } @@ -8747,6 +9466,9 @@ func (m *SyncStatus) Unmarshal(dAtA []byte) error { return ErrInvalidLengthDmworker } postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthDmworker + } if postIndex > l { return io.ErrUnexpectedEOF } @@ -8766,7 +9488,7 @@ func (m *SyncStatus) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= (uint64(b) & 0x7F) << shift + stringLen |= uint64(b&0x7F) << shift if b < 0x80 { break } @@ -8776,6 +9498,9 @@ func (m *SyncStatus) Unmarshal(dAtA []byte) error { return ErrInvalidLengthDmworker } postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthDmworker + } if postIndex > l { return io.ErrUnexpectedEOF } @@ -8795,7 +9520,7 @@ func (m *SyncStatus) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= (uint64(b) & 0x7F) << shift + stringLen |= uint64(b&0x7F) << shift if b < 0x80 { break } @@ -8805,6 +9530,9 @@ func (m *SyncStatus) Unmarshal(dAtA []byte) error { return ErrInvalidLengthDmworker } postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthDmworker + } if postIndex > l { return io.ErrUnexpectedEOF } @@ -8824,7 +9552,7 @@ func (m *SyncStatus) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= (uint64(b) & 0x7F) << shift + stringLen |= uint64(b&0x7F) << shift if b < 0x80 { break } @@ -8834,6 +9562,9 @@ func (m *SyncStatus) Unmarshal(dAtA []byte) error { return ErrInvalidLengthDmworker } postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthDmworker + } if postIndex > l { return io.ErrUnexpectedEOF } @@ -8853,7 +9584,7 @@ func (m *SyncStatus) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= (uint64(b) & 0x7F) << shift + stringLen |= uint64(b&0x7F) << shift if b < 0x80 { break } @@ -8863,6 +9594,9 @@ func (m *SyncStatus) Unmarshal(dAtA []byte) error { return ErrInvalidLengthDmworker } postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthDmworker + } if postIndex > l { return io.ErrUnexpectedEOF } @@ -8882,7 +9616,7 @@ func (m *SyncStatus) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - msglen |= (int(b) & 0x7F) << shift + msglen |= int(b&0x7F) << shift if b < 0x80 { break } @@ -8891,6 +9625,9 @@ func (m *SyncStatus) Unmarshal(dAtA []byte) error { return ErrInvalidLengthDmworker } postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthDmworker + } if postIndex > l { return io.ErrUnexpectedEOF } @@ -8913,7 +9650,7 @@ func (m *SyncStatus) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - v |= (int(b) & 0x7F) << shift + v |= int(b&0x7F) << shift if b < 0x80 { break } @@ -8928,6 +9665,9 @@ func (m *SyncStatus) Unmarshal(dAtA []byte) error { if skippy < 0 { return ErrInvalidLengthDmworker } + if (iNdEx + skippy) < 0 { + return ErrInvalidLengthDmworker + } if (iNdEx + skippy) > l { return io.ErrUnexpectedEOF } @@ -8955,7 +9695,7 @@ func (m *RelayStatus) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - wire |= (uint64(b) & 0x7F) << shift + wire |= uint64(b&0x7F) << shift if b < 0x80 { break } @@ -8983,7 +9723,7 @@ func (m *RelayStatus) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= (uint64(b) & 0x7F) << shift + stringLen |= uint64(b&0x7F) << shift if b < 0x80 { break } @@ -8993,6 +9733,9 @@ func (m *RelayStatus) Unmarshal(dAtA []byte) error { return ErrInvalidLengthDmworker } postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthDmworker + } if postIndex > l { return io.ErrUnexpectedEOF } @@ -9012,7 +9755,7 @@ func (m *RelayStatus) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= (uint64(b) & 0x7F) << shift + stringLen |= uint64(b&0x7F) << shift if b < 0x80 { break } @@ -9022,6 +9765,9 @@ func (m *RelayStatus) Unmarshal(dAtA []byte) error { return ErrInvalidLengthDmworker } postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthDmworker + } if postIndex > l { return io.ErrUnexpectedEOF } @@ -9041,7 +9787,7 @@ func (m *RelayStatus) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= (uint64(b) & 0x7F) << shift + stringLen |= uint64(b&0x7F) << shift if b < 0x80 { break } @@ -9051,6 +9797,9 @@ func (m *RelayStatus) Unmarshal(dAtA []byte) error { return ErrInvalidLengthDmworker } postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthDmworker + } if postIndex > l { return io.ErrUnexpectedEOF } @@ -9070,7 +9819,7 @@ func (m *RelayStatus) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= (uint64(b) & 0x7F) << shift + stringLen |= uint64(b&0x7F) << shift if b < 0x80 { break } @@ -9080,6 +9829,9 @@ func (m *RelayStatus) Unmarshal(dAtA []byte) error { return ErrInvalidLengthDmworker } postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthDmworker + } if postIndex > l { return io.ErrUnexpectedEOF } @@ -9099,7 +9851,7 @@ func (m *RelayStatus) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= (uint64(b) & 0x7F) << shift + stringLen |= uint64(b&0x7F) << shift if b < 0x80 { break } @@ -9109,6 +9861,9 @@ func (m *RelayStatus) Unmarshal(dAtA []byte) error { return ErrInvalidLengthDmworker } postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthDmworker + } if postIndex > l { return io.ErrUnexpectedEOF } @@ -9128,7 +9883,7 @@ func (m *RelayStatus) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - v |= (int(b) & 0x7F) << shift + v |= int(b&0x7F) << shift if b < 0x80 { break } @@ -9148,7 +9903,7 @@ func (m *RelayStatus) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - m.Stage |= (Stage(b) & 0x7F) << shift + m.Stage |= Stage(b&0x7F) << shift if b < 0x80 { break } @@ -9167,7 +9922,7 @@ func (m *RelayStatus) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - msglen |= (int(b) & 0x7F) << shift + msglen |= int(b&0x7F) << shift if b < 0x80 { break } @@ -9176,6 +9931,9 @@ func (m *RelayStatus) Unmarshal(dAtA []byte) error { return ErrInvalidLengthDmworker } postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthDmworker + } if postIndex > l { return io.ErrUnexpectedEOF } @@ -9195,6 +9953,9 @@ func (m *RelayStatus) Unmarshal(dAtA []byte) error { if skippy < 0 { return ErrInvalidLengthDmworker } + if (iNdEx + skippy) < 0 { + return ErrInvalidLengthDmworker + } if (iNdEx + skippy) > l { return io.ErrUnexpectedEOF } @@ -9222,7 +9983,7 @@ func (m *SubTaskStatus) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - wire |= (uint64(b) & 0x7F) << shift + wire |= uint64(b&0x7F) << shift if b < 0x80 { break } @@ -9250,7 +10011,7 @@ func (m *SubTaskStatus) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= (uint64(b) & 0x7F) << shift + stringLen |= uint64(b&0x7F) << shift if b < 0x80 { break } @@ -9260,6 +10021,9 @@ func (m *SubTaskStatus) Unmarshal(dAtA []byte) error { return ErrInvalidLengthDmworker } postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthDmworker + } if postIndex > l { return io.ErrUnexpectedEOF } @@ -9279,7 +10043,7 @@ func (m *SubTaskStatus) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - m.Stage |= (Stage(b) & 0x7F) << shift + m.Stage |= Stage(b&0x7F) << shift if b < 0x80 { break } @@ -9298,7 +10062,7 @@ func (m *SubTaskStatus) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - m.Unit |= (UnitType(b) & 0x7F) << shift + m.Unit |= UnitType(b&0x7F) << shift if b < 0x80 { break } @@ -9317,7 +10081,7 @@ func (m *SubTaskStatus) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - msglen |= (int(b) & 0x7F) << shift + msglen |= int(b&0x7F) << shift if b < 0x80 { break } @@ -9326,6 +10090,9 @@ func (m *SubTaskStatus) Unmarshal(dAtA []byte) error { return ErrInvalidLengthDmworker } postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthDmworker + } if postIndex > l { return io.ErrUnexpectedEOF } @@ -9350,7 +10117,7 @@ func (m *SubTaskStatus) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= (uint64(b) & 0x7F) << shift + stringLen |= uint64(b&0x7F) << shift if b < 0x80 { break } @@ -9360,6 +10127,9 @@ func (m *SubTaskStatus) Unmarshal(dAtA []byte) error { return ErrInvalidLengthDmworker } postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthDmworker + } if postIndex > l { return io.ErrUnexpectedEOF } @@ -9379,7 +10149,7 @@ func (m *SubTaskStatus) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= (uint64(b) & 0x7F) << shift + stringLen |= uint64(b&0x7F) << shift if b < 0x80 { break } @@ -9389,6 +10159,9 @@ func (m *SubTaskStatus) Unmarshal(dAtA []byte) error { return ErrInvalidLengthDmworker } postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthDmworker + } if postIndex > l { return io.ErrUnexpectedEOF } @@ -9408,7 +10181,7 @@ func (m *SubTaskStatus) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - msglen |= (int(b) & 0x7F) << shift + msglen |= int(b&0x7F) << shift if b < 0x80 { break } @@ -9417,6 +10190,9 @@ func (m *SubTaskStatus) Unmarshal(dAtA []byte) error { return ErrInvalidLengthDmworker } postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthDmworker + } if postIndex > l { return io.ErrUnexpectedEOF } @@ -9440,7 +10216,7 @@ func (m *SubTaskStatus) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - msglen |= (int(b) & 0x7F) << shift + msglen |= int(b&0x7F) << shift if b < 0x80 { break } @@ -9449,6 +10225,9 @@ func (m *SubTaskStatus) Unmarshal(dAtA []byte) error { return ErrInvalidLengthDmworker } postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthDmworker + } if postIndex > l { return io.ErrUnexpectedEOF } @@ -9472,7 +10251,7 @@ func (m *SubTaskStatus) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - msglen |= (int(b) & 0x7F) << shift + msglen |= int(b&0x7F) << shift if b < 0x80 { break } @@ -9481,6 +10260,9 @@ func (m *SubTaskStatus) Unmarshal(dAtA []byte) error { return ErrInvalidLengthDmworker } postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthDmworker + } if postIndex > l { return io.ErrUnexpectedEOF } @@ -9504,7 +10286,7 @@ func (m *SubTaskStatus) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - msglen |= (int(b) & 0x7F) << shift + msglen |= int(b&0x7F) << shift if b < 0x80 { break } @@ -9513,6 +10295,9 @@ func (m *SubTaskStatus) Unmarshal(dAtA []byte) error { return ErrInvalidLengthDmworker } postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthDmworker + } if postIndex > l { return io.ErrUnexpectedEOF } @@ -9531,6 +10316,9 @@ func (m *SubTaskStatus) Unmarshal(dAtA []byte) error { if skippy < 0 { return ErrInvalidLengthDmworker } + if (iNdEx + skippy) < 0 { + return ErrInvalidLengthDmworker + } if (iNdEx + skippy) > l { return io.ErrUnexpectedEOF } @@ -9558,7 +10346,7 @@ func (m *SubTaskStatusList) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - wire |= (uint64(b) & 0x7F) << shift + wire |= uint64(b&0x7F) << shift if b < 0x80 { break } @@ -9586,7 +10374,7 @@ func (m *SubTaskStatusList) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - msglen |= (int(b) & 0x7F) << shift + msglen |= int(b&0x7F) << shift if b < 0x80 { break } @@ -9595,6 +10383,9 @@ func (m *SubTaskStatusList) Unmarshal(dAtA []byte) error { return ErrInvalidLengthDmworker } postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthDmworker + } if postIndex > l { return io.ErrUnexpectedEOF } @@ -9612,6 +10403,9 @@ func (m *SubTaskStatusList) Unmarshal(dAtA []byte) error { if skippy < 0 { return ErrInvalidLengthDmworker } + if (iNdEx + skippy) < 0 { + return ErrInvalidLengthDmworker + } if (iNdEx + skippy) > l { return io.ErrUnexpectedEOF } @@ -9639,7 +10433,7 @@ func (m *CheckError) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - wire |= (uint64(b) & 0x7F) << shift + wire |= uint64(b&0x7F) << shift if b < 0x80 { break } @@ -9667,7 +10461,7 @@ func (m *CheckError) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= (uint64(b) & 0x7F) << shift + stringLen |= uint64(b&0x7F) << shift if b < 0x80 { break } @@ -9677,6 +10471,9 @@ func (m *CheckError) Unmarshal(dAtA []byte) error { return ErrInvalidLengthDmworker } postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthDmworker + } if postIndex > l { return io.ErrUnexpectedEOF } @@ -9691,6 +10488,9 @@ func (m *CheckError) Unmarshal(dAtA []byte) error { if skippy < 0 { return ErrInvalidLengthDmworker } + if (iNdEx + skippy) < 0 { + return ErrInvalidLengthDmworker + } if (iNdEx + skippy) > l { return io.ErrUnexpectedEOF } @@ -9718,7 +10518,7 @@ func (m *DumpError) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - wire |= (uint64(b) & 0x7F) << shift + wire |= uint64(b&0x7F) << shift if b < 0x80 { break } @@ -9746,7 +10546,7 @@ func (m *DumpError) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= (uint64(b) & 0x7F) << shift + stringLen |= uint64(b&0x7F) << shift if b < 0x80 { break } @@ -9756,6 +10556,9 @@ func (m *DumpError) Unmarshal(dAtA []byte) error { return ErrInvalidLengthDmworker } postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthDmworker + } if postIndex > l { return io.ErrUnexpectedEOF } @@ -9770,6 +10573,9 @@ func (m *DumpError) Unmarshal(dAtA []byte) error { if skippy < 0 { return ErrInvalidLengthDmworker } + if (iNdEx + skippy) < 0 { + return ErrInvalidLengthDmworker + } if (iNdEx + skippy) > l { return io.ErrUnexpectedEOF } @@ -9797,7 +10603,7 @@ func (m *LoadError) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - wire |= (uint64(b) & 0x7F) << shift + wire |= uint64(b&0x7F) << shift if b < 0x80 { break } @@ -9825,7 +10631,7 @@ func (m *LoadError) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= (uint64(b) & 0x7F) << shift + stringLen |= uint64(b&0x7F) << shift if b < 0x80 { break } @@ -9835,6 +10641,9 @@ func (m *LoadError) Unmarshal(dAtA []byte) error { return ErrInvalidLengthDmworker } postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthDmworker + } if postIndex > l { return io.ErrUnexpectedEOF } @@ -9849,6 +10658,9 @@ func (m *LoadError) Unmarshal(dAtA []byte) error { if skippy < 0 { return ErrInvalidLengthDmworker } + if (iNdEx + skippy) < 0 { + return ErrInvalidLengthDmworker + } if (iNdEx + skippy) > l { return io.ErrUnexpectedEOF } @@ -9876,7 +10688,7 @@ func (m *SyncSQLError) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - wire |= (uint64(b) & 0x7F) << shift + wire |= uint64(b&0x7F) << shift if b < 0x80 { break } @@ -9904,7 +10716,7 @@ func (m *SyncSQLError) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= (uint64(b) & 0x7F) << shift + stringLen |= uint64(b&0x7F) << shift if b < 0x80 { break } @@ -9914,6 +10726,9 @@ func (m *SyncSQLError) Unmarshal(dAtA []byte) error { return ErrInvalidLengthDmworker } postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthDmworker + } if postIndex > l { return io.ErrUnexpectedEOF } @@ -9933,7 +10748,7 @@ func (m *SyncSQLError) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= (uint64(b) & 0x7F) << shift + stringLen |= uint64(b&0x7F) << shift if b < 0x80 { break } @@ -9943,6 +10758,9 @@ func (m *SyncSQLError) Unmarshal(dAtA []byte) error { return ErrInvalidLengthDmworker } postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthDmworker + } if postIndex > l { return io.ErrUnexpectedEOF } @@ -9962,7 +10780,7 @@ func (m *SyncSQLError) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= (uint64(b) & 0x7F) << shift + stringLen |= uint64(b&0x7F) << shift if b < 0x80 { break } @@ -9972,6 +10790,9 @@ func (m *SyncSQLError) Unmarshal(dAtA []byte) error { return ErrInvalidLengthDmworker } postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthDmworker + } if postIndex > l { return io.ErrUnexpectedEOF } @@ -9986,6 +10807,9 @@ func (m *SyncSQLError) Unmarshal(dAtA []byte) error { if skippy < 0 { return ErrInvalidLengthDmworker } + if (iNdEx + skippy) < 0 { + return ErrInvalidLengthDmworker + } if (iNdEx + skippy) > l { return io.ErrUnexpectedEOF } @@ -10013,7 +10837,7 @@ func (m *SyncError) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - wire |= (uint64(b) & 0x7F) << shift + wire |= uint64(b&0x7F) << shift if b < 0x80 { break } @@ -10041,7 +10865,7 @@ func (m *SyncError) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - msglen |= (int(b) & 0x7F) << shift + msglen |= int(b&0x7F) << shift if b < 0x80 { break } @@ -10050,6 +10874,9 @@ func (m *SyncError) Unmarshal(dAtA []byte) error { return ErrInvalidLengthDmworker } postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthDmworker + } if postIndex > l { return io.ErrUnexpectedEOF } @@ -10067,6 +10894,9 @@ func (m *SyncError) Unmarshal(dAtA []byte) error { if skippy < 0 { return ErrInvalidLengthDmworker } + if (iNdEx + skippy) < 0 { + return ErrInvalidLengthDmworker + } if (iNdEx + skippy) > l { return io.ErrUnexpectedEOF } @@ -10094,7 +10924,7 @@ func (m *RelayError) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - wire |= (uint64(b) & 0x7F) << shift + wire |= uint64(b&0x7F) << shift if b < 0x80 { break } @@ -10122,7 +10952,7 @@ func (m *RelayError) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= (uint64(b) & 0x7F) << shift + stringLen |= uint64(b&0x7F) << shift if b < 0x80 { break } @@ -10132,6 +10962,9 @@ func (m *RelayError) Unmarshal(dAtA []byte) error { return ErrInvalidLengthDmworker } postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthDmworker + } if postIndex > l { return io.ErrUnexpectedEOF } @@ -10146,6 +10979,9 @@ func (m *RelayError) Unmarshal(dAtA []byte) error { if skippy < 0 { return ErrInvalidLengthDmworker } + if (iNdEx + skippy) < 0 { + return ErrInvalidLengthDmworker + } if (iNdEx + skippy) > l { return io.ErrUnexpectedEOF } @@ -10173,7 +11009,7 @@ func (m *SubTaskError) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - wire |= (uint64(b) & 0x7F) << shift + wire |= uint64(b&0x7F) << shift if b < 0x80 { break } @@ -10201,7 +11037,7 @@ func (m *SubTaskError) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= (uint64(b) & 0x7F) << shift + stringLen |= uint64(b&0x7F) << shift if b < 0x80 { break } @@ -10211,6 +11047,9 @@ func (m *SubTaskError) Unmarshal(dAtA []byte) error { return ErrInvalidLengthDmworker } postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthDmworker + } if postIndex > l { return io.ErrUnexpectedEOF } @@ -10230,7 +11069,7 @@ func (m *SubTaskError) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - m.Stage |= (Stage(b) & 0x7F) << shift + m.Stage |= Stage(b&0x7F) << shift if b < 0x80 { break } @@ -10249,7 +11088,7 @@ func (m *SubTaskError) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - m.Unit |= (UnitType(b) & 0x7F) << shift + m.Unit |= UnitType(b&0x7F) << shift if b < 0x80 { break } @@ -10268,7 +11107,7 @@ func (m *SubTaskError) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= (uint64(b) & 0x7F) << shift + stringLen |= uint64(b&0x7F) << shift if b < 0x80 { break } @@ -10278,6 +11117,9 @@ func (m *SubTaskError) Unmarshal(dAtA []byte) error { return ErrInvalidLengthDmworker } postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthDmworker + } if postIndex > l { return io.ErrUnexpectedEOF } @@ -10297,7 +11139,7 @@ func (m *SubTaskError) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - msglen |= (int(b) & 0x7F) << shift + msglen |= int(b&0x7F) << shift if b < 0x80 { break } @@ -10306,6 +11148,9 @@ func (m *SubTaskError) Unmarshal(dAtA []byte) error { return ErrInvalidLengthDmworker } postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthDmworker + } if postIndex > l { return io.ErrUnexpectedEOF } @@ -10329,7 +11174,7 @@ func (m *SubTaskError) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - msglen |= (int(b) & 0x7F) << shift + msglen |= int(b&0x7F) << shift if b < 0x80 { break } @@ -10338,6 +11183,9 @@ func (m *SubTaskError) Unmarshal(dAtA []byte) error { return ErrInvalidLengthDmworker } postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthDmworker + } if postIndex > l { return io.ErrUnexpectedEOF } @@ -10361,7 +11209,7 @@ func (m *SubTaskError) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - msglen |= (int(b) & 0x7F) << shift + msglen |= int(b&0x7F) << shift if b < 0x80 { break } @@ -10370,6 +11218,9 @@ func (m *SubTaskError) Unmarshal(dAtA []byte) error { return ErrInvalidLengthDmworker } postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthDmworker + } if postIndex > l { return io.ErrUnexpectedEOF } @@ -10393,7 +11244,7 @@ func (m *SubTaskError) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - msglen |= (int(b) & 0x7F) << shift + msglen |= int(b&0x7F) << shift if b < 0x80 { break } @@ -10402,6 +11253,9 @@ func (m *SubTaskError) Unmarshal(dAtA []byte) error { return ErrInvalidLengthDmworker } postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthDmworker + } if postIndex > l { return io.ErrUnexpectedEOF } @@ -10420,6 +11274,9 @@ func (m *SubTaskError) Unmarshal(dAtA []byte) error { if skippy < 0 { return ErrInvalidLengthDmworker } + if (iNdEx + skippy) < 0 { + return ErrInvalidLengthDmworker + } if (iNdEx + skippy) > l { return io.ErrUnexpectedEOF } @@ -10447,7 +11304,7 @@ func (m *SubTaskErrorList) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - wire |= (uint64(b) & 0x7F) << shift + wire |= uint64(b&0x7F) << shift if b < 0x80 { break } @@ -10475,7 +11332,7 @@ func (m *SubTaskErrorList) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - msglen |= (int(b) & 0x7F) << shift + msglen |= int(b&0x7F) << shift if b < 0x80 { break } @@ -10484,6 +11341,9 @@ func (m *SubTaskErrorList) Unmarshal(dAtA []byte) error { return ErrInvalidLengthDmworker } postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthDmworker + } if postIndex > l { return io.ErrUnexpectedEOF } @@ -10501,6 +11361,9 @@ func (m *SubTaskErrorList) Unmarshal(dAtA []byte) error { if skippy < 0 { return ErrInvalidLengthDmworker } + if (iNdEx + skippy) < 0 { + return ErrInvalidLengthDmworker + } if (iNdEx + skippy) > l { return io.ErrUnexpectedEOF } @@ -10528,7 +11391,7 @@ func (m *ProcessResult) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - wire |= (uint64(b) & 0x7F) << shift + wire |= uint64(b&0x7F) << shift if b < 0x80 { break } @@ -10556,7 +11419,7 @@ func (m *ProcessResult) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - v |= (int(b) & 0x7F) << shift + v |= int(b&0x7F) << shift if b < 0x80 { break } @@ -10576,7 +11439,7 @@ func (m *ProcessResult) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - msglen |= (int(b) & 0x7F) << shift + msglen |= int(b&0x7F) << shift if b < 0x80 { break } @@ -10585,6 +11448,9 @@ func (m *ProcessResult) Unmarshal(dAtA []byte) error { return ErrInvalidLengthDmworker } postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthDmworker + } if postIndex > l { return io.ErrUnexpectedEOF } @@ -10607,7 +11473,7 @@ func (m *ProcessResult) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - byteLen |= (int(b) & 0x7F) << shift + byteLen |= int(b&0x7F) << shift if b < 0x80 { break } @@ -10616,6 +11482,9 @@ func (m *ProcessResult) Unmarshal(dAtA []byte) error { return ErrInvalidLengthDmworker } postIndex := iNdEx + byteLen + if postIndex < 0 { + return ErrInvalidLengthDmworker + } if postIndex > l { return io.ErrUnexpectedEOF } @@ -10633,6 +11502,9 @@ func (m *ProcessResult) Unmarshal(dAtA []byte) error { if skippy < 0 { return ErrInvalidLengthDmworker } + if (iNdEx + skippy) < 0 { + return ErrInvalidLengthDmworker + } if (iNdEx + skippy) > l { return io.ErrUnexpectedEOF } @@ -10660,7 +11532,7 @@ func (m *TError) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - wire |= (uint64(b) & 0x7F) << shift + wire |= uint64(b&0x7F) << shift if b < 0x80 { break } @@ -10688,7 +11560,7 @@ func (m *TError) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - m.ErrCode |= (int32(b) & 0x7F) << shift + m.ErrCode |= int32(b&0x7F) << shift if b < 0x80 { break } @@ -10707,7 +11579,7 @@ func (m *TError) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - m.ErrClass |= (int32(b) & 0x7F) << shift + m.ErrClass |= int32(b&0x7F) << shift if b < 0x80 { break } @@ -10726,7 +11598,7 @@ func (m *TError) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - m.ErrScope |= (int32(b) & 0x7F) << shift + m.ErrScope |= int32(b&0x7F) << shift if b < 0x80 { break } @@ -10745,7 +11617,7 @@ func (m *TError) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - m.ErrLevel |= (int32(b) & 0x7F) << shift + m.ErrLevel |= int32(b&0x7F) << shift if b < 0x80 { break } @@ -10764,7 +11636,7 @@ func (m *TError) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= (uint64(b) & 0x7F) << shift + stringLen |= uint64(b&0x7F) << shift if b < 0x80 { break } @@ -10774,6 +11646,9 @@ func (m *TError) Unmarshal(dAtA []byte) error { return ErrInvalidLengthDmworker } postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthDmworker + } if postIndex > l { return io.ErrUnexpectedEOF } @@ -10793,7 +11668,7 @@ func (m *TError) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= (uint64(b) & 0x7F) << shift + stringLen |= uint64(b&0x7F) << shift if b < 0x80 { break } @@ -10803,6 +11678,9 @@ func (m *TError) Unmarshal(dAtA []byte) error { return ErrInvalidLengthDmworker } postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthDmworker + } if postIndex > l { return io.ErrUnexpectedEOF } @@ -10817,6 +11695,9 @@ func (m *TError) Unmarshal(dAtA []byte) error { if skippy < 0 { return ErrInvalidLengthDmworker } + if (iNdEx + skippy) < 0 { + return ErrInvalidLengthDmworker + } if (iNdEx + skippy) > l { return io.ErrUnexpectedEOF } @@ -10844,7 +11725,7 @@ func (m *ProcessError) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - wire |= (uint64(b) & 0x7F) << shift + wire |= uint64(b&0x7F) << shift if b < 0x80 { break } @@ -10872,7 +11753,7 @@ func (m *ProcessError) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - m.Type |= (ErrorType(b) & 0x7F) << shift + m.Type |= ErrorType(b&0x7F) << shift if b < 0x80 { break } @@ -10891,7 +11772,7 @@ func (m *ProcessError) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= (uint64(b) & 0x7F) << shift + stringLen |= uint64(b&0x7F) << shift if b < 0x80 { break } @@ -10901,6 +11782,9 @@ func (m *ProcessError) Unmarshal(dAtA []byte) error { return ErrInvalidLengthDmworker } postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthDmworker + } if postIndex > l { return io.ErrUnexpectedEOF } @@ -10920,7 +11804,7 @@ func (m *ProcessError) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - msglen |= (int(b) & 0x7F) << shift + msglen |= int(b&0x7F) << shift if b < 0x80 { break } @@ -10929,6 +11813,9 @@ func (m *ProcessError) Unmarshal(dAtA []byte) error { return ErrInvalidLengthDmworker } postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthDmworker + } if postIndex > l { return io.ErrUnexpectedEOF } @@ -10948,6 +11835,9 @@ func (m *ProcessError) Unmarshal(dAtA []byte) error { if skippy < 0 { return ErrInvalidLengthDmworker } + if (iNdEx + skippy) < 0 { + return ErrInvalidLengthDmworker + } if (iNdEx + skippy) > l { return io.ErrUnexpectedEOF } @@ -10975,7 +11865,7 @@ func (m *DDLInfo) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - wire |= (uint64(b) & 0x7F) << shift + wire |= uint64(b&0x7F) << shift if b < 0x80 { break } @@ -11003,7 +11893,7 @@ func (m *DDLInfo) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= (uint64(b) & 0x7F) << shift + stringLen |= uint64(b&0x7F) << shift if b < 0x80 { break } @@ -11013,6 +11903,9 @@ func (m *DDLInfo) Unmarshal(dAtA []byte) error { return ErrInvalidLengthDmworker } postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthDmworker + } if postIndex > l { return io.ErrUnexpectedEOF } @@ -11032,7 +11925,7 @@ func (m *DDLInfo) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= (uint64(b) & 0x7F) << shift + stringLen |= uint64(b&0x7F) << shift if b < 0x80 { break } @@ -11042,6 +11935,9 @@ func (m *DDLInfo) Unmarshal(dAtA []byte) error { return ErrInvalidLengthDmworker } postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthDmworker + } if postIndex > l { return io.ErrUnexpectedEOF } @@ -11061,7 +11957,7 @@ func (m *DDLInfo) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= (uint64(b) & 0x7F) << shift + stringLen |= uint64(b&0x7F) << shift if b < 0x80 { break } @@ -11071,6 +11967,9 @@ func (m *DDLInfo) Unmarshal(dAtA []byte) error { return ErrInvalidLengthDmworker } postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthDmworker + } if postIndex > l { return io.ErrUnexpectedEOF } @@ -11090,7 +11989,7 @@ func (m *DDLInfo) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= (uint64(b) & 0x7F) << shift + stringLen |= uint64(b&0x7F) << shift if b < 0x80 { break } @@ -11100,6 +11999,9 @@ func (m *DDLInfo) Unmarshal(dAtA []byte) error { return ErrInvalidLengthDmworker } postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthDmworker + } if postIndex > l { return io.ErrUnexpectedEOF } @@ -11114,6 +12016,9 @@ func (m *DDLInfo) Unmarshal(dAtA []byte) error { if skippy < 0 { return ErrInvalidLengthDmworker } + if (iNdEx + skippy) < 0 { + return ErrInvalidLengthDmworker + } if (iNdEx + skippy) > l { return io.ErrUnexpectedEOF } @@ -11141,7 +12046,7 @@ func (m *DDLLockInfo) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - wire |= (uint64(b) & 0x7F) << shift + wire |= uint64(b&0x7F) << shift if b < 0x80 { break } @@ -11169,7 +12074,7 @@ func (m *DDLLockInfo) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= (uint64(b) & 0x7F) << shift + stringLen |= uint64(b&0x7F) << shift if b < 0x80 { break } @@ -11179,6 +12084,9 @@ func (m *DDLLockInfo) Unmarshal(dAtA []byte) error { return ErrInvalidLengthDmworker } postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthDmworker + } if postIndex > l { return io.ErrUnexpectedEOF } @@ -11198,7 +12106,7 @@ func (m *DDLLockInfo) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= (uint64(b) & 0x7F) << shift + stringLen |= uint64(b&0x7F) << shift if b < 0x80 { break } @@ -11208,6 +12116,9 @@ func (m *DDLLockInfo) Unmarshal(dAtA []byte) error { return ErrInvalidLengthDmworker } postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthDmworker + } if postIndex > l { return io.ErrUnexpectedEOF } @@ -11222,6 +12133,9 @@ func (m *DDLLockInfo) Unmarshal(dAtA []byte) error { if skippy < 0 { return ErrInvalidLengthDmworker } + if (iNdEx + skippy) < 0 { + return ErrInvalidLengthDmworker + } if (iNdEx + skippy) > l { return io.ErrUnexpectedEOF } @@ -11249,7 +12163,7 @@ func (m *ExecDDLRequest) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - wire |= (uint64(b) & 0x7F) << shift + wire |= uint64(b&0x7F) << shift if b < 0x80 { break } @@ -11277,7 +12191,7 @@ func (m *ExecDDLRequest) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= (uint64(b) & 0x7F) << shift + stringLen |= uint64(b&0x7F) << shift if b < 0x80 { break } @@ -11287,6 +12201,9 @@ func (m *ExecDDLRequest) Unmarshal(dAtA []byte) error { return ErrInvalidLengthDmworker } postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthDmworker + } if postIndex > l { return io.ErrUnexpectedEOF } @@ -11306,7 +12223,7 @@ func (m *ExecDDLRequest) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= (uint64(b) & 0x7F) << shift + stringLen |= uint64(b&0x7F) << shift if b < 0x80 { break } @@ -11316,6 +12233,9 @@ func (m *ExecDDLRequest) Unmarshal(dAtA []byte) error { return ErrInvalidLengthDmworker } postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthDmworker + } if postIndex > l { return io.ErrUnexpectedEOF } @@ -11335,7 +12255,7 @@ func (m *ExecDDLRequest) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - v |= (int(b) & 0x7F) << shift + v |= int(b&0x7F) << shift if b < 0x80 { break } @@ -11355,7 +12275,7 @@ func (m *ExecDDLRequest) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= (uint64(b) & 0x7F) << shift + stringLen |= uint64(b&0x7F) << shift if b < 0x80 { break } @@ -11365,6 +12285,9 @@ func (m *ExecDDLRequest) Unmarshal(dAtA []byte) error { return ErrInvalidLengthDmworker } postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthDmworker + } if postIndex > l { return io.ErrUnexpectedEOF } @@ -11384,7 +12307,7 @@ func (m *ExecDDLRequest) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= (uint64(b) & 0x7F) << shift + stringLen |= uint64(b&0x7F) << shift if b < 0x80 { break } @@ -11394,6 +12317,9 @@ func (m *ExecDDLRequest) Unmarshal(dAtA []byte) error { return ErrInvalidLengthDmworker } postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthDmworker + } if postIndex > l { return io.ErrUnexpectedEOF } @@ -11408,6 +12334,9 @@ func (m *ExecDDLRequest) Unmarshal(dAtA []byte) error { if skippy < 0 { return ErrInvalidLengthDmworker } + if (iNdEx + skippy) < 0 { + return ErrInvalidLengthDmworker + } if (iNdEx + skippy) > l { return io.ErrUnexpectedEOF } @@ -11435,7 +12364,7 @@ func (m *BreakDDLLockRequest) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - wire |= (uint64(b) & 0x7F) << shift + wire |= uint64(b&0x7F) << shift if b < 0x80 { break } @@ -11463,7 +12392,7 @@ func (m *BreakDDLLockRequest) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= (uint64(b) & 0x7F) << shift + stringLen |= uint64(b&0x7F) << shift if b < 0x80 { break } @@ -11473,6 +12402,9 @@ func (m *BreakDDLLockRequest) Unmarshal(dAtA []byte) error { return ErrInvalidLengthDmworker } postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthDmworker + } if postIndex > l { return io.ErrUnexpectedEOF } @@ -11492,7 +12424,7 @@ func (m *BreakDDLLockRequest) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= (uint64(b) & 0x7F) << shift + stringLen |= uint64(b&0x7F) << shift if b < 0x80 { break } @@ -11502,6 +12434,9 @@ func (m *BreakDDLLockRequest) Unmarshal(dAtA []byte) error { return ErrInvalidLengthDmworker } postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthDmworker + } if postIndex > l { return io.ErrUnexpectedEOF } @@ -11521,7 +12456,7 @@ func (m *BreakDDLLockRequest) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - v |= (int(b) & 0x7F) << shift + v |= int(b&0x7F) << shift if b < 0x80 { break } @@ -11541,7 +12476,7 @@ func (m *BreakDDLLockRequest) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - v |= (int(b) & 0x7F) << shift + v |= int(b&0x7F) << shift if b < 0x80 { break } @@ -11556,6 +12491,9 @@ func (m *BreakDDLLockRequest) Unmarshal(dAtA []byte) error { if skippy < 0 { return ErrInvalidLengthDmworker } + if (iNdEx + skippy) < 0 { + return ErrInvalidLengthDmworker + } if (iNdEx + skippy) > l { return io.ErrUnexpectedEOF } @@ -11583,7 +12521,7 @@ func (m *SwitchRelayMasterRequest) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - wire |= (uint64(b) & 0x7F) << shift + wire |= uint64(b&0x7F) << shift if b < 0x80 { break } @@ -11606,6 +12544,9 @@ func (m *SwitchRelayMasterRequest) Unmarshal(dAtA []byte) error { if skippy < 0 { return ErrInvalidLengthDmworker } + if (iNdEx + skippy) < 0 { + return ErrInvalidLengthDmworker + } if (iNdEx + skippy) > l { return io.ErrUnexpectedEOF } @@ -11633,7 +12574,7 @@ func (m *OperateRelayRequest) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - wire |= (uint64(b) & 0x7F) << shift + wire |= uint64(b&0x7F) << shift if b < 0x80 { break } @@ -11661,7 +12602,7 @@ func (m *OperateRelayRequest) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - m.Op |= (RelayOp(b) & 0x7F) << shift + m.Op |= RelayOp(b&0x7F) << shift if b < 0x80 { break } @@ -11675,6 +12616,9 @@ func (m *OperateRelayRequest) Unmarshal(dAtA []byte) error { if skippy < 0 { return ErrInvalidLengthDmworker } + if (iNdEx + skippy) < 0 { + return ErrInvalidLengthDmworker + } if (iNdEx + skippy) > l { return io.ErrUnexpectedEOF } @@ -11702,7 +12646,7 @@ func (m *OperateRelayResponse) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - wire |= (uint64(b) & 0x7F) << shift + wire |= uint64(b&0x7F) << shift if b < 0x80 { break } @@ -11730,7 +12674,7 @@ func (m *OperateRelayResponse) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - m.Op |= (RelayOp(b) & 0x7F) << shift + m.Op |= RelayOp(b&0x7F) << shift if b < 0x80 { break } @@ -11749,7 +12693,7 @@ func (m *OperateRelayResponse) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - v |= (int(b) & 0x7F) << shift + v |= int(b&0x7F) << shift if b < 0x80 { break } @@ -11769,7 +12713,7 @@ func (m *OperateRelayResponse) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= (uint64(b) & 0x7F) << shift + stringLen |= uint64(b&0x7F) << shift if b < 0x80 { break } @@ -11779,6 +12723,9 @@ func (m *OperateRelayResponse) Unmarshal(dAtA []byte) error { return ErrInvalidLengthDmworker } postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthDmworker + } if postIndex > l { return io.ErrUnexpectedEOF } @@ -11798,7 +12745,7 @@ func (m *OperateRelayResponse) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= (uint64(b) & 0x7F) << shift + stringLen |= uint64(b&0x7F) << shift if b < 0x80 { break } @@ -11808,6 +12755,9 @@ func (m *OperateRelayResponse) Unmarshal(dAtA []byte) error { return ErrInvalidLengthDmworker } postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthDmworker + } if postIndex > l { return io.ErrUnexpectedEOF } @@ -11822,6 +12772,9 @@ func (m *OperateRelayResponse) Unmarshal(dAtA []byte) error { if skippy < 0 { return ErrInvalidLengthDmworker } + if (iNdEx + skippy) < 0 { + return ErrInvalidLengthDmworker + } if (iNdEx + skippy) > l { return io.ErrUnexpectedEOF } @@ -11849,7 +12802,7 @@ func (m *PurgeRelayRequest) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - wire |= (uint64(b) & 0x7F) << shift + wire |= uint64(b&0x7F) << shift if b < 0x80 { break } @@ -11877,7 +12830,7 @@ func (m *PurgeRelayRequest) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - v |= (int(b) & 0x7F) << shift + v |= int(b&0x7F) << shift if b < 0x80 { break } @@ -11897,7 +12850,7 @@ func (m *PurgeRelayRequest) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - m.Time |= (int64(b) & 0x7F) << shift + m.Time |= int64(b&0x7F) << shift if b < 0x80 { break } @@ -11916,7 +12869,7 @@ func (m *PurgeRelayRequest) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= (uint64(b) & 0x7F) << shift + stringLen |= uint64(b&0x7F) << shift if b < 0x80 { break } @@ -11926,6 +12879,9 @@ func (m *PurgeRelayRequest) Unmarshal(dAtA []byte) error { return ErrInvalidLengthDmworker } postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthDmworker + } if postIndex > l { return io.ErrUnexpectedEOF } @@ -11945,7 +12901,7 @@ func (m *PurgeRelayRequest) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= (uint64(b) & 0x7F) << shift + stringLen |= uint64(b&0x7F) << shift if b < 0x80 { break } @@ -11955,6 +12911,9 @@ func (m *PurgeRelayRequest) Unmarshal(dAtA []byte) error { return ErrInvalidLengthDmworker } postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthDmworker + } if postIndex > l { return io.ErrUnexpectedEOF } @@ -11969,6 +12928,9 @@ func (m *PurgeRelayRequest) Unmarshal(dAtA []byte) error { if skippy < 0 { return ErrInvalidLengthDmworker } + if (iNdEx + skippy) < 0 { + return ErrInvalidLengthDmworker + } if (iNdEx + skippy) > l { return io.ErrUnexpectedEOF } @@ -11996,7 +12958,7 @@ func (m *QueryWorkerConfigRequest) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - wire |= (uint64(b) & 0x7F) << shift + wire |= uint64(b&0x7F) << shift if b < 0x80 { break } @@ -12019,6 +12981,9 @@ func (m *QueryWorkerConfigRequest) Unmarshal(dAtA []byte) error { if skippy < 0 { return ErrInvalidLengthDmworker } + if (iNdEx + skippy) < 0 { + return ErrInvalidLengthDmworker + } if (iNdEx + skippy) > l { return io.ErrUnexpectedEOF } @@ -12046,7 +13011,7 @@ func (m *QueryWorkerConfigResponse) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - wire |= (uint64(b) & 0x7F) << shift + wire |= uint64(b&0x7F) << shift if b < 0x80 { break } @@ -12074,7 +13039,7 @@ func (m *QueryWorkerConfigResponse) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - v |= (int(b) & 0x7F) << shift + v |= int(b&0x7F) << shift if b < 0x80 { break } @@ -12094,7 +13059,7 @@ func (m *QueryWorkerConfigResponse) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= (uint64(b) & 0x7F) << shift + stringLen |= uint64(b&0x7F) << shift if b < 0x80 { break } @@ -12104,6 +13069,9 @@ func (m *QueryWorkerConfigResponse) Unmarshal(dAtA []byte) error { return ErrInvalidLengthDmworker } postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthDmworker + } if postIndex > l { return io.ErrUnexpectedEOF } @@ -12123,7 +13091,7 @@ func (m *QueryWorkerConfigResponse) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= (uint64(b) & 0x7F) << shift + stringLen |= uint64(b&0x7F) << shift if b < 0x80 { break } @@ -12133,6 +13101,9 @@ func (m *QueryWorkerConfigResponse) Unmarshal(dAtA []byte) error { return ErrInvalidLengthDmworker } postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthDmworker + } if postIndex > l { return io.ErrUnexpectedEOF } @@ -12152,7 +13123,7 @@ func (m *QueryWorkerConfigResponse) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= (uint64(b) & 0x7F) << shift + stringLen |= uint64(b&0x7F) << shift if b < 0x80 { break } @@ -12162,6 +13133,9 @@ func (m *QueryWorkerConfigResponse) Unmarshal(dAtA []byte) error { return ErrInvalidLengthDmworker } postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthDmworker + } if postIndex > l { return io.ErrUnexpectedEOF } @@ -12181,7 +13155,7 @@ func (m *QueryWorkerConfigResponse) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= (uint64(b) & 0x7F) << shift + stringLen |= uint64(b&0x7F) << shift if b < 0x80 { break } @@ -12191,6 +13165,9 @@ func (m *QueryWorkerConfigResponse) Unmarshal(dAtA []byte) error { return ErrInvalidLengthDmworker } postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthDmworker + } if postIndex > l { return io.ErrUnexpectedEOF } @@ -12205,6 +13182,218 @@ func (m *QueryWorkerConfigResponse) Unmarshal(dAtA []byte) error { if skippy < 0 { return ErrInvalidLengthDmworker } + if (iNdEx + skippy) < 0 { + return ErrInvalidLengthDmworker + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *MysqlTaskRequest) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowDmworker + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: MysqlTaskRequest: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: MysqlTaskRequest: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Op", wireType) + } + m.Op = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowDmworker + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.Op |= WorkerOp(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Config", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowDmworker + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLengthDmworker + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthDmworker + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Config = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipDmworker(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthDmworker + } + if (iNdEx + skippy) < 0 { + return ErrInvalidLengthDmworker + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *MysqlTaskResponse) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowDmworker + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: MysqlTaskResponse: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: MysqlTaskResponse: 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 ErrIntOverflowDmworker + } + 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 ErrIntOverflowDmworker + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLengthDmworker + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthDmworker + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Msg = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipDmworker(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthDmworker + } + if (iNdEx + skippy) < 0 { + return ErrInvalidLengthDmworker + } if (iNdEx + skippy) > l { return io.ErrUnexpectedEOF } @@ -12220,6 +13409,7 @@ func (m *QueryWorkerConfigResponse) Unmarshal(dAtA []byte) error { func skipDmworker(dAtA []byte) (n int, err error) { l := len(dAtA) iNdEx := 0 + depth := 0 for iNdEx < l { var wire uint64 for shift := uint(0); ; shift += 7 { @@ -12251,10 +13441,8 @@ func skipDmworker(dAtA []byte) (n int, err error) { break } } - return iNdEx, nil case 1: iNdEx += 8 - return iNdEx, nil case 2: var length int for shift := uint(0); ; shift += 7 { @@ -12271,198 +13459,34 @@ func skipDmworker(dAtA []byte) (n int, err error) { break } } - iNdEx += length if length < 0 { return 0, ErrInvalidLengthDmworker } - return iNdEx, nil + iNdEx += length case 3: - for { - var innerWire uint64 - var start int = iNdEx - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return 0, ErrIntOverflowDmworker - } - if iNdEx >= l { - return 0, io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - innerWire |= (uint64(b) & 0x7F) << shift - if b < 0x80 { - break - } - } - innerWireType := int(innerWire & 0x7) - if innerWireType == 4 { - break - } - next, err := skipDmworker(dAtA[start:]) - if err != nil { - return 0, err - } - iNdEx = start + next - } - return iNdEx, nil + depth++ case 4: - return iNdEx, nil + if depth == 0 { + return 0, ErrUnexpectedEndOfGroupDmworker + } + depth-- case 5: iNdEx += 4 - return iNdEx, nil default: return 0, fmt.Errorf("proto: illegal wireType %d", wireType) } + if iNdEx < 0 { + return 0, ErrInvalidLengthDmworker + } + if depth == 0 { + return iNdEx, nil + } } - panic("unreachable") + return 0, io.ErrUnexpectedEOF } var ( - ErrInvalidLengthDmworker = fmt.Errorf("proto: negative length found during unmarshaling") - ErrIntOverflowDmworker = fmt.Errorf("proto: integer overflow") + ErrInvalidLengthDmworker = fmt.Errorf("proto: negative length found during unmarshaling") + ErrIntOverflowDmworker = fmt.Errorf("proto: integer overflow") + ErrUnexpectedEndOfGroupDmworker = fmt.Errorf("proto: unexpected end of group") ) - -func init() { proto.RegisterFile("dmworker.proto", fileDescriptor_dmworker_84482ac67274de9a) } - -var fileDescriptor_dmworker_84482ac67274de9a = []byte{ - // 2210 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xb4, 0x59, 0xcf, 0x6f, 0x23, 0x59, - 0xf1, 0x77, 0xb7, 0x7f, 0xc4, 0x2e, 0x3b, 0x99, 0xce, 0x4b, 0x36, 0xeb, 0xf1, 0x77, 0x37, 0xdf, - 0xd0, 0xb3, 0xda, 0xcd, 0xe6, 0x10, 0xed, 0x06, 0x10, 0x12, 0xb0, 0xc0, 0xc6, 0xce, 0x64, 0x0c, - 0xce, 0x4c, 0xd2, 0xce, 0x00, 0xb7, 0x55, 0xa7, 0xfd, 0xe2, 0xb4, 0xd2, 0xee, 0xee, 0xe9, 0x1f, - 0xc9, 0xe6, 0x06, 0xe2, 0x88, 0x84, 0x90, 0x90, 0x90, 0x10, 0x67, 0x6e, 0xfc, 0x09, 0xdc, 0xe1, - 0xb8, 0x47, 0x6e, 0xa0, 0x99, 0x7f, 0x83, 0x03, 0xaa, 0x7a, 0xaf, 0xbb, 0x5f, 0x27, 0xb6, 0x77, - 0x0f, 0xc3, 0x25, 0x72, 0xfd, 0x78, 0xf5, 0xea, 0x7d, 0xaa, 0xba, 0xea, 0xbd, 0x0a, 0xac, 0x4d, - 0x66, 0xb7, 0x41, 0x74, 0xcd, 0xa3, 0xfd, 0x30, 0x0a, 0x92, 0x80, 0xe9, 0xe1, 0x85, 0xf9, 0x31, - 0x6c, 0x8c, 0x13, 0x3b, 0x4a, 0xc6, 0xe9, 0xc5, 0xb9, 0x1d, 0x5f, 0x5b, 0xfc, 0x55, 0xca, 0xe3, - 0x84, 0x31, 0xa8, 0x25, 0x76, 0x7c, 0xdd, 0xd5, 0x76, 0xb4, 0xdd, 0x96, 0x45, 0xbf, 0xcd, 0x7d, - 0x60, 0x2f, 0xc3, 0x89, 0x9d, 0x70, 0x8b, 0x7b, 0xf6, 0x5d, 0xa6, 0xd9, 0x85, 0x15, 0x27, 0xf0, - 0x13, 0xee, 0x27, 0x52, 0x39, 0x23, 0xcd, 0x31, 0x6c, 0x9c, 0xb8, 0xd3, 0xe8, 0xfe, 0x82, 0x6d, - 0x80, 0x43, 0xd7, 0xf7, 0x82, 0xe9, 0x73, 0x7b, 0xc6, 0xe5, 0x1a, 0x85, 0xc3, 0xde, 0x83, 0x96, - 0xa0, 0x4e, 0x83, 0xb8, 0xab, 0xef, 0x68, 0xbb, 0xab, 0x56, 0xc1, 0x30, 0x8f, 0xe1, 0x9d, 0x17, - 0x21, 0x47, 0xa3, 0xf7, 0x3c, 0xee, 0x81, 0x1e, 0x84, 0x64, 0x6e, 0xed, 0x00, 0xf6, 0xc3, 0x8b, - 0x7d, 0x14, 0xbe, 0x08, 0x2d, 0x3d, 0x08, 0xf1, 0x34, 0x3e, 0x6e, 0xa6, 0x8b, 0xd3, 0xe0, 0x6f, - 0xf3, 0x06, 0xb6, 0xee, 0x1b, 0x8a, 0xc3, 0xc0, 0x8f, 0xf9, 0x52, 0x4b, 0x5b, 0xd0, 0x88, 0x78, - 0x9c, 0x7a, 0x09, 0xd9, 0x6a, 0x5a, 0x92, 0x42, 0xbe, 0x80, 0xb6, 0x5b, 0xa5, 0x3d, 0x24, 0xc5, - 0x0c, 0xa8, 0xce, 0xe2, 0x69, 0xb7, 0x46, 0x4c, 0xfc, 0x69, 0xee, 0xc1, 0xa6, 0x40, 0xf1, 0x1b, - 0x20, 0xbe, 0x0b, 0xec, 0x2c, 0xe5, 0xd1, 0xdd, 0x38, 0xb1, 0x93, 0x34, 0x56, 0x34, 0xfd, 0x02, - 0x3a, 0x71, 0x9a, 0x8f, 0x60, 0x9d, 0x34, 0x8f, 0xa2, 0x28, 0x88, 0x96, 0x29, 0xfe, 0x59, 0x83, - 0xee, 0x33, 0xdb, 0x9f, 0x78, 0xd9, 0xfe, 0xe3, 0xb3, 0xd1, 0x32, 0xcb, 0xec, 0x31, 0xa1, 0xa1, - 0x13, 0x1a, 0x2d, 0x44, 0x63, 0x7c, 0x36, 0x2a, 0x60, 0xb5, 0xa3, 0x69, 0xdc, 0xad, 0xee, 0x54, - 0x51, 0x1d, 0x7f, 0x63, 0xf4, 0x2e, 0xf2, 0xe8, 0x89, 0x63, 0x17, 0x0c, 0x8c, 0x7d, 0xfc, 0xca, - 0x3b, 0xb5, 0x93, 0x84, 0x47, 0x7e, 0xb7, 0x2e, 0x62, 0x5f, 0x70, 0xcc, 0x5f, 0xc2, 0x66, 0x3f, - 0x98, 0xcd, 0x02, 0xff, 0x17, 0x04, 0x5f, 0x1e, 0x92, 0x02, 0x76, 0x6d, 0x01, 0xec, 0xfa, 0x3c, - 0xd8, 0xab, 0x05, 0xec, 0xff, 0xd2, 0x60, 0xa3, 0x84, 0xe5, 0xdb, 0xb2, 0xcc, 0xbe, 0x07, 0xab, - 0xb1, 0x84, 0x92, 0x4c, 0x77, 0x6b, 0x3b, 0xd5, 0xdd, 0xf6, 0xc1, 0x3a, 0x61, 0xa5, 0x0a, 0xac, - 0xb2, 0x1e, 0xfb, 0x14, 0xda, 0x11, 0x7e, 0x18, 0x72, 0x19, 0xa2, 0xd1, 0x3e, 0x78, 0x84, 0xcb, - 0xac, 0x82, 0x6d, 0xa9, 0x3a, 0xac, 0x07, 0xcd, 0x38, 0x48, 0x23, 0x87, 0x0f, 0x07, 0xdd, 0x06, - 0xb9, 0x90, 0xd3, 0xe6, 0xdf, 0x34, 0x99, 0x2d, 0x32, 0x07, 0xde, 0xda, 0x01, 0xbf, 0x03, 0x1d, - 0xe9, 0x38, 0x59, 0x96, 0xe7, 0x33, 0x94, 0xf3, 0x89, 0x1d, 0x4b, 0x5a, 0x6c, 0x1f, 0x80, 0x8e, - 0x21, 0xd6, 0x88, 0xc3, 0xad, 0xe5, 0x87, 0x13, 0x2b, 0x14, 0x0d, 0xf3, 0x2f, 0x1a, 0xb4, 0xfb, - 0x57, 0xdc, 0xc9, 0xd0, 0xd9, 0x82, 0x46, 0x68, 0xc7, 0x31, 0x9f, 0x64, 0x7e, 0x0b, 0x8a, 0x6d, - 0x42, 0x3d, 0x09, 0x12, 0xdb, 0x23, 0xb7, 0xeb, 0x96, 0x20, 0x28, 0xb1, 0x52, 0xc7, 0xe1, 0x71, - 0x7c, 0x99, 0x7a, 0xe4, 0x7c, 0xdd, 0x52, 0x38, 0x68, 0xed, 0xd2, 0x76, 0x3d, 0x3e, 0xa1, 0x9c, - 0xac, 0x5b, 0x92, 0xc2, 0xea, 0x75, 0x6b, 0x47, 0xbe, 0xeb, 0x4f, 0xc9, 0xc5, 0xba, 0x95, 0x91, - 0xb8, 0x62, 0xc2, 0x13, 0xdb, 0xf5, 0x08, 0xe8, 0x8e, 0x25, 0x29, 0xb3, 0x03, 0x30, 0x48, 0x67, - 0xa1, 0xf0, 0xd2, 0xfc, 0x9d, 0x06, 0x30, 0x0a, 0xec, 0x89, 0x74, 0xfa, 0x03, 0x58, 0xbd, 0x74, - 0x7d, 0x37, 0xbe, 0xe2, 0x93, 0xc3, 0xbb, 0x84, 0xc7, 0xe4, 0x7b, 0xd5, 0x2a, 0x33, 0xd1, 0x59, - 0xf2, 0x5a, 0xa8, 0xe8, 0xa4, 0xa2, 0x70, 0x30, 0xca, 0x61, 0x14, 0x4c, 0x23, 0x1e, 0xc7, 0x32, - 0x0e, 0x39, 0x8d, 0x6b, 0x67, 0x3c, 0xb1, 0x45, 0x41, 0x94, 0x1f, 0x98, 0xc2, 0x31, 0x7f, 0xab, - 0xc1, 0xea, 0xf8, 0xca, 0x8e, 0x26, 0xae, 0x3f, 0x3d, 0x8e, 0x82, 0x94, 0x4a, 0x56, 0x62, 0x47, - 0x53, 0x9e, 0xd5, 0x67, 0x49, 0xe1, 0xd7, 0x3b, 0x18, 0x8c, 0x70, 0x7f, 0xfa, 0x7a, 0xf1, 0x37, - 0xee, 0x7c, 0xe9, 0x46, 0x71, 0x82, 0x1f, 0xaf, 0xdc, 0x39, 0xa3, 0xd1, 0x4e, 0x7c, 0xe7, 0x3b, - 0x04, 0x21, 0xae, 0x90, 0x14, 0xae, 0x49, 0x7d, 0x29, 0xa9, 0x93, 0x24, 0xa7, 0xcd, 0xdf, 0x54, - 0x01, 0xc6, 0x77, 0xbe, 0x23, 0xe1, 0xd9, 0x81, 0x36, 0x1d, 0xf3, 0xe8, 0x86, 0xfb, 0x49, 0x06, - 0x8e, 0xca, 0x42, 0x63, 0x44, 0x9e, 0x87, 0x19, 0x30, 0x39, 0x8d, 0xa5, 0x25, 0xe2, 0x0e, 0xf7, - 0x13, 0x14, 0x56, 0x49, 0x58, 0x30, 0x98, 0x09, 0x9d, 0x99, 0x1d, 0x27, 0x3c, 0x2a, 0x41, 0x53, - 0xe2, 0xb1, 0x3d, 0x30, 0x54, 0xfa, 0x38, 0x71, 0x27, 0xb2, 0x08, 0x3d, 0xe0, 0xa3, 0x3d, 0x3a, - 0x44, 0x66, 0x4f, 0x7c, 0x6e, 0x25, 0x1e, 0xda, 0x53, 0x69, 0xb2, 0xb7, 0x22, 0xec, 0xdd, 0xe7, - 0xa3, 0xbd, 0x0b, 0x2f, 0x70, 0xae, 0x5d, 0x7f, 0x4a, 0xb0, 0x37, 0x09, 0xaa, 0x12, 0x8f, 0x7d, - 0x06, 0x46, 0xea, 0x47, 0x3c, 0x0e, 0xbc, 0x1b, 0x3e, 0xa1, 0xe8, 0xc5, 0xdd, 0x96, 0x52, 0x4d, - 0xd4, 0xb8, 0x5a, 0x0f, 0x54, 0x95, 0x08, 0x81, 0xf8, 0x64, 0x64, 0x14, 0xfe, 0xae, 0x43, 0x5b, - 0x29, 0x29, 0x0f, 0xa0, 0xd2, 0xbe, 0x21, 0x54, 0xfa, 0x02, 0xa8, 0x76, 0xb2, 0x42, 0x96, 0x5e, - 0x0c, 0xdc, 0xac, 0x03, 0xaa, 0xac, 0x5c, 0xa3, 0x14, 0x1b, 0x95, 0xc5, 0x76, 0xe1, 0x91, 0x42, - 0x2a, 0x91, 0xb9, 0xcf, 0x66, 0xfb, 0xc0, 0x88, 0xd5, 0xb7, 0x13, 0xe7, 0xea, 0x65, 0x78, 0x42, - 0xde, 0x50, 0x78, 0x9a, 0xd6, 0x1c, 0x09, 0xfb, 0x7f, 0xa8, 0xc7, 0x89, 0x3d, 0xe5, 0x14, 0x99, - 0xac, 0x87, 0x21, 0xc3, 0x12, 0x7c, 0xf6, 0x71, 0x5e, 0x21, 0x9b, 0x54, 0xa5, 0x08, 0xeb, 0xd3, - 0x28, 0xc0, 0xda, 0x61, 0x91, 0x20, 0x2b, 0x9a, 0xe6, 0x7f, 0x74, 0x58, 0x2d, 0xd5, 0xf4, 0xb9, - 0x2d, 0x33, 0xdf, 0x51, 0x5f, 0xb0, 0xe3, 0x0e, 0xd4, 0x52, 0xdf, 0x4d, 0x08, 0xa9, 0xb5, 0x83, - 0x0e, 0xca, 0x5f, 0xfa, 0x6e, 0x72, 0x7e, 0x17, 0x72, 0x8b, 0x24, 0x8a, 0x4f, 0xb5, 0xaf, 0xf1, - 0x89, 0x7d, 0x02, 0x1b, 0x45, 0x26, 0x0c, 0x06, 0xa3, 0x51, 0xe0, 0x5c, 0x0f, 0x07, 0x12, 0xbd, - 0x79, 0x22, 0xc6, 0x44, 0x89, 0xa7, 0x8c, 0x7e, 0x56, 0x11, 0x45, 0xfe, 0x23, 0xa8, 0x3b, 0x58, - 0x7d, 0x09, 0x25, 0xd9, 0x86, 0x94, 0x72, 0xfc, 0xac, 0x62, 0x09, 0x39, 0xfb, 0x00, 0x6a, 0x93, - 0x74, 0x16, 0x4a, 0xac, 0xa8, 0xa2, 0x17, 0xf5, 0xf0, 0x59, 0xc5, 0x22, 0x29, 0x6a, 0x79, 0x81, - 0x3d, 0xe9, 0xb6, 0x0a, 0xad, 0xa2, 0x4c, 0xa2, 0x16, 0x4a, 0x51, 0x0b, 0x53, 0x94, 0xd2, 0x55, - 0x6a, 0x15, 0xd5, 0x02, 0xb5, 0x50, 0x7a, 0xd8, 0x84, 0x46, 0x2c, 0xaa, 0xed, 0x8f, 0x60, 0xbd, - 0x84, 0xfe, 0xc8, 0x8d, 0x09, 0x2a, 0x21, 0xee, 0x6a, 0x8b, 0x1a, 0x6f, 0xb6, 0x7e, 0x1b, 0x80, - 0xce, 0x24, 0x3a, 0x94, 0xec, 0x74, 0x5a, 0x71, 0x49, 0x78, 0x1f, 0x5a, 0x78, 0x96, 0x25, 0x62, - 0x3c, 0xc4, 0x22, 0x71, 0x08, 0x1d, 0xf2, 0xfe, 0x6c, 0xb4, 0x40, 0x83, 0x1d, 0xc0, 0xa6, 0xe8, - 0x3b, 0xf9, 0x7d, 0xd6, 0x4d, 0xdc, 0xc0, 0x97, 0x1f, 0xd6, 0x5c, 0x19, 0x56, 0x44, 0x8e, 0xe6, - 0xc6, 0x67, 0xa3, 0xac, 0x24, 0x67, 0xb4, 0xf9, 0x5d, 0x68, 0xe1, 0x8e, 0x62, 0xbb, 0x5d, 0x68, - 0x90, 0x20, 0xc3, 0xc1, 0xc8, 0xe1, 0x94, 0x0e, 0x59, 0x52, 0x8e, 0x30, 0x14, 0x8d, 0x77, 0xce, - 0x41, 0xfe, 0xa4, 0x43, 0x47, 0xed, 0xec, 0xff, 0xab, 0x24, 0x67, 0xca, 0xe5, 0x38, 0xcb, 0xc3, - 0x0f, 0xb3, 0x3c, 0x54, 0x6e, 0x0c, 0x45, 0xcc, 0x8a, 0x34, 0x7c, 0x22, 0xd3, 0xb0, 0x41, 0x6a, - 0xab, 0x59, 0x1a, 0x66, 0x5a, 0x22, 0x0b, 0x9f, 0xc8, 0x2c, 0x5c, 0x29, 0x94, 0xf2, 0x00, 0xe6, - 0x49, 0xf8, 0x44, 0x26, 0x61, 0xb3, 0x50, 0xca, 0x41, 0xcd, 0x73, 0x70, 0x05, 0xea, 0x04, 0x9e, - 0xf9, 0x7d, 0x30, 0x54, 0x68, 0x28, 0x03, 0x3f, 0x94, 0xc2, 0x12, 0xf0, 0xea, 0xcd, 0x48, 0xae, - 0x7d, 0x05, 0xab, 0xa5, 0x4f, 0x18, 0x9b, 0xb9, 0x1b, 0xf7, 0x6d, 0xdf, 0xe1, 0x5e, 0x7e, 0xcf, - 0x51, 0x38, 0x4a, 0x48, 0xf5, 0xc2, 0xb2, 0x34, 0x51, 0x0a, 0xa9, 0x72, 0x5b, 0xa9, 0x96, 0x6e, - 0x2b, 0x7f, 0xd5, 0xa0, 0x71, 0x2e, 0x82, 0xd8, 0x85, 0x95, 0xa3, 0x28, 0xea, 0x07, 0x13, 0x11, - 0xc7, 0xba, 0x95, 0x91, 0x98, 0x62, 0xf8, 0xd3, 0xb3, 0xe3, 0x58, 0xde, 0xaa, 0x72, 0x5a, 0xca, - 0xc6, 0x4e, 0x10, 0x72, 0x79, 0xad, 0xca, 0x69, 0x29, 0x1b, 0xf1, 0x1b, 0xee, 0xc9, 0x6b, 0x55, - 0x4e, 0xe3, 0x6e, 0x27, 0x3c, 0x8e, 0x31, 0x41, 0x44, 0x25, 0xca, 0x48, 0x5c, 0x65, 0xd9, 0xb7, - 0x7d, 0x3b, 0x8d, 0x79, 0x76, 0x87, 0xcd, 0x68, 0x93, 0x43, 0x47, 0x3d, 0x1e, 0xfb, 0x16, 0xd4, - 0x30, 0x5f, 0xe4, 0x63, 0x8c, 0x62, 0x43, 0x02, 0x91, 0x44, 0xf8, 0x37, 0x4b, 0x5f, 0xbd, 0xf8, - 0xca, 0x76, 0xb2, 0x70, 0x54, 0x29, 0xa2, 0xe2, 0x09, 0x57, 0x0a, 0xc4, 0x17, 0xb0, 0x32, 0x18, - 0x8c, 0x86, 0xfe, 0x65, 0x30, 0xef, 0xd9, 0x45, 0x7d, 0xd4, 0xb9, 0xe2, 0x33, 0x3b, 0xbb, 0x1a, - 0x0b, 0x8a, 0xae, 0x9e, 0xf6, 0x85, 0xc7, 0xe5, 0x77, 0x28, 0x88, 0xfc, 0x1e, 0x55, 0x2b, 0xee, - 0x51, 0xe6, 0xa7, 0xd0, 0xce, 0xca, 0xed, 0xa2, 0x4d, 0xd6, 0x40, 0x1f, 0x0e, 0xe4, 0x06, 0xfa, - 0x70, 0x60, 0xfe, 0x4a, 0x83, 0xb5, 0xa3, 0x2f, 0xb9, 0x33, 0x18, 0x8c, 0x96, 0x3c, 0x09, 0xd1, - 0x37, 0x4f, 0x14, 0x78, 0xe9, 0x9b, 0x97, 0xd5, 0xf4, 0x1a, 0xff, 0x92, 0x3b, 0xe4, 0x5a, 0xd3, - 0xa2, 0xdf, 0x74, 0x99, 0x8a, 0x6c, 0x87, 0x1f, 0x0f, 0x07, 0xb2, 0xe5, 0xe6, 0x74, 0xee, 0x75, - 0x5d, 0xf1, 0xfa, 0xd7, 0x1a, 0x6c, 0x1c, 0x46, 0xdc, 0xbe, 0x96, 0xbe, 0x2f, 0xf3, 0xc3, 0x84, - 0x4e, 0xc4, 0x67, 0xc1, 0x0d, 0x1f, 0xa9, 0xde, 0x94, 0x78, 0x98, 0x03, 0x5c, 0x9c, 0x48, 0xba, - 0x95, 0x91, 0x28, 0x89, 0xaf, 0xdd, 0x10, 0x25, 0x35, 0x21, 0x91, 0xa4, 0xd9, 0x83, 0xee, 0xf8, - 0xd6, 0x4d, 0x9c, 0x2b, 0xaa, 0x50, 0xa2, 0x85, 0x4b, 0x3f, 0xcc, 0x03, 0xd8, 0x90, 0x4f, 0xf6, - 0xd2, 0x40, 0xe1, 0xff, 0x94, 0xf7, 0x7a, 0x3b, 0x7f, 0x61, 0x88, 0x37, 0xaa, 0x99, 0xc2, 0x66, - 0x79, 0x8d, 0x7c, 0x16, 0x2d, 0x5b, 0xf4, 0x16, 0x5e, 0xf9, 0xb7, 0xb0, 0x7e, 0x9a, 0x46, 0xd3, - 0xb2, 0xa3, 0x3d, 0x68, 0xba, 0xbe, 0xed, 0x24, 0xee, 0x0d, 0x97, 0x1f, 0x7b, 0x4e, 0x13, 0xc6, - 0xae, 0x1c, 0x51, 0x54, 0x2d, 0xfa, 0x2d, 0x6e, 0xe3, 0x1e, 0xa7, 0xd2, 0x9b, 0xdf, 0xc6, 0x05, - 0x4d, 0x39, 0x2a, 0xae, 0x5b, 0x35, 0x99, 0xa3, 0x44, 0x21, 0x7e, 0xf4, 0x08, 0x14, 0x0f, 0xe8, - 0x7e, 0xe0, 0x5f, 0xba, 0xd3, 0x0c, 0xbf, 0x3f, 0x68, 0xf0, 0x78, 0x8e, 0xf0, 0xad, 0x3d, 0x14, - 0xd5, 0xd7, 0x69, 0xad, 0xfc, 0x3a, 0x55, 0xc7, 0x44, 0xf5, 0xd2, 0x98, 0x68, 0xef, 0x0b, 0x68, - 0x88, 0x01, 0x0b, 0x5b, 0x85, 0xd6, 0xd0, 0xbf, 0xb1, 0x3d, 0x77, 0xf2, 0x22, 0x34, 0x2a, 0xac, - 0x09, 0xb5, 0x71, 0x12, 0x84, 0x86, 0xc6, 0x5a, 0x50, 0x3f, 0xc5, 0xfa, 0x60, 0xe8, 0x0c, 0xa0, - 0x81, 0xc5, 0x73, 0xc6, 0x8d, 0x2a, 0xb2, 0x69, 0x76, 0x65, 0xd4, 0x90, 0x2d, 0xa6, 0x2a, 0x46, - 0x9d, 0xad, 0x01, 0x7c, 0x9e, 0x26, 0x81, 0x54, 0x6b, 0xec, 0xed, 0x41, 0x9d, 0x66, 0x16, 0x64, - 0xf0, 0x67, 0xc3, 0x53, 0xa3, 0xc2, 0xda, 0xb0, 0x62, 0x1d, 0x9d, 0x8e, 0x3e, 0xef, 0x1f, 0x19, - 0x1a, 0xae, 0x1d, 0x3e, 0xff, 0xe9, 0x51, 0xff, 0xdc, 0xd0, 0xf7, 0x7e, 0x4e, 0x26, 0xa7, 0x58, - 0x56, 0x3a, 0xd2, 0x17, 0xa2, 0x8d, 0x0a, 0x5b, 0x81, 0xea, 0x73, 0x7e, 0x6b, 0x68, 0xb4, 0x38, - 0xf5, 0xf1, 0x91, 0x28, 0xfc, 0x21, 0xd7, 0x26, 0x46, 0x15, 0x05, 0xe8, 0x70, 0xc8, 0x27, 0x46, - 0x8d, 0x75, 0xa0, 0xf9, 0x54, 0xbe, 0xfa, 0x8c, 0xfa, 0xde, 0x0b, 0x68, 0x66, 0xcd, 0x8f, 0x3d, - 0x82, 0xb6, 0x34, 0x8d, 0x2c, 0xa3, 0x82, 0xe7, 0xa0, 0x16, 0x67, 0x68, 0xe8, 0x22, 0xb6, 0x31, - 0x43, 0xc7, 0x5f, 0xd8, 0xab, 0x8c, 0x2a, 0xb9, 0x7d, 0xe7, 0x3b, 0x46, 0x0d, 0x15, 0x29, 0xa1, - 0x8c, 0xc9, 0xde, 0x0f, 0xa0, 0x95, 0x57, 0x42, 0x74, 0xf6, 0xa5, 0x7f, 0xed, 0x07, 0xb7, 0x3e, - 0xf1, 0xc4, 0x01, 0xb1, 0x98, 0x8c, 0xcf, 0x46, 0x86, 0x86, 0x1b, 0x92, 0xfd, 0xa7, 0x74, 0xbf, - 0x30, 0xf4, 0xbd, 0x13, 0x58, 0x91, 0xe9, 0xce, 0x18, 0xac, 0x49, 0x67, 0x24, 0xc7, 0xa8, 0x60, - 0x1c, 0xf0, 0x1c, 0x62, 0x2b, 0x0d, 0xf1, 0xa4, 0x23, 0x0a, 0x5a, 0x47, 0x73, 0x02, 0x5b, 0xc1, - 0xa8, 0x1e, 0xfc, 0xb1, 0x09, 0x0d, 0x91, 0x52, 0xac, 0x0f, 0x1d, 0x75, 0x9c, 0xc8, 0xde, 0x95, - 0xd7, 0x82, 0xfb, 0x03, 0xc6, 0x5e, 0x97, 0x1a, 0xfb, 0x9c, 0x59, 0x8f, 0x59, 0x61, 0x43, 0x58, - 0x2b, 0x8f, 0xe6, 0xd8, 0x63, 0xd4, 0x9e, 0x3b, 0xf7, 0xeb, 0xf5, 0xe6, 0x89, 0x72, 0x53, 0x47, - 0xb0, 0x5a, 0x9a, 0xb6, 0x31, 0xda, 0x77, 0xde, 0x00, 0x6e, 0xa9, 0x47, 0x3f, 0x81, 0xb6, 0x32, - 0x3c, 0x62, 0x5b, 0xa8, 0xfa, 0x70, 0x32, 0xd7, 0x7b, 0xf7, 0x01, 0x3f, 0xb7, 0xf0, 0x19, 0x40, - 0x31, 0x9c, 0x61, 0xef, 0xe4, 0x8a, 0xea, 0xc0, 0xae, 0xb7, 0x75, 0x9f, 0x9d, 0x2f, 0x7f, 0x0a, - 0x20, 0xa7, 0x76, 0x67, 0xa3, 0x98, 0xbd, 0x87, 0x7a, 0x8b, 0xa6, 0x78, 0x4b, 0x0f, 0x72, 0x00, - 0x9d, 0xa7, 0x3c, 0x71, 0xae, 0xb2, 0xf6, 0x47, 0xf7, 0x7c, 0xa5, 0x55, 0xf5, 0xda, 0x92, 0x81, - 0x84, 0x59, 0xd9, 0xd5, 0x3e, 0xd1, 0xd8, 0x0f, 0x01, 0x30, 0x97, 0xd2, 0x84, 0x63, 0xe9, 0x66, - 0xd4, 0x84, 0x4b, 0x8d, 0x6a, 0xe9, 0x8e, 0x7d, 0xe8, 0xa8, 0x3d, 0x45, 0x64, 0xc4, 0x9c, 0x2e, - 0xb3, 0xd4, 0xc8, 0x09, 0xac, 0x3f, 0xe8, 0x0a, 0x02, 0x85, 0x45, 0xcd, 0xe2, 0xeb, 0x7c, 0x52, - 0x9b, 0x82, 0xf0, 0x69, 0x4e, 0x6b, 0x11, 0x46, 0xe6, 0xf5, 0x0f, 0xb3, 0xc2, 0x7e, 0x0c, 0x50, - 0x94, 0x78, 0x11, 0xd1, 0x07, 0x25, 0x7f, 0xa9, 0x17, 0xc7, 0xb0, 0xae, 0xcc, 0xd3, 0x45, 0x35, - 0x16, 0xa9, 0xf5, 0x70, 0xcc, 0xbe, 0xd4, 0x90, 0x25, 0x87, 0xbf, 0x6a, 0x59, 0x17, 0xe8, 0x2c, - 0x6a, 0x05, 0xbd, 0xf7, 0x17, 0x48, 0x55, 0x88, 0xd4, 0xe1, 0xbd, 0x80, 0x68, 0xce, 0x38, 0x7f, - 0x99, 0x63, 0x87, 0xdd, 0x7f, 0xbc, 0xde, 0xd6, 0xbe, 0x7a, 0xbd, 0xad, 0xfd, 0xfb, 0xf5, 0xb6, - 0xf6, 0xfb, 0x37, 0xdb, 0x95, 0xaf, 0xde, 0x6c, 0x57, 0xfe, 0xf9, 0x66, 0xbb, 0x72, 0xd1, 0xa0, - 0xff, 0x40, 0x7c, 0xfb, 0xbf, 0x01, 0x00, 0x00, 0xff, 0xff, 0x6e, 0x74, 0x29, 0x1e, 0x93, 0x18, - 0x00, 0x00, -} diff --git a/dm/pb/tracer.pb.go b/dm/pb/tracer.pb.go index 4646582de1..ed4e99f436 100644 --- a/dm/pb/tracer.pb.go +++ b/dm/pb/tracer.pb.go @@ -4,17 +4,16 @@ package pb import ( - "fmt" - - proto "github.com/gogo/protobuf/proto" - + context "context" + fmt "fmt" + io "io" math "math" + math_bits "math/bits" - context "golang.org/x/net/context" - + proto "github.com/gogo/protobuf/proto" grpc "google.golang.org/grpc" - - io "io" + codes "google.golang.org/grpc/codes" + status "google.golang.org/grpc/status" ) // Reference imports to suppress errors if they are not otherwise used. @@ -26,7 +25,7 @@ var _ = math.Inf // is compatible with the proto package it is being compiled against. // A compilation error at this line likely means your copy of the // proto package needs to be updated. -const _ = proto.GoGoProtoPackageIsVersion2 // please upgrade the proto package +const _ = proto.GoGoProtoPackageIsVersion3 // please upgrade the proto package type GetTSORequest struct { Id string `protobuf:"bytes,1,opt,name=id,proto3" json:"id,omitempty"` @@ -36,7 +35,7 @@ func (m *GetTSORequest) Reset() { *m = GetTSORequest{} } func (m *GetTSORequest) String() string { return proto.CompactTextString(m) } func (*GetTSORequest) ProtoMessage() {} func (*GetTSORequest) Descriptor() ([]byte, []int) { - return fileDescriptor_tracer_1adb952205408096, []int{0} + return fileDescriptor_6d422d7c66fbbd8f, []int{0} } func (m *GetTSORequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -46,15 +45,15 @@ func (m *GetTSORequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error return xxx_messageInfo_GetTSORequest.Marshal(b, m, deterministic) } else { b = b[:cap(b)] - n, err := m.MarshalTo(b) + n, err := m.MarshalToSizedBuffer(b) if err != nil { return nil, err } return b[:n], nil } } -func (dst *GetTSORequest) XXX_Merge(src proto.Message) { - xxx_messageInfo_GetTSORequest.Merge(dst, src) +func (m *GetTSORequest) XXX_Merge(src proto.Message) { + xxx_messageInfo_GetTSORequest.Merge(m, src) } func (m *GetTSORequest) XXX_Size() int { return m.Size() @@ -82,7 +81,7 @@ func (m *GetTSOResponse) Reset() { *m = GetTSOResponse{} } func (m *GetTSOResponse) String() string { return proto.CompactTextString(m) } func (*GetTSOResponse) ProtoMessage() {} func (*GetTSOResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_tracer_1adb952205408096, []int{1} + return fileDescriptor_6d422d7c66fbbd8f, []int{1} } func (m *GetTSOResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -92,15 +91,15 @@ func (m *GetTSOResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, erro return xxx_messageInfo_GetTSOResponse.Marshal(b, m, deterministic) } else { b = b[:cap(b)] - n, err := m.MarshalTo(b) + n, err := m.MarshalToSizedBuffer(b) if err != nil { return nil, err } return b[:n], nil } } -func (dst *GetTSOResponse) XXX_Merge(src proto.Message) { - xxx_messageInfo_GetTSOResponse.Merge(dst, src) +func (m *GetTSOResponse) XXX_Merge(src proto.Message) { + xxx_messageInfo_GetTSOResponse.Merge(m, src) } func (m *GetTSOResponse) XXX_Size() int { return m.Size() @@ -141,7 +140,7 @@ func (m *CommonUploadResponse) Reset() { *m = CommonUploadResponse{} } func (m *CommonUploadResponse) String() string { return proto.CompactTextString(m) } func (*CommonUploadResponse) ProtoMessage() {} func (*CommonUploadResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_tracer_1adb952205408096, []int{2} + return fileDescriptor_6d422d7c66fbbd8f, []int{2} } func (m *CommonUploadResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -151,15 +150,15 @@ func (m *CommonUploadResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte return xxx_messageInfo_CommonUploadResponse.Marshal(b, m, deterministic) } else { b = b[:cap(b)] - n, err := m.MarshalTo(b) + n, err := m.MarshalToSizedBuffer(b) if err != nil { return nil, err } return b[:n], nil } } -func (dst *CommonUploadResponse) XXX_Merge(src proto.Message) { - xxx_messageInfo_CommonUploadResponse.Merge(dst, src) +func (m *CommonUploadResponse) XXX_Merge(src proto.Message) { + xxx_messageInfo_CommonUploadResponse.Merge(m, src) } func (m *CommonUploadResponse) XXX_Size() int { return m.Size() @@ -185,14 +184,14 @@ func (m *CommonUploadResponse) GetMsg() string { } type UploadSyncerBinlogEventRequest struct { - Events []*SyncerBinlogEvent `protobuf:"bytes,1,rep,name=events" json:"events,omitempty"` + Events []*SyncerBinlogEvent `protobuf:"bytes,1,rep,name=events,proto3" json:"events,omitempty"` } func (m *UploadSyncerBinlogEventRequest) Reset() { *m = UploadSyncerBinlogEventRequest{} } func (m *UploadSyncerBinlogEventRequest) String() string { return proto.CompactTextString(m) } func (*UploadSyncerBinlogEventRequest) ProtoMessage() {} func (*UploadSyncerBinlogEventRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_tracer_1adb952205408096, []int{3} + return fileDescriptor_6d422d7c66fbbd8f, []int{3} } func (m *UploadSyncerBinlogEventRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -202,15 +201,15 @@ func (m *UploadSyncerBinlogEventRequest) XXX_Marshal(b []byte, deterministic boo return xxx_messageInfo_UploadSyncerBinlogEventRequest.Marshal(b, m, deterministic) } else { b = b[:cap(b)] - n, err := m.MarshalTo(b) + n, err := m.MarshalToSizedBuffer(b) if err != nil { return nil, err } return b[:n], nil } } -func (dst *UploadSyncerBinlogEventRequest) XXX_Merge(src proto.Message) { - xxx_messageInfo_UploadSyncerBinlogEventRequest.Merge(dst, src) +func (m *UploadSyncerBinlogEventRequest) XXX_Merge(src proto.Message) { + xxx_messageInfo_UploadSyncerBinlogEventRequest.Merge(m, src) } func (m *UploadSyncerBinlogEventRequest) XXX_Size() int { return m.Size() @@ -229,14 +228,14 @@ func (m *UploadSyncerBinlogEventRequest) GetEvents() []*SyncerBinlogEvent { } type UploadSyncerJobEventRequest struct { - Events []*SyncerJobEvent `protobuf:"bytes,1,rep,name=events" json:"events,omitempty"` + Events []*SyncerJobEvent `protobuf:"bytes,1,rep,name=events,proto3" json:"events,omitempty"` } func (m *UploadSyncerJobEventRequest) Reset() { *m = UploadSyncerJobEventRequest{} } func (m *UploadSyncerJobEventRequest) String() string { return proto.CompactTextString(m) } func (*UploadSyncerJobEventRequest) ProtoMessage() {} func (*UploadSyncerJobEventRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_tracer_1adb952205408096, []int{4} + return fileDescriptor_6d422d7c66fbbd8f, []int{4} } func (m *UploadSyncerJobEventRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -246,15 +245,15 @@ func (m *UploadSyncerJobEventRequest) XXX_Marshal(b []byte, deterministic bool) return xxx_messageInfo_UploadSyncerJobEventRequest.Marshal(b, m, deterministic) } else { b = b[:cap(b)] - n, err := m.MarshalTo(b) + n, err := m.MarshalToSizedBuffer(b) if err != nil { return nil, err } return b[:n], nil } } -func (dst *UploadSyncerJobEventRequest) XXX_Merge(src proto.Message) { - xxx_messageInfo_UploadSyncerJobEventRequest.Merge(dst, src) +func (m *UploadSyncerJobEventRequest) XXX_Merge(src proto.Message) { + xxx_messageInfo_UploadSyncerJobEventRequest.Merge(m, src) } func (m *UploadSyncerJobEventRequest) XXX_Size() int { return m.Size() @@ -280,6 +279,33 @@ func init() { proto.RegisterType((*UploadSyncerJobEventRequest)(nil), "pb.UploadSyncerJobEventRequest") } +func init() { proto.RegisterFile("tracer.proto", fileDescriptor_6d422d7c66fbbd8f) } + +var fileDescriptor_6d422d7c66fbbd8f = []byte{ + // 325 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x94, 0x92, 0x41, 0x4b, 0xc3, 0x30, + 0x18, 0x86, 0x9b, 0x16, 0x8a, 0x7e, 0xea, 0xd0, 0xcf, 0xa9, 0x61, 0x42, 0x36, 0x72, 0x1a, 0x82, + 0x03, 0xe7, 0x1f, 0x90, 0x89, 0x88, 0xbb, 0x0c, 0xba, 0x79, 0xf0, 0x24, 0xab, 0x0b, 0x63, 0xb0, + 0x35, 0xb5, 0xc9, 0x04, 0xff, 0x85, 0x3f, 0xcb, 0xe3, 0x8e, 0x1e, 0xa5, 0x3d, 0xfa, 0x27, 0x24, + 0x6d, 0x37, 0xb7, 0x3a, 0x07, 0xde, 0xf2, 0x7d, 0x79, 0xdf, 0x87, 0xbc, 0x2f, 0x81, 0x5d, 0x1d, + 0xf5, 0x9f, 0x44, 0xd4, 0x08, 0x23, 0xa9, 0x25, 0xda, 0xa1, 0x5f, 0x39, 0xcc, 0x36, 0x8f, 0xea, + 0x35, 0x58, 0x5c, 0xf0, 0x2a, 0xec, 0xdd, 0x0a, 0xdd, 0xeb, 0x76, 0x3c, 0xf1, 0x3c, 0x15, 0x4a, + 0x63, 0x09, 0xec, 0xd1, 0x80, 0x92, 0x1a, 0xa9, 0x6f, 0x7b, 0xf6, 0x68, 0xc0, 0xdb, 0x50, 0x9a, + 0x0b, 0x54, 0x28, 0x03, 0x25, 0xf0, 0x18, 0xdc, 0x48, 0xa8, 0xe9, 0x58, 0xa7, 0xaa, 0x2d, 0x2f, + 0x9f, 0x70, 0x1f, 0x9c, 0x89, 0x1a, 0x52, 0x3b, 0xb5, 0x9a, 0xa3, 0x61, 0x69, 0x45, 0x9d, 0x1a, + 0xa9, 0x3b, 0x9e, 0xad, 0x15, 0xbf, 0x82, 0xf2, 0xb5, 0x9c, 0x4c, 0x64, 0x70, 0x1f, 0x8e, 0x65, + 0x7f, 0xf0, 0x7f, 0x22, 0xef, 0x00, 0xcb, 0xbc, 0xdd, 0x34, 0x44, 0x6b, 0x14, 0x8c, 0xe5, 0xf0, + 0xe6, 0x45, 0x04, 0x7a, 0xfe, 0xfe, 0x73, 0x70, 0x85, 0x99, 0x15, 0x25, 0x35, 0xa7, 0xbe, 0xd3, + 0x3c, 0x6a, 0x84, 0x7e, 0xe3, 0xb7, 0x3a, 0x17, 0xf1, 0x3b, 0x38, 0x5d, 0x06, 0xb6, 0xa5, 0xbf, + 0x42, 0x3b, 0x2b, 0xd0, 0xf0, 0x87, 0xb6, 0x90, 0xe6, 0x8a, 0xe6, 0x17, 0x01, 0xb7, 0x97, 0x56, + 0x8c, 0x17, 0xe0, 0x66, 0xa5, 0xe1, 0x81, 0x31, 0xac, 0x34, 0x5c, 0xc1, 0xe5, 0x55, 0xd6, 0x00, + 0xb7, 0xf0, 0x01, 0x4e, 0xfe, 0x48, 0x86, 0xdc, 0x18, 0x36, 0xc7, 0xae, 0x50, 0xa3, 0x59, 0x57, + 0x2e, 0xb7, 0xb0, 0x0b, 0xe5, 0x75, 0x19, 0xb1, 0x5a, 0xe4, 0x16, 0xd2, 0x6f, 0x82, 0xb6, 0xe8, + 0x7b, 0xcc, 0xc8, 0x2c, 0x66, 0xe4, 0x33, 0x66, 0xe4, 0x2d, 0x61, 0xd6, 0x2c, 0x61, 0xd6, 0x47, + 0xc2, 0x2c, 0xdf, 0x4d, 0x7f, 0xd6, 0xe5, 0x77, 0x00, 0x00, 0x00, 0xff, 0xff, 0x89, 0x04, 0x0e, + 0x15, 0x82, 0x02, 0x00, 0x00, +} + // Reference imports to suppress errors if they are not otherwise used. var _ context.Context var _ grpc.ClientConn @@ -339,6 +365,20 @@ type TracerServer interface { UploadSyncerJobEvent(context.Context, *UploadSyncerJobEventRequest) (*CommonUploadResponse, error) } +// UnimplementedTracerServer can be embedded to have forward compatible implementations. +type UnimplementedTracerServer struct { +} + +func (*UnimplementedTracerServer) GetTSO(ctx context.Context, req *GetTSORequest) (*GetTSOResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method GetTSO not implemented") +} +func (*UnimplementedTracerServer) UploadSyncerBinlogEvent(ctx context.Context, req *UploadSyncerBinlogEventRequest) (*CommonUploadResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method UploadSyncerBinlogEvent not implemented") +} +func (*UnimplementedTracerServer) UploadSyncerJobEvent(ctx context.Context, req *UploadSyncerJobEventRequest) (*CommonUploadResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method UploadSyncerJobEvent not implemented") +} + func RegisterTracerServer(s *grpc.Server, srv TracerServer) { s.RegisterService(&_Tracer_serviceDesc, srv) } @@ -421,7 +461,7 @@ var _Tracer_serviceDesc = grpc.ServiceDesc{ func (m *GetTSORequest) Marshal() (dAtA []byte, err error) { size := m.Size() dAtA = make([]byte, size) - n, err := m.MarshalTo(dAtA) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) if err != nil { return nil, err } @@ -429,23 +469,29 @@ func (m *GetTSORequest) Marshal() (dAtA []byte, err error) { } func (m *GetTSORequest) MarshalTo(dAtA []byte) (int, error) { - var i int + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *GetTSORequest) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) _ = i var l int _ = l if len(m.Id) > 0 { - dAtA[i] = 0xa - i++ + i -= len(m.Id) + copy(dAtA[i:], m.Id) i = encodeVarintTracer(dAtA, i, uint64(len(m.Id))) - i += copy(dAtA[i:], m.Id) + i-- + dAtA[i] = 0xa } - return i, nil + return len(dAtA) - i, nil } func (m *GetTSOResponse) Marshal() (dAtA []byte, err error) { size := m.Size() dAtA = make([]byte, size) - n, err := m.MarshalTo(dAtA) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) if err != nil { return nil, err } @@ -453,38 +499,44 @@ func (m *GetTSOResponse) Marshal() (dAtA []byte, err error) { } func (m *GetTSOResponse) MarshalTo(dAtA []byte) (int, error) { - var i int + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *GetTSOResponse) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) _ = i var l int _ = l + if m.Ts != 0 { + i = encodeVarintTracer(dAtA, i, uint64(m.Ts)) + i-- + dAtA[i] = 0x18 + } + if len(m.Msg) > 0 { + i -= len(m.Msg) + copy(dAtA[i:], m.Msg) + i = encodeVarintTracer(dAtA, i, uint64(len(m.Msg))) + i-- + dAtA[i] = 0x12 + } if m.Result { - dAtA[i] = 0x8 - i++ + i-- if m.Result { dAtA[i] = 1 } else { dAtA[i] = 0 } - i++ - } - if len(m.Msg) > 0 { - dAtA[i] = 0x12 - i++ - i = encodeVarintTracer(dAtA, i, uint64(len(m.Msg))) - i += copy(dAtA[i:], m.Msg) - } - if m.Ts != 0 { - dAtA[i] = 0x18 - i++ - i = encodeVarintTracer(dAtA, i, uint64(m.Ts)) + i-- + dAtA[i] = 0x8 } - return i, nil + return len(dAtA) - i, nil } func (m *CommonUploadResponse) Marshal() (dAtA []byte, err error) { size := m.Size() dAtA = make([]byte, size) - n, err := m.MarshalTo(dAtA) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) if err != nil { return nil, err } @@ -492,33 +544,39 @@ func (m *CommonUploadResponse) Marshal() (dAtA []byte, err error) { } func (m *CommonUploadResponse) MarshalTo(dAtA []byte) (int, error) { - var i int + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *CommonUploadResponse) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) _ = i var l int _ = l + if len(m.Msg) > 0 { + i -= len(m.Msg) + copy(dAtA[i:], m.Msg) + i = encodeVarintTracer(dAtA, i, uint64(len(m.Msg))) + i-- + dAtA[i] = 0x12 + } if m.Result { - dAtA[i] = 0x8 - i++ + i-- if m.Result { dAtA[i] = 1 } else { dAtA[i] = 0 } - i++ - } - if len(m.Msg) > 0 { - dAtA[i] = 0x12 - i++ - i = encodeVarintTracer(dAtA, i, uint64(len(m.Msg))) - i += copy(dAtA[i:], m.Msg) + i-- + dAtA[i] = 0x8 } - return i, nil + return len(dAtA) - i, nil } func (m *UploadSyncerBinlogEventRequest) Marshal() (dAtA []byte, err error) { size := m.Size() dAtA = make([]byte, size) - n, err := m.MarshalTo(dAtA) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) if err != nil { return nil, err } @@ -526,29 +584,36 @@ func (m *UploadSyncerBinlogEventRequest) Marshal() (dAtA []byte, err error) { } func (m *UploadSyncerBinlogEventRequest) MarshalTo(dAtA []byte) (int, error) { - var i int + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *UploadSyncerBinlogEventRequest) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) _ = i var l int _ = l if len(m.Events) > 0 { - for _, msg := range m.Events { - dAtA[i] = 0xa - i++ - i = encodeVarintTracer(dAtA, i, uint64(msg.Size())) - n, err := msg.MarshalTo(dAtA[i:]) - if err != nil { - return 0, err + for iNdEx := len(m.Events) - 1; iNdEx >= 0; iNdEx-- { + { + size, err := m.Events[iNdEx].MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintTracer(dAtA, i, uint64(size)) } - i += n + i-- + dAtA[i] = 0xa } } - return i, nil + return len(dAtA) - i, nil } func (m *UploadSyncerJobEventRequest) Marshal() (dAtA []byte, err error) { size := m.Size() dAtA = make([]byte, size) - n, err := m.MarshalTo(dAtA) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) if err != nil { return nil, err } @@ -556,33 +621,42 @@ func (m *UploadSyncerJobEventRequest) Marshal() (dAtA []byte, err error) { } func (m *UploadSyncerJobEventRequest) MarshalTo(dAtA []byte) (int, error) { - var i int + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *UploadSyncerJobEventRequest) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) _ = i var l int _ = l if len(m.Events) > 0 { - for _, msg := range m.Events { - dAtA[i] = 0xa - i++ - i = encodeVarintTracer(dAtA, i, uint64(msg.Size())) - n, err := msg.MarshalTo(dAtA[i:]) - if err != nil { - return 0, err + for iNdEx := len(m.Events) - 1; iNdEx >= 0; iNdEx-- { + { + size, err := m.Events[iNdEx].MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintTracer(dAtA, i, uint64(size)) } - i += n + i-- + dAtA[i] = 0xa } } - return i, nil + return len(dAtA) - i, nil } func encodeVarintTracer(dAtA []byte, offset int, v uint64) int { + offset -= sovTracer(v) + base := offset for v >= 1<<7 { dAtA[offset] = uint8(v&0x7f | 0x80) v >>= 7 offset++ } dAtA[offset] = uint8(v) - return offset + 1 + return base } func (m *GetTSORequest) Size() (n int) { if m == nil { @@ -663,14 +737,7 @@ func (m *UploadSyncerJobEventRequest) Size() (n int) { } func sovTracer(x uint64) (n int) { - for { - n++ - x >>= 7 - if x == 0 { - break - } - } - return n + return (math_bits.Len64(x|1) + 6) / 7 } func sozTracer(x uint64) (n int) { return sovTracer(uint64((x << 1) ^ uint64((int64(x) >> 63)))) @@ -690,7 +757,7 @@ func (m *GetTSORequest) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - wire |= (uint64(b) & 0x7F) << shift + wire |= uint64(b&0x7F) << shift if b < 0x80 { break } @@ -718,7 +785,7 @@ func (m *GetTSORequest) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= (uint64(b) & 0x7F) << shift + stringLen |= uint64(b&0x7F) << shift if b < 0x80 { break } @@ -728,6 +795,9 @@ func (m *GetTSORequest) Unmarshal(dAtA []byte) error { return ErrInvalidLengthTracer } postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthTracer + } if postIndex > l { return io.ErrUnexpectedEOF } @@ -742,6 +812,9 @@ func (m *GetTSORequest) Unmarshal(dAtA []byte) error { if skippy < 0 { return ErrInvalidLengthTracer } + if (iNdEx + skippy) < 0 { + return ErrInvalidLengthTracer + } if (iNdEx + skippy) > l { return io.ErrUnexpectedEOF } @@ -769,7 +842,7 @@ func (m *GetTSOResponse) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - wire |= (uint64(b) & 0x7F) << shift + wire |= uint64(b&0x7F) << shift if b < 0x80 { break } @@ -797,7 +870,7 @@ func (m *GetTSOResponse) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - v |= (int(b) & 0x7F) << shift + v |= int(b&0x7F) << shift if b < 0x80 { break } @@ -817,7 +890,7 @@ func (m *GetTSOResponse) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= (uint64(b) & 0x7F) << shift + stringLen |= uint64(b&0x7F) << shift if b < 0x80 { break } @@ -827,6 +900,9 @@ func (m *GetTSOResponse) Unmarshal(dAtA []byte) error { return ErrInvalidLengthTracer } postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthTracer + } if postIndex > l { return io.ErrUnexpectedEOF } @@ -846,7 +922,7 @@ func (m *GetTSOResponse) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - m.Ts |= (int64(b) & 0x7F) << shift + m.Ts |= int64(b&0x7F) << shift if b < 0x80 { break } @@ -860,6 +936,9 @@ func (m *GetTSOResponse) Unmarshal(dAtA []byte) error { if skippy < 0 { return ErrInvalidLengthTracer } + if (iNdEx + skippy) < 0 { + return ErrInvalidLengthTracer + } if (iNdEx + skippy) > l { return io.ErrUnexpectedEOF } @@ -887,7 +966,7 @@ func (m *CommonUploadResponse) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - wire |= (uint64(b) & 0x7F) << shift + wire |= uint64(b&0x7F) << shift if b < 0x80 { break } @@ -915,7 +994,7 @@ func (m *CommonUploadResponse) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - v |= (int(b) & 0x7F) << shift + v |= int(b&0x7F) << shift if b < 0x80 { break } @@ -935,7 +1014,7 @@ func (m *CommonUploadResponse) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= (uint64(b) & 0x7F) << shift + stringLen |= uint64(b&0x7F) << shift if b < 0x80 { break } @@ -945,6 +1024,9 @@ func (m *CommonUploadResponse) Unmarshal(dAtA []byte) error { return ErrInvalidLengthTracer } postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthTracer + } if postIndex > l { return io.ErrUnexpectedEOF } @@ -959,6 +1041,9 @@ func (m *CommonUploadResponse) Unmarshal(dAtA []byte) error { if skippy < 0 { return ErrInvalidLengthTracer } + if (iNdEx + skippy) < 0 { + return ErrInvalidLengthTracer + } if (iNdEx + skippy) > l { return io.ErrUnexpectedEOF } @@ -986,7 +1071,7 @@ func (m *UploadSyncerBinlogEventRequest) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - wire |= (uint64(b) & 0x7F) << shift + wire |= uint64(b&0x7F) << shift if b < 0x80 { break } @@ -1014,7 +1099,7 @@ func (m *UploadSyncerBinlogEventRequest) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - msglen |= (int(b) & 0x7F) << shift + msglen |= int(b&0x7F) << shift if b < 0x80 { break } @@ -1023,6 +1108,9 @@ func (m *UploadSyncerBinlogEventRequest) Unmarshal(dAtA []byte) error { return ErrInvalidLengthTracer } postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthTracer + } if postIndex > l { return io.ErrUnexpectedEOF } @@ -1040,6 +1128,9 @@ func (m *UploadSyncerBinlogEventRequest) Unmarshal(dAtA []byte) error { if skippy < 0 { return ErrInvalidLengthTracer } + if (iNdEx + skippy) < 0 { + return ErrInvalidLengthTracer + } if (iNdEx + skippy) > l { return io.ErrUnexpectedEOF } @@ -1067,7 +1158,7 @@ func (m *UploadSyncerJobEventRequest) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - wire |= (uint64(b) & 0x7F) << shift + wire |= uint64(b&0x7F) << shift if b < 0x80 { break } @@ -1095,7 +1186,7 @@ func (m *UploadSyncerJobEventRequest) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - msglen |= (int(b) & 0x7F) << shift + msglen |= int(b&0x7F) << shift if b < 0x80 { break } @@ -1104,6 +1195,9 @@ func (m *UploadSyncerJobEventRequest) Unmarshal(dAtA []byte) error { return ErrInvalidLengthTracer } postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthTracer + } if postIndex > l { return io.ErrUnexpectedEOF } @@ -1121,6 +1215,9 @@ func (m *UploadSyncerJobEventRequest) Unmarshal(dAtA []byte) error { if skippy < 0 { return ErrInvalidLengthTracer } + if (iNdEx + skippy) < 0 { + return ErrInvalidLengthTracer + } if (iNdEx + skippy) > l { return io.ErrUnexpectedEOF } @@ -1136,6 +1233,7 @@ func (m *UploadSyncerJobEventRequest) Unmarshal(dAtA []byte) error { func skipTracer(dAtA []byte) (n int, err error) { l := len(dAtA) iNdEx := 0 + depth := 0 for iNdEx < l { var wire uint64 for shift := uint(0); ; shift += 7 { @@ -1167,10 +1265,8 @@ func skipTracer(dAtA []byte) (n int, err error) { break } } - return iNdEx, nil case 1: iNdEx += 8 - return iNdEx, nil case 2: var length int for shift := uint(0); ; shift += 7 { @@ -1187,80 +1283,34 @@ func skipTracer(dAtA []byte) (n int, err error) { break } } - iNdEx += length if length < 0 { return 0, ErrInvalidLengthTracer } - return iNdEx, nil + iNdEx += length case 3: - for { - var innerWire uint64 - var start int = iNdEx - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return 0, ErrIntOverflowTracer - } - if iNdEx >= l { - return 0, io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - innerWire |= (uint64(b) & 0x7F) << shift - if b < 0x80 { - break - } - } - innerWireType := int(innerWire & 0x7) - if innerWireType == 4 { - break - } - next, err := skipTracer(dAtA[start:]) - if err != nil { - return 0, err - } - iNdEx = start + next - } - return iNdEx, nil + depth++ case 4: - return iNdEx, nil + if depth == 0 { + return 0, ErrUnexpectedEndOfGroupTracer + } + depth-- case 5: iNdEx += 4 - return iNdEx, nil default: return 0, fmt.Errorf("proto: illegal wireType %d", wireType) } + if iNdEx < 0 { + return 0, ErrInvalidLengthTracer + } + if depth == 0 { + return iNdEx, nil + } } - panic("unreachable") + return 0, io.ErrUnexpectedEOF } var ( - ErrInvalidLengthTracer = fmt.Errorf("proto: negative length found during unmarshaling") - ErrIntOverflowTracer = fmt.Errorf("proto: integer overflow") + ErrInvalidLengthTracer = fmt.Errorf("proto: negative length found during unmarshaling") + ErrIntOverflowTracer = fmt.Errorf("proto: integer overflow") + ErrUnexpectedEndOfGroupTracer = fmt.Errorf("proto: unexpected end of group") ) - -func init() { proto.RegisterFile("tracer.proto", fileDescriptor_tracer_1adb952205408096) } - -var fileDescriptor_tracer_1adb952205408096 = []byte{ - // 325 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x94, 0x92, 0x41, 0x4b, 0xc3, 0x30, - 0x18, 0x86, 0x9b, 0x16, 0x8a, 0x7e, 0xea, 0xd0, 0xcf, 0xa9, 0x61, 0x42, 0x36, 0x72, 0x1a, 0x82, - 0x03, 0xe7, 0x1f, 0x90, 0x89, 0x88, 0xbb, 0x0c, 0xba, 0x79, 0xf0, 0x24, 0xab, 0x0b, 0x63, 0xb0, - 0x35, 0xb5, 0xc9, 0x04, 0xff, 0x85, 0x3f, 0xcb, 0xe3, 0x8e, 0x1e, 0xa5, 0x3d, 0xfa, 0x27, 0x24, - 0x6d, 0x37, 0xb7, 0x3a, 0x07, 0xde, 0xf2, 0x7d, 0x79, 0xdf, 0x87, 0xbc, 0x2f, 0x81, 0x5d, 0x1d, - 0xf5, 0x9f, 0x44, 0xd4, 0x08, 0x23, 0xa9, 0x25, 0xda, 0xa1, 0x5f, 0x39, 0xcc, 0x36, 0x8f, 0xea, - 0x35, 0x58, 0x5c, 0xf0, 0x2a, 0xec, 0xdd, 0x0a, 0xdd, 0xeb, 0x76, 0x3c, 0xf1, 0x3c, 0x15, 0x4a, - 0x63, 0x09, 0xec, 0xd1, 0x80, 0x92, 0x1a, 0xa9, 0x6f, 0x7b, 0xf6, 0x68, 0xc0, 0xdb, 0x50, 0x9a, - 0x0b, 0x54, 0x28, 0x03, 0x25, 0xf0, 0x18, 0xdc, 0x48, 0xa8, 0xe9, 0x58, 0xa7, 0xaa, 0x2d, 0x2f, - 0x9f, 0x70, 0x1f, 0x9c, 0x89, 0x1a, 0x52, 0x3b, 0xb5, 0x9a, 0xa3, 0x61, 0x69, 0x45, 0x9d, 0x1a, - 0xa9, 0x3b, 0x9e, 0xad, 0x15, 0xbf, 0x82, 0xf2, 0xb5, 0x9c, 0x4c, 0x64, 0x70, 0x1f, 0x8e, 0x65, - 0x7f, 0xf0, 0x7f, 0x22, 0xef, 0x00, 0xcb, 0xbc, 0xdd, 0x34, 0x44, 0x6b, 0x14, 0x8c, 0xe5, 0xf0, - 0xe6, 0x45, 0x04, 0x7a, 0xfe, 0xfe, 0x73, 0x70, 0x85, 0x99, 0x15, 0x25, 0x35, 0xa7, 0xbe, 0xd3, - 0x3c, 0x6a, 0x84, 0x7e, 0xe3, 0xb7, 0x3a, 0x17, 0xf1, 0x3b, 0x38, 0x5d, 0x06, 0xb6, 0xa5, 0xbf, - 0x42, 0x3b, 0x2b, 0xd0, 0xf0, 0x87, 0xb6, 0x90, 0xe6, 0x8a, 0xe6, 0x17, 0x01, 0xb7, 0x97, 0x56, - 0x8c, 0x17, 0xe0, 0x66, 0xa5, 0xe1, 0x81, 0x31, 0xac, 0x34, 0x5c, 0xc1, 0xe5, 0x55, 0xd6, 0x00, - 0xb7, 0xf0, 0x01, 0x4e, 0xfe, 0x48, 0x86, 0xdc, 0x18, 0x36, 0xc7, 0xae, 0x50, 0xa3, 0x59, 0x57, - 0x2e, 0xb7, 0xb0, 0x0b, 0xe5, 0x75, 0x19, 0xb1, 0x5a, 0xe4, 0x16, 0xd2, 0x6f, 0x82, 0xb6, 0xe8, - 0x7b, 0xcc, 0xc8, 0x2c, 0x66, 0xe4, 0x33, 0x66, 0xe4, 0x2d, 0x61, 0xd6, 0x2c, 0x61, 0xd6, 0x47, - 0xc2, 0x2c, 0xdf, 0x4d, 0x7f, 0xd6, 0xe5, 0x77, 0x00, 0x00, 0x00, 0xff, 0xff, 0x89, 0x04, 0x0e, - 0x15, 0x82, 0x02, 0x00, 0x00, -} diff --git a/dm/pb/tracer_base.pb.go b/dm/pb/tracer_base.pb.go index 77e5367bc3..176eb34bc3 100644 --- a/dm/pb/tracer_base.pb.go +++ b/dm/pb/tracer_base.pb.go @@ -4,13 +4,12 @@ package pb import ( - "fmt" - - proto "github.com/gogo/protobuf/proto" - + fmt "fmt" + io "io" math "math" + math_bits "math/bits" - io "io" + proto "github.com/gogo/protobuf/proto" ) // Reference imports to suppress errors if they are not otherwise used. @@ -22,7 +21,7 @@ var _ = math.Inf // is compatible with the proto package it is being compiled against. // A compilation error at this line likely means your copy of the // proto package needs to be updated. -const _ = proto.GoGoProtoPackageIsVersion2 // please upgrade the proto package +const _ = proto.GoGoProtoPackageIsVersion3 // please upgrade the proto package type TraceType int32 @@ -37,6 +36,7 @@ var TraceType_name = map[int32]string{ 1: "BinlogEvent", 2: "JobEvent", } + var TraceType_value = map[string]int32{ "DummyEvent": 0, "BinlogEvent": 1, @@ -46,8 +46,9 @@ var TraceType_value = map[string]int32{ func (x TraceType) String() string { return proto.EnumName(TraceType_name, int32(x)) } + func (TraceType) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_tracer_base_eb78fe39c9a10136, []int{0} + return fileDescriptor_be09eaf025e7bb5c, []int{0} } type BaseEvent struct { @@ -63,7 +64,7 @@ func (m *BaseEvent) Reset() { *m = BaseEvent{} } func (m *BaseEvent) String() string { return proto.CompactTextString(m) } func (*BaseEvent) ProtoMessage() {} func (*BaseEvent) Descriptor() ([]byte, []int) { - return fileDescriptor_tracer_base_eb78fe39c9a10136, []int{0} + return fileDescriptor_be09eaf025e7bb5c, []int{0} } func (m *BaseEvent) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -73,15 +74,15 @@ func (m *BaseEvent) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { return xxx_messageInfo_BaseEvent.Marshal(b, m, deterministic) } else { b = b[:cap(b)] - n, err := m.MarshalTo(b) + n, err := m.MarshalToSizedBuffer(b) if err != nil { return nil, err } return b[:n], nil } } -func (dst *BaseEvent) XXX_Merge(src proto.Message) { - xxx_messageInfo_BaseEvent.Merge(dst, src) +func (m *BaseEvent) XXX_Merge(src proto.Message) { + xxx_messageInfo_BaseEvent.Merge(m, src) } func (m *BaseEvent) XXX_Size() int { return m.Size() @@ -135,13 +136,36 @@ func (m *BaseEvent) GetType() TraceType { } func init() { - proto.RegisterType((*BaseEvent)(nil), "pb.BaseEvent") proto.RegisterEnum("pb.TraceType", TraceType_name, TraceType_value) + proto.RegisterType((*BaseEvent)(nil), "pb.BaseEvent") +} + +func init() { proto.RegisterFile("tracer_base.proto", fileDescriptor_be09eaf025e7bb5c) } + +var fileDescriptor_be09eaf025e7bb5c = []byte{ + // 246 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x3c, 0x90, 0xb1, 0x4e, 0x84, 0x40, + 0x10, 0x86, 0x19, 0xe0, 0xce, 0x63, 0xd4, 0xf3, 0x9c, 0x6a, 0x63, 0xb1, 0x41, 0x2b, 0x62, 0x41, + 0xa1, 0x9d, 0x25, 0xc1, 0xe2, 0x2c, 0xc9, 0xf5, 0x86, 0x35, 0xeb, 0x85, 0x04, 0xd8, 0x0d, 0xec, + 0x99, 0xf0, 0x16, 0x3e, 0x84, 0x0f, 0x63, 0x79, 0xa5, 0xa5, 0x81, 0x17, 0x31, 0x2c, 0x1e, 0xdd, + 0xff, 0xfd, 0x5f, 0x76, 0x67, 0x32, 0x78, 0x6d, 0x9a, 0xfc, 0x4d, 0x36, 0xaf, 0x22, 0x6f, 0x65, + 0xac, 0x1b, 0x65, 0x14, 0xb9, 0x5a, 0xdc, 0x7d, 0x01, 0x06, 0x49, 0xde, 0xca, 0xe7, 0x0f, 0x59, + 0x1b, 0xba, 0xc1, 0xd5, 0x7b, 0x51, 0xca, 0x3a, 0xaf, 0x24, 0x83, 0x10, 0xa2, 0x20, 0x9b, 0x99, + 0x08, 0xfd, 0xb2, 0xa8, 0x25, 0x73, 0x43, 0x88, 0x16, 0x99, 0xcd, 0xb4, 0x41, 0xcf, 0xb4, 0x8a, + 0x79, 0x21, 0x44, 0x5e, 0x36, 0x46, 0x62, 0x78, 0x66, 0x07, 0x6d, 0x53, 0xe6, 0xdb, 0x0f, 0x4e, + 0x38, 0x9a, 0x7d, 0xa3, 0x0e, 0x7a, 0x9b, 0xb2, 0xc5, 0x64, 0xfe, 0x91, 0x6e, 0xd1, 0x37, 0x9d, + 0x96, 0x6c, 0x19, 0x42, 0xb4, 0x7e, 0xb8, 0x8c, 0xb5, 0x88, 0x77, 0xe3, 0xa3, 0x5d, 0xa7, 0x65, + 0x66, 0xd5, 0xfd, 0x13, 0x06, 0x73, 0x45, 0x6b, 0xc4, 0xf4, 0x50, 0x55, 0x9d, 0xdd, 0x79, 0xe3, + 0xd0, 0x15, 0x9e, 0x27, 0x45, 0x5d, 0xaa, 0xfd, 0x54, 0x00, 0x5d, 0xe0, 0xea, 0x45, 0x89, 0x89, + 0xdc, 0x84, 0x7d, 0xf7, 0x1c, 0x8e, 0x3d, 0x87, 0xdf, 0x9e, 0xc3, 0xe7, 0xc0, 0x9d, 0xe3, 0xc0, + 0x9d, 0x9f, 0x81, 0x3b, 0x62, 0x69, 0xef, 0xf0, 0xf8, 0x17, 0x00, 0x00, 0xff, 0xff, 0x25, 0x73, + 0x7a, 0x49, 0x1c, 0x01, 0x00, 0x00, } + func (m *BaseEvent) Marshal() (dAtA []byte, err error) { size := m.Size() dAtA = make([]byte, size) - n, err := m.MarshalTo(dAtA) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) if err != nil { return nil, err } @@ -149,54 +173,64 @@ func (m *BaseEvent) Marshal() (dAtA []byte, err error) { } func (m *BaseEvent) MarshalTo(dAtA []byte) (int, error) { - var i int + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *BaseEvent) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) _ = i var l int _ = l - if len(m.Filename) > 0 { - dAtA[i] = 0xa - i++ - i = encodeVarintTracerBase(dAtA, i, uint64(len(m.Filename))) - i += copy(dAtA[i:], m.Filename) - } - if m.Line != 0 { - dAtA[i] = 0x10 - i++ - i = encodeVarintTracerBase(dAtA, i, uint64(m.Line)) + if m.Type != 0 { + i = encodeVarintTracerBase(dAtA, i, uint64(m.Type)) + i-- + dAtA[i] = 0x30 } - if m.Tso != 0 { - dAtA[i] = 0x18 - i++ - i = encodeVarintTracerBase(dAtA, i, uint64(m.Tso)) + if len(m.GroupID) > 0 { + i -= len(m.GroupID) + copy(dAtA[i:], m.GroupID) + i = encodeVarintTracerBase(dAtA, i, uint64(len(m.GroupID))) + i-- + dAtA[i] = 0x2a } if len(m.TraceID) > 0 { - dAtA[i] = 0x22 - i++ + i -= len(m.TraceID) + copy(dAtA[i:], m.TraceID) i = encodeVarintTracerBase(dAtA, i, uint64(len(m.TraceID))) - i += copy(dAtA[i:], m.TraceID) + i-- + dAtA[i] = 0x22 } - if len(m.GroupID) > 0 { - dAtA[i] = 0x2a - i++ - i = encodeVarintTracerBase(dAtA, i, uint64(len(m.GroupID))) - i += copy(dAtA[i:], m.GroupID) + if m.Tso != 0 { + i = encodeVarintTracerBase(dAtA, i, uint64(m.Tso)) + i-- + dAtA[i] = 0x18 } - if m.Type != 0 { - dAtA[i] = 0x30 - i++ - i = encodeVarintTracerBase(dAtA, i, uint64(m.Type)) + if m.Line != 0 { + i = encodeVarintTracerBase(dAtA, i, uint64(m.Line)) + i-- + dAtA[i] = 0x10 } - return i, nil + if len(m.Filename) > 0 { + i -= len(m.Filename) + copy(dAtA[i:], m.Filename) + i = encodeVarintTracerBase(dAtA, i, uint64(len(m.Filename))) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil } func encodeVarintTracerBase(dAtA []byte, offset int, v uint64) int { + offset -= sovTracerBase(v) + base := offset for v >= 1<<7 { dAtA[offset] = uint8(v&0x7f | 0x80) v >>= 7 offset++ } dAtA[offset] = uint8(v) - return offset + 1 + return base } func (m *BaseEvent) Size() (n int) { if m == nil { @@ -229,14 +263,7 @@ func (m *BaseEvent) Size() (n int) { } func sovTracerBase(x uint64) (n int) { - for { - n++ - x >>= 7 - if x == 0 { - break - } - } - return n + return (math_bits.Len64(x|1) + 6) / 7 } func sozTracerBase(x uint64) (n int) { return sovTracerBase(uint64((x << 1) ^ uint64((int64(x) >> 63)))) @@ -256,7 +283,7 @@ func (m *BaseEvent) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - wire |= (uint64(b) & 0x7F) << shift + wire |= uint64(b&0x7F) << shift if b < 0x80 { break } @@ -284,7 +311,7 @@ func (m *BaseEvent) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= (uint64(b) & 0x7F) << shift + stringLen |= uint64(b&0x7F) << shift if b < 0x80 { break } @@ -294,6 +321,9 @@ func (m *BaseEvent) Unmarshal(dAtA []byte) error { return ErrInvalidLengthTracerBase } postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthTracerBase + } if postIndex > l { return io.ErrUnexpectedEOF } @@ -313,7 +343,7 @@ func (m *BaseEvent) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - m.Line |= (int32(b) & 0x7F) << shift + m.Line |= int32(b&0x7F) << shift if b < 0x80 { break } @@ -332,7 +362,7 @@ func (m *BaseEvent) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - m.Tso |= (int64(b) & 0x7F) << shift + m.Tso |= int64(b&0x7F) << shift if b < 0x80 { break } @@ -351,7 +381,7 @@ func (m *BaseEvent) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= (uint64(b) & 0x7F) << shift + stringLen |= uint64(b&0x7F) << shift if b < 0x80 { break } @@ -361,6 +391,9 @@ func (m *BaseEvent) Unmarshal(dAtA []byte) error { return ErrInvalidLengthTracerBase } postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthTracerBase + } if postIndex > l { return io.ErrUnexpectedEOF } @@ -380,7 +413,7 @@ func (m *BaseEvent) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= (uint64(b) & 0x7F) << shift + stringLen |= uint64(b&0x7F) << shift if b < 0x80 { break } @@ -390,6 +423,9 @@ func (m *BaseEvent) Unmarshal(dAtA []byte) error { return ErrInvalidLengthTracerBase } postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthTracerBase + } if postIndex > l { return io.ErrUnexpectedEOF } @@ -409,7 +445,7 @@ func (m *BaseEvent) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - m.Type |= (TraceType(b) & 0x7F) << shift + m.Type |= TraceType(b&0x7F) << shift if b < 0x80 { break } @@ -423,6 +459,9 @@ func (m *BaseEvent) Unmarshal(dAtA []byte) error { if skippy < 0 { return ErrInvalidLengthTracerBase } + if (iNdEx + skippy) < 0 { + return ErrInvalidLengthTracerBase + } if (iNdEx + skippy) > l { return io.ErrUnexpectedEOF } @@ -438,6 +477,7 @@ func (m *BaseEvent) Unmarshal(dAtA []byte) error { func skipTracerBase(dAtA []byte) (n int, err error) { l := len(dAtA) iNdEx := 0 + depth := 0 for iNdEx < l { var wire uint64 for shift := uint(0); ; shift += 7 { @@ -469,10 +509,8 @@ func skipTracerBase(dAtA []byte) (n int, err error) { break } } - return iNdEx, nil case 1: iNdEx += 8 - return iNdEx, nil case 2: var length int for shift := uint(0); ; shift += 7 { @@ -489,75 +527,34 @@ func skipTracerBase(dAtA []byte) (n int, err error) { break } } - iNdEx += length if length < 0 { return 0, ErrInvalidLengthTracerBase } - return iNdEx, nil + iNdEx += length case 3: - for { - var innerWire uint64 - var start int = iNdEx - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return 0, ErrIntOverflowTracerBase - } - if iNdEx >= l { - return 0, io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - innerWire |= (uint64(b) & 0x7F) << shift - if b < 0x80 { - break - } - } - innerWireType := int(innerWire & 0x7) - if innerWireType == 4 { - break - } - next, err := skipTracerBase(dAtA[start:]) - if err != nil { - return 0, err - } - iNdEx = start + next - } - return iNdEx, nil + depth++ case 4: - return iNdEx, nil + if depth == 0 { + return 0, ErrUnexpectedEndOfGroupTracerBase + } + depth-- case 5: iNdEx += 4 - return iNdEx, nil default: return 0, fmt.Errorf("proto: illegal wireType %d", wireType) } + if iNdEx < 0 { + return 0, ErrInvalidLengthTracerBase + } + if depth == 0 { + return iNdEx, nil + } } - panic("unreachable") + return 0, io.ErrUnexpectedEOF } var ( - ErrInvalidLengthTracerBase = fmt.Errorf("proto: negative length found during unmarshaling") - ErrIntOverflowTracerBase = fmt.Errorf("proto: integer overflow") + ErrInvalidLengthTracerBase = fmt.Errorf("proto: negative length found during unmarshaling") + ErrIntOverflowTracerBase = fmt.Errorf("proto: integer overflow") + ErrUnexpectedEndOfGroupTracerBase = fmt.Errorf("proto: unexpected end of group") ) - -func init() { proto.RegisterFile("tracer_base.proto", fileDescriptor_tracer_base_eb78fe39c9a10136) } - -var fileDescriptor_tracer_base_eb78fe39c9a10136 = []byte{ - // 246 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x3c, 0x90, 0xb1, 0x4e, 0x84, 0x40, - 0x10, 0x86, 0x19, 0xe0, 0xce, 0x63, 0xd4, 0xf3, 0x9c, 0x6a, 0x63, 0xb1, 0x41, 0x2b, 0x62, 0x41, - 0xa1, 0x9d, 0x25, 0xc1, 0xe2, 0x2c, 0xc9, 0xf5, 0x86, 0x35, 0xeb, 0x85, 0x04, 0xd8, 0x0d, 0xec, - 0x99, 0xf0, 0x16, 0x3e, 0x84, 0x0f, 0x63, 0x79, 0xa5, 0xa5, 0x81, 0x17, 0x31, 0x2c, 0x1e, 0xdd, - 0xff, 0xfd, 0x5f, 0x76, 0x67, 0x32, 0x78, 0x6d, 0x9a, 0xfc, 0x4d, 0x36, 0xaf, 0x22, 0x6f, 0x65, - 0xac, 0x1b, 0x65, 0x14, 0xb9, 0x5a, 0xdc, 0x7d, 0x01, 0x06, 0x49, 0xde, 0xca, 0xe7, 0x0f, 0x59, - 0x1b, 0xba, 0xc1, 0xd5, 0x7b, 0x51, 0xca, 0x3a, 0xaf, 0x24, 0x83, 0x10, 0xa2, 0x20, 0x9b, 0x99, - 0x08, 0xfd, 0xb2, 0xa8, 0x25, 0x73, 0x43, 0x88, 0x16, 0x99, 0xcd, 0xb4, 0x41, 0xcf, 0xb4, 0x8a, - 0x79, 0x21, 0x44, 0x5e, 0x36, 0x46, 0x62, 0x78, 0x66, 0x07, 0x6d, 0x53, 0xe6, 0xdb, 0x0f, 0x4e, - 0x38, 0x9a, 0x7d, 0xa3, 0x0e, 0x7a, 0x9b, 0xb2, 0xc5, 0x64, 0xfe, 0x91, 0x6e, 0xd1, 0x37, 0x9d, - 0x96, 0x6c, 0x19, 0x42, 0xb4, 0x7e, 0xb8, 0x8c, 0xb5, 0x88, 0x77, 0xe3, 0xa3, 0x5d, 0xa7, 0x65, - 0x66, 0xd5, 0xfd, 0x13, 0x06, 0x73, 0x45, 0x6b, 0xc4, 0xf4, 0x50, 0x55, 0x9d, 0xdd, 0x79, 0xe3, - 0xd0, 0x15, 0x9e, 0x27, 0x45, 0x5d, 0xaa, 0xfd, 0x54, 0x00, 0x5d, 0xe0, 0xea, 0x45, 0x89, 0x89, - 0xdc, 0x84, 0x7d, 0xf7, 0x1c, 0x8e, 0x3d, 0x87, 0xdf, 0x9e, 0xc3, 0xe7, 0xc0, 0x9d, 0xe3, 0xc0, - 0x9d, 0x9f, 0x81, 0x3b, 0x62, 0x69, 0xef, 0xf0, 0xf8, 0x17, 0x00, 0x00, 0xff, 0xff, 0x25, 0x73, - 0x7a, 0x49, 0x1c, 0x01, 0x00, 0x00, -} diff --git a/dm/pb/tracer_syncer.pb.go b/dm/pb/tracer_syncer.pb.go index 864be1a33a..136951eb84 100644 --- a/dm/pb/tracer_syncer.pb.go +++ b/dm/pb/tracer_syncer.pb.go @@ -4,13 +4,12 @@ package pb import ( - "fmt" - - proto "github.com/gogo/protobuf/proto" - + fmt "fmt" + io "io" math "math" + math_bits "math/bits" - io "io" + proto "github.com/gogo/protobuf/proto" ) // Reference imports to suppress errors if they are not otherwise used. @@ -22,7 +21,7 @@ var _ = math.Inf // is compatible with the proto package it is being compiled against. // A compilation error at this line likely means your copy of the // proto package needs to be updated. -const _ = proto.GoGoProtoPackageIsVersion2 // please upgrade the proto package +const _ = proto.GoGoProtoPackageIsVersion3 // please upgrade the proto package type SyncerJobState int32 @@ -39,6 +38,7 @@ var SyncerJobState_name = map[int32]string{ 2: "success", 3: "error", } + var SyncerJobState_value = map[string]int32{ "init": 0, "queued": 1, @@ -49,8 +49,9 @@ var SyncerJobState_value = map[string]int32{ func (x SyncerJobState) String() string { return proto.EnumName(SyncerJobState_name, int32(x)) } + func (SyncerJobState) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_tracer_syncer_1170dba324220d8f, []int{0} + return fileDescriptor_aa4988ddb6d489fb, []int{0} } type MySQLPosition struct { @@ -62,7 +63,7 @@ func (m *MySQLPosition) Reset() { *m = MySQLPosition{} } func (m *MySQLPosition) String() string { return proto.CompactTextString(m) } func (*MySQLPosition) ProtoMessage() {} func (*MySQLPosition) Descriptor() ([]byte, []int) { - return fileDescriptor_tracer_syncer_1170dba324220d8f, []int{0} + return fileDescriptor_aa4988ddb6d489fb, []int{0} } func (m *MySQLPosition) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -72,15 +73,15 @@ func (m *MySQLPosition) XXX_Marshal(b []byte, deterministic bool) ([]byte, error return xxx_messageInfo_MySQLPosition.Marshal(b, m, deterministic) } else { b = b[:cap(b)] - n, err := m.MarshalTo(b) + n, err := m.MarshalToSizedBuffer(b) if err != nil { return nil, err } return b[:n], nil } } -func (dst *MySQLPosition) XXX_Merge(src proto.Message) { - xxx_messageInfo_MySQLPosition.Merge(dst, src) +func (m *MySQLPosition) XXX_Merge(src proto.Message) { + xxx_messageInfo_MySQLPosition.Merge(m, src) } func (m *MySQLPosition) XXX_Size() int { return m.Size() @@ -108,15 +109,15 @@ func (m *MySQLPosition) GetPos() uint32 { type SyncerState struct { SafeMode bool `protobuf:"varint,1,opt,name=safeMode,proto3" json:"safeMode,omitempty"` TryReSync bool `protobuf:"varint,2,opt,name=tryReSync,proto3" json:"tryReSync,omitempty"` - LastPos *MySQLPosition `protobuf:"bytes,3,opt,name=lastPos" json:"lastPos,omitempty"` - CurrentPos *MySQLPosition `protobuf:"bytes,4,opt,name=currentPos" json:"currentPos,omitempty"` + LastPos *MySQLPosition `protobuf:"bytes,3,opt,name=lastPos,proto3" json:"lastPos,omitempty"` + CurrentPos *MySQLPosition `protobuf:"bytes,4,opt,name=currentPos,proto3" json:"currentPos,omitempty"` } func (m *SyncerState) Reset() { *m = SyncerState{} } func (m *SyncerState) String() string { return proto.CompactTextString(m) } func (*SyncerState) ProtoMessage() {} func (*SyncerState) Descriptor() ([]byte, []int) { - return fileDescriptor_tracer_syncer_1170dba324220d8f, []int{1} + return fileDescriptor_aa4988ddb6d489fb, []int{1} } func (m *SyncerState) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -126,15 +127,15 @@ func (m *SyncerState) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) return xxx_messageInfo_SyncerState.Marshal(b, m, deterministic) } else { b = b[:cap(b)] - n, err := m.MarshalTo(b) + n, err := m.MarshalToSizedBuffer(b) if err != nil { return nil, err } return b[:n], nil } } -func (dst *SyncerState) XXX_Merge(src proto.Message) { - xxx_messageInfo_SyncerState.Merge(dst, src) +func (m *SyncerState) XXX_Merge(src proto.Message) { + xxx_messageInfo_SyncerState.Merge(m, src) } func (m *SyncerState) XXX_Size() int { return m.Size() @@ -182,7 +183,7 @@ func (m *ExecDDLInfo) Reset() { *m = ExecDDLInfo{} } func (m *ExecDDLInfo) String() string { return proto.CompactTextString(m) } func (*ExecDDLInfo) ProtoMessage() {} func (*ExecDDLInfo) Descriptor() ([]byte, []int) { - return fileDescriptor_tracer_syncer_1170dba324220d8f, []int{2} + return fileDescriptor_aa4988ddb6d489fb, []int{2} } func (m *ExecDDLInfo) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -192,15 +193,15 @@ func (m *ExecDDLInfo) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) return xxx_messageInfo_ExecDDLInfo.Marshal(b, m, deterministic) } else { b = b[:cap(b)] - n, err := m.MarshalTo(b) + n, err := m.MarshalToSizedBuffer(b) if err != nil { return nil, err } return b[:n], nil } } -func (dst *ExecDDLInfo) XXX_Merge(src proto.Message) { - xxx_messageInfo_ExecDDLInfo.Merge(dst, src) +func (m *ExecDDLInfo) XXX_Merge(src proto.Message) { + xxx_messageInfo_ExecDDLInfo.Merge(m, src) } func (m *ExecDDLInfo) XXX_Size() int { return m.Size() @@ -226,8 +227,8 @@ func (m *ExecDDLInfo) GetExec() bool { } type SyncerBinlogEvent struct { - Base *BaseEvent `protobuf:"bytes,1,opt,name=base" json:"base,omitempty"` - State *SyncerState `protobuf:"bytes,2,opt,name=state" json:"state,omitempty"` + Base *BaseEvent `protobuf:"bytes,1,opt,name=base,proto3" json:"base,omitempty"` + State *SyncerState `protobuf:"bytes,2,opt,name=state,proto3" json:"state,omitempty"` EventType int32 `protobuf:"varint,3,opt,name=eventType,proto3" json:"eventType,omitempty"` OpType int32 `protobuf:"varint,4,opt,name=opType,proto3" json:"opType,omitempty"` } @@ -236,7 +237,7 @@ func (m *SyncerBinlogEvent) Reset() { *m = SyncerBinlogEvent{} } func (m *SyncerBinlogEvent) String() string { return proto.CompactTextString(m) } func (*SyncerBinlogEvent) ProtoMessage() {} func (*SyncerBinlogEvent) Descriptor() ([]byte, []int) { - return fileDescriptor_tracer_syncer_1170dba324220d8f, []int{3} + return fileDescriptor_aa4988ddb6d489fb, []int{3} } func (m *SyncerBinlogEvent) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -246,15 +247,15 @@ func (m *SyncerBinlogEvent) XXX_Marshal(b []byte, deterministic bool) ([]byte, e return xxx_messageInfo_SyncerBinlogEvent.Marshal(b, m, deterministic) } else { b = b[:cap(b)] - n, err := m.MarshalTo(b) + n, err := m.MarshalToSizedBuffer(b) if err != nil { return nil, err } return b[:n], nil } } -func (dst *SyncerBinlogEvent) XXX_Merge(src proto.Message) { - xxx_messageInfo_SyncerBinlogEvent.Merge(dst, src) +func (m *SyncerBinlogEvent) XXX_Merge(src proto.Message) { + xxx_messageInfo_SyncerBinlogEvent.Merge(m, src) } func (m *SyncerBinlogEvent) XXX_Size() int { return m.Size() @@ -294,14 +295,14 @@ func (m *SyncerBinlogEvent) GetOpType() int32 { } type SyncerJobEvent struct { - Base *BaseEvent `protobuf:"bytes,1,opt,name=base" json:"base,omitempty"` + Base *BaseEvent `protobuf:"bytes,1,opt,name=base,proto3" json:"base,omitempty"` OpType int32 `protobuf:"varint,2,opt,name=opType,proto3" json:"opType,omitempty"` - Pos *MySQLPosition `protobuf:"bytes,3,opt,name=pos" json:"pos,omitempty"` - CurrentPos *MySQLPosition `protobuf:"bytes,4,opt,name=currentPos" json:"currentPos,omitempty"` + Pos *MySQLPosition `protobuf:"bytes,3,opt,name=pos,proto3" json:"pos,omitempty"` + CurrentPos *MySQLPosition `protobuf:"bytes,4,opt,name=currentPos,proto3" json:"currentPos,omitempty"` Sql string `protobuf:"bytes,5,opt,name=sql,proto3" json:"sql,omitempty"` - Ddls []string `protobuf:"bytes,6,rep,name=ddls" json:"ddls,omitempty"` + Ddls []string `protobuf:"bytes,6,rep,name=ddls,proto3" json:"ddls,omitempty"` ArgsChecksum uint32 `protobuf:"varint,7,opt,name=argsChecksum,proto3" json:"argsChecksum,omitempty"` - DdlInfo *ExecDDLInfo `protobuf:"bytes,8,opt,name=ddlInfo" json:"ddlInfo,omitempty"` + DdlInfo *ExecDDLInfo `protobuf:"bytes,8,opt,name=ddlInfo,proto3" json:"ddlInfo,omitempty"` QueueBucket string `protobuf:"bytes,9,opt,name=queueBucket,proto3" json:"queueBucket,omitempty"` State SyncerJobState `protobuf:"varint,10,opt,name=state,proto3,enum=pb.SyncerJobState" json:"state,omitempty"` } @@ -310,7 +311,7 @@ func (m *SyncerJobEvent) Reset() { *m = SyncerJobEvent{} } func (m *SyncerJobEvent) String() string { return proto.CompactTextString(m) } func (*SyncerJobEvent) ProtoMessage() {} func (*SyncerJobEvent) Descriptor() ([]byte, []int) { - return fileDescriptor_tracer_syncer_1170dba324220d8f, []int{4} + return fileDescriptor_aa4988ddb6d489fb, []int{4} } func (m *SyncerJobEvent) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -320,15 +321,15 @@ func (m *SyncerJobEvent) XXX_Marshal(b []byte, deterministic bool) ([]byte, erro return xxx_messageInfo_SyncerJobEvent.Marshal(b, m, deterministic) } else { b = b[:cap(b)] - n, err := m.MarshalTo(b) + n, err := m.MarshalToSizedBuffer(b) if err != nil { return nil, err } return b[:n], nil } } -func (dst *SyncerJobEvent) XXX_Merge(src proto.Message) { - xxx_messageInfo_SyncerJobEvent.Merge(dst, src) +func (m *SyncerJobEvent) XXX_Merge(src proto.Message) { + xxx_messageInfo_SyncerJobEvent.Merge(m, src) } func (m *SyncerJobEvent) XXX_Size() int { return m.Size() @@ -410,17 +411,56 @@ func (m *SyncerJobEvent) GetState() SyncerJobState { } func init() { + proto.RegisterEnum("pb.SyncerJobState", SyncerJobState_name, SyncerJobState_value) proto.RegisterType((*MySQLPosition)(nil), "pb.MySQLPosition") proto.RegisterType((*SyncerState)(nil), "pb.SyncerState") proto.RegisterType((*ExecDDLInfo)(nil), "pb.ExecDDLInfo") proto.RegisterType((*SyncerBinlogEvent)(nil), "pb.SyncerBinlogEvent") proto.RegisterType((*SyncerJobEvent)(nil), "pb.SyncerJobEvent") - proto.RegisterEnum("pb.SyncerJobState", SyncerJobState_name, SyncerJobState_value) } + +func init() { proto.RegisterFile("tracer_syncer.proto", fileDescriptor_aa4988ddb6d489fb) } + +var fileDescriptor_aa4988ddb6d489fb = []byte{ + // 510 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x9c, 0x53, 0xcf, 0x6e, 0xd3, 0x4e, + 0x10, 0xce, 0xe6, 0x7f, 0xc6, 0xbf, 0xf4, 0x97, 0x0c, 0x12, 0xb2, 0x2a, 0x64, 0x19, 0x23, 0xa4, + 0x00, 0x52, 0x24, 0x82, 0x38, 0x70, 0xe1, 0x10, 0xd2, 0x43, 0xab, 0x56, 0x2a, 0x1b, 0xee, 0xc8, + 0x7f, 0xa6, 0x25, 0x8a, 0xeb, 0x75, 0x77, 0xd7, 0xa8, 0x79, 0x0b, 0x24, 0x1e, 0x82, 0x57, 0xe1, + 0xd8, 0x23, 0x47, 0x94, 0xf0, 0x20, 0x68, 0xd7, 0x09, 0x71, 0x0f, 0x95, 0x10, 0xb7, 0xd9, 0xf9, + 0x66, 0x3e, 0xcf, 0x7c, 0xdf, 0x18, 0x1e, 0x68, 0x19, 0xc6, 0x24, 0x3f, 0xaa, 0x55, 0x16, 0x93, + 0x1c, 0xe7, 0x52, 0x68, 0x81, 0xf5, 0x3c, 0x3a, 0x1c, 0x6e, 0x81, 0x28, 0x54, 0x54, 0xa6, 0x83, + 0xd7, 0xd0, 0x3f, 0x5b, 0xcd, 0xdf, 0x9f, 0x9e, 0x0b, 0xb5, 0xd0, 0x0b, 0x91, 0x21, 0x42, 0x33, + 0x0b, 0xaf, 0xc8, 0x65, 0x3e, 0x1b, 0xf5, 0xb8, 0x8d, 0x71, 0x00, 0x8d, 0x5c, 0x28, 0xb7, 0xee, + 0xb3, 0x51, 0x9f, 0x9b, 0x30, 0xf8, 0xc6, 0xc0, 0x99, 0x5b, 0xfa, 0xb9, 0x0e, 0x35, 0xe1, 0x21, + 0x74, 0x55, 0x78, 0x41, 0x67, 0x22, 0x29, 0x3b, 0xbb, 0xfc, 0xcf, 0x1b, 0x1f, 0x41, 0x4f, 0xcb, + 0x15, 0x27, 0x53, 0x6f, 0x39, 0xba, 0x7c, 0x9f, 0xc0, 0x17, 0xd0, 0x49, 0x43, 0xa5, 0xcf, 0x85, + 0x72, 0x1b, 0x3e, 0x1b, 0x39, 0x93, 0xe1, 0x38, 0x8f, 0xc6, 0x77, 0x66, 0xe2, 0xbb, 0x0a, 0x7c, + 0x09, 0x10, 0x17, 0x52, 0x52, 0x66, 0xeb, 0x9b, 0xf7, 0xd5, 0x57, 0x8a, 0x82, 0x37, 0xe0, 0x1c, + 0xdd, 0x50, 0x3c, 0x9b, 0x9d, 0x1e, 0x67, 0x17, 0x02, 0x1f, 0x42, 0x3b, 0x15, 0xf1, 0xf2, 0x78, + 0xb6, 0x5d, 0x70, 0xfb, 0x32, 0x6b, 0xd3, 0x0d, 0xed, 0xe6, 0xb3, 0x71, 0xf0, 0x95, 0xc1, 0xb0, + 0x5c, 0x72, 0xba, 0xc8, 0x52, 0x71, 0x79, 0xf4, 0x99, 0x32, 0x8d, 0x8f, 0xa1, 0x69, 0xf4, 0xb3, + 0xfd, 0xce, 0xa4, 0x6f, 0xbe, 0x3e, 0x0d, 0x15, 0x59, 0x90, 0x5b, 0x08, 0x9f, 0x42, 0x4b, 0x19, + 0x59, 0x2c, 0x9b, 0x33, 0xf9, 0xdf, 0xd4, 0x54, 0xd4, 0xe2, 0x25, 0x6a, 0x84, 0x21, 0xd3, 0xf5, + 0x61, 0x95, 0x93, 0x5d, 0xbe, 0xc5, 0xf7, 0x09, 0x33, 0xa9, 0xc8, 0x2d, 0xd4, 0xb4, 0xd0, 0xf6, + 0x15, 0xfc, 0xaa, 0xc3, 0x41, 0x49, 0x76, 0x22, 0xa2, 0xbf, 0x1e, 0x69, 0xcf, 0x56, 0xaf, 0xb2, + 0xe1, 0x93, 0xd2, 0xda, 0x7b, 0xa5, 0x37, 0xe8, 0x3f, 0xc8, 0x6e, 0x4e, 0x46, 0x5d, 0xa7, 0x6e, + 0xcb, 0x8a, 0x6c, 0x42, 0xa3, 0x70, 0x92, 0xa4, 0xca, 0x6d, 0xfb, 0x0d, 0x73, 0x58, 0x26, 0xc6, + 0x00, 0xfe, 0x0b, 0xe5, 0xa5, 0x7a, 0xf7, 0x89, 0xe2, 0xa5, 0x2a, 0xae, 0xdc, 0x8e, 0xbd, 0xb0, + 0x3b, 0x39, 0x7c, 0x06, 0x9d, 0x24, 0x49, 0x8d, 0x79, 0x6e, 0x77, 0x2f, 0x67, 0xc5, 0x53, 0xbe, + 0xc3, 0xd1, 0x07, 0xe7, 0xba, 0xa0, 0x82, 0xa6, 0x45, 0xbc, 0x24, 0xed, 0xf6, 0xec, 0xc7, 0xab, + 0x29, 0x1c, 0xed, 0x9c, 0x01, 0x9f, 0x8d, 0x0e, 0x26, 0xb8, 0x77, 0xe6, 0x44, 0x44, 0x55, 0x73, + 0x9e, 0xbf, 0xad, 0xa8, 0x5c, 0xde, 0x78, 0x17, 0x9a, 0x8b, 0x6c, 0xa1, 0x07, 0x35, 0x04, 0x68, + 0x5b, 0xd2, 0x64, 0xc0, 0xd0, 0x81, 0x8e, 0x2a, 0xe2, 0x98, 0x94, 0x1a, 0xd4, 0xb1, 0x07, 0x2d, + 0x92, 0x52, 0xc8, 0x41, 0x63, 0xea, 0x7e, 0x5f, 0x7b, 0xec, 0x76, 0xed, 0xb1, 0x9f, 0x6b, 0x8f, + 0x7d, 0xd9, 0x78, 0xb5, 0xdb, 0x8d, 0x57, 0xfb, 0xb1, 0xf1, 0x6a, 0x51, 0xdb, 0xfe, 0x79, 0xaf, + 0x7e, 0x07, 0x00, 0x00, 0xff, 0xff, 0xae, 0x90, 0xfe, 0xb8, 0xa7, 0x03, 0x00, 0x00, +} + func (m *MySQLPosition) Marshal() (dAtA []byte, err error) { size := m.Size() dAtA = make([]byte, size) - n, err := m.MarshalTo(dAtA) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) if err != nil { return nil, err } @@ -428,28 +468,34 @@ func (m *MySQLPosition) Marshal() (dAtA []byte, err error) { } func (m *MySQLPosition) MarshalTo(dAtA []byte) (int, error) { - var i int + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *MySQLPosition) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) _ = i var l int _ = l - if len(m.Name) > 0 { - dAtA[i] = 0xa - i++ - i = encodeVarintTracerSyncer(dAtA, i, uint64(len(m.Name))) - i += copy(dAtA[i:], m.Name) - } if m.Pos != 0 { - dAtA[i] = 0x10 - i++ i = encodeVarintTracerSyncer(dAtA, i, uint64(m.Pos)) + i-- + dAtA[i] = 0x10 + } + if len(m.Name) > 0 { + i -= len(m.Name) + copy(dAtA[i:], m.Name) + i = encodeVarintTracerSyncer(dAtA, i, uint64(len(m.Name))) + i-- + dAtA[i] = 0xa } - return i, nil + return len(dAtA) - i, nil } func (m *SyncerState) Marshal() (dAtA []byte, err error) { size := m.Size() dAtA = make([]byte, size) - n, err := m.MarshalTo(dAtA) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) if err != nil { return nil, err } @@ -457,57 +503,66 @@ func (m *SyncerState) Marshal() (dAtA []byte, err error) { } func (m *SyncerState) MarshalTo(dAtA []byte) (int, error) { - var i int + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *SyncerState) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) _ = i var l int _ = l - if m.SafeMode { - dAtA[i] = 0x8 - i++ - if m.SafeMode { - dAtA[i] = 1 - } else { - dAtA[i] = 0 + if m.CurrentPos != nil { + { + size, err := m.CurrentPos.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintTracerSyncer(dAtA, i, uint64(size)) } - i++ + i-- + dAtA[i] = 0x22 + } + if m.LastPos != nil { + { + size, err := m.LastPos.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintTracerSyncer(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x1a } if m.TryReSync { - dAtA[i] = 0x10 - i++ + i-- if m.TryReSync { dAtA[i] = 1 } else { dAtA[i] = 0 } - i++ - } - if m.LastPos != nil { - dAtA[i] = 0x1a - i++ - i = encodeVarintTracerSyncer(dAtA, i, uint64(m.LastPos.Size())) - n1, err := m.LastPos.MarshalTo(dAtA[i:]) - if err != nil { - return 0, err - } - i += n1 + i-- + dAtA[i] = 0x10 } - if m.CurrentPos != nil { - dAtA[i] = 0x22 - i++ - i = encodeVarintTracerSyncer(dAtA, i, uint64(m.CurrentPos.Size())) - n2, err := m.CurrentPos.MarshalTo(dAtA[i:]) - if err != nil { - return 0, err + if m.SafeMode { + i-- + if m.SafeMode { + dAtA[i] = 1 + } else { + dAtA[i] = 0 } - i += n2 + i-- + dAtA[i] = 0x8 } - return i, nil + return len(dAtA) - i, nil } func (m *ExecDDLInfo) Marshal() (dAtA []byte, err error) { size := m.Size() dAtA = make([]byte, size) - n, err := m.MarshalTo(dAtA) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) if err != nil { return nil, err } @@ -515,33 +570,39 @@ func (m *ExecDDLInfo) Marshal() (dAtA []byte, err error) { } func (m *ExecDDLInfo) MarshalTo(dAtA []byte) (int, error) { - var i int + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *ExecDDLInfo) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) _ = i var l int _ = l - if len(m.LockID) > 0 { - dAtA[i] = 0xa - i++ - i = encodeVarintTracerSyncer(dAtA, i, uint64(len(m.LockID))) - i += copy(dAtA[i:], m.LockID) - } if m.Exec { - dAtA[i] = 0x10 - i++ + i-- if m.Exec { dAtA[i] = 1 } else { dAtA[i] = 0 } - i++ + i-- + dAtA[i] = 0x10 } - return i, nil + if len(m.LockID) > 0 { + i -= len(m.LockID) + copy(dAtA[i:], m.LockID) + i = encodeVarintTracerSyncer(dAtA, i, uint64(len(m.LockID))) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil } func (m *SyncerBinlogEvent) Marshal() (dAtA []byte, err error) { size := m.Size() dAtA = make([]byte, size) - n, err := m.MarshalTo(dAtA) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) if err != nil { return nil, err } @@ -549,47 +610,56 @@ func (m *SyncerBinlogEvent) Marshal() (dAtA []byte, err error) { } func (m *SyncerBinlogEvent) MarshalTo(dAtA []byte) (int, error) { - var i int + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *SyncerBinlogEvent) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) _ = i var l int _ = l - if m.Base != nil { - dAtA[i] = 0xa - i++ - i = encodeVarintTracerSyncer(dAtA, i, uint64(m.Base.Size())) - n3, err := m.Base.MarshalTo(dAtA[i:]) - if err != nil { - return 0, err - } - i += n3 - } - if m.State != nil { - dAtA[i] = 0x12 - i++ - i = encodeVarintTracerSyncer(dAtA, i, uint64(m.State.Size())) - n4, err := m.State.MarshalTo(dAtA[i:]) - if err != nil { - return 0, err - } - i += n4 + if m.OpType != 0 { + i = encodeVarintTracerSyncer(dAtA, i, uint64(m.OpType)) + i-- + dAtA[i] = 0x20 } if m.EventType != 0 { - dAtA[i] = 0x18 - i++ i = encodeVarintTracerSyncer(dAtA, i, uint64(m.EventType)) + i-- + dAtA[i] = 0x18 } - if m.OpType != 0 { - dAtA[i] = 0x20 - i++ - i = encodeVarintTracerSyncer(dAtA, i, uint64(m.OpType)) + if m.State != nil { + { + size, err := m.State.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintTracerSyncer(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x12 } - return i, nil + if m.Base != nil { + { + size, err := m.Base.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintTracerSyncer(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil } func (m *SyncerJobEvent) Marshal() (dAtA []byte, err error) { size := m.Size() dAtA = make([]byte, size) - n, err := m.MarshalTo(dAtA) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) if err != nil { return nil, err } @@ -597,103 +667,114 @@ func (m *SyncerJobEvent) Marshal() (dAtA []byte, err error) { } func (m *SyncerJobEvent) MarshalTo(dAtA []byte) (int, error) { - var i int + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *SyncerJobEvent) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) _ = i var l int _ = l - if m.Base != nil { - dAtA[i] = 0xa - i++ - i = encodeVarintTracerSyncer(dAtA, i, uint64(m.Base.Size())) - n5, err := m.Base.MarshalTo(dAtA[i:]) - if err != nil { - return 0, err - } - i += n5 + if m.State != 0 { + i = encodeVarintTracerSyncer(dAtA, i, uint64(m.State)) + i-- + dAtA[i] = 0x50 } - if m.OpType != 0 { - dAtA[i] = 0x10 - i++ - i = encodeVarintTracerSyncer(dAtA, i, uint64(m.OpType)) + if len(m.QueueBucket) > 0 { + i -= len(m.QueueBucket) + copy(dAtA[i:], m.QueueBucket) + i = encodeVarintTracerSyncer(dAtA, i, uint64(len(m.QueueBucket))) + i-- + dAtA[i] = 0x4a } - if m.Pos != nil { - dAtA[i] = 0x1a - i++ - i = encodeVarintTracerSyncer(dAtA, i, uint64(m.Pos.Size())) - n6, err := m.Pos.MarshalTo(dAtA[i:]) - if err != nil { - return 0, err + if m.DdlInfo != nil { + { + size, err := m.DdlInfo.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintTracerSyncer(dAtA, i, uint64(size)) } - i += n6 + i-- + dAtA[i] = 0x42 } - if m.CurrentPos != nil { - dAtA[i] = 0x22 - i++ - i = encodeVarintTracerSyncer(dAtA, i, uint64(m.CurrentPos.Size())) - n7, err := m.CurrentPos.MarshalTo(dAtA[i:]) - if err != nil { - return 0, err + if m.ArgsChecksum != 0 { + i = encodeVarintTracerSyncer(dAtA, i, uint64(m.ArgsChecksum)) + i-- + dAtA[i] = 0x38 + } + if len(m.Ddls) > 0 { + for iNdEx := len(m.Ddls) - 1; iNdEx >= 0; iNdEx-- { + i -= len(m.Ddls[iNdEx]) + copy(dAtA[i:], m.Ddls[iNdEx]) + i = encodeVarintTracerSyncer(dAtA, i, uint64(len(m.Ddls[iNdEx]))) + i-- + dAtA[i] = 0x32 } - i += n7 } if len(m.Sql) > 0 { - dAtA[i] = 0x2a - i++ + i -= len(m.Sql) + copy(dAtA[i:], m.Sql) i = encodeVarintTracerSyncer(dAtA, i, uint64(len(m.Sql))) - i += copy(dAtA[i:], m.Sql) + i-- + dAtA[i] = 0x2a } - if len(m.Ddls) > 0 { - for _, s := range m.Ddls { - dAtA[i] = 0x32 - i++ - l = len(s) - for l >= 1<<7 { - dAtA[i] = uint8(uint64(l)&0x7f | 0x80) - l >>= 7 - i++ - } - dAtA[i] = uint8(l) - i++ - i += copy(dAtA[i:], s) + if m.CurrentPos != nil { + { + size, err := m.CurrentPos.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintTracerSyncer(dAtA, i, uint64(size)) } + i-- + dAtA[i] = 0x22 } - if m.ArgsChecksum != 0 { - dAtA[i] = 0x38 - i++ - i = encodeVarintTracerSyncer(dAtA, i, uint64(m.ArgsChecksum)) - } - if m.DdlInfo != nil { - dAtA[i] = 0x42 - i++ - i = encodeVarintTracerSyncer(dAtA, i, uint64(m.DdlInfo.Size())) - n8, err := m.DdlInfo.MarshalTo(dAtA[i:]) - if err != nil { - return 0, err + if m.Pos != nil { + { + size, err := m.Pos.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintTracerSyncer(dAtA, i, uint64(size)) } - i += n8 + i-- + dAtA[i] = 0x1a } - if len(m.QueueBucket) > 0 { - dAtA[i] = 0x4a - i++ - i = encodeVarintTracerSyncer(dAtA, i, uint64(len(m.QueueBucket))) - i += copy(dAtA[i:], m.QueueBucket) + if m.OpType != 0 { + i = encodeVarintTracerSyncer(dAtA, i, uint64(m.OpType)) + i-- + dAtA[i] = 0x10 } - if m.State != 0 { - dAtA[i] = 0x50 - i++ - i = encodeVarintTracerSyncer(dAtA, i, uint64(m.State)) + if m.Base != nil { + { + size, err := m.Base.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintTracerSyncer(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0xa } - return i, nil + return len(dAtA) - i, nil } func encodeVarintTracerSyncer(dAtA []byte, offset int, v uint64) int { + offset -= sovTracerSyncer(v) + base := offset for v >= 1<<7 { dAtA[offset] = uint8(v&0x7f | 0x80) v >>= 7 offset++ } dAtA[offset] = uint8(v) - return offset + 1 + return base } func (m *MySQLPosition) Size() (n int) { if m == nil { @@ -822,14 +903,7 @@ func (m *SyncerJobEvent) Size() (n int) { } func sovTracerSyncer(x uint64) (n int) { - for { - n++ - x >>= 7 - if x == 0 { - break - } - } - return n + return (math_bits.Len64(x|1) + 6) / 7 } func sozTracerSyncer(x uint64) (n int) { return sovTracerSyncer(uint64((x << 1) ^ uint64((int64(x) >> 63)))) @@ -849,7 +923,7 @@ func (m *MySQLPosition) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - wire |= (uint64(b) & 0x7F) << shift + wire |= uint64(b&0x7F) << shift if b < 0x80 { break } @@ -877,7 +951,7 @@ func (m *MySQLPosition) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= (uint64(b) & 0x7F) << shift + stringLen |= uint64(b&0x7F) << shift if b < 0x80 { break } @@ -887,6 +961,9 @@ func (m *MySQLPosition) Unmarshal(dAtA []byte) error { return ErrInvalidLengthTracerSyncer } postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthTracerSyncer + } if postIndex > l { return io.ErrUnexpectedEOF } @@ -906,7 +983,7 @@ func (m *MySQLPosition) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - m.Pos |= (uint32(b) & 0x7F) << shift + m.Pos |= uint32(b&0x7F) << shift if b < 0x80 { break } @@ -920,6 +997,9 @@ func (m *MySQLPosition) Unmarshal(dAtA []byte) error { if skippy < 0 { return ErrInvalidLengthTracerSyncer } + if (iNdEx + skippy) < 0 { + return ErrInvalidLengthTracerSyncer + } if (iNdEx + skippy) > l { return io.ErrUnexpectedEOF } @@ -947,7 +1027,7 @@ func (m *SyncerState) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - wire |= (uint64(b) & 0x7F) << shift + wire |= uint64(b&0x7F) << shift if b < 0x80 { break } @@ -975,7 +1055,7 @@ func (m *SyncerState) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - v |= (int(b) & 0x7F) << shift + v |= int(b&0x7F) << shift if b < 0x80 { break } @@ -995,7 +1075,7 @@ func (m *SyncerState) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - v |= (int(b) & 0x7F) << shift + v |= int(b&0x7F) << shift if b < 0x80 { break } @@ -1015,7 +1095,7 @@ func (m *SyncerState) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - msglen |= (int(b) & 0x7F) << shift + msglen |= int(b&0x7F) << shift if b < 0x80 { break } @@ -1024,6 +1104,9 @@ func (m *SyncerState) Unmarshal(dAtA []byte) error { return ErrInvalidLengthTracerSyncer } postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthTracerSyncer + } if postIndex > l { return io.ErrUnexpectedEOF } @@ -1048,7 +1131,7 @@ func (m *SyncerState) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - msglen |= (int(b) & 0x7F) << shift + msglen |= int(b&0x7F) << shift if b < 0x80 { break } @@ -1057,6 +1140,9 @@ func (m *SyncerState) Unmarshal(dAtA []byte) error { return ErrInvalidLengthTracerSyncer } postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthTracerSyncer + } if postIndex > l { return io.ErrUnexpectedEOF } @@ -1076,6 +1162,9 @@ func (m *SyncerState) Unmarshal(dAtA []byte) error { if skippy < 0 { return ErrInvalidLengthTracerSyncer } + if (iNdEx + skippy) < 0 { + return ErrInvalidLengthTracerSyncer + } if (iNdEx + skippy) > l { return io.ErrUnexpectedEOF } @@ -1103,7 +1192,7 @@ func (m *ExecDDLInfo) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - wire |= (uint64(b) & 0x7F) << shift + wire |= uint64(b&0x7F) << shift if b < 0x80 { break } @@ -1131,7 +1220,7 @@ func (m *ExecDDLInfo) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= (uint64(b) & 0x7F) << shift + stringLen |= uint64(b&0x7F) << shift if b < 0x80 { break } @@ -1141,6 +1230,9 @@ func (m *ExecDDLInfo) Unmarshal(dAtA []byte) error { return ErrInvalidLengthTracerSyncer } postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthTracerSyncer + } if postIndex > l { return io.ErrUnexpectedEOF } @@ -1160,7 +1252,7 @@ func (m *ExecDDLInfo) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - v |= (int(b) & 0x7F) << shift + v |= int(b&0x7F) << shift if b < 0x80 { break } @@ -1175,6 +1267,9 @@ func (m *ExecDDLInfo) Unmarshal(dAtA []byte) error { if skippy < 0 { return ErrInvalidLengthTracerSyncer } + if (iNdEx + skippy) < 0 { + return ErrInvalidLengthTracerSyncer + } if (iNdEx + skippy) > l { return io.ErrUnexpectedEOF } @@ -1202,7 +1297,7 @@ func (m *SyncerBinlogEvent) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - wire |= (uint64(b) & 0x7F) << shift + wire |= uint64(b&0x7F) << shift if b < 0x80 { break } @@ -1230,7 +1325,7 @@ func (m *SyncerBinlogEvent) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - msglen |= (int(b) & 0x7F) << shift + msglen |= int(b&0x7F) << shift if b < 0x80 { break } @@ -1239,6 +1334,9 @@ func (m *SyncerBinlogEvent) Unmarshal(dAtA []byte) error { return ErrInvalidLengthTracerSyncer } postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthTracerSyncer + } if postIndex > l { return io.ErrUnexpectedEOF } @@ -1263,7 +1361,7 @@ func (m *SyncerBinlogEvent) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - msglen |= (int(b) & 0x7F) << shift + msglen |= int(b&0x7F) << shift if b < 0x80 { break } @@ -1272,6 +1370,9 @@ func (m *SyncerBinlogEvent) Unmarshal(dAtA []byte) error { return ErrInvalidLengthTracerSyncer } postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthTracerSyncer + } if postIndex > l { return io.ErrUnexpectedEOF } @@ -1296,7 +1397,7 @@ func (m *SyncerBinlogEvent) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - m.EventType |= (int32(b) & 0x7F) << shift + m.EventType |= int32(b&0x7F) << shift if b < 0x80 { break } @@ -1315,7 +1416,7 @@ func (m *SyncerBinlogEvent) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - m.OpType |= (int32(b) & 0x7F) << shift + m.OpType |= int32(b&0x7F) << shift if b < 0x80 { break } @@ -1329,6 +1430,9 @@ func (m *SyncerBinlogEvent) Unmarshal(dAtA []byte) error { if skippy < 0 { return ErrInvalidLengthTracerSyncer } + if (iNdEx + skippy) < 0 { + return ErrInvalidLengthTracerSyncer + } if (iNdEx + skippy) > l { return io.ErrUnexpectedEOF } @@ -1356,7 +1460,7 @@ func (m *SyncerJobEvent) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - wire |= (uint64(b) & 0x7F) << shift + wire |= uint64(b&0x7F) << shift if b < 0x80 { break } @@ -1384,7 +1488,7 @@ func (m *SyncerJobEvent) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - msglen |= (int(b) & 0x7F) << shift + msglen |= int(b&0x7F) << shift if b < 0x80 { break } @@ -1393,6 +1497,9 @@ func (m *SyncerJobEvent) Unmarshal(dAtA []byte) error { return ErrInvalidLengthTracerSyncer } postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthTracerSyncer + } if postIndex > l { return io.ErrUnexpectedEOF } @@ -1417,7 +1524,7 @@ func (m *SyncerJobEvent) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - m.OpType |= (int32(b) & 0x7F) << shift + m.OpType |= int32(b&0x7F) << shift if b < 0x80 { break } @@ -1436,7 +1543,7 @@ func (m *SyncerJobEvent) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - msglen |= (int(b) & 0x7F) << shift + msglen |= int(b&0x7F) << shift if b < 0x80 { break } @@ -1445,6 +1552,9 @@ func (m *SyncerJobEvent) Unmarshal(dAtA []byte) error { return ErrInvalidLengthTracerSyncer } postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthTracerSyncer + } if postIndex > l { return io.ErrUnexpectedEOF } @@ -1469,7 +1579,7 @@ func (m *SyncerJobEvent) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - msglen |= (int(b) & 0x7F) << shift + msglen |= int(b&0x7F) << shift if b < 0x80 { break } @@ -1478,6 +1588,9 @@ func (m *SyncerJobEvent) Unmarshal(dAtA []byte) error { return ErrInvalidLengthTracerSyncer } postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthTracerSyncer + } if postIndex > l { return io.ErrUnexpectedEOF } @@ -1502,7 +1615,7 @@ func (m *SyncerJobEvent) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= (uint64(b) & 0x7F) << shift + stringLen |= uint64(b&0x7F) << shift if b < 0x80 { break } @@ -1512,6 +1625,9 @@ func (m *SyncerJobEvent) Unmarshal(dAtA []byte) error { return ErrInvalidLengthTracerSyncer } postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthTracerSyncer + } if postIndex > l { return io.ErrUnexpectedEOF } @@ -1531,7 +1647,7 @@ func (m *SyncerJobEvent) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= (uint64(b) & 0x7F) << shift + stringLen |= uint64(b&0x7F) << shift if b < 0x80 { break } @@ -1541,6 +1657,9 @@ func (m *SyncerJobEvent) Unmarshal(dAtA []byte) error { return ErrInvalidLengthTracerSyncer } postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthTracerSyncer + } if postIndex > l { return io.ErrUnexpectedEOF } @@ -1560,7 +1679,7 @@ func (m *SyncerJobEvent) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - m.ArgsChecksum |= (uint32(b) & 0x7F) << shift + m.ArgsChecksum |= uint32(b&0x7F) << shift if b < 0x80 { break } @@ -1579,7 +1698,7 @@ func (m *SyncerJobEvent) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - msglen |= (int(b) & 0x7F) << shift + msglen |= int(b&0x7F) << shift if b < 0x80 { break } @@ -1588,6 +1707,9 @@ func (m *SyncerJobEvent) Unmarshal(dAtA []byte) error { return ErrInvalidLengthTracerSyncer } postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthTracerSyncer + } if postIndex > l { return io.ErrUnexpectedEOF } @@ -1612,7 +1734,7 @@ func (m *SyncerJobEvent) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= (uint64(b) & 0x7F) << shift + stringLen |= uint64(b&0x7F) << shift if b < 0x80 { break } @@ -1622,6 +1744,9 @@ func (m *SyncerJobEvent) Unmarshal(dAtA []byte) error { return ErrInvalidLengthTracerSyncer } postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthTracerSyncer + } if postIndex > l { return io.ErrUnexpectedEOF } @@ -1641,7 +1766,7 @@ func (m *SyncerJobEvent) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - m.State |= (SyncerJobState(b) & 0x7F) << shift + m.State |= SyncerJobState(b&0x7F) << shift if b < 0x80 { break } @@ -1655,6 +1780,9 @@ func (m *SyncerJobEvent) Unmarshal(dAtA []byte) error { if skippy < 0 { return ErrInvalidLengthTracerSyncer } + if (iNdEx + skippy) < 0 { + return ErrInvalidLengthTracerSyncer + } if (iNdEx + skippy) > l { return io.ErrUnexpectedEOF } @@ -1670,6 +1798,7 @@ func (m *SyncerJobEvent) Unmarshal(dAtA []byte) error { func skipTracerSyncer(dAtA []byte) (n int, err error) { l := len(dAtA) iNdEx := 0 + depth := 0 for iNdEx < l { var wire uint64 for shift := uint(0); ; shift += 7 { @@ -1701,10 +1830,8 @@ func skipTracerSyncer(dAtA []byte) (n int, err error) { break } } - return iNdEx, nil case 1: iNdEx += 8 - return iNdEx, nil case 2: var length int for shift := uint(0); ; shift += 7 { @@ -1721,91 +1848,34 @@ func skipTracerSyncer(dAtA []byte) (n int, err error) { break } } - iNdEx += length if length < 0 { return 0, ErrInvalidLengthTracerSyncer } - return iNdEx, nil + iNdEx += length case 3: - for { - var innerWire uint64 - var start int = iNdEx - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return 0, ErrIntOverflowTracerSyncer - } - if iNdEx >= l { - return 0, io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - innerWire |= (uint64(b) & 0x7F) << shift - if b < 0x80 { - break - } - } - innerWireType := int(innerWire & 0x7) - if innerWireType == 4 { - break - } - next, err := skipTracerSyncer(dAtA[start:]) - if err != nil { - return 0, err - } - iNdEx = start + next - } - return iNdEx, nil + depth++ case 4: - return iNdEx, nil + if depth == 0 { + return 0, ErrUnexpectedEndOfGroupTracerSyncer + } + depth-- case 5: iNdEx += 4 - return iNdEx, nil default: return 0, fmt.Errorf("proto: illegal wireType %d", wireType) } + if iNdEx < 0 { + return 0, ErrInvalidLengthTracerSyncer + } + if depth == 0 { + return iNdEx, nil + } } - panic("unreachable") + return 0, io.ErrUnexpectedEOF } var ( - ErrInvalidLengthTracerSyncer = fmt.Errorf("proto: negative length found during unmarshaling") - ErrIntOverflowTracerSyncer = fmt.Errorf("proto: integer overflow") + ErrInvalidLengthTracerSyncer = fmt.Errorf("proto: negative length found during unmarshaling") + ErrIntOverflowTracerSyncer = fmt.Errorf("proto: integer overflow") + ErrUnexpectedEndOfGroupTracerSyncer = fmt.Errorf("proto: unexpected end of group") ) - -func init() { proto.RegisterFile("tracer_syncer.proto", fileDescriptor_tracer_syncer_1170dba324220d8f) } - -var fileDescriptor_tracer_syncer_1170dba324220d8f = []byte{ - // 510 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x9c, 0x53, 0xcf, 0x6e, 0xd3, 0x4e, - 0x10, 0xce, 0xe6, 0x7f, 0xc6, 0xbf, 0xf4, 0x97, 0x0c, 0x12, 0xb2, 0x2a, 0x64, 0x19, 0x23, 0xa4, - 0x00, 0x52, 0x24, 0x82, 0x38, 0x70, 0xe1, 0x10, 0xd2, 0x43, 0xab, 0x56, 0x2a, 0x1b, 0xee, 0xc8, - 0x7f, 0xa6, 0x25, 0x8a, 0xeb, 0x75, 0x77, 0xd7, 0xa8, 0x79, 0x0b, 0x24, 0x1e, 0x82, 0x57, 0xe1, - 0xd8, 0x23, 0x47, 0x94, 0xf0, 0x20, 0x68, 0xd7, 0x09, 0x71, 0x0f, 0x95, 0x10, 0xb7, 0xd9, 0xf9, - 0x66, 0x3e, 0xcf, 0x7c, 0xdf, 0x18, 0x1e, 0x68, 0x19, 0xc6, 0x24, 0x3f, 0xaa, 0x55, 0x16, 0x93, - 0x1c, 0xe7, 0x52, 0x68, 0x81, 0xf5, 0x3c, 0x3a, 0x1c, 0x6e, 0x81, 0x28, 0x54, 0x54, 0xa6, 0x83, - 0xd7, 0xd0, 0x3f, 0x5b, 0xcd, 0xdf, 0x9f, 0x9e, 0x0b, 0xb5, 0xd0, 0x0b, 0x91, 0x21, 0x42, 0x33, - 0x0b, 0xaf, 0xc8, 0x65, 0x3e, 0x1b, 0xf5, 0xb8, 0x8d, 0x71, 0x00, 0x8d, 0x5c, 0x28, 0xb7, 0xee, - 0xb3, 0x51, 0x9f, 0x9b, 0x30, 0xf8, 0xc6, 0xc0, 0x99, 0x5b, 0xfa, 0xb9, 0x0e, 0x35, 0xe1, 0x21, - 0x74, 0x55, 0x78, 0x41, 0x67, 0x22, 0x29, 0x3b, 0xbb, 0xfc, 0xcf, 0x1b, 0x1f, 0x41, 0x4f, 0xcb, - 0x15, 0x27, 0x53, 0x6f, 0x39, 0xba, 0x7c, 0x9f, 0xc0, 0x17, 0xd0, 0x49, 0x43, 0xa5, 0xcf, 0x85, - 0x72, 0x1b, 0x3e, 0x1b, 0x39, 0x93, 0xe1, 0x38, 0x8f, 0xc6, 0x77, 0x66, 0xe2, 0xbb, 0x0a, 0x7c, - 0x09, 0x10, 0x17, 0x52, 0x52, 0x66, 0xeb, 0x9b, 0xf7, 0xd5, 0x57, 0x8a, 0x82, 0x37, 0xe0, 0x1c, - 0xdd, 0x50, 0x3c, 0x9b, 0x9d, 0x1e, 0x67, 0x17, 0x02, 0x1f, 0x42, 0x3b, 0x15, 0xf1, 0xf2, 0x78, - 0xb6, 0x5d, 0x70, 0xfb, 0x32, 0x6b, 0xd3, 0x0d, 0xed, 0xe6, 0xb3, 0x71, 0xf0, 0x95, 0xc1, 0xb0, - 0x5c, 0x72, 0xba, 0xc8, 0x52, 0x71, 0x79, 0xf4, 0x99, 0x32, 0x8d, 0x8f, 0xa1, 0x69, 0xf4, 0xb3, - 0xfd, 0xce, 0xa4, 0x6f, 0xbe, 0x3e, 0x0d, 0x15, 0x59, 0x90, 0x5b, 0x08, 0x9f, 0x42, 0x4b, 0x19, - 0x59, 0x2c, 0x9b, 0x33, 0xf9, 0xdf, 0xd4, 0x54, 0xd4, 0xe2, 0x25, 0x6a, 0x84, 0x21, 0xd3, 0xf5, - 0x61, 0x95, 0x93, 0x5d, 0xbe, 0xc5, 0xf7, 0x09, 0x33, 0xa9, 0xc8, 0x2d, 0xd4, 0xb4, 0xd0, 0xf6, - 0x15, 0xfc, 0xaa, 0xc3, 0x41, 0x49, 0x76, 0x22, 0xa2, 0xbf, 0x1e, 0x69, 0xcf, 0x56, 0xaf, 0xb2, - 0xe1, 0x93, 0xd2, 0xda, 0x7b, 0xa5, 0x37, 0xe8, 0x3f, 0xc8, 0x6e, 0x4e, 0x46, 0x5d, 0xa7, 0x6e, - 0xcb, 0x8a, 0x6c, 0x42, 0xa3, 0x70, 0x92, 0xa4, 0xca, 0x6d, 0xfb, 0x0d, 0x73, 0x58, 0x26, 0xc6, - 0x00, 0xfe, 0x0b, 0xe5, 0xa5, 0x7a, 0xf7, 0x89, 0xe2, 0xa5, 0x2a, 0xae, 0xdc, 0x8e, 0xbd, 0xb0, - 0x3b, 0x39, 0x7c, 0x06, 0x9d, 0x24, 0x49, 0x8d, 0x79, 0x6e, 0x77, 0x2f, 0x67, 0xc5, 0x53, 0xbe, - 0xc3, 0xd1, 0x07, 0xe7, 0xba, 0xa0, 0x82, 0xa6, 0x45, 0xbc, 0x24, 0xed, 0xf6, 0xec, 0xc7, 0xab, - 0x29, 0x1c, 0xed, 0x9c, 0x01, 0x9f, 0x8d, 0x0e, 0x26, 0xb8, 0x77, 0xe6, 0x44, 0x44, 0x55, 0x73, - 0x9e, 0xbf, 0xad, 0xa8, 0x5c, 0xde, 0x78, 0x17, 0x9a, 0x8b, 0x6c, 0xa1, 0x07, 0x35, 0x04, 0x68, - 0x5b, 0xd2, 0x64, 0xc0, 0xd0, 0x81, 0x8e, 0x2a, 0xe2, 0x98, 0x94, 0x1a, 0xd4, 0xb1, 0x07, 0x2d, - 0x92, 0x52, 0xc8, 0x41, 0x63, 0xea, 0x7e, 0x5f, 0x7b, 0xec, 0x76, 0xed, 0xb1, 0x9f, 0x6b, 0x8f, - 0x7d, 0xd9, 0x78, 0xb5, 0xdb, 0x8d, 0x57, 0xfb, 0xb1, 0xf1, 0x6a, 0x51, 0xdb, 0xfe, 0x79, 0xaf, - 0x7e, 0x07, 0x00, 0x00, 0xff, 0xff, 0xae, 0x90, 0xfe, 0xb8, 0xa7, 0x03, 0x00, 0x00, -} diff --git a/dm/pbmock/dmmaster.go b/dm/pbmock/dmmaster.go index 9c9f1215a2..65f94b3f9a 100644 --- a/dm/pbmock/dmmaster.go +++ b/dm/pbmock/dmmaster.go @@ -115,6 +115,26 @@ func (mr *MockMasterClientMockRecorder) MigrateWorkerRelay(arg0, arg1 interface{ return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "MigrateWorkerRelay", reflect.TypeOf((*MockMasterClient)(nil).MigrateWorkerRelay), varargs...) } +// OperateMysqlWorker mocks base method +func (m *MockMasterClient) OperateMysqlWorker(arg0 context.Context, arg1 *pb.MysqlTaskRequest, arg2 ...grpc.CallOption) (*pb.MysqlTaskResponse, error) { + m.ctrl.T.Helper() + varargs := []interface{}{arg0, arg1} + for _, a := range arg2 { + varargs = append(varargs, a) + } + ret := m.ctrl.Call(m, "OperateMysqlWorker", varargs...) + ret0, _ := ret[0].(*pb.MysqlTaskResponse) + ret1, _ := ret[1].(error) + return ret0, ret1 +} + +// OperateMysqlWorker indicates an expected call of OperateMysqlWorker +func (mr *MockMasterClientMockRecorder) OperateMysqlWorker(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, "OperateMysqlWorker", reflect.TypeOf((*MockMasterClient)(nil).OperateMysqlWorker), varargs...) +} + // OperateTask mocks base method func (m *MockMasterClient) OperateTask(arg0 context.Context, arg1 *pb.OperateTaskRequest, arg2 ...grpc.CallOption) (*pb.OperateTaskResponse, error) { m.ctrl.T.Helper() @@ -215,24 +235,24 @@ func (mr *MockMasterClientMockRecorder) QueryStatus(arg0, arg1 interface{}, arg2 return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "QueryStatus", reflect.TypeOf((*MockMasterClient)(nil).QueryStatus), varargs...) } -// RefreshWorkerTasks mocks base method -func (m *MockMasterClient) RefreshWorkerTasks(arg0 context.Context, arg1 *pb.RefreshWorkerTasksRequest, arg2 ...grpc.CallOption) (*pb.RefreshWorkerTasksResponse, error) { +// RegisterWorker mocks base method +func (m *MockMasterClient) RegisterWorker(arg0 context.Context, arg1 *pb.RegisterWorkerRequest, arg2 ...grpc.CallOption) (*pb.RegisterWorkerResponse, error) { m.ctrl.T.Helper() varargs := []interface{}{arg0, arg1} for _, a := range arg2 { varargs = append(varargs, a) } - ret := m.ctrl.Call(m, "RefreshWorkerTasks", varargs...) - ret0, _ := ret[0].(*pb.RefreshWorkerTasksResponse) + ret := m.ctrl.Call(m, "RegisterWorker", varargs...) + ret0, _ := ret[0].(*pb.RegisterWorkerResponse) ret1, _ := ret[1].(error) return ret0, ret1 } -// RefreshWorkerTasks indicates an expected call of RefreshWorkerTasks -func (mr *MockMasterClientMockRecorder) RefreshWorkerTasks(arg0, arg1 interface{}, arg2 ...interface{}) *gomock.Call { +// RegisterWorker indicates an expected call of RegisterWorker +func (mr *MockMasterClientMockRecorder) RegisterWorker(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, "RefreshWorkerTasks", reflect.TypeOf((*MockMasterClient)(nil).RefreshWorkerTasks), varargs...) + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "RegisterWorker", reflect.TypeOf((*MockMasterClient)(nil).RegisterWorker), varargs...) } // ShowDDLLocks mocks base method @@ -458,6 +478,21 @@ func (mr *MockMasterServerMockRecorder) MigrateWorkerRelay(arg0, arg1 interface{ return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "MigrateWorkerRelay", reflect.TypeOf((*MockMasterServer)(nil).MigrateWorkerRelay), arg0, arg1) } +// OperateMysqlWorker mocks base method +func (m *MockMasterServer) OperateMysqlWorker(arg0 context.Context, arg1 *pb.MysqlTaskRequest) (*pb.MysqlTaskResponse, error) { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "OperateMysqlWorker", arg0, arg1) + ret0, _ := ret[0].(*pb.MysqlTaskResponse) + ret1, _ := ret[1].(error) + return ret0, ret1 +} + +// OperateMysqlWorker indicates an expected call of OperateMysqlWorker +func (mr *MockMasterServerMockRecorder) OperateMysqlWorker(arg0, arg1 interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "OperateMysqlWorker", reflect.TypeOf((*MockMasterServer)(nil).OperateMysqlWorker), arg0, arg1) +} + // OperateTask mocks base method func (m *MockMasterServer) OperateTask(arg0 context.Context, arg1 *pb.OperateTaskRequest) (*pb.OperateTaskResponse, error) { m.ctrl.T.Helper() @@ -533,19 +568,19 @@ func (mr *MockMasterServerMockRecorder) QueryStatus(arg0, arg1 interface{}) *gom return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "QueryStatus", reflect.TypeOf((*MockMasterServer)(nil).QueryStatus), arg0, arg1) } -// RefreshWorkerTasks mocks base method -func (m *MockMasterServer) RefreshWorkerTasks(arg0 context.Context, arg1 *pb.RefreshWorkerTasksRequest) (*pb.RefreshWorkerTasksResponse, error) { +// RegisterWorker mocks base method +func (m *MockMasterServer) RegisterWorker(arg0 context.Context, arg1 *pb.RegisterWorkerRequest) (*pb.RegisterWorkerResponse, error) { m.ctrl.T.Helper() - ret := m.ctrl.Call(m, "RefreshWorkerTasks", arg0, arg1) - ret0, _ := ret[0].(*pb.RefreshWorkerTasksResponse) + ret := m.ctrl.Call(m, "RegisterWorker", arg0, arg1) + ret0, _ := ret[0].(*pb.RegisterWorkerResponse) ret1, _ := ret[1].(error) return ret0, ret1 } -// RefreshWorkerTasks indicates an expected call of RefreshWorkerTasks -func (mr *MockMasterServerMockRecorder) RefreshWorkerTasks(arg0, arg1 interface{}) *gomock.Call { +// RegisterWorker indicates an expected call of RegisterWorker +func (mr *MockMasterServerMockRecorder) RegisterWorker(arg0, arg1 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "RefreshWorkerTasks", reflect.TypeOf((*MockMasterServer)(nil).RefreshWorkerTasks), arg0, arg1) + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "RegisterWorker", reflect.TypeOf((*MockMasterServer)(nil).RegisterWorker), arg0, arg1) } // ShowDDLLocks mocks base method diff --git a/dm/pbmock/dmworker.go b/dm/pbmock/dmworker.go index 5d046b6c43..2ccb8d2bb5 100644 --- a/dm/pbmock/dmworker.go +++ b/dm/pbmock/dmworker.go @@ -136,6 +136,26 @@ func (mr *MockWorkerClientMockRecorder) MigrateRelay(arg0, arg1 interface{}, arg return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "MigrateRelay", reflect.TypeOf((*MockWorkerClient)(nil).MigrateRelay), varargs...) } +// OperateMysqlTask mocks base method +func (m *MockWorkerClient) OperateMysqlTask(arg0 context.Context, arg1 *pb.MysqlTaskRequest, arg2 ...grpc.CallOption) (*pb.MysqlTaskResponse, error) { + m.ctrl.T.Helper() + varargs := []interface{}{arg0, arg1} + for _, a := range arg2 { + varargs = append(varargs, a) + } + ret := m.ctrl.Call(m, "OperateMysqlTask", varargs...) + ret0, _ := ret[0].(*pb.MysqlTaskResponse) + ret1, _ := ret[1].(error) + return ret0, ret1 +} + +// OperateMysqlTask indicates an expected call of OperateMysqlTask +func (mr *MockWorkerClientMockRecorder) OperateMysqlTask(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, "OperateMysqlTask", reflect.TypeOf((*MockWorkerClient)(nil).OperateMysqlTask), varargs...) +} + // OperateRelay mocks base method func (m *MockWorkerClient) OperateRelay(arg0 context.Context, arg1 *pb.OperateRelayRequest, arg2 ...grpc.CallOption) (*pb.OperateRelayResponse, error) { m.ctrl.T.Helper() @@ -570,6 +590,21 @@ func (mr *MockWorkerServerMockRecorder) MigrateRelay(arg0, arg1 interface{}) *go return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "MigrateRelay", reflect.TypeOf((*MockWorkerServer)(nil).MigrateRelay), arg0, arg1) } +// OperateMysqlTask mocks base method +func (m *MockWorkerServer) OperateMysqlTask(arg0 context.Context, arg1 *pb.MysqlTaskRequest) (*pb.MysqlTaskResponse, error) { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "OperateMysqlTask", arg0, arg1) + ret0, _ := ret[0].(*pb.MysqlTaskResponse) + ret1, _ := ret[1].(error) + return ret0, ret1 +} + +// OperateMysqlTask indicates an expected call of OperateMysqlTask +func (mr *MockWorkerServerMockRecorder) OperateMysqlTask(arg0, arg1 interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "OperateMysqlTask", reflect.TypeOf((*MockWorkerServer)(nil).OperateMysqlTask), arg0, arg1) +} + // OperateRelay mocks base method func (m *MockWorkerServer) OperateRelay(arg0 context.Context, arg1 *pb.OperateRelayRequest) (*pb.OperateRelayResponse, error) { m.ctrl.T.Helper() diff --git a/dm/proto/dmmaster.proto b/dm/proto/dmmaster.proto index eec0be17df..8a109000b7 100644 --- a/dm/proto/dmmaster.proto +++ b/dm/proto/dmmaster.proto @@ -52,16 +52,15 @@ service Master { // PurgeWorkerRelay purges relay log files for some dm-workers rpc PurgeWorkerRelay(PurgeWorkerRelayRequest) returns (PurgeWorkerRelayResponse) {} - // used by dmctl, to force refresh the task -> workers mapper - // it should be used rarely only when task -> workers mapper corrupted - rpc RefreshWorkerTasks (RefreshWorkerTasksRequest) returns (RefreshWorkerTasksResponse) {} - // MigrateRelay request migrate old dm-woker to a new one. rpc MigrateWorkerRelay(MigrateWorkerRelayRequest) returns (CommonWorkerResponse) {} // CheckTask checks legality of task configuration rpc CheckTask(CheckTaskRequest) returns (CheckTaskResponse) {} + // Operate mysql-worker for server + rpc OperateMysqlWorker(MysqlTaskRequest) returns (MysqlTaskResponse) {} + // RegisterWorker register the dm-workers. rpc RegisterWorker(RegisterWorkerRequest) returns(RegisterWorkerResponse) {} } @@ -79,7 +78,7 @@ message UpdateWorkerRelayConfigRequest { message StartTaskRequest { string task = 1; // task's configuration, yaml format - repeated string workers = 2; // workers need to do start task, empty for all matched workers in deployment + repeated string sources = 2; // mysql source need to do start task, empty for all matched workers in deployment } message StartTaskResponse { @@ -101,7 +100,7 @@ message UpdateMasterConfigResponse { message OperateTaskRequest { TaskOp op = 1; // Stop / Pause / Resume string name = 2; // task's name - repeated string workers = 3; // workers need to do operation, empty for matched workers in processing the task + repeated string sources = 3; // workers need to do operation, empty for matched workers in processing the task } message OperateTaskResponse { @@ -119,7 +118,7 @@ message OperateTaskResponse { // workers need to do update, empty for all workers in processing the task message UpdateTaskRequest { string task = 1; - repeated string workers = 2; + repeated string sources = 2; } message UpdateTaskResponse { @@ -131,7 +130,7 @@ message UpdateTaskResponse { message QueryStatusListRequest { string name = 1; // task's name, empty for all tasks - repeated string workers = 2; // workers need to query, empty for all workers + repeated string sources = 2; // workers need to query, empty for all workers } message QueryStatusListResponse { @@ -142,7 +141,7 @@ message QueryStatusListResponse { message QueryErrorListRequest { string name = 1; // task's name, empty for all tasks - repeated string workers = 2; // workers need to query, empty for all workers + repeated string sources = 2; // workers need to query, empty for all workers } message QueryErrorListResponse { @@ -158,7 +157,7 @@ message QueryErrorListResponse { // if specify task and workers both, and workers not doing the task , it will return empty DDL locks message ShowDDLLocksRequest { string task = 1; - repeated string workers = 2; // workers need to query, empty for all workers + repeated string sources = 2; // workers need to query, empty for all workers } // DDLLock represents a DDL lock info (I known the name confused with DDLLockInfo, any suggestion?) @@ -193,7 +192,7 @@ message ShowDDLLocksResponse { message UnlockDDLLockRequest { string ID = 1; string replaceOwner = 2; - repeated string workers = 3; + repeated string sources = 3; bool forceRemove = 4; } @@ -211,7 +210,7 @@ message UnlockDDLLockResponse { // 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 workers = 1; + repeated string sources = 1; string task = 2; string removeLockID = 3; bool execDDL = 4; @@ -227,7 +226,7 @@ message BreakWorkerDDLLockResponse { // 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 message SwitchWorkerRelayMasterRequest { - repeated string workers = 1; + repeated string sources = 1; } message SwitchWorkerRelayMasterResponse { @@ -239,7 +238,7 @@ message SwitchWorkerRelayMasterResponse { // OperateWorkerRelayRequest represents a request for some dm-workers to operate relay unit message OperateWorkerRelayRequest { RelayOp op = 1; // Stop / Pause / Resume - repeated string workers = 2; + repeated string sources = 2; } message OperateWorkerRelayResponse { @@ -249,20 +248,6 @@ message OperateWorkerRelayResponse { repeated OperateRelayResponse workers = 4; } - -message RefreshWorkerTasksRequest { -} - -message RefreshWorkerTasksMsg { - string worker = 1; - string msg = 2; -} - -message RefreshWorkerTasksResponse { - bool result = 1; - repeated RefreshWorkerTasksMsg workers = 2; -} - message HandleSQLsRequest { string name = 1; // sub task's name SQLOp op = 2; // operation type @@ -286,7 +271,7 @@ message HandleSQLsResponse { // filename: whether purge relay log files before this filename // subDir: specify relay sub directory for @filename message PurgeWorkerRelayRequest { - repeated string workers = 1; + repeated string sources = 1; bool inactive = 2; int64 time = 3; string filename = 4; @@ -308,7 +293,6 @@ message CheckTaskResponse { string msg = 2; } - message RegisterWorkerRequest { string name = 1; string address = 2; diff --git a/dm/proto/dmworker.proto b/dm/proto/dmworker.proto index 1bd373a44b..e6721002e5 100644 --- a/dm/proto/dmworker.proto +++ b/dm/proto/dmworker.proto @@ -39,6 +39,8 @@ service Worker { rpc QueryWorkerConfig (QueryWorkerConfigRequest) returns (QueryWorkerConfigResponse) {} rpc MigrateRelay(MigrateRelayRequest) returns (CommonWorkerResponse) {} + + rpc OperateMysqlTask(MysqlTaskRequest) returns (MysqlTaskResponse) {} } message StartSubTaskRequest { @@ -448,3 +450,19 @@ message QueryWorkerConfigResponse { string sourceID = 4; // source ID string content = 5; // marshaled config content } + +enum WorkerOp { + StartWorker = 0; + UpdateConfig = 1; + StopWorker = 2; +} + +message MysqlTaskRequest { + WorkerOp op = 1; + string config = 2; +} + +message MysqlTaskResponse { + bool result = 1; + string msg = 2; +} diff --git a/dm/worker/config.go b/dm/worker/config.go index 444df4bce0..5cc227eed5 100644 --- a/dm/worker/config.go +++ b/dm/worker/config.go @@ -15,40 +15,17 @@ package worker import ( "bytes" - "context" - "database/sql" "encoding/base64" "encoding/json" "flag" "fmt" - "io/ioutil" - "math" - "math/rand" "strings" - "time" "github.com/BurntSushi/toml" - "github.com/siddontang/go-mysql/mysql" - "github.com/pingcap/dm/dm/config" - "github.com/pingcap/dm/pkg/binlog" - "github.com/pingcap/dm/pkg/conn" - "github.com/pingcap/dm/pkg/gtid" "github.com/pingcap/dm/pkg/log" "github.com/pingcap/dm/pkg/terror" - "github.com/pingcap/dm/pkg/tracing" "github.com/pingcap/dm/pkg/utils" - "github.com/pingcap/dm/relay/purger" -) - -const ( - // dbReadTimeout is readTimeout for DB connection in adjust - dbReadTimeout = "30s" - // dbGetTimeout is timeout for getting some information from DB - dbGetTimeout = 30 * time.Second - - // the default base(min) server id generated by random - defaultBaseServerID = math.MaxUint32 / 10 ) // SampleConfigFile is sample config file of dm-worker @@ -73,20 +50,8 @@ func NewConfig() *Config { fs.StringVar(&cfg.LogLevel, "L", "info", "log level: debug, info, warn, error, fatal") fs.StringVar(&cfg.LogFile, "log-file", "", "log file path") //fs.StringVar(&cfg.LogRotate, "log-rotate", "day", "log file rotate type, hour/day") - fs.StringVar(&cfg.RelayDir, "relay-dir", "./relay_log", "relay log directory") - fs.Int64Var(&cfg.Purge.Interval, "purge-interval", 60*60, "interval (seconds) try to check whether needing to purge relay log files") - fs.Int64Var(&cfg.Purge.Expires, "purge-expires", 0, "try to purge relay log files if their modified time is older than this (hours)") - fs.Int64Var(&cfg.Purge.RemainSpace, "purge-remain-space", 15, "try to purge relay log files if remain space is less than this (GB)") - fs.BoolVar(&cfg.Checker.CheckEnable, "checker-check-enable", true, "whether enable task status checker") - fs.DurationVar(&cfg.Checker.BackoffRollback.Duration, "checker-backoff-rollback", DefaultBackoffRollback, "task status checker backoff rollback interval") - fs.DurationVar(&cfg.Checker.BackoffMax.Duration, "checker-backoff-max", DefaultBackoffMax, "task status checker backoff max delay duration") - fs.BoolVar(&cfg.Tracer.Enable, "tracer-enable", false, "whether to enable tracing") - fs.StringVar(&cfg.Tracer.TracerAddr, "tracer-server-addr", "", "tracing service rpc address") - fs.IntVar(&cfg.Tracer.BatchSize, "tracer-batch-size", 20, "upload to tracing service batch size") - fs.BoolVar(&cfg.Tracer.Checksum, "tracer-checksum", false, "whether to calculate checksum of some data") fs.StringVar(&cfg.Join, "join", "", "join to an existing cluster (usage: cluster's '${advertise-client-urls}'") fs.StringVar(&cfg.Name, "name", "", "human-readable name for DM-worker member") - return cfg } @@ -102,30 +67,6 @@ type Config struct { Join string `toml:"string" json:"join" ` WorkerAddr string `toml:"worker-addr" json:"worker-addr"` - EnableGTID bool `toml:"enable-gtid" json:"enable-gtid"` - AutoFixGTID bool `toml:"auto-fix-gtid" json:"auto-fix-gtid"` - RelayDir string `toml:"relay-dir" json:"relay-dir"` - MetaDir string `toml:"meta-dir" json:"meta-dir"` - ServerID uint32 `toml:"server-id" json:"server-id"` - Flavor string `toml:"flavor" json:"flavor"` - Charset string `toml:"charset" json:"charset"` - - // relay synchronous starting point (if specified) - RelayBinLogName string `toml:"relay-binlog-name" json:"relay-binlog-name"` - RelayBinlogGTID string `toml:"relay-binlog-gtid" json:"relay-binlog-gtid"` - - SourceID string `toml:"source-id" json:"source-id"` - From config.DBConfig `toml:"from" json:"from"` - - // config items for purger - Purge purger.Config `toml:"purge" json:"purge"` - - // config items for task status checker - Checker CheckerConfig `toml:"checker" json:"checker"` - - // config items for tracer - Tracer tracing.Config `toml:"tracer" json:"tracer"` - ConfigFile string `json:"config-file"` printVersion bool @@ -204,47 +145,6 @@ func (c *Config) Parse(arguments []string) error { return terror.ErrWorkerInvalidFlag.Generate(c.flagSet.Arg(0)) } - if len(c.MetaDir) == 0 { - c.MetaDir = "./dm_worker_meta" - } - - // assign tracer id to source id - c.Tracer.Source = c.SourceID - - err = c.adjust() - if err != nil { - return err - } - return c.verify() -} - -// verify verifies the config -func (c *Config) verify() error { - if len(c.SourceID) == 0 { - return terror.ErrWorkerNeedSourceID.Generate() - } - if len(c.SourceID) > config.MaxSourceIDLength { - return terror.ErrWorkerTooLongSourceID.Generate(c.SourceID, config.MaxSourceIDLength) - } - - var err error - if len(c.RelayBinLogName) > 0 { - if !binlog.VerifyFilename(c.RelayBinLogName) { - return terror.ErrWorkerRelayBinlogName.Generate(c.RelayBinLogName) - } - } - if len(c.RelayBinlogGTID) > 0 { - _, err = gtid.ParserGTID(c.Flavor, c.RelayBinlogGTID) - if err != nil { - return terror.WithClass(terror.Annotatef(err, "relay-binlog-gtid %s", c.RelayBinlogGTID), terror.ClassDMWorker) - } - } - - _, err = c.DecryptPassword() - if err != nil { - return err - } - return nil } @@ -262,108 +162,6 @@ func (c *Config) configFromFile(path string) error { } return terror.ErrWorkerUndecodedItemFromFile.Generate(strings.Join(undecodedItems, ",")) } - - return c.verify() -} - -func (c *Config) adjust() error { - c.From.Adjust() - c.Checker.adjust() - - if c.Flavor == "" || c.ServerID == 0 { - fromDB, err := c.createFromDB() - if err != nil { - return err - } - defer fromDB.Close() - - ctx, cancel := context.WithTimeout(context.Background(), dbGetTimeout) - defer cancel() - - err = c.adjustFlavor(ctx, fromDB.DB) - if err != nil { - return err - } - - err = c.adjustServerID(ctx, fromDB.DB) - if err != nil { - return err - } - } - - return nil -} - -func (c *Config) createFromDB() (*conn.BaseDB, error) { - // decrypt password - clone, err := c.DecryptPassword() - if err != nil { - return nil, err - } - from := clone.From - from.RawDBCfg = config.DefaultRawDBConfig().SetReadTimeout(dbReadTimeout) - fromDB, err := conn.DefaultDBProvider.Apply(from) - if err != nil { - return nil, terror.WithScope(err, terror.ScopeUpstream) - } - - return fromDB, nil -} - -// adjustFlavor adjusts flavor through querying from given database -func (c *Config) adjustFlavor(ctx context.Context, db *sql.DB) (err error) { - if c.Flavor != "" { - switch c.Flavor { - case mysql.MariaDBFlavor, mysql.MySQLFlavor: - return nil - default: - return terror.ErrNotSupportedFlavor.Generate(c.Flavor) - } - } - - c.Flavor, err = utils.GetFlavor(ctx, db) - if ctx.Err() != nil { - err = terror.Annotatef(err, "time cost to get flavor info exceeds %s", dbGetTimeout) - } - return terror.WithScope(err, terror.ScopeUpstream) -} - -func (c *Config) adjustServerID(ctx context.Context, db *sql.DB) error { - if c.ServerID != 0 { - return nil - } - - serverIDs, err := getAllServerIDFunc(ctx, db) - if ctx.Err() != nil { - err = terror.Annotatef(err, "time cost to get server-id info exceeds %s", dbGetTimeout) - } - if err != nil { - return terror.WithScope(err, terror.ScopeUpstream) - } - - for i := 0; i < 5; i++ { - randomValue := uint32(rand.Intn(100000)) - randomServerID := defaultBaseServerID + randomValue - if _, ok := serverIDs[randomServerID]; ok { - continue - } - - c.ServerID = randomServerID - return nil - } - - return terror.ErrInvalidServerID.Generatef("can't find a random available server ID") -} - -// UpdateConfigFile write configure to local file -func (c *Config) UpdateConfigFile(content string) error { - if c.ConfigFile == "" { - c.ConfigFile = "dm-worker-config.bak" - } - err := ioutil.WriteFile(c.ConfigFile, []byte(content), 0666) - if err != nil { - return terror.ErrWorkerWriteConfigFile.Delegate(err) - } return nil } @@ -380,20 +178,3 @@ func (c *Config) Reload() error { return nil } - -// DecryptPassword returns a decrypted config replica in config -func (c *Config) DecryptPassword() (*Config, error) { - clone := c.Clone() - var ( - pswdFrom string - err error - ) - if len(clone.From.Password) > 0 { - pswdFrom, err = utils.Decrypt(clone.From.Password) - if err != nil { - return nil, terror.WithClass(err, terror.ClassDMWorker) - } - } - clone.From.Password = pswdFrom - return clone, nil -} diff --git a/dm/worker/config_test.go b/dm/worker/config_test.go index ee2f782ef7..8385200f25 100644 --- a/dm/worker/config_test.go +++ b/dm/worker/config_test.go @@ -29,14 +29,15 @@ import ( ) func (t *testServer) TestConfig(c *C) { - cfg := NewConfig() + cfg := &config.MysqlConfig{} - c.Assert(cfg.Parse([]string{"-config=./dm-worker.toml", "-relay-dir=./xx"}), IsNil) + c.Assert(cfg.LoadFromFile("./dm-mysql.toml"), IsNil) + cfg.RelayDir = "./xx" c.Assert(cfg.RelayDir, Equals, "./xx") c.Assert(cfg.ServerID, Equals, uint32(101)) - dir := c.MkDir() - cfg.ConfigFile = path.Join(dir, "dm-worker.toml") + // dir := c.MkDir() + // cfg.ConfigFile = path.Join(dir, "dm-worker.toml") // test clone clone1 := cfg.Clone() @@ -54,11 +55,9 @@ func (t *testServer) TestConfig(c *C) { c.Assert(originCfgStr, Matches, `(.|\n)*server-id = 101(.|\n)*`) // test update config file and reload - c.Assert(cfg.UpdateConfigFile(tomlStr), IsNil) - c.Assert(cfg.Reload(), IsNil) + c.Assert(cfg.Parse(tomlStr), IsNil) c.Assert(cfg.ServerID, Equals, uint32(100)) - c.Assert(cfg.UpdateConfigFile(originCfgStr), IsNil) - c.Assert(cfg.Reload(), IsNil) + c.Assert(cfg.Parse(originCfgStr), IsNil) c.Assert(cfg.ServerID, Equals, uint32(101)) // test decrypt password @@ -81,34 +80,35 @@ func (t *testServer) TestConfig(c *C) { dir2 := c.MkDir() configFile := path.Join(dir2, "dm-worker-invalid.toml") configContent := []byte(` -worker-addr = ":8262" +source-id = "haha" aaa = "xxx" `) err = ioutil.WriteFile(configFile, configContent, 0644) c.Assert(err, IsNil) - err = cfg.configFromFile(configFile) + err = cfg.LoadFromFile(configFile) c.Assert(err, NotNil) c.Assert(err, ErrorMatches, ".*worker config contains unknown configuration options: aaa") } func (t *testServer) TestConfigVerify(c *C) { - newConfig := func() *Config { - cfg := NewConfig() - c.Assert(cfg.Parse([]string{"-config=./dm-worker.toml", "-relay-dir=./xx"}), IsNil) + newConfig := func() *config.MysqlConfig { + cfg := &config.MysqlConfig{} + c.Assert(cfg.LoadFromFile("./dm-mysql.toml"), IsNil) + cfg.RelayDir = "./xx" return cfg } testCases := []struct { - genFunc func() *Config + genFunc func() *config.MysqlConfig errorFormat string }{ { - func() *Config { + func() *config.MysqlConfig { return newConfig() }, "", }, { - func() *Config { + func() *config.MysqlConfig { cfg := newConfig() cfg.SourceID = "" return cfg @@ -116,7 +116,7 @@ func (t *testServer) TestConfigVerify(c *C) { ".*dm-worker should bind a non-empty source ID which represents a MySQL/MariaDB instance or a replica group.*", }, { - func() *Config { + func() *config.MysqlConfig { cfg := newConfig() cfg.SourceID = "source-id-length-more-than-thirty-two" return cfg @@ -124,7 +124,7 @@ func (t *testServer) TestConfigVerify(c *C) { fmt.Sprintf(".*the length of source ID .* is more than max allowed value %d", config.MaxSourceIDLength), }, { - func() *Config { + func() *config.MysqlConfig { cfg := newConfig() cfg.RelayBinLogName = "mysql-binlog" return cfg @@ -132,7 +132,7 @@ func (t *testServer) TestConfigVerify(c *C) { ".*not valid.*", }, { - func() *Config { + func() *config.MysqlConfig { cfg := newConfig() cfg.RelayBinlogGTID = "9afe121c-40c2-11e9-9ec7-0242ac110002:1-rtc" return cfg @@ -140,7 +140,7 @@ func (t *testServer) TestConfigVerify(c *C) { ".*relay-binlog-gtid 9afe121c-40c2-11e9-9ec7-0242ac110002:1-rtc:.*", }, { - func() *Config { + func() *config.MysqlConfig { cfg := newConfig() cfg.From.Password = "not-encrypt" return cfg @@ -151,7 +151,7 @@ func (t *testServer) TestConfigVerify(c *C) { for _, tc := range testCases { cfg := tc.genFunc() - err := cfg.verify() + err := cfg.Verify() if tc.errorFormat != "" { c.Assert(err, NotNil) lines := strings.Split(err.Error(), "\n") @@ -163,7 +163,7 @@ func (t *testServer) TestConfigVerify(c *C) { } -func subtestFlavor(c *C, cfg *Config, sqlInfo, expectedFlavor, expectedError string) { +func subtestFlavor(c *C, cfg *config.MysqlConfig, sqlInfo, expectedFlavor, expectedError string) { cfg.Flavor = "" db, mock, err := sqlmock.New() c.Assert(err, IsNil) @@ -172,7 +172,7 @@ func subtestFlavor(c *C, cfg *Config, sqlInfo, expectedFlavor, expectedError str AddRow("version", sqlInfo)) mock.ExpectClose() - err = cfg.adjustFlavor(context.Background(), db) + err = cfg.AdjustFlavor(context.Background(), db) if expectedError == "" { c.Assert(err, IsNil) c.Assert(cfg.Flavor, Equals, expectedFlavor) @@ -182,15 +182,16 @@ func subtestFlavor(c *C, cfg *Config, sqlInfo, expectedFlavor, expectedError str } func (t *testServer) TestAdjustFlavor(c *C) { - cfg := NewConfig() - c.Assert(cfg.Parse([]string{"-config=./dm-worker.toml", "-relay-dir=./xx"}), IsNil) + cfg := &config.MysqlConfig{} + c.Assert(cfg.LoadFromFile("./dm-mysql.toml"), IsNil) + cfg.RelayDir = "./xx" cfg.Flavor = "mariadb" - err := cfg.adjustFlavor(context.Background(), nil) + err := cfg.AdjustFlavor(context.Background(), nil) c.Assert(err, IsNil) c.Assert(cfg.Flavor, Equals, mysql.MariaDBFlavor) cfg.Flavor = "MongoDB" - err = cfg.adjustFlavor(context.Background(), nil) + err = cfg.AdjustFlavor(context.Background(), nil) c.Assert(err, ErrorMatches, ".*flavor MongoDB not supported") subtestFlavor(c, cfg, "10.4.8-MariaDB-1:10.4.8+maria~bionic", mysql.MariaDBFlavor, "") @@ -204,14 +205,15 @@ func (t *testServer) TestAdjustServerID(c *C) { }() getAllServerIDFunc = getMockServerIDs - cfg := NewConfig() - c.Assert(cfg.Parse([]string{"-config=./dm-worker.toml", "-relay-dir=./xx"}), IsNil) + cfg := &config.MysqlConfig{} + c.Assert(cfg.LoadFromFile("./dm-mysql.toml"), IsNil) + cfg.RelayDir = "./xx" - cfg.adjustServerID(context.Background(), nil) + cfg.AdjustServerID(context.Background(), nil) c.Assert(cfg.ServerID, Equals, uint32(101)) cfg.ServerID = 0 - cfg.adjustServerID(context.Background(), nil) + cfg.AdjustServerID(context.Background(), nil) c.Assert(cfg.ServerID, Not(Equals), 0) } diff --git a/dm/worker/dm-mysql.toml b/dm/worker/dm-mysql.toml new file mode 100644 index 0000000000..91bb198bbd --- /dev/null +++ b/dm/worker/dm-mysql.toml @@ -0,0 +1,37 @@ + +#server id of slave for binlog replication +#each instance (master and slave) in replication group should have different server id +server-id = 101 + +#represents a MySQL/MariaDB instance or a replication group +source-id = "mysql-replica-01" + +#flavor: mysql/mariadb +flavor = "mysql" + +#directory that used to store relay log +relay-dir = "./relay_log" + +#enable gtid in relay log unit +enable-gtid = false + +#charset of DSN of source mysql/mariadb instance +# charset= "" + +[from] +host = "127.0.0.1" +user = "root" +password = "Up8156jArvIPymkVC+5LxkAT6rek" +port = 3306 + +#relay log purge strategy +#[purge] +#interval = 3600 +#expires = 24 +#remain-space = 15 + +#task status checker +#[checker] +#check-enable = true +#backoff-rollback = "5m" +#backoff-max = "5m" \ No newline at end of file diff --git a/dm/worker/dm-worker.toml b/dm/worker/dm-worker.toml index 51a144873a..363a808ed0 100644 --- a/dm/worker/dm-worker.toml +++ b/dm/worker/dm-worker.toml @@ -7,39 +7,4 @@ log-file = "dm-worker.log" #dm-worker listen address worker-addr = ":8262" -#server id of slave for binlog replication -#each instance (master and slave) in replication group should have different server id -server-id = 101 -#represents a MySQL/MariaDB instance or a replication group -source-id = "mysql-replica-01" - -#flavor: mysql/mariadb -flavor = "mysql" - -#directory that used to store relay log -relay-dir = "./relay_log" - -#enable gtid in relay log unit -enable-gtid = false - -#charset of DSN of source mysql/mariadb instance -# charset= "" - -[from] -host = "127.0.0.1" -user = "root" -password = "Up8156jArvIPymkVC+5LxkAT6rek" -port = 3306 - -#relay log purge strategy -#[purge] -#interval = 3600 -#expires = 24 -#remain-space = 15 - -#task status checker -#[checker] -#check-enable = true -#backoff-rollback = "5m" -#backoff-max = "5m" diff --git a/dm/worker/join.go b/dm/worker/join.go index 51e9b1b0cb..718a306cc4 100644 --- a/dm/worker/join.go +++ b/dm/worker/join.go @@ -15,6 +15,7 @@ package worker import ( "context" + "go.uber.org/zap" "strings" "time" @@ -69,35 +70,44 @@ var ( ) // KeepAlive attempts to keep the lease of the server alive forever. -func (s *Server) KeepAlive(ctx context.Context) error { +func (s *Server) KeepAlive(ctx context.Context) (bool, error) { // TODO: fetch the actual master endpoints, the master member maybe changed. endpoints := GetJoinURLs(s.cfg.Join) client, err := clientv3.NewFromURLs(endpoints) if err != nil { - return err + return false, err } - // FIXME: use a context from server. - lease, err := client.Grant(ctx, defaultKeepAliveTTL) + cliCtx, canc := context.WithTimeout(ctx, revokeLeaseTimeout) + defer canc() + lease, err := client.Grant(cliCtx, defaultKeepAliveTTL) + if err != nil { + return false, err + } k := strings.Join([]string{workerKeepAlivePath, s.cfg.WorkerAddr, s.cfg.Name}, ",") - _, err = client.Put(ctx, k, time.Now().String(), clientv3.WithLease(lease.ID)) + _, err = client.Put(cliCtx, k, time.Now().String(), clientv3.WithLease(lease.ID)) if err != nil { - return err + return false, err } ch, err := client.KeepAlive(ctx, lease.ID) + if err != nil { + return false, err + } + log.L().Info("keep alive to ", zap.String("master", s.cfg.Join)) for { select { case _, ok := <-ch: if !ok { log.L().Info("keep alive channel is closed") - return nil + return false, nil } case <-ctx.Done(): log.L().Info("server is closing, exits keepalive") ctx, cancel := context.WithTimeout(client.Ctx(), revokeLeaseTimeout) defer cancel() client.Revoke(ctx, lease.ID) - return nil + return true, nil } } + } diff --git a/dm/worker/relay.go b/dm/worker/relay.go index 36384ac518..454854178e 100644 --- a/dm/worker/relay.go +++ b/dm/worker/relay.go @@ -15,6 +15,7 @@ package worker import ( "context" + "github.com/pingcap/dm/dm/config" "sync" "github.com/pingcap/errors" @@ -52,7 +53,7 @@ type RelayHolder interface { // Result returns the result of the relay Result() *pb.ProcessResult // Update updates relay config online - Update(ctx context.Context, cfg *Config) error + Update(ctx context.Context, cfg *config.MysqlConfig) error // Migrate resets binlog name and binlog position for relay unit Migrate(ctx context.Context, binlogName string, binlogPos uint32) error } @@ -67,7 +68,7 @@ type realRelayHolder struct { wg sync.WaitGroup relay relay.Process - cfg *Config + cfg *config.MysqlConfig ctx context.Context cancel context.CancelFunc @@ -80,7 +81,7 @@ type realRelayHolder struct { } // NewRealRelayHolder creates a new RelayHolder -func NewRealRelayHolder(cfg *Config) RelayHolder { +func NewRealRelayHolder(cfg *config.MysqlConfig) RelayHolder { clone, _ := cfg.DecryptPassword() relayCfg := &relay.Config{ EnableGTID: clone.EnableGTID, @@ -320,7 +321,7 @@ func (h *realRelayHolder) Result() *pb.ProcessResult { } // Update update relay config online -func (h *realRelayHolder) Update(ctx context.Context, cfg *Config) error { +func (h *realRelayHolder) Update(ctx context.Context, cfg *config.MysqlConfig) error { relayCfg := &relay.Config{ AutoFixGTID: cfg.AutoFixGTID, Charset: cfg.Charset, @@ -383,18 +384,18 @@ func (h *realRelayHolder) Migrate(ctx context.Context, binlogName string, binlog type dummyRelayHolder struct { initError error - cfg *Config + cfg *config.MysqlConfig } // NewDummyRelayHolder creates a new RelayHolder -func NewDummyRelayHolder(cfg *Config) RelayHolder { +func NewDummyRelayHolder(cfg *config.MysqlConfig) RelayHolder { return &dummyRelayHolder{ cfg: cfg, } } // NewDummyRelayHolderWithInitError creates a new RelayHolder with init error -func NewDummyRelayHolderWithInitError(cfg *Config) RelayHolder { +func NewDummyRelayHolderWithInitError(cfg *config.MysqlConfig) RelayHolder { return &dummyRelayHolder{ initError: errors.New("init error"), cfg: cfg, @@ -443,7 +444,7 @@ func (d *dummyRelayHolder) Result() *pb.ProcessResult { } // Update implements interface of RelayHolder -func (d *dummyRelayHolder) Update(ctx context.Context, cfg *Config) error { +func (d *dummyRelayHolder) Update(ctx context.Context, cfg *config.MysqlConfig) error { return nil } diff --git a/dm/worker/relay_test.go b/dm/worker/relay_test.go index cb523dbce5..350fc159cd 100644 --- a/dm/worker/relay_test.go +++ b/dm/worker/relay_test.go @@ -134,8 +134,8 @@ func (t *testRelay) TestRelay(c *C) { purger.NewPurger = originNewPurger }() - cfg := NewConfig() - c.Assert(cfg.Parse([]string{"-config=./dm-worker.toml"}), IsNil) + cfg := &config.MysqlConfig{} + c.Assert(cfg.LoadFromFile("./dm-mysql.toml"), IsNil) dir := c.MkDir() cfg.RelayDir = dir @@ -271,7 +271,7 @@ func (t *testRelay) testPauseAndResume(c *C, holder *realRelayHolder) { } func (t *testRelay) testUpdate(c *C, holder *realRelayHolder) { - cfg := &Config{ + cfg := &config.MysqlConfig{ From: config.DBConfig{ Host: "127.0.0.1", Port: 3306, diff --git a/dm/worker/server.go b/dm/worker/server.go index 64ea2705e3..99d2c35650 100644 --- a/dm/worker/server.go +++ b/dm/worker/server.go @@ -44,6 +44,8 @@ type Server struct { sync.Mutex wg sync.WaitGroup closed sync2.AtomicBool + ctx context.Context + cancel context.CancelFunc cfg *Config @@ -75,23 +77,41 @@ func (s *Server) Start() error { return terror.ErrWorkerStartService.Delegate(err) } - s.worker, err = NewWorker(s.cfg) - if err != nil { - return err - } - - s.wg.Add(2) - go func() { - defer s.wg.Done() - // start running worker to handle requests - s.worker.Start() - }() - + s.worker = nil + s.ctx, s.cancel = context.WithCancel(context.Background()) + s.wg.Add(1) go func() { defer s.wg.Done() // worker keepalive with master - // FIXME: use global context - s.KeepAlive(s.worker.ctx) + // If worker loses connect from master, it would stop all task and try to connect master again. + shouldExit := false + shouldStop := false + for !shouldExit { + shouldExit, err = s.KeepAlive(s.ctx) + if err != nil || !shouldExit { + if shouldStop { + s.Lock() + if s.worker != nil { + s.worker.Close() + s.worker = nil + } + s.Unlock() + shouldStop = false + } else { + // Try to connect master again before stop worker + shouldStop = true + } + ch := time.NewTicker(5 * time.Second) + select { + case <-s.ctx.Done(): + shouldExit = true + break + case <-ch.C: + // Try to connect master again + break + } + } + } }() // create a cmux @@ -121,8 +141,7 @@ func (s *Server) Start() error { return terror.ErrWorkerStartService.Delegate(err) } -// Close close the RPC server, this function can be called multiple times -func (s *Server) Close() { +func (s *Server) doClose() { s.Lock() defer s.Unlock() if s.closed.Get() { @@ -143,18 +162,28 @@ func (s *Server) Close() { } // close worker and wait for return + s.cancel() if s.worker != nil { s.worker.Close() - s.wg.Wait() } - s.closed.Set(true) } +// Close close the RPC server, this function can be called multiple times +func (s *Server) Close() { + s.doClose() + s.wg.Wait() +} + +func (s *Server) checkWorkerStart() *Worker { + s.Lock() + defer s.Unlock() + return s.worker +} + // StartSubTask implements WorkerServer.StartSubTask func (s *Server) StartSubTask(ctx context.Context, req *pb.StartSubTaskRequest) (*pb.CommonWorkerResponse, error) { log.L().Info("", zap.String("request", "StartSubTask"), zap.Stringer("payload", req)) - cfg := config.NewSubTaskConfig() err := cfg.Decode(req.Task) if err != nil { @@ -165,40 +194,55 @@ func (s *Server) StartSubTask(ctx context.Context, req *pb.StartSubTaskRequest) Msg: err.Error(), }, nil } + resp := &pb.CommonWorkerResponse{ + Result: true, + Msg: "", + } + w := s.checkWorkerStart() + if w == nil { + log.L().Error("fail to call StartSubTask, because mysql worker has not been started") + resp.Result = false + resp.Msg = terror.ErrWorkerNoStart.Error() + return resp, nil + } + + cfg.LogLevel = s.cfg.LogLevel + cfg.LogFile = s.cfg.LogFile + err = w.StartSubTask(cfg) - err = s.worker.StartSubTask(cfg) if err != nil { err = terror.Annotatef(err, "start sub task %s", cfg.Name) log.L().Error("fail to start subtask", zap.String("request", "StartSubTask"), zap.Stringer("payload", req), zap.Error(err)) - return &pb.CommonWorkerResponse{ - Result: false, - Msg: err.Error(), - }, nil + resp.Result = false + resp.Msg = err.Error() } - - return &pb.CommonWorkerResponse{ - Result: true, - }, nil + return resp, nil } // OperateSubTask implements WorkerServer.OperateSubTask func (s *Server) OperateSubTask(ctx context.Context, req *pb.OperateSubTaskRequest) (*pb.OperateSubTaskResponse, error) { + resp := &pb.OperateSubTaskResponse{ + Result: true, + Op: req.Op, + Msg: "", + } + w := s.checkWorkerStart() + if w == nil { + log.L().Error("fail to call OperateSubTask, because mysql worker has not been started") + resp.Result = false + resp.Msg = terror.ErrWorkerNoStart.Error() + return resp, nil + } + log.L().Info("", zap.String("request", "OperateSubTask"), zap.Stringer("payload", req)) - err := s.worker.OperateSubTask(req.Name, req.Op) + err := w.OperateSubTask(req.Name, req.Op) if err != nil { err = terror.Annotatef(err, "operate(%s) sub task %s", req.Op.String(), req.Name) log.L().Error("fail to operate task", zap.String("request", "OperateSubTask"), zap.Stringer("payload", req), zap.Error(err)) - return &pb.OperateSubTaskResponse{ - Op: req.Op, - Result: false, - Msg: err.Error(), - }, nil + resp.Result = false + resp.Msg = err.Error() } - - return &pb.OperateSubTaskResponse{ - Op: req.Op, - Result: true, - }, nil + return resp, nil } // UpdateSubTask implements WorkerServer.UpdateSubTask @@ -214,33 +258,45 @@ func (s *Server) UpdateSubTask(ctx context.Context, req *pb.UpdateSubTaskRequest Msg: err.Error(), }, nil } - - err = s.worker.UpdateSubTask(cfg) + resp := &pb.CommonWorkerResponse{ + Result: true, + Msg: "", + } + w := s.checkWorkerStart() + if w == nil { + log.L().Error("fail to call StartSubTask, because mysql worker has not been started") + resp.Result = false + resp.Msg = terror.ErrWorkerNoStart.Error() + return resp, nil + } + err = w.UpdateSubTask(cfg) if err != nil { err = terror.Annotatef(err, "update sub task %s", cfg.Name) log.L().Error("fail to update task", zap.String("request", "UpdateSubTask"), zap.Stringer("payload", req), zap.Error(err)) - return &pb.CommonWorkerResponse{ - Result: false, - Msg: err.Error(), - }, nil + resp.Result = false + resp.Msg = err.Error() } - - return &pb.CommonWorkerResponse{ - Result: true, - }, nil + return resp, nil } // QueryStatus implements WorkerServer.QueryStatus func (s *Server) QueryStatus(ctx context.Context, req *pb.QueryStatusRequest) (*pb.QueryStatusResponse, error) { log.L().Info("", zap.String("request", "QueryStatus"), zap.Stringer("payload", req)) - resp := &pb.QueryStatusResponse{ - Result: true, - SubTaskStatus: s.worker.QueryStatus(req.Name), - RelayStatus: s.worker.relayHolder.Status(), - SourceID: s.worker.cfg.SourceID, + Result: true, + } + + w := s.checkWorkerStart() + if w == nil { + log.L().Error("fail to call QueryStatus, because mysql worker has not been started") + resp.Result = false + resp.Msg = terror.ErrWorkerNoStart.Error() + return resp, nil } + resp.SubTaskStatus = w.QueryStatus(req.Name) + resp.RelayStatus = w.relayHolder.Status() + resp.SourceID = w.cfg.SourceID if len(resp.SubTaskStatus) == 0 { resp.Msg = "no sub task started" } @@ -250,13 +306,20 @@ func (s *Server) QueryStatus(ctx context.Context, req *pb.QueryStatusRequest) (* // QueryError implements WorkerServer.QueryError func (s *Server) QueryError(ctx context.Context, req *pb.QueryErrorRequest) (*pb.QueryErrorResponse, error) { log.L().Info("", zap.String("request", "QueryError"), zap.Stringer("payload", req)) - resp := &pb.QueryErrorResponse{ - Result: true, - SubTaskError: s.worker.QueryError(req.Name), - RelayError: s.worker.relayHolder.Error(), + Result: true, + } + + w := s.checkWorkerStart() + if w == nil { + log.L().Error("fail to call StartSubTask, because mysql worker has not been started") + resp.Result = false + resp.Msg = terror.ErrWorkerNoStart.Error() + return resp, nil } + resp.SubTaskError = w.QueryError(req.Name) + resp.RelayError = w.relayHolder.Error() return resp, nil } @@ -265,10 +328,16 @@ func (s *Server) QueryError(ctx context.Context, req *pb.QueryErrorRequest) (*pb // if error occurred in Send / Recv, just retry in client func (s *Server) FetchDDLInfo(stream pb.Worker_FetchDDLInfoServer) error { log.L().Info("", zap.String("request", "FetchDDLInfo")) + w := s.checkWorkerStart() + if w == nil { + log.L().Error("fail to call StartSubTask, because mysql worker has not been started") + return terror.ErrWorkerNoStart.Generate() + } + var ddlInfo *pb.DDLInfo for { // try fetch pending to sync DDL info from worker - ddlInfo = s.worker.FetchDDLInfo(stream.Context()) + ddlInfo = w.FetchDDLInfo(stream.Context()) if ddlInfo == nil { return nil // worker closed or context canceled } @@ -293,7 +362,7 @@ func (s *Server) FetchDDLInfo(stream pb.Worker_FetchDDLInfoServer) error { //ddlInfo = nil // clear and protect to put it back - err = s.worker.RecordDDLLockInfo(in) + err = w.RecordDDLLockInfo(in) if err != nil { // if error occurred when recording DDLLockInfo, log an error // user can handle this case using dmctl @@ -305,8 +374,13 @@ func (s *Server) FetchDDLInfo(stream pb.Worker_FetchDDLInfoServer) error { // ExecuteDDL implements WorkerServer.ExecuteDDL func (s *Server) ExecuteDDL(ctx context.Context, req *pb.ExecDDLRequest) (*pb.CommonWorkerResponse, error) { log.L().Info("", zap.String("request", "ExecuteDDL"), zap.Stringer("payload", req)) + w := s.checkWorkerStart() + if w == nil { + log.L().Error("fail to call StartSubTask, because mysql worker has not been started") + return makeCommonWorkerResponse(terror.ErrWorkerNoStart.Generate()), nil + } - err := s.worker.ExecuteDDL(ctx, req) + err := w.ExecuteDDL(ctx, req) if err != nil { log.L().Error("fail to execute ddl", zap.String("request", "ExecuteDDL"), zap.Stringer("payload", req), zap.Error(err)) } @@ -316,8 +390,13 @@ func (s *Server) ExecuteDDL(ctx context.Context, req *pb.ExecDDLRequest) (*pb.Co // BreakDDLLock implements WorkerServer.BreakDDLLock func (s *Server) BreakDDLLock(ctx context.Context, req *pb.BreakDDLLockRequest) (*pb.CommonWorkerResponse, error) { log.L().Info("", zap.String("request", "BreakDDLLock"), zap.Stringer("payload", req)) + w := s.checkWorkerStart() + if w == nil { + log.L().Error("fail to call StartSubTask, because mysql worker has not been started") + return makeCommonWorkerResponse(terror.ErrWorkerNoStart.Generate()), nil + } - err := s.worker.BreakDDLLock(ctx, req) + err := w.BreakDDLLock(ctx, req) if err != nil { log.L().Error("fail to break ddl lock", zap.String("request", "BreakDDLLock"), zap.Stringer("payload", req), zap.Error(err)) } @@ -327,8 +406,13 @@ func (s *Server) BreakDDLLock(ctx context.Context, req *pb.BreakDDLLockRequest) // HandleSQLs implements WorkerServer.HandleSQLs func (s *Server) HandleSQLs(ctx context.Context, req *pb.HandleSubTaskSQLsRequest) (*pb.CommonWorkerResponse, error) { log.L().Info("", zap.String("request", "HandleSQLs"), zap.Stringer("payload", req)) + w := s.checkWorkerStart() + if w == nil { + log.L().Error("fail to call StartSubTask, because mysql worker has not been started") + return makeCommonWorkerResponse(terror.ErrWorkerNoStart.Generate()), nil + } - err := s.worker.HandleSQLs(ctx, req) + err := w.HandleSQLs(ctx, req) if err != nil { log.L().Error("fail to handle sqls", zap.String("request", "HandleSQLs"), zap.Stringer("payload", req), zap.Error(err)) } @@ -338,8 +422,13 @@ func (s *Server) HandleSQLs(ctx context.Context, req *pb.HandleSubTaskSQLsReques // SwitchRelayMaster implements WorkerServer.SwitchRelayMaster func (s *Server) SwitchRelayMaster(ctx context.Context, req *pb.SwitchRelayMasterRequest) (*pb.CommonWorkerResponse, error) { log.L().Info("", zap.String("request", "SwitchRelayMaster"), zap.Stringer("payload", req)) + w := s.checkWorkerStart() + if w == nil { + log.L().Error("fail to call StartSubTask, because mysql worker has not been started") + return makeCommonWorkerResponse(terror.ErrWorkerNoStart.Generate()), nil + } - err := s.worker.SwitchRelayMaster(ctx, req) + err := w.SwitchRelayMaster(ctx, req) if err != nil { log.L().Error("fail to switch relay master", zap.String("request", "SwitchRelayMaster"), zap.Stringer("payload", req), zap.Error(err)) } @@ -349,13 +438,19 @@ func (s *Server) SwitchRelayMaster(ctx context.Context, req *pb.SwitchRelayMaste // OperateRelay implements WorkerServer.OperateRelay func (s *Server) OperateRelay(ctx context.Context, req *pb.OperateRelayRequest) (*pb.OperateRelayResponse, error) { log.L().Info("", zap.String("request", "OperateRelay"), zap.Stringer("payload", req)) - resp := &pb.OperateRelayResponse{ Op: req.Op, Result: false, } - err := s.worker.OperateRelay(ctx, req) + w := s.checkWorkerStart() + if w == nil { + log.L().Error("fail to call StartSubTask, because mysql worker has not been started") + resp.Msg = terror.ErrWorkerNoStart.Error() + return resp, nil + } + + err := w.OperateRelay(ctx, req) if err != nil { log.L().Error("fail to operate relay", zap.String("request", "OperateRelay"), zap.Stringer("payload", req), zap.Error(err)) resp.Msg = errors.ErrorStack(err) @@ -369,8 +464,13 @@ func (s *Server) OperateRelay(ctx context.Context, req *pb.OperateRelayRequest) // PurgeRelay implements WorkerServer.PurgeRelay func (s *Server) PurgeRelay(ctx context.Context, req *pb.PurgeRelayRequest) (*pb.CommonWorkerResponse, error) { log.L().Info("", zap.String("request", "PurgeRelay"), zap.Stringer("payload", req)) + w := s.checkWorkerStart() + if w == nil { + log.L().Error("fail to call StartSubTask, because mysql worker has not been started") + return makeCommonWorkerResponse(terror.ErrWorkerNoStart.Generate()), nil + } - err := s.worker.PurgeRelay(ctx, req) + err := w.PurgeRelay(ctx, req) if err != nil { log.L().Error("fail to purge relay", zap.String("request", "PurgeRelay"), zap.Stringer("payload", req), zap.Error(err)) } @@ -380,8 +480,13 @@ func (s *Server) PurgeRelay(ctx context.Context, req *pb.PurgeRelayRequest) (*pb // UpdateRelayConfig updates config for relay and (dm-worker) func (s *Server) UpdateRelayConfig(ctx context.Context, req *pb.UpdateRelayRequest) (*pb.CommonWorkerResponse, error) { log.L().Info("", zap.String("request", "UpdateRelayConfig"), zap.Stringer("payload", req)) + w := s.checkWorkerStart() + if w == nil { + log.L().Error("fail to call StartSubTask, because mysql worker has not been started") + return makeCommonWorkerResponse(terror.ErrWorkerNoStart.Generate()), nil + } - err := s.worker.UpdateRelayConfig(ctx, req.Content) + err := w.UpdateRelayConfig(ctx, req.Content) if err != nil { log.L().Error("fail to update relay config", zap.String("request", "UpdateRelayConfig"), zap.Stringer("payload", req), zap.Error(err)) } @@ -393,12 +498,19 @@ func (s *Server) UpdateRelayConfig(ctx context.Context, req *pb.UpdateRelayReque // to avoid circular import, we only return db config func (s *Server) QueryWorkerConfig(ctx context.Context, req *pb.QueryWorkerConfigRequest) (*pb.QueryWorkerConfigResponse, error) { log.L().Info("", zap.String("request", "QueryWorkerConfig"), zap.Stringer("payload", req)) - resp := &pb.QueryWorkerConfigResponse{ Result: true, } - workerCfg, err := s.worker.QueryConfig(ctx) + w := s.checkWorkerStart() + if w == nil { + log.L().Error("fail to call StartSubTask, because mysql worker has not been started") + resp.Result = false + resp.Msg = terror.ErrWorkerNoStart.Error() + return resp, nil + } + + workerCfg, err := w.QueryConfig(ctx) if err != nil { resp.Result = false resp.Msg = errors.ErrorStack(err) @@ -421,14 +533,76 @@ func (s *Server) QueryWorkerConfig(ctx context.Context, req *pb.QueryWorkerConfi // MigrateRelay migrate relay to original binlog pos func (s *Server) MigrateRelay(ctx context.Context, req *pb.MigrateRelayRequest) (*pb.CommonWorkerResponse, error) { log.L().Info("", zap.String("request", "MigrateRelay"), zap.Stringer("payload", req)) + w := s.checkWorkerStart() + if w == nil { + log.L().Error("fail to call StartSubTask, because mysql worker has not been started") + return makeCommonWorkerResponse(terror.ErrWorkerNoStart.Generate()), nil + } - err := s.worker.MigrateRelay(ctx, req.BinlogName, req.BinlogPos) + err := w.MigrateRelay(ctx, req.BinlogName, req.BinlogPos) if err != nil { log.L().Error("fail to migrate relay", zap.String("request", "MigrateRelay"), zap.Stringer("payload", req), zap.Error(err)) } return makeCommonWorkerResponse(err), nil } +func (s *Server) startWorker(cfg *config.MysqlConfig) error { + s.Lock() + if s.worker != nil { + return terror.ErrWorkerAlreadyClosed.Generate() + } + w, err := NewWorker(cfg) + if err != nil { + return err + } + s.worker = w + s.Unlock() + go func() { + s.worker.Start() + }() + return nil +} + +// OperateMysqlTask create a new mysql task which will be running in this Server +func (s *Server) OperateMysqlTask(ctx context.Context, req *pb.MysqlTaskRequest) (*pb.MysqlTaskResponse, error) { + resp := &pb.MysqlTaskResponse{ + Result: true, + Msg: "Operate mysql task successfully", + } + cfg := config.NewWorkerConfig() + err := cfg.Parse(req.Config) + if err != nil { + resp.Result = false + resp.Msg = errors.ErrorStack(err) + return resp, nil + } + if req.Op == pb.WorkerOp_UpdateConfig || req.Op == pb.WorkerOp_StopWorker { + s.Lock() + if s.worker == nil { + resp.Result = false + resp.Msg = "Mysql task has not been created, please call CreateMysqlTask" + return resp, nil + } + if cfg.SourceID != s.worker.cfg.SourceID { + resp.Result = false + resp.Msg = "stop config has not match the source id of worker, it may be a wrong request" + return resp, nil + } + w := s.worker + s.worker = nil + s.Unlock() + w.Close() + } + if req.Op == pb.WorkerOp_UpdateConfig || req.Op == pb.WorkerOp_StartWorker { + err = s.startWorker(cfg) + } + if err != nil { + resp.Result = false + resp.Msg = errors.ErrorStack(err) + } + return resp, nil +} + func makeCommonWorkerResponse(reqErr error) *pb.CommonWorkerResponse { resp := &pb.CommonWorkerResponse{ Result: true, diff --git a/dm/worker/server_test.go b/dm/worker/server_test.go index f61a14bffb..da1a76c8d2 100644 --- a/dm/worker/server_test.go +++ b/dm/worker/server_test.go @@ -15,6 +15,7 @@ package worker import ( "context" + "github.com/pingcap/dm/dm/config" "io/ioutil" "net/http" "testing" @@ -41,8 +42,9 @@ func (t *testServer) TestServer(c *C) { c.Assert(cfg.Parse([]string{"-config=./dm-worker.toml"}), IsNil) dir := c.MkDir() - cfg.RelayDir = dir - cfg.MetaDir = dir + workerCfg := &config.MysqlConfig{} + workerCfg.RelayDir = dir + workerCfg.MetaDir = dir NewRelayHolder = NewDummyRelayHolder defer func() { @@ -66,6 +68,7 @@ func (t *testServer) TestServer(c *C) { c.Assert(utils.WaitSomething(30, 100*time.Millisecond, func() bool { return !s.closed.Get() }), IsTrue) + c.Assert(s.startWorker(workerCfg), IsNil) // test condition hub t.testConidtionHub(c, s) diff --git a/dm/worker/task_checker.go b/dm/worker/task_checker.go index 4fe56ca528..7232f7748b 100644 --- a/dm/worker/task_checker.go +++ b/dm/worker/task_checker.go @@ -17,6 +17,7 @@ import ( "context" "encoding/json" "fmt" + "github.com/pingcap/dm/dm/config" "strings" "sync" "time" @@ -32,15 +33,15 @@ import ( "github.com/pingcap/dm/pkg/terror" ) -// Backoff related constants -var ( - DefaultCheckInterval = 5 * time.Second - DefaultBackoffRollback = 5 * time.Minute - DefaultBackoffMin = 1 * time.Second - DefaultBackoffMax = 5 * time.Minute - DefaultBackoffJitter = true - DefaultBackoffFactor float64 = 2 -) +//// Backoff related constants +//var ( +// DefaultCheckInterval = 5 * time.Second +// DefaultBackoffRollback = 5 * time.Minute +// DefaultBackoffMin = 1 * time.Second +// DefaultBackoffMax = 5 * time.Minute +// DefaultBackoffJitter = true +// DefaultBackoffFactor float64 = 2 +//) // ResumeStrategy represents what we can do when we meet a paused task in task status checker type ResumeStrategy int @@ -117,25 +118,6 @@ func (d *duration) MarshalJSON() ([]byte, error) { }) } -// CheckerConfig is configuration used for TaskStatusChecker -type CheckerConfig struct { - CheckEnable bool `toml:"check-enable" json:"check-enable"` - BackoffRollback duration `toml:"backoff-rollback" json:"backoff-rollback"` - BackoffMax duration `toml:"backoff-max" json:"backoff-max"` - // unexpose config - CheckInterval duration `json:"-"` - BackoffMin duration `json:"-"` - BackoffJitter bool `json:"-"` - BackoffFactor float64 `json:"-"` -} - -func (cc *CheckerConfig) adjust() { - cc.CheckInterval = duration{Duration: DefaultCheckInterval} - cc.BackoffMin = duration{Duration: DefaultBackoffMin} - cc.BackoffJitter = DefaultBackoffJitter - cc.BackoffFactor = DefaultBackoffFactor -} - // TaskStatusChecker is an interface that defines how we manage task status type TaskStatusChecker interface { // Init initializes the checker @@ -182,14 +164,14 @@ type realTaskStatusChecker struct { wg sync.WaitGroup closed sync2.AtomicInt32 - cfg CheckerConfig + cfg config.CheckerConfig l log.Logger w *Worker bc *backoffController } // NewRealTaskStatusChecker creates a new realTaskStatusChecker instance -func NewRealTaskStatusChecker(cfg CheckerConfig, w *Worker) TaskStatusChecker { +func NewRealTaskStatusChecker(cfg config.CheckerConfig, w *Worker) TaskStatusChecker { tsc := &realTaskStatusChecker{ cfg: cfg, l: log.With(zap.String("component", "task checker")), @@ -238,7 +220,7 @@ func (tsc *realTaskStatusChecker) run() { if err != nil { tsc.l.Warn("inject failpoint TaskCheckInterval failed", zap.Reflect("value", val), zap.Error(err)) } else { - tsc.cfg.CheckInterval = duration{Duration: interval} + tsc.cfg.CheckInterval = config.Duration{Duration: interval} tsc.l.Info("set TaskCheckInterval", zap.String("failpoint", "TaskCheckInterval"), zap.Duration("value", interval)) } }) diff --git a/dm/worker/task_checker_test.go b/dm/worker/task_checker_test.go index 23301dfd50..a9bade5281 100644 --- a/dm/worker/task_checker_test.go +++ b/dm/worker/task_checker_test.go @@ -60,13 +60,13 @@ func (s *testTaskCheckerSuite) TestResumeStrategy(c *check.C) { {&pb.SubTaskStatus{Name: taskName, Stage: pb.Stage_Paused, Result: &pb.ProcessResult{IsCanceled: false}}, now, -2 * time.Millisecond, 1 * time.Millisecond, ResumeDispatch}, } - tsc := NewRealTaskStatusChecker(CheckerConfig{ + tsc := NewRealTaskStatusChecker(config.CheckerConfig{ CheckEnable: true, - CheckInterval: duration{Duration: DefaultCheckInterval}, - BackoffRollback: duration{Duration: DefaultBackoffRollback}, - BackoffMin: duration{Duration: DefaultBackoffMin}, - BackoffMax: duration{Duration: DefaultBackoffMax}, - BackoffFactor: DefaultBackoffFactor, + CheckInterval: config.Duration{Duration: config.DefaultCheckInterval}, + BackoffRollback: config.Duration{Duration: config.DefaultBackoffRollback}, + BackoffMin: config.Duration{Duration: config.DefaultBackoffMin}, + BackoffMax: config.Duration{Duration: config.DefaultBackoffMax}, + BackoffFactor: config.DefaultBackoffFactor, }, nil) for _, tc := range testCases { rtsc, ok := tsc.(*realTaskStatusChecker) @@ -87,20 +87,20 @@ func (s *testTaskCheckerSuite) TestCheck(c *check.C) { NewRelayHolder = NewDummyRelayHolder dir := c.MkDir() - cfg := NewConfig() - c.Assert(cfg.Parse([]string{"-config=./dm-worker.toml"}), check.IsNil) + cfg := &config.MysqlConfig{} + c.Assert(cfg.LoadFromFile("./dm-mysql.toml"), check.IsNil) cfg.RelayDir = dir cfg.MetaDir = dir w, err := NewWorker(cfg) c.Assert(err, check.IsNil) - tsc := NewRealTaskStatusChecker(CheckerConfig{ + tsc := NewRealTaskStatusChecker(config.CheckerConfig{ CheckEnable: true, - CheckInterval: duration{Duration: DefaultCheckInterval}, - BackoffRollback: duration{Duration: 200 * time.Millisecond}, - BackoffMin: duration{Duration: 1 * time.Millisecond}, - BackoffMax: duration{Duration: 1 * time.Second}, - BackoffFactor: DefaultBackoffFactor, + CheckInterval: config.Duration{Duration: config.DefaultCheckInterval}, + BackoffRollback: config.Duration{Duration: 200 * time.Millisecond}, + BackoffMin: config.Duration{Duration: 1 * time.Millisecond}, + BackoffMax: config.Duration{Duration: 1 * time.Second}, + BackoffFactor: config.DefaultBackoffFactor, }, nil) c.Assert(tsc.Init(), check.IsNil) rtsc, ok := tsc.(*realTaskStatusChecker) @@ -154,13 +154,13 @@ func (s *testTaskCheckerSuite) TestCheck(c *check.C) { c.Assert(bf.Current(), check.Equals, current) // test resume skip strategy - tsc = NewRealTaskStatusChecker(CheckerConfig{ + tsc = NewRealTaskStatusChecker(config.CheckerConfig{ CheckEnable: true, - CheckInterval: duration{Duration: DefaultCheckInterval}, - BackoffRollback: duration{Duration: 200 * time.Millisecond}, - BackoffMin: duration{Duration: 10 * time.Second}, - BackoffMax: duration{Duration: 100 * time.Second}, - BackoffFactor: DefaultBackoffFactor, + CheckInterval: config.Duration{Duration: config.DefaultCheckInterval}, + BackoffRollback: config.Duration{Duration: 200 * time.Millisecond}, + BackoffMin: config.Duration{Duration: 10 * time.Second}, + BackoffMax: config.Duration{Duration: 100 * time.Second}, + BackoffFactor: config.DefaultBackoffFactor, }, w) c.Assert(tsc.Init(), check.IsNil) rtsc, ok = tsc.(*realTaskStatusChecker) @@ -204,19 +204,20 @@ func (s *testTaskCheckerSuite) TestCheckTaskIndependent(c *check.C) { NewRelayHolder = NewDummyRelayHolder dir := c.MkDir() - cfg := NewConfig() - c.Assert(cfg.Parse([]string{"-config=./dm-worker.toml"}), check.IsNil) + // cfg := NewConfig() + cfg := &config.MysqlConfig{} + c.Assert(cfg.LoadFromFile("./dm-mysql.toml"), check.IsNil) cfg.RelayDir = dir cfg.MetaDir = dir w, err := NewWorker(cfg) c.Assert(err, check.IsNil) - tsc := NewRealTaskStatusChecker(CheckerConfig{ + tsc := NewRealTaskStatusChecker(config.CheckerConfig{ CheckEnable: true, - CheckInterval: duration{Duration: DefaultCheckInterval}, - BackoffRollback: duration{Duration: 200 * time.Millisecond}, - BackoffMin: duration{Duration: backoffMin}, - BackoffMax: duration{Duration: 10 * time.Second}, + CheckInterval: config.Duration{Duration: config.DefaultCheckInterval}, + BackoffRollback: config.Duration{Duration: 200 * time.Millisecond}, + BackoffMin: config.Duration{Duration: backoffMin}, + BackoffMax: config.Duration{Duration: 10 * time.Second}, BackoffFactor: 1.0, }, nil) c.Assert(tsc.Init(), check.IsNil) diff --git a/dm/worker/worker.go b/dm/worker/worker.go index bafa2f4396..a305373089 100644 --- a/dm/worker/worker.go +++ b/dm/worker/worker.go @@ -52,7 +52,7 @@ type Worker struct { ctx context.Context cancel context.CancelFunc - cfg *Config + cfg *config.MysqlConfig l log.Logger subTaskHolder *subTaskHolder @@ -63,10 +63,11 @@ type Worker struct { tracer *tracing.Tracer taskStatusChecker TaskStatusChecker + configFile string } // NewWorker creates a new Worker -func NewWorker(cfg *Config) (w *Worker, err error) { +func NewWorker(cfg *config.MysqlConfig) (w *Worker, err error) { w = &Worker{ cfg: cfg, relayHolder: NewRelayHolder(cfg), @@ -75,6 +76,7 @@ func NewWorker(cfg *Config) (w *Worker, err error) { l: log.With(zap.String("component", "worker controller")), } w.ctx, w.cancel = context.WithCancel(context.Background()) + w.closed.Set(closedTrue) defer func(w2 *Worker) { if err != nil { // when err != nil, `w` will become nil in this func, so we pass `w` in defer. @@ -112,10 +114,6 @@ func NewWorker(cfg *Config) (w *Worker, err error) { // Start starts working func (w *Worker) Start() { - if w.closed.Get() == closedTrue { - w.l.Warn("already closed") - return - } // start relay w.relayHolder.Start() @@ -139,6 +137,7 @@ func (w *Worker) Start() { w.l.Info("start running") ticker := time.NewTicker(5 * time.Second) + w.closed.Set(closedFalse) defer ticker.Stop() for { select { @@ -185,6 +184,7 @@ func (w *Worker) Close() { } w.closed.Set(closedTrue) + w.l.Info("Stop worker") } // StartSubTask creates a sub task an run it @@ -515,7 +515,7 @@ func (w *Worker) ForbidPurge() (bool, string) { } // QueryConfig returns worker's config -func (w *Worker) QueryConfig(ctx context.Context) (*Config, error) { +func (w *Worker) QueryConfig(ctx context.Context) (*config.MysqlConfig, error) { w.RLock() defer w.RUnlock() @@ -550,14 +550,10 @@ func (w *Worker) UpdateRelayConfig(ctx context.Context, content string) error { } } - // Save configure to local file. - newCfg := NewConfig() - err := newCfg.UpdateConfigFile(content) - if err != nil { - return err - } + // No need to store config in local + newCfg := &config.MysqlConfig{} - err = newCfg.Reload() + err := newCfg.Parse(content) if err != nil { return err } @@ -613,19 +609,15 @@ func (w *Worker) UpdateRelayConfig(ctx context.Context, content string) error { w.cfg.AutoFixGTID = newCfg.AutoFixGTID w.cfg.Charset = newCfg.Charset - if w.cfg.ConfigFile == "" { - w.cfg.ConfigFile = "dm-worker.toml" + if w.configFile == "" { + w.configFile = "dm-worker.toml" } content, err = w.cfg.Toml() if err != nil { return err } - err = w.cfg.UpdateConfigFile(content) - if err != nil { - return err - } - w.l.Info("update relay config successfully, save config to local file", zap.String("local file", w.cfg.ConfigFile)) + w.l.Info("update relay config successfully, save config to local file", zap.String("local file", w.configFile)) return nil } @@ -663,8 +655,7 @@ func (w *Worker) copyConfigFromWorker(cfg *config.SubTaskConfig) { cfg.AutoFixGTID = w.cfg.AutoFixGTID // log config items, mydumper unit use it - cfg.LogLevel = w.cfg.LogLevel - cfg.LogFile = w.cfg.LogFile + } // getAllSubTaskStatus returns all subtask status of this worker, note the field diff --git a/dm/worker/worker_test.go b/dm/worker/worker_test.go index fd75282c7d..e98f15fe41 100644 --- a/dm/worker/worker_test.go +++ b/dm/worker/worker_test.go @@ -31,8 +31,8 @@ import ( var emptyWorkerStatusInfoJSONLength = 25 func (t *testServer) testWorker(c *C) { - cfg := NewConfig() - c.Assert(cfg.Parse([]string{"-config=./dm-worker.toml"}), IsNil) + cfg := &config.MysqlConfig{} + c.Assert(cfg.LoadFromFile("./dm-mysql.toml"), IsNil) dir := c.MkDir() cfg.RelayDir = dir @@ -50,7 +50,10 @@ func (t *testServer) testWorker(c *C) { w, err := NewWorker(cfg) c.Assert(err, IsNil) c.Assert(w.StatusJSON(""), HasLen, emptyWorkerStatusInfoJSONLength) - c.Assert(w.closed.Get(), Equals, closedFalse) + //c.Assert(w.closed.Get(), Equals, closedFalse) + //go func() { + // w.Start() + //}() // close twice w.Close() @@ -59,10 +62,12 @@ func (t *testServer) testWorker(c *C) { w.Close() c.Assert(w.closed.Get(), Equals, closedTrue) c.Assert(w.subTaskHolder.getAllSubTasks(), HasLen, 0) + c.Assert(w.closed.Get(), Equals, closedTrue) err = w.StartSubTask(&config.SubTaskConfig{ Name: "testStartTask", }) + c.Assert(err, NotNil) c.Assert(err, ErrorMatches, ".*worker already closed.*") err = w.UpdateSubTask(&config.SubTaskConfig{ @@ -80,16 +85,19 @@ func (t *testServer) TestTaskAutoResume(c *C) { port = 8263 ) cfg := NewConfig() + workerCfg := config.NewWorkerConfig() + workerCfg.LoadFromFile("./dm-mysql.toml") c.Assert(cfg.Parse([]string{"-config=./dm-worker.toml"}), IsNil) - cfg.From.Password = "" // no password set - cfg.Checker.CheckInterval = duration{Duration: 40 * time.Millisecond} - cfg.Checker.BackoffMin = duration{Duration: 20 * time.Millisecond} - cfg.Checker.BackoffMax = duration{Duration: 1 * time.Second} + workerCfg.Checker.CheckInterval = config.Duration{Duration: 40 * time.Millisecond} + workerCfg.Checker.BackoffMin = config.Duration{Duration: 20 * time.Millisecond} + workerCfg.Checker.BackoffMax = config.Duration{Duration: 1 * time.Second} + workerCfg.From.Password = "" // no password set + cfg.WorkerAddr = fmt.Sprintf(":%d", port) dir := c.MkDir() - cfg.RelayDir = dir - cfg.MetaDir = dir + workerCfg.RelayDir = dir + workerCfg.MetaDir = dir NewRelayHolder = NewDummyRelayHolder defer func() { @@ -110,7 +118,11 @@ func (t *testServer) TestTaskAutoResume(c *C) { c.Assert(s.Start(), IsNil) }() c.Assert(utils.WaitSomething(10, 100*time.Millisecond, func() bool { - return !s.closed.Get() + if s.closed.Get() { + return false + } + c.Assert(s.startWorker(workerCfg), IsNil) + return true }), IsTrue) // start task diff --git a/pkg/terror/error_list.go b/pkg/terror/error_list.go index 9b819c8a9b..9151996539 100644 --- a/pkg/terror/error_list.go +++ b/pkg/terror/error_list.go @@ -470,6 +470,8 @@ const ( codeWorkerWaitRelayCatchupTimeout codeWorkerRelayIsPurging codeWorkerHostPortNotValid + codeWorkerNoStart + codeWorkerAlreadyStarted ) // DM-tracer error code @@ -900,7 +902,9 @@ var ( ErrWorkerSaveVersionToKV = New(codeWorkerSaveVersionToKV, ClassDMWorker, ScopeInternal, LevelHigh, "save version %v into levelDB with key %v") ErrWorkerVerAutoDowngrade = New(codeWorkerVerAutoDowngrade, ClassDMWorker, ScopeInternal, LevelHigh, "the previous version %s is newer than current %s, automatic downgrade is not supported now, please handle it manually") ErrWorkerStartService = New(codeWorkerStartService, ClassDMWorker, ScopeInternal, LevelHigh, "start server") + ErrWorkerNoStart = New(codeWorkerNoStart, ClassDMWorker, ScopeInternal, LevelHigh, "worker has not started") ErrWorkerAlreadyClosed = New(codeWorkerAlreadyClosed, ClassDMWorker, ScopeInternal, LevelHigh, "worker already closed") + ErrWorkerAlreadyStart = New(codeWorkerAlreadyStarted, ClassDMWorker, ScopeInternal, LevelHigh, "worker already started") ErrWorkerNotRunningStage = New(codeWorkerNotRunningStage, ClassDMWorker, ScopeInternal, LevelHigh, "current stage is not running not valid") ErrWorkerNotPausedStage = New(codeWorkerNotPausedStage, ClassDMWorker, ScopeInternal, LevelHigh, "current stage is not paused not valid") ErrWorkerUpdateTaskStage = New(codeWorkerUpdateTaskStage, ClassDMWorker, ScopeInternal, LevelHigh, "can only update task on Paused stage, but current stage is %s") diff --git a/relay/purger/config.go b/relay/purger/config.go deleted file mode 100644 index 91b24fc06f..0000000000 --- a/relay/purger/config.go +++ /dev/null @@ -1,21 +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 purger - -// Config is the configuration for Purger -type Config struct { - Interval int64 `toml:"interval" json:"interval"` // check whether need to purge at this @Interval (seconds) - Expires int64 `toml:"expires" json:"expires"` // if file's modified time is older than @Expires (hours), then it can be purged - RemainSpace int64 `toml:"remain-space" json:"remain-space"` // if remain space in @RelayBaseDir less than @RemainSpace (GB), then it can be purged -} diff --git a/relay/purger/purger.go b/relay/purger/purger.go index 9561e2ea88..4efeb83a7c 100644 --- a/relay/purger/purger.go +++ b/relay/purger/purger.go @@ -15,6 +15,7 @@ package purger import ( "context" + "github.com/pingcap/dm/dm/config" "path/filepath" "sync" "time" @@ -71,7 +72,7 @@ type RelayPurger struct { running sync2.AtomicInt32 purgingStrategy sync2.AtomicUint32 - cfg Config + cfg config.PurgeConfig baseRelayDir string indexPath string // server-uuid.index file path operators []RelayOperator @@ -82,7 +83,7 @@ type RelayPurger struct { } // NewRelayPurger creates a new purger -func NewRelayPurger(cfg Config, baseRelayDir string, operators []RelayOperator, interceptors []PurgeInterceptor) Purger { +func NewRelayPurger(cfg config.PurgeConfig, baseRelayDir string, operators []RelayOperator, interceptors []PurgeInterceptor) Purger { p := &RelayPurger{ cfg: cfg, baseRelayDir: baseRelayDir, @@ -307,7 +308,7 @@ func (p *RelayPurger) earliestActiveRelayLog() *streamer.RelayLogInfo { type dummyPurger struct{} // NewDummyPurger returns a dummy purger -func NewDummyPurger(cfg Config, baseRelayDir string, operators []RelayOperator, interceptors []PurgeInterceptor) Purger { +func NewDummyPurger(cfg config.PurgeConfig, baseRelayDir string, operators []RelayOperator, interceptors []PurgeInterceptor) Purger { return &dummyPurger{} } diff --git a/relay/purger/purger_test.go b/relay/purger/purger_test.go index 354b3bcbe5..619cddbba7 100644 --- a/relay/purger/purger_test.go +++ b/relay/purger/purger_test.go @@ -16,6 +16,7 @@ package purger import ( "bytes" "context" + "github.com/pingcap/dm/dm/config" "io/ioutil" "os" "path/filepath" @@ -79,7 +80,7 @@ func (t *testPurgerSuite) TestPurgeManuallyInactive(c *C) { err = t.genUUIDIndexFile(baseDir) c.Assert(err, IsNil) - cfg := Config{ + cfg := config.PurgeConfig{ Interval: 0, // disable automatically } @@ -118,7 +119,7 @@ func (t *testPurgerSuite) TestPurgeManuallyTime(c *C) { err = t.genUUIDIndexFile(baseDir) c.Assert(err, IsNil) - cfg := Config{ + cfg := config.PurgeConfig{ Interval: 0, // disable automatically } @@ -157,7 +158,7 @@ func (t *testPurgerSuite) TestPurgeManuallyFilename(c *C) { err = t.genUUIDIndexFile(baseDir) c.Assert(err, IsNil) - cfg := Config{ + cfg := config.PurgeConfig{ Interval: 0, // disable automatically } @@ -200,7 +201,7 @@ func (t *testPurgerSuite) TestPurgeAutomaticallyTime(c *C) { err = t.genUUIDIndexFile(baseDir) c.Assert(err, IsNil) - cfg := Config{ + cfg := config.PurgeConfig{ Interval: 1, // enable automatically Expires: 1, } @@ -250,7 +251,7 @@ func (t *testPurgerSuite) TestPurgeAutomaticallySpace(c *C) { storageSize, err := utils.GetStorageSize(baseDir) c.Assert(err, IsNil) - cfg := Config{ + cfg := config.PurgeConfig{ Interval: 1, // enable automatically RemainSpace: int64(storageSize.Available)/1024/1024/1024 + 1024, // always trigger purge } @@ -334,7 +335,7 @@ func (i *fakeInterceptor) ForbidPurge() (bool, string) { } func (t *testPurgerSuite) TestPurgerInterceptor(c *C) { - cfg := Config{} + cfg := config.PurgeConfig{} interceptor := newFakeInterceptor() purger := NewPurger(cfg, "", []RelayOperator{t}, []PurgeInterceptor{interceptor}) diff --git a/tests/http_apis/conf/dm-mysql.toml b/tests/http_apis/conf/dm-mysql.toml new file mode 100644 index 0000000000..bdae2fdf14 --- /dev/null +++ b/tests/http_apis/conf/dm-mysql.toml @@ -0,0 +1,13 @@ +# Worker Configuration. + +source-id = "mysql-replica-01" +flavor = "" +enable-gtid = false +relay-binlog-name = "" +relay-binlog-gtid = "" + +[from] +host = "172.16.5.40" +user = "root" +password = "" +port = 10097 \ No newline at end of file diff --git a/tests/utils/dmctl.go b/tests/utils/dmctl.go index cea552564b..fc3c27b807 100644 --- a/tests/utils/dmctl.go +++ b/tests/utils/dmctl.go @@ -41,7 +41,7 @@ func StartTask(ctx context.Context, cli pb.MasterClient, configFile string, work resp, err := cli.StartTask(ctx, &pb.StartTaskRequest{ Task: string(content), - Workers: workers, + Sources: workers, }) if err != nil { return errors.Trace(err) @@ -64,7 +64,7 @@ func OperateTask(ctx context.Context, cli pb.MasterClient, op pb.TaskOp, name st resp, err := cli.OperateTask(ctx, &pb.OperateTaskRequest{ Op: op, Name: name, - Workers: workers, + Sources: workers, }) if err != nil { return errors.Trace(err)