From 16e24e24182e04822a514752dc811268bea737be Mon Sep 17 00:00:00 2001 From: gmhdbjd Date: Thu, 30 Jul 2020 22:54:51 +0800 Subject: [PATCH 01/31] remove handle-sql --- _utils/terror_gen/errors_release.txt | 8 - dm/command/argument_test.go | 131 -- dm/ctl/ctl.go | 3 - dm/ctl/master/sql_inject.go | 100 -- dm/ctl/master/sql_operation.go | 43 - dm/ctl/master/sql_replace.go | 105 -- dm/ctl/master/sql_skip.go | 98 -- dm/master/server.go | 79 +- dm/master/shardddl/pessimist.go | 1 - dm/master/sql-operator/operator.go | 163 --- dm/master/sql-operator/operator_test.go | 91 -- dm/master/workerrpc/interface.go | 6 - dm/master/workerrpc/rawgrpc.go | 2 - dm/pb/dmmaster.pb.go | 1151 +++-------------- dm/pb/dmworker.pb.go | 763 +++-------- dm/pbmock/dmmaster.go | 35 - dm/pbmock/dmworker.go | 35 - dm/proto/dmmaster.proto | 18 - dm/proto/dmworker.proto | 17 - dm/worker/server.go | 17 - dm/worker/subtask.go | 15 - dm/worker/worker.go | 14 - errors.toml | 48 - pkg/terror/error_list.go | 8 - syncer/inject_sql.go | 108 -- syncer/operator.go | 31 - syncer/optimist.go | 12 - syncer/sql-operator/operator.go | 175 --- syncer/sql-operator/operator_test.go | 220 ---- syncer/syncer.go | 37 +- tests/dmctl_advance/check_list/sql_replace.sh | 20 - tests/dmctl_advance/check_list/sql_skip.sh | 39 - tests/dmctl_advance/run.sh | 10 - tests/dmctl_command/run.sh | 2 +- 34 files changed, 357 insertions(+), 3248 deletions(-) delete mode 100644 dm/command/argument_test.go delete mode 100644 dm/ctl/master/sql_inject.go delete mode 100644 dm/ctl/master/sql_operation.go delete mode 100644 dm/ctl/master/sql_replace.go delete mode 100644 dm/ctl/master/sql_skip.go delete mode 100644 dm/master/sql-operator/operator.go delete mode 100644 dm/master/sql-operator/operator_test.go delete mode 100644 syncer/inject_sql.go delete mode 100644 syncer/operator.go delete mode 100644 syncer/sql-operator/operator.go delete mode 100644 syncer/sql-operator/operator_test.go delete mode 100644 tests/dmctl_advance/check_list/sql_replace.sh delete mode 100644 tests/dmctl_advance/check_list/sql_skip.sh diff --git a/_utils/terror_gen/errors_release.txt b/_utils/terror_gen/errors_release.txt index a2dfc6bd94..ea753d9262 100644 --- a/_utils/terror_gen/errors_release.txt +++ b/_utils/terror_gen/errors_release.txt @@ -258,10 +258,6 @@ ErrSyncerUnitDDLExecChanCloseOrBusy,[code=36019:class=sync-unit:scope=internal:l ErrSyncerUnitDDLChanDone,[code=36020:class=sync-unit:scope=internal:level=high], "Message: canceled from external" ErrSyncerUnitDDLChanCanceled,[code=36021:class=sync-unit:scope=internal:level=high], "Message: canceled by Close or Renew" ErrSyncerUnitDDLOnMultipleTable,[code=36022:class=sync-unit:scope=internal:level=high], "Message: ddl on multiple table: %s not supported, Workaround: It is recommended to include only one DDL operation in a statement executed upstream. Please manually handle it using dmctl (skipping the DDL statement or replacing the DDL statement with a specified DDL statement). For details, see https://docs.pingcap.com/tidb-data-migration/stable/skip-or-replace-abnormal-sql-statements" -ErrSyncerUnitInjectDDLOnly,[code=36023:class=sync-unit:scope=internal:level=low], "Message: only support inject DDL for sharding group to be synced currently, but got %s" -ErrSyncerUnitInjectDDLWithoutSchema,[code=36024:class=sync-unit:scope=internal:level=low], "Message: injected DDL %s without schema name not valid" -ErrSyncerUnitNotSupportedOperate,[code=36025:class=sync-unit:scope=internal:level=medium], "Message: op %s not supported" -ErrSyncerUnitNilOperatorReq,[code=36026:class=sync-unit:scope=internal:level=medium], "Message: nil request not valid" ErrSyncerUnitDMLColumnNotMatch,[code=36027:class=sync-unit:scope=internal:level=high], "Message: Column count doesn't match value count: %d (columns) vs %d (values)" ErrSyncerUnitDMLOldNewValueMismatch,[code=36028:class=sync-unit:scope=internal:level=high], "Message: Old value count doesn't match new value count: %d (old) vs %d (new)" ErrSyncerUnitDMLPruneColumnMismatch,[code=36029:class=sync-unit:scope=internal:level=high], "Message: prune DML columns and data mismatch in length: %d (columns) %d (data)" @@ -299,9 +295,6 @@ ErrSyncerUnitGenBAList,[code=36060:class=sync-unit:scope=internal:level=high], " ErrSyncerUnitHandleDDLFailed,[code=36061:class=sync-unit:scope=internal:level=high], "Message: fail to handle ddl job for %s" ErrSyncerShardDDLConflict,[code=36062:class=sync-unit:scope=internal:level=high], "Message: fail to handle shard ddl %v in optimistic mode, because schema conflict detected, Workaround: Please use show-ddl-locks command for more details." ErrSyncerFailpoint,[code=36063:class=sync-unit:scope=internal:level=low], "Message: failpoint specified error" -ErrMasterSQLOpNilRequest,[code=38001:class=dm-master:scope=internal:level=medium], "Message: nil request not valid" -ErrMasterSQLOpNotSupport,[code=38002:class=dm-master:scope=internal:level=medium], "Message: op %s not supported" -ErrMasterSQLOpWithoutSharding,[code=38003:class=dm-master:scope=internal:level=medium], "Message: operate request without --sharding specified not valid" ErrMasterGRPCCreateConn,[code=38004:class=dm-master:scope=internal:level=high], "Message: create grpc connection" ErrMasterGRPCSendOnCloseConn,[code=38005:class=dm-master:scope=internal:level=high], "Message: send request on a closed client" ErrMasterGRPCClientClose,[code=38006:class=dm-master:scope=internal:level=high], "Message: close rpc client" @@ -321,7 +314,6 @@ ErrMasterLockNotFound,[code=38019:class=dm-master:scope=internal:level=high], "M ErrMasterLockIsResolving,[code=38020:class=dm-master:scope=internal:level=high], "Message: lock %s is resolving" ErrMasterWorkerCliNotFound,[code=38021:class=dm-master:scope=internal:level=high], "Message: source %s relevant worker-client not found" ErrMasterWorkerNotWaitLock,[code=38022:class=dm-master:scope=internal:level=high], "Message: worker %s not waiting for DDL lock %s" -ErrMasterHandleSQLReqFail,[code=38023:class=dm-master:scope=internal:level=high], "Message: request DDL lock %s owner %s handle SQLs request %s fail %s" ErrMasterOwnerExecDDL,[code=38024:class=dm-master:scope=internal:level=high], "Message: owner %s ExecuteDDL fail" ErrMasterPartWorkerExecDDLFail,[code=38025:class=dm-master:scope=internal:level=high], "Message: DDL lock %s owner ExecuteDDL successfully, so DDL lock removed. but some dm-workers ExecuteDDL fail, you should to handle dm-worker directly" ErrMasterWorkerExistDDLLock,[code=38026:class=dm-master:scope=internal:level=high], "Message: worker %s exist ddl lock, Workaround: Please unlock ddl lock first." diff --git a/dm/command/argument_test.go b/dm/command/argument_test.go deleted file mode 100644 index 9f2d36afdc..0000000000 --- a/dm/command/argument_test.go +++ /dev/null @@ -1,131 +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 command - -import ( - "testing" - - . "github.com/pingcap/check" - "github.com/siddontang/go-mysql/mysql" -) - -var _ = Suite(&testArgumentSuite{}) - -func TestSuite(t *testing.T) { - TestingT(t) -} - -type testArgumentSuite struct { -} - -func (t *testArgumentSuite) TestTrimQuoteMark(c *C) { - cases := []struct { - input string - output string - }{ - { - input: "mysql-bin.000001:234", - output: "mysql-bin.000001:234", - }, - { - input: "\"mysql-bin.000001:234\"", - output: "mysql-bin.000001:234", - }, - { - input: "\"mysql-bin.000001:234", - output: "\"mysql-bin.000001:234", - }, - { - input: "mysql-bin.000001:234\"", - output: "mysql-bin.000001:234\"", - }, - { - input: "\"\"mysql-bin.000001:234\"", - output: "\"mysql-bin.000001:234", - }, - } - - for _, cs := range cases { - c.Assert(TrimQuoteMark(cs.input), Equals, cs.output) - } -} - -func (t *testArgumentSuite) TestVerifySQLOperateArgs(c *C) { - cases := []struct { - binlogPosStr string - sqlPattern string - shardingIn bool - binlogPos *mysql.Position - regNotNil bool - hasError bool - }{ - { - hasError: true, // must specify one of --binlog-pos and --sql-pattern, - }, - { - binlogPosStr: "mysql-bin.000001:234", - sqlPattern: "~(?i)ALTER\\s+TABLE\\s+", - hasError: true, // cannot specify both --binlog-pos and --sql-pattern - }, - { - binlogPosStr: "mysql-bin.000001:234", - shardingIn: true, - hasError: true, // cannot specify --binlog-pos with --sharding - }, - { - binlogPosStr: "mysql-bin.000001;234", - hasError: true, // invalid --binlog-pos - }, - { - sqlPattern: "~abc[def", - hasError: true, // invalid --sql-pattern - }, - { - binlogPosStr: "mysql-bin.000001:234", - binlogPos: &mysql.Position{Name: "mysql-bin.000001", Pos: 234}, - hasError: false, // valid --binlog-pos - }, - { - sqlPattern: "~(?i)ALTER\\s+TABLE\\s+", - regNotNil: true, - hasError: false, // valid --sql-pattern, regexp - }, - { - sqlPattern: "ALTER TABLE", - regNotNil: true, - hasError: false, // valid --sql-pattern, non-regexp - }, - { - sqlPattern: "~(?i)ALTER\\s+TABLE\\s+", - regNotNil: true, - shardingIn: true, - hasError: false, // valid --sql-pattern with --sharding - }, - } - - for _, cs := range cases { - pos, reg, err := VerifySQLOperateArgs(cs.binlogPosStr, cs.sqlPattern, cs.shardingIn) - if cs.hasError { - c.Assert(err, NotNil) - } else { - c.Assert(err, IsNil) - } - c.Assert(pos, DeepEquals, cs.binlogPos) - if cs.regNotNil { - c.Assert(reg, NotNil) - } else { - c.Assert(reg, IsNil) - } - } -} diff --git a/dm/ctl/ctl.go b/dm/ctl/ctl.go index 0c17e3b57d..029410ac8b 100644 --- a/dm/ctl/ctl.go +++ b/dm/ctl/ctl.go @@ -63,9 +63,6 @@ func NewRootCmd() *cobra.Command { master.NewUpdateTaskCmd(), master.NewQueryStatusCmd(), master.NewQueryErrorCmd(), - master.NewSQLReplaceCmd(), - master.NewSQLSkipCmd(), - master.NewSQLInjectCmd(), master.NewShowDDLLocksCmd(), master.NewUnlockDDLLockCmd(), master.NewSwitchRelayMasterCmd(), diff --git a/dm/ctl/master/sql_inject.go b/dm/ctl/master/sql_inject.go deleted file mode 100644 index 1545db4107..0000000000 --- a/dm/ctl/master/sql_inject.go +++ /dev/null @@ -1,100 +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" - "errors" - "os" - "strings" - - "github.com/spf13/cobra" - - "github.com/pingcap/dm/dm/ctl/common" - "github.com/pingcap/dm/dm/pb" -) - -// NewSQLInjectCmd creates a SQLInject command -func NewSQLInjectCmd() *cobra.Command { - cmd := &cobra.Command{ - Use: "sql-inject <-s source> ", - Short: "inject (limited) SQLs into binlog replication unit as binlog events", - RunE: sqlInjectFunc, - } - return cmd -} - -// sqlInjectFunc does sql inject request -func sqlInjectFunc(cmd *cobra.Command, _ []string) (err error) { - if len(cmd.Flags().Args()) < 2 { - cmd.SetOut(os.Stdout) - cmd.Usage() - err = errors.New("please check output to see error") - return - } - - sources, err := common.GetSourceArgs(cmd) - if err != nil { - common.PrintLines("%v", err) - return - } - if len(sources) != 1 { - common.PrintLines("want only one source, but got %v", sources) - err = errors.New("please check output to see error") - return - } - - taskName := cmd.Flags().Arg(0) - if strings.TrimSpace(taskName) == "" { - common.PrintLines("task-name is empty") - err = errors.New("please check output to see error") - return - } - - extraArgs := cmd.Flags().Args()[1:] - realSQLs, err := common.ExtractSQLsFromArgs(extraArgs) - if err != nil { - common.PrintLines("check sqls err") - return - } - for _, sql := range realSQLs { - var isDDL bool - isDDL, err = common.IsDDL(sql) - if err != nil { - common.PrintLines("check sql err") - return - } - if !isDDL { - common.PrintLines("only support inject DDL currently, but got '%s'", sql) - err = errors.New("please check output to see error") - return - } - } - - ctx, cancel := context.WithCancel(context.Background()) - defer cancel() - cli := common.MasterClient() - resp, err := cli.HandleSQLs(ctx, &pb.HandleSQLsRequest{ - Name: taskName, - Op: pb.SQLOp_INJECT, - Args: realSQLs, - Source: sources[0], - }) - if err != nil { - return - } - - common.PrettyPrintResponse(resp) - return -} diff --git a/dm/ctl/master/sql_operation.go b/dm/ctl/master/sql_operation.go deleted file mode 100644 index 92b6b68f4b..0000000000 --- a/dm/ctl/master/sql_operation.go +++ /dev/null @@ -1,43 +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 ( - "github.com/pingcap/errors" - "github.com/spf13/cobra" - - "github.com/pingcap/dm/dm/command" -) - -// extractBinlogPosSQLPattern extracts binlog-pos and sql-pattern from command -// return binlog-pos, sql-pattern, sharding, error -func extractBinlogPosSQLPattern(cmd *cobra.Command) (string, string, bool, error) { - binlogPos, err := cmd.Flags().GetString("binlog-pos") - if err != nil { - return "", "", false, errors.Trace(err) - } - - sqlPattern, err := cmd.Flags().GetString("sql-pattern") - if err != nil { - return "", "", false, errors.Trace(err) - } - - sharding, err := cmd.Flags().GetBool("sharding") - if err != nil { - return "", "", false, errors.Trace(err) - } - - _, _, err = command.VerifySQLOperateArgs(binlogPos, sqlPattern, sharding) - return binlogPos, sqlPattern, sharding, errors.Trace(err) -} diff --git a/dm/ctl/master/sql_replace.go b/dm/ctl/master/sql_replace.go deleted file mode 100644 index 7d02b6ee82..0000000000 --- a/dm/ctl/master/sql_replace.go +++ /dev/null @@ -1,105 +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" - "errors" - "os" - "strings" - - "github.com/pingcap/dm/dm/ctl/common" - "github.com/pingcap/dm/dm/pb" - - "github.com/spf13/cobra" -) - -// NewSQLReplaceCmd creates a SQLReplace command -func NewSQLReplaceCmd() *cobra.Command { - cmd := &cobra.Command{ - Use: "sql-replace <-s source> [-b binlog-pos] [-p sql-pattern] [--sharding] ", - Short: "replace SQLs matched by a specific binlog position (binlog-pos) or a SQL pattern (sql-pattern); each SQL must end with a semicolon", - RunE: sqlReplaceFunc, - } - cmd.Flags().StringP("binlog-pos", "b", "", "position used to match binlog event if matched the sql-replace operation will be applied. The format like \"mysql-bin|000001.000003:3270\"") - cmd.Flags().StringP("sql-pattern", "p", "", "SQL pattern used to match the DDL converted by optional router-rules if matched the sql-replace operation will be applied. The format like \"~(?i)ALTER\\s+TABLE\\s+`db1`.`tbl1`\\s+ADD\\s+COLUMN\\s+col1\\s+INT\". Whitespace is not supported, and must be replaced by \"\\s\". Staring with ~ as regular expression. This can only be used for DDL (converted by optional router-rules), and if multi DDLs in one binlog event, one of them matched is enough, but all of them will be replaced") - cmd.Flags().BoolP("sharding", "", false, "whether are handing sharding DDL, which will only take effect on DDL lock's owner") - return cmd -} - -// sqlReplaceFunc does sql replace request -func sqlReplaceFunc(cmd *cobra.Command, _ []string) (err error) { - if len(cmd.Flags().Args()) < 2 { - cmd.SetOut(os.Stdout) - cmd.Usage() - err = errors.New("please check output to see error") - return - } - - binlogPos, sqlPattern, sharding, err := extractBinlogPosSQLPattern(cmd) - if err != nil { - return - } - - var source string - sources, err := common.GetSourceArgs(cmd) - if err != nil { - return - } - if sharding { - if len(sources) != 0 { - common.PrintLines("--sharding operator always takes effect on DDL lock's owner, specified sources %v arguments will be ignored", sources) - } - } else { - if len(sources) != 1 { - common.PrintLines("should only specify one source, but got %v", sources) - err = errors.New("please check output to see error") - return - } - source = sources[0] - } - - taskName := cmd.Flags().Arg(0) - if strings.TrimSpace(taskName) == "" { - common.PrintLines("must specify the task-name") - err = errors.New("please check output to see error") - return - } - - extraArgs := cmd.Flags().Args()[1:] - realSQLs, err := common.ExtractSQLsFromArgs(extraArgs) - if err != nil { - common.PrintLines("check SQLs error") - return - } - - ctx, cancel := context.WithCancel(context.Background()) - defer cancel() - cli := common.MasterClient() - resp, err := cli.HandleSQLs(ctx, &pb.HandleSQLsRequest{ - Name: taskName, - Source: source, - Op: pb.SQLOp_REPLACE, - Args: realSQLs, - BinlogPos: binlogPos, - SqlPattern: sqlPattern, - Sharding: sharding, - }) - if err != nil { - return - } - - common.PrettyPrintResponse(resp) - return -} diff --git a/dm/ctl/master/sql_skip.go b/dm/ctl/master/sql_skip.go deleted file mode 100644 index 3e81940ff6..0000000000 --- a/dm/ctl/master/sql_skip.go +++ /dev/null @@ -1,98 +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" - "errors" - "os" - "strings" - - "github.com/spf13/cobra" - - "github.com/pingcap/dm/dm/ctl/common" - "github.com/pingcap/dm/dm/pb" -) - -// NewSQLSkipCmd creates a SQLSkip command -func NewSQLSkipCmd() *cobra.Command { - cmd := &cobra.Command{ - Use: "sql-skip <-s source> [-b binlog-pos] [-p sql-pattern] [--sharding] ", - Short: "skip the binlog event matched by a specific binlog position (binlog-pos) or a SQL pattern (sql-pattern)", - RunE: sqlSkipFunc, - } - cmd.Flags().StringP("binlog-pos", "b", "", "position used to match binlog event if matched the sql-skip operation will be applied. The format like \"mysql-bin|000001.000003:3270\"") - cmd.Flags().StringP("sql-pattern", "p", "", "SQL pattern used to match the DDL converted by optional router-rules if matched the sql-skip operation will be applied. The format like \"~(?i)ALTER\\s+TABLE\\s+`db1`.`tbl1`\\s+ADD\\s+COLUMN\\s+col1\\s+INT\". Whitespace is not supported, and must be replaced by \"\\s\". Staring with ~ as regular expression. This can only be used for DDL (converted by optional router-rules), and if multi DDLs in one binlog event, one of them matched is enough, but all of them will be skipped") - cmd.Flags().BoolP("sharding", "", false, "whether are handing sharding DDL, which will only take effect on DDL lock's owner") - return cmd -} - -// sqlSkipFunc does sql skip request -func sqlSkipFunc(cmd *cobra.Command, _ []string) (err error) { - if len(cmd.Flags().Args()) != 1 { - cmd.SetOut(os.Stdout) - cmd.Usage() - err = errors.New("please check output to see error") - return - } - - binlogPos, sqlPattern, sharding, err := extractBinlogPosSQLPattern(cmd) - if err != nil { - return - } - - var source string - sources, err := common.GetSourceArgs(cmd) - if err != nil { - return - } - if sharding { - if len(sources) != 0 { - common.PrintLines("--sharding operator always takes effect on DDL lock's owner, specified sources %v arguments will be ignored", sources) - } - } else { - if len(sources) != 1 { - common.PrintLines("should only specify one source, but got %v", sources) - err = errors.New("please check output to see error") - return - } - source = sources[0] - } - - taskName := cmd.Flags().Arg(0) - if strings.TrimSpace(taskName) == "" { - common.PrintLines("must specify the task-name") - err = errors.New("please check output to see error") - return - } - - ctx, cancel := context.WithCancel(context.Background()) - defer cancel() - cli := common.MasterClient() - - resp, err := cli.HandleSQLs(ctx, &pb.HandleSQLsRequest{ - Name: taskName, - Source: source, - Op: pb.SQLOp_SKIP, - BinlogPos: binlogPos, - SqlPattern: sqlPattern, - Sharding: sharding, - }) - if err != nil { - return - } - - common.PrettyPrintResponse(resp) - return -} diff --git a/dm/master/server.go b/dm/master/server.go index e9f53e7c9e..b607bd48c1 100644 --- a/dm/master/server.go +++ b/dm/master/server.go @@ -40,7 +40,6 @@ import ( "github.com/pingcap/dm/dm/master/metrics" "github.com/pingcap/dm/dm/master/scheduler" "github.com/pingcap/dm/dm/master/shardddl" - operator "github.com/pingcap/dm/dm/master/sql-operator" "github.com/pingcap/dm/dm/master/workerrpc" "github.com/pingcap/dm/dm/pb" "github.com/pingcap/dm/pkg/conn" @@ -112,9 +111,6 @@ type Server struct { // shard DDL optimist optimist *shardddl.Optimist - // SQL operator holder - sqlOperatorHolder *operator.Holder - // trace group id generator idGen *tracing.IDGenerator @@ -128,11 +124,10 @@ type Server struct { func NewServer(cfg *Config) *Server { logger := log.L() server := Server{ - cfg: cfg, - scheduler: scheduler.NewScheduler(&logger, cfg.Security), - sqlOperatorHolder: operator.NewHolder(), - idGen: tracing.NewIDGen(), - ap: NewAgentPool(&RateLimitConfig{rate: cfg.RPCRateLimit, burst: cfg.RPCRateBurst}), + cfg: cfg, + scheduler: scheduler.NewScheduler(&logger, cfg.Security), + idGen: tracing.NewIDGen(), + ap: NewAgentPool(&RateLimitConfig{rate: cfg.RPCRateLimit, burst: cfg.RPCRateBurst}), } server.pessimist = shardddl.NewPessimist(&logger, server.getTaskResources) server.optimist = shardddl.NewOptimist(&logger) @@ -859,72 +854,6 @@ func (s *Server) UnlockDDLLock(ctx context.Context, req *pb.UnlockDDLLockRequest return resp, nil } -// HandleSQLs implements MasterServer.HandleSQLs -func (s *Server) HandleSQLs(ctx context.Context, req *pb.HandleSQLsRequest) (*pb.HandleSQLsResponse, error) { - log.L().Info("", zap.String("task name", req.Name), zap.Stringer("payload", req), zap.String("request", "HandleSQLs")) - - isLeader, needForward := s.isLeaderAndNeedForward() - if !isLeader { - if needForward { - return s.leaderClient.HandleSQLs(ctx, req) - } - return nil, terror.ErrMasterRequestIsNotForwardToLeader - } - - // save request for --sharding operation - if req.Sharding { - err := s.sqlOperatorHolder.Set(req) - if err != nil { - return &pb.HandleSQLsResponse{ - Result: false, - Msg: fmt.Sprintf("save request with --sharding error:\n%v", err), - }, nil - } - log.L().Info("handle sqls request was saved", zap.String("task name", req.Name), zap.String("request", "HandleSQLs")) - return &pb.HandleSQLsResponse{ - Result: true, - Msg: "request with --sharding saved and will be sent to DDL lock's owner when resolving DDL lock", - }, nil - } - - resp := &pb.HandleSQLsResponse{ - Result: false, - Msg: "", - } - - if !s.checkTaskAndWorkerMatch(req.Name, req.Source) { - resp.Msg = fmt.Sprintf("task %s and worker %s not match, can try `refresh-worker-tasks` cmd first", req.Name, req.Source) - return resp, nil - } - - // execute grpc call - subReq := &workerrpc.Request{ - Type: workerrpc.CmdHandleSubTaskSQLs, - HandleSubTaskSQLs: &pb.HandleSubTaskSQLsRequest{ - Name: req.Name, - Op: req.Op, - Args: req.Args, - BinlogPos: req.BinlogPos, - SqlPattern: req.SqlPattern, - }, - } - worker := s.scheduler.GetWorkerBySource(req.Source) - if worker == nil { - resp.Msg = fmt.Sprintf("source %s not found in bound sources %v", req.Source, s.scheduler.BoundSources()) - return resp, nil - } - response, err := worker.SendRequest(ctx, subReq, s.cfg.RPCTimeout) - workerResp := &pb.CommonWorkerResponse{} - if err != nil { - workerResp = errorCommonWorkerResponse(err.Error(), req.Source, worker.BaseInfo().Name) - } else { - workerResp = response.HandleSubTaskSQLs - } - resp.Sources = []*pb.CommonWorkerResponse{workerResp} - resp.Result = true - return resp, nil -} - // PurgeWorkerRelay implements MasterServer.PurgeWorkerRelay func (s *Server) PurgeWorkerRelay(ctx context.Context, req *pb.PurgeWorkerRelayRequest) (*pb.PurgeWorkerRelayResponse, error) { log.L().Info("", zap.Stringer("payload", req), zap.String("request", "PurgeWorkerRelay")) diff --git a/dm/master/shardddl/pessimist.go b/dm/master/shardddl/pessimist.go index bd085550fe..1a7d51a0a5 100644 --- a/dm/master/shardddl/pessimist.go +++ b/dm/master/shardddl/pessimist.go @@ -330,7 +330,6 @@ func (p *Pessimist) UnlockLock(ctx context.Context, ID, replaceOwner string, for }() // 6. put `exec` operation for the owner, and wait for the owner to be done. - // TODO: `sql-skip`/`sql-replace` supported later. done, err := p.waitOwnerToBeDone(ctx, lock, owner) if err != nil { revertLockSync = true diff --git a/dm/master/sql-operator/operator.go b/dm/master/sql-operator/operator.go deleted file mode 100644 index 96186aaa1c..0000000000 --- a/dm/master/sql-operator/operator.go +++ /dev/null @@ -1,163 +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 operator - -import ( - "fmt" - "regexp" - "sync" - - "github.com/golang/protobuf/proto" - uuid "github.com/satori/go.uuid" - "go.uber.org/zap" - - "github.com/pingcap/dm/dm/command" - "github.com/pingcap/dm/dm/pb" - "github.com/pingcap/dm/pkg/log" - "github.com/pingcap/dm/pkg/terror" -) - -// Operator contains an operation for specified binlog pos -// used by `sql-skip` and `sql-replace` -type Operator struct { - uuid string - Req *pb.HandleSQLsRequest - reg *regexp.Regexp -} - -// newOperator creates a new Operator with a random UUID -func newOperator(req *pb.HandleSQLsRequest, reg *regexp.Regexp) *Operator { - return &Operator{ - uuid: uuid.NewV4().String(), - Req: req, - reg: reg, - } -} - -// matchPattern tries to match SQL with the regexp -func (o *Operator) matchPattern(sql string) bool { - if o.reg == nil { - return false - } - return o.reg.MatchString(sql) -} - -// clone returns a deep copy -func (o *Operator) clone() *Operator { - return &Operator{ - uuid: o.uuid, - reg: o.reg.Copy(), - Req: proto.Clone(o.Req).(*pb.HandleSQLsRequest), - } -} - -func (o *Operator) String() string { - return fmt.Sprintf("uuid: %s, request: %s", o.uuid, o.Req) -} - -// Holder holds SQL operators -type Holder struct { - mu sync.RWMutex - operators map[string]map[string]*Operator // taskName -> Key(sql-pattern) -> Operator - logger log.Logger -} - -// NewHolder creates a new Holder -func NewHolder() *Holder { - return &Holder{ - operators: make(map[string]map[string]*Operator), - logger: log.With(zap.String("component", "sql operator")), - } -} - -// Set sets an operator according to request -func (h *Holder) Set(req *pb.HandleSQLsRequest) error { - if req == nil { - return terror.ErrMasterSQLOpNilRequest.Generate() - } - switch req.Op { - case pb.SQLOp_SKIP, pb.SQLOp_REPLACE: - default: - return terror.ErrMasterSQLOpNotSupport.Generate(req.Op) - } - - // now, only support --sharding operate request - if !req.Sharding { - return terror.ErrMasterSQLOpWithoutSharding.Generate() - } - - _, sqlReg, err := command.VerifySQLOperateArgs(req.BinlogPos, req.SqlPattern, req.Sharding) - if err != nil { - return terror.WithClass(err, terror.ClassDMMaster) - } - - h.mu.Lock() - defer h.mu.Unlock() - - key := req.SqlPattern - oper := newOperator(req, sqlReg) - operators, ok1 := h.operators[req.Name] - if ok1 { - prev, ok2 := operators[key] - if ok2 { - h.logger.Warn("overwrite operation", zap.Stringer("previous operation", prev), zap.Stringer("current operation", oper)) - } - } else { - operators = make(map[string]*Operator) - h.operators[req.Name] = operators - } - operators[key] = oper - h.logger.Info("set operation", zap.Stringer("operation", oper)) - return nil -} - -// Get tries to get an operator by taskName and SQLs, returns key and operator -func (h *Holder) Get(taskName string, sqls []string) (string, *Operator) { - h.mu.RLock() - defer h.mu.RUnlock() - - operators, ok := h.operators[taskName] - if !ok { - return "", nil - } - - for _, sql := range sqls { - for key, oper := range operators { - if oper.matchPattern(sql) { // matched one SQL of all is enough - h.logger.Info("get a matched operator", zap.Stringer("operation", oper), zap.String("key", key), zap.String("sql", sql)) - return key, oper.clone() - } - } - } - return "", nil -} - -// Remove removes the operator with the key -func (h *Holder) Remove(taskName, key string) { - h.mu.Lock() - defer h.mu.Unlock() - operators, ok := h.operators[taskName] - if !ok { - return - } - - oper, ok := operators[key] - if ok { - delete(operators, key) - if len(operators) == 0 { - delete(h.operators, taskName) - } - h.logger.Info("remove operator", zap.Stringer("operation", oper)) - } -} diff --git a/dm/master/sql-operator/operator_test.go b/dm/master/sql-operator/operator_test.go deleted file mode 100644 index 3fb37461e6..0000000000 --- a/dm/master/sql-operator/operator_test.go +++ /dev/null @@ -1,91 +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 operator - -import ( - "testing" - - . "github.com/pingcap/check" - - "github.com/pingcap/dm/dm/pb" -) - -var _ = Suite(&testOperatorSuite{}) - -func TestSuite(t *testing.T) { - TestingT(t) -} - -type testOperatorSuite struct { -} - -func (o *testOperatorSuite) TestOperator(c *C) { - h := NewHolder() - - // nil request - err := h.Set(nil) - c.Assert(err, NotNil) - - // not support op - req := &pb.HandleSQLsRequest{Name: "task-A", Op: pb.SQLOp_INJECT} - err = h.Set(req) - c.Assert(err, NotNil) - - // no --sharding - req.Op = pb.SQLOp_SKIP - err = h.Set(req) - c.Assert(err, NotNil) - - // with --binlog-pos - req.Sharding = true - req.BinlogPos = "mysql-bin.000001:234" - err = h.Set(req) - c.Assert(err, NotNil) - - // without --sql-pattern - req.BinlogPos = "" - err = h.Set(req) - c.Assert(err, NotNil) - - // valid - req.SqlPattern = "~(?i)ALTER\\s+TABLE\\s+`db1`.`tbl1`\\s+ADD\\s+COLUMN\\s+col1\\s+INT" - err = h.Set(req) - c.Assert(err, IsNil) - - // get, SQLs mismatch - sqls := []string{"INSERT INTO `d1`.`t1` VALUES (1, 2)"} - key, oper := h.Get(req.Name, sqls) - c.Assert(key, Equals, "") - c.Assert(oper, IsNil) - - // get, taskName mismatch - sqls = []string{"ALTER TABLE `db1`.`tbl1` ADD COLUMN col1 INT"} - key, oper = h.Get("not-exist-task", sqls) - c.Assert(key, Equals, "") - c.Assert(oper, IsNil) - - // get, matched - key, oper = h.Get(req.Name, sqls) - c.Assert(key, Equals, req.SqlPattern) - c.Assert(oper, NotNil) - c.Assert(oper.Req, DeepEquals, req) - - // remove - h.Remove(req.Name, key) - - // get, not exists - key, oper = h.Get(req.Name, sqls) - c.Assert(key, Equals, "") - c.Assert(oper, IsNil) -} diff --git a/dm/master/workerrpc/interface.go b/dm/master/workerrpc/interface.go index ad98f08740..2762d056f2 100644 --- a/dm/master/workerrpc/interface.go +++ b/dm/master/workerrpc/interface.go @@ -34,8 +34,6 @@ const ( CmdQueryTaskOperation CmdQueryWorkerConfig - CmdHandleSubTaskSQLs - CmdSwitchRelayMaster CmdOperateRelay CmdPurgeRelay @@ -59,8 +57,6 @@ type Request struct { QueryError *pb.QueryErrorRequest QueryWorkerConfig *pb.QueryWorkerConfigRequest - HandleSubTaskSQLs *pb.HandleSubTaskSQLsRequest - SwitchRelayMaster *pb.SwitchRelayMasterRequest OperateRelay *pb.OperateRelayRequest PurgeRelay *pb.PurgeRelayRequest @@ -82,8 +78,6 @@ type Response struct { QueryError *pb.QueryErrorResponse QueryWorkerConfig *pb.QueryWorkerConfigResponse - HandleSubTaskSQLs *pb.CommonWorkerResponse - SwitchRelayMaster *pb.CommonWorkerResponse OperateRelay *pb.OperateRelayResponse PurgeRelay *pb.CommonWorkerResponse diff --git a/dm/master/workerrpc/rawgrpc.go b/dm/master/workerrpc/rawgrpc.go index 78300e9fd7..4a6bf07342 100644 --- a/dm/master/workerrpc/rawgrpc.go +++ b/dm/master/workerrpc/rawgrpc.go @@ -114,8 +114,6 @@ func callRPC(ctx context.Context, client pb.WorkerClient, req *Request) (*Respon resp.QueryError, err = client.QueryError(ctx, req.QueryError) case CmdQueryWorkerConfig: resp.QueryWorkerConfig, err = client.QueryWorkerConfig(ctx, req.QueryWorkerConfig) - case CmdHandleSubTaskSQLs: - resp.HandleSubTaskSQLs, err = client.HandleSQLs(ctx, req.HandleSubTaskSQLs) case CmdSwitchRelayMaster: resp.SwitchRelayMaster, err = client.SwitchRelayMaster(ctx, req.SwitchRelayMaster) case CmdOperateRelay: diff --git a/dm/pb/dmmaster.pb.go b/dm/pb/dmmaster.pb.go index fdc6bdb44f..7394f69b6a 100644 --- a/dm/pb/dmmaster.pb.go +++ b/dm/pb/dmmaster.pb.go @@ -1456,158 +1456,6 @@ func (m *OperateWorkerRelayResponse) GetSources() []*CommonWorkerResponse { 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,proto3" json:"args,omitempty"` - BinlogPos string `protobuf:"bytes,4,opt,name=binlogPos,proto3" json:"binlogPos,omitempty"` - Source string `protobuf:"bytes,5,opt,name=source,proto3" json:"source,omitempty"` - SqlPattern string `protobuf:"bytes,6,opt,name=sqlPattern,proto3" json:"sqlPattern,omitempty"` - Sharding bool `protobuf:"varint,7,opt,name=sharding,proto3" json:"sharding,omitempty"` -} - -func (m *HandleSQLsRequest) Reset() { *m = HandleSQLsRequest{} } -func (m *HandleSQLsRequest) String() string { return proto.CompactTextString(m) } -func (*HandleSQLsRequest) ProtoMessage() {} -func (*HandleSQLsRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_f9bef11f2a341f03, []int{23} -} -func (m *HandleSQLsRequest) XXX_Unmarshal(b []byte) error { - return m.Unmarshal(b) -} -func (m *HandleSQLsRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { - if deterministic { - return xxx_messageInfo_HandleSQLsRequest.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 *HandleSQLsRequest) XXX_Merge(src proto.Message) { - xxx_messageInfo_HandleSQLsRequest.Merge(m, src) -} -func (m *HandleSQLsRequest) XXX_Size() int { - return m.Size() -} -func (m *HandleSQLsRequest) XXX_DiscardUnknown() { - xxx_messageInfo_HandleSQLsRequest.DiscardUnknown(m) -} - -var xxx_messageInfo_HandleSQLsRequest proto.InternalMessageInfo - -func (m *HandleSQLsRequest) GetName() string { - if m != nil { - return m.Name - } - return "" -} - -func (m *HandleSQLsRequest) GetOp() SQLOp { - if m != nil { - return m.Op - } - return SQLOp_SKIP -} - -func (m *HandleSQLsRequest) GetArgs() []string { - if m != nil { - return m.Args - } - return nil -} - -func (m *HandleSQLsRequest) GetBinlogPos() string { - if m != nil { - return m.BinlogPos - } - return "" -} - -func (m *HandleSQLsRequest) GetSource() string { - if m != nil { - return m.Source - } - return "" -} - -func (m *HandleSQLsRequest) GetSqlPattern() string { - if m != nil { - return m.SqlPattern - } - return "" -} - -func (m *HandleSQLsRequest) GetSharding() bool { - if m != nil { - return m.Sharding - } - return false -} - -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"` - Sources []*CommonWorkerResponse `protobuf:"bytes,3,rep,name=sources,proto3" json:"sources,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_f9bef11f2a341f03, []int{24} -} -func (m *HandleSQLsResponse) XXX_Unmarshal(b []byte) error { - return m.Unmarshal(b) -} -func (m *HandleSQLsResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { - if deterministic { - return xxx_messageInfo_HandleSQLsResponse.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 *HandleSQLsResponse) XXX_Merge(src proto.Message) { - xxx_messageInfo_HandleSQLsResponse.Merge(m, src) -} -func (m *HandleSQLsResponse) XXX_Size() int { - return m.Size() -} -func (m *HandleSQLsResponse) XXX_DiscardUnknown() { - xxx_messageInfo_HandleSQLsResponse.DiscardUnknown(m) -} - -var xxx_messageInfo_HandleSQLsResponse proto.InternalMessageInfo - -func (m *HandleSQLsResponse) GetResult() bool { - if m != nil { - return m.Result - } - return false -} - -func (m *HandleSQLsResponse) GetMsg() string { - if m != nil { - return m.Msg - } - return "" -} - -func (m *HandleSQLsResponse) GetSources() []*CommonWorkerResponse { - if m != nil { - return m.Sources - } - return nil -} - // PurgeWorkerRelayRequest represents a request to purge relay log files for some dm-workers // workers: dm-workers need to purge relay log files // inactive: whether purge inactive relay log files @@ -1626,7 +1474,7 @@ func (m *PurgeWorkerRelayRequest) Reset() { *m = PurgeWorkerRelayRequest func (m *PurgeWorkerRelayRequest) String() string { return proto.CompactTextString(m) } func (*PurgeWorkerRelayRequest) ProtoMessage() {} func (*PurgeWorkerRelayRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_f9bef11f2a341f03, []int{25} + return fileDescriptor_f9bef11f2a341f03, []int{23} } func (m *PurgeWorkerRelayRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1700,7 +1548,7 @@ func (m *PurgeWorkerRelayResponse) Reset() { *m = PurgeWorkerRelayRespon func (m *PurgeWorkerRelayResponse) String() string { return proto.CompactTextString(m) } func (*PurgeWorkerRelayResponse) ProtoMessage() {} func (*PurgeWorkerRelayResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_f9bef11f2a341f03, []int{26} + return fileDescriptor_f9bef11f2a341f03, []int{24} } func (m *PurgeWorkerRelayResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1758,7 +1606,7 @@ func (m *CheckTaskRequest) Reset() { *m = CheckTaskRequest{} } func (m *CheckTaskRequest) String() string { return proto.CompactTextString(m) } func (*CheckTaskRequest) ProtoMessage() {} func (*CheckTaskRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_f9bef11f2a341f03, []int{27} + return fileDescriptor_f9bef11f2a341f03, []int{25} } func (m *CheckTaskRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1803,7 +1651,7 @@ func (m *CheckTaskResponse) Reset() { *m = CheckTaskResponse{} } func (m *CheckTaskResponse) String() string { return proto.CompactTextString(m) } func (*CheckTaskResponse) ProtoMessage() {} func (*CheckTaskResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_f9bef11f2a341f03, []int{28} + return fileDescriptor_f9bef11f2a341f03, []int{26} } func (m *CheckTaskResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1855,7 +1703,7 @@ func (m *OperateSourceRequest) Reset() { *m = OperateSourceRequest{} } func (m *OperateSourceRequest) String() string { return proto.CompactTextString(m) } func (*OperateSourceRequest) ProtoMessage() {} func (*OperateSourceRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_f9bef11f2a341f03, []int{29} + return fileDescriptor_f9bef11f2a341f03, []int{27} } func (m *OperateSourceRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1908,7 +1756,7 @@ func (m *OperateSourceResponse) Reset() { *m = OperateSourceResponse{} } func (m *OperateSourceResponse) String() string { return proto.CompactTextString(m) } func (*OperateSourceResponse) ProtoMessage() {} func (*OperateSourceResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_f9bef11f2a341f03, []int{30} + return fileDescriptor_f9bef11f2a341f03, []int{28} } func (m *OperateSourceResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1967,7 +1815,7 @@ func (m *RegisterWorkerRequest) Reset() { *m = RegisterWorkerRequest{} } func (m *RegisterWorkerRequest) String() string { return proto.CompactTextString(m) } func (*RegisterWorkerRequest) ProtoMessage() {} func (*RegisterWorkerRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_f9bef11f2a341f03, []int{31} + return fileDescriptor_f9bef11f2a341f03, []int{29} } func (m *RegisterWorkerRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2019,7 +1867,7 @@ func (m *RegisterWorkerResponse) Reset() { *m = RegisterWorkerResponse{} func (m *RegisterWorkerResponse) String() string { return proto.CompactTextString(m) } func (*RegisterWorkerResponse) ProtoMessage() {} func (*RegisterWorkerResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_f9bef11f2a341f03, []int{32} + return fileDescriptor_f9bef11f2a341f03, []int{30} } func (m *RegisterWorkerResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2071,7 +1919,7 @@ func (m *OfflineMemberRequest) Reset() { *m = OfflineMemberRequest{} } func (m *OfflineMemberRequest) String() string { return proto.CompactTextString(m) } func (*OfflineMemberRequest) ProtoMessage() {} func (*OfflineMemberRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_f9bef11f2a341f03, []int{33} + return fileDescriptor_f9bef11f2a341f03, []int{31} } func (m *OfflineMemberRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2123,7 +1971,7 @@ func (m *OfflineMemberResponse) Reset() { *m = OfflineMemberResponse{} } func (m *OfflineMemberResponse) String() string { return proto.CompactTextString(m) } func (*OfflineMemberResponse) ProtoMessage() {} func (*OfflineMemberResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_f9bef11f2a341f03, []int{34} + return fileDescriptor_f9bef11f2a341f03, []int{32} } func (m *OfflineMemberResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2174,7 +2022,7 @@ func (m *OperateLeaderRequest) Reset() { *m = OperateLeaderRequest{} } func (m *OperateLeaderRequest) String() string { return proto.CompactTextString(m) } func (*OperateLeaderRequest) ProtoMessage() {} func (*OperateLeaderRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_f9bef11f2a341f03, []int{35} + return fileDescriptor_f9bef11f2a341f03, []int{33} } func (m *OperateLeaderRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2219,7 +2067,7 @@ func (m *OperateLeaderResponse) Reset() { *m = OperateLeaderResponse{} } func (m *OperateLeaderResponse) String() string { return proto.CompactTextString(m) } func (*OperateLeaderResponse) ProtoMessage() {} func (*OperateLeaderResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_f9bef11f2a341f03, []int{36} + return fileDescriptor_f9bef11f2a341f03, []int{34} } func (m *OperateLeaderResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2274,7 +2122,7 @@ func (m *MasterInfo) Reset() { *m = MasterInfo{} } func (m *MasterInfo) String() string { return proto.CompactTextString(m) } func (*MasterInfo) ProtoMessage() {} func (*MasterInfo) Descriptor() ([]byte, []int) { - return fileDescriptor_f9bef11f2a341f03, []int{37} + return fileDescriptor_f9bef11f2a341f03, []int{35} } func (m *MasterInfo) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2349,7 +2197,7 @@ func (m *WorkerInfo) Reset() { *m = WorkerInfo{} } func (m *WorkerInfo) String() string { return proto.CompactTextString(m) } func (*WorkerInfo) ProtoMessage() {} func (*WorkerInfo) Descriptor() ([]byte, []int) { - return fileDescriptor_f9bef11f2a341f03, []int{38} + return fileDescriptor_f9bef11f2a341f03, []int{36} } func (m *WorkerInfo) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2416,7 +2264,7 @@ func (m *ListLeaderMember) Reset() { *m = ListLeaderMember{} } func (m *ListLeaderMember) String() string { return proto.CompactTextString(m) } func (*ListLeaderMember) ProtoMessage() {} func (*ListLeaderMember) Descriptor() ([]byte, []int) { - return fileDescriptor_f9bef11f2a341f03, []int{39} + return fileDescriptor_f9bef11f2a341f03, []int{37} } func (m *ListLeaderMember) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2475,7 +2323,7 @@ func (m *ListMasterMember) Reset() { *m = ListMasterMember{} } func (m *ListMasterMember) String() string { return proto.CompactTextString(m) } func (*ListMasterMember) ProtoMessage() {} func (*ListMasterMember) Descriptor() ([]byte, []int) { - return fileDescriptor_f9bef11f2a341f03, []int{40} + return fileDescriptor_f9bef11f2a341f03, []int{38} } func (m *ListMasterMember) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2527,7 +2375,7 @@ func (m *ListWorkerMember) Reset() { *m = ListWorkerMember{} } func (m *ListWorkerMember) String() string { return proto.CompactTextString(m) } func (*ListWorkerMember) ProtoMessage() {} func (*ListWorkerMember) Descriptor() ([]byte, []int) { - return fileDescriptor_f9bef11f2a341f03, []int{41} + return fileDescriptor_f9bef11f2a341f03, []int{39} } func (m *ListWorkerMember) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2582,7 +2430,7 @@ func (m *Members) Reset() { *m = Members{} } func (m *Members) String() string { return proto.CompactTextString(m) } func (*Members) ProtoMessage() {} func (*Members) Descriptor() ([]byte, []int) { - return fileDescriptor_f9bef11f2a341f03, []int{42} + return fileDescriptor_f9bef11f2a341f03, []int{40} } func (m *Members) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2679,7 +2527,7 @@ func (m *ListMemberRequest) Reset() { *m = ListMemberRequest{} } func (m *ListMemberRequest) String() string { return proto.CompactTextString(m) } func (*ListMemberRequest) ProtoMessage() {} func (*ListMemberRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_f9bef11f2a341f03, []int{43} + return fileDescriptor_f9bef11f2a341f03, []int{41} } func (m *ListMemberRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2746,7 +2594,7 @@ func (m *ListMemberResponse) Reset() { *m = ListMemberResponse{} } func (m *ListMemberResponse) String() string { return proto.CompactTextString(m) } func (*ListMemberResponse) ProtoMessage() {} func (*ListMemberResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_f9bef11f2a341f03, []int{44} + return fileDescriptor_f9bef11f2a341f03, []int{42} } func (m *ListMemberResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2809,7 +2657,7 @@ func (m *OperateSchemaRequest) Reset() { *m = OperateSchemaRequest{} } func (m *OperateSchemaRequest) String() string { return proto.CompactTextString(m) } func (*OperateSchemaRequest) ProtoMessage() {} func (*OperateSchemaRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_f9bef11f2a341f03, []int{45} + return fileDescriptor_f9bef11f2a341f03, []int{43} } func (m *OperateSchemaRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2890,7 +2738,7 @@ func (m *OperateSchemaResponse) Reset() { *m = OperateSchemaResponse{} } func (m *OperateSchemaResponse) String() string { return proto.CompactTextString(m) } func (*OperateSchemaResponse) ProtoMessage() {} func (*OperateSchemaResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_f9bef11f2a341f03, []int{46} + return fileDescriptor_f9bef11f2a341f03, []int{44} } func (m *OperateSchemaResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2949,7 +2797,7 @@ func (m *GetSubTaskCfgRequest) Reset() { *m = GetSubTaskCfgRequest{} } func (m *GetSubTaskCfgRequest) String() string { return proto.CompactTextString(m) } func (*GetSubTaskCfgRequest) ProtoMessage() {} func (*GetSubTaskCfgRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_f9bef11f2a341f03, []int{47} + return fileDescriptor_f9bef11f2a341f03, []int{45} } func (m *GetSubTaskCfgRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2995,7 +2843,7 @@ func (m *GetSubTaskCfgResponse) Reset() { *m = GetSubTaskCfgResponse{} } func (m *GetSubTaskCfgResponse) String() string { return proto.CompactTextString(m) } func (*GetSubTaskCfgResponse) ProtoMessage() {} func (*GetSubTaskCfgResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_f9bef11f2a341f03, []int{48} + return fileDescriptor_f9bef11f2a341f03, []int{46} } func (m *GetSubTaskCfgResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3054,7 +2902,7 @@ func (m *GetTaskCfgRequest) Reset() { *m = GetTaskCfgRequest{} } func (m *GetTaskCfgRequest) String() string { return proto.CompactTextString(m) } func (*GetTaskCfgRequest) ProtoMessage() {} func (*GetTaskCfgRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_f9bef11f2a341f03, []int{49} + return fileDescriptor_f9bef11f2a341f03, []int{47} } func (m *GetTaskCfgRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3100,7 +2948,7 @@ func (m *GetTaskCfgResponse) Reset() { *m = GetTaskCfgResponse{} } func (m *GetTaskCfgResponse) String() string { return proto.CompactTextString(m) } func (*GetTaskCfgResponse) ProtoMessage() {} func (*GetTaskCfgResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_f9bef11f2a341f03, []int{50} + return fileDescriptor_f9bef11f2a341f03, []int{48} } func (m *GetTaskCfgResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3176,8 +3024,6 @@ func init() { proto.RegisterType((*SwitchWorkerRelayMasterResponse)(nil), "pb.SwitchWorkerRelayMasterResponse") proto.RegisterType((*OperateWorkerRelayRequest)(nil), "pb.OperateWorkerRelayRequest") proto.RegisterType((*OperateWorkerRelayResponse)(nil), "pb.OperateWorkerRelayResponse") - proto.RegisterType((*HandleSQLsRequest)(nil), "pb.HandleSQLsRequest") - proto.RegisterType((*HandleSQLsResponse)(nil), "pb.HandleSQLsResponse") proto.RegisterType((*PurgeWorkerRelayRequest)(nil), "pb.PurgeWorkerRelayRequest") proto.RegisterType((*PurgeWorkerRelayResponse)(nil), "pb.PurgeWorkerRelayResponse") proto.RegisterType((*CheckTaskRequest)(nil), "pb.CheckTaskRequest") @@ -3209,134 +3055,128 @@ func init() { func init() { proto.RegisterFile("dmmaster.proto", fileDescriptor_f9bef11f2a341f03) } var fileDescriptor_f9bef11f2a341f03 = []byte{ - // 2022 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xbc, 0x59, 0xcd, 0x6f, 0xe3, 0xc6, - 0x15, 0x17, 0x25, 0xaf, 0x2d, 0x3f, 0xaf, 0x1d, 0x79, 0xd6, 0x96, 0x65, 0xae, 0x57, 0xeb, 0x4c, - 0x92, 0xad, 0x61, 0x14, 0xeb, 0xae, 0xd3, 0xd3, 0x02, 0x29, 0x90, 0xb5, 0x9c, 0xc4, 0xa8, 0x36, - 0xf6, 0x52, 0x35, 0x8a, 0xa0, 0x40, 0x11, 0x8a, 0x1a, 0xc9, 0x84, 0x29, 0x92, 0x26, 0x29, 0xbb, - 0xc6, 0x22, 0x28, 0xd0, 0x43, 0x8f, 0x6d, 0x81, 0x1e, 0x72, 0xec, 0xa1, 0xd7, 0xfe, 0x21, 0x39, - 0x06, 0xe8, 0xa5, 0xc7, 0x62, 0xb7, 0x7f, 0x48, 0x31, 0x6f, 0x86, 0xe4, 0xf0, 0x43, 0xea, 0x6a, - 0x81, 0xfa, 0xc6, 0x37, 0x1f, 0xef, 0xfd, 0xe6, 0xbd, 0x37, 0xef, 0x63, 0x08, 0x6b, 0x83, 0xf1, - 0xd8, 0x0c, 0x23, 0x16, 0x3c, 0xf5, 0x03, 0x2f, 0xf2, 0x48, 0xd5, 0xef, 0xeb, 0x6b, 0x83, 0xf1, - 0x8d, 0x17, 0x5c, 0xc6, 0x63, 0xfa, 0xce, 0xc8, 0xf3, 0x46, 0x0e, 0x3b, 0x30, 0x7d, 0xfb, 0xc0, - 0x74, 0x5d, 0x2f, 0x32, 0x23, 0xdb, 0x73, 0x43, 0x31, 0x4b, 0xaf, 0x60, 0xfb, 0xa5, 0x3d, 0x0a, - 0xcc, 0x88, 0xfd, 0x1a, 0x37, 0x19, 0xcc, 0x31, 0x6f, 0x0d, 0x76, 0x35, 0x61, 0x61, 0x44, 0xda, - 0x00, 0x2f, 0x6c, 0xd7, 0xf1, 0x46, 0x5f, 0x9b, 0x63, 0xd6, 0xd2, 0x76, 0xb5, 0xbd, 0x65, 0x43, - 0x19, 0x21, 0x3b, 0xb0, 0x2c, 0xa8, 0x33, 0x2f, 0x6c, 0x55, 0x77, 0xb5, 0xbd, 0x55, 0x23, 0x1d, - 0x20, 0x4d, 0x58, 0x0c, 0xbd, 0x49, 0x60, 0xb1, 0x56, 0x0d, 0x77, 0x4a, 0x8a, 0x9e, 0x41, 0xfb, - 0xdc, 0x1f, 0x64, 0x25, 0x1e, 0x79, 0xee, 0xd0, 0x1e, 0xc5, 0x72, 0x9b, 0xb0, 0x68, 0xe1, 0x80, - 0x94, 0x29, 0x29, 0x85, 0x63, 0x35, 0xc3, 0xf1, 0x5b, 0x68, 0xf4, 0x22, 0x33, 0x88, 0x7e, 0x65, - 0x86, 0x97, 0x31, 0x0f, 0x02, 0x0b, 0x91, 0x19, 0x5e, 0x4a, 0x0e, 0xf8, 0x4d, 0x5a, 0xb0, 0x24, - 0x76, 0x70, 0xb4, 0xb5, 0xbd, 0x65, 0x23, 0x26, 0xf9, 0x49, 0x03, 0x36, 0xf6, 0xae, 0xd9, 0x4b, - 0x16, 0x99, 0x88, 0xb7, 0x6e, 0x28, 0x23, 0xf4, 0x0a, 0xd6, 0x15, 0x09, 0xa1, 0xef, 0xb9, 0x21, - 0xe3, 0x70, 0x02, 0x16, 0x4e, 0x9c, 0x08, 0x85, 0xd4, 0x0d, 0x49, 0x91, 0x06, 0xd4, 0xc6, 0xe1, - 0x48, 0x62, 0xe4, 0x9f, 0xe4, 0x30, 0x15, 0x5c, 0xdb, 0xad, 0xed, 0xad, 0x1c, 0xb6, 0x9e, 0xfa, - 0xfd, 0xa7, 0x47, 0xde, 0x78, 0xec, 0xb9, 0xb1, 0x16, 0x04, 0xd3, 0x04, 0x12, 0xfd, 0x14, 0xb6, - 0x85, 0x9a, 0x5e, 0xa2, 0x85, 0xdf, 0x49, 0x43, 0xf4, 0x16, 0xf4, 0xb2, 0x4d, 0x73, 0x03, 0x7e, - 0x96, 0x07, 0xbc, 0xc5, 0x01, 0xbf, 0x9a, 0xb0, 0xe0, 0xb6, 0x17, 0x99, 0xd1, 0x24, 0x2c, 0xe2, - 0xfd, 0x2d, 0x90, 0x53, 0x9f, 0x71, 0x4f, 0x52, 0xcd, 0xa0, 0x43, 0xd5, 0xf3, 0x51, 0xdc, 0xda, - 0x21, 0x70, 0x1e, 0x7c, 0xf2, 0xd4, 0x37, 0xaa, 0x9e, 0xcf, 0x4d, 0xe4, 0x72, 0xc7, 0x12, 0x72, - 0xf1, 0x5b, 0x35, 0x51, 0x2d, 0x63, 0x22, 0xfa, 0x67, 0x0d, 0x1e, 0x64, 0x04, 0xc8, 0x43, 0xcd, - 0x92, 0x90, 0x1e, 0xb8, 0x5a, 0x76, 0xe0, 0x5a, 0xa9, 0x85, 0x16, 0xde, 0xd5, 0x42, 0x9f, 0xc3, - 0xba, 0x50, 0xf6, 0x7b, 0xfb, 0x1d, 0x0d, 0x80, 0xa8, 0x2c, 0xee, 0xc4, 0xb1, 0xbe, 0x80, 0xa6, - 0x62, 0xc8, 0xae, 0x1d, 0x46, 0x0a, 0x76, 0x37, 0xbd, 0xe9, 0x05, 0x83, 0xe4, 0xb0, 0x5f, 0xc3, - 0x56, 0x81, 0xcf, 0x5d, 0x38, 0xda, 0x31, 0x6c, 0xe2, 0xfc, 0x71, 0x10, 0x78, 0xc1, 0xfb, 0xc3, - 0x8f, 0xa4, 0x1a, 0x14, 0x36, 0x73, 0xa3, 0xff, 0x59, 0x1e, 0x7d, 0x33, 0x41, 0x8f, 0x6c, 0x8b, - 0xe0, 0x8f, 0xe0, 0x41, 0xef, 0xc2, 0xbb, 0xe9, 0x74, 0xba, 0x5d, 0xcf, 0xba, 0x0c, 0xdf, 0xcf, - 0x6b, 0xfe, 0xa6, 0xc1, 0x92, 0xe4, 0x40, 0xd6, 0xa0, 0x7a, 0xd2, 0x91, 0xfb, 0xaa, 0x27, 0x9d, - 0x84, 0x53, 0x55, 0xe1, 0x44, 0x60, 0x61, 0xec, 0x0d, 0xe2, 0x38, 0x8c, 0xdf, 0x64, 0x03, 0xee, - 0x79, 0x37, 0x2e, 0x0b, 0x5a, 0x0b, 0x38, 0x28, 0x08, 0xbe, 0xb2, 0xd3, 0xe9, 0x86, 0xad, 0x7b, - 0x28, 0x10, 0xbf, 0x31, 0xea, 0xde, 0xba, 0x16, 0x1b, 0xb4, 0x16, 0x71, 0x54, 0x52, 0x44, 0x87, - 0xfa, 0xc4, 0x95, 0x33, 0x4b, 0x38, 0x93, 0xd0, 0xd4, 0x82, 0x8d, 0xec, 0x31, 0xe7, 0x56, 0xed, - 0x87, 0x70, 0xcf, 0xe1, 0x5b, 0xa5, 0x62, 0x57, 0xb8, 0x62, 0x25, 0x3b, 0x43, 0xcc, 0x50, 0x07, - 0x36, 0xce, 0x5d, 0xfe, 0x19, 0x8f, 0x4b, 0x65, 0xe6, 0x55, 0x42, 0xe1, 0x7e, 0xc0, 0x7c, 0xc7, - 0xb4, 0xd8, 0x29, 0x9e, 0x58, 0x48, 0xc9, 0x8c, 0x91, 0x5d, 0x58, 0x19, 0x7a, 0x81, 0xc5, 0x0c, - 0x8c, 0xf9, 0x32, 0x03, 0xa8, 0x43, 0xf4, 0x73, 0xd8, 0xcc, 0x49, 0x9b, 0xf7, 0x4c, 0xf4, 0x39, - 0xb4, 0x7b, 0x37, 0x76, 0x64, 0x5d, 0x28, 0x99, 0x4f, 0x04, 0xea, 0x18, 0xba, 0x62, 0x73, 0x2d, - 0x6b, 0xf3, 0xdf, 0xc3, 0xe3, 0xa9, 0x7b, 0xef, 0x24, 0x6c, 0x18, 0xb0, 0x2d, 0xc3, 0x6f, 0x49, - 0xa5, 0xf0, 0x50, 0x09, 0xc2, 0x68, 0x2a, 0x9c, 0x95, 0x51, 0x78, 0xba, 0x23, 0x7f, 0xaf, 0x81, - 0x5e, 0xc6, 0x54, 0x1e, 0x68, 0x26, 0xd7, 0xff, 0x6f, 0x6c, 0xff, 0x41, 0x83, 0xf5, 0xaf, 0x4c, - 0x77, 0xe0, 0xb0, 0xde, 0xab, 0x6e, 0x38, 0x2b, 0xc2, 0x6c, 0x23, 0xc8, 0x2a, 0x82, 0x5c, 0xe6, - 0x8c, 0x7b, 0xaf, 0xba, 0x69, 0x82, 0x33, 0x83, 0x51, 0x9c, 0xc9, 0xf0, 0x9b, 0xd7, 0x4c, 0xfd, - 0xa4, 0x66, 0x12, 0x77, 0x2f, 0x1d, 0x50, 0x2a, 0x9c, 0x7b, 0x6a, 0x85, 0xc3, 0xeb, 0x93, 0xf0, - 0xca, 0x39, 0x33, 0xa3, 0x88, 0x05, 0x6e, 0x6b, 0x51, 0x54, 0x62, 0xe9, 0x08, 0xbf, 0x8b, 0xe1, - 0x85, 0x19, 0x0c, 0x6c, 0x77, 0xd4, 0x5a, 0x42, 0x75, 0x24, 0x34, 0xcf, 0x31, 0xea, 0x49, 0xee, - 0xc4, 0x59, 0xbe, 0xd7, 0x60, 0xeb, 0x6c, 0x12, 0x8c, 0xca, 0x7c, 0x65, 0xaa, 0x8f, 0xf3, 0x53, - 0xd8, 0xae, 0x69, 0x45, 0xf6, 0x35, 0x93, 0x46, 0x4d, 0x68, 0x8c, 0x6b, 0xf6, 0x58, 0xdc, 0xcc, - 0x9a, 0x81, 0xdf, 0x7c, 0xfd, 0xd0, 0x76, 0x18, 0x9a, 0x44, 0xa8, 0x32, 0xa1, 0x51, 0x93, 0x93, - 0x7e, 0xc7, 0x0e, 0x12, 0x4d, 0x22, 0x45, 0x7f, 0x07, 0xad, 0x22, 0xb0, 0x3b, 0xd1, 0xc9, 0x13, - 0x68, 0x1c, 0x5d, 0x30, 0xeb, 0xf2, 0x7f, 0x54, 0x0b, 0xf4, 0x33, 0x58, 0x57, 0xd6, 0xcd, 0x1d, - 0x64, 0xba, 0xb0, 0x21, 0xaf, 0x54, 0x0f, 0x05, 0xc7, 0xa2, 0x76, 0x94, 0xcb, 0x74, 0x1f, 0xfd, - 0x14, 0xa7, 0xd3, 0xdb, 0x24, 0x0b, 0x4a, 0x71, 0x45, 0xe3, 0x82, 0x72, 0x02, 0x9b, 0x39, 0x6e, - 0x77, 0xa2, 0xab, 0x63, 0xd8, 0x34, 0xd8, 0xc8, 0xe6, 0xe1, 0x2d, 0x5e, 0x32, 0x33, 0xc7, 0x9b, - 0x83, 0x41, 0xc0, 0xc2, 0x50, 0x8a, 0x8d, 0x49, 0xfa, 0x02, 0x9a, 0x79, 0x36, 0x73, 0xeb, 0xf3, - 0x17, 0xb0, 0x71, 0x3a, 0x1c, 0x3a, 0xb6, 0xcb, 0x5e, 0xb2, 0x71, 0x3f, 0x83, 0x24, 0xba, 0xf5, - 0x13, 0x24, 0xfc, 0xbb, 0xac, 0xa2, 0xe5, 0x79, 0x23, 0xb7, 0x7f, 0x6e, 0x08, 0x3f, 0x4f, 0x4c, - 0xda, 0x65, 0xe6, 0x20, 0x85, 0x50, 0x30, 0xa9, 0x98, 0x16, 0x26, 0x45, 0xc1, 0xd9, 0x5d, 0x73, - 0x0b, 0xfe, 0x93, 0x06, 0x20, 0x92, 0xcc, 0x89, 0x3b, 0xf4, 0x4a, 0x95, 0xaf, 0x43, 0x7d, 0x8c, - 0xe7, 0x3a, 0xe9, 0xe0, 0xce, 0x05, 0x23, 0xa1, 0x79, 0x8d, 0x61, 0x3a, 0x76, 0x92, 0x4e, 0x05, - 0xc1, 0x77, 0xf8, 0x8c, 0x05, 0xe7, 0x46, 0x57, 0xc4, 0xe3, 0x65, 0x23, 0xa1, 0x79, 0x9c, 0xb3, - 0x1c, 0x9b, 0xb9, 0x11, 0xce, 0x8a, 0x2a, 0x44, 0x19, 0xa1, 0x7d, 0x00, 0x61, 0xc8, 0xa9, 0x78, - 0x78, 0xcc, 0x1d, 0x0c, 0xe2, 0x24, 0x8f, 0xdf, 0x1c, 0x47, 0x18, 0x99, 0xa3, 0xb8, 0x00, 0x12, - 0x84, 0x12, 0x6b, 0x17, 0x32, 0xdd, 0x64, 0x17, 0x1a, 0xbc, 0x1c, 0x14, 0x4a, 0x13, 0x36, 0x8b, - 0x55, 0xa3, 0xa5, 0x5e, 0x5d, 0xd6, 0xbc, 0xc4, 0xb2, 0x6b, 0xa9, 0x6c, 0xfa, 0xb5, 0xe0, 0x26, - 0xb4, 0x38, 0x95, 0xdb, 0x1e, 0x2c, 0x89, 0x46, 0x5e, 0xa4, 0xc8, 0x95, 0xc3, 0x35, 0x6e, 0xce, - 0x54, 0xf5, 0x46, 0x3c, 0x1d, 0xf3, 0x13, 0x5a, 0x98, 0xc5, 0x4f, 0x3c, 0x02, 0x64, 0xf8, 0xa5, - 0xaa, 0x33, 0xe2, 0x69, 0xfa, 0x77, 0x0d, 0x96, 0x04, 0x9b, 0x90, 0x3c, 0x85, 0x45, 0x07, 0x4f, - 0x8d, 0xac, 0x56, 0x0e, 0x37, 0xd0, 0xa7, 0x72, 0xba, 0xf8, 0xaa, 0x62, 0xc8, 0x55, 0x7c, 0xbd, - 0x80, 0x85, 0x5a, 0x50, 0xd6, 0xab, 0xa7, 0xe5, 0xeb, 0xc5, 0x2a, 0xbe, 0x5e, 0x88, 0x45, 0x0d, - 0x29, 0xeb, 0xd5, 0xd3, 0xf0, 0xf5, 0x62, 0xd5, 0x8b, 0x3a, 0x2c, 0x0a, 0x5f, 0xe2, 0xfd, 0x37, - 0xf2, 0xcd, 0xdc, 0xc0, 0x66, 0x06, 0x6e, 0x3d, 0x81, 0xd5, 0xcc, 0xc0, 0xaa, 0x27, 0xe2, 0x9b, - 0x19, 0xf1, 0xf5, 0x58, 0x0c, 0x77, 0x0f, 0x6e, 0xbe, 0xd8, 0x1b, 0x05, 0x41, 0x19, 0x10, 0x55, - 0xe4, 0xdc, 0x61, 0xef, 0x13, 0x58, 0x12, 0xe0, 0x33, 0x25, 0xac, 0x54, 0xb5, 0x11, 0xcf, 0xd1, - 0x7f, 0x68, 0x69, 0xbc, 0xb6, 0x2e, 0xd8, 0xd8, 0x9c, 0x1e, 0xaf, 0x71, 0x3a, 0x2d, 0x2d, 0x0a, - 0x65, 0xfe, 0xd4, 0xde, 0x99, 0x5f, 0xb9, 0x81, 0x19, 0x99, 0x7d, 0x33, 0x4c, 0x12, 0x65, 0x4c, - 0xf3, 0xd3, 0x47, 0x66, 0xdf, 0x89, 0x2b, 0x0e, 0x41, 0xe0, 0xe5, 0x40, 0x79, 0xb2, 0xd8, 0x90, - 0x94, 0x9a, 0x0f, 0x24, 0xda, 0x3b, 0xc9, 0x07, 0xfb, 0xb0, 0xf1, 0x25, 0x8b, 0x7a, 0x93, 0x3e, - 0x4f, 0x8a, 0x47, 0xc3, 0xd1, 0x8c, 0x74, 0x40, 0xcf, 0x61, 0x33, 0xb7, 0x76, 0x6e, 0x88, 0x04, - 0x16, 0xac, 0x61, 0x5a, 0xb8, 0xf1, 0x6f, 0xfa, 0x13, 0x58, 0xff, 0x92, 0x45, 0xef, 0x20, 0xff, - 0x0c, 0x88, 0xba, 0x70, 0x6e, 0xe1, 0x0d, 0xa8, 0x59, 0xc3, 0xa4, 0x80, 0xb5, 0x86, 0xa3, 0xfd, - 0x3e, 0xd4, 0xe3, 0x64, 0x4d, 0x1e, 0xc0, 0x07, 0x27, 0xee, 0xb5, 0xe9, 0xd8, 0x83, 0x78, 0xa8, - 0x51, 0x21, 0x1f, 0xc0, 0x0a, 0x3e, 0x4f, 0x89, 0xa1, 0x86, 0x46, 0x1a, 0x70, 0x5f, 0xbc, 0x2b, - 0xc8, 0x91, 0x2a, 0x59, 0x03, 0xe8, 0x45, 0x9e, 0x2f, 0xe9, 0x1a, 0xd2, 0x17, 0xde, 0x8d, 0xa4, - 0x17, 0xf6, 0x7f, 0x09, 0xf5, 0x38, 0x7b, 0x28, 0x32, 0xe2, 0xa1, 0x46, 0x85, 0xac, 0xc3, 0xea, - 0xf1, 0xb5, 0x6d, 0x45, 0xc9, 0x90, 0x46, 0xb6, 0xe0, 0xc1, 0x91, 0xe9, 0x5a, 0xcc, 0xc9, 0x4e, - 0x54, 0x0f, 0xff, 0xd8, 0x80, 0x45, 0x11, 0x03, 0xc8, 0x37, 0xb0, 0x9c, 0xbc, 0x9c, 0x11, 0xbc, - 0xf0, 0xf9, 0xa7, 0x3a, 0x7d, 0x33, 0x37, 0x2a, 0x34, 0x46, 0x1f, 0xff, 0xe1, 0x9f, 0xff, 0xf9, - 0x6b, 0x75, 0x9b, 0x6e, 0x1c, 0x98, 0xbe, 0x1d, 0x1e, 0x5c, 0x3f, 0x33, 0x1d, 0xff, 0xc2, 0x7c, - 0x76, 0xc0, 0x7d, 0x3d, 0x7c, 0xae, 0xed, 0x93, 0x21, 0xac, 0x28, 0x0f, 0x42, 0x04, 0x7b, 0xef, - 0xe2, 0x13, 0x94, 0xbe, 0x55, 0x18, 0x97, 0x02, 0x9e, 0xa0, 0x80, 0x5d, 0xfd, 0x61, 0x99, 0x80, - 0x83, 0xd7, 0xdc, 0x98, 0xdf, 0x71, 0x39, 0x9f, 0x01, 0xa4, 0x8f, 0x34, 0x04, 0xd1, 0x16, 0xde, - 0x7d, 0xf4, 0x66, 0x7e, 0x58, 0x0a, 0xa9, 0x10, 0x07, 0x56, 0x94, 0xf7, 0x0c, 0xa2, 0xe7, 0x1e, - 0x38, 0x94, 0x17, 0x0c, 0xfd, 0x61, 0xe9, 0x9c, 0xe4, 0xf4, 0x31, 0xc2, 0x6d, 0x93, 0x9d, 0x1c, - 0xdc, 0x10, 0x97, 0x4a, 0xbc, 0xe4, 0x18, 0x20, 0x7d, 0x7f, 0x20, 0xdb, 0xd9, 0xf7, 0x08, 0x55, - 0x96, 0x5e, 0x36, 0x95, 0x80, 0x3e, 0x82, 0xfb, 0x6a, 0x03, 0x4f, 0x50, 0x89, 0x25, 0x2f, 0x17, - 0x7a, 0xab, 0x38, 0x91, 0x30, 0xf9, 0x02, 0x56, 0x33, 0x2d, 0x33, 0xc1, 0xc5, 0x65, 0x3d, 0xbb, - 0xbe, 0x5d, 0x32, 0x93, 0xf0, 0x39, 0x8f, 0x5f, 0xc9, 0xd4, 0x57, 0x4d, 0xf2, 0x28, 0xd5, 0x78, - 0xc9, 0x13, 0xa9, 0xde, 0x9e, 0x36, 0x9d, 0xb0, 0xfd, 0x06, 0xb6, 0xa6, 0x3c, 0x44, 0x13, 0x9a, - 0x6e, 0x9e, 0xf6, 0x4a, 0xad, 0x4f, 0x0d, 0x5b, 0xb4, 0xc2, 0x5d, 0x26, 0xed, 0xb9, 0x84, 0xcb, - 0x14, 0xba, 0x49, 0xe1, 0x32, 0xc5, 0xd6, 0x8c, 0x56, 0xc8, 0x00, 0xb6, 0xa6, 0x34, 0xfb, 0x02, - 0xd9, 0xec, 0x57, 0x04, 0xfd, 0xa3, 0x99, 0x6b, 0x94, 0xf3, 0x37, 0x8b, 0xcd, 0x37, 0xfa, 0xf8, - 0x23, 0xe5, 0xca, 0x14, 0x3b, 0x38, 0xa1, 0xda, 0xe9, 0x7d, 0x3b, 0xad, 0x90, 0x53, 0x68, 0xe4, - 0xbb, 0x2c, 0x82, 0xce, 0x3d, 0xa5, 0x29, 0xd4, 0x77, 0xca, 0x27, 0x15, 0x86, 0xa4, 0xf8, 0x9f, - 0x42, 0xe0, 0x9c, 0xfa, 0xff, 0x62, 0xa6, 0x85, 0x9e, 0xc3, 0x72, 0xd2, 0x65, 0x89, 0xb8, 0x94, - 0x6f, 0xce, 0x44, 0x5c, 0x2a, 0xb4, 0x62, 0xb4, 0x42, 0x46, 0xb0, 0x9a, 0x69, 0x8a, 0x84, 0x5f, - 0x97, 0x75, 0x5d, 0xc2, 0xaf, 0x4b, 0x3b, 0x28, 0xfa, 0x21, 0xde, 0xe7, 0x87, 0x7a, 0x33, 0x7f, - 0x9f, 0x45, 0xce, 0xe3, 0x91, 0xe7, 0x04, 0xd6, 0xb2, 0xfd, 0x8b, 0xb8, 0xd0, 0xa5, 0xad, 0x91, - 0xb8, 0xd0, 0xe5, 0xed, 0x0e, 0xad, 0x90, 0x00, 0x56, 0x33, 0x6d, 0x88, 0xc4, 0x5c, 0xd2, 0xd9, - 0x48, 0xcc, 0x65, 0x3d, 0x0b, 0xfd, 0x29, 0x62, 0x7e, 0xb2, 0xff, 0x71, 0x0e, 0xb3, 0xac, 0x66, - 0x0e, 0x5e, 0xf3, 0x36, 0xe8, 0xbb, 0x38, 0x16, 0x5d, 0x26, 0x7a, 0x12, 0xb9, 0x21, 0xa3, 0xa7, - 0x4c, 0x2b, 0x93, 0xd1, 0x53, 0xb6, 0x5d, 0xa1, 0x9f, 0xa0, 0xcc, 0xc7, 0xba, 0x9e, 0x93, 0x29, - 0xaa, 0xbd, 0x83, 0xd7, 0x9e, 0x8f, 0x51, 0xfa, 0x37, 0x00, 0x69, 0xbd, 0x26, 0xae, 0x5c, 0xa1, - 0x64, 0x14, 0x57, 0xae, 0x58, 0xd6, 0xd1, 0x36, 0xca, 0x68, 0x91, 0x66, 0xf9, 0xb9, 0xc8, 0x30, - 0xb5, 0x38, 0x96, 0x3d, 0x59, 0x8b, 0xab, 0x75, 0x5b, 0xd6, 0xe2, 0x99, 0x1a, 0x89, 0xee, 0xa2, - 0x14, 0x5d, 0xdf, 0xcc, 0x5b, 0x1c, 0x97, 0xf1, 0x43, 0x38, 0xb0, 0x9a, 0xa9, 0x5d, 0x84, 0x9c, - 0xb2, 0xd2, 0x47, 0xc8, 0x29, 0x2d, 0x74, 0xe2, 0xc4, 0x46, 0xda, 0x79, 0x39, 0x93, 0xbe, 0x9a, - 0xdb, 0xc8, 0xb7, 0x00, 0x69, 0xa5, 0x22, 0x54, 0x56, 0x28, 0x71, 0x84, 0xca, 0x8a, 0x05, 0x0d, - 0xfd, 0x08, 0x85, 0x3c, 0x22, 0xb3, 0xb2, 0xe7, 0x8b, 0xd6, 0x0f, 0x6f, 0xda, 0xda, 0x8f, 0x6f, - 0xda, 0xda, 0xbf, 0xdf, 0xb4, 0xb5, 0xbf, 0xbc, 0x6d, 0x57, 0x7e, 0x7c, 0xdb, 0xae, 0xfc, 0xeb, - 0x6d, 0xbb, 0xd2, 0x5f, 0xc4, 0xff, 0x8f, 0x9f, 0xfe, 0x37, 0x00, 0x00, 0xff, 0xff, 0xfc, 0xbb, - 0xa5, 0xe9, 0xc3, 0x1c, 0x00, 0x00, + // 1921 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xbc, 0x59, 0x5f, 0x6f, 0xdb, 0xc8, + 0x11, 0x17, 0x25, 0xc5, 0x96, 0xc7, 0xb1, 0x23, 0x6f, 0x6c, 0x59, 0x66, 0x1c, 0xc5, 0xb7, 0x77, + 0x97, 0x1a, 0x46, 0x11, 0x35, 0xbe, 0x3e, 0x05, 0xb8, 0x02, 0x17, 0xcb, 0x77, 0x35, 0xaa, 0x9c, + 0x53, 0xba, 0x46, 0x71, 0x28, 0x50, 0x1c, 0x45, 0xad, 0x64, 0xc2, 0x14, 0xc9, 0x90, 0x94, 0x5d, + 0x23, 0x38, 0x14, 0xe8, 0x07, 0x68, 0x0b, 0xf4, 0xe1, 0x1e, 0xfb, 0xd0, 0xd7, 0x7e, 0x8f, 0xf6, + 0x31, 0x40, 0x5f, 0xfa, 0x58, 0x24, 0xfd, 0x20, 0xc5, 0xce, 0x2e, 0xc9, 0xe5, 0x1f, 0xa9, 0x51, + 0x80, 0xf3, 0x1b, 0x67, 0x67, 0x77, 0xe6, 0xb7, 0x33, 0xb3, 0xb3, 0x33, 0x4b, 0x58, 0x1f, 0x4e, + 0x26, 0x66, 0x18, 0xb1, 0xe0, 0x89, 0x1f, 0x78, 0x91, 0x47, 0xaa, 0xfe, 0x40, 0x5f, 0x1f, 0x4e, + 0xae, 0xbd, 0xe0, 0x32, 0x1e, 0xd3, 0x77, 0xc7, 0x9e, 0x37, 0x76, 0x58, 0xd7, 0xf4, 0xed, 0xae, + 0xe9, 0xba, 0x5e, 0x64, 0x46, 0xb6, 0xe7, 0x86, 0x82, 0x4b, 0x5f, 0xc1, 0xce, 0x0b, 0x7b, 0x1c, + 0x98, 0x11, 0xfb, 0x35, 0x2e, 0x32, 0x98, 0x63, 0xde, 0x18, 0xec, 0xd5, 0x94, 0x85, 0x11, 0xe9, + 0x00, 0x3c, 0xb7, 0x5d, 0xc7, 0x1b, 0x7f, 0x6d, 0x4e, 0x58, 0x5b, 0xdb, 0xd3, 0xf6, 0x57, 0x0c, + 0x65, 0x84, 0xec, 0xc2, 0x8a, 0xa0, 0x5e, 0x7a, 0x61, 0xbb, 0xba, 0xa7, 0xed, 0xaf, 0x19, 0xe9, + 0x00, 0x69, 0xc1, 0x52, 0xe8, 0x4d, 0x03, 0x8b, 0xb5, 0x6b, 0xb8, 0x52, 0x52, 0xf4, 0x25, 0x74, + 0xce, 0xfd, 0x61, 0x56, 0xe3, 0x91, 0xe7, 0x8e, 0xec, 0x71, 0xac, 0xb7, 0x05, 0x4b, 0x16, 0x0e, + 0x48, 0x9d, 0x92, 0x52, 0x24, 0x56, 0x33, 0x12, 0xbf, 0x85, 0xe6, 0x59, 0x64, 0x06, 0xd1, 0xaf, + 0xcc, 0xf0, 0x32, 0x96, 0x41, 0xa0, 0x1e, 0x99, 0xe1, 0xa5, 0x94, 0x80, 0xdf, 0xa4, 0x0d, 0xcb, + 0x62, 0x05, 0x47, 0x5b, 0xdb, 0x5f, 0x31, 0x62, 0x92, 0xef, 0x34, 0x60, 0x13, 0xef, 0x8a, 0xbd, + 0x60, 0x91, 0x89, 0x78, 0x1b, 0x86, 0x32, 0x42, 0x5f, 0xc1, 0x86, 0xa2, 0x21, 0xf4, 0x3d, 0x37, + 0x64, 0x1c, 0x4e, 0xc0, 0xc2, 0xa9, 0x13, 0xa1, 0x92, 0x86, 0x21, 0x29, 0xd2, 0x84, 0xda, 0x24, + 0x1c, 0x4b, 0x8c, 0xfc, 0x93, 0x1c, 0xa6, 0x8a, 0x6b, 0x7b, 0xb5, 0xfd, 0xd5, 0xc3, 0xf6, 0x13, + 0x7f, 0xf0, 0xe4, 0xc8, 0x9b, 0x4c, 0x3c, 0x37, 0xb6, 0x82, 0x10, 0x9a, 0x40, 0xa2, 0x9f, 0xc1, + 0x8e, 0x30, 0xd3, 0x0b, 0xf4, 0xf0, 0x7b, 0x59, 0x88, 0xde, 0x80, 0x5e, 0xb6, 0x68, 0x61, 0xc0, + 0x4f, 0xf3, 0x80, 0xb7, 0x39, 0xe0, 0x5f, 0x4e, 0x59, 0x70, 0x73, 0x16, 0x99, 0xd1, 0x34, 0x2c, + 0xe2, 0xfd, 0x2d, 0x90, 0x53, 0x9f, 0xf1, 0x48, 0x52, 0xdd, 0xa0, 0x43, 0xd5, 0xf3, 0x51, 0xdd, + 0xfa, 0x21, 0x70, 0x19, 0x9c, 0x79, 0xea, 0x1b, 0x55, 0xcf, 0xe7, 0x2e, 0x72, 0x79, 0x60, 0x09, + 0xbd, 0xf8, 0xad, 0xba, 0xa8, 0x96, 0x71, 0x11, 0xfd, 0x93, 0x06, 0xf7, 0x33, 0x0a, 0xe4, 0xa6, + 0xe6, 0x69, 0x48, 0x37, 0x5c, 0x2d, 0xdb, 0x70, 0xad, 0xd4, 0x43, 0xf5, 0xf7, 0xf5, 0xd0, 0x17, + 0xb0, 0x21, 0x8c, 0xfd, 0xc1, 0x71, 0x47, 0x03, 0x20, 0xaa, 0x88, 0x5b, 0x09, 0xac, 0x2f, 0xa1, + 0xa5, 0x38, 0xb2, 0x6f, 0x87, 0x91, 0x82, 0xdd, 0x4d, 0x4f, 0x7a, 0xc1, 0x21, 0x39, 0xec, 0x57, + 0xb0, 0x5d, 0x90, 0x73, 0x1b, 0x81, 0x76, 0x0c, 0x5b, 0xc8, 0x3f, 0x0e, 0x02, 0x2f, 0xf8, 0x70, + 0xf8, 0x91, 0x34, 0x83, 0x22, 0x66, 0x61, 0xf4, 0x3f, 0xc9, 0xa3, 0x6f, 0x25, 0xe8, 0x51, 0x6c, + 0x11, 0xfc, 0x11, 0xdc, 0x3f, 0xbb, 0xf0, 0xae, 0x7b, 0xbd, 0x7e, 0xdf, 0xb3, 0x2e, 0xc3, 0x0f, + 0x8b, 0x9a, 0xbf, 0x6a, 0xb0, 0x2c, 0x25, 0x90, 0x75, 0xa8, 0x9e, 0xf4, 0xe4, 0xba, 0xea, 0x49, + 0x2f, 0x91, 0x54, 0x55, 0x24, 0x11, 0xa8, 0x4f, 0xbc, 0x61, 0x9c, 0x87, 0xf1, 0x9b, 0x6c, 0xc2, + 0x1d, 0xef, 0xda, 0x65, 0x41, 0xbb, 0x8e, 0x83, 0x82, 0xe0, 0x33, 0x7b, 0xbd, 0x7e, 0xd8, 0xbe, + 0x83, 0x0a, 0xf1, 0x1b, 0xb3, 0xee, 0x8d, 0x6b, 0xb1, 0x61, 0x7b, 0x09, 0x47, 0x25, 0x45, 0x74, + 0x68, 0x4c, 0x5d, 0xc9, 0x59, 0x46, 0x4e, 0x42, 0x53, 0x0b, 0x36, 0xb3, 0xdb, 0x5c, 0xd8, 0xb4, + 0x1f, 0xc1, 0x1d, 0x87, 0x2f, 0x95, 0x86, 0x5d, 0xe5, 0x86, 0x95, 0xe2, 0x0c, 0xc1, 0xa1, 0x0e, + 0x6c, 0x9e, 0xbb, 0xfc, 0x33, 0x1e, 0x97, 0xc6, 0xcc, 0x9b, 0x84, 0xc2, 0xdd, 0x80, 0xf9, 0x8e, + 0x69, 0xb1, 0x53, 0xdc, 0xb1, 0xd0, 0x92, 0x19, 0x23, 0x7b, 0xb0, 0x3a, 0xf2, 0x02, 0x8b, 0x19, + 0x98, 0xf3, 0xe5, 0x0d, 0xa0, 0x0e, 0xd1, 0x2f, 0x60, 0x2b, 0xa7, 0x6d, 0xd1, 0x3d, 0xd1, 0x67, + 0xd0, 0x39, 0xbb, 0xb6, 0x23, 0xeb, 0x42, 0xb9, 0xf9, 0x44, 0xa2, 0x8e, 0xa1, 0x2b, 0x3e, 0xd7, + 0xb2, 0x3e, 0xff, 0x3d, 0x3c, 0x9a, 0xb9, 0xf6, 0x56, 0xd2, 0x86, 0x01, 0x3b, 0x32, 0xfd, 0x96, + 0x54, 0x0a, 0x0f, 0x94, 0x24, 0x8c, 0xae, 0x42, 0xae, 0xcc, 0xc2, 0xb3, 0x03, 0xf9, 0x7b, 0x0d, + 0xf4, 0x32, 0xa1, 0x72, 0x43, 0x73, 0xa5, 0xfe, 0xb0, 0xb9, 0xfd, 0x7b, 0x0d, 0xb6, 0x5f, 0x4e, + 0x83, 0x71, 0xd9, 0x66, 0x67, 0x3a, 0x89, 0x1f, 0x09, 0xdb, 0x35, 0xad, 0xc8, 0xbe, 0x62, 0x12, + 0x55, 0x42, 0xe3, 0xc1, 0xb4, 0x27, 0x22, 0xb4, 0x6a, 0x06, 0x7e, 0xf3, 0xf9, 0x23, 0xdb, 0x61, + 0x98, 0xb5, 0xc4, 0x39, 0x4c, 0x68, 0x3c, 0x76, 0xd3, 0x41, 0xcf, 0x0e, 0xda, 0x77, 0x64, 0xb1, + 0x83, 0x14, 0xfd, 0x1d, 0xb4, 0x8b, 0xc0, 0x6e, 0x25, 0x02, 0x1e, 0x43, 0xf3, 0xe8, 0x82, 0x59, + 0x97, 0xff, 0xe7, 0xba, 0xa3, 0x9f, 0xc3, 0x86, 0x32, 0x6f, 0xe1, 0x53, 0xd2, 0x87, 0x4d, 0x19, + 0x13, 0x67, 0xa8, 0x38, 0x56, 0xb5, 0xab, 0x44, 0xc3, 0x5d, 0x8e, 0x56, 0xb0, 0xd3, 0x70, 0x90, + 0x15, 0x91, 0x88, 0xb1, 0xb8, 0x22, 0x9a, 0xc2, 0x56, 0x4e, 0xda, 0xad, 0xd8, 0xea, 0x18, 0xb6, + 0x0c, 0x36, 0xb6, 0xf9, 0xf9, 0x8c, 0xa7, 0xcc, 0xbd, 0xa4, 0xcc, 0xe1, 0x30, 0x60, 0x61, 0x28, + 0xd5, 0xc6, 0x24, 0x7d, 0x0e, 0xad, 0xbc, 0x98, 0x85, 0xed, 0xf9, 0x33, 0xd8, 0x3c, 0x1d, 0x8d, + 0x1c, 0xdb, 0x65, 0x2f, 0xd8, 0x64, 0x90, 0x41, 0x12, 0xdd, 0xf8, 0x09, 0x12, 0xfe, 0x5d, 0x56, + 0x92, 0xf1, 0xc4, 0x97, 0x5b, 0xbf, 0x30, 0x84, 0x9f, 0x26, 0x2e, 0xed, 0x33, 0x73, 0x98, 0x42, + 0x28, 0xb8, 0x54, 0xb0, 0x85, 0x4b, 0x51, 0x71, 0x76, 0xd5, 0xc2, 0x8a, 0xff, 0xa8, 0x01, 0x88, + 0x2c, 0x79, 0xe2, 0x8e, 0xbc, 0x52, 0xe3, 0xeb, 0xd0, 0x98, 0xe0, 0xbe, 0x4e, 0x7a, 0xb8, 0xb2, + 0x6e, 0x24, 0x34, 0xbf, 0x24, 0x4d, 0xc7, 0x4e, 0xee, 0x03, 0x41, 0xf0, 0x15, 0x3e, 0x63, 0xc1, + 0xb9, 0xd1, 0x17, 0x09, 0x65, 0xc5, 0x48, 0x68, 0xde, 0x48, 0x58, 0x8e, 0xcd, 0xdc, 0x08, 0xb9, + 0xe2, 0x1a, 0x55, 0x46, 0xe8, 0x00, 0x40, 0x38, 0x72, 0x26, 0x1e, 0x02, 0x75, 0xee, 0xfd, 0xd8, + 0x05, 0xfc, 0x9b, 0xe3, 0x08, 0x23, 0x73, 0x1c, 0xdf, 0xe0, 0x82, 0x50, 0xda, 0xa1, 0x7a, 0xa6, + 0x1d, 0xea, 0x43, 0x93, 0xd7, 0x33, 0xc2, 0x68, 0xc2, 0x67, 0xb1, 0x69, 0xb4, 0x34, 0xaa, 0xcb, + 0xaa, 0xef, 0x58, 0x77, 0x2d, 0xd5, 0x4d, 0xbf, 0x16, 0xd2, 0x84, 0x15, 0x67, 0x4a, 0xdb, 0x87, + 0x65, 0xd1, 0x89, 0x8a, 0x1c, 0xbf, 0x7a, 0xb8, 0xce, 0xdd, 0x99, 0x9a, 0xde, 0x88, 0xd9, 0xb1, + 0x3c, 0x61, 0x85, 0x79, 0xf2, 0x44, 0x17, 0x9b, 0x91, 0x97, 0x9a, 0xce, 0x88, 0xd9, 0xf4, 0x6f, + 0x1a, 0x2c, 0x0b, 0x31, 0x21, 0x79, 0x02, 0x4b, 0x0e, 0xee, 0x1a, 0x45, 0xad, 0x1e, 0x6e, 0x62, + 0x4c, 0xe5, 0x6c, 0xf1, 0xf3, 0x8a, 0x21, 0x67, 0xf1, 0xf9, 0x02, 0x16, 0x5a, 0x41, 0x99, 0xaf, + 0xee, 0x96, 0xcf, 0x17, 0xb3, 0xf8, 0x7c, 0xa1, 0x16, 0x2d, 0xa4, 0xcc, 0x57, 0x77, 0xc3, 0xe7, + 0x8b, 0x59, 0xcf, 0x1b, 0xb0, 0x24, 0x62, 0x89, 0x37, 0x90, 0x28, 0x37, 0x73, 0x02, 0x5b, 0x19, + 0xb8, 0x8d, 0x04, 0x56, 0x2b, 0x03, 0xab, 0x91, 0xa8, 0x6f, 0x65, 0xd4, 0x37, 0x62, 0x35, 0x3c, + 0x3c, 0xb8, 0xfb, 0xe2, 0x68, 0x14, 0x04, 0x65, 0x40, 0x54, 0x95, 0x0b, 0xa7, 0xbd, 0x4f, 0x61, + 0x59, 0x80, 0xcf, 0xd4, 0x60, 0xd2, 0xd4, 0x46, 0xcc, 0xa3, 0x7f, 0xd7, 0xd2, 0x7c, 0x6d, 0x5d, + 0xb0, 0x89, 0x39, 0x3b, 0x5f, 0x23, 0x3b, 0x6d, 0xfe, 0x0a, 0x75, 0xea, 0xcc, 0xe6, 0x8f, 0x1f, + 0xb9, 0xa1, 0x19, 0x99, 0x03, 0x33, 0x4c, 0x2e, 0xca, 0x98, 0xe6, 0xbb, 0x8f, 0xcc, 0x81, 0xc3, + 0xe4, 0x3d, 0x29, 0x08, 0x3c, 0x1c, 0xa8, 0xaf, 0xbd, 0x24, 0x0f, 0x07, 0x52, 0xea, 0x7d, 0x20, + 0xd1, 0xde, 0xca, 0x7d, 0x70, 0x00, 0x9b, 0x5f, 0xb1, 0xe8, 0x6c, 0x3a, 0xe0, 0x97, 0xe2, 0xd1, + 0x68, 0x3c, 0xe7, 0x3a, 0xa0, 0xe7, 0xb0, 0x95, 0x9b, 0xbb, 0x30, 0x44, 0x02, 0x75, 0x6b, 0x34, + 0x8e, 0xcd, 0x88, 0xdf, 0xf4, 0x47, 0xb0, 0xf1, 0x15, 0x8b, 0xde, 0x43, 0xff, 0x4b, 0x20, 0xea, + 0xc4, 0x85, 0x95, 0x37, 0xa1, 0x66, 0x8d, 0x92, 0x0a, 0xcc, 0x1a, 0x8d, 0x0f, 0x06, 0xd0, 0x88, + 0x2f, 0x6b, 0x72, 0x1f, 0xee, 0x9d, 0xb8, 0x57, 0xa6, 0x63, 0x0f, 0xe3, 0xa1, 0x66, 0x85, 0xdc, + 0x83, 0x55, 0x7c, 0x5f, 0x11, 0x43, 0x4d, 0x8d, 0x34, 0xe1, 0xae, 0x68, 0x8c, 0xe5, 0x48, 0x95, + 0xac, 0x03, 0x9c, 0x45, 0x9e, 0x2f, 0xe9, 0x1a, 0xd2, 0x17, 0xde, 0xb5, 0xa4, 0xeb, 0x07, 0xbf, + 0x80, 0x46, 0x7c, 0x7b, 0x28, 0x3a, 0xe2, 0xa1, 0x66, 0x85, 0x6c, 0xc0, 0xda, 0xf1, 0x95, 0x6d, + 0x45, 0xc9, 0x90, 0x46, 0xb6, 0xe1, 0xfe, 0x91, 0xe9, 0x5a, 0xcc, 0xc9, 0x32, 0xaa, 0x87, 0xff, + 0xb8, 0x07, 0x4b, 0x22, 0x07, 0x90, 0x6f, 0x60, 0x25, 0x79, 0xfa, 0x21, 0x78, 0xe0, 0xf3, 0x6f, + 0x4d, 0xfa, 0x56, 0x6e, 0x54, 0x58, 0x8c, 0x3e, 0xfa, 0xc3, 0xbf, 0xfe, 0xfb, 0x97, 0xea, 0x0e, + 0xdd, 0xec, 0x9a, 0xbe, 0x1d, 0x76, 0xaf, 0x9e, 0x9a, 0x8e, 0x7f, 0x61, 0x3e, 0xed, 0xf2, 0x58, + 0x0f, 0x9f, 0x69, 0x07, 0x64, 0x04, 0xab, 0xca, 0x8b, 0x06, 0xc1, 0xe6, 0xb1, 0xf8, 0x86, 0xa2, + 0x6f, 0x17, 0xc6, 0xa5, 0x82, 0xc7, 0xa8, 0x60, 0x4f, 0x7f, 0x50, 0xa6, 0xa0, 0xfb, 0x9a, 0x3b, + 0xf3, 0x3b, 0xae, 0xe7, 0x73, 0x80, 0xf4, 0x95, 0x81, 0x20, 0xda, 0xc2, 0xc3, 0x85, 0xde, 0xca, + 0x0f, 0x4b, 0x25, 0x15, 0xe2, 0xc0, 0xaa, 0xd2, 0x90, 0x13, 0x3d, 0xd7, 0xa1, 0x2b, 0x2d, 0xb8, + 0xfe, 0xa0, 0x94, 0x27, 0x25, 0x7d, 0x82, 0x70, 0x3b, 0x64, 0x37, 0x07, 0x37, 0xc4, 0xa9, 0x12, + 0x2f, 0x39, 0x06, 0x48, 0x1b, 0x68, 0xb2, 0x93, 0x6d, 0xa8, 0x55, 0x5d, 0x7a, 0x19, 0x2b, 0x01, + 0x7d, 0x04, 0x77, 0xd5, 0x0e, 0x94, 0xa0, 0x11, 0x4b, 0x5a, 0x6f, 0xbd, 0x5d, 0x64, 0x24, 0x42, + 0xbe, 0x84, 0xb5, 0x4c, 0xcf, 0x47, 0x70, 0x72, 0x59, 0xd3, 0xa9, 0xef, 0x94, 0x70, 0x12, 0x39, + 0xe7, 0xf1, 0x33, 0x8f, 0xfa, 0x2c, 0x47, 0x1e, 0xa6, 0x16, 0x2f, 0x79, 0xe3, 0xd3, 0x3b, 0xb3, + 0xd8, 0x89, 0xd8, 0x6f, 0x60, 0x7b, 0xc6, 0x4b, 0x2a, 0xa1, 0xe9, 0xe2, 0x59, 0xcf, 0xac, 0xfa, + 0xcc, 0xb4, 0x45, 0x2b, 0x64, 0x08, 0xdb, 0x33, 0xda, 0x4d, 0x21, 0x7a, 0x7e, 0x1f, 0xab, 0x7f, + 0x3c, 0x77, 0x8e, 0xb2, 0x81, 0x56, 0xb1, 0xfd, 0xc3, 0x20, 0x7d, 0xa8, 0xc4, 0x7c, 0xb1, 0x05, + 0x13, 0xb6, 0x99, 0xdd, 0x39, 0xd2, 0x0a, 0x39, 0x85, 0x66, 0xbe, 0x4d, 0x22, 0x18, 0x9d, 0x33, + 0xba, 0x3a, 0x7d, 0xb7, 0x9c, 0xa9, 0x08, 0x24, 0xc5, 0x97, 0x72, 0x81, 0x73, 0xe6, 0x0b, 0xfa, + 0x5c, 0x13, 0x3f, 0x83, 0x95, 0xa4, 0x4d, 0x12, 0x89, 0x25, 0xdf, 0x5d, 0x89, 0xc4, 0x52, 0xe8, + 0xa5, 0x68, 0x85, 0x8c, 0x61, 0x2d, 0xd3, 0xd5, 0x88, 0xc0, 0x2c, 0x6b, 0x9b, 0x44, 0x60, 0x96, + 0xb6, 0x40, 0xf4, 0x23, 0x3c, 0x90, 0x0f, 0xf4, 0x56, 0xfe, 0x40, 0x8a, 0x4b, 0x8b, 0xa7, 0x8e, + 0x13, 0x58, 0xcf, 0x36, 0x20, 0xe2, 0x44, 0x96, 0xf6, 0x36, 0xe2, 0x44, 0x96, 0xf7, 0x2b, 0xb4, + 0x42, 0x02, 0x58, 0xcb, 0xf4, 0x11, 0x12, 0x73, 0x49, 0x6b, 0x22, 0x31, 0x97, 0x35, 0x1d, 0xf4, + 0xc7, 0x88, 0xf9, 0xf1, 0xc1, 0x27, 0x39, 0xcc, 0xb2, 0x1c, 0xe9, 0xbe, 0xe6, 0x7d, 0xcc, 0x77, + 0x71, 0x32, 0xb9, 0x4c, 0xec, 0x24, 0x92, 0x7b, 0xc6, 0x4e, 0x99, 0x5e, 0x24, 0x63, 0xa7, 0x6c, + 0xbf, 0x41, 0x3f, 0x45, 0x9d, 0x8f, 0x74, 0x3d, 0xa7, 0x53, 0x94, 0x6b, 0xdd, 0xd7, 0x9e, 0x8f, + 0x69, 0xf6, 0x37, 0x00, 0x69, 0xc1, 0x25, 0xd2, 0x6c, 0xa1, 0xe6, 0x13, 0x69, 0xb6, 0x58, 0x97, + 0xd1, 0x0e, 0xea, 0x68, 0x93, 0x56, 0xf9, 0xbe, 0xc8, 0x28, 0xf5, 0x38, 0xd6, 0x2d, 0x59, 0x8f, + 0xab, 0x85, 0x57, 0xd6, 0xe3, 0x99, 0x22, 0x87, 0xee, 0xa1, 0x16, 0x5d, 0xdf, 0xca, 0x7b, 0x1c, + 0xa7, 0xf1, 0x4d, 0x38, 0xb0, 0x96, 0x29, 0x3e, 0x84, 0x9e, 0xb2, 0xda, 0x45, 0xe8, 0x29, 0xad, + 0x54, 0xe2, 0x9b, 0x89, 0x74, 0xf2, 0x7a, 0xa6, 0x03, 0xf5, 0x72, 0x22, 0xdf, 0x02, 0xa4, 0xa5, + 0x86, 0x30, 0x59, 0xa1, 0x46, 0x11, 0x26, 0x2b, 0x56, 0x24, 0xf4, 0x63, 0x54, 0xf2, 0x90, 0xcc, + 0xbb, 0xfe, 0x9e, 0xb7, 0xff, 0xf9, 0xb6, 0xa3, 0xbd, 0x79, 0xdb, 0xd1, 0xfe, 0xf3, 0xb6, 0xa3, + 0xfd, 0xf9, 0x5d, 0xa7, 0xf2, 0xe6, 0x5d, 0xa7, 0xf2, 0xef, 0x77, 0x9d, 0xca, 0x60, 0x09, 0xff, + 0x80, 0x7d, 0xf6, 0xbf, 0x00, 0x00, 0x00, 0xff, 0xff, 0x02, 0xfe, 0xc8, 0x6e, 0x45, 0x1b, 0x00, + 0x00, } // Reference imports to suppress errors if they are not otherwise used. @@ -3362,7 +3202,6 @@ type MasterClient interface { UnlockDDLLock(ctx context.Context, in *UnlockDDLLockRequest, opts ...grpc.CallOption) (*UnlockDDLLockResponse, error) UpdateMasterConfig(ctx context.Context, in *UpdateMasterConfigRequest, opts ...grpc.CallOption) (*UpdateMasterConfigResponse, error) UpdateWorkerRelayConfig(ctx context.Context, in *UpdateWorkerRelayConfigRequest, opts ...grpc.CallOption) (*CommonWorkerResponse, error) - HandleSQLs(ctx context.Context, in *HandleSQLsRequest, opts ...grpc.CallOption) (*HandleSQLsResponse, error) // SwitchWorkerRelayMaster requests some dm-workers to switch relay unit's master server SwitchWorkerRelayMaster(ctx context.Context, in *SwitchWorkerRelayMasterRequest, opts ...grpc.CallOption) (*SwitchWorkerRelayMasterResponse, error) // OperateWorkerRelayTask requests some dm-workers to operate relay unit @@ -3480,15 +3319,6 @@ func (c *masterClient) UpdateWorkerRelayConfig(ctx context.Context, in *UpdateWo return out, nil } -func (c *masterClient) HandleSQLs(ctx context.Context, in *HandleSQLsRequest, opts ...grpc.CallOption) (*HandleSQLsResponse, error) { - out := new(HandleSQLsResponse) - err := c.cc.Invoke(ctx, "/pb.Master/HandleSQLs", in, out, opts...) - if err != nil { - return nil, err - } - return out, nil -} - func (c *masterClient) SwitchWorkerRelayMaster(ctx context.Context, in *SwitchWorkerRelayMasterRequest, opts ...grpc.CallOption) (*SwitchWorkerRelayMasterResponse, error) { out := new(SwitchWorkerRelayMasterResponse) err := c.cc.Invoke(ctx, "/pb.Master/SwitchWorkerRelayMaster", in, out, opts...) @@ -3619,7 +3449,6 @@ type MasterServer interface { UnlockDDLLock(context.Context, *UnlockDDLLockRequest) (*UnlockDDLLockResponse, error) UpdateMasterConfig(context.Context, *UpdateMasterConfigRequest) (*UpdateMasterConfigResponse, error) UpdateWorkerRelayConfig(context.Context, *UpdateWorkerRelayConfigRequest) (*CommonWorkerResponse, error) - HandleSQLs(context.Context, *HandleSQLsRequest) (*HandleSQLsResponse, error) // SwitchWorkerRelayMaster requests some dm-workers to switch relay unit's master server SwitchWorkerRelayMaster(context.Context, *SwitchWorkerRelayMasterRequest) (*SwitchWorkerRelayMasterResponse, error) // OperateWorkerRelayTask requests some dm-workers to operate relay unit @@ -3679,9 +3508,6 @@ func (*UnimplementedMasterServer) UpdateMasterConfig(ctx context.Context, req *U func (*UnimplementedMasterServer) UpdateWorkerRelayConfig(ctx context.Context, req *UpdateWorkerRelayConfigRequest) (*CommonWorkerResponse, error) { return nil, status.Errorf(codes.Unimplemented, "method UpdateWorkerRelayConfig not implemented") } -func (*UnimplementedMasterServer) 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") } @@ -3888,24 +3714,6 @@ func _Master_UpdateWorkerRelayConfig_Handler(srv interface{}, ctx context.Contex return interceptor(ctx, in, info, handler) } -func _Master_HandleSQLs_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { - in := new(HandleSQLsRequest) - if err := dec(in); err != nil { - return nil, err - } - if interceptor == nil { - return srv.(MasterServer).HandleSQLs(ctx, in) - } - info := &grpc.UnaryServerInfo{ - Server: srv, - FullMethod: "/pb.Master/HandleSQLs", - } - handler := func(ctx context.Context, req interface{}) (interface{}, error) { - return srv.(MasterServer).HandleSQLs(ctx, req.(*HandleSQLsRequest)) - } - return interceptor(ctx, in, info, handler) -} - func _Master_SwitchWorkerRelayMaster_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { in := new(SwitchWorkerRelayMasterRequest) if err := dec(in); err != nil { @@ -4180,10 +3988,6 @@ var _Master_serviceDesc = grpc.ServiceDesc{ MethodName: "UpdateWorkerRelayConfig", Handler: _Master_UpdateWorkerRelayConfig_Handler, }, - { - MethodName: "HandleSQLs", - Handler: _Master_HandleSQLs_Handler, - }, { MethodName: "SwitchWorkerRelayMaster", Handler: _Master_SwitchWorkerRelayMaster_Handler, @@ -5329,7 +5133,7 @@ func (m *OperateWorkerRelayResponse) MarshalToSizedBuffer(dAtA []byte) (int, err return len(dAtA) - i, nil } -func (m *HandleSQLsRequest) Marshal() (dAtA []byte, err error) { +func (m *PurgeWorkerRelayRequest) Marshal() (dAtA []byte, err error) { size := m.Size() dAtA = make([]byte, size) n, err := m.MarshalToSizedBuffer(dAtA[:size]) @@ -5339,72 +5143,58 @@ func (m *HandleSQLsRequest) Marshal() (dAtA []byte, err error) { return dAtA[:n], nil } -func (m *HandleSQLsRequest) MarshalTo(dAtA []byte) (int, error) { +func (m *PurgeWorkerRelayRequest) MarshalTo(dAtA []byte) (int, error) { size := m.Size() return m.MarshalToSizedBuffer(dAtA[:size]) } -func (m *HandleSQLsRequest) MarshalToSizedBuffer(dAtA []byte) (int, error) { +func (m *PurgeWorkerRelayRequest) MarshalToSizedBuffer(dAtA []byte) (int, error) { i := len(dAtA) _ = i var l int _ = l - if m.Sharding { - i-- - if m.Sharding { - dAtA[i] = 1 - } else { - dAtA[i] = 0 - } + 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] = 0x38 + dAtA[i] = 0x2a } - if len(m.SqlPattern) > 0 { - i -= len(m.SqlPattern) - copy(dAtA[i:], m.SqlPattern) - i = encodeVarintDmmaster(dAtA, i, uint64(len(m.SqlPattern))) + 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] = 0x32 + dAtA[i] = 0x22 } - if len(m.Source) > 0 { - i -= len(m.Source) - copy(dAtA[i:], m.Source) - i = encodeVarintDmmaster(dAtA, i, uint64(len(m.Source))) + if m.Time != 0 { + i = encodeVarintDmmaster(dAtA, i, uint64(m.Time)) i-- - dAtA[i] = 0x2a + dAtA[i] = 0x18 } - if len(m.BinlogPos) > 0 { - i -= len(m.BinlogPos) - copy(dAtA[i:], m.BinlogPos) - i = encodeVarintDmmaster(dAtA, i, uint64(len(m.BinlogPos))) + if m.Inactive { i-- - dAtA[i] = 0x22 - } - 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 + if m.Inactive { + dAtA[i] = 1 + } else { + dAtA[i] = 0 } - } - 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 + 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 len(dAtA) - i, nil } -func (m *HandleSQLsResponse) Marshal() (dAtA []byte, err error) { +func (m *PurgeWorkerRelayResponse) Marshal() (dAtA []byte, err error) { size := m.Size() dAtA = make([]byte, size) n, err := m.MarshalToSizedBuffer(dAtA[:size]) @@ -5414,12 +5204,12 @@ func (m *HandleSQLsResponse) Marshal() (dAtA []byte, err error) { return dAtA[:n], nil } -func (m *HandleSQLsResponse) MarshalTo(dAtA []byte) (int, error) { +func (m *PurgeWorkerRelayResponse) MarshalTo(dAtA []byte) (int, error) { size := m.Size() return m.MarshalToSizedBuffer(dAtA[:size]) } -func (m *HandleSQLsResponse) MarshalToSizedBuffer(dAtA []byte) (int, error) { +func (m *PurgeWorkerRelayResponse) MarshalToSizedBuffer(dAtA []byte) (int, error) { i := len(dAtA) _ = i var l int @@ -5458,7 +5248,7 @@ func (m *HandleSQLsResponse) MarshalToSizedBuffer(dAtA []byte) (int, error) { return len(dAtA) - i, nil } -func (m *PurgeWorkerRelayRequest) Marshal() (dAtA []byte, err error) { +func (m *CheckTaskRequest) Marshal() (dAtA []byte, err error) { size := m.Size() dAtA = make([]byte, size) n, err := m.MarshalToSizedBuffer(dAtA[:size]) @@ -5468,137 +5258,22 @@ func (m *PurgeWorkerRelayRequest) Marshal() (dAtA []byte, err error) { return dAtA[:n], nil } -func (m *PurgeWorkerRelayRequest) MarshalTo(dAtA []byte) (int, error) { +func (m *CheckTaskRequest) MarshalTo(dAtA []byte) (int, error) { size := m.Size() return m.MarshalToSizedBuffer(dAtA[:size]) } -func (m *PurgeWorkerRelayRequest) MarshalToSizedBuffer(dAtA []byte) (int, error) { +func (m *CheckTaskRequest) 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 = 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))) + 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] = 0x22 - } - if m.Time != 0 { - i = encodeVarintDmmaster(dAtA, i, uint64(m.Time)) - i-- - dAtA[i] = 0x18 - } - if m.Inactive { - i-- - if m.Inactive { - dAtA[i] = 1 - } else { - dAtA[i] = 0 - } - i-- - dAtA[i] = 0x10 - } - 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 len(dAtA) - i, nil -} - -func (m *PurgeWorkerRelayResponse) 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 *PurgeWorkerRelayResponse) MarshalTo(dAtA []byte) (int, error) { - 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 len(m.Sources) > 0 { - for iNdEx := len(m.Sources) - 1; iNdEx >= 0; iNdEx-- { - { - size, err := m.Sources[iNdEx].MarshalToSizedBuffer(dAtA[:i]) - if err != nil { - return 0, err - } - i -= size - i = encodeVarintDmmaster(dAtA, i, uint64(size)) - } - i-- - dAtA[i] = 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 { - i-- - if m.Result { - dAtA[i] = 1 - } else { - dAtA[i] = 0 - } - i-- - dAtA[i] = 0x8 - } - return len(dAtA) - i, nil -} - -func (m *CheckTaskRequest) 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 *CheckTaskRequest) MarshalTo(dAtA []byte) (int, error) { - 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 { - i -= len(m.Task) - copy(dAtA[i:], m.Task) - i = encodeVarintDmmaster(dAtA, i, uint64(len(m.Task))) - i-- - dAtA[i] = 0xa + dAtA[i] = 0xa } return len(dAtA) - i, nil } @@ -7185,65 +6860,6 @@ func (m *OperateWorkerRelayResponse) Size() (n int) { return n } -func (m *HandleSQLsRequest) Size() (n int) { - if m == nil { - return 0 - } - var l int - _ = l - l = len(m.Name) - if l > 0 { - n += 1 + l + sovDmmaster(uint64(l)) - } - if m.Op != 0 { - n += 1 + sovDmmaster(uint64(m.Op)) - } - if len(m.Args) > 0 { - for _, s := range m.Args { - l = len(s) - n += 1 + l + sovDmmaster(uint64(l)) - } - } - l = len(m.BinlogPos) - if l > 0 { - n += 1 + l + sovDmmaster(uint64(l)) - } - l = len(m.Source) - if l > 0 { - n += 1 + l + sovDmmaster(uint64(l)) - } - l = len(m.SqlPattern) - if l > 0 { - n += 1 + l + sovDmmaster(uint64(l)) - } - if m.Sharding { - n += 2 - } - return n -} - -func (m *HandleSQLsResponse) Size() (n int) { - if m == nil { - return 0 - } - var l int - _ = l - if m.Result { - n += 2 - } - l = len(m.Msg) - if l > 0 { - n += 1 + l + sovDmmaster(uint64(l)) - } - if len(m.Sources) > 0 { - for _, e := range m.Sources { - l = e.Size() - n += 1 + l + sovDmmaster(uint64(l)) - } - } - return n -} - func (m *PurgeWorkerRelayRequest) Size() (n int) { if m == nil { return 0 @@ -10873,397 +10489,6 @@ func (m *OperateWorkerRelayResponse) Unmarshal(dAtA []byte) error { } return nil } -func (m *HandleSQLsRequest) 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: HandleSQLsRequest: wiretype end group for non-group") - } - if fieldNum <= 0 { - return fmt.Errorf("proto: HandleSQLsRequest: 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) - } - 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.Name = string(dAtA[iNdEx:postIndex]) - iNdEx = postIndex - case 2: - 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 |= SQLOp(b&0x7F) << shift - if b < 0x80 { - break - } - } - case 3: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Args", 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.Args = append(m.Args, string(dAtA[iNdEx:postIndex])) - iNdEx = postIndex - case 4: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field BinlogPos", 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.BinlogPos = string(dAtA[iNdEx:postIndex]) - iNdEx = postIndex - case 5: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Source", wireType) - } - var stringLen uint64 - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowDmmaster - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - stringLen |= uint64(b&0x7F) << shift - if b < 0x80 { - break - } - } - intStringLen := int(stringLen) - if intStringLen < 0 { - return ErrInvalidLengthDmmaster - } - postIndex := iNdEx + intStringLen - if postIndex < 0 { - return ErrInvalidLengthDmmaster - } - if postIndex > l { - return io.ErrUnexpectedEOF - } - m.Source = string(dAtA[iNdEx:postIndex]) - iNdEx = postIndex - case 6: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field SqlPattern", 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.SqlPattern = string(dAtA[iNdEx:postIndex]) - iNdEx = postIndex - case 7: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field Sharding", 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.Sharding = bool(v != 0) - default: - iNdEx = preIndex - skippy, err := skipDmmaster(dAtA[iNdEx:]) - if err != nil { - return err - } - if skippy < 0 { - return ErrInvalidLengthDmmaster - } - if (iNdEx + skippy) < 0 { - return ErrInvalidLengthDmmaster - } - if (iNdEx + skippy) > l { - return io.ErrUnexpectedEOF - } - iNdEx += skippy - } - } - - if iNdEx > l { - return io.ErrUnexpectedEOF - } - return nil -} -func (m *HandleSQLsResponse) 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: HandleSQLsResponse: wiretype end group for non-group") - } - if fieldNum <= 0 { - return fmt.Errorf("proto: HandleSQLsResponse: illegal tag %d (wire type %d)", fieldNum, wire) - } - switch fieldNum { - case 1: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field Result", wireType) - } - var v int - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowDmmaster - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - v |= int(b&0x7F) << shift - if b < 0x80 { - break - } - } - m.Result = bool(v != 0) - case 2: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Msg", wireType) - } - var stringLen uint64 - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowDmmaster - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - stringLen |= uint64(b&0x7F) << shift - if b < 0x80 { - break - } - } - intStringLen := int(stringLen) - if intStringLen < 0 { - return ErrInvalidLengthDmmaster - } - postIndex := iNdEx + intStringLen - if postIndex < 0 { - return ErrInvalidLengthDmmaster - } - if postIndex > l { - return io.ErrUnexpectedEOF - } - m.Msg = string(dAtA[iNdEx:postIndex]) - iNdEx = postIndex - case 3: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Sources", wireType) - } - var msglen int - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowDmmaster - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - msglen |= int(b&0x7F) << shift - if b < 0x80 { - break - } - } - if msglen < 0 { - return ErrInvalidLengthDmmaster - } - postIndex := iNdEx + msglen - if postIndex < 0 { - return ErrInvalidLengthDmmaster - } - if postIndex > l { - return io.ErrUnexpectedEOF - } - m.Sources = append(m.Sources, &CommonWorkerResponse{}) - if err := m.Sources[len(m.Sources)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { - return err - } - iNdEx = postIndex - default: - iNdEx = preIndex - skippy, err := skipDmmaster(dAtA[iNdEx:]) - if err != nil { - return err - } - if skippy < 0 { - return ErrInvalidLengthDmmaster - } - if (iNdEx + skippy) < 0 { - return ErrInvalidLengthDmmaster - } - if (iNdEx + skippy) > l { - return io.ErrUnexpectedEOF - } - iNdEx += skippy - } - } - - if iNdEx > l { - return io.ErrUnexpectedEOF - } - return nil -} func (m *PurgeWorkerRelayRequest) Unmarshal(dAtA []byte) error { l := len(dAtA) iNdEx := 0 diff --git a/dm/pb/dmworker.pb.go b/dm/pb/dmworker.pb.go index 5752093a94..4269dd8a7f 100644 --- a/dm/pb/dmworker.pb.go +++ b/dm/pb/dmworker.pb.go @@ -67,34 +67,6 @@ func (TaskOp) EnumDescriptor() ([]byte, []int) { return fileDescriptor_51a1b9e17fd67b10, []int{0} } -type SQLOp int32 - -const ( - SQLOp_SKIP SQLOp = 0 - SQLOp_REPLACE SQLOp = 1 - SQLOp_INJECT SQLOp = 2 -) - -var SQLOp_name = map[int32]string{ - 0: "SKIP", - 1: "REPLACE", - 2: "INJECT", -} - -var SQLOp_value = map[string]int32{ - "SKIP": 0, - "REPLACE": 1, - "INJECT": 2, -} - -func (x SQLOp) String() string { - return proto.EnumName(SQLOp_name, int32(x)) -} - -func (SQLOp) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_51a1b9e17fd67b10, []int{1} -} - // Stage represents current stage for a (sub) task // a (sub) task should be always in one stage of the following stages // (sub) task can transfer from on stage to some special other stages @@ -157,7 +129,7 @@ func (x Stage) String() string { } func (Stage) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_51a1b9e17fd67b10, []int{2} + return fileDescriptor_51a1b9e17fd67b10, []int{1} } // UnitType represents the dm unit's type @@ -195,7 +167,7 @@ func (x UnitType) String() string { } func (UnitType) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_51a1b9e17fd67b10, []int{3} + return fileDescriptor_51a1b9e17fd67b10, []int{2} } // RelayOp differs from TaskOp @@ -227,7 +199,7 @@ func (x RelayOp) String() string { } func (RelayOp) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_51a1b9e17fd67b10, []int{4} + return fileDescriptor_51a1b9e17fd67b10, []int{3} } type SchemaOp int32 @@ -258,7 +230,7 @@ func (x SchemaOp) String() string { } func (SchemaOp) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_51a1b9e17fd67b10, []int{5} + return fileDescriptor_51a1b9e17fd67b10, []int{4} } type StartSubTaskRequest struct { @@ -653,83 +625,6 @@ func (m *QueryErrorRequest) GetName() string { return "" } -// NOTE: we can extract some common(optional) arguments from `args`, like `meta` -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,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"` -} - -func (m *HandleSubTaskSQLsRequest) Reset() { *m = HandleSubTaskSQLsRequest{} } -func (m *HandleSubTaskSQLsRequest) String() string { return proto.CompactTextString(m) } -func (*HandleSubTaskSQLsRequest) ProtoMessage() {} -func (*HandleSubTaskSQLsRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_51a1b9e17fd67b10, []int{8} -} -func (m *HandleSubTaskSQLsRequest) XXX_Unmarshal(b []byte) error { - return m.Unmarshal(b) -} -func (m *HandleSubTaskSQLsRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { - if deterministic { - return xxx_messageInfo_HandleSubTaskSQLsRequest.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 *HandleSubTaskSQLsRequest) XXX_Merge(src proto.Message) { - xxx_messageInfo_HandleSubTaskSQLsRequest.Merge(m, src) -} -func (m *HandleSubTaskSQLsRequest) XXX_Size() int { - return m.Size() -} -func (m *HandleSubTaskSQLsRequest) XXX_DiscardUnknown() { - xxx_messageInfo_HandleSubTaskSQLsRequest.DiscardUnknown(m) -} - -var xxx_messageInfo_HandleSubTaskSQLsRequest proto.InternalMessageInfo - -func (m *HandleSubTaskSQLsRequest) GetName() string { - if m != nil { - return m.Name - } - return "" -} - -func (m *HandleSubTaskSQLsRequest) GetOp() SQLOp { - if m != nil { - return m.Op - } - return SQLOp_SKIP -} - -func (m *HandleSubTaskSQLsRequest) GetArgs() []string { - if m != nil { - return m.Args - } - return nil -} - -func (m *HandleSubTaskSQLsRequest) GetBinlogPos() string { - if m != nil { - return m.BinlogPos - } - return "" -} - -func (m *HandleSubTaskSQLsRequest) GetSqlPattern() string { - if m != nil { - return m.SqlPattern - } - return "" -} - type CommonWorkerResponse 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"` @@ -741,7 +636,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_51a1b9e17fd67b10, []int{9} + return fileDescriptor_51a1b9e17fd67b10, []int{8} } func (m *CommonWorkerResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -811,7 +706,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_51a1b9e17fd67b10, []int{10} + return fileDescriptor_51a1b9e17fd67b10, []int{9} } func (m *QueryStatusResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -880,7 +775,7 @@ 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_51a1b9e17fd67b10, []int{11} + return fileDescriptor_51a1b9e17fd67b10, []int{10} } func (m *QueryErrorResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -952,7 +847,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_51a1b9e17fd67b10, []int{12} + return fileDescriptor_51a1b9e17fd67b10, []int{11} } func (m *CheckStatus) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1032,7 +927,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_51a1b9e17fd67b10, []int{13} + return fileDescriptor_51a1b9e17fd67b10, []int{12} } func (m *DumpStatus) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1073,7 +968,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_51a1b9e17fd67b10, []int{14} + return fileDescriptor_51a1b9e17fd67b10, []int{13} } func (m *LoadStatus) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1148,7 +1043,7 @@ 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_51a1b9e17fd67b10, []int{15} + return fileDescriptor_51a1b9e17fd67b10, []int{14} } func (m *ShardingGroup) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1231,7 +1126,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_51a1b9e17fd67b10, []int{16} + return fileDescriptor_51a1b9e17fd67b10, []int{15} } func (m *SyncStatus) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1349,7 +1244,7 @@ func (m *SourceStatus) Reset() { *m = SourceStatus{} } func (m *SourceStatus) String() string { return proto.CompactTextString(m) } func (*SourceStatus) ProtoMessage() {} func (*SourceStatus) Descriptor() ([]byte, []int) { - return fileDescriptor_51a1b9e17fd67b10, []int{17} + return fileDescriptor_51a1b9e17fd67b10, []int{16} } func (m *SourceStatus) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1422,7 +1317,7 @@ 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_51a1b9e17fd67b10, []int{18} + return fileDescriptor_51a1b9e17fd67b10, []int{17} } func (m *RelayStatus) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1536,7 +1431,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_51a1b9e17fd67b10, []int{19} + return fileDescriptor_51a1b9e17fd67b10, []int{18} } func (m *SubTaskStatus) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1690,7 +1585,7 @@ 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_51a1b9e17fd67b10, []int{20} + return fileDescriptor_51a1b9e17fd67b10, []int{19} } func (m *SubTaskStatusList) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1736,7 +1631,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_51a1b9e17fd67b10, []int{21} + return fileDescriptor_51a1b9e17fd67b10, []int{20} } func (m *CheckError) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1782,7 +1677,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_51a1b9e17fd67b10, []int{22} + return fileDescriptor_51a1b9e17fd67b10, []int{21} } func (m *DumpError) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1827,7 +1722,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_51a1b9e17fd67b10, []int{23} + return fileDescriptor_51a1b9e17fd67b10, []int{22} } func (m *LoadError) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1874,7 +1769,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_51a1b9e17fd67b10, []int{24} + return fileDescriptor_51a1b9e17fd67b10, []int{23} } func (m *SyncSQLError) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1933,7 +1828,7 @@ 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_51a1b9e17fd67b10, []int{25} + return fileDescriptor_51a1b9e17fd67b10, []int{24} } func (m *SyncError) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1981,7 +1876,7 @@ func (m *SourceError) Reset() { *m = SourceError{} } func (m *SourceError) String() string { return proto.CompactTextString(m) } func (*SourceError) ProtoMessage() {} func (*SourceError) Descriptor() ([]byte, []int) { - return fileDescriptor_51a1b9e17fd67b10, []int{26} + return fileDescriptor_51a1b9e17fd67b10, []int{25} } func (m *SourceError) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2047,7 +1942,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_51a1b9e17fd67b10, []int{27} + return fileDescriptor_51a1b9e17fd67b10, []int{26} } func (m *RelayError) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2106,7 +2001,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_51a1b9e17fd67b10, []int{28} + return fileDescriptor_51a1b9e17fd67b10, []int{27} } func (m *SubTaskError) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2246,7 +2141,7 @@ 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_51a1b9e17fd67b10, []int{29} + return fileDescriptor_51a1b9e17fd67b10, []int{28} } func (m *SubTaskErrorList) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2296,7 +2191,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_51a1b9e17fd67b10, []int{30} + return fileDescriptor_51a1b9e17fd67b10, []int{29} } func (m *ProcessResult) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2363,7 +2258,7 @@ 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_51a1b9e17fd67b10, []int{31} + return fileDescriptor_51a1b9e17fd67b10, []int{30} } func (m *ProcessError) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2449,7 +2344,7 @@ func (m *SwitchRelayMasterRequest) Reset() { *m = SwitchRelayMasterReque func (m *SwitchRelayMasterRequest) String() string { return proto.CompactTextString(m) } func (*SwitchRelayMasterRequest) ProtoMessage() {} func (*SwitchRelayMasterRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_51a1b9e17fd67b10, []int{32} + return fileDescriptor_51a1b9e17fd67b10, []int{31} } func (m *SwitchRelayMasterRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2487,7 +2382,7 @@ func (m *OperateRelayRequest) Reset() { *m = OperateRelayRequest{} } func (m *OperateRelayRequest) String() string { return proto.CompactTextString(m) } func (*OperateRelayRequest) ProtoMessage() {} func (*OperateRelayRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_51a1b9e17fd67b10, []int{33} + return fileDescriptor_51a1b9e17fd67b10, []int{32} } func (m *OperateRelayRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2534,7 +2429,7 @@ func (m *OperateRelayResponse) Reset() { *m = OperateRelayResponse{} } func (m *OperateRelayResponse) String() string { return proto.CompactTextString(m) } func (*OperateRelayResponse) ProtoMessage() {} func (*OperateRelayResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_51a1b9e17fd67b10, []int{34} + return fileDescriptor_51a1b9e17fd67b10, []int{33} } func (m *OperateRelayResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2607,7 +2502,7 @@ func (m *PurgeRelayRequest) Reset() { *m = PurgeRelayRequest{} } func (m *PurgeRelayRequest) String() string { return proto.CompactTextString(m) } func (*PurgeRelayRequest) ProtoMessage() {} func (*PurgeRelayRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_51a1b9e17fd67b10, []int{35} + return fileDescriptor_51a1b9e17fd67b10, []int{34} } func (m *PurgeRelayRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2671,7 +2566,7 @@ func (m *QueryWorkerConfigRequest) Reset() { *m = QueryWorkerConfigReque func (m *QueryWorkerConfigRequest) String() string { return proto.CompactTextString(m) } func (*QueryWorkerConfigRequest) ProtoMessage() {} func (*QueryWorkerConfigRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_51a1b9e17fd67b10, []int{36} + return fileDescriptor_51a1b9e17fd67b10, []int{35} } func (m *QueryWorkerConfigRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2711,7 +2606,7 @@ func (m *QueryWorkerConfigResponse) Reset() { *m = QueryWorkerConfigResp func (m *QueryWorkerConfigResponse) String() string { return proto.CompactTextString(m) } func (*QueryWorkerConfigResponse) ProtoMessage() {} func (*QueryWorkerConfigResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_51a1b9e17fd67b10, []int{37} + return fileDescriptor_51a1b9e17fd67b10, []int{36} } func (m *QueryWorkerConfigResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2781,7 +2676,7 @@ func (m *OperateWorkerSchemaRequest) Reset() { *m = OperateWorkerSchemaR func (m *OperateWorkerSchemaRequest) String() string { return proto.CompactTextString(m) } func (*OperateWorkerSchemaRequest) ProtoMessage() {} func (*OperateWorkerSchemaRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_51a1b9e17fd67b10, []int{38} + return fileDescriptor_51a1b9e17fd67b10, []int{37} } func (m *OperateWorkerSchemaRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2854,7 +2749,6 @@ func (m *OperateWorkerSchemaRequest) GetSchema() string { 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.RelayOp", RelayOp_name, RelayOp_value) @@ -2867,7 +2761,6 @@ func init() { proto.RegisterType((*UpdateSubTaskRequest)(nil), "pb.UpdateSubTaskRequest") proto.RegisterType((*QueryStatusRequest)(nil), "pb.QueryStatusRequest") proto.RegisterType((*QueryErrorRequest)(nil), "pb.QueryErrorRequest") - proto.RegisterType((*HandleSubTaskSQLsRequest)(nil), "pb.HandleSubTaskSQLsRequest") proto.RegisterType((*CommonWorkerResponse)(nil), "pb.CommonWorkerResponse") proto.RegisterType((*QueryStatusResponse)(nil), "pb.QueryStatusResponse") proto.RegisterType((*QueryErrorResponse)(nil), "pb.QueryErrorResponse") @@ -2903,141 +2796,135 @@ func init() { func init() { proto.RegisterFile("dmworker.proto", fileDescriptor_51a1b9e17fd67b10) } var fileDescriptor_51a1b9e17fd67b10 = []byte{ - // 2143 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xb4, 0x19, 0x4d, 0x6f, 0xe4, 0x48, - 0xb5, 0xed, 0xfe, 0x48, 0xf7, 0xeb, 0xee, 0x8c, 0x53, 0x99, 0xcd, 0xf6, 0x34, 0xb3, 0x4d, 0xe4, - 0x59, 0xed, 0x66, 0x73, 0x88, 0xd8, 0x00, 0x42, 0x42, 0x5a, 0x60, 0xa7, 0x93, 0xcd, 0x84, 0xed, - 0xcc, 0x24, 0xee, 0x19, 0x71, 0x44, 0x8e, 0x5d, 0xe9, 0x58, 0x71, 0xdb, 0x1e, 0x7f, 0x24, 0xca, - 0x9f, 0x00, 0x2e, 0x1c, 0x10, 0xdc, 0x10, 0xe2, 0x82, 0x10, 0xff, 0x02, 0x8e, 0x2b, 0x4e, 0x1c, - 0xd1, 0xcc, 0xdf, 0xe0, 0x80, 0xde, 0xab, 0xb2, 0x5d, 0x4e, 0xba, 0x7b, 0x76, 0x24, 0xb8, 0xf9, - 0x7d, 0xd4, 0xab, 0x57, 0xef, 0xb3, 0xea, 0x19, 0xd6, 0xdd, 0xf9, 0x4d, 0x18, 0x5f, 0xf1, 0x78, - 0x2f, 0x8a, 0xc3, 0x34, 0x64, 0x7a, 0x74, 0x6e, 0x7e, 0x06, 0x9b, 0xd3, 0xd4, 0x8e, 0xd3, 0x69, - 0x76, 0xfe, 0xd2, 0x4e, 0xae, 0x2c, 0xfe, 0x3a, 0xe3, 0x49, 0xca, 0x18, 0x34, 0x52, 0x3b, 0xb9, - 0x1a, 0x68, 0xdb, 0xda, 0x4e, 0xc7, 0xa2, 0x6f, 0x73, 0x0f, 0xd8, 0xab, 0xc8, 0xb5, 0x53, 0x6e, - 0x71, 0xdf, 0xbe, 0xcd, 0x39, 0x07, 0xb0, 0xe6, 0x84, 0x41, 0xca, 0x83, 0x54, 0x32, 0xe7, 0xa0, - 0x39, 0x85, 0xcd, 0x13, 0x6f, 0x16, 0xdf, 0x5d, 0x30, 0x02, 0x78, 0xea, 0x05, 0x7e, 0x38, 0x7b, - 0x6e, 0xcf, 0xb9, 0x5c, 0xa3, 0x60, 0xd8, 0x63, 0xe8, 0x08, 0xe8, 0x34, 0x4c, 0x06, 0xfa, 0xb6, - 0xb6, 0xd3, 0xb7, 0x4a, 0x84, 0x79, 0x04, 0x1f, 0xbc, 0x88, 0x38, 0x0a, 0xbd, 0xa3, 0xf1, 0x10, - 0xf4, 0x30, 0x22, 0x71, 0xeb, 0xfb, 0xb0, 0x17, 0x9d, 0xef, 0x21, 0xf1, 0x45, 0x64, 0xe9, 0x61, - 0x84, 0xa7, 0x09, 0x70, 0x33, 0x5d, 0x9c, 0x06, 0xbf, 0xcd, 0x6b, 0xd8, 0xba, 0x2b, 0x28, 0x89, - 0xc2, 0x20, 0xe1, 0x2b, 0x25, 0x6d, 0x41, 0x2b, 0xe6, 0x49, 0xe6, 0xa7, 0x24, 0xab, 0x6d, 0x49, - 0x08, 0xf1, 0x49, 0x98, 0xc5, 0x0e, 0x1f, 0xd4, 0x69, 0x0f, 0x09, 0x31, 0x03, 0xea, 0xf3, 0x64, - 0x36, 0x68, 0x10, 0x12, 0x3f, 0xcd, 0x5d, 0x78, 0x28, 0xac, 0xf8, 0x2d, 0x2c, 0xbe, 0x03, 0xec, - 0x2c, 0xe3, 0xf1, 0xed, 0x34, 0xb5, 0xd3, 0x2c, 0x51, 0x38, 0x83, 0xd2, 0x74, 0xe2, 0x34, 0x9f, - 0xc2, 0x06, 0x71, 0x1e, 0xc6, 0x71, 0x18, 0xaf, 0x62, 0xfc, 0xbd, 0x06, 0x83, 0x67, 0x76, 0xe0, - 0xfa, 0xf9, 0xfe, 0xd3, 0xb3, 0xc9, 0x2a, 0xc9, 0xec, 0x11, 0x59, 0x43, 0x27, 0x6b, 0x74, 0xd0, - 0x1a, 0xd3, 0xb3, 0x49, 0x69, 0x56, 0x3b, 0x9e, 0x25, 0x83, 0xfa, 0x76, 0x1d, 0xd9, 0xf1, 0x1b, - 0xbd, 0x77, 0x5e, 0x78, 0x4f, 0x1c, 0xbb, 0x44, 0xa0, 0xef, 0x93, 0xd7, 0xfe, 0xa9, 0x9d, 0xa6, - 0x3c, 0x0e, 0x06, 0x4d, 0xe1, 0xfb, 0x12, 0x63, 0x46, 0xf0, 0x70, 0x1c, 0xce, 0xe7, 0x61, 0xf0, - 0x0b, 0x8a, 0xd3, 0xc2, 0x25, 0xa5, 0xd9, 0xb5, 0x8a, 0xd9, 0xa5, 0x79, 0xf5, 0xc2, 0xbc, 0x4b, - 0x1d, 0xb1, 0x05, 0x2d, 0x11, 0xfb, 0x52, 0x29, 0x09, 0x99, 0x7f, 0xd5, 0x60, 0xb3, 0x62, 0xe3, - 0xf7, 0xde, 0xf1, 0x07, 0xd0, 0x13, 0x7b, 0x08, 0x09, 0xb4, 0x6f, 0x77, 0xdf, 0x20, 0x53, 0x29, - 0x78, 0xab, 0xc2, 0xc5, 0x7e, 0x04, 0xfd, 0x44, 0x3a, 0x40, 0x2c, 0x6b, 0x6c, 0xd7, 0x77, 0xba, - 0xfb, 0x1b, 0xb4, 0x4c, 0x25, 0x58, 0x55, 0x3e, 0xf3, 0xcf, 0x9a, 0x0c, 0x0a, 0xe9, 0xea, 0xf7, - 0xd6, 0xf7, 0x73, 0xe8, 0x0a, 0xbd, 0x48, 0x80, 0x54, 0xf7, 0x41, 0xa9, 0xae, 0x90, 0xab, 0xf2, - 0xd0, 0x11, 0x85, 0x12, 0x62, 0x8d, 0xd0, 0xd5, 0x50, 0x74, 0x15, 0x8b, 0x2a, 0x5c, 0xe6, 0x9f, - 0x34, 0xe8, 0x8e, 0x2f, 0xb9, 0x23, 0x35, 0x47, 0x15, 0x23, 0x3b, 0x49, 0xb8, 0x9b, 0xab, 0x28, - 0x20, 0xf6, 0x10, 0x9a, 0x69, 0x98, 0xda, 0x3e, 0x29, 0xd9, 0xb4, 0x04, 0x40, 0xa1, 0x92, 0x39, - 0x0e, 0x4f, 0x92, 0x8b, 0xcc, 0x27, 0x2d, 0x9b, 0x96, 0x82, 0x41, 0x69, 0x17, 0xb6, 0xe7, 0x73, - 0x97, 0x1c, 0xda, 0xb4, 0x24, 0x84, 0xf5, 0xe8, 0xc6, 0x8e, 0x03, 0x2f, 0x98, 0x51, 0x7c, 0x35, - 0xad, 0x1c, 0xc4, 0x15, 0x2e, 0x4f, 0x6d, 0xcf, 0x1f, 0xb4, 0xb6, 0xb5, 0x9d, 0x9e, 0x25, 0x21, - 0xb3, 0x07, 0x70, 0x90, 0xcd, 0x23, 0x69, 0xdf, 0x5f, 0x69, 0x00, 0x93, 0xd0, 0x76, 0xa5, 0xd2, - 0x1f, 0x43, 0xff, 0xc2, 0x0b, 0xbc, 0xe4, 0x92, 0xbb, 0x4f, 0x6f, 0x53, 0x9e, 0x90, 0xee, 0x75, - 0xab, 0x8a, 0x44, 0x65, 0x49, 0x6b, 0xc1, 0xa2, 0x13, 0x8b, 0x82, 0x61, 0x43, 0x68, 0x47, 0x71, - 0x38, 0x8b, 0x79, 0x92, 0xc8, 0xb8, 0x2c, 0x60, 0x5c, 0x3b, 0xe7, 0xa9, 0x2d, 0x4a, 0x9c, 0x8c, - 0x4e, 0x05, 0x63, 0xfe, 0x56, 0x83, 0xfe, 0xf4, 0xd2, 0x8e, 0x5d, 0x2f, 0x98, 0x1d, 0xc5, 0x61, - 0x46, 0x45, 0x28, 0xb5, 0xe3, 0x19, 0xcf, 0x2b, 0xae, 0x84, 0x30, 0x1f, 0x0f, 0x0e, 0x26, 0xb8, - 0x3f, 0xe5, 0x23, 0x7e, 0x0b, 0xfd, 0xe3, 0x24, 0x9d, 0x84, 0x8e, 0x9d, 0x7a, 0x61, 0x20, 0xb7, - 0xaf, 0x22, 0x29, 0x6b, 0x6e, 0x03, 0x87, 0x8c, 0x59, 0xa7, 0xac, 0x21, 0x08, 0xf5, 0xce, 0x02, - 0x49, 0x69, 0x12, 0xa5, 0x80, 0xcd, 0x3f, 0xd6, 0x01, 0xa6, 0xb7, 0x81, 0x23, 0x0d, 0xb5, 0x0d, - 0x5d, 0x3a, 0xf0, 0xe1, 0x35, 0x0f, 0xd2, 0xdc, 0x4c, 0x2a, 0x0a, 0x85, 0x11, 0xf8, 0x32, 0xca, - 0x4d, 0x54, 0xc0, 0x58, 0x36, 0x62, 0xee, 0xf0, 0x20, 0x45, 0x62, 0x9d, 0x88, 0x25, 0x82, 0x99, - 0xd0, 0x9b, 0xdb, 0x49, 0xca, 0xe3, 0x8a, 0x91, 0x2a, 0x38, 0xb6, 0x0b, 0x86, 0x0a, 0x1f, 0xa5, - 0x9e, 0x2b, 0x0b, 0xcc, 0x3d, 0x3c, 0xca, 0xa3, 0x43, 0xe4, 0xf2, 0x5a, 0x42, 0x9e, 0x8a, 0x43, - 0x79, 0x2a, 0x4c, 0xf2, 0xd6, 0x84, 0xbc, 0xbb, 0x78, 0x94, 0x77, 0xee, 0x87, 0xce, 0x95, 0x17, - 0xcc, 0xc8, 0x01, 0x6d, 0x32, 0x55, 0x05, 0xc7, 0xbe, 0x00, 0x23, 0x0b, 0x62, 0x9e, 0x84, 0xfe, - 0x35, 0x77, 0xc9, 0x8f, 0xc9, 0xa0, 0xa3, 0xe4, 0xbc, 0xea, 0x61, 0xeb, 0x1e, 0xab, 0xe2, 0x21, - 0x10, 0xc9, 0x23, 0x3d, 0x34, 0x02, 0x10, 0xe5, 0xf5, 0xe5, 0x6d, 0xc4, 0x07, 0x5d, 0x11, 0x3d, - 0x25, 0xc6, 0xfc, 0x83, 0x06, 0x3d, 0xb5, 0x0c, 0x29, 0x05, 0x52, 0x5b, 0x52, 0x20, 0x75, 0xb5, - 0x40, 0xb2, 0xcf, 0x8a, 0xc2, 0x22, 0x2a, 0x05, 0x69, 0x7b, 0x1a, 0x87, 0x98, 0x87, 0x16, 0x11, - 0x8a, 0x5a, 0xf3, 0x39, 0x74, 0x63, 0xec, 0xf4, 0x45, 0x45, 0x2b, 0x2a, 0x8b, 0x55, 0xa2, 0x2d, - 0x95, 0xc7, 0xfc, 0xbb, 0x0e, 0x5d, 0x85, 0x78, 0xcf, 0xd3, 0xda, 0xb7, 0xf4, 0xb4, 0xbe, 0xc4, - 0xd3, 0xdb, 0xb9, 0x4a, 0xd9, 0xf9, 0x81, 0x17, 0xcb, 0xe0, 0x57, 0x51, 0x05, 0x47, 0x25, 0xb4, - 0x54, 0x14, 0xdb, 0x81, 0x07, 0x0a, 0xa8, 0x04, 0xd6, 0x5d, 0x34, 0xdb, 0x03, 0x46, 0xa8, 0xb1, - 0x9d, 0x3a, 0x97, 0xaf, 0xa2, 0x13, 0xd2, 0x86, 0xa2, 0xab, 0x6d, 0x2d, 0xa0, 0xb0, 0xef, 0x42, - 0x33, 0x49, 0xed, 0x19, 0xa7, 0xc0, 0xca, 0xdb, 0x2b, 0x22, 0x2c, 0x81, 0x57, 0x8c, 0xdf, 0x7e, - 0x87, 0xf1, 0xcd, 0xff, 0xe8, 0xd0, 0xaf, 0x34, 0x8e, 0x85, 0xdd, 0xbc, 0xd8, 0x51, 0x5f, 0xb2, - 0xe3, 0x36, 0x34, 0xb2, 0xc0, 0x13, 0xce, 0x5e, 0xdf, 0xef, 0x21, 0xfd, 0x55, 0xe0, 0xa5, 0x18, - 0x4b, 0x16, 0x51, 0x14, 0x9d, 0x1a, 0xef, 0x0a, 0x88, 0xef, 0xc1, 0x66, 0x19, 0xc8, 0x07, 0x07, - 0x93, 0x49, 0xe8, 0x5c, 0x1d, 0x1f, 0x48, 0xeb, 0x2d, 0x22, 0x31, 0x26, 0xda, 0x15, 0x25, 0xe4, - 0xb3, 0x9a, 0x68, 0x58, 0x9f, 0x42, 0xd3, 0xc1, 0x36, 0x42, 0x56, 0x92, 0x01, 0xa5, 0xf4, 0x95, - 0x67, 0x35, 0x4b, 0xd0, 0xd9, 0xc7, 0xd0, 0x70, 0xb3, 0x79, 0x24, 0x6d, 0xb5, 0x8e, 0x7c, 0x65, - 0x61, 0x7f, 0x56, 0xb3, 0x88, 0x8a, 0x5c, 0x7e, 0x68, 0xbb, 0x83, 0x4e, 0xc9, 0x55, 0xd6, 0x7b, - 0xe4, 0x42, 0x2a, 0x72, 0x61, 0x86, 0x51, 0xb6, 0x49, 0xae, 0xb2, 0xd8, 0x21, 0x17, 0x52, 0x9f, - 0xb6, 0xa1, 0x95, 0x88, 0x40, 0xfe, 0x09, 0x6c, 0x54, 0xac, 0x3f, 0xf1, 0x12, 0x32, 0x95, 0x20, - 0x0f, 0xb4, 0x65, 0xdd, 0x3d, 0x5f, 0x3f, 0x02, 0xa0, 0x33, 0x89, 0x86, 0x2b, 0xbb, 0xb6, 0x56, - 0x5e, 0x1b, 0x3f, 0x82, 0x0e, 0x9e, 0x65, 0x05, 0x19, 0x0f, 0xb1, 0x8c, 0x1c, 0x41, 0x8f, 0xb4, - 0x3f, 0x9b, 0x2c, 0xe1, 0x60, 0xfb, 0xf0, 0x50, 0x34, 0xd0, 0xe2, 0xaa, 0xed, 0x51, 0xbb, 0x10, - 0x89, 0xb5, 0x90, 0x86, 0x05, 0x9d, 0xa3, 0xb8, 0xe9, 0xd9, 0x24, 0xef, 0x6a, 0x39, 0x6c, 0xfe, - 0x10, 0x3a, 0xb8, 0xa3, 0xd8, 0x6e, 0x07, 0x5a, 0x44, 0xc8, 0xed, 0x60, 0x14, 0xe6, 0x94, 0x0a, - 0x59, 0x92, 0x6e, 0xfe, 0x5a, 0xab, 0xdc, 0x4e, 0xde, 0xbb, 0x5a, 0x6d, 0xdf, 0xbf, 0xdc, 0x74, - 0xaa, 0x77, 0x99, 0x3d, 0x00, 0x2a, 0x38, 0xf9, 0x4d, 0xa6, 0x70, 0x6f, 0x89, 0xb5, 0x14, 0x0e, - 0x74, 0x4c, 0x09, 0x2d, 0x30, 0xed, 0xef, 0x74, 0xe8, 0xa9, 0x97, 0xa0, 0xff, 0x57, 0xda, 0x31, - 0xe5, 0x25, 0x91, 0x67, 0xc6, 0x27, 0x79, 0x66, 0x34, 0xcb, 0x63, 0x94, 0x51, 0x54, 0x26, 0xc6, - 0x13, 0x99, 0x18, 0x2d, 0x62, 0xeb, 0xe7, 0x89, 0x91, 0x73, 0x89, 0xbc, 0x78, 0x22, 0xf3, 0x62, - 0xad, 0x64, 0x2a, 0x42, 0xaa, 0x48, 0x8b, 0x27, 0x32, 0x2d, 0xda, 0x25, 0x53, 0xe1, 0xe6, 0x22, - 0x2b, 0xd6, 0xa0, 0x49, 0xee, 0x34, 0x7f, 0x0c, 0x86, 0x6a, 0x1a, 0xca, 0x89, 0x4f, 0x24, 0xb1, - 0x12, 0x0a, 0xea, 0x25, 0x52, 0xae, 0x7d, 0x0d, 0xfd, 0x4a, 0x51, 0xc1, 0x4e, 0xe7, 0x25, 0x63, - 0x3b, 0x70, 0xb8, 0x5f, 0x5c, 0x21, 0x15, 0x8c, 0x12, 0x64, 0x7a, 0x29, 0x59, 0x8a, 0xa8, 0x04, - 0x99, 0x72, 0x11, 0xac, 0x57, 0x2e, 0x82, 0xff, 0xd4, 0xa0, 0xa7, 0x2e, 0xc0, 0xbb, 0xe4, 0x61, - 0x1c, 0x8f, 0x43, 0x57, 0x78, 0xb3, 0x69, 0xe5, 0x20, 0x86, 0x3e, 0x7e, 0xfa, 0x76, 0x92, 0xc8, - 0x08, 0x2c, 0x60, 0x49, 0x9b, 0x3a, 0x61, 0x94, 0x3f, 0x42, 0x0a, 0x58, 0xd2, 0x26, 0xfc, 0x9a, - 0xfb, 0xb2, 0xd5, 0x14, 0x30, 0xee, 0x76, 0xc2, 0x93, 0x04, 0xc3, 0x44, 0x54, 0xc8, 0x1c, 0xc4, - 0x55, 0x96, 0x7d, 0x33, 0xb6, 0xb3, 0x84, 0xcb, 0xbb, 0x4a, 0x01, 0xa3, 0x59, 0xf0, 0xb1, 0x64, - 0xc7, 0x61, 0x16, 0xe4, 0x37, 0x14, 0x05, 0x63, 0x0e, 0x61, 0x30, 0xbd, 0xf1, 0x52, 0xe7, 0x92, - 0xa2, 0x58, 0x34, 0x1e, 0xf9, 0xde, 0x33, 0xf7, 0x61, 0x53, 0xbe, 0x81, 0x2b, 0x2f, 0xf4, 0xef, - 0x28, 0x0f, 0xe0, 0x6e, 0x91, 0x1a, 0xe2, 0xd1, 0x67, 0x66, 0xf0, 0xb0, 0xba, 0x46, 0x3e, 0x40, - 0x56, 0x2d, 0xfa, 0x1f, 0x3c, 0x9b, 0x6f, 0x60, 0xe3, 0x34, 0x8b, 0x67, 0x55, 0x45, 0x87, 0xd0, - 0xf6, 0x02, 0xdb, 0x49, 0xbd, 0x6b, 0x2e, 0x03, 0xa2, 0x80, 0xe9, 0x3d, 0xed, 0xc9, 0x37, 0x7f, - 0xdd, 0xa2, 0x6f, 0xe4, 0xbf, 0xf0, 0x7c, 0x4e, 0xe9, 0x29, 0x3d, 0x93, 0xc3, 0xa4, 0x8a, 0xb8, - 0x24, 0xc8, 0x07, 0xa2, 0x80, 0xd0, 0x7e, 0xf4, 0xdc, 0x12, 0x2f, 0xd2, 0x71, 0x18, 0x5c, 0x78, - 0xb3, 0xdc, 0x7e, 0x37, 0xf0, 0x68, 0x01, 0xed, 0x1d, 0x2f, 0xb2, 0xf2, 0xcc, 0xfa, 0xa2, 0x33, - 0xd7, 0xcb, 0x9a, 0xac, 0x8c, 0x56, 0x1a, 0xd5, 0xd1, 0xca, 0xdf, 0x34, 0x18, 0x4a, 0x2f, 0x88, - 0xbd, 0xa7, 0xce, 0x25, 0x9f, 0xdb, 0xb9, 0x5d, 0x1e, 0x2b, 0xbe, 0xa0, 0x5a, 0x22, 0xc8, 0xe5, - 0xb3, 0x9d, 0x26, 0x0d, 0x7a, 0x39, 0x69, 0x58, 0xea, 0x88, 0x21, 0xb4, 0x5d, 0x3b, 0xb5, 0xcf, - 0xed, 0x84, 0xe7, 0xf1, 0x9a, 0xc3, 0xf4, 0x6e, 0xb3, 0xcf, 0xfd, 0x3c, 0x5a, 0x05, 0x40, 0x92, - 0x68, 0x37, 0x19, 0xa9, 0x12, 0xda, 0xfd, 0x25, 0xb4, 0xc4, 0x1c, 0x85, 0xf5, 0xa1, 0x73, 0x1c, - 0x5c, 0xdb, 0xbe, 0xe7, 0xbe, 0x88, 0x8c, 0x1a, 0x6b, 0x43, 0x63, 0x9a, 0x86, 0x91, 0xa1, 0xb1, - 0x0e, 0x34, 0x4f, 0x31, 0xa6, 0x0d, 0x9d, 0x01, 0xb4, 0x30, 0xed, 0xe7, 0xdc, 0xa8, 0x23, 0x9a, - 0x46, 0x54, 0x46, 0x03, 0xd1, 0x62, 0x78, 0x62, 0x34, 0xd9, 0x3a, 0xc0, 0x97, 0x59, 0x1a, 0x4a, - 0xb6, 0xd6, 0xee, 0x2e, 0x34, 0x69, 0x34, 0x41, 0x02, 0xbf, 0x3e, 0x3e, 0x35, 0x6a, 0xac, 0x0b, - 0x6b, 0xd6, 0xe1, 0xe9, 0xe4, 0xcb, 0xf1, 0xa1, 0xa1, 0xe1, 0xda, 0xe3, 0xe7, 0x3f, 0x3f, 0x1c, - 0xbf, 0x34, 0xf4, 0xdd, 0xd7, 0x24, 0x72, 0x86, 0x46, 0xef, 0x49, 0x5d, 0x08, 0x36, 0x6a, 0x6c, - 0x0d, 0xea, 0xcf, 0xf9, 0x8d, 0xa1, 0xd1, 0xe2, 0x2c, 0xc0, 0x97, 0xa3, 0xd0, 0x87, 0x54, 0x73, - 0x8d, 0x3a, 0x12, 0x50, 0xe1, 0x88, 0xbb, 0x46, 0x83, 0xf5, 0xa0, 0xfd, 0x95, 0x7c, 0x0a, 0x1a, - 0x4d, 0x24, 0x21, 0x1b, 0xae, 0x69, 0x21, 0x89, 0x94, 0x43, 0x68, 0x6d, 0xf7, 0x05, 0xb4, 0xf3, - 0x8a, 0xce, 0x1e, 0x40, 0x57, 0xee, 0x8a, 0x28, 0xa3, 0x86, 0x47, 0xa4, 0xba, 0x6d, 0x68, 0xa8, - 0x3d, 0xd6, 0x66, 0x43, 0xc7, 0x2f, 0x2c, 0xc0, 0x46, 0x9d, 0x4e, 0x74, 0x1b, 0x38, 0x46, 0x03, - 0x19, 0x29, 0x03, 0x0c, 0x77, 0xf7, 0x04, 0xd6, 0x64, 0x8a, 0x31, 0x06, 0xeb, 0x52, 0x9e, 0xc4, - 0x18, 0x35, 0xb4, 0x32, 0x6a, 0x29, 0xb8, 0x35, 0xb4, 0x16, 0x1d, 0x40, 0xc0, 0x3a, 0xaa, 0x20, - 0x2c, 0x27, 0x10, 0x75, 0xd4, 0x2f, 0x8f, 0x12, 0xb6, 0x09, 0x0f, 0x72, 0xab, 0x48, 0x94, 0x10, - 0x78, 0xc4, 0x53, 0x81, 0x30, 0x34, 0x92, 0x5f, 0x80, 0x3a, 0x1a, 0xd2, 0xe2, 0xf3, 0xf0, 0x9a, - 0x4b, 0x4c, 0x7d, 0xff, 0x2f, 0x6b, 0xd0, 0x12, 0xc1, 0xc9, 0xc6, 0xd0, 0x53, 0x87, 0x8c, 0xec, - 0x43, 0xd9, 0xff, 0xee, 0x8e, 0x1d, 0x87, 0x03, 0xea, 0x60, 0x0b, 0x26, 0x40, 0x66, 0x8d, 0x1d, - 0xc3, 0x7a, 0x75, 0x60, 0xc7, 0x1e, 0x21, 0xf7, 0xc2, 0x69, 0xe0, 0x70, 0xb8, 0x88, 0x54, 0x88, - 0x3a, 0x84, 0x7e, 0x65, 0x06, 0xc7, 0x68, 0xdf, 0x45, 0x63, 0xb9, 0x95, 0x1a, 0xfd, 0x0c, 0xba, - 0xca, 0xe8, 0x88, 0x6d, 0x21, 0xeb, 0xfd, 0x79, 0xdd, 0xf0, 0xc3, 0x7b, 0xf8, 0x42, 0xc2, 0x17, - 0x00, 0xe5, 0x2c, 0x87, 0x7d, 0x50, 0x30, 0xaa, 0x63, 0xbc, 0xe1, 0xd6, 0x5d, 0x74, 0xb1, 0xfc, - 0x2b, 0x00, 0x39, 0xcb, 0x3b, 0x9b, 0x24, 0xec, 0x31, 0xf2, 0x2d, 0x9b, 0xed, 0xad, 0x3c, 0xc8, - 0x09, 0x6c, 0xdc, 0xeb, 0x11, 0x42, 0xdc, 0xb2, 0xd6, 0xb1, 0x52, 0xdc, 0x18, 0x7a, 0x6a, 0x8b, - 0x10, 0xee, 0x5e, 0xd0, 0x68, 0x84, 0x90, 0x45, 0xdd, 0xc4, 0xac, 0xb1, 0x9f, 0x02, 0x94, 0x05, - 0x5f, 0x98, 0xe6, 0x5e, 0x03, 0x58, 0xa9, 0xc5, 0x11, 0x6c, 0x28, 0xe3, 0x6a, 0x51, 0x9c, 0x85, - 0x8f, 0xee, 0x4f, 0xb1, 0x57, 0x0a, 0xb2, 0xe4, 0x6c, 0x55, 0xad, 0xf2, 0xc2, 0x3a, 0xcb, 0x1a, - 0xc3, 0xf0, 0xa3, 0x25, 0x54, 0xd5, 0x44, 0xea, 0x6c, 0x5c, 0x98, 0x68, 0xc1, 0xb4, 0x7c, 0xa5, - 0x62, 0x5f, 0x43, 0x3f, 0x0f, 0x71, 0x4a, 0x3a, 0x36, 0x52, 0xec, 0xb9, 0xa0, 0x2f, 0xac, 0x12, - 0xf6, 0x74, 0xf0, 0x8f, 0x37, 0x23, 0xed, 0x9b, 0x37, 0x23, 0xed, 0xdf, 0x6f, 0x46, 0xda, 0x6f, - 0xde, 0x8e, 0x6a, 0xdf, 0xbc, 0x1d, 0xd5, 0xfe, 0xf5, 0x76, 0x54, 0x3b, 0x6f, 0xd1, 0xdf, 0x82, - 0xef, 0xff, 0x37, 0x00, 0x00, 0xff, 0xff, 0x19, 0xf9, 0x8a, 0x86, 0x3f, 0x18, 0x00, 0x00, + // 2037 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xb4, 0x18, 0x4d, 0x6f, 0xdb, 0xc8, + 0x55, 0x24, 0x25, 0x59, 0x7a, 0x92, 0x1c, 0x7a, 0xec, 0xf5, 0x6a, 0xd5, 0xac, 0x6a, 0x30, 0x8b, + 0x5d, 0xaf, 0x0f, 0x46, 0xd7, 0x6d, 0x51, 0xa0, 0xc0, 0xf6, 0x23, 0x72, 0xea, 0x04, 0x95, 0x37, + 0x0e, 0x95, 0xa0, 0xc7, 0x82, 0x22, 0xc7, 0x32, 0x61, 0x89, 0x64, 0xf8, 0x61, 0xc3, 0x7f, 0xa2, + 0xed, 0xa5, 0x87, 0x02, 0xbd, 0x15, 0x45, 0x6f, 0x45, 0xff, 0x45, 0x7b, 0x5c, 0xf4, 0xd4, 0x63, + 0x91, 0xf4, 0x67, 0xf4, 0x50, 0xbc, 0x37, 0x43, 0x72, 0x68, 0x7d, 0x64, 0x03, 0x74, 0x6f, 0xf3, + 0x3e, 0xe6, 0xcd, 0xfb, 0x9e, 0x99, 0x07, 0xdb, 0xde, 0xe2, 0x36, 0x8c, 0xaf, 0x79, 0x7c, 0x1c, + 0xc5, 0x61, 0x1a, 0x32, 0x3d, 0x9a, 0x5a, 0x9f, 0xc3, 0xee, 0x24, 0x75, 0xe2, 0x74, 0x92, 0x4d, + 0x5f, 0x3a, 0xc9, 0xb5, 0xcd, 0x5f, 0x67, 0x3c, 0x49, 0x19, 0x83, 0x7a, 0xea, 0x24, 0xd7, 0x7d, + 0xed, 0x40, 0x3b, 0x6c, 0xdb, 0xb4, 0xb6, 0x8e, 0x81, 0xbd, 0x8a, 0x3c, 0x27, 0xe5, 0x36, 0x9f, + 0x3b, 0x77, 0x39, 0x67, 0x1f, 0xb6, 0xdc, 0x30, 0x48, 0x79, 0x90, 0x4a, 0xe6, 0x1c, 0xb4, 0x26, + 0xb0, 0x7b, 0xee, 0xcf, 0xe2, 0xfb, 0x1b, 0x86, 0x00, 0x8f, 0xfd, 0x60, 0x1e, 0xce, 0xbe, 0x72, + 0x16, 0x5c, 0xee, 0x51, 0x30, 0xec, 0x21, 0xb4, 0x05, 0x74, 0x11, 0x26, 0x7d, 0xfd, 0x40, 0x3b, + 0xec, 0xd9, 0x25, 0xc2, 0x3a, 0x83, 0x0f, 0x9e, 0x47, 0x1c, 0x85, 0xde, 0xd3, 0x78, 0x00, 0x7a, + 0x18, 0x91, 0xb8, 0xed, 0x13, 0x38, 0x8e, 0xa6, 0xc7, 0x48, 0x7c, 0x1e, 0xd9, 0x7a, 0x18, 0xa1, + 0x35, 0x01, 0x1e, 0xa6, 0x0b, 0x6b, 0x70, 0x6d, 0xdd, 0xc0, 0xfe, 0x7d, 0x41, 0x49, 0x14, 0x06, + 0x09, 0xdf, 0x28, 0x69, 0x1f, 0x9a, 0x31, 0x4f, 0xb2, 0x79, 0x4a, 0xb2, 0x5a, 0xb6, 0x84, 0x10, + 0x9f, 0x84, 0x59, 0xec, 0xf2, 0xbe, 0x41, 0x67, 0x48, 0x88, 0x99, 0x60, 0x2c, 0x92, 0x59, 0xbf, + 0x4e, 0x48, 0x5c, 0x5a, 0x47, 0xb0, 0x27, 0xbc, 0xf8, 0x0d, 0x3c, 0x7e, 0x08, 0xec, 0x45, 0xc6, + 0xe3, 0xbb, 0x49, 0xea, 0xa4, 0x59, 0xa2, 0x70, 0x06, 0xa5, 0xeb, 0x84, 0x35, 0x9f, 0xc1, 0x0e, + 0x71, 0x3e, 0x89, 0xe3, 0x30, 0xde, 0xc4, 0x18, 0xc1, 0xde, 0x28, 0x5c, 0x2c, 0xc2, 0xe0, 0x57, + 0x94, 0x09, 0x85, 0xd1, 0xa5, 0x61, 0x5a, 0xc5, 0x30, 0x69, 0x80, 0x5e, 0x18, 0xb0, 0xd6, 0xd4, + 0x7d, 0x68, 0x8a, 0xec, 0x92, 0xd6, 0x4a, 0xc8, 0xfa, 0xab, 0x06, 0xbb, 0x15, 0x2b, 0xde, 0xfb, + 0xc4, 0x1f, 0x40, 0x57, 0x9c, 0x21, 0x24, 0xd0, 0xb9, 0x9d, 0x13, 0x13, 0x43, 0x33, 0x51, 0xf0, + 0x76, 0x85, 0x8b, 0xfd, 0x08, 0x7a, 0x89, 0x70, 0xb1, 0xdc, 0x56, 0x3f, 0x30, 0x0e, 0x3b, 0x27, + 0x3b, 0xb4, 0x4d, 0x25, 0xd8, 0x55, 0x3e, 0xeb, 0x2f, 0x9a, 0x74, 0xbb, 0x74, 0xe6, 0x7b, 0xeb, + 0xfb, 0x05, 0x74, 0x84, 0x5e, 0x24, 0x40, 0xaa, 0xfb, 0xa0, 0x54, 0x57, 0xc8, 0x55, 0x79, 0xc8, + 0x44, 0xa1, 0x84, 0xd8, 0x23, 0x74, 0x35, 0x15, 0x5d, 0xc5, 0xa6, 0x0a, 0x97, 0xf5, 0x67, 0x0d, + 0x3a, 0xa3, 0x2b, 0xee, 0x4a, 0xcd, 0x51, 0xc5, 0xc8, 0x49, 0x12, 0xee, 0xe5, 0x2a, 0x0a, 0x88, + 0xed, 0x41, 0x23, 0x0d, 0x53, 0x67, 0x4e, 0x4a, 0x36, 0x6c, 0x01, 0x60, 0x21, 0x26, 0x99, 0xeb, + 0xf2, 0x24, 0xb9, 0xcc, 0xe6, 0xa4, 0x65, 0xc3, 0x56, 0x30, 0x28, 0xed, 0xd2, 0xf1, 0xe7, 0xdc, + 0xa3, 0x80, 0x36, 0x6c, 0x09, 0x61, 0xc5, 0xdf, 0x3a, 0x71, 0xe0, 0x07, 0xb3, 0x7e, 0x83, 0x08, + 0x39, 0x88, 0x3b, 0x3c, 0x9e, 0x3a, 0xfe, 0xbc, 0xdf, 0x3c, 0xd0, 0x0e, 0xbb, 0xb6, 0x84, 0xac, + 0x2e, 0xc0, 0x69, 0xb6, 0x88, 0xa4, 0x7f, 0x7f, 0xa3, 0x01, 0x8c, 0x43, 0xc7, 0x93, 0x4a, 0x7f, + 0x02, 0xbd, 0x4b, 0x3f, 0xf0, 0x93, 0x2b, 0xee, 0x3d, 0xbe, 0x4b, 0x79, 0x42, 0xba, 0x1b, 0x76, + 0x15, 0x89, 0xca, 0x92, 0xd6, 0x82, 0x45, 0x27, 0x16, 0x05, 0xc3, 0x06, 0xd0, 0x8a, 0xe2, 0x70, + 0x16, 0xf3, 0x24, 0x91, 0x79, 0x59, 0xc0, 0xb8, 0x77, 0xc1, 0x53, 0x47, 0x34, 0x11, 0x99, 0x9d, + 0x0a, 0xc6, 0xfa, 0xbd, 0x06, 0xbd, 0xc9, 0x95, 0x13, 0x7b, 0x7e, 0x30, 0x3b, 0x8b, 0xc3, 0x8c, + 0xca, 0x3c, 0x75, 0xe2, 0x19, 0xcf, 0x7b, 0x9a, 0x84, 0xb0, 0xa2, 0x4e, 0x4f, 0xc7, 0x78, 0xbe, + 0x81, 0x15, 0x85, 0x6b, 0xa1, 0x7f, 0x9c, 0xa4, 0xe3, 0xd0, 0x75, 0x52, 0x3f, 0x0c, 0xe4, 0xf1, + 0x55, 0x24, 0x55, 0xcd, 0x5d, 0xe0, 0x92, 0x33, 0x0d, 0xaa, 0x1a, 0x82, 0x50, 0xef, 0x2c, 0x90, + 0x94, 0x06, 0x51, 0x0a, 0xd8, 0xfa, 0x93, 0x01, 0x30, 0xb9, 0x0b, 0x5c, 0xe9, 0xa8, 0x03, 0xe8, + 0x90, 0xc1, 0x4f, 0x6e, 0x78, 0x90, 0xe6, 0x6e, 0x52, 0x51, 0x28, 0x8c, 0xc0, 0x97, 0x51, 0xee, + 0xa2, 0x02, 0xc6, 0xb6, 0x1a, 0x73, 0x97, 0x07, 0x29, 0x12, 0x0d, 0x22, 0x96, 0x08, 0x66, 0x41, + 0x77, 0xe1, 0x24, 0x29, 0x8f, 0x2b, 0x4e, 0xaa, 0xe0, 0xd8, 0x11, 0x98, 0x2a, 0x7c, 0x96, 0xfa, + 0x1e, 0x25, 0x40, 0xdb, 0x5e, 0xc2, 0xa3, 0x3c, 0x32, 0x22, 0x97, 0xd7, 0x14, 0xf2, 0x54, 0x1c, + 0xca, 0x53, 0x61, 0x92, 0xb7, 0x25, 0xe4, 0xdd, 0xc7, 0xa3, 0xbc, 0xe9, 0x3c, 0x74, 0xaf, 0xfd, + 0x60, 0x46, 0x01, 0x68, 0x91, 0xab, 0x2a, 0x38, 0xf6, 0x25, 0x98, 0x59, 0x10, 0xf3, 0x24, 0x9c, + 0xdf, 0x70, 0x8f, 0xe2, 0x98, 0xf4, 0xdb, 0x4a, 0xcd, 0xab, 0x11, 0xb6, 0x97, 0x58, 0x95, 0x08, + 0x81, 0x28, 0x1e, 0x19, 0xa1, 0x21, 0xc0, 0x94, 0x14, 0x79, 0x79, 0x17, 0xf1, 0x7e, 0x47, 0x64, + 0x4f, 0x89, 0xb1, 0xfe, 0xa8, 0x41, 0x57, 0x6d, 0x43, 0x4a, 0x83, 0xd4, 0xd6, 0x34, 0x48, 0x5d, + 0x6d, 0x90, 0xec, 0xf3, 0xa2, 0xb1, 0x88, 0x4e, 0x41, 0xda, 0x5e, 0xc4, 0x21, 0xd6, 0xa1, 0x4d, + 0x84, 0xa2, 0xd7, 0x7c, 0x01, 0x9d, 0x18, 0xef, 0xd2, 0xa2, 0xa3, 0x15, 0x9d, 0xc5, 0x2e, 0xd1, + 0xb6, 0xca, 0x63, 0xfd, 0x5d, 0x87, 0x8e, 0x42, 0x5c, 0x8a, 0xb4, 0xf6, 0x0d, 0x23, 0xad, 0xaf, + 0x89, 0xf4, 0x41, 0xae, 0x52, 0x36, 0x3d, 0xf5, 0x63, 0x99, 0xfc, 0x2a, 0xaa, 0xe0, 0xa8, 0xa4, + 0x96, 0x8a, 0x62, 0x87, 0xf0, 0x40, 0x01, 0x95, 0xc4, 0xba, 0x8f, 0x66, 0xc7, 0xc0, 0x08, 0x35, + 0x72, 0x52, 0xf7, 0xea, 0x55, 0x74, 0x4e, 0xda, 0x50, 0x76, 0xb5, 0xec, 0x15, 0x14, 0xf6, 0x5d, + 0x68, 0x24, 0xa9, 0x33, 0xe3, 0x94, 0x58, 0xdb, 0x27, 0x6d, 0x4a, 0x04, 0x44, 0xd8, 0x02, 0xaf, + 0x38, 0xbf, 0xf5, 0x0e, 0xe7, 0x5b, 0xff, 0xd5, 0xa1, 0x57, 0xb9, 0x38, 0x56, 0x5d, 0xb0, 0xe5, + 0x89, 0xfa, 0x9a, 0x13, 0x0f, 0xa0, 0x9e, 0x05, 0xbe, 0x08, 0xf6, 0xf6, 0x49, 0x17, 0xe9, 0xaf, + 0x02, 0x3f, 0xc5, 0x5c, 0xb2, 0x89, 0xa2, 0xe8, 0x54, 0x7f, 0x57, 0x42, 0x7c, 0x0f, 0x76, 0xcb, + 0x44, 0x3e, 0x3d, 0x1d, 0x8f, 0x43, 0xf7, 0xfa, 0xd9, 0xa9, 0xf4, 0xde, 0x2a, 0x12, 0x63, 0xe2, + 0xba, 0xa2, 0x82, 0x7c, 0x5a, 0x13, 0x17, 0xd6, 0x67, 0xd0, 0x70, 0xf1, 0x1a, 0x21, 0x2f, 0xc9, + 0x84, 0x52, 0xee, 0x95, 0xa7, 0x35, 0x5b, 0xd0, 0xd9, 0x27, 0x50, 0xf7, 0xb2, 0x45, 0x24, 0x7d, + 0xb5, 0x8d, 0x7c, 0x65, 0x63, 0x7f, 0x5a, 0xb3, 0x89, 0x8a, 0x5c, 0xf3, 0xd0, 0xf1, 0xfa, 0xed, + 0x92, 0xab, 0xec, 0xf7, 0xc8, 0x85, 0x54, 0xe4, 0xc2, 0x0a, 0xa3, 0x6a, 0x93, 0x5c, 0x65, 0xb3, + 0x43, 0x2e, 0xa4, 0x3e, 0x6e, 0x41, 0x33, 0x11, 0x89, 0xfc, 0x13, 0xd8, 0xa9, 0x78, 0x7f, 0xec, + 0x27, 0xe4, 0x2a, 0x41, 0xee, 0x6b, 0xeb, 0x6e, 0xf7, 0x7c, 0xff, 0x10, 0x80, 0x6c, 0x12, 0x17, + 0xae, 0xbc, 0xb5, 0xb5, 0xf2, 0x61, 0xf6, 0x31, 0xb4, 0xd1, 0x96, 0x0d, 0x64, 0x34, 0x62, 0x1d, + 0x39, 0x82, 0x2e, 0x69, 0xff, 0x62, 0xbc, 0x86, 0x83, 0x9d, 0xc0, 0x9e, 0xb8, 0x40, 0x8b, 0xc7, + 0xac, 0x4f, 0xd7, 0x85, 0x28, 0xac, 0x95, 0x34, 0x6c, 0xe8, 0x1c, 0xc5, 0x4d, 0x5e, 0x8c, 0xf3, + 0x5b, 0x2d, 0x87, 0xad, 0x1f, 0x42, 0x1b, 0x4f, 0x14, 0xc7, 0x1d, 0x42, 0x93, 0x08, 0xb9, 0x1f, + 0xcc, 0xc2, 0x9d, 0x52, 0x21, 0x5b, 0xd2, 0xad, 0xdf, 0x6a, 0x95, 0xd7, 0xc9, 0x7b, 0x77, 0xab, + 0x83, 0xe5, 0xc7, 0x4d, 0xbb, 0xfa, 0x96, 0x39, 0x06, 0xa0, 0x86, 0x93, 0xbf, 0x64, 0x8a, 0xf0, + 0x96, 0x58, 0x5b, 0xe1, 0xc0, 0xc0, 0x94, 0xd0, 0x0a, 0xd7, 0xfe, 0x41, 0x87, 0xae, 0xfa, 0x08, + 0xfa, 0xb6, 0xca, 0x8e, 0x29, 0x6f, 0xf5, 0xbc, 0x32, 0x3e, 0xcd, 0x2b, 0xa3, 0x51, 0x9a, 0x51, + 0x66, 0x51, 0x59, 0x18, 0x8f, 0x64, 0x61, 0x34, 0x89, 0xad, 0x97, 0x17, 0x46, 0xce, 0x25, 0xea, + 0xe2, 0x91, 0xac, 0x8b, 0xad, 0x92, 0xa9, 0x48, 0xa9, 0xa2, 0x2c, 0x1e, 0xc9, 0xb2, 0x68, 0x95, + 0x4c, 0x45, 0x98, 0x8b, 0xaa, 0xd8, 0x82, 0x06, 0x85, 0xd3, 0xfa, 0x31, 0x98, 0xaa, 0x6b, 0xa8, + 0x26, 0x3e, 0x95, 0xc4, 0x4a, 0x2a, 0xa8, 0x8f, 0x48, 0xb9, 0xf7, 0x35, 0xf4, 0x2a, 0x4d, 0x05, + 0x6f, 0x3a, 0x3f, 0x19, 0x39, 0x81, 0xcb, 0xe7, 0xc5, 0x13, 0x52, 0xc1, 0x28, 0x49, 0xa6, 0x97, + 0x92, 0xa5, 0x88, 0x4a, 0x92, 0x29, 0x0f, 0x41, 0xa3, 0xf2, 0x10, 0xfc, 0xa7, 0x06, 0x5d, 0x75, + 0x03, 0xbe, 0x25, 0x9f, 0xc4, 0xf1, 0x28, 0xf4, 0x44, 0x34, 0x1b, 0x76, 0x0e, 0x62, 0xea, 0xe3, + 0x72, 0xee, 0x24, 0x89, 0xcc, 0xc0, 0x02, 0x96, 0xb4, 0x89, 0x1b, 0x46, 0xf9, 0x27, 0xa4, 0x80, + 0x25, 0x6d, 0xcc, 0x6f, 0xf8, 0x5c, 0x5e, 0x35, 0x05, 0x8c, 0xa7, 0x9d, 0xf3, 0x24, 0xc1, 0x34, + 0x11, 0x1d, 0x32, 0x07, 0x71, 0x97, 0xed, 0xdc, 0x8e, 0x9c, 0x2c, 0xe1, 0xf2, 0xad, 0x52, 0xc0, + 0xe8, 0x16, 0xfc, 0x2c, 0x39, 0x71, 0x98, 0x05, 0xf9, 0x0b, 0x45, 0xc1, 0x58, 0x03, 0xe8, 0x4f, + 0x6e, 0xfd, 0xd4, 0xbd, 0xa2, 0x2c, 0x16, 0x17, 0x8f, 0xfc, 0x82, 0x59, 0x27, 0xb0, 0x2b, 0x7f, + 0x99, 0x95, 0x3f, 0xf0, 0x77, 0x94, 0x2f, 0x66, 0xa7, 0x28, 0x0d, 0xf1, 0xc7, 0xb4, 0x32, 0xd8, + 0xab, 0xee, 0x91, 0x1f, 0x90, 0x4d, 0x9b, 0xfe, 0x0f, 0x1f, 0xd3, 0x5b, 0xd8, 0xb9, 0xc8, 0xe2, + 0x59, 0x55, 0xd1, 0x01, 0xb4, 0xfc, 0xc0, 0x71, 0x53, 0xff, 0x86, 0xcb, 0x84, 0x28, 0x60, 0xfa, + 0xb1, 0xfa, 0xf2, 0x57, 0x6d, 0xd8, 0xb4, 0x46, 0xfe, 0x4b, 0x7f, 0xce, 0xa9, 0x3c, 0x65, 0x64, + 0x72, 0x98, 0x54, 0x11, 0x8f, 0x04, 0xf9, 0x41, 0x14, 0x10, 0xfa, 0x8f, 0xbe, 0x5b, 0xe2, 0x47, + 0x3a, 0x0a, 0x83, 0x4b, 0x7f, 0x96, 0xfb, 0xef, 0x16, 0x3e, 0x5a, 0x41, 0x7b, 0xc7, 0x8f, 0xac, + 0xb4, 0x59, 0x5f, 0x65, 0xb3, 0x51, 0xf6, 0x64, 0x65, 0x78, 0x51, 0xaf, 0x0e, 0x2f, 0xfe, 0xa6, + 0xc1, 0x40, 0x46, 0x41, 0x9c, 0x3d, 0x71, 0xaf, 0xf8, 0xc2, 0xc9, 0xfd, 0xf2, 0x50, 0x89, 0x05, + 0xf5, 0x12, 0x41, 0x2e, 0xe7, 0x0d, 0xf4, 0x97, 0xd7, 0xcb, 0xbf, 0xfc, 0xda, 0x40, 0x0c, 0xa0, + 0xe5, 0x39, 0xa9, 0x33, 0x75, 0x12, 0x9e, 0xe7, 0x6b, 0x0e, 0xd3, 0xbf, 0xcd, 0x99, 0xce, 0xf3, + 0x6c, 0x15, 0x00, 0x49, 0xa2, 0xd3, 0x64, 0xa6, 0x4a, 0xe8, 0xe8, 0xd7, 0xd0, 0x14, 0x93, 0x0a, + 0xd6, 0x83, 0xf6, 0xb3, 0xe0, 0xc6, 0x99, 0xfb, 0xde, 0xf3, 0xc8, 0xac, 0xb1, 0x16, 0xd4, 0x27, + 0x69, 0x18, 0x99, 0x1a, 0x6b, 0x43, 0xe3, 0x02, 0x73, 0xda, 0xd4, 0x19, 0x40, 0x13, 0xcb, 0x7e, + 0xc1, 0x4d, 0x03, 0xd1, 0x34, 0x04, 0x32, 0xeb, 0x88, 0x16, 0xe3, 0x09, 0xb3, 0xc1, 0xb6, 0x01, + 0x7e, 0x9e, 0xa5, 0xa1, 0x64, 0x6b, 0x1e, 0xbd, 0x26, 0xb6, 0x19, 0x3a, 0xb2, 0x2b, 0xe5, 0x13, + 0x6c, 0xd6, 0xd8, 0x16, 0x18, 0x5f, 0xf1, 0x5b, 0x53, 0x63, 0x1d, 0xd8, 0xb2, 0xb3, 0x00, 0x7f, + 0x83, 0xe2, 0x0c, 0x3a, 0xce, 0x33, 0x0d, 0x24, 0xa0, 0x12, 0x11, 0xf7, 0xcc, 0x3a, 0xeb, 0x42, + 0xeb, 0x17, 0xf2, 0x7b, 0x67, 0x36, 0x90, 0x84, 0x6c, 0xb8, 0xa7, 0x89, 0x24, 0x3a, 0x10, 0xa1, + 0xad, 0xa3, 0xe7, 0xd0, 0xca, 0xbb, 0x34, 0x7b, 0x00, 0x1d, 0x79, 0x2a, 0xa2, 0xcc, 0x1a, 0xaa, + 0x4d, 0xbd, 0xd8, 0xd4, 0xd0, 0x44, 0xec, 0xb7, 0xa6, 0x8e, 0x2b, 0x6c, 0xaa, 0xa6, 0x41, 0x66, + 0xdf, 0x05, 0xae, 0x59, 0x47, 0x46, 0xca, 0x6a, 0xd3, 0x3b, 0x3a, 0x87, 0x2d, 0x59, 0x36, 0x8c, + 0xc1, 0xb6, 0x94, 0x27, 0x31, 0x66, 0x0d, 0x3d, 0x87, 0x5a, 0x0a, 0x6e, 0x0d, 0x3d, 0x40, 0x06, + 0x08, 0x58, 0x47, 0x15, 0x84, 0x37, 0x04, 0xc2, 0x40, 0xfd, 0xf2, 0xc8, 0xb3, 0x5d, 0x78, 0x90, + 0x7b, 0x45, 0xa2, 0x84, 0xc0, 0x33, 0x9e, 0x0a, 0x84, 0xa9, 0x91, 0xfc, 0x02, 0xd4, 0xd1, 0x91, + 0x36, 0x5f, 0x84, 0x37, 0x5c, 0x62, 0x8c, 0x93, 0xff, 0x34, 0xa1, 0x29, 0x12, 0x8e, 0x8d, 0xa0, + 0xab, 0x8e, 0xe6, 0xd8, 0x87, 0xf2, 0x4e, 0xbb, 0x3f, 0xac, 0x1b, 0xf4, 0xe9, 0x56, 0x5a, 0x31, + 0xd5, 0xb1, 0x6a, 0xec, 0x19, 0x6c, 0x57, 0xc7, 0x5c, 0xec, 0x23, 0xe4, 0x5e, 0x39, 0x43, 0x1b, + 0x0c, 0x56, 0x91, 0x0a, 0x51, 0x4f, 0xa0, 0x57, 0x99, 0x5c, 0x31, 0x3a, 0x77, 0xd5, 0x30, 0x6b, + 0xa3, 0x46, 0x3f, 0x83, 0x8e, 0x32, 0x0e, 0x62, 0xfb, 0xc8, 0xba, 0x3c, 0xe5, 0x1a, 0x7c, 0xb8, + 0x84, 0x2f, 0x24, 0x7c, 0x09, 0x50, 0xce, 0x67, 0xd8, 0x07, 0x05, 0xa3, 0x3a, 0xfc, 0x1a, 0xec, + 0xdf, 0x47, 0x17, 0xdb, 0xcf, 0x61, 0x67, 0xa9, 0x5f, 0xb3, 0x87, 0xe4, 0xdc, 0x35, 0x6d, 0x7c, + 0xa3, 0x3d, 0x23, 0xe8, 0xaa, 0xed, 0x5a, 0x84, 0x69, 0x45, 0xd3, 0x17, 0x42, 0x56, 0x75, 0x76, + 0xab, 0xc6, 0x7e, 0x0a, 0x50, 0x36, 0x5f, 0x61, 0xd2, 0x52, 0x33, 0xde, 0xa8, 0xc5, 0x19, 0xec, + 0x28, 0xc3, 0x59, 0xd1, 0x28, 0x85, 0x6f, 0x97, 0x67, 0xb6, 0x1b, 0x05, 0xd9, 0x72, 0x92, 0xa8, + 0x76, 0x5c, 0xe1, 0x9d, 0x75, 0x4d, 0x7a, 0xf0, 0xf1, 0x1a, 0xaa, 0xea, 0x22, 0x75, 0x12, 0x2c, + 0x5c, 0xb4, 0x62, 0x36, 0xbc, 0x51, 0xb1, 0x5f, 0x42, 0x2f, 0x4f, 0x4d, 0x2a, 0x16, 0x36, 0x54, + 0xfc, 0xb9, 0xa2, 0x47, 0x6f, 0x12, 0xf6, 0xb8, 0xff, 0x8f, 0x37, 0x43, 0xed, 0xeb, 0x37, 0x43, + 0xed, 0xdf, 0x6f, 0x86, 0xda, 0xef, 0xde, 0x0e, 0x6b, 0x5f, 0xbf, 0x1d, 0xd6, 0xfe, 0xf5, 0x76, + 0x58, 0x9b, 0x36, 0x69, 0x36, 0xfe, 0xfd, 0xff, 0x05, 0x00, 0x00, 0xff, 0xff, 0xfc, 0x3b, 0x55, + 0x52, 0x2d, 0x17, 0x00, 0x00, } // Reference imports to suppress errors if they are not otherwise used. @@ -3057,7 +2944,6 @@ type WorkerClient interface { UpdateSubTask(ctx context.Context, in *UpdateSubTaskRequest, opts ...grpc.CallOption) (*CommonWorkerResponse, error) QueryStatus(ctx context.Context, in *QueryStatusRequest, opts ...grpc.CallOption) (*QueryStatusResponse, error) QueryError(ctx context.Context, in *QueryErrorRequest, opts ...grpc.CallOption) (*QueryErrorResponse, error) - HandleSQLs(ctx context.Context, in *HandleSubTaskSQLsRequest, opts ...grpc.CallOption) (*CommonWorkerResponse, error) // SwitchRelayMaster request a dm-worker's relay unit switching to another master server SwitchRelayMaster(ctx context.Context, in *SwitchRelayMasterRequest, opts ...grpc.CallOption) (*CommonWorkerResponse, error) // OperateRelay operates relay unit @@ -3127,15 +3013,6 @@ func (c *workerClient) QueryError(ctx context.Context, in *QueryErrorRequest, op return out, nil } -func (c *workerClient) HandleSQLs(ctx context.Context, in *HandleSubTaskSQLsRequest, opts ...grpc.CallOption) (*CommonWorkerResponse, error) { - out := new(CommonWorkerResponse) - err := c.cc.Invoke(ctx, "/pb.Worker/HandleSQLs", in, out, opts...) - if err != nil { - return nil, err - } - return out, nil -} - func (c *workerClient) SwitchRelayMaster(ctx context.Context, in *SwitchRelayMasterRequest, opts ...grpc.CallOption) (*CommonWorkerResponse, error) { out := new(CommonWorkerResponse) err := c.cc.Invoke(ctx, "/pb.Worker/SwitchRelayMaster", in, out, opts...) @@ -3206,7 +3083,6 @@ type WorkerServer interface { UpdateSubTask(context.Context, *UpdateSubTaskRequest) (*CommonWorkerResponse, error) QueryStatus(context.Context, *QueryStatusRequest) (*QueryStatusResponse, error) QueryError(context.Context, *QueryErrorRequest) (*QueryErrorResponse, error) - HandleSQLs(context.Context, *HandleSubTaskSQLsRequest) (*CommonWorkerResponse, error) // SwitchRelayMaster request a dm-worker's relay unit switching to another master server SwitchRelayMaster(context.Context, *SwitchRelayMasterRequest) (*CommonWorkerResponse, error) // OperateRelay operates relay unit @@ -3242,9 +3118,6 @@ func (*UnimplementedWorkerServer) QueryStatus(ctx context.Context, req *QuerySta 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) SwitchRelayMaster(ctx context.Context, req *SwitchRelayMasterRequest) (*CommonWorkerResponse, error) { return nil, status.Errorf(codes.Unimplemented, "method SwitchRelayMaster not implemented") } @@ -3361,24 +3234,6 @@ func _Worker_QueryError_Handler(srv interface{}, ctx context.Context, dec func(i return interceptor(ctx, in, info, handler) } -func _Worker_HandleSQLs_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { - in := new(HandleSubTaskSQLsRequest) - if err := dec(in); err != nil { - return nil, err - } - if interceptor == nil { - return srv.(WorkerServer).HandleSQLs(ctx, in) - } - info := &grpc.UnaryServerInfo{ - Server: srv, - FullMethod: "/pb.Worker/HandleSQLs", - } - handler := func(ctx context.Context, req interface{}) (interface{}, error) { - return srv.(WorkerServer).HandleSQLs(ctx, req.(*HandleSubTaskSQLsRequest)) - } - return interceptor(ctx, in, info, handler) -} - func _Worker_SwitchRelayMaster_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { in := new(SwitchRelayMasterRequest) if err := dec(in); err != nil { @@ -3529,10 +3384,6 @@ var _Worker_serviceDesc = grpc.ServiceDesc{ MethodName: "QueryError", Handler: _Worker_QueryError_Handler, }, - { - MethodName: "HandleSQLs", - Handler: _Worker_HandleSQLs_Handler, - }, { MethodName: "SwitchRelayMaster", Handler: _Worker_SwitchRelayMaster_Handler, @@ -3838,64 +3689,6 @@ func (m *QueryErrorRequest) MarshalToSizedBuffer(dAtA []byte) (int, error) { return len(dAtA) - i, nil } -func (m *HandleSubTaskSQLsRequest) 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 *HandleSubTaskSQLsRequest) MarshalTo(dAtA []byte) (int, error) { - 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.SqlPattern) > 0 { - i -= len(m.SqlPattern) - copy(dAtA[i:], m.SqlPattern) - i = encodeVarintDmworker(dAtA, i, uint64(len(m.SqlPattern))) - i-- - dAtA[i] = 0x2a - } - 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 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 - } - } - if m.Op != 0 { - i = encodeVarintDmworker(dAtA, i, uint64(m.Op)) - 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 *CommonWorkerResponse) Marshal() (dAtA []byte, err error) { size := m.Size() dAtA = make([]byte, size) @@ -5722,36 +5515,6 @@ func (m *QueryErrorRequest) Size() (n int) { return n } -func (m *HandleSubTaskSQLsRequest) Size() (n int) { - if m == nil { - return 0 - } - var l int - _ = l - l = len(m.Name) - if l > 0 { - n += 1 + l + sovDmworker(uint64(l)) - } - if m.Op != 0 { - n += 1 + sovDmworker(uint64(m.Op)) - } - if len(m.Args) > 0 { - for _, s := range m.Args { - l = len(s) - n += 1 + l + sovDmworker(uint64(l)) - } - } - l = len(m.BinlogPos) - if l > 0 { - n += 1 + l + sovDmworker(uint64(l)) - } - l = len(m.SqlPattern) - if l > 0 { - n += 1 + l + sovDmworker(uint64(l)) - } - return n -} - func (m *CommonWorkerResponse) Size() (n int) { if m == nil { return 0 @@ -7336,206 +7099,6 @@ func (m *QueryErrorRequest) Unmarshal(dAtA []byte) error { } return nil } -func (m *HandleSubTaskSQLsRequest) 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: HandleSubTaskSQLsRequest: wiretype end group for non-group") - } - if fieldNum <= 0 { - return fmt.Errorf("proto: HandleSubTaskSQLsRequest: 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) - } - 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.Name = string(dAtA[iNdEx:postIndex]) - iNdEx = postIndex - case 2: - 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 |= SQLOp(b&0x7F) << shift - if b < 0x80 { - break - } - } - case 3: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Args", 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.Args = append(m.Args, string(dAtA[iNdEx:postIndex])) - iNdEx = postIndex - case 4: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field BinlogPos", 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.BinlogPos = string(dAtA[iNdEx:postIndex]) - iNdEx = postIndex - case 5: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field SqlPattern", 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.SqlPattern = 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 *CommonWorkerResponse) Unmarshal(dAtA []byte) error { l := len(dAtA) iNdEx := 0 diff --git a/dm/pbmock/dmmaster.go b/dm/pbmock/dmmaster.go index 8744d39f42..4e3a0e4250 100644 --- a/dm/pbmock/dmmaster.go +++ b/dm/pbmock/dmmaster.go @@ -95,26 +95,6 @@ func (mr *MockMasterClientMockRecorder) GetTaskCfg(arg0, arg1 interface{}, arg2 return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "GetTaskCfg", reflect.TypeOf((*MockMasterClient)(nil).GetTaskCfg), varargs...) } -// HandleSQLs mocks base method -func (m *MockMasterClient) HandleSQLs(arg0 context.Context, arg1 *pb.HandleSQLsRequest, arg2 ...grpc.CallOption) (*pb.HandleSQLsResponse, error) { - m.ctrl.T.Helper() - varargs := []interface{}{arg0, arg1} - for _, a := range arg2 { - varargs = append(varargs, a) - } - ret := m.ctrl.Call(m, "HandleSQLs", varargs...) - ret0, _ := ret[0].(*pb.HandleSQLsResponse) - ret1, _ := ret[1].(error) - return ret0, ret1 -} - -// HandleSQLs indicates an expected call of HandleSQLs -func (mr *MockMasterClientMockRecorder) HandleSQLs(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, "HandleSQLs", reflect.TypeOf((*MockMasterClient)(nil).HandleSQLs), varargs...) -} - // ListMember mocks base method func (m *MockMasterClient) ListMember(arg0 context.Context, arg1 *pb.ListMemberRequest, arg2 ...grpc.CallOption) (*pb.ListMemberResponse, error) { m.ctrl.T.Helper() @@ -563,21 +543,6 @@ func (mr *MockMasterServerMockRecorder) GetTaskCfg(arg0, arg1 interface{}) *gomo return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "GetTaskCfg", reflect.TypeOf((*MockMasterServer)(nil).GetTaskCfg), arg0, arg1) } -// HandleSQLs mocks base method -func (m *MockMasterServer) HandleSQLs(arg0 context.Context, arg1 *pb.HandleSQLsRequest) (*pb.HandleSQLsResponse, error) { - m.ctrl.T.Helper() - ret := m.ctrl.Call(m, "HandleSQLs", arg0, arg1) - ret0, _ := ret[0].(*pb.HandleSQLsResponse) - ret1, _ := ret[1].(error) - return ret0, ret1 -} - -// HandleSQLs indicates an expected call of HandleSQLs -func (mr *MockMasterServerMockRecorder) HandleSQLs(arg0, arg1 interface{}) *gomock.Call { - mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "HandleSQLs", reflect.TypeOf((*MockMasterServer)(nil).HandleSQLs), arg0, arg1) -} - // ListMember mocks base method func (m *MockMasterServer) ListMember(arg0 context.Context, arg1 *pb.ListMemberRequest) (*pb.ListMemberResponse, error) { m.ctrl.T.Helper() diff --git a/dm/pbmock/dmworker.go b/dm/pbmock/dmworker.go index 0522243f70..ae1114faf5 100644 --- a/dm/pbmock/dmworker.go +++ b/dm/pbmock/dmworker.go @@ -35,26 +35,6 @@ func (m *MockWorkerClient) EXPECT() *MockWorkerClientMockRecorder { return m.recorder } -// HandleSQLs mocks base method -func (m *MockWorkerClient) HandleSQLs(arg0 context.Context, arg1 *pb.HandleSubTaskSQLsRequest, arg2 ...grpc.CallOption) (*pb.CommonWorkerResponse, error) { - m.ctrl.T.Helper() - varargs := []interface{}{arg0, arg1} - for _, a := range arg2 { - varargs = append(varargs, a) - } - ret := m.ctrl.Call(m, "HandleSQLs", varargs...) - ret0, _ := ret[0].(*pb.CommonWorkerResponse) - ret1, _ := ret[1].(error) - return ret0, ret1 -} - -// HandleSQLs indicates an expected call of HandleSQLs -func (mr *MockWorkerClientMockRecorder) HandleSQLs(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, "HandleSQLs", reflect.TypeOf((*MockWorkerClient)(nil).HandleSQLs), varargs...) -} - // MigrateRelay mocks base method func (m *MockWorkerClient) MigrateRelay(arg0 context.Context, arg1 *pb.MigrateRelayRequest, arg2 ...grpc.CallOption) (*pb.CommonWorkerResponse, error) { m.ctrl.T.Helper() @@ -318,21 +298,6 @@ func (m *MockWorkerServer) EXPECT() *MockWorkerServerMockRecorder { return m.recorder } -// HandleSQLs mocks base method -func (m *MockWorkerServer) HandleSQLs(arg0 context.Context, arg1 *pb.HandleSubTaskSQLsRequest) (*pb.CommonWorkerResponse, error) { - m.ctrl.T.Helper() - ret := m.ctrl.Call(m, "HandleSQLs", arg0, arg1) - ret0, _ := ret[0].(*pb.CommonWorkerResponse) - ret1, _ := ret[1].(error) - return ret0, ret1 -} - -// HandleSQLs indicates an expected call of HandleSQLs -func (mr *MockWorkerServerMockRecorder) HandleSQLs(arg0, arg1 interface{}) *gomock.Call { - mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "HandleSQLs", reflect.TypeOf((*MockWorkerServer)(nil).HandleSQLs), arg0, arg1) -} - // MigrateRelay mocks base method func (m *MockWorkerServer) MigrateRelay(arg0 context.Context, arg1 *pb.MigrateRelayRequest) (*pb.CommonWorkerResponse, error) { m.ctrl.T.Helper() diff --git a/dm/proto/dmmaster.proto b/dm/proto/dmmaster.proto index 5862f7360c..23c93d0e00 100644 --- a/dm/proto/dmmaster.proto +++ b/dm/proto/dmmaster.proto @@ -37,8 +37,6 @@ service Master { rpc UpdateWorkerRelayConfig(UpdateWorkerRelayConfigRequest) returns (CommonWorkerResponse) {} - rpc HandleSQLs (HandleSQLsRequest) returns (HandleSQLsResponse) {} - // SwitchWorkerRelayMaster requests some dm-workers to switch relay unit's master server rpc SwitchWorkerRelayMaster(SwitchWorkerRelayMasterRequest) returns (SwitchWorkerRelayMasterResponse) {} @@ -271,22 +269,6 @@ message OperateWorkerRelayResponse { repeated CommonWorkerResponse sources = 4; } -message HandleSQLsRequest { - string name = 1; // sub task's name - SQLOp op = 2; // operation type - repeated string args = 3; // args for operation - string binlogPos = 4; // binlog-pos (that's file:pos format) - string source = 5; // source - string sqlPattern = 6; // sql-pattern (for regexp, starts with ~) - bool sharding = 7; // whether are handing sharding DDL, which will only take effect on DDL lock's owner -} - -message HandleSQLsResponse { - bool result = 1; - string msg = 2; - repeated CommonWorkerResponse sources = 3; -} - // PurgeWorkerRelayRequest represents a request to purge relay log files for some dm-workers // workers: dm-workers need to purge relay log files // inactive: whether purge inactive relay log files diff --git a/dm/proto/dmworker.proto b/dm/proto/dmworker.proto index d0bd70642d..25b6c584e8 100644 --- a/dm/proto/dmworker.proto +++ b/dm/proto/dmworker.proto @@ -11,8 +11,6 @@ service Worker { rpc QueryStatus (QueryStatusRequest) returns (QueryStatusResponse) {} rpc QueryError (QueryErrorRequest) returns (QueryErrorResponse) {} - rpc HandleSQLs (HandleSubTaskSQLsRequest) returns (CommonWorkerResponse) {} - // SwitchRelayMaster request a dm-worker's relay unit switching to another master server rpc SwitchRelayMaster (SwitchRelayMasterRequest) returns (CommonWorkerResponse) {} @@ -82,21 +80,6 @@ message QueryErrorRequest { string name = 1; // sub task's name, empty for all sub tasks } -enum SQLOp { - SKIP = 0; // skip the SQL - REPLACE = 1; // replace the SQL with a specified SQL - INJECT = 2; // inject the SQL to syncer as binlog event -} - -// NOTE: we can extract some common(optional) arguments from `args`, like `meta` -message HandleSubTaskSQLsRequest { - string name = 1; // sub task's name - SQLOp op = 2; // operation type - repeated string args = 3; // args for operation - string binlogPos = 4; // binlog-pos (that's file:pos format) - string sqlPattern = 5; // sql-pattern (for regexp, starts with ~) -} - message CommonWorkerResponse { bool result = 1; string msg = 2; // when result is true, msg is empty diff --git a/dm/worker/server.go b/dm/worker/server.go index 25b6c2ce58..7eb85915e2 100644 --- a/dm/worker/server.go +++ b/dm/worker/server.go @@ -562,23 +562,6 @@ func (s *Server) QueryError(ctx context.Context, req *pb.QueryErrorRequest) (*pb return resp, nil } -// HandleSQLs implements WorkerServer.HandleSQLs -func (s *Server) HandleSQLs(ctx context.Context, req *pb.HandleSubTaskSQLsRequest) (*pb.CommonWorkerResponse, error) { - log.L().Info("", zap.String("request", "HandleSQLs"), zap.Stringer("payload", req)) - w := s.getWorker(true) - if w == nil { - log.L().Error("fail to call StartSubTask, because mysql worker has not been started") - return makeCommonWorkerResponse(terror.ErrWorkerNoStart.Generate()), nil - } - - err := w.HandleSQLs(ctx, req) - if err != nil { - log.L().Error("fail to handle sqls", zap.String("request", "HandleSQLs"), zap.Stringer("payload", req), zap.Error(err)) - } - // TODO: check whether this interface need to store message in ETCD - return makeCommonWorkerResponse(err), nil -} - // 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)) diff --git a/dm/worker/subtask.go b/dm/worker/subtask.go index 8046de8878..80610a7d12 100644 --- a/dm/worker/subtask.go +++ b/dm/worker/subtask.go @@ -507,21 +507,6 @@ func (st *SubTask) Update(cfg *config.SubTaskConfig) error { return nil } -// SetSyncerSQLOperator sets an operator to syncer. -func (st *SubTask) SetSyncerSQLOperator(ctx context.Context, req *pb.HandleSubTaskSQLsRequest) error { - syncUnit, ok := st.currUnit.(*syncer.Syncer) - if !ok { - return terror.ErrWorkerOperSyncUnitOnly.Generate(st.currUnit.Type()) - } - - // special handle for INJECT - if req.Op == pb.SQLOp_INJECT { - return syncUnit.InjectSQLs(ctx, req.Args) - } - - return syncUnit.SetSQLOperator(req) -} - // OperateSchema operates schema for an upstream table. func (st *SubTask) OperateSchema(ctx context.Context, req *pb.OperateWorkerSchemaRequest) (schema string, err error) { if st.Stage() != pb.Stage_Paused { diff --git a/dm/worker/worker.go b/dm/worker/worker.go index 2296488fc6..4b39b96e8b 100644 --- a/dm/worker/worker.go +++ b/dm/worker/worker.go @@ -562,20 +562,6 @@ func (w *Worker) operateRelayStage(ctx context.Context, stage ha.Stage) (string, return op.String(), w.OperateRelay(ctx, &pb.OperateRelayRequest{Op: op}) } -// HandleSQLs implements Handler.HandleSQLs. -func (w *Worker) HandleSQLs(ctx context.Context, req *pb.HandleSubTaskSQLsRequest) error { - if w.closed.Get() == closedTrue { - return terror.ErrWorkerAlreadyClosed.Generate() - } - - st := w.subTaskHolder.findSubTask(req.Name) - if st == nil { - return terror.ErrWorkerSubTaskNotFound.Generate(req.Name) - } - - return st.SetSyncerSQLOperator(ctx, req) -} - // SwitchRelayMaster switches relay unit's master server func (w *Worker) SwitchRelayMaster(ctx context.Context, req *pb.SwitchRelayMasterRequest) error { if w.closed.Get() == closedTrue { diff --git a/errors.toml b/errors.toml index c6f2248034..f830c3159d 100644 --- a/errors.toml +++ b/errors.toml @@ -1558,30 +1558,6 @@ description = "" workaround = "It is recommended to include only one DDL operation in a statement executed upstream. Please manually handle it using dmctl (skipping the DDL statement or replacing the DDL statement with a specified DDL statement). For details, see https://docs.pingcap.com/tidb-data-migration/stable/skip-or-replace-abnormal-sql-statements" tags = ["internal", "high"] -[error.DM-sync-unit-36023] -message = "only support inject DDL for sharding group to be synced currently, but got %s" -description = "" -workaround = "" -tags = ["internal", "low"] - -[error.DM-sync-unit-36024] -message = "injected DDL %s without schema name not valid" -description = "" -workaround = "" -tags = ["internal", "low"] - -[error.DM-sync-unit-36025] -message = "op %s not supported" -description = "" -workaround = "" -tags = ["internal", "medium"] - -[error.DM-sync-unit-36026] -message = "nil request not valid" -description = "" -workaround = "" -tags = ["internal", "medium"] - [error.DM-sync-unit-36027] message = "Column count doesn't match value count: %d (columns) vs %d (values)" description = "" @@ -1804,24 +1780,6 @@ description = "" workaround = "" tags = ["internal", "low"] -[error.DM-dm-master-38001] -message = "nil request not valid" -description = "" -workaround = "" -tags = ["internal", "medium"] - -[error.DM-dm-master-38002] -message = "op %s not supported" -description = "" -workaround = "" -tags = ["internal", "medium"] - -[error.DM-dm-master-38003] -message = "operate request without --sharding specified not valid" -description = "" -workaround = "" -tags = ["internal", "medium"] - [error.DM-dm-master-38004] message = "create grpc connection" description = "" @@ -1936,12 +1894,6 @@ description = "" workaround = "" tags = ["internal", "high"] -[error.DM-dm-master-38023] -message = "request DDL lock %s owner %s handle SQLs request %s fail %s" -description = "" -workaround = "" -tags = ["internal", "high"] - [error.DM-dm-master-38024] message = "owner %s ExecuteDDL fail" description = "" diff --git a/pkg/terror/error_list.go b/pkg/terror/error_list.go index c0044c0ce9..421d8f4281 100644 --- a/pkg/terror/error_list.go +++ b/pkg/terror/error_list.go @@ -882,10 +882,6 @@ var ( ErrSyncerUnitDDLChanDone = New(codeSyncerUnitDDLChanDone, ClassSyncUnit, ScopeInternal, LevelHigh, "canceled from external", "") ErrSyncerUnitDDLChanCanceled = New(codeSyncerUnitDDLChanCanceled, ClassSyncUnit, ScopeInternal, LevelHigh, "canceled by Close or Renew", "") ErrSyncerUnitDDLOnMultipleTable = New(codeSyncerUnitDDLOnMultipleTable, ClassSyncUnit, ScopeInternal, LevelHigh, "ddl on multiple table: %s not supported", "It is recommended to include only one DDL operation in a statement executed upstream. Please manually handle it using dmctl (skipping the DDL statement or replacing the DDL statement with a specified DDL statement). For details, see https://docs.pingcap.com/tidb-data-migration/stable/skip-or-replace-abnormal-sql-statements") - ErrSyncerUnitInjectDDLOnly = New(codeSyncerUnitInjectDDLOnly, ClassSyncUnit, ScopeInternal, LevelLow, "only support inject DDL for sharding group to be synced currently, but got %s", "") - ErrSyncerUnitInjectDDLWithoutSchema = New(codeSyncerUnitInjectDDLWithoutSchema, ClassSyncUnit, ScopeInternal, LevelLow, "injected DDL %s without schema name not valid", "") - ErrSyncerUnitNotSupportedOperate = New(codeSyncerUnitNotSupportedOperate, ClassSyncUnit, ScopeInternal, LevelMedium, "op %s not supported", "") - ErrSyncerUnitNilOperatorReq = New(codeSyncerUnitNilOperatorReq, ClassSyncUnit, ScopeInternal, LevelMedium, "nil request not valid", "") ErrSyncerUnitDMLColumnNotMatch = New(codeSyncerUnitDMLColumnNotMatch, ClassSyncUnit, ScopeInternal, LevelHigh, "Column count doesn't match value count: %d (columns) vs %d (values)", "") ErrSyncerUnitDMLOldNewValueMismatch = New(codeSyncerUnitDMLOldNewValueMismatch, ClassSyncUnit, ScopeInternal, LevelHigh, "Old value count doesn't match new value count: %d (old) vs %d (new)", "") ErrSyncerUnitDMLPruneColumnMismatch = New(codeSyncerUnitDMLPruneColumnMismatch, ClassSyncUnit, ScopeInternal, LevelHigh, "prune DML columns and data mismatch in length: %d (columns) %d (data)", "") @@ -925,9 +921,6 @@ var ( ErrSyncerFailpoint = New(codeSyncerFailpoint, ClassSyncUnit, ScopeInternal, LevelLow, "failpoint specified error", "") // DM-master error - ErrMasterSQLOpNilRequest = New(codeMasterSQLOpNilRequest, ClassDMMaster, ScopeInternal, LevelMedium, "nil request not valid", "") - ErrMasterSQLOpNotSupport = New(codeMasterSQLOpNotSupport, ClassDMMaster, ScopeInternal, LevelMedium, "op %s not supported", "") - ErrMasterSQLOpWithoutSharding = New(codeMasterSQLOpWithoutSharding, ClassDMMaster, ScopeInternal, LevelMedium, "operate request without --sharding specified not valid", "") ErrMasterGRPCCreateConn = New(codeMasterGRPCCreateConn, ClassDMMaster, ScopeInternal, LevelHigh, "create grpc connection", "") ErrMasterGRPCSendOnCloseConn = New(codeMasterGRPCSendOnCloseConn, ClassDMMaster, ScopeInternal, LevelHigh, "send request on a closed client", "") ErrMasterGRPCClientClose = New(codeMasterGRPCClientClose, ClassDMMaster, ScopeInternal, LevelHigh, "close rpc client", "") @@ -947,7 +940,6 @@ var ( ErrMasterLockIsResolving = New(codeMasterLockIsResolving, ClassDMMaster, ScopeInternal, LevelHigh, "lock %s is resolving", "") ErrMasterWorkerCliNotFound = New(codeMasterWorkerCliNotFound, ClassDMMaster, ScopeInternal, LevelHigh, "source %s relevant worker-client not found", "") ErrMasterWorkerNotWaitLock = New(codeMasterWorkerNotWaitLock, ClassDMMaster, ScopeInternal, LevelHigh, "worker %s not waiting for DDL lock %s", "") - ErrMasterHandleSQLReqFail = New(codeMasterHandleSQLReqFail, ClassDMMaster, ScopeInternal, LevelHigh, "request DDL lock %s owner %s handle SQLs request %s fail %s", "") ErrMasterOwnerExecDDL = New(codeMasterOwnerExecDDL, ClassDMMaster, ScopeInternal, LevelHigh, "owner %s ExecuteDDL fail", "") ErrMasterPartWorkerExecDDLFail = New(codeMasterPartWorkerExecDDLFail, ClassDMMaster, ScopeInternal, LevelHigh, "DDL lock %s owner ExecuteDDL successfully, so DDL lock removed. but some dm-workers ExecuteDDL fail, you should to handle dm-worker directly", "") ErrMasterWorkerExistDDLLock = New(codeMasterWorkerExistDDLLock, ClassDMMaster, ScopeInternal, LevelHigh, "worker %s exist ddl lock", "Please unlock ddl lock first.") diff --git a/syncer/inject_sql.go b/syncer/inject_sql.go deleted file mode 100644 index db29203df6..0000000000 --- a/syncer/inject_sql.go +++ /dev/null @@ -1,108 +0,0 @@ -// Copyright 2019 PingCAP, Inc. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// See the License for the specific language governing permissions and -// limitations under the License. - -package syncer - -import ( - "context" - "time" - - "github.com/pingcap/parser" - "github.com/pingcap/parser/ast" - "github.com/siddontang/go-mysql/replication" - "go.uber.org/zap" - - "github.com/pingcap/dm/pkg/binlog" - parserpkg "github.com/pingcap/dm/pkg/parser" - "github.com/pingcap/dm/pkg/terror" -) - -// InjectSQLs injects ddl into syncer as binlog events while meet xid/query event -// TODO: let user to specify special xid/query event position -// TODO: inject dml sqls -func (s *Syncer) InjectSQLs(ctx context.Context, sqls []string) error { - // verify and fetch schema name - schemas := make([]string, 0, len(sqls)) - parser2 := parser.New() - for _, sql := range sqls { - node, err := parser2.ParseOneStmt(sql, "", "") - if err != nil { - return terror.Annotatef(terror.ErrSyncerUnitParseStmt.New(err.Error()), "sql %s", sql) - } - ddlNode, ok := node.(ast.DDLNode) - if !ok { - return terror.ErrSyncerUnitInjectDDLOnly.Generate(sql) - } - tableNames, err := parserpkg.FetchDDLTableNames("", ddlNode) - if err != nil { - return err - } - if len(tableNames[0].Schema) == 0 { - return terror.ErrSyncerUnitInjectDDLWithoutSchema.Generate(sql) - } - schemas = append(schemas, tableNames[0].Schema) - } - - for i, sql := range sqls { - schema := schemas[i] - ev := genIncompleteQueryEvent([]byte(schema), []byte(sql)) - newCtx, cancel := context.WithTimeout(ctx, 30*time.Second) - s.tctx.L().Info("injecting sql", zap.String("sql", sql), zap.String("schema", schema)) - - select { - case s.injectEventCh <- ev: - case <-newCtx.Done(): - cancel() - return newCtx.Err() - } - cancel() - } - return nil -} - -func (s *Syncer) tryInject(op opType, location binlog.Location) *replication.BinlogEvent { - if op != xid && op != ddl { - return nil - } - - select { - case e := <-s.injectEventCh: - // try receive from extra binlog event chan - // NOTE: now we simply set EventSize to 0, make event's start / end pos are the same - // TODO: support GTID - e.Header.LogPos = location.Position.Pos - e.Header.EventSize = 0 - s.tctx.L().Info("inject binlog event from inject chan", zap.Reflect("header", e.Header), zap.Reflect("event", e.Event)) - return e - default: - return nil - } -} - -// generates an incomplete QueryEvent, only partial fields are valid -// now, it only used to generate QueryEvent to force sharding group to be synced -// NOTE: using only if you know want your are doing -func genIncompleteQueryEvent(schema, query []byte) *replication.BinlogEvent { - header := &replication.EventHeader{ - EventType: replication.QUERY_EVENT, - } - queryEvent := &replication.QueryEvent{ - Schema: schema, - Query: query, - } - e := &replication.BinlogEvent{ - Header: header, - Event: queryEvent, - } - return e -} diff --git a/syncer/operator.go b/syncer/operator.go deleted file mode 100644 index d6ba766325..0000000000 --- a/syncer/operator.go +++ /dev/null @@ -1,31 +0,0 @@ -// Copyright 2019 PingCAP, Inc. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// See the License for the specific language governing permissions and -// limitations under the License. - -package syncer - -import ( - "github.com/pingcap/dm/dm/pb" - "github.com/pingcap/dm/pkg/binlog" -) - -// SetSQLOperator sets an SQL operator to syncer -func (s *Syncer) SetSQLOperator(req *pb.HandleSubTaskSQLsRequest) error { - return s.sqlOperatorHolder.Set(s.tctx, req) -} - -// tryApplySQLOperator tries to get SQLs by applying an possible operator -// return whether applied, and the applied SQLs -func (s *Syncer) tryApplySQLOperator(location binlog.Location, sqls []string) (bool, []string, error) { - // TODO: support GTID - return s.sqlOperatorHolder.Apply(s.tctx, location.Position, sqls) -} diff --git a/syncer/optimist.go b/syncer/optimist.go index 0e4ef9f3e7..fc32baf4d0 100644 --- a/syncer/optimist.go +++ b/syncer/optimist.go @@ -183,18 +183,6 @@ func (s *Syncer) handleQueryEventOptimistic( s.tctx.L().Info("start to handle ddls in optimistic shard mode", zap.String("event", "query"), zap.Strings("ddls", needHandleDDLs), zap.ByteString("raw statement", ev.Query), log.WrapStringerField("location", ec.currentLocation)) - // try apply SQL operator before addJob. now, one query event only has one DDL job, if updating to multi DDL jobs, refine this. - applied, appliedSQLs, err := s.tryApplySQLOperator(ec.currentLocation.Clone(), needHandleDDLs) - if err != nil { - return terror.Annotatef(err, "try apply SQL operator on binlog-location %s with DDLs %v", ec.currentLocation, needHandleDDLs) - } - if applied { - s.tctx.L().Info("replace ddls to preset ddls by sql operator in shard mode", zap.String("event", "query"), - zap.Strings("preset ddls", appliedSQLs), zap.Strings("ddls", needHandleDDLs), zap.ByteString("raw statement", ev.Query), - log.WrapStringerField("location", ec.currentLocation)) - needHandleDDLs = appliedSQLs // maybe nil - } - // interrupted after track DDL and before execute DDL. failpoint.Inject("FlushCheckpointStage", func(val failpoint.Value) { err = handleFlushCheckpointStage(2, val.(int), "before execute DDL") diff --git a/syncer/sql-operator/operator.go b/syncer/sql-operator/operator.go deleted file mode 100644 index 50bdb8764e..0000000000 --- a/syncer/sql-operator/operator.go +++ /dev/null @@ -1,175 +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 operator - -import ( - "fmt" - "regexp" - "strings" - "sync" - - uuid "github.com/satori/go.uuid" - "github.com/siddontang/go-mysql/mysql" - "go.uber.org/zap" - - "github.com/pingcap/dm/dm/command" - "github.com/pingcap/dm/dm/pb" - tcontext "github.com/pingcap/dm/pkg/context" - "github.com/pingcap/dm/pkg/log" - "github.com/pingcap/dm/pkg/terror" -) - -// operator contains an operation for specified binlog pos or SQL pattern -// used by `sql-skip` and `sql-replace` -type operator struct { - uuid string // add a UUID, make it more friendly to be traced in log - pos *mysql.Position - pattern string - reg *regexp.Regexp - op pb.SQLOp - args []string // if op == SQLOp_REPLACE, it has arguments -} - -// newOperator creates a new Operator with a random UUID -func newOperator(tctx *tcontext.Context, pos *mysql.Position, pattern string, reg *regexp.Regexp, op pb.SQLOp, args []string) *operator { - switch op { - case pb.SQLOp_SKIP: - if len(args) > 0 { - tctx.L().Warn("ignore operation", zap.Stringer("operation", op), zap.Strings("arguments", args)) - args = nil - } - } - - return &operator{ - uuid: uuid.NewV4().String(), - pos: pos, - pattern: pattern, - reg: reg, - op: op, - args: args, - } -} - -// operate do the operation to return the args -func (o *operator) operate() ([]string, error) { - switch o.op { - case pb.SQLOp_SKIP: - return nil, nil - case pb.SQLOp_REPLACE: - return o.args, nil - default: - return nil, terror.ErrSyncerUnitNotSupportedOperate.Generate(o.op) - } -} - -// matchPattern tries to match SQL with the regexp -func (o *operator) matchPattern(sql string) bool { - if o.reg == nil { - return false - } - return o.reg.MatchString(sql) -} - -func (o *operator) String() string { - if len(o.pattern) > 0 { - return fmt.Sprintf("uuid: %s, pattern: %s, op: %s, args: %s", o.uuid, o.pattern, o.op, strings.Join(o.args, " ")) - } - return fmt.Sprintf("uuid: %s, pos: %s, op: %s, args: %s", o.uuid, o.pos, o.op, strings.Join(o.args, " ")) -} - -// Holder holds SQL operators -type Holder struct { - mu sync.Mutex - operators map[string]*operator -} - -// NewHolder creates a new Holder -func NewHolder() *Holder { - return &Holder{ - operators: make(map[string]*operator), - } -} - -// Set sets an operator according request -func (h *Holder) Set(tctx *tcontext.Context, req *pb.HandleSubTaskSQLsRequest) error { - if req == nil { - return terror.ErrSyncerUnitNilOperatorReq.Generate() - } - switch req.Op { - case pb.SQLOp_SKIP, pb.SQLOp_REPLACE: - default: - return terror.ErrSyncerUnitNotSupportedOperate.Generate(req.Op) - } - - binlogPos, sqlReg, err := command.VerifySQLOperateArgs(req.BinlogPos, req.SqlPattern, false) // sharding only be used in DM-master - if err != nil { - return err - } - - var key string - if binlogPos != nil { - key = binlogPos.String() - } else if sqlReg != nil { - key = req.SqlPattern // use sql-pattern as the key - } - - h.mu.Lock() - defer h.mu.Unlock() - - oper := newOperator(tctx, binlogPos, req.SqlPattern, sqlReg, req.Op, req.Args) - prev, ok := h.operators[key] - if ok { - tctx.L().Warn("overwrite operator", log.WrapStringerField("old operator", prev), log.WrapStringerField("new operator", oper)) - } - h.operators[key] = oper - tctx.L().Info("set a new operator", log.WrapStringerField("new operator", oper)) - return nil -} - -// Apply tries to apply operator by pos or SQLs, returns applied, args, error -func (h *Holder) Apply(tctx *tcontext.Context, pos mysql.Position, sqls []string) (bool, []string, error) { - h.mu.Lock() - defer h.mu.Unlock() - - var cause string - key := pos.String() - oper, ok := h.operators[key] - if !ok { - OUTER_FOR: - for _, sql := range sqls { - for key2, oper2 := range h.operators { - if oper2.matchPattern(sql) { // matched one SQL of all is enough - key = key2 - oper = oper2 - cause = fmt.Sprintf("sql-pattern %s matched SQL %s", key, sql) - break OUTER_FOR - } - } - } - if oper == nil { - return false, nil, nil - } - } else { - cause = fmt.Sprintf("binlog-pos %s matched", pos) - } - - delete(h.operators, key) // always delete the operator - args, err := oper.operate() - if err != nil { - return false, nil, terror.Annotatef(err, "operator %s", oper) - } - - tctx.L().Info("applying operator", zap.String("chance", cause), log.WrapStringerField("operation", oper)) - return true, args, nil -} diff --git a/syncer/sql-operator/operator_test.go b/syncer/sql-operator/operator_test.go deleted file mode 100644 index 37f885d40d..0000000000 --- a/syncer/sql-operator/operator_test.go +++ /dev/null @@ -1,220 +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 operator - -import ( - "testing" - - . "github.com/pingcap/check" - "github.com/siddontang/go-mysql/mysql" - - "github.com/pingcap/dm/dm/pb" - tcontext "github.com/pingcap/dm/pkg/context" -) - -var _ = Suite(&testOperatorSuite{}) - -func TestSuite(t *testing.T) { - TestingT(t) -} - -type testOperatorSuite struct { -} - -func (o *testOperatorSuite) TestOperatorSet(c *C) { - reqPos := &pb.HandleSubTaskSQLsRequest{ - Op: pb.SQLOp_SKIP, - BinlogPos: "mysql-bin.000001:234", - } - reqPattern := &pb.HandleSubTaskSQLsRequest{ - Op: pb.SQLOp_SKIP, - SqlPattern: "~(?i)ALTER\\s+TABLE\\s+`db1`.`tbl1`\\s+ADD\\s+COLUMN\\s+col1\\s+INT", - } - sql := "ALTER TABLE `db1`.`tbl1` ADD COLUMN col1 INT" - - tctx := tcontext.Background() - - h := NewHolder() - - // nil request - err := h.Set(tctx, nil) - c.Assert(err, NotNil) - - // not supported op - err = h.Set(tctx, &pb.HandleSubTaskSQLsRequest{Op: pb.SQLOp_INJECT}) - c.Assert(err, NotNil) - - // none of binlog-pos, sql-pattern set - err = h.Set(tctx, &pb.HandleSubTaskSQLsRequest{Op: pb.SQLOp_SKIP}) - c.Assert(err, NotNil) - - // both binlog-pos, sql-pattern set - err = h.Set(tctx, &pb.HandleSubTaskSQLsRequest{ - Op: pb.SQLOp_SKIP, - BinlogPos: reqPos.BinlogPos, - SqlPattern: reqPattern.SqlPattern, - }) - c.Assert(err, NotNil) - - // no operator set, apply got nothing - applied, args, err := h.Apply(tctx, mysql.Position{}, []string{sql}) - c.Assert(err, IsNil) - c.Assert(applied, IsFalse) - c.Assert(args, IsNil) -} - -func (o *testOperatorSuite) TestBinlogPos(c *C) { - cases := []struct { - req *pb.HandleSubTaskSQLsRequest - pos mysql.Position - }{ - { - req: &pb.HandleSubTaskSQLsRequest{ - Op: pb.SQLOp_SKIP, - BinlogPos: "mysql-bin.000001:234", - Args: []string{"CREATE DATABASE shard_db_1;", "CREATE TABLE shard_db_1.shard_table_1 (c1 int PRIMARY KEY, c2 int, INDEX idx_c2 (c2))"}, - }, - pos: mysql.Position{Name: "mysql-bin.000001", Pos: 234}, - }, - { - req: &pb.HandleSubTaskSQLsRequest{ - Op: pb.SQLOp_SKIP, - BinlogPos: "mysql-bin.000005:678", - Args: []string{"CREATE DATABASE shard_db_2;", "CREATE TABLE shard_db_2.shard_table_1 (c1 int PRIMARY KEY, c2 int, INDEX idx_c2 (c2))"}, - }, - pos: mysql.Position{Name: "mysql-bin.000005", Pos: 678}, - }, - } - - tctx := tcontext.Background() - h := NewHolder() - - // invalid binlog-pos - err := h.Set(tctx, &pb.HandleSubTaskSQLsRequest{Op: pb.SQLOp_SKIP, BinlogPos: "invalid-binlog-pos.123"}) - c.Assert(err, NotNil) - - // no operator set, mismatch - applied, args, err := h.Apply(tctx, cases[0].pos, nil) - c.Assert(err, IsNil) - c.Assert(applied, IsFalse) - c.Assert(args, IsNil) // mismatch, no args used - - // set skip operator - err = h.Set(tctx, cases[0].req) - c.Assert(err, IsNil) - - // binlog-pos mismatch - applied, args, err = h.Apply(tctx, mysql.Position{}, nil) - c.Assert(err, IsNil) - c.Assert(applied, IsFalse) - c.Assert(args, IsNil) - - // matched - applied, args, err = h.Apply(tctx, cases[0].pos, nil) - c.Assert(err, IsNil) - c.Assert(applied, IsTrue) - c.Assert(args, IsNil) // for skip, no args used - - // op replace, multi operators - for _, cs := range cases { - cs.req.Op = pb.SQLOp_REPLACE - err = h.Set(tctx, cs.req) - c.Assert(err, IsNil) - } - for _, cs := range cases { - applied, args, err = h.Apply(tctx, cs.pos, nil) - c.Assert(err, IsNil) - c.Assert(applied, IsTrue) - c.Assert(args, DeepEquals, cs.req.Args) - } - - // all operators applied, match nothing - applied, args, err = h.Apply(tctx, cases[0].pos, nil) - c.Assert(err, IsNil) - c.Assert(applied, IsFalse) - c.Assert(args, IsNil) -} - -func (o *testOperatorSuite) TestSQLPattern(c *C) { - cases := []struct { - req *pb.HandleSubTaskSQLsRequest - sqls []string - }{ - { - req: &pb.HandleSubTaskSQLsRequest{ - Op: pb.SQLOp_SKIP, - SqlPattern: "~(?i)ALTER\\s+TABLE\\s+`db1`.`tbl1`\\s+ADD\\s+COLUMN\\s+col1\\s+INT", - Args: []string{"CREATE DATABASE shard_db_1;", "CREATE TABLE shard_db_1.shard_table_1 (c1 int PRIMARY KEY, c2 int, INDEX idx_c2 (c2))"}, - }, - sqls: []string{"ALTER TABLE `db1`.`tbl1` ADD COLUMN col1 INT"}, - }, - { - req: &pb.HandleSubTaskSQLsRequest{ - Op: pb.SQLOp_SKIP, - SqlPattern: "~(?i)DROP\\s+TABLE", - Args: []string{"CREATE DATABASE shard_db_2;", "DROP TABLE shard_db_2.shard_table_1"}, - }, - sqls: []string{"INSERT INTO `db1`.`tbl` VALUES (1, 2)", "DROP TABLE `db1`.`tbl1`", "INSERT INTO `db1`.`tbl` VALUES (3, 3)"}, - }, - } - emptyPos := mysql.Position{} - - tctx := tcontext.Background() - h := NewHolder() - - // invalid sql-pattern - err := h.Set(tctx, &pb.HandleSubTaskSQLsRequest{Op: pb.SQLOp_SKIP, SqlPattern: "~(invalid-regexp"}) - c.Assert(err, NotNil) - - // no operator set, mismatch - applied, args, err := h.Apply(tctx, emptyPos, cases[0].sqls) - c.Assert(err, IsNil) - c.Assert(applied, IsFalse) - c.Assert(args, IsNil) // mismatch, no args used - - // set skip operator - err = h.Set(tctx, cases[0].req) - c.Assert(err, IsNil) - - // sql-pattern mismatch - applied, args, err = h.Apply(tctx, emptyPos, []string{"INSERT INTO `db1`.`tbl1` VALUES (1, 2)"}) - c.Assert(err, IsNil) - c.Assert(applied, IsFalse) - c.Assert(args, IsNil) - - // matched - applied, args, err = h.Apply(tctx, emptyPos, cases[0].sqls) - c.Assert(err, IsNil) - c.Assert(applied, IsTrue) - c.Assert(args, IsNil) // for skip, no args used - - // op replace, multi operators - for _, cs := range cases { - cs.req.Op = pb.SQLOp_REPLACE - err = h.Set(tctx, cs.req) - c.Assert(err, IsNil) - } - for _, cs := range cases { - applied, args, err = h.Apply(tctx, emptyPos, cs.sqls) - c.Assert(err, IsNil) - c.Assert(applied, IsTrue) - c.Assert(args, DeepEquals, cs.req.Args) - } - - // all operators applied, match nothing - applied, args, err = h.Apply(tctx, emptyPos, cases[0].sqls) - c.Assert(err, IsNil) - c.Assert(applied, IsFalse) - c.Assert(args, IsNil) -} diff --git a/syncer/syncer.go b/syncer/syncer.go index 6e80f23033..fbc6219013 100644 --- a/syncer/syncer.go +++ b/syncer/syncer.go @@ -59,7 +59,6 @@ import ( "github.com/pingcap/dm/pkg/utils" sm "github.com/pingcap/dm/syncer/safe-mode" "github.com/pingcap/dm/syncer/shardddl" - operator "github.com/pingcap/dm/syncer/sql-operator" ) var ( @@ -165,8 +164,6 @@ type Syncer struct { errors []*ExecErrorContext } - sqlOperatorHolder *operator.Holder - heartbeat *Heartbeat readerHub *streamer.ReaderHub @@ -210,7 +207,6 @@ func NewSyncer(cfg *config.SubTaskConfig, etcdClient *clientv3.Client) *Syncer { syncer.setSyncCfg() syncer.binlogType = toBinlogType(cfg.UseRelay) - syncer.sqlOperatorHolder = operator.NewHolder() syncer.readerHub = streamer.GetReaderHub() if cfg.ShardMode == config.ShardPessimistic { @@ -1235,14 +1231,11 @@ func (s *Syncer) Run(ctx context.Context) (err error) { // we only inject sqls in global streaming to avoid DDL position confusion if shardingReSync == nil { - e = s.tryInject(latestOp, currentLocation.Clone()) latestOp = null } startTime := time.Now() - if e == nil { - e, err = s.streamerController.GetEvent(tctx) - } + e, err = s.streamerController.GetEvent(tctx) if err == context.Canceled { s.tctx.L().Info("binlog replication main routine quit(context canceled)!", zap.Stringer("last location", lastLocation)) @@ -1521,13 +1514,6 @@ func (s *Syncer) handleRowsEvent(ev *replication.RowsEvent, ec eventContext) err args [][]interface{} ) - // for RowsEvent, one event may have multi SQLs and multi keys, (eg. INSERT INTO t1 VALUES (11, 12), (21, 22) ) - // to cover them dispatched to different channels, we still apply operator here - // ugly, but I have no better solution yet. - applied, sqls, err = s.tryApplySQLOperator(ec.currentLocation.Clone(), nil) // forbidden sql-pattern for DMLs - if err != nil { - return err - } param := &genDMLParam{ schema: schemaName, table: tableName, @@ -1652,8 +1638,6 @@ func (s *Syncer) handleQueryEvent(ev *replication.QueryEvent, ec eventContext) e onlineDDLTableNames map[string]*filter.Table ) - // for DDL, we don't apply operator until we try to execute it. - // so can handle sharding cases sqls, onlineDDLTableNames, err = s.resolveDDLSQL(ec.tctx, ec.parser2, parseResult.stmt, usedSchema) if err != nil { s.tctx.L().Error("fail to resolve statement", zap.String("event", "query"), zap.String("statement", sql), zap.String("schema", usedSchema), zap.Stringer("last location", ec.lastLocation), log.WrapStringerField("location", ec.currentLocation), log.ShortError(err)) @@ -1770,15 +1754,6 @@ func (s *Syncer) handleQueryEvent(ev *replication.QueryEvent, ec eventContext) e if s.cfg.ShardMode == "" { s.tctx.L().Info("start to handle ddls in normal mode", zap.String("event", "query"), zap.Strings("ddls", needHandleDDLs), zap.ByteString("raw statement", ev.Query), log.WrapStringerField("location", ec.currentLocation)) - // try apply SQL operator before addJob. now, one query event only has one DDL job, if updating to multi DDL jobs, refine this. - applied, appliedSQLs, applyErr := s.tryApplySQLOperator(ec.currentLocation.Clone(), needHandleDDLs) - if applyErr != nil { - return terror.Annotatef(applyErr, "try apply SQL operator on binlog-location %s with DDLs %v", ec.currentLocation, needHandleDDLs) - } - if applied { - s.tctx.L().Info("replace ddls to preset ddls by sql operator in normal mode", zap.String("event", "query"), zap.Strings("preset ddls", appliedSQLs), zap.Strings("ddls", needHandleDDLs), zap.ByteString("raw statement", ev.Query), log.WrapStringerField("location", ec.currentLocation)) - needHandleDDLs = appliedSQLs // maybe nil - } // interrupted after flush old checkpoint and before track DDL. failpoint.Inject("FlushCheckpointStage", func(val failpoint.Value) { @@ -1986,16 +1961,6 @@ func (s *Syncer) handleQueryEvent(ev *replication.QueryEvent, ec eventContext) e s.tctx.L().Info("start to handle ddls in shard mode", zap.String("event", "query"), zap.Strings("ddls", needHandleDDLs), zap.ByteString("raw statement", ev.Query), zap.Stringer("start location", startLocation), log.WrapStringerField("end location", ec.currentLocation)) - // try apply SQL operator before addJob. now, one query event only has one DDL job, if updating to multi DDL jobs, refine this. - applied, appliedSQLs, err := s.tryApplySQLOperator(ec.currentLocation.Clone(), needHandleDDLs) - if err != nil { - return terror.Annotatef(err, "try apply SQL operator on binlog-location %s with DDLs %v", ec.currentLocation, needHandleDDLs) - } - if applied { - s.tctx.L().Info("replace ddls to preset ddls by sql operator in shard mode", zap.String("event", "query"), zap.Strings("preset ddls", appliedSQLs), zap.Strings("ddls", needHandleDDLs), zap.ByteString("raw statement", ev.Query), zap.Stringer("start location", startLocation), log.WrapStringerField("end location", ec.currentLocation)) - needHandleDDLs = appliedSQLs // maybe nil - } - // interrupted after track DDL and before execute DDL. failpoint.Inject("FlushCheckpointStage", func(val failpoint.Value) { err = handleFlushCheckpointStage(2, val.(int), "before execute DDL") diff --git a/tests/dmctl_advance/check_list/sql_replace.sh b/tests/dmctl_advance/check_list/sql_replace.sh deleted file mode 100644 index bf4a1d4858..0000000000 --- a/tests/dmctl_advance/check_list/sql_replace.sh +++ /dev/null @@ -1,20 +0,0 @@ -#!/bin/bash - -function sql_replace_wrong_arg() { - run_dm_ctl $WORK_DIR "127.0.0.1:$MASTER_PORT" \ - "sql-replace" \ - "sql-replace <-s source> \[-b binlog-pos\] \[-p sql-pattern\] \[--sharding\] \[flags\]" 1 -} - -function sql_replace_invalid_binlog_pos() { - binlog_pos="mysql-bin:shoud-bin-digital" - run_dm_ctl $WORK_DIR "127.0.0.1:$MASTER_PORT" \ - "sql-replace test-task --binlog-pos $binlog_pos sql1" \ - "\[.*\], Message: invalid --binlog-pos $binlog_pos in sql operation: the pos should be digital" 1 -} - -function sql_replace_non_sharding_without_one_worker() { - run_dm_ctl $WORK_DIR "127.0.0.1:$MASTER_PORT" \ - "sql-replace test-task --source $SOURCE_ID1,$SOURCE_ID2 --binlog-pos mysql-bin:13426 sql1" \ - "should only specify one source, but got \[$SOURCE_ID1 $SOURCE_ID2\]" 1 -} diff --git a/tests/dmctl_advance/check_list/sql_skip.sh b/tests/dmctl_advance/check_list/sql_skip.sh deleted file mode 100644 index db439a466e..0000000000 --- a/tests/dmctl_advance/check_list/sql_skip.sh +++ /dev/null @@ -1,39 +0,0 @@ -#!/bin/bash - -function sql_skip_wrong_arg() { - run_dm_ctl $WORK_DIR "127.0.0.1:$MASTER_PORT" \ - "sql-skip" \ - "sql-skip <-s source> \[-b binlog-pos\] \[-p sql-pattern\] \[--sharding\] \[flags\]" 1 -} - -function sql_skip_binlogpos_sqlpattern_conflict() { - run_dm_ctl $WORK_DIR "127.0.0.1:$MASTER_PORT" \ - "sql-skip test-task --binlog-pos mysql-bin:194 --sql-pattern ~(?i)ALTER\\s+TABLE\\s+" \ - "cannot specify both --binlog-pos and --sql-pattern in sql operation" 1 -} - -function sql_skip_invalid_binlog_pos() { - binlog_pos="mysql-bin:shoud-bin-digital" - run_dm_ctl $WORK_DIR "127.0.0.1:$MASTER_PORT" \ - "sql-skip test-task --binlog-pos $binlog_pos" \ - "\[.*\], Message: invalid --binlog-pos $binlog_pos in sql operation: the pos should be digital" 1 -} - -function sql_skip_invalid_regex() { - regex="~(\[a-z\])\\1" - run_dm_ctl $WORK_DIR "127.0.0.1:$MASTER_PORT" \ - "sql-skip test-task --sql-pattern $regex" \ - "invalid --sql-pattern .* in sql operation" 1 -} - -function sql_skip_sharding_with_binlogpos() { - run_dm_ctl $WORK_DIR "127.0.0.1:$MASTER_PORT" \ - "sql-skip test-task --sharding --binlog-pos mysql-bin:13426" \ - "cannot specify --binlog-pos with --sharding in sql operation" 1 -} - -function sql_skip_non_sharding_without_one_worker() { - run_dm_ctl $WORK_DIR "127.0.0.1:$MASTER_PORT" \ - "sql-skip test-task --source $SOURCE_ID1,$SOURCE_ID2 --binlog-pos mysql-bin:13426" \ - "should only specify one source, but got \[$SOURCE_ID1 $SOURCE_ID2\]" 1 -} diff --git a/tests/dmctl_advance/run.sh b/tests/dmctl_advance/run.sh index 2ea516b0a8..b777dec292 100755 --- a/tests/dmctl_advance/run.sh +++ b/tests/dmctl_advance/run.sh @@ -23,16 +23,6 @@ function usage_and_arg_test() { query_error_wrong_arg - sql_skip_wrong_arg - sql_skip_binlogpos_sqlpattern_conflict - sql_skip_invalid_binlog_pos - sql_skip_invalid_regex - sql_skip_sharding_with_binlogpos - sql_skip_non_sharding_without_one_worker - - sql_replace_wrong_arg - sql_replace_invalid_binlog_pos - sql_replace_non_sharding_without_one_worker # TODO: check SQLs error test } diff --git a/tests/dmctl_command/run.sh b/tests/dmctl_command/run.sh index 8ee3cfd933..963f22e407 100644 --- a/tests/dmctl_command/run.sh +++ b/tests/dmctl_command/run.sh @@ -6,7 +6,7 @@ cur=$( cd "$( dirname "${BASH_SOURCE[0]}" )" && pwd ) source $cur/../_utils/test_prepare WORK_DIR=$TEST_DIR/$TEST_NAME -help_cnt=42 +help_cnt=39 function run() { # check dmctl alone output From 33a9be6177a35ce94a4bc97b2717480260026169 Mon Sep 17 00:00:00 2001 From: gmhdbjd Date: Fri, 31 Jul 2020 14:02:11 +0800 Subject: [PATCH 02/31] add grpc method --- _utils/terror_gen/errors_release.txt | 2 +- dm/command/argument.go | 52 +- dm/ctl/common/util.go | 2 + dm/ctl/ctl.go | 1 + dm/ctl/master/handle_error.go | 120 +++ dm/master/server.go | 76 ++ dm/master/workerrpc/interface.go | 6 + dm/master/workerrpc/rawgrpc.go | 2 + dm/pb/dmmaster.pb.go | 983 +++++++++++++++--- dm/pb/dmmaster.pb.gw.go | 78 ++ dm/pb/dmworker.pb.go | 642 +++++++++--- dm/pbmock/dmmaster.go | 35 + dm/pbmock/dmworker.go | 35 + dm/proto/dmmaster.proto | 21 + dm/proto/dmworker.proto | 15 + dm/worker/server.go | 21 + dm/worker/subtask.go | 18 + dm/worker/worker.go | 17 + errors.toml | 12 +- pkg/terror/error_list.go | 4 +- syncer/handle_error.go | 25 + .../dmctl_advance/check_list/handle_error.sh | 27 + tests/dmctl_advance/run.sh | 5 + tests/dmctl_command/run.sh | 2 +- 24 files changed, 1869 insertions(+), 332 deletions(-) create mode 100644 dm/ctl/master/handle_error.go create mode 100644 syncer/handle_error.go create mode 100644 tests/dmctl_advance/check_list/handle_error.sh diff --git a/_utils/terror_gen/errors_release.txt b/_utils/terror_gen/errors_release.txt index ea753d9262..f2875f9afc 100644 --- a/_utils/terror_gen/errors_release.txt +++ b/_utils/terror_gen/errors_release.txt @@ -20,12 +20,12 @@ ErrNotUUIDString,[code=11013:class=functional:scope=internal:level=high], "Messa ErrMariaDBDomainID,[code=11014:class=functional:scope=internal:level=high], "Message: %v is not uint32" ErrInvalidServerID,[code=11015:class=functional:scope=internal:level=high], "Message: invalid server id %s" ErrGetSQLModeFromStr,[code=11016:class=functional:scope=internal:level=high], "Message: get sql mode from string literal %s" -ErrVerifySQLOperateArgs,[code=11017:class=functional:scope=internal:level=low], "Workaround: Please make sure the args are correct." ErrStatFileSize,[code=11018:class=functional:scope=internal:level=high], "Message: get file statfs" ErrReaderAlreadyRunning,[code=11019:class=functional:scope=internal:level=high], "Message: binlog reader is already running" ErrReaderAlreadyStarted,[code=11020:class=functional:scope=internal:level=high], "Message: stage %s, expect %s, already started" ErrReaderStateCannotClose,[code=11021:class=functional:scope=internal:level=high], "Message: stage %s, expect %s, can not close" ErrReaderShouldStartSync,[code=11022:class=functional:scope=internal:level=high], "Message: stage %s, expect %s" +ErrVerifyHandleErrorArgs,[code=11115:class=functional:scope=internal:level=low], "Workaround: Please make sure the args are correct." ErrEmptyRelayDir,[code=11023:class=functional:scope=internal:level=high], "Message: empty relay dir, Workaround: Please check `relay-dir` config in task configuration file." ErrReadDir,[code=11024:class=functional:scope=internal:level=high], "Message: read dir: %s" ErrBaseFileNotFound,[code=11025:class=functional:scope=internal:level=high], "Message: base file %s in directory %s not found" diff --git a/dm/command/argument.go b/dm/command/argument.go index 4d4e3ccb7e..4cbc13cb87 100644 --- a/dm/command/argument.go +++ b/dm/command/argument.go @@ -14,9 +14,6 @@ package command import ( - "regexp" - "strings" - "github.com/siddontang/go-mysql/mysql" "github.com/pingcap/dm/pkg/binlog" @@ -32,47 +29,12 @@ func TrimQuoteMark(s string) string { return s } -// VerifySQLOperateArgs verify args for sql operation, including sql-skip, sql-replace. -// NOTE: only part of args verified in here and others need to be verified from outer. -func VerifySQLOperateArgs(binlogPosStr, sqlPattern string, sharding bool) (*mysql.Position, *regexp.Regexp, error) { - binlogPosStr = TrimQuoteMark(binlogPosStr) - sqlPattern = TrimQuoteMark(sqlPattern) - - var ( - pos *mysql.Position - reg *regexp.Regexp - ) - - if len(binlogPosStr) > 0 && len(sqlPattern) > 0 { - return nil, nil, terror.ErrVerifySQLOperateArgs.New("cannot specify both --binlog-pos and --sql-pattern in sql operation") +// VerifyBinlogPos verify binlog pos string +func VerifyBinlogPos(pos string) (*mysql.Position, error) { + binlogPosStr := TrimQuoteMark(pos) + pos2, err := binlog.PositionFromStr(binlogPosStr) + if err != nil { + return nil, terror.ErrVerifyHandleErrorArgs.Generatef("invalid --binlog-pos %s in handle-error operation: %s", binlogPosStr, terror.Message(err)) } - - if len(binlogPosStr) > 0 { - pos2, err := binlog.PositionFromStr(binlogPosStr) - if err != nil { - return nil, nil, terror.ErrVerifySQLOperateArgs.Generatef("invalid --binlog-pos %s in sql operation: %s", binlogPosStr, terror.Message(err)) - } - pos = &pos2 - } else if len(sqlPattern) > 0 { - var pattern string - if strings.HasPrefix(sqlPattern, "~") { - pattern = sqlPattern[1:] - } else { - pattern = "^" + regexp.QuoteMeta(sqlPattern) + "$" - } - - var err error - reg, err = regexp.Compile(pattern) - if err != nil { - return nil, nil, terror.ErrVerifySQLOperateArgs.AnnotateDelegate(err, "invalid --sql-pattern %s in sql operation", sqlPattern) - } - } else { - return nil, nil, terror.ErrVerifySQLOperateArgs.New("must specify one of --binlog-pos and --sql-pattern in sql operation") - } - - if sharding && len(binlogPosStr) > 0 { - return nil, nil, terror.ErrVerifySQLOperateArgs.New("cannot specify --binlog-pos with --sharding in sql operation") - } - - return pos, reg, nil + return &pos2, nil } diff --git a/dm/ctl/common/util.go b/dm/ctl/common/util.go index 1129f36628..b2c5582339 100644 --- a/dm/ctl/common/util.go +++ b/dm/ctl/common/util.go @@ -20,6 +20,7 @@ import ( "strings" "time" + "github.com/pingcap/dm/dm/command" "github.com/pingcap/dm/dm/config" "github.com/pingcap/dm/dm/pb" parserpkg "github.com/pingcap/dm/pkg/parser" @@ -189,6 +190,7 @@ func ExtractSQLsFromArgs(args []string) ([]string, error) { } concat := strings.TrimSpace(strings.Join(args, " ")) + concat = command.TrimQuoteMark(concat) parser := parser.New() nodes, err := parserpkg.Parse(parser, concat, "", "") diff --git a/dm/ctl/ctl.go b/dm/ctl/ctl.go index 029410ac8b..689e539b87 100644 --- a/dm/ctl/ctl.go +++ b/dm/ctl/ctl.go @@ -78,6 +78,7 @@ func NewRootCmd() *cobra.Command { master.NewListMemberCmd(), master.NewOperateSchemaCmd(), master.NewGetTaskCfgCmd(), + master.NewHandleErrorCmd(), ) return cmd } diff --git a/dm/ctl/master/handle_error.go b/dm/ctl/master/handle_error.go new file mode 100644 index 0000000000..ef6276cc14 --- /dev/null +++ b/dm/ctl/master/handle_error.go @@ -0,0 +1,120 @@ +// Copyright 2020 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package master + +import ( + "context" + "os" + + "github.com/spf13/cobra" + + "github.com/pingcap/dm/dm/command" + "github.com/pingcap/dm/dm/ctl/common" + "github.com/pingcap/dm/dm/pb" +) + +// NewHandleErrorCmd creates a HandleError command +func NewHandleErrorCmd() *cobra.Command { + cmd := &cobra.Command{ + Use: "handle-error [-s source ...] [-b binlog-pos] [replace-sql1;replace-sql2;]", + Short: "skip/replace the current error event or a specific binlog position (binlog-pos) event", + Run: handleErrorFunc, + } + cmd.Flags().StringP("binlog-pos", "b", "", "position used to match binlog event if matched the handler-error operation will be applied. The format like \"mysql-bin|000001.000003:3270\"") + return cmd +} + +func convertOp(t string) pb.ErrorOp { + switch t { + case "skip": + return pb.ErrorOp_Skip + case "replace": + return pb.ErrorOp_Replace + default: + return pb.ErrorOp_InvalidErrorOp + } +} + +// handleErrorFunc does handle error request +func handleErrorFunc(cmd *cobra.Command, _ []string) { + if len(cmd.Flags().Args()) < 2 { + cmd.SetOut(os.Stdout) + cmd.Usage() + return + } + + taskName := cmd.Flags().Arg(0) + operation := cmd.Flags().Arg(1) + var sqls []string + + op := convertOp(operation) + switch op { + case pb.ErrorOp_Skip: + if len(cmd.Flags().Args()) > 2 { + common.PrintLines("replace-sqls will be ignored for 'skip' operation") + } + case pb.ErrorOp_Replace: + if len(cmd.Flags().Args()) <= 2 { + common.PrintLines("must specify the replace-sqls for replace operation") + return + } + + var err error + sqls, err = common.ExtractSQLsFromArgs(cmd.Flags().Args()[2:]) + if err != nil { + common.PrintLines("%v", err) + return + } + default: + common.PrintLines("invalid operation '%s', please use `skip` or `relpace`", operation) + return + } + + binlogPos, err := cmd.Flags().GetString("binlog-pos") + if err != nil { + common.PrintLines("%v", err) + return + } + if len(binlogPos) != 0 { + _, err = command.VerifyBinlogPos(binlogPos) + if err != nil { + common.PrintLines("%v", err) + return + } + } + + sources, err := common.GetSourceArgs(cmd) + if err != nil { + common.PrintLines("%v", err) + return + } + + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + cli := common.MasterClient() + + resp, err := cli.HandleError(ctx, &pb.HandleErrorRequest{ + Op: op, + Task: taskName, + BinlogPos: binlogPos, + Sqls: sqls, + Sources: sources, + }) + if err != nil { + common.PrintLines("can not handle error:\n%v", err) + return + } + + common.PrettyPrintResponse(resp) +} diff --git a/dm/master/server.go b/dm/master/server.go index b607bd48c1..1d83f9abe0 100644 --- a/dm/master/server.go +++ b/dm/master/server.go @@ -2084,3 +2084,79 @@ func (s *Server) GetTaskCfg(ctx context.Context, req *pb.GetTaskCfgRequest) (*pb Cfg: cfg, }, nil } + +// HandleError implements MasterServer.HandleError +func (s *Server) HandleError(ctx context.Context, req *pb.HandleErrorRequest) (*pb.HandleErrorResponse, error) { + log.L().Info("", zap.Stringer("payload", req), zap.String("request", "HandleError")) + + isLeader, needForward := s.isLeaderAndNeedForward() + if !isLeader { + if needForward { + return s.leaderClient.HandleError(ctx, req) + } + return nil, terror.ErrMasterRequestIsNotForwardToLeader + } + + sources := req.Sources + if len(sources) == 0 { + sources = s.getTaskResources(req.Task) + log.L().Info(fmt.Sprintf("sources: %s", sources)) + if len(sources) == 0 { + return &pb.HandleErrorResponse{ + Result: false, + Msg: fmt.Sprintf("task %s has no source or not exist, please check the task name and status", req.Task), + }, nil + } + } + + workerReq := workerrpc.Request{ + Type: workerrpc.CmdHandleError, + HandleError: &pb.HandleWorkerErrorRequest{ + Op: req.Op, + Task: req.Task, + BinlogPos: req.BinlogPos, + Sqls: req.Sqls, + }, + } + + workerRespCh := make(chan *pb.CommonWorkerResponse, len(sources)) + var wg sync.WaitGroup + for _, source := range sources { + wg.Add(1) + go func(source string) { + defer wg.Done() + worker := s.scheduler.GetWorkerBySource(source) + if worker == nil { + workerRespCh <- errorCommonWorkerResponse(fmt.Sprintf("source %s relevant worker-client not found", source), source, "") + return + } + resp, err := worker.SendRequest(ctx, &workerReq, s.cfg.RPCTimeout) + workerResp := &pb.CommonWorkerResponse{} + if err != nil { + workerResp = errorCommonWorkerResponse(err.Error(), source, worker.BaseInfo().Name) + } else { + workerResp = resp.HandleError + } + workerResp.Source = source + workerRespCh <- workerResp + }(source) + } + wg.Wait() + + workerRespMap := make(map[string]*pb.CommonWorkerResponse, len(sources)) + for len(workerRespCh) > 0 { + workerResp := <-workerRespCh + workerRespMap[workerResp.Source] = workerResp + } + + sort.Strings(sources) + workerResps := make([]*pb.CommonWorkerResponse, 0, len(sources)) + for _, worker := range sources { + workerResps = append(workerResps, workerRespMap[worker]) + } + + return &pb.HandleErrorResponse{ + Result: true, + Sources: workerResps, + }, nil +} diff --git a/dm/master/workerrpc/interface.go b/dm/master/workerrpc/interface.go index 2762d056f2..a977ec0e71 100644 --- a/dm/master/workerrpc/interface.go +++ b/dm/master/workerrpc/interface.go @@ -43,6 +43,8 @@ const ( CmdFetchDDLInfo CmdOperateSchema + + CmdHandleError ) // Request wraps all dm-worker rpc requests. @@ -64,6 +66,8 @@ type Request struct { MigrateRelay *pb.MigrateRelayRequest OperateSchema *pb.OperateWorkerSchemaRequest + + HandleError *pb.HandleWorkerErrorRequest } // Response wraps all dm-worker rpc responses. @@ -85,6 +89,8 @@ type Response struct { MigrateRelay *pb.CommonWorkerResponse OperateSchema *pb.CommonWorkerResponse + + HandleError *pb.CommonWorkerResponse } // Client is a client that sends RPC. diff --git a/dm/master/workerrpc/rawgrpc.go b/dm/master/workerrpc/rawgrpc.go index 4a6bf07342..c874cae208 100644 --- a/dm/master/workerrpc/rawgrpc.go +++ b/dm/master/workerrpc/rawgrpc.go @@ -126,6 +126,8 @@ func callRPC(ctx context.Context, client pb.WorkerClient, req *Request) (*Respon resp.MigrateRelay, err = client.MigrateRelay(ctx, req.MigrateRelay) case CmdOperateSchema: resp.OperateSchema, err = client.OperateSchema(ctx, req.OperateSchema) + case CmdHandleError: + resp.HandleError, err = client.HandleError(ctx, req.HandleError) default: return nil, terror.ErrMasterGRPCInvalidReqType.Generate(req.Type) } diff --git a/dm/pb/dmmaster.pb.go b/dm/pb/dmmaster.pb.go index 7394f69b6a..c751adfdfd 100644 --- a/dm/pb/dmmaster.pb.go +++ b/dm/pb/dmmaster.pb.go @@ -2998,6 +2998,142 @@ func (m *GetTaskCfgResponse) GetCfg() string { return "" } +type HandleErrorRequest struct { + Op ErrorOp `protobuf:"varint,1,opt,name=op,proto3,enum=pb.ErrorOp" json:"op,omitempty"` + Task string `protobuf:"bytes,2,opt,name=task,proto3" json:"task,omitempty"` + Sources []string `protobuf:"bytes,3,rep,name=sources,proto3" json:"sources,omitempty"` + BinlogPos string `protobuf:"bytes,4,opt,name=binlogPos,proto3" json:"binlogPos,omitempty"` + Sqls []string `protobuf:"bytes,5,rep,name=sqls,proto3" json:"sqls,omitempty"` +} + +func (m *HandleErrorRequest) Reset() { *m = HandleErrorRequest{} } +func (m *HandleErrorRequest) String() string { return proto.CompactTextString(m) } +func (*HandleErrorRequest) ProtoMessage() {} +func (*HandleErrorRequest) Descriptor() ([]byte, []int) { + return fileDescriptor_f9bef11f2a341f03, []int{49} +} +func (m *HandleErrorRequest) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *HandleErrorRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_HandleErrorRequest.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 *HandleErrorRequest) XXX_Merge(src proto.Message) { + xxx_messageInfo_HandleErrorRequest.Merge(m, src) +} +func (m *HandleErrorRequest) XXX_Size() int { + return m.Size() +} +func (m *HandleErrorRequest) XXX_DiscardUnknown() { + xxx_messageInfo_HandleErrorRequest.DiscardUnknown(m) +} + +var xxx_messageInfo_HandleErrorRequest proto.InternalMessageInfo + +func (m *HandleErrorRequest) GetOp() ErrorOp { + if m != nil { + return m.Op + } + return ErrorOp_Skip +} + +func (m *HandleErrorRequest) GetTask() string { + if m != nil { + return m.Task + } + return "" +} + +func (m *HandleErrorRequest) GetSources() []string { + if m != nil { + return m.Sources + } + return nil +} + +func (m *HandleErrorRequest) GetBinlogPos() string { + if m != nil { + return m.BinlogPos + } + return "" +} + +func (m *HandleErrorRequest) GetSqls() []string { + if m != nil { + return m.Sqls + } + return nil +} + +type HandleErrorResponse struct { + Result bool `protobuf:"varint,1,opt,name=result,proto3" json:"result,omitempty"` + Msg string `protobuf:"bytes,2,opt,name=msg,proto3" json:"msg,omitempty"` + Sources []*CommonWorkerResponse `protobuf:"bytes,3,rep,name=sources,proto3" json:"sources,omitempty"` +} + +func (m *HandleErrorResponse) Reset() { *m = HandleErrorResponse{} } +func (m *HandleErrorResponse) String() string { return proto.CompactTextString(m) } +func (*HandleErrorResponse) ProtoMessage() {} +func (*HandleErrorResponse) Descriptor() ([]byte, []int) { + return fileDescriptor_f9bef11f2a341f03, []int{50} +} +func (m *HandleErrorResponse) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *HandleErrorResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_HandleErrorResponse.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 *HandleErrorResponse) XXX_Merge(src proto.Message) { + xxx_messageInfo_HandleErrorResponse.Merge(m, src) +} +func (m *HandleErrorResponse) XXX_Size() int { + return m.Size() +} +func (m *HandleErrorResponse) XXX_DiscardUnknown() { + xxx_messageInfo_HandleErrorResponse.DiscardUnknown(m) +} + +var xxx_messageInfo_HandleErrorResponse proto.InternalMessageInfo + +func (m *HandleErrorResponse) GetResult() bool { + if m != nil { + return m.Result + } + return false +} + +func (m *HandleErrorResponse) GetMsg() string { + if m != nil { + return m.Msg + } + return "" +} + +func (m *HandleErrorResponse) GetSources() []*CommonWorkerResponse { + if m != nil { + return m.Sources + } + return nil +} + func init() { proto.RegisterEnum("pb.SourceOp", SourceOp_name, SourceOp_value) proto.RegisterEnum("pb.LeaderOp", LeaderOp_name, LeaderOp_value) @@ -3050,133 +3186,139 @@ func init() { proto.RegisterType((*GetSubTaskCfgResponse)(nil), "pb.GetSubTaskCfgResponse") proto.RegisterType((*GetTaskCfgRequest)(nil), "pb.GetTaskCfgRequest") proto.RegisterType((*GetTaskCfgResponse)(nil), "pb.GetTaskCfgResponse") + proto.RegisterType((*HandleErrorRequest)(nil), "pb.HandleErrorRequest") + proto.RegisterType((*HandleErrorResponse)(nil), "pb.HandleErrorResponse") } func init() { proto.RegisterFile("dmmaster.proto", fileDescriptor_f9bef11f2a341f03) } var fileDescriptor_f9bef11f2a341f03 = []byte{ - // 1921 bytes of a gzipped FileDescriptorProto + // 1991 bytes of a gzipped FileDescriptorProto 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xbc, 0x59, 0x5f, 0x6f, 0xdb, 0xc8, - 0x11, 0x17, 0x25, 0xc5, 0x96, 0xc7, 0xb1, 0x23, 0x6f, 0x6c, 0x59, 0x66, 0x1c, 0xc5, 0xb7, 0x77, - 0x97, 0x1a, 0x46, 0x11, 0x35, 0xbe, 0x3e, 0x05, 0xb8, 0x02, 0x17, 0xcb, 0x77, 0x35, 0xaa, 0x9c, - 0x53, 0xba, 0x46, 0x71, 0x28, 0x50, 0x1c, 0x45, 0xad, 0x64, 0xc2, 0x14, 0xc9, 0x90, 0x94, 0x5d, - 0x23, 0x38, 0x14, 0xe8, 0x07, 0x68, 0x0b, 0xf4, 0xe1, 0x1e, 0xfb, 0xd0, 0xd7, 0x7e, 0x8f, 0xf6, - 0x31, 0x40, 0x5f, 0xfa, 0x58, 0x24, 0xfd, 0x20, 0xc5, 0xce, 0x2e, 0xc9, 0xe5, 0x1f, 0xa9, 0x51, - 0x80, 0xf3, 0x1b, 0x67, 0x67, 0x77, 0xe6, 0xb7, 0x33, 0xb3, 0xb3, 0x33, 0x4b, 0x58, 0x1f, 0x4e, - 0x26, 0x66, 0x18, 0xb1, 0xe0, 0x89, 0x1f, 0x78, 0x91, 0x47, 0xaa, 0xfe, 0x40, 0x5f, 0x1f, 0x4e, - 0xae, 0xbd, 0xe0, 0x32, 0x1e, 0xd3, 0x77, 0xc7, 0x9e, 0x37, 0x76, 0x58, 0xd7, 0xf4, 0xed, 0xae, - 0xe9, 0xba, 0x5e, 0x64, 0x46, 0xb6, 0xe7, 0x86, 0x82, 0x4b, 0x5f, 0xc1, 0xce, 0x0b, 0x7b, 0x1c, - 0x98, 0x11, 0xfb, 0x35, 0x2e, 0x32, 0x98, 0x63, 0xde, 0x18, 0xec, 0xd5, 0x94, 0x85, 0x11, 0xe9, - 0x00, 0x3c, 0xb7, 0x5d, 0xc7, 0x1b, 0x7f, 0x6d, 0x4e, 0x58, 0x5b, 0xdb, 0xd3, 0xf6, 0x57, 0x0c, - 0x65, 0x84, 0xec, 0xc2, 0x8a, 0xa0, 0x5e, 0x7a, 0x61, 0xbb, 0xba, 0xa7, 0xed, 0xaf, 0x19, 0xe9, - 0x00, 0x69, 0xc1, 0x52, 0xe8, 0x4d, 0x03, 0x8b, 0xb5, 0x6b, 0xb8, 0x52, 0x52, 0xf4, 0x25, 0x74, - 0xce, 0xfd, 0x61, 0x56, 0xe3, 0x91, 0xe7, 0x8e, 0xec, 0x71, 0xac, 0xb7, 0x05, 0x4b, 0x16, 0x0e, - 0x48, 0x9d, 0x92, 0x52, 0x24, 0x56, 0x33, 0x12, 0xbf, 0x85, 0xe6, 0x59, 0x64, 0x06, 0xd1, 0xaf, - 0xcc, 0xf0, 0x32, 0x96, 0x41, 0xa0, 0x1e, 0x99, 0xe1, 0xa5, 0x94, 0x80, 0xdf, 0xa4, 0x0d, 0xcb, - 0x62, 0x05, 0x47, 0x5b, 0xdb, 0x5f, 0x31, 0x62, 0x92, 0xef, 0x34, 0x60, 0x13, 0xef, 0x8a, 0xbd, - 0x60, 0x91, 0x89, 0x78, 0x1b, 0x86, 0x32, 0x42, 0x5f, 0xc1, 0x86, 0xa2, 0x21, 0xf4, 0x3d, 0x37, - 0x64, 0x1c, 0x4e, 0xc0, 0xc2, 0xa9, 0x13, 0xa1, 0x92, 0x86, 0x21, 0x29, 0xd2, 0x84, 0xda, 0x24, - 0x1c, 0x4b, 0x8c, 0xfc, 0x93, 0x1c, 0xa6, 0x8a, 0x6b, 0x7b, 0xb5, 0xfd, 0xd5, 0xc3, 0xf6, 0x13, - 0x7f, 0xf0, 0xe4, 0xc8, 0x9b, 0x4c, 0x3c, 0x37, 0xb6, 0x82, 0x10, 0x9a, 0x40, 0xa2, 0x9f, 0xc1, - 0x8e, 0x30, 0xd3, 0x0b, 0xf4, 0xf0, 0x7b, 0x59, 0x88, 0xde, 0x80, 0x5e, 0xb6, 0x68, 0x61, 0xc0, - 0x4f, 0xf3, 0x80, 0xb7, 0x39, 0xe0, 0x5f, 0x4e, 0x59, 0x70, 0x73, 0x16, 0x99, 0xd1, 0x34, 0x2c, - 0xe2, 0xfd, 0x2d, 0x90, 0x53, 0x9f, 0xf1, 0x48, 0x52, 0xdd, 0xa0, 0x43, 0xd5, 0xf3, 0x51, 0xdd, - 0xfa, 0x21, 0x70, 0x19, 0x9c, 0x79, 0xea, 0x1b, 0x55, 0xcf, 0xe7, 0x2e, 0x72, 0x79, 0x60, 0x09, - 0xbd, 0xf8, 0xad, 0xba, 0xa8, 0x96, 0x71, 0x11, 0xfd, 0x93, 0x06, 0xf7, 0x33, 0x0a, 0xe4, 0xa6, - 0xe6, 0x69, 0x48, 0x37, 0x5c, 0x2d, 0xdb, 0x70, 0xad, 0xd4, 0x43, 0xf5, 0xf7, 0xf5, 0xd0, 0x17, - 0xb0, 0x21, 0x8c, 0xfd, 0xc1, 0x71, 0x47, 0x03, 0x20, 0xaa, 0x88, 0x5b, 0x09, 0xac, 0x2f, 0xa1, - 0xa5, 0x38, 0xb2, 0x6f, 0x87, 0x91, 0x82, 0xdd, 0x4d, 0x4f, 0x7a, 0xc1, 0x21, 0x39, 0xec, 0x57, - 0xb0, 0x5d, 0x90, 0x73, 0x1b, 0x81, 0x76, 0x0c, 0x5b, 0xc8, 0x3f, 0x0e, 0x02, 0x2f, 0xf8, 0x70, - 0xf8, 0x91, 0x34, 0x83, 0x22, 0x66, 0x61, 0xf4, 0x3f, 0xc9, 0xa3, 0x6f, 0x25, 0xe8, 0x51, 0x6c, - 0x11, 0xfc, 0x11, 0xdc, 0x3f, 0xbb, 0xf0, 0xae, 0x7b, 0xbd, 0x7e, 0xdf, 0xb3, 0x2e, 0xc3, 0x0f, - 0x8b, 0x9a, 0xbf, 0x6a, 0xb0, 0x2c, 0x25, 0x90, 0x75, 0xa8, 0x9e, 0xf4, 0xe4, 0xba, 0xea, 0x49, - 0x2f, 0x91, 0x54, 0x55, 0x24, 0x11, 0xa8, 0x4f, 0xbc, 0x61, 0x9c, 0x87, 0xf1, 0x9b, 0x6c, 0xc2, - 0x1d, 0xef, 0xda, 0x65, 0x41, 0xbb, 0x8e, 0x83, 0x82, 0xe0, 0x33, 0x7b, 0xbd, 0x7e, 0xd8, 0xbe, - 0x83, 0x0a, 0xf1, 0x1b, 0xb3, 0xee, 0x8d, 0x6b, 0xb1, 0x61, 0x7b, 0x09, 0x47, 0x25, 0x45, 0x74, - 0x68, 0x4c, 0x5d, 0xc9, 0x59, 0x46, 0x4e, 0x42, 0x53, 0x0b, 0x36, 0xb3, 0xdb, 0x5c, 0xd8, 0xb4, - 0x1f, 0xc1, 0x1d, 0x87, 0x2f, 0x95, 0x86, 0x5d, 0xe5, 0x86, 0x95, 0xe2, 0x0c, 0xc1, 0xa1, 0x0e, - 0x6c, 0x9e, 0xbb, 0xfc, 0x33, 0x1e, 0x97, 0xc6, 0xcc, 0x9b, 0x84, 0xc2, 0xdd, 0x80, 0xf9, 0x8e, - 0x69, 0xb1, 0x53, 0xdc, 0xb1, 0xd0, 0x92, 0x19, 0x23, 0x7b, 0xb0, 0x3a, 0xf2, 0x02, 0x8b, 0x19, - 0x98, 0xf3, 0xe5, 0x0d, 0xa0, 0x0e, 0xd1, 0x2f, 0x60, 0x2b, 0xa7, 0x6d, 0xd1, 0x3d, 0xd1, 0x67, - 0xd0, 0x39, 0xbb, 0xb6, 0x23, 0xeb, 0x42, 0xb9, 0xf9, 0x44, 0xa2, 0x8e, 0xa1, 0x2b, 0x3e, 0xd7, - 0xb2, 0x3e, 0xff, 0x3d, 0x3c, 0x9a, 0xb9, 0xf6, 0x56, 0xd2, 0x86, 0x01, 0x3b, 0x32, 0xfd, 0x96, - 0x54, 0x0a, 0x0f, 0x94, 0x24, 0x8c, 0xae, 0x42, 0xae, 0xcc, 0xc2, 0xb3, 0x03, 0xf9, 0x7b, 0x0d, - 0xf4, 0x32, 0xa1, 0x72, 0x43, 0x73, 0xa5, 0xfe, 0xb0, 0xb9, 0xfd, 0x7b, 0x0d, 0xb6, 0x5f, 0x4e, - 0x83, 0x71, 0xd9, 0x66, 0x67, 0x3a, 0x89, 0x1f, 0x09, 0xdb, 0x35, 0xad, 0xc8, 0xbe, 0x62, 0x12, - 0x55, 0x42, 0xe3, 0xc1, 0xb4, 0x27, 0x22, 0xb4, 0x6a, 0x06, 0x7e, 0xf3, 0xf9, 0x23, 0xdb, 0x61, - 0x98, 0xb5, 0xc4, 0x39, 0x4c, 0x68, 0x3c, 0x76, 0xd3, 0x41, 0xcf, 0x0e, 0xda, 0x77, 0x64, 0xb1, - 0x83, 0x14, 0xfd, 0x1d, 0xb4, 0x8b, 0xc0, 0x6e, 0x25, 0x02, 0x1e, 0x43, 0xf3, 0xe8, 0x82, 0x59, - 0x97, 0xff, 0xe7, 0xba, 0xa3, 0x9f, 0xc3, 0x86, 0x32, 0x6f, 0xe1, 0x53, 0xd2, 0x87, 0x4d, 0x19, - 0x13, 0x67, 0xa8, 0x38, 0x56, 0xb5, 0xab, 0x44, 0xc3, 0x5d, 0x8e, 0x56, 0xb0, 0xd3, 0x70, 0x90, - 0x15, 0x91, 0x88, 0xb1, 0xb8, 0x22, 0x9a, 0xc2, 0x56, 0x4e, 0xda, 0xad, 0xd8, 0xea, 0x18, 0xb6, - 0x0c, 0x36, 0xb6, 0xf9, 0xf9, 0x8c, 0xa7, 0xcc, 0xbd, 0xa4, 0xcc, 0xe1, 0x30, 0x60, 0x61, 0x28, - 0xd5, 0xc6, 0x24, 0x7d, 0x0e, 0xad, 0xbc, 0x98, 0x85, 0xed, 0xf9, 0x33, 0xd8, 0x3c, 0x1d, 0x8d, - 0x1c, 0xdb, 0x65, 0x2f, 0xd8, 0x64, 0x90, 0x41, 0x12, 0xdd, 0xf8, 0x09, 0x12, 0xfe, 0x5d, 0x56, - 0x92, 0xf1, 0xc4, 0x97, 0x5b, 0xbf, 0x30, 0x84, 0x9f, 0x26, 0x2e, 0xed, 0x33, 0x73, 0x98, 0x42, - 0x28, 0xb8, 0x54, 0xb0, 0x85, 0x4b, 0x51, 0x71, 0x76, 0xd5, 0xc2, 0x8a, 0xff, 0xa8, 0x01, 0x88, - 0x2c, 0x79, 0xe2, 0x8e, 0xbc, 0x52, 0xe3, 0xeb, 0xd0, 0x98, 0xe0, 0xbe, 0x4e, 0x7a, 0xb8, 0xb2, - 0x6e, 0x24, 0x34, 0xbf, 0x24, 0x4d, 0xc7, 0x4e, 0xee, 0x03, 0x41, 0xf0, 0x15, 0x3e, 0x63, 0xc1, - 0xb9, 0xd1, 0x17, 0x09, 0x65, 0xc5, 0x48, 0x68, 0xde, 0x48, 0x58, 0x8e, 0xcd, 0xdc, 0x08, 0xb9, - 0xe2, 0x1a, 0x55, 0x46, 0xe8, 0x00, 0x40, 0x38, 0x72, 0x26, 0x1e, 0x02, 0x75, 0xee, 0xfd, 0xd8, - 0x05, 0xfc, 0x9b, 0xe3, 0x08, 0x23, 0x73, 0x1c, 0xdf, 0xe0, 0x82, 0x50, 0xda, 0xa1, 0x7a, 0xa6, - 0x1d, 0xea, 0x43, 0x93, 0xd7, 0x33, 0xc2, 0x68, 0xc2, 0x67, 0xb1, 0x69, 0xb4, 0x34, 0xaa, 0xcb, - 0xaa, 0xef, 0x58, 0x77, 0x2d, 0xd5, 0x4d, 0xbf, 0x16, 0xd2, 0x84, 0x15, 0x67, 0x4a, 0xdb, 0x87, - 0x65, 0xd1, 0x89, 0x8a, 0x1c, 0xbf, 0x7a, 0xb8, 0xce, 0xdd, 0x99, 0x9a, 0xde, 0x88, 0xd9, 0xb1, - 0x3c, 0x61, 0x85, 0x79, 0xf2, 0x44, 0x17, 0x9b, 0x91, 0x97, 0x9a, 0xce, 0x88, 0xd9, 0xf4, 0x6f, - 0x1a, 0x2c, 0x0b, 0x31, 0x21, 0x79, 0x02, 0x4b, 0x0e, 0xee, 0x1a, 0x45, 0xad, 0x1e, 0x6e, 0x62, - 0x4c, 0xe5, 0x6c, 0xf1, 0xf3, 0x8a, 0x21, 0x67, 0xf1, 0xf9, 0x02, 0x16, 0x5a, 0x41, 0x99, 0xaf, - 0xee, 0x96, 0xcf, 0x17, 0xb3, 0xf8, 0x7c, 0xa1, 0x16, 0x2d, 0xa4, 0xcc, 0x57, 0x77, 0xc3, 0xe7, - 0x8b, 0x59, 0xcf, 0x1b, 0xb0, 0x24, 0x62, 0x89, 0x37, 0x90, 0x28, 0x37, 0x73, 0x02, 0x5b, 0x19, - 0xb8, 0x8d, 0x04, 0x56, 0x2b, 0x03, 0xab, 0x91, 0xa8, 0x6f, 0x65, 0xd4, 0x37, 0x62, 0x35, 0x3c, - 0x3c, 0xb8, 0xfb, 0xe2, 0x68, 0x14, 0x04, 0x65, 0x40, 0x54, 0x95, 0x0b, 0xa7, 0xbd, 0x4f, 0x61, - 0x59, 0x80, 0xcf, 0xd4, 0x60, 0xd2, 0xd4, 0x46, 0xcc, 0xa3, 0x7f, 0xd7, 0xd2, 0x7c, 0x6d, 0x5d, - 0xb0, 0x89, 0x39, 0x3b, 0x5f, 0x23, 0x3b, 0x6d, 0xfe, 0x0a, 0x75, 0xea, 0xcc, 0xe6, 0x8f, 0x1f, - 0xb9, 0xa1, 0x19, 0x99, 0x03, 0x33, 0x4c, 0x2e, 0xca, 0x98, 0xe6, 0xbb, 0x8f, 0xcc, 0x81, 0xc3, - 0xe4, 0x3d, 0x29, 0x08, 0x3c, 0x1c, 0xa8, 0xaf, 0xbd, 0x24, 0x0f, 0x07, 0x52, 0xea, 0x7d, 0x20, - 0xd1, 0xde, 0xca, 0x7d, 0x70, 0x00, 0x9b, 0x5f, 0xb1, 0xe8, 0x6c, 0x3a, 0xe0, 0x97, 0xe2, 0xd1, - 0x68, 0x3c, 0xe7, 0x3a, 0xa0, 0xe7, 0xb0, 0x95, 0x9b, 0xbb, 0x30, 0x44, 0x02, 0x75, 0x6b, 0x34, - 0x8e, 0xcd, 0x88, 0xdf, 0xf4, 0x47, 0xb0, 0xf1, 0x15, 0x8b, 0xde, 0x43, 0xff, 0x4b, 0x20, 0xea, - 0xc4, 0x85, 0x95, 0x37, 0xa1, 0x66, 0x8d, 0x92, 0x0a, 0xcc, 0x1a, 0x8d, 0x0f, 0x06, 0xd0, 0x88, - 0x2f, 0x6b, 0x72, 0x1f, 0xee, 0x9d, 0xb8, 0x57, 0xa6, 0x63, 0x0f, 0xe3, 0xa1, 0x66, 0x85, 0xdc, - 0x83, 0x55, 0x7c, 0x5f, 0x11, 0x43, 0x4d, 0x8d, 0x34, 0xe1, 0xae, 0x68, 0x8c, 0xe5, 0x48, 0x95, - 0xac, 0x03, 0x9c, 0x45, 0x9e, 0x2f, 0xe9, 0x1a, 0xd2, 0x17, 0xde, 0xb5, 0xa4, 0xeb, 0x07, 0xbf, - 0x80, 0x46, 0x7c, 0x7b, 0x28, 0x3a, 0xe2, 0xa1, 0x66, 0x85, 0x6c, 0xc0, 0xda, 0xf1, 0x95, 0x6d, - 0x45, 0xc9, 0x90, 0x46, 0xb6, 0xe1, 0xfe, 0x91, 0xe9, 0x5a, 0xcc, 0xc9, 0x32, 0xaa, 0x87, 0xff, - 0xb8, 0x07, 0x4b, 0x22, 0x07, 0x90, 0x6f, 0x60, 0x25, 0x79, 0xfa, 0x21, 0x78, 0xe0, 0xf3, 0x6f, - 0x4d, 0xfa, 0x56, 0x6e, 0x54, 0x58, 0x8c, 0x3e, 0xfa, 0xc3, 0xbf, 0xfe, 0xfb, 0x97, 0xea, 0x0e, - 0xdd, 0xec, 0x9a, 0xbe, 0x1d, 0x76, 0xaf, 0x9e, 0x9a, 0x8e, 0x7f, 0x61, 0x3e, 0xed, 0xf2, 0x58, - 0x0f, 0x9f, 0x69, 0x07, 0x64, 0x04, 0xab, 0xca, 0x8b, 0x06, 0xc1, 0xe6, 0xb1, 0xf8, 0x86, 0xa2, - 0x6f, 0x17, 0xc6, 0xa5, 0x82, 0xc7, 0xa8, 0x60, 0x4f, 0x7f, 0x50, 0xa6, 0xa0, 0xfb, 0x9a, 0x3b, - 0xf3, 0x3b, 0xae, 0xe7, 0x73, 0x80, 0xf4, 0x95, 0x81, 0x20, 0xda, 0xc2, 0xc3, 0x85, 0xde, 0xca, - 0x0f, 0x4b, 0x25, 0x15, 0xe2, 0xc0, 0xaa, 0xd2, 0x90, 0x13, 0x3d, 0xd7, 0xa1, 0x2b, 0x2d, 0xb8, - 0xfe, 0xa0, 0x94, 0x27, 0x25, 0x7d, 0x82, 0x70, 0x3b, 0x64, 0x37, 0x07, 0x37, 0xc4, 0xa9, 0x12, - 0x2f, 0x39, 0x06, 0x48, 0x1b, 0x68, 0xb2, 0x93, 0x6d, 0xa8, 0x55, 0x5d, 0x7a, 0x19, 0x2b, 0x01, - 0x7d, 0x04, 0x77, 0xd5, 0x0e, 0x94, 0xa0, 0x11, 0x4b, 0x5a, 0x6f, 0xbd, 0x5d, 0x64, 0x24, 0x42, - 0xbe, 0x84, 0xb5, 0x4c, 0xcf, 0x47, 0x70, 0x72, 0x59, 0xd3, 0xa9, 0xef, 0x94, 0x70, 0x12, 0x39, - 0xe7, 0xf1, 0x33, 0x8f, 0xfa, 0x2c, 0x47, 0x1e, 0xa6, 0x16, 0x2f, 0x79, 0xe3, 0xd3, 0x3b, 0xb3, - 0xd8, 0x89, 0xd8, 0x6f, 0x60, 0x7b, 0xc6, 0x4b, 0x2a, 0xa1, 0xe9, 0xe2, 0x59, 0xcf, 0xac, 0xfa, - 0xcc, 0xb4, 0x45, 0x2b, 0x64, 0x08, 0xdb, 0x33, 0xda, 0x4d, 0x21, 0x7a, 0x7e, 0x1f, 0xab, 0x7f, - 0x3c, 0x77, 0x8e, 0xb2, 0x81, 0x56, 0xb1, 0xfd, 0xc3, 0x20, 0x7d, 0xa8, 0xc4, 0x7c, 0xb1, 0x05, - 0x13, 0xb6, 0x99, 0xdd, 0x39, 0xd2, 0x0a, 0x39, 0x85, 0x66, 0xbe, 0x4d, 0x22, 0x18, 0x9d, 0x33, - 0xba, 0x3a, 0x7d, 0xb7, 0x9c, 0xa9, 0x08, 0x24, 0xc5, 0x97, 0x72, 0x81, 0x73, 0xe6, 0x0b, 0xfa, - 0x5c, 0x13, 0x3f, 0x83, 0x95, 0xa4, 0x4d, 0x12, 0x89, 0x25, 0xdf, 0x5d, 0x89, 0xc4, 0x52, 0xe8, - 0xa5, 0x68, 0x85, 0x8c, 0x61, 0x2d, 0xd3, 0xd5, 0x88, 0xc0, 0x2c, 0x6b, 0x9b, 0x44, 0x60, 0x96, - 0xb6, 0x40, 0xf4, 0x23, 0x3c, 0x90, 0x0f, 0xf4, 0x56, 0xfe, 0x40, 0x8a, 0x4b, 0x8b, 0xa7, 0x8e, - 0x13, 0x58, 0xcf, 0x36, 0x20, 0xe2, 0x44, 0x96, 0xf6, 0x36, 0xe2, 0x44, 0x96, 0xf7, 0x2b, 0xb4, - 0x42, 0x02, 0x58, 0xcb, 0xf4, 0x11, 0x12, 0x73, 0x49, 0x6b, 0x22, 0x31, 0x97, 0x35, 0x1d, 0xf4, - 0xc7, 0x88, 0xf9, 0xf1, 0xc1, 0x27, 0x39, 0xcc, 0xb2, 0x1c, 0xe9, 0xbe, 0xe6, 0x7d, 0xcc, 0x77, - 0x71, 0x32, 0xb9, 0x4c, 0xec, 0x24, 0x92, 0x7b, 0xc6, 0x4e, 0x99, 0x5e, 0x24, 0x63, 0xa7, 0x6c, - 0xbf, 0x41, 0x3f, 0x45, 0x9d, 0x8f, 0x74, 0x3d, 0xa7, 0x53, 0x94, 0x6b, 0xdd, 0xd7, 0x9e, 0x8f, - 0x69, 0xf6, 0x37, 0x00, 0x69, 0xc1, 0x25, 0xd2, 0x6c, 0xa1, 0xe6, 0x13, 0x69, 0xb6, 0x58, 0x97, - 0xd1, 0x0e, 0xea, 0x68, 0x93, 0x56, 0xf9, 0xbe, 0xc8, 0x28, 0xf5, 0x38, 0xd6, 0x2d, 0x59, 0x8f, - 0xab, 0x85, 0x57, 0xd6, 0xe3, 0x99, 0x22, 0x87, 0xee, 0xa1, 0x16, 0x5d, 0xdf, 0xca, 0x7b, 0x1c, - 0xa7, 0xf1, 0x4d, 0x38, 0xb0, 0x96, 0x29, 0x3e, 0x84, 0x9e, 0xb2, 0xda, 0x45, 0xe8, 0x29, 0xad, - 0x54, 0xe2, 0x9b, 0x89, 0x74, 0xf2, 0x7a, 0xa6, 0x03, 0xf5, 0x72, 0x22, 0xdf, 0x02, 0xa4, 0xa5, - 0x86, 0x30, 0x59, 0xa1, 0x46, 0x11, 0x26, 0x2b, 0x56, 0x24, 0xf4, 0x63, 0x54, 0xf2, 0x90, 0xcc, - 0xbb, 0xfe, 0x9e, 0xb7, 0xff, 0xf9, 0xb6, 0xa3, 0xbd, 0x79, 0xdb, 0xd1, 0xfe, 0xf3, 0xb6, 0xa3, - 0xfd, 0xf9, 0x5d, 0xa7, 0xf2, 0xe6, 0x5d, 0xa7, 0xf2, 0xef, 0x77, 0x9d, 0xca, 0x60, 0x09, 0xff, - 0x80, 0x7d, 0xf6, 0xbf, 0x00, 0x00, 0x00, 0xff, 0xff, 0x02, 0xfe, 0xc8, 0x6e, 0x45, 0x1b, 0x00, - 0x00, + 0x11, 0x17, 0x25, 0xc5, 0x96, 0x47, 0xb1, 0x4f, 0x59, 0xdb, 0xb2, 0xcc, 0x38, 0x8a, 0x6f, 0xef, + 0x2e, 0x35, 0x8c, 0x22, 0x6e, 0x7c, 0x7d, 0x0a, 0x70, 0x05, 0x2e, 0xb6, 0xef, 0xce, 0xa8, 0x73, + 0x4e, 0xe9, 0x1a, 0xc5, 0xa1, 0x40, 0x11, 0x8a, 0x5a, 0xc9, 0x84, 0x29, 0x92, 0x21, 0x29, 0xbb, + 0x46, 0x70, 0x28, 0xd0, 0x0f, 0xd0, 0x3f, 0xe8, 0xc3, 0x3d, 0xf6, 0xa1, 0x4f, 0x05, 0xfa, 0x41, + 0xfa, 0x78, 0x40, 0x5f, 0xfa, 0x58, 0x24, 0xfd, 0x20, 0xc5, 0xce, 0x2e, 0xc9, 0xe5, 0x3f, 0x35, + 0x0a, 0x70, 0x7e, 0xe3, 0xec, 0xec, 0xce, 0xfc, 0x76, 0x66, 0x76, 0x76, 0x66, 0x09, 0x2b, 0xc3, + 0xc9, 0xc4, 0x0c, 0x23, 0x16, 0x3c, 0xf6, 0x03, 0x2f, 0xf2, 0x48, 0xdd, 0x1f, 0xe8, 0x2b, 0xc3, + 0xc9, 0xb5, 0x17, 0x5c, 0xc6, 0x63, 0xfa, 0xd6, 0xd8, 0xf3, 0xc6, 0x0e, 0xdb, 0x33, 0x7d, 0x7b, + 0xcf, 0x74, 0x5d, 0x2f, 0x32, 0x23, 0xdb, 0x73, 0x43, 0xc1, 0xa5, 0xaf, 0x60, 0xf3, 0xb9, 0x3d, + 0x0e, 0xcc, 0x88, 0xfd, 0x0a, 0x17, 0x19, 0xcc, 0x31, 0x6f, 0x0c, 0xf6, 0x6a, 0xca, 0xc2, 0x88, + 0xf4, 0x01, 0x9e, 0xd9, 0xae, 0xe3, 0x8d, 0xbf, 0x36, 0x27, 0xac, 0xa7, 0x6d, 0x6b, 0x3b, 0x4b, + 0x86, 0x32, 0x42, 0xb6, 0x60, 0x49, 0x50, 0x2f, 0xbc, 0xb0, 0x57, 0xdf, 0xd6, 0x76, 0x96, 0x8d, + 0x74, 0x80, 0x74, 0x61, 0x21, 0xf4, 0xa6, 0x81, 0xc5, 0x7a, 0x0d, 0x5c, 0x29, 0x29, 0xfa, 0x02, + 0xfa, 0xe7, 0xfe, 0x30, 0xab, 0xf1, 0xc0, 0x73, 0x47, 0xf6, 0x38, 0xd6, 0xdb, 0x85, 0x05, 0x0b, + 0x07, 0xa4, 0x4e, 0x49, 0x29, 0x12, 0xeb, 0x19, 0x89, 0x2f, 0xa1, 0x73, 0x16, 0x99, 0x41, 0xf4, + 0x4b, 0x33, 0xbc, 0x8c, 0x65, 0x10, 0x68, 0x46, 0x66, 0x78, 0x29, 0x25, 0xe0, 0x37, 0xe9, 0xc1, + 0xa2, 0x58, 0xc1, 0xd1, 0x36, 0x76, 0x96, 0x8c, 0x98, 0xe4, 0x3b, 0x0d, 0xd8, 0xc4, 0xbb, 0x62, + 0xcf, 0x59, 0x64, 0x22, 0xde, 0x96, 0xa1, 0x8c, 0xd0, 0x57, 0x70, 0x4f, 0xd1, 0x10, 0xfa, 0x9e, + 0x1b, 0x32, 0x0e, 0x27, 0x60, 0xe1, 0xd4, 0x89, 0x50, 0x49, 0xcb, 0x90, 0x14, 0xe9, 0x40, 0x63, + 0x12, 0x8e, 0x25, 0x46, 0xfe, 0x49, 0xf6, 0x53, 0xc5, 0x8d, 0xed, 0xc6, 0x4e, 0x7b, 0xbf, 0xf7, + 0xd8, 0x1f, 0x3c, 0x3e, 0xf0, 0x26, 0x13, 0xcf, 0x8d, 0xad, 0x20, 0x84, 0x26, 0x90, 0xe8, 0xa7, + 0xb0, 0x29, 0xcc, 0xf4, 0x1c, 0x3d, 0xfc, 0x4e, 0x16, 0xa2, 0x37, 0xa0, 0x97, 0x2d, 0x9a, 0x1b, + 0xf0, 0x93, 0x3c, 0xe0, 0x0d, 0x0e, 0xf8, 0x17, 0x53, 0x16, 0xdc, 0x9c, 0x45, 0x66, 0x34, 0x0d, + 0x8b, 0x78, 0x7f, 0x03, 0xe4, 0xd4, 0x67, 0x3c, 0x92, 0x54, 0x37, 0xe8, 0x50, 0xf7, 0x7c, 0x54, + 0xb7, 0xb2, 0x0f, 0x5c, 0x06, 0x67, 0x9e, 0xfa, 0x46, 0xdd, 0xf3, 0xb9, 0x8b, 0x5c, 0x1e, 0x58, + 0x42, 0x2f, 0x7e, 0xab, 0x2e, 0x6a, 0x64, 0x5c, 0x44, 0xff, 0xa8, 0xc1, 0x6a, 0x46, 0x81, 0xdc, + 0xd4, 0x2c, 0x0d, 0xe9, 0x86, 0xeb, 0x65, 0x1b, 0x6e, 0x94, 0x7a, 0xa8, 0xf9, 0xae, 0x1e, 0xfa, + 0x1c, 0xee, 0x09, 0x63, 0xbf, 0x77, 0xdc, 0xd1, 0x00, 0x88, 0x2a, 0xe2, 0x56, 0x02, 0xeb, 0x0b, + 0xe8, 0x2a, 0x8e, 0x3c, 0xb1, 0xc3, 0x48, 0xc1, 0xee, 0xa6, 0x27, 0xbd, 0xe0, 0x90, 0x1c, 0xf6, + 0x2b, 0xd8, 0x28, 0xc8, 0xb9, 0x8d, 0x40, 0x3b, 0x82, 0x75, 0xe4, 0x1f, 0x05, 0x81, 0x17, 0xbc, + 0x3f, 0xfc, 0x48, 0x9a, 0x41, 0x11, 0x33, 0x37, 0xfa, 0x9f, 0xe4, 0xd1, 0x77, 0x13, 0xf4, 0x28, + 0xb6, 0x08, 0xfe, 0x00, 0x56, 0xcf, 0x2e, 0xbc, 0xeb, 0xc3, 0xc3, 0x93, 0x13, 0xcf, 0xba, 0x0c, + 0xdf, 0x2f, 0x6a, 0xfe, 0xaa, 0xc1, 0xa2, 0x94, 0x40, 0x56, 0xa0, 0x7e, 0x7c, 0x28, 0xd7, 0xd5, + 0x8f, 0x0f, 0x13, 0x49, 0x75, 0x45, 0x12, 0x81, 0xe6, 0xc4, 0x1b, 0xc6, 0x79, 0x18, 0xbf, 0xc9, + 0x1a, 0xdc, 0xf1, 0xae, 0x5d, 0x16, 0xf4, 0x9a, 0x38, 0x28, 0x08, 0x3e, 0xf3, 0xf0, 0xf0, 0x24, + 0xec, 0xdd, 0x41, 0x85, 0xf8, 0x8d, 0x59, 0xf7, 0xc6, 0xb5, 0xd8, 0xb0, 0xb7, 0x80, 0xa3, 0x92, + 0x22, 0x3a, 0xb4, 0xa6, 0xae, 0xe4, 0x2c, 0x22, 0x27, 0xa1, 0xa9, 0x05, 0x6b, 0xd9, 0x6d, 0xce, + 0x6d, 0xda, 0x0f, 0xe1, 0x8e, 0xc3, 0x97, 0x4a, 0xc3, 0xb6, 0xb9, 0x61, 0xa5, 0x38, 0x43, 0x70, + 0xa8, 0x03, 0x6b, 0xe7, 0x2e, 0xff, 0x8c, 0xc7, 0xa5, 0x31, 0xf3, 0x26, 0xa1, 0x70, 0x37, 0x60, + 0xbe, 0x63, 0x5a, 0xec, 0x14, 0x77, 0x2c, 0xb4, 0x64, 0xc6, 0xc8, 0x36, 0xb4, 0x47, 0x5e, 0x60, + 0x31, 0x03, 0x73, 0xbe, 0xbc, 0x01, 0xd4, 0x21, 0xfa, 0x39, 0xac, 0xe7, 0xb4, 0xcd, 0xbb, 0x27, + 0xfa, 0x14, 0xfa, 0x67, 0xd7, 0x76, 0x64, 0x5d, 0x28, 0x37, 0x9f, 0x48, 0xd4, 0x31, 0x74, 0xc5, + 0xe7, 0x5a, 0xd6, 0xe7, 0xbf, 0x83, 0x87, 0x95, 0x6b, 0x6f, 0x25, 0x6d, 0x18, 0xb0, 0x29, 0xd3, + 0x6f, 0x49, 0xa5, 0x70, 0x5f, 0x49, 0xc2, 0xe8, 0x2a, 0xe4, 0xca, 0x2c, 0x5c, 0x1d, 0xc8, 0xdf, + 0x69, 0xa0, 0x97, 0x09, 0x95, 0x1b, 0x9a, 0x29, 0xf5, 0x87, 0xcd, 0xed, 0xdf, 0x69, 0xb0, 0xf1, + 0x62, 0x1a, 0x8c, 0xcb, 0x36, 0x5b, 0xe9, 0x24, 0x7e, 0x24, 0x6c, 0xd7, 0xb4, 0x22, 0xfb, 0x8a, + 0x49, 0x54, 0x09, 0x8d, 0x07, 0xd3, 0x9e, 0x88, 0xd0, 0x6a, 0x18, 0xf8, 0xcd, 0xe7, 0x8f, 0x6c, + 0x87, 0x61, 0xd6, 0x12, 0xe7, 0x30, 0xa1, 0xf1, 0xd8, 0x4d, 0x07, 0x87, 0x76, 0xd0, 0xbb, 0x23, + 0x8b, 0x1d, 0xa4, 0xe8, 0x6f, 0xa1, 0x57, 0x04, 0x76, 0x2b, 0x11, 0xf0, 0x08, 0x3a, 0x07, 0x17, + 0xcc, 0xba, 0xfc, 0x3f, 0xd7, 0x1d, 0xfd, 0x0c, 0xee, 0x29, 0xf3, 0xe6, 0x3e, 0x25, 0x27, 0xb0, + 0x26, 0x63, 0xe2, 0x0c, 0x15, 0xc7, 0xaa, 0xb6, 0x94, 0x68, 0xb8, 0xcb, 0xd1, 0x0a, 0x76, 0x1a, + 0x0e, 0xb2, 0x22, 0x12, 0x31, 0x16, 0x57, 0x44, 0x53, 0x58, 0xcf, 0x49, 0xbb, 0x15, 0x5b, 0x1d, + 0xc1, 0xba, 0xc1, 0xc6, 0x36, 0x3f, 0x9f, 0xf1, 0x94, 0x99, 0x97, 0x94, 0x39, 0x1c, 0x06, 0x2c, + 0x0c, 0xa5, 0xda, 0x98, 0xa4, 0xcf, 0xa0, 0x9b, 0x17, 0x33, 0xb7, 0x3d, 0x7f, 0x06, 0x6b, 0xa7, + 0xa3, 0x91, 0x63, 0xbb, 0xec, 0x39, 0x9b, 0x0c, 0x32, 0x48, 0xa2, 0x1b, 0x3f, 0x41, 0xc2, 0xbf, + 0xcb, 0x4a, 0x32, 0x9e, 0xf8, 0x72, 0xeb, 0xe7, 0x86, 0xf0, 0xd3, 0xc4, 0xa5, 0x27, 0xcc, 0x1c, + 0xa6, 0x10, 0x0a, 0x2e, 0x15, 0x6c, 0xe1, 0x52, 0x54, 0x9c, 0x5d, 0x35, 0xb7, 0xe2, 0x3f, 0x68, + 0x00, 0x22, 0x4b, 0x1e, 0xbb, 0x23, 0xaf, 0xd4, 0xf8, 0x3a, 0xb4, 0x26, 0xb8, 0xaf, 0xe3, 0x43, + 0x5c, 0xd9, 0x34, 0x12, 0x9a, 0x5f, 0x92, 0xa6, 0x63, 0x27, 0xf7, 0x81, 0x20, 0xf8, 0x0a, 0x9f, + 0xb1, 0xe0, 0xdc, 0x38, 0x11, 0x09, 0x65, 0xc9, 0x48, 0x68, 0xde, 0x48, 0x58, 0x8e, 0xcd, 0xdc, + 0x08, 0xb9, 0xe2, 0x1a, 0x55, 0x46, 0xe8, 0x00, 0x40, 0x38, 0xb2, 0x12, 0x0f, 0x81, 0x26, 0xf7, + 0x7e, 0xec, 0x02, 0xfe, 0xcd, 0x71, 0x84, 0x91, 0x39, 0x8e, 0x6f, 0x70, 0x41, 0x28, 0xed, 0x50, + 0x33, 0xd3, 0x0e, 0x9d, 0x40, 0x87, 0xd7, 0x33, 0xc2, 0x68, 0xc2, 0x67, 0xb1, 0x69, 0xb4, 0x34, + 0xaa, 0xcb, 0xaa, 0xef, 0x58, 0x77, 0x23, 0xd5, 0x4d, 0xbf, 0x16, 0xd2, 0x84, 0x15, 0x2b, 0xa5, + 0xed, 0xc0, 0xa2, 0xe8, 0x44, 0x45, 0x8e, 0x6f, 0xef, 0xaf, 0x70, 0x77, 0xa6, 0xa6, 0x37, 0x62, + 0x76, 0x2c, 0x4f, 0x58, 0x61, 0x96, 0x3c, 0xd1, 0xc5, 0x66, 0xe4, 0xa5, 0xa6, 0x33, 0x62, 0x36, + 0xfd, 0x9b, 0x06, 0x8b, 0x42, 0x4c, 0x48, 0x1e, 0xc3, 0x82, 0x83, 0xbb, 0x46, 0x51, 0xed, 0xfd, + 0x35, 0x8c, 0xa9, 0x9c, 0x2d, 0xbe, 0xaa, 0x19, 0x72, 0x16, 0x9f, 0x2f, 0x60, 0xa1, 0x15, 0x94, + 0xf9, 0xea, 0x6e, 0xf9, 0x7c, 0x31, 0x8b, 0xcf, 0x17, 0x6a, 0xd1, 0x42, 0xca, 0x7c, 0x75, 0x37, + 0x7c, 0xbe, 0x98, 0xf5, 0xac, 0x05, 0x0b, 0x22, 0x96, 0x78, 0x03, 0x89, 0x72, 0x33, 0x27, 0xb0, + 0x9b, 0x81, 0xdb, 0x4a, 0x60, 0x75, 0x33, 0xb0, 0x5a, 0x89, 0xfa, 0x6e, 0x46, 0x7d, 0x2b, 0x56, + 0xc3, 0xc3, 0x83, 0xbb, 0x2f, 0x8e, 0x46, 0x41, 0x50, 0x06, 0x44, 0x55, 0x39, 0x77, 0xda, 0xfb, + 0x04, 0x16, 0x05, 0xf8, 0x4c, 0x0d, 0x26, 0x4d, 0x6d, 0xc4, 0x3c, 0xfa, 0x0f, 0x2d, 0xcd, 0xd7, + 0xd6, 0x05, 0x9b, 0x98, 0xd5, 0xf9, 0x1a, 0xd9, 0x69, 0xf3, 0x57, 0xa8, 0x53, 0x2b, 0x9b, 0x3f, + 0x7e, 0xe4, 0x86, 0x66, 0x64, 0x0e, 0xcc, 0x30, 0xb9, 0x28, 0x63, 0x9a, 0xef, 0x3e, 0x32, 0x07, + 0x0e, 0x93, 0xf7, 0xa4, 0x20, 0xf0, 0x70, 0xa0, 0xbe, 0xde, 0x82, 0x3c, 0x1c, 0x48, 0xa9, 0xf7, + 0x81, 0x44, 0x7b, 0x2b, 0xf7, 0xc1, 0x2e, 0xac, 0x7d, 0xc9, 0xa2, 0xb3, 0xe9, 0x80, 0x5f, 0x8a, + 0x07, 0xa3, 0xf1, 0x8c, 0xeb, 0x80, 0x9e, 0xc3, 0x7a, 0x6e, 0xee, 0xdc, 0x10, 0x09, 0x34, 0xad, + 0xd1, 0x38, 0x36, 0x23, 0x7e, 0xd3, 0x1f, 0xc1, 0xbd, 0x2f, 0x59, 0xf4, 0x0e, 0xfa, 0x5f, 0x00, + 0x51, 0x27, 0xce, 0xad, 0xbc, 0x03, 0x0d, 0x6b, 0x94, 0x54, 0x60, 0xd6, 0x68, 0x4c, 0xff, 0xac, + 0x01, 0xf9, 0xca, 0x74, 0x87, 0x0e, 0x93, 0x6d, 0x51, 0x45, 0xd5, 0x88, 0xdc, 0xf7, 0x0a, 0x90, + 0x2d, 0x58, 0x1a, 0x24, 0x4f, 0x51, 0x22, 0x42, 0xd2, 0x01, 0x2e, 0x2b, 0x7c, 0xe5, 0x24, 0x6d, + 0x0d, 0xff, 0xa6, 0x21, 0xac, 0x66, 0x20, 0xdd, 0x46, 0x18, 0xec, 0x0e, 0xa0, 0x15, 0x57, 0x2d, + 0x64, 0x15, 0x3e, 0x38, 0x76, 0xaf, 0x4c, 0xc7, 0x1e, 0xc6, 0x43, 0x9d, 0x1a, 0xf9, 0x00, 0xda, + 0xf8, 0xd0, 0x24, 0x86, 0x3a, 0x1a, 0xe9, 0xc0, 0x5d, 0xf1, 0x42, 0x20, 0x47, 0xea, 0x64, 0x05, + 0xe0, 0x2c, 0xf2, 0x7c, 0x49, 0x37, 0x90, 0xbe, 0xf0, 0xae, 0x25, 0xdd, 0xdc, 0xfd, 0x39, 0xb4, + 0xe2, 0x6b, 0x54, 0xd1, 0x11, 0x0f, 0x75, 0x6a, 0xe4, 0x1e, 0x2c, 0x1f, 0x5d, 0xd9, 0x56, 0x94, + 0x0c, 0x69, 0x64, 0x03, 0x56, 0x0f, 0x4c, 0xd7, 0x62, 0x4e, 0x96, 0x51, 0xdf, 0xff, 0x7b, 0x07, + 0x16, 0x44, 0x32, 0x24, 0xdf, 0xc0, 0x52, 0xf2, 0x06, 0x46, 0x30, 0xf3, 0xe5, 0x1f, 0xdd, 0xf4, + 0xf5, 0xdc, 0xa8, 0xd8, 0x3e, 0x7d, 0xf8, 0xfb, 0x7f, 0xfd, 0xf7, 0x2f, 0xf5, 0x4d, 0xba, 0xb6, + 0x67, 0xfa, 0x76, 0xb8, 0x77, 0xf5, 0xc4, 0x74, 0xfc, 0x0b, 0xf3, 0xc9, 0x1e, 0xf7, 0x69, 0xf8, + 0x54, 0xdb, 0x25, 0x23, 0x68, 0x2b, 0x4f, 0x3b, 0x04, 0xbb, 0xe8, 0xe2, 0x63, 0x92, 0xbe, 0x51, + 0x18, 0x97, 0x0a, 0x1e, 0xa1, 0x82, 0x6d, 0xfd, 0x7e, 0x99, 0x82, 0xbd, 0xd7, 0x3c, 0xaa, 0xbf, + 0xe5, 0x7a, 0x3e, 0x03, 0x48, 0x9f, 0x5b, 0x08, 0xa2, 0x2d, 0xbc, 0xe0, 0xe8, 0xdd, 0xfc, 0xb0, + 0x54, 0x52, 0x23, 0x0e, 0xb4, 0x95, 0x97, 0x09, 0xa2, 0xe7, 0x9e, 0x2a, 0x94, 0xb7, 0x08, 0xfd, + 0x7e, 0x29, 0x4f, 0x4a, 0xfa, 0x18, 0xe1, 0xf6, 0xc9, 0x56, 0x0e, 0x6e, 0x88, 0x53, 0x25, 0x5e, + 0x72, 0x04, 0x90, 0xbe, 0x24, 0x90, 0xcd, 0xec, 0xcb, 0x82, 0xaa, 0x4b, 0x2f, 0x63, 0x25, 0xa0, + 0x0f, 0xe0, 0xae, 0xda, 0x8a, 0x13, 0x34, 0x62, 0xc9, 0x1b, 0x84, 0xde, 0x2b, 0x32, 0x12, 0x21, + 0x5f, 0xc0, 0x72, 0xa6, 0xf9, 0x25, 0x38, 0xb9, 0xac, 0xfb, 0xd6, 0x37, 0x4b, 0x38, 0x89, 0x9c, + 0xf3, 0xf8, 0xbd, 0x4b, 0x7d, 0x9f, 0x24, 0x0f, 0x52, 0x8b, 0x97, 0x3c, 0x76, 0xea, 0xfd, 0x2a, + 0x76, 0x22, 0xf6, 0x1b, 0xd8, 0xa8, 0x78, 0x52, 0x26, 0x34, 0x5d, 0x5c, 0xf5, 0xde, 0xac, 0x57, + 0x1e, 0x5c, 0x5a, 0x23, 0x43, 0xd8, 0xa8, 0xe8, 0xbb, 0x85, 0xe8, 0xd9, 0x0d, 0xbd, 0xfe, 0xd1, + 0xcc, 0x39, 0xca, 0x06, 0xba, 0xc5, 0x3e, 0x18, 0x83, 0xf4, 0x81, 0x12, 0xf3, 0xc5, 0x5e, 0x54, + 0xd8, 0xa6, 0xba, 0x85, 0xa6, 0x35, 0x72, 0x0a, 0x9d, 0x7c, 0xbf, 0x48, 0x30, 0x3a, 0x2b, 0xda, + 0x5b, 0x7d, 0xab, 0x9c, 0xa9, 0x08, 0x24, 0xc5, 0x5f, 0x06, 0x02, 0x67, 0xe5, 0xaf, 0x84, 0x99, + 0x26, 0x7e, 0x0a, 0x4b, 0x49, 0xbf, 0x28, 0x12, 0x4b, 0xbe, 0xcd, 0x14, 0x89, 0xa5, 0xd0, 0x54, + 0xd2, 0x1a, 0x19, 0xc3, 0x72, 0xa6, 0xbd, 0x13, 0x81, 0x59, 0xd6, 0x3f, 0x8a, 0xc0, 0x2c, 0xed, + 0x05, 0xe9, 0x87, 0x78, 0x20, 0xef, 0xeb, 0xdd, 0xfc, 0x81, 0x14, 0x69, 0x9b, 0xa7, 0x8e, 0x63, + 0x58, 0xc9, 0x76, 0x62, 0xe2, 0x44, 0x96, 0x36, 0x79, 0xe2, 0x44, 0x96, 0x37, 0x6e, 0xb4, 0x46, + 0x02, 0x58, 0xce, 0x34, 0x54, 0x12, 0x73, 0x49, 0x8f, 0x26, 0x31, 0x97, 0x75, 0x5f, 0xf4, 0xc7, + 0x88, 0xf9, 0xd1, 0xee, 0xc7, 0x39, 0xcc, 0xb2, 0x2e, 0xdb, 0x7b, 0xcd, 0x1b, 0xba, 0x6f, 0xe3, + 0x64, 0x72, 0x99, 0xd8, 0x49, 0x24, 0xf7, 0x8c, 0x9d, 0x32, 0x4d, 0x59, 0xc6, 0x4e, 0xd9, 0xc6, + 0x8b, 0x7e, 0x82, 0x3a, 0x1f, 0xea, 0x7a, 0x4e, 0xa7, 0xa8, 0x5b, 0xf7, 0x5e, 0x7b, 0x3e, 0xa6, + 0xd9, 0x5f, 0x03, 0xa4, 0x95, 0xa7, 0x48, 0xb3, 0x85, 0xe2, 0x57, 0xa4, 0xd9, 0x62, 0x81, 0x4a, + 0xfb, 0xa8, 0xa3, 0x47, 0xba, 0xe5, 0xfb, 0x22, 0xa3, 0xd4, 0xe3, 0x58, 0xc0, 0x65, 0x3d, 0xae, + 0x56, 0xa0, 0x59, 0x8f, 0x67, 0xaa, 0x3d, 0xba, 0x8d, 0x5a, 0x74, 0x7d, 0x3d, 0xef, 0x71, 0x9c, + 0xc6, 0x37, 0xe1, 0xc0, 0x72, 0xa6, 0x0a, 0x13, 0x7a, 0xca, 0x8a, 0x38, 0xa1, 0xa7, 0xb4, 0x64, + 0x8b, 0x6f, 0x26, 0xd2, 0xcf, 0xeb, 0x99, 0x0e, 0xd4, 0xcb, 0x89, 0xbc, 0x04, 0x48, 0x6b, 0x2e, + 0x61, 0xb2, 0x42, 0xb1, 0x26, 0x4c, 0x56, 0x2c, 0xcd, 0xe8, 0x47, 0xa8, 0xe4, 0x01, 0x99, 0x75, + 0xfd, 0x91, 0x97, 0xd0, 0x56, 0xea, 0x1d, 0x71, 0xc7, 0x16, 0x6b, 0x32, 0x71, 0xc7, 0x96, 0x14, + 0x46, 0x95, 0x16, 0x63, 0x7c, 0x16, 0x3f, 0x22, 0xcf, 0x7a, 0xff, 0x7c, 0xd3, 0xd7, 0xbe, 0x7f, + 0xd3, 0xd7, 0xfe, 0xf3, 0xa6, 0xaf, 0xfd, 0xe9, 0x6d, 0xbf, 0xf6, 0xfd, 0xdb, 0x7e, 0xed, 0xdf, + 0x6f, 0xfb, 0xb5, 0xc1, 0x02, 0xfe, 0x6c, 0xfc, 0xf4, 0x7f, 0x01, 0x00, 0x00, 0xff, 0xff, 0x37, + 0xd7, 0x5e, 0xdf, 0xb0, 0x1c, 0x00, 0x00, } // Reference imports to suppress errors if they are not otherwise used. @@ -3228,6 +3370,7 @@ type MasterClient interface { GetSubTaskCfg(ctx context.Context, in *GetSubTaskCfgRequest, opts ...grpc.CallOption) (*GetSubTaskCfgResponse, error) // GetTaskCfg get task config GetTaskCfg(ctx context.Context, in *GetTaskCfgRequest, opts ...grpc.CallOption) (*GetTaskCfgResponse, error) + HandleError(ctx context.Context, in *HandleErrorRequest, opts ...grpc.CallOption) (*HandleErrorResponse, error) } type masterClient struct { @@ -3436,6 +3579,15 @@ func (c *masterClient) GetTaskCfg(ctx context.Context, in *GetTaskCfgRequest, op return out, nil } +func (c *masterClient) HandleError(ctx context.Context, in *HandleErrorRequest, opts ...grpc.CallOption) (*HandleErrorResponse, error) { + out := new(HandleErrorResponse) + err := c.cc.Invoke(ctx, "/pb.Master/HandleError", in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + // MasterServer is the server API for Master service. type MasterServer interface { StartTask(context.Context, *StartTaskRequest) (*StartTaskResponse, error) @@ -3475,6 +3627,7 @@ type MasterServer interface { GetSubTaskCfg(context.Context, *GetSubTaskCfgRequest) (*GetSubTaskCfgResponse, error) // GetTaskCfg get task config GetTaskCfg(context.Context, *GetTaskCfgRequest) (*GetTaskCfgResponse, error) + HandleError(context.Context, *HandleErrorRequest) (*HandleErrorResponse, error) } // UnimplementedMasterServer can be embedded to have forward compatible implementations. @@ -3547,6 +3700,9 @@ func (*UnimplementedMasterServer) GetSubTaskCfg(ctx context.Context, req *GetSub func (*UnimplementedMasterServer) GetTaskCfg(ctx context.Context, req *GetTaskCfgRequest) (*GetTaskCfgResponse, error) { return nil, status.Errorf(codes.Unimplemented, "method GetTaskCfg not implemented") } +func (*UnimplementedMasterServer) HandleError(ctx context.Context, req *HandleErrorRequest) (*HandleErrorResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method HandleError not implemented") +} func RegisterMasterServer(s *grpc.Server, srv MasterServer) { s.RegisterService(&_Master_serviceDesc, srv) @@ -3948,6 +4104,24 @@ func _Master_GetTaskCfg_Handler(srv interface{}, ctx context.Context, dec func(i return interceptor(ctx, in, info, handler) } +func _Master_HandleError_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(HandleErrorRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(MasterServer).HandleError(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: "/pb.Master/HandleError", + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(MasterServer).HandleError(ctx, req.(*HandleErrorRequest)) + } + return interceptor(ctx, in, info, handler) +} + var _Master_serviceDesc = grpc.ServiceDesc{ ServiceName: "pb.Master", HandlerType: (*MasterServer)(nil), @@ -4040,6 +4214,10 @@ var _Master_serviceDesc = grpc.ServiceDesc{ MethodName: "GetTaskCfg", Handler: _Master_GetTaskCfg_Handler, }, + { + MethodName: "HandleError", + Handler: _Master_HandleError_Handler, + }, }, Streams: []grpc.StreamDesc{}, Metadata: "dmmaster.proto", @@ -6363,6 +6541,120 @@ func (m *GetTaskCfgResponse) MarshalToSizedBuffer(dAtA []byte) (int, error) { return len(dAtA) - i, nil } +func (m *HandleErrorRequest) 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 *HandleErrorRequest) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *HandleErrorRequest) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + _ = i + var l int + _ = l + if len(m.Sqls) > 0 { + for iNdEx := len(m.Sqls) - 1; iNdEx >= 0; iNdEx-- { + i -= len(m.Sqls[iNdEx]) + copy(dAtA[i:], m.Sqls[iNdEx]) + i = encodeVarintDmmaster(dAtA, i, uint64(len(m.Sqls[iNdEx]))) + i-- + dAtA[i] = 0x2a + } + } + 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 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.Task) > 0 { + i -= len(m.Task) + copy(dAtA[i:], m.Task) + i = encodeVarintDmmaster(dAtA, i, uint64(len(m.Task))) + i-- + dAtA[i] = 0x12 + } + if m.Op != 0 { + i = encodeVarintDmmaster(dAtA, i, uint64(m.Op)) + i-- + dAtA[i] = 0x8 + } + return len(dAtA) - i, nil +} + +func (m *HandleErrorResponse) 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 *HandleErrorResponse) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *HandleErrorResponse) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + _ = i + var l int + _ = l + if len(m.Sources) > 0 { + for iNdEx := len(m.Sources) - 1; iNdEx >= 0; iNdEx-- { + { + size, err := m.Sources[iNdEx].MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintDmmaster(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 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 { + i-- + if m.Result { + dAtA[i] = 1 + } else { + dAtA[i] = 0 + } + i-- + dAtA[i] = 0x8 + } + return len(dAtA) - i, nil +} + func encodeVarintDmmaster(dAtA []byte, offset int, v uint64) int { offset -= sovDmmaster(v) base := offset @@ -7407,35 +7699,89 @@ func (m *GetTaskCfgResponse) Size() (n int) { return n } -func sovDmmaster(x uint64) (n int) { - return (math_bits.Len64(x|1) + 6) / 7 -} -func sozDmmaster(x uint64) (n int) { - return sovDmmaster(uint64((x << 1) ^ uint64((int64(x) >> 63)))) -} -func (m *MigrateWorkerRelayRequest) 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 - } +func (m *HandleErrorRequest) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Op != 0 { + n += 1 + sovDmmaster(uint64(m.Op)) + } + l = len(m.Task) + 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)) } - fieldNum := int32(wire >> 3) - wireType := int(wire & 0x7) - if wireType == 4 { + } + l = len(m.BinlogPos) + if l > 0 { + n += 1 + l + sovDmmaster(uint64(l)) + } + if len(m.Sqls) > 0 { + for _, s := range m.Sqls { + l = len(s) + n += 1 + l + sovDmmaster(uint64(l)) + } + } + return n +} + +func (m *HandleErrorResponse) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Result { + n += 2 + } + l = len(m.Msg) + if l > 0 { + n += 1 + l + sovDmmaster(uint64(l)) + } + if len(m.Sources) > 0 { + for _, e := range m.Sources { + l = e.Size() + n += 1 + l + sovDmmaster(uint64(l)) + } + } + return n +} + +func sovDmmaster(x uint64) (n int) { + return (math_bits.Len64(x|1) + 6) / 7 +} +func sozDmmaster(x uint64) (n int) { + return sovDmmaster(uint64((x << 1) ^ uint64((int64(x) >> 63)))) +} +func (m *MigrateWorkerRelayRequest) 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: MigrateWorkerRelayRequest: wiretype end group for non-group") } if fieldNum <= 0 { @@ -13883,6 +14229,345 @@ func (m *GetTaskCfgResponse) Unmarshal(dAtA []byte) error { } return nil } +func (m *HandleErrorRequest) 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: HandleErrorRequest: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: HandleErrorRequest: 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 |= ErrorOp(b&0x7F) << shift + if b < 0x80 { + break + } + } + 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 Sources", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowDmmaster + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLengthDmmaster + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthDmmaster + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Sources = append(m.Sources, string(dAtA[iNdEx:postIndex])) + iNdEx = postIndex + case 4: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field BinlogPos", 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.BinlogPos = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 5: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Sqls", 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.Sqls = append(m.Sqls, string(dAtA[iNdEx:postIndex])) + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipDmmaster(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthDmmaster + } + if (iNdEx + skippy) < 0 { + return ErrInvalidLengthDmmaster + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *HandleErrorResponse) 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: HandleErrorResponse: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: HandleErrorResponse: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Result", wireType) + } + var v int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowDmmaster + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + m.Result = bool(v != 0) + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Msg", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowDmmaster + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLengthDmmaster + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthDmmaster + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Msg = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Sources", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowDmmaster + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthDmmaster + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthDmmaster + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Sources = append(m.Sources, &CommonWorkerResponse{}) + if err := m.Sources[len(m.Sources)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipDmmaster(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthDmmaster + } + if (iNdEx + skippy) < 0 { + return ErrInvalidLengthDmmaster + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} func skipDmmaster(dAtA []byte) (n int, err error) { l := len(dAtA) iNdEx := 0 diff --git a/dm/pb/dmmaster.pb.gw.go b/dm/pb/dmmaster.pb.gw.go index 93d75c2371..2b77fd8033 100644 --- a/dm/pb/dmmaster.pb.gw.go +++ b/dm/pb/dmmaster.pb.gw.go @@ -565,6 +565,40 @@ func local_request_Master_GetTaskCfg_0(ctx context.Context, marshaler runtime.Ma } +func request_Master_HandleError_0(ctx context.Context, marshaler runtime.Marshaler, client MasterClient, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq HandleErrorRequest + 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 := client.HandleError(ctx, &protoReq, grpc.Header(&metadata.HeaderMD), grpc.Trailer(&metadata.TrailerMD)) + return msg, metadata, err + +} + +func local_request_Master_HandleError_0(ctx context.Context, marshaler runtime.Marshaler, server MasterServer, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq HandleErrorRequest + 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.HandleError(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. @@ -770,6 +804,26 @@ func RegisterMasterHandlerServer(ctx context.Context, mux *runtime.ServeMux, ser }) + mux.Handle("PUT", pattern_Master_HandleError_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_HandleError_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_HandleError_0(ctx, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + return nil } @@ -1011,6 +1065,26 @@ func RegisterMasterHandlerClient(ctx context.Context, mux *runtime.ServeMux, cli }) + mux.Handle("PUT", pattern_Master_HandleError_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.AnnotateContext(ctx, mux, req) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := request_Master_HandleError_0(rctx, inboundMarshaler, client, req, pathParams) + ctx = runtime.NewServerMetadataContext(ctx, md) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + + forward_Master_HandleError_0(ctx, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + return nil } @@ -1034,6 +1108,8 @@ var ( pattern_Master_GetSubTaskCfg_0 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 2, 1, 2, 2, 1, 0, 4, 1, 5, 3}, []string{"apis", "v1alpha1", "subtasks", "name"}, "", runtime.AssumeColonVerbOpt(true))) pattern_Master_GetTaskCfg_0 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 2, 1, 2, 2, 1, 0, 4, 1, 5, 3}, []string{"apis", "v1alpha1", "tasks", "name"}, "", runtime.AssumeColonVerbOpt(true))) + + pattern_Master_HandleError_0 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 2, 1, 2, 2}, []string{"apis", "v1alpha1", "errors"}, "", runtime.AssumeColonVerbOpt(true))) ) var ( @@ -1056,4 +1132,6 @@ var ( forward_Master_GetSubTaskCfg_0 = runtime.ForwardResponseMessage forward_Master_GetTaskCfg_0 = runtime.ForwardResponseMessage + + forward_Master_HandleError_0 = runtime.ForwardResponseMessage ) diff --git a/dm/pb/dmworker.pb.go b/dm/pb/dmworker.pb.go index 4269dd8a7f..88abfdccb7 100644 --- a/dm/pb/dmworker.pb.go +++ b/dm/pb/dmworker.pb.go @@ -233,6 +233,34 @@ func (SchemaOp) EnumDescriptor() ([]byte, []int) { return fileDescriptor_51a1b9e17fd67b10, []int{4} } +type ErrorOp int32 + +const ( + ErrorOp_Skip ErrorOp = 0 + ErrorOp_Replace ErrorOp = 1 + ErrorOp_InvalidErrorOp ErrorOp = 2 +) + +var ErrorOp_name = map[int32]string{ + 0: "Skip", + 1: "Replace", + 2: "InvalidErrorOp", +} + +var ErrorOp_value = map[string]int32{ + "Skip": 0, + "Replace": 1, + "InvalidErrorOp": 2, +} + +func (x ErrorOp) String() string { + return proto.EnumName(ErrorOp_name, int32(x)) +} + +func (ErrorOp) EnumDescriptor() ([]byte, []int) { + return fileDescriptor_51a1b9e17fd67b10, []int{5} +} + type StartSubTaskRequest struct { Task string `protobuf:"bytes,1,opt,name=task,proto3" json:"task,omitempty"` } @@ -2747,12 +2775,81 @@ func (m *OperateWorkerSchemaRequest) GetSchema() string { return "" } +type HandleWorkerErrorRequest struct { + Op ErrorOp `protobuf:"varint,1,opt,name=op,proto3,enum=pb.ErrorOp" json:"op,omitempty"` + Task string `protobuf:"bytes,2,opt,name=task,proto3" json:"task,omitempty"` + BinlogPos string `protobuf:"bytes,3,opt,name=binlogPos,proto3" json:"binlogPos,omitempty"` + Sqls []string `protobuf:"bytes,4,rep,name=sqls,proto3" json:"sqls,omitempty"` +} + +func (m *HandleWorkerErrorRequest) Reset() { *m = HandleWorkerErrorRequest{} } +func (m *HandleWorkerErrorRequest) String() string { return proto.CompactTextString(m) } +func (*HandleWorkerErrorRequest) ProtoMessage() {} +func (*HandleWorkerErrorRequest) Descriptor() ([]byte, []int) { + return fileDescriptor_51a1b9e17fd67b10, []int{38} +} +func (m *HandleWorkerErrorRequest) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *HandleWorkerErrorRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_HandleWorkerErrorRequest.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 *HandleWorkerErrorRequest) XXX_Merge(src proto.Message) { + xxx_messageInfo_HandleWorkerErrorRequest.Merge(m, src) +} +func (m *HandleWorkerErrorRequest) XXX_Size() int { + return m.Size() +} +func (m *HandleWorkerErrorRequest) XXX_DiscardUnknown() { + xxx_messageInfo_HandleWorkerErrorRequest.DiscardUnknown(m) +} + +var xxx_messageInfo_HandleWorkerErrorRequest proto.InternalMessageInfo + +func (m *HandleWorkerErrorRequest) GetOp() ErrorOp { + if m != nil { + return m.Op + } + return ErrorOp_Skip +} + +func (m *HandleWorkerErrorRequest) GetTask() string { + if m != nil { + return m.Task + } + return "" +} + +func (m *HandleWorkerErrorRequest) GetBinlogPos() string { + if m != nil { + return m.BinlogPos + } + return "" +} + +func (m *HandleWorkerErrorRequest) GetSqls() []string { + if m != nil { + return m.Sqls + } + return nil +} + func init() { proto.RegisterEnum("pb.TaskOp", TaskOp_name, TaskOp_value) proto.RegisterEnum("pb.Stage", Stage_name, Stage_value) proto.RegisterEnum("pb.UnitType", UnitType_name, UnitType_value) proto.RegisterEnum("pb.RelayOp", RelayOp_name, RelayOp_value) proto.RegisterEnum("pb.SchemaOp", SchemaOp_name, SchemaOp_value) + proto.RegisterEnum("pb.ErrorOp", ErrorOp_name, ErrorOp_value) proto.RegisterType((*StartSubTaskRequest)(nil), "pb.StartSubTaskRequest") proto.RegisterType((*UpdateRelayRequest)(nil), "pb.UpdateRelayRequest") proto.RegisterType((*MigrateRelayRequest)(nil), "pb.MigrateRelayRequest") @@ -2791,140 +2888,146 @@ func init() { proto.RegisterType((*QueryWorkerConfigRequest)(nil), "pb.QueryWorkerConfigRequest") proto.RegisterType((*QueryWorkerConfigResponse)(nil), "pb.QueryWorkerConfigResponse") proto.RegisterType((*OperateWorkerSchemaRequest)(nil), "pb.OperateWorkerSchemaRequest") + proto.RegisterType((*HandleWorkerErrorRequest)(nil), "pb.HandleWorkerErrorRequest") } func init() { proto.RegisterFile("dmworker.proto", fileDescriptor_51a1b9e17fd67b10) } var fileDescriptor_51a1b9e17fd67b10 = []byte{ - // 2037 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xb4, 0x18, 0x4d, 0x6f, 0xdb, 0xc8, - 0x55, 0x24, 0x25, 0x59, 0x7a, 0x92, 0x1c, 0x7a, 0xec, 0xf5, 0x6a, 0xd5, 0xac, 0x6a, 0x30, 0x8b, - 0x5d, 0xaf, 0x0f, 0x46, 0xd7, 0x6d, 0x51, 0xa0, 0xc0, 0xf6, 0x23, 0x72, 0xea, 0x04, 0x95, 0x37, - 0x0e, 0x95, 0xa0, 0xc7, 0x82, 0x22, 0xc7, 0x32, 0x61, 0x89, 0x64, 0xf8, 0x61, 0xc3, 0x7f, 0xa2, - 0xed, 0xa5, 0x87, 0x02, 0xbd, 0x15, 0x45, 0x6f, 0x45, 0xff, 0x45, 0x7b, 0x5c, 0xf4, 0xd4, 0x63, - 0x91, 0xf4, 0x67, 0xf4, 0x50, 0xbc, 0x37, 0x43, 0x72, 0x68, 0x7d, 0x64, 0x03, 0x74, 0x6f, 0xf3, - 0x3e, 0xe6, 0xcd, 0xfb, 0x9e, 0x99, 0x07, 0xdb, 0xde, 0xe2, 0x36, 0x8c, 0xaf, 0x79, 0x7c, 0x1c, - 0xc5, 0x61, 0x1a, 0x32, 0x3d, 0x9a, 0x5a, 0x9f, 0xc3, 0xee, 0x24, 0x75, 0xe2, 0x74, 0x92, 0x4d, - 0x5f, 0x3a, 0xc9, 0xb5, 0xcd, 0x5f, 0x67, 0x3c, 0x49, 0x19, 0x83, 0x7a, 0xea, 0x24, 0xd7, 0x7d, - 0xed, 0x40, 0x3b, 0x6c, 0xdb, 0xb4, 0xb6, 0x8e, 0x81, 0xbd, 0x8a, 0x3c, 0x27, 0xe5, 0x36, 0x9f, - 0x3b, 0x77, 0x39, 0x67, 0x1f, 0xb6, 0xdc, 0x30, 0x48, 0x79, 0x90, 0x4a, 0xe6, 0x1c, 0xb4, 0x26, - 0xb0, 0x7b, 0xee, 0xcf, 0xe2, 0xfb, 0x1b, 0x86, 0x00, 0x8f, 0xfd, 0x60, 0x1e, 0xce, 0xbe, 0x72, - 0x16, 0x5c, 0xee, 0x51, 0x30, 0xec, 0x21, 0xb4, 0x05, 0x74, 0x11, 0x26, 0x7d, 0xfd, 0x40, 0x3b, - 0xec, 0xd9, 0x25, 0xc2, 0x3a, 0x83, 0x0f, 0x9e, 0x47, 0x1c, 0x85, 0xde, 0xd3, 0x78, 0x00, 0x7a, - 0x18, 0x91, 0xb8, 0xed, 0x13, 0x38, 0x8e, 0xa6, 0xc7, 0x48, 0x7c, 0x1e, 0xd9, 0x7a, 0x18, 0xa1, - 0x35, 0x01, 0x1e, 0xa6, 0x0b, 0x6b, 0x70, 0x6d, 0xdd, 0xc0, 0xfe, 0x7d, 0x41, 0x49, 0x14, 0x06, - 0x09, 0xdf, 0x28, 0x69, 0x1f, 0x9a, 0x31, 0x4f, 0xb2, 0x79, 0x4a, 0xb2, 0x5a, 0xb6, 0x84, 0x10, - 0x9f, 0x84, 0x59, 0xec, 0xf2, 0xbe, 0x41, 0x67, 0x48, 0x88, 0x99, 0x60, 0x2c, 0x92, 0x59, 0xbf, - 0x4e, 0x48, 0x5c, 0x5a, 0x47, 0xb0, 0x27, 0xbc, 0xf8, 0x0d, 0x3c, 0x7e, 0x08, 0xec, 0x45, 0xc6, - 0xe3, 0xbb, 0x49, 0xea, 0xa4, 0x59, 0xa2, 0x70, 0x06, 0xa5, 0xeb, 0x84, 0x35, 0x9f, 0xc1, 0x0e, - 0x71, 0x3e, 0x89, 0xe3, 0x30, 0xde, 0xc4, 0x18, 0xc1, 0xde, 0x28, 0x5c, 0x2c, 0xc2, 0xe0, 0x57, - 0x94, 0x09, 0x85, 0xd1, 0xa5, 0x61, 0x5a, 0xc5, 0x30, 0x69, 0x80, 0x5e, 0x18, 0xb0, 0xd6, 0xd4, - 0x7d, 0x68, 0x8a, 0xec, 0x92, 0xd6, 0x4a, 0xc8, 0xfa, 0xab, 0x06, 0xbb, 0x15, 0x2b, 0xde, 0xfb, - 0xc4, 0x1f, 0x40, 0x57, 0x9c, 0x21, 0x24, 0xd0, 0xb9, 0x9d, 0x13, 0x13, 0x43, 0x33, 0x51, 0xf0, - 0x76, 0x85, 0x8b, 0xfd, 0x08, 0x7a, 0x89, 0x70, 0xb1, 0xdc, 0x56, 0x3f, 0x30, 0x0e, 0x3b, 0x27, - 0x3b, 0xb4, 0x4d, 0x25, 0xd8, 0x55, 0x3e, 0xeb, 0x2f, 0x9a, 0x74, 0xbb, 0x74, 0xe6, 0x7b, 0xeb, - 0xfb, 0x05, 0x74, 0x84, 0x5e, 0x24, 0x40, 0xaa, 0xfb, 0xa0, 0x54, 0x57, 0xc8, 0x55, 0x79, 0xc8, - 0x44, 0xa1, 0x84, 0xd8, 0x23, 0x74, 0x35, 0x15, 0x5d, 0xc5, 0xa6, 0x0a, 0x97, 0xf5, 0x67, 0x0d, - 0x3a, 0xa3, 0x2b, 0xee, 0x4a, 0xcd, 0x51, 0xc5, 0xc8, 0x49, 0x12, 0xee, 0xe5, 0x2a, 0x0a, 0x88, - 0xed, 0x41, 0x23, 0x0d, 0x53, 0x67, 0x4e, 0x4a, 0x36, 0x6c, 0x01, 0x60, 0x21, 0x26, 0x99, 0xeb, - 0xf2, 0x24, 0xb9, 0xcc, 0xe6, 0xa4, 0x65, 0xc3, 0x56, 0x30, 0x28, 0xed, 0xd2, 0xf1, 0xe7, 0xdc, - 0xa3, 0x80, 0x36, 0x6c, 0x09, 0x61, 0xc5, 0xdf, 0x3a, 0x71, 0xe0, 0x07, 0xb3, 0x7e, 0x83, 0x08, - 0x39, 0x88, 0x3b, 0x3c, 0x9e, 0x3a, 0xfe, 0xbc, 0xdf, 0x3c, 0xd0, 0x0e, 0xbb, 0xb6, 0x84, 0xac, - 0x2e, 0xc0, 0x69, 0xb6, 0x88, 0xa4, 0x7f, 0x7f, 0xa3, 0x01, 0x8c, 0x43, 0xc7, 0x93, 0x4a, 0x7f, - 0x02, 0xbd, 0x4b, 0x3f, 0xf0, 0x93, 0x2b, 0xee, 0x3d, 0xbe, 0x4b, 0x79, 0x42, 0xba, 0x1b, 0x76, - 0x15, 0x89, 0xca, 0x92, 0xd6, 0x82, 0x45, 0x27, 0x16, 0x05, 0xc3, 0x06, 0xd0, 0x8a, 0xe2, 0x70, - 0x16, 0xf3, 0x24, 0x91, 0x79, 0x59, 0xc0, 0xb8, 0x77, 0xc1, 0x53, 0x47, 0x34, 0x11, 0x99, 0x9d, - 0x0a, 0xc6, 0xfa, 0xbd, 0x06, 0xbd, 0xc9, 0x95, 0x13, 0x7b, 0x7e, 0x30, 0x3b, 0x8b, 0xc3, 0x8c, - 0xca, 0x3c, 0x75, 0xe2, 0x19, 0xcf, 0x7b, 0x9a, 0x84, 0xb0, 0xa2, 0x4e, 0x4f, 0xc7, 0x78, 0xbe, - 0x81, 0x15, 0x85, 0x6b, 0xa1, 0x7f, 0x9c, 0xa4, 0xe3, 0xd0, 0x75, 0x52, 0x3f, 0x0c, 0xe4, 0xf1, - 0x55, 0x24, 0x55, 0xcd, 0x5d, 0xe0, 0x92, 0x33, 0x0d, 0xaa, 0x1a, 0x82, 0x50, 0xef, 0x2c, 0x90, - 0x94, 0x06, 0x51, 0x0a, 0xd8, 0xfa, 0x93, 0x01, 0x30, 0xb9, 0x0b, 0x5c, 0xe9, 0xa8, 0x03, 0xe8, - 0x90, 0xc1, 0x4f, 0x6e, 0x78, 0x90, 0xe6, 0x6e, 0x52, 0x51, 0x28, 0x8c, 0xc0, 0x97, 0x51, 0xee, - 0xa2, 0x02, 0xc6, 0xb6, 0x1a, 0x73, 0x97, 0x07, 0x29, 0x12, 0x0d, 0x22, 0x96, 0x08, 0x66, 0x41, - 0x77, 0xe1, 0x24, 0x29, 0x8f, 0x2b, 0x4e, 0xaa, 0xe0, 0xd8, 0x11, 0x98, 0x2a, 0x7c, 0x96, 0xfa, - 0x1e, 0x25, 0x40, 0xdb, 0x5e, 0xc2, 0xa3, 0x3c, 0x32, 0x22, 0x97, 0xd7, 0x14, 0xf2, 0x54, 0x1c, - 0xca, 0x53, 0x61, 0x92, 0xb7, 0x25, 0xe4, 0xdd, 0xc7, 0xa3, 0xbc, 0xe9, 0x3c, 0x74, 0xaf, 0xfd, - 0x60, 0x46, 0x01, 0x68, 0x91, 0xab, 0x2a, 0x38, 0xf6, 0x25, 0x98, 0x59, 0x10, 0xf3, 0x24, 0x9c, - 0xdf, 0x70, 0x8f, 0xe2, 0x98, 0xf4, 0xdb, 0x4a, 0xcd, 0xab, 0x11, 0xb6, 0x97, 0x58, 0x95, 0x08, - 0x81, 0x28, 0x1e, 0x19, 0xa1, 0x21, 0xc0, 0x94, 0x14, 0x79, 0x79, 0x17, 0xf1, 0x7e, 0x47, 0x64, - 0x4f, 0x89, 0xb1, 0xfe, 0xa8, 0x41, 0x57, 0x6d, 0x43, 0x4a, 0x83, 0xd4, 0xd6, 0x34, 0x48, 0x5d, - 0x6d, 0x90, 0xec, 0xf3, 0xa2, 0xb1, 0x88, 0x4e, 0x41, 0xda, 0x5e, 0xc4, 0x21, 0xd6, 0xa1, 0x4d, - 0x84, 0xa2, 0xd7, 0x7c, 0x01, 0x9d, 0x18, 0xef, 0xd2, 0xa2, 0xa3, 0x15, 0x9d, 0xc5, 0x2e, 0xd1, - 0xb6, 0xca, 0x63, 0xfd, 0x5d, 0x87, 0x8e, 0x42, 0x5c, 0x8a, 0xb4, 0xf6, 0x0d, 0x23, 0xad, 0xaf, - 0x89, 0xf4, 0x41, 0xae, 0x52, 0x36, 0x3d, 0xf5, 0x63, 0x99, 0xfc, 0x2a, 0xaa, 0xe0, 0xa8, 0xa4, - 0x96, 0x8a, 0x62, 0x87, 0xf0, 0x40, 0x01, 0x95, 0xc4, 0xba, 0x8f, 0x66, 0xc7, 0xc0, 0x08, 0x35, - 0x72, 0x52, 0xf7, 0xea, 0x55, 0x74, 0x4e, 0xda, 0x50, 0x76, 0xb5, 0xec, 0x15, 0x14, 0xf6, 0x5d, - 0x68, 0x24, 0xa9, 0x33, 0xe3, 0x94, 0x58, 0xdb, 0x27, 0x6d, 0x4a, 0x04, 0x44, 0xd8, 0x02, 0xaf, - 0x38, 0xbf, 0xf5, 0x0e, 0xe7, 0x5b, 0xff, 0xd5, 0xa1, 0x57, 0xb9, 0x38, 0x56, 0x5d, 0xb0, 0xe5, - 0x89, 0xfa, 0x9a, 0x13, 0x0f, 0xa0, 0x9e, 0x05, 0xbe, 0x08, 0xf6, 0xf6, 0x49, 0x17, 0xe9, 0xaf, - 0x02, 0x3f, 0xc5, 0x5c, 0xb2, 0x89, 0xa2, 0xe8, 0x54, 0x7f, 0x57, 0x42, 0x7c, 0x0f, 0x76, 0xcb, - 0x44, 0x3e, 0x3d, 0x1d, 0x8f, 0x43, 0xf7, 0xfa, 0xd9, 0xa9, 0xf4, 0xde, 0x2a, 0x12, 0x63, 0xe2, - 0xba, 0xa2, 0x82, 0x7c, 0x5a, 0x13, 0x17, 0xd6, 0x67, 0xd0, 0x70, 0xf1, 0x1a, 0x21, 0x2f, 0xc9, - 0x84, 0x52, 0xee, 0x95, 0xa7, 0x35, 0x5b, 0xd0, 0xd9, 0x27, 0x50, 0xf7, 0xb2, 0x45, 0x24, 0x7d, - 0xb5, 0x8d, 0x7c, 0x65, 0x63, 0x7f, 0x5a, 0xb3, 0x89, 0x8a, 0x5c, 0xf3, 0xd0, 0xf1, 0xfa, 0xed, - 0x92, 0xab, 0xec, 0xf7, 0xc8, 0x85, 0x54, 0xe4, 0xc2, 0x0a, 0xa3, 0x6a, 0x93, 0x5c, 0x65, 0xb3, - 0x43, 0x2e, 0xa4, 0x3e, 0x6e, 0x41, 0x33, 0x11, 0x89, 0xfc, 0x13, 0xd8, 0xa9, 0x78, 0x7f, 0xec, - 0x27, 0xe4, 0x2a, 0x41, 0xee, 0x6b, 0xeb, 0x6e, 0xf7, 0x7c, 0xff, 0x10, 0x80, 0x6c, 0x12, 0x17, - 0xae, 0xbc, 0xb5, 0xb5, 0xf2, 0x61, 0xf6, 0x31, 0xb4, 0xd1, 0x96, 0x0d, 0x64, 0x34, 0x62, 0x1d, - 0x39, 0x82, 0x2e, 0x69, 0xff, 0x62, 0xbc, 0x86, 0x83, 0x9d, 0xc0, 0x9e, 0xb8, 0x40, 0x8b, 0xc7, - 0xac, 0x4f, 0xd7, 0x85, 0x28, 0xac, 0x95, 0x34, 0x6c, 0xe8, 0x1c, 0xc5, 0x4d, 0x5e, 0x8c, 0xf3, - 0x5b, 0x2d, 0x87, 0xad, 0x1f, 0x42, 0x1b, 0x4f, 0x14, 0xc7, 0x1d, 0x42, 0x93, 0x08, 0xb9, 0x1f, - 0xcc, 0xc2, 0x9d, 0x52, 0x21, 0x5b, 0xd2, 0xad, 0xdf, 0x6a, 0x95, 0xd7, 0xc9, 0x7b, 0x77, 0xab, - 0x83, 0xe5, 0xc7, 0x4d, 0xbb, 0xfa, 0x96, 0x39, 0x06, 0xa0, 0x86, 0x93, 0xbf, 0x64, 0x8a, 0xf0, - 0x96, 0x58, 0x5b, 0xe1, 0xc0, 0xc0, 0x94, 0xd0, 0x0a, 0xd7, 0xfe, 0x41, 0x87, 0xae, 0xfa, 0x08, - 0xfa, 0xb6, 0xca, 0x8e, 0x29, 0x6f, 0xf5, 0xbc, 0x32, 0x3e, 0xcd, 0x2b, 0xa3, 0x51, 0x9a, 0x51, - 0x66, 0x51, 0x59, 0x18, 0x8f, 0x64, 0x61, 0x34, 0x89, 0xad, 0x97, 0x17, 0x46, 0xce, 0x25, 0xea, - 0xe2, 0x91, 0xac, 0x8b, 0xad, 0x92, 0xa9, 0x48, 0xa9, 0xa2, 0x2c, 0x1e, 0xc9, 0xb2, 0x68, 0x95, - 0x4c, 0x45, 0x98, 0x8b, 0xaa, 0xd8, 0x82, 0x06, 0x85, 0xd3, 0xfa, 0x31, 0x98, 0xaa, 0x6b, 0xa8, - 0x26, 0x3e, 0x95, 0xc4, 0x4a, 0x2a, 0xa8, 0x8f, 0x48, 0xb9, 0xf7, 0x35, 0xf4, 0x2a, 0x4d, 0x05, - 0x6f, 0x3a, 0x3f, 0x19, 0x39, 0x81, 0xcb, 0xe7, 0xc5, 0x13, 0x52, 0xc1, 0x28, 0x49, 0xa6, 0x97, - 0x92, 0xa5, 0x88, 0x4a, 0x92, 0x29, 0x0f, 0x41, 0xa3, 0xf2, 0x10, 0xfc, 0xa7, 0x06, 0x5d, 0x75, - 0x03, 0xbe, 0x25, 0x9f, 0xc4, 0xf1, 0x28, 0xf4, 0x44, 0x34, 0x1b, 0x76, 0x0e, 0x62, 0xea, 0xe3, - 0x72, 0xee, 0x24, 0x89, 0xcc, 0xc0, 0x02, 0x96, 0xb4, 0x89, 0x1b, 0x46, 0xf9, 0x27, 0xa4, 0x80, - 0x25, 0x6d, 0xcc, 0x6f, 0xf8, 0x5c, 0x5e, 0x35, 0x05, 0x8c, 0xa7, 0x9d, 0xf3, 0x24, 0xc1, 0x34, - 0x11, 0x1d, 0x32, 0x07, 0x71, 0x97, 0xed, 0xdc, 0x8e, 0x9c, 0x2c, 0xe1, 0xf2, 0xad, 0x52, 0xc0, - 0xe8, 0x16, 0xfc, 0x2c, 0x39, 0x71, 0x98, 0x05, 0xf9, 0x0b, 0x45, 0xc1, 0x58, 0x03, 0xe8, 0x4f, - 0x6e, 0xfd, 0xd4, 0xbd, 0xa2, 0x2c, 0x16, 0x17, 0x8f, 0xfc, 0x82, 0x59, 0x27, 0xb0, 0x2b, 0x7f, - 0x99, 0x95, 0x3f, 0xf0, 0x77, 0x94, 0x2f, 0x66, 0xa7, 0x28, 0x0d, 0xf1, 0xc7, 0xb4, 0x32, 0xd8, - 0xab, 0xee, 0x91, 0x1f, 0x90, 0x4d, 0x9b, 0xfe, 0x0f, 0x1f, 0xd3, 0x5b, 0xd8, 0xb9, 0xc8, 0xe2, - 0x59, 0x55, 0xd1, 0x01, 0xb4, 0xfc, 0xc0, 0x71, 0x53, 0xff, 0x86, 0xcb, 0x84, 0x28, 0x60, 0xfa, - 0xb1, 0xfa, 0xf2, 0x57, 0x6d, 0xd8, 0xb4, 0x46, 0xfe, 0x4b, 0x7f, 0xce, 0xa9, 0x3c, 0x65, 0x64, - 0x72, 0x98, 0x54, 0x11, 0x8f, 0x04, 0xf9, 0x41, 0x14, 0x10, 0xfa, 0x8f, 0xbe, 0x5b, 0xe2, 0x47, - 0x3a, 0x0a, 0x83, 0x4b, 0x7f, 0x96, 0xfb, 0xef, 0x16, 0x3e, 0x5a, 0x41, 0x7b, 0xc7, 0x8f, 0xac, - 0xb4, 0x59, 0x5f, 0x65, 0xb3, 0x51, 0xf6, 0x64, 0x65, 0x78, 0x51, 0xaf, 0x0e, 0x2f, 0xfe, 0xa6, - 0xc1, 0x40, 0x46, 0x41, 0x9c, 0x3d, 0x71, 0xaf, 0xf8, 0xc2, 0xc9, 0xfd, 0xf2, 0x50, 0x89, 0x05, - 0xf5, 0x12, 0x41, 0x2e, 0xe7, 0x0d, 0xf4, 0x97, 0xd7, 0xcb, 0xbf, 0xfc, 0xda, 0x40, 0x0c, 0xa0, - 0xe5, 0x39, 0xa9, 0x33, 0x75, 0x12, 0x9e, 0xe7, 0x6b, 0x0e, 0xd3, 0xbf, 0xcd, 0x99, 0xce, 0xf3, - 0x6c, 0x15, 0x00, 0x49, 0xa2, 0xd3, 0x64, 0xa6, 0x4a, 0xe8, 0xe8, 0xd7, 0xd0, 0x14, 0x93, 0x0a, - 0xd6, 0x83, 0xf6, 0xb3, 0xe0, 0xc6, 0x99, 0xfb, 0xde, 0xf3, 0xc8, 0xac, 0xb1, 0x16, 0xd4, 0x27, - 0x69, 0x18, 0x99, 0x1a, 0x6b, 0x43, 0xe3, 0x02, 0x73, 0xda, 0xd4, 0x19, 0x40, 0x13, 0xcb, 0x7e, - 0xc1, 0x4d, 0x03, 0xd1, 0x34, 0x04, 0x32, 0xeb, 0x88, 0x16, 0xe3, 0x09, 0xb3, 0xc1, 0xb6, 0x01, - 0x7e, 0x9e, 0xa5, 0xa1, 0x64, 0x6b, 0x1e, 0xbd, 0x26, 0xb6, 0x19, 0x3a, 0xb2, 0x2b, 0xe5, 0x13, - 0x6c, 0xd6, 0xd8, 0x16, 0x18, 0x5f, 0xf1, 0x5b, 0x53, 0x63, 0x1d, 0xd8, 0xb2, 0xb3, 0x00, 0x7f, - 0x83, 0xe2, 0x0c, 0x3a, 0xce, 0x33, 0x0d, 0x24, 0xa0, 0x12, 0x11, 0xf7, 0xcc, 0x3a, 0xeb, 0x42, - 0xeb, 0x17, 0xf2, 0x7b, 0x67, 0x36, 0x90, 0x84, 0x6c, 0xb8, 0xa7, 0x89, 0x24, 0x3a, 0x10, 0xa1, - 0xad, 0xa3, 0xe7, 0xd0, 0xca, 0xbb, 0x34, 0x7b, 0x00, 0x1d, 0x79, 0x2a, 0xa2, 0xcc, 0x1a, 0xaa, - 0x4d, 0xbd, 0xd8, 0xd4, 0xd0, 0x44, 0xec, 0xb7, 0xa6, 0x8e, 0x2b, 0x6c, 0xaa, 0xa6, 0x41, 0x66, - 0xdf, 0x05, 0xae, 0x59, 0x47, 0x46, 0xca, 0x6a, 0xd3, 0x3b, 0x3a, 0x87, 0x2d, 0x59, 0x36, 0x8c, - 0xc1, 0xb6, 0x94, 0x27, 0x31, 0x66, 0x0d, 0x3d, 0x87, 0x5a, 0x0a, 0x6e, 0x0d, 0x3d, 0x40, 0x06, - 0x08, 0x58, 0x47, 0x15, 0x84, 0x37, 0x04, 0xc2, 0x40, 0xfd, 0xf2, 0xc8, 0xb3, 0x5d, 0x78, 0x90, - 0x7b, 0x45, 0xa2, 0x84, 0xc0, 0x33, 0x9e, 0x0a, 0x84, 0xa9, 0x91, 0xfc, 0x02, 0xd4, 0xd1, 0x91, - 0x36, 0x5f, 0x84, 0x37, 0x5c, 0x62, 0x8c, 0x93, 0xff, 0x34, 0xa1, 0x29, 0x12, 0x8e, 0x8d, 0xa0, - 0xab, 0x8e, 0xe6, 0xd8, 0x87, 0xf2, 0x4e, 0xbb, 0x3f, 0xac, 0x1b, 0xf4, 0xe9, 0x56, 0x5a, 0x31, - 0xd5, 0xb1, 0x6a, 0xec, 0x19, 0x6c, 0x57, 0xc7, 0x5c, 0xec, 0x23, 0xe4, 0x5e, 0x39, 0x43, 0x1b, - 0x0c, 0x56, 0x91, 0x0a, 0x51, 0x4f, 0xa0, 0x57, 0x99, 0x5c, 0x31, 0x3a, 0x77, 0xd5, 0x30, 0x6b, - 0xa3, 0x46, 0x3f, 0x83, 0x8e, 0x32, 0x0e, 0x62, 0xfb, 0xc8, 0xba, 0x3c, 0xe5, 0x1a, 0x7c, 0xb8, - 0x84, 0x2f, 0x24, 0x7c, 0x09, 0x50, 0xce, 0x67, 0xd8, 0x07, 0x05, 0xa3, 0x3a, 0xfc, 0x1a, 0xec, - 0xdf, 0x47, 0x17, 0xdb, 0xcf, 0x61, 0x67, 0xa9, 0x5f, 0xb3, 0x87, 0xe4, 0xdc, 0x35, 0x6d, 0x7c, - 0xa3, 0x3d, 0x23, 0xe8, 0xaa, 0xed, 0x5a, 0x84, 0x69, 0x45, 0xd3, 0x17, 0x42, 0x56, 0x75, 0x76, - 0xab, 0xc6, 0x7e, 0x0a, 0x50, 0x36, 0x5f, 0x61, 0xd2, 0x52, 0x33, 0xde, 0xa8, 0xc5, 0x19, 0xec, - 0x28, 0xc3, 0x59, 0xd1, 0x28, 0x85, 0x6f, 0x97, 0x67, 0xb6, 0x1b, 0x05, 0xd9, 0x72, 0x92, 0xa8, - 0x76, 0x5c, 0xe1, 0x9d, 0x75, 0x4d, 0x7a, 0xf0, 0xf1, 0x1a, 0xaa, 0xea, 0x22, 0x75, 0x12, 0x2c, - 0x5c, 0xb4, 0x62, 0x36, 0xbc, 0x51, 0xb1, 0x5f, 0x42, 0x2f, 0x4f, 0x4d, 0x2a, 0x16, 0x36, 0x54, - 0xfc, 0xb9, 0xa2, 0x47, 0x6f, 0x12, 0xf6, 0xb8, 0xff, 0x8f, 0x37, 0x43, 0xed, 0xeb, 0x37, 0x43, - 0xed, 0xdf, 0x6f, 0x86, 0xda, 0xef, 0xde, 0x0e, 0x6b, 0x5f, 0xbf, 0x1d, 0xd6, 0xfe, 0xf5, 0x76, - 0x58, 0x9b, 0x36, 0x69, 0x36, 0xfe, 0xfd, 0xff, 0x05, 0x00, 0x00, 0xff, 0xff, 0xfc, 0x3b, 0x55, - 0x52, 0x2d, 0x17, 0x00, 0x00, + // 2116 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xb4, 0x19, 0xcb, 0x6e, 0xdc, 0xc8, + 0x71, 0xc8, 0x79, 0x68, 0xa6, 0x66, 0x46, 0xa6, 0x5a, 0x5a, 0xed, 0xec, 0xc4, 0x3b, 0x11, 0xe8, + 0xc5, 0xae, 0x56, 0x07, 0x21, 0xab, 0x6c, 0x10, 0x20, 0xc0, 0xe6, 0xe1, 0x91, 0x23, 0x1b, 0x19, + 0xad, 0x6d, 0x8e, 0x8d, 0x1c, 0x03, 0x0e, 0xd9, 0x1a, 0x11, 0xe2, 0x90, 0x34, 0x1f, 0x12, 0x74, + 0xc8, 0x2f, 0x24, 0xb9, 0xe4, 0x10, 0x20, 0xb7, 0x20, 0xc8, 0x29, 0x41, 0xfe, 0x22, 0x39, 0x2e, + 0x72, 0xca, 0x31, 0xb0, 0x7f, 0x23, 0x87, 0xa0, 0xaa, 0x9b, 0x64, 0x53, 0xf3, 0xb0, 0x0d, 0x64, + 0x6f, 0xac, 0x47, 0x57, 0x57, 0x55, 0xd7, 0xa3, 0xbb, 0x08, 0xdb, 0xee, 0xe2, 0x26, 0x8c, 0xaf, + 0x78, 0x7c, 0x1c, 0xc5, 0x61, 0x1a, 0x32, 0x3d, 0x9a, 0x99, 0x9f, 0xc3, 0xee, 0x34, 0xb5, 0xe3, + 0x74, 0x9a, 0xcd, 0x5e, 0xd8, 0xc9, 0x95, 0xc5, 0x5f, 0x65, 0x3c, 0x49, 0x19, 0x83, 0x46, 0x6a, + 0x27, 0x57, 0x03, 0xed, 0x40, 0x3b, 0xec, 0x58, 0xf4, 0x6d, 0x1e, 0x03, 0x7b, 0x19, 0xb9, 0x76, + 0xca, 0x2d, 0xee, 0xdb, 0xb7, 0x39, 0xe7, 0x00, 0xb6, 0x9c, 0x30, 0x48, 0x79, 0x90, 0x4a, 0xe6, + 0x1c, 0x34, 0xa7, 0xb0, 0x7b, 0xee, 0xcd, 0xe3, 0xbb, 0x0b, 0x46, 0x00, 0x0f, 0xbd, 0xc0, 0x0f, + 0xe7, 0x5f, 0xdb, 0x0b, 0x2e, 0xd7, 0x28, 0x18, 0x76, 0x1f, 0x3a, 0x02, 0x7a, 0x16, 0x26, 0x03, + 0xfd, 0x40, 0x3b, 0xec, 0x5b, 0x25, 0xc2, 0x3c, 0x83, 0x0f, 0x9e, 0x46, 0x1c, 0x85, 0xde, 0xd1, + 0x78, 0x08, 0x7a, 0x18, 0x91, 0xb8, 0xed, 0x13, 0x38, 0x8e, 0x66, 0xc7, 0x48, 0x7c, 0x1a, 0x59, + 0x7a, 0x18, 0xa1, 0x35, 0x01, 0x6e, 0xa6, 0x0b, 0x6b, 0xf0, 0xdb, 0xbc, 0x86, 0xfd, 0xbb, 0x82, + 0x92, 0x28, 0x0c, 0x12, 0xbe, 0x51, 0xd2, 0x3e, 0xb4, 0x62, 0x9e, 0x64, 0x7e, 0x4a, 0xb2, 0xda, + 0x96, 0x84, 0x10, 0x9f, 0x84, 0x59, 0xec, 0xf0, 0x41, 0x9d, 0xf6, 0x90, 0x10, 0x33, 0xa0, 0xbe, + 0x48, 0xe6, 0x83, 0x06, 0x21, 0xf1, 0xd3, 0x3c, 0x82, 0x3d, 0xe1, 0xc5, 0x77, 0xf0, 0xf8, 0x21, + 0xb0, 0xe7, 0x19, 0x8f, 0x6f, 0xa7, 0xa9, 0x9d, 0x66, 0x89, 0xc2, 0x19, 0x94, 0xae, 0x13, 0xd6, + 0x7c, 0x06, 0x3b, 0xc4, 0xf9, 0x28, 0x8e, 0xc3, 0x78, 0x13, 0x63, 0x04, 0x7b, 0xe3, 0x70, 0xb1, + 0x08, 0x83, 0x5f, 0x52, 0x24, 0x14, 0x46, 0x97, 0x86, 0x69, 0x15, 0xc3, 0xa4, 0x01, 0x7a, 0x61, + 0xc0, 0x5a, 0x53, 0xf7, 0xa1, 0x25, 0xa2, 0x4b, 0x5a, 0x2b, 0x21, 0xf3, 0x6f, 0x1a, 0xec, 0x56, + 0xac, 0x78, 0xef, 0x1d, 0xbf, 0x84, 0x9e, 0xd8, 0x43, 0x48, 0xa0, 0x7d, 0xbb, 0x27, 0x06, 0x1e, + 0xcd, 0x54, 0xc1, 0x5b, 0x15, 0x2e, 0xf6, 0x43, 0xe8, 0x27, 0xc2, 0xc5, 0x72, 0x59, 0xe3, 0xa0, + 0x7e, 0xd8, 0x3d, 0xd9, 0xa1, 0x65, 0x2a, 0xc1, 0xaa, 0xf2, 0x99, 0x7f, 0xd1, 0xa4, 0xdb, 0xa5, + 0x33, 0xdf, 0x5b, 0xdf, 0x2f, 0xa0, 0x2b, 0xf4, 0x22, 0x01, 0x52, 0xdd, 0x7b, 0xa5, 0xba, 0x42, + 0xae, 0xca, 0x43, 0x26, 0x0a, 0x25, 0xc4, 0x1a, 0xa1, 0xab, 0xa1, 0xe8, 0x2a, 0x16, 0x55, 0xb8, + 0xcc, 0x3f, 0x6b, 0xd0, 0x1d, 0x5f, 0x72, 0x47, 0x6a, 0x8e, 0x2a, 0x46, 0x76, 0x92, 0x70, 0x37, + 0x57, 0x51, 0x40, 0x6c, 0x0f, 0x9a, 0x69, 0x98, 0xda, 0x3e, 0x29, 0xd9, 0xb4, 0x04, 0x80, 0x89, + 0x98, 0x64, 0x8e, 0xc3, 0x93, 0xe4, 0x22, 0xf3, 0x49, 0xcb, 0xa6, 0xa5, 0x60, 0x50, 0xda, 0x85, + 0xed, 0xf9, 0xdc, 0xa5, 0x03, 0x6d, 0x5a, 0x12, 0xc2, 0x8c, 0xbf, 0xb1, 0xe3, 0xc0, 0x0b, 0xe6, + 0x83, 0x26, 0x11, 0x72, 0x10, 0x57, 0xb8, 0x3c, 0xb5, 0x3d, 0x7f, 0xd0, 0x3a, 0xd0, 0x0e, 0x7b, + 0x96, 0x84, 0xcc, 0x1e, 0xc0, 0x69, 0xb6, 0x88, 0xa4, 0x7f, 0x7f, 0xa3, 0x01, 0x4c, 0x42, 0xdb, + 0x95, 0x4a, 0x7f, 0x02, 0xfd, 0x0b, 0x2f, 0xf0, 0x92, 0x4b, 0xee, 0x3e, 0xbc, 0x4d, 0x79, 0x42, + 0xba, 0xd7, 0xad, 0x2a, 0x12, 0x95, 0x25, 0xad, 0x05, 0x8b, 0x4e, 0x2c, 0x0a, 0x86, 0x0d, 0xa1, + 0x1d, 0xc5, 0xe1, 0x3c, 0xe6, 0x49, 0x22, 0xe3, 0xb2, 0x80, 0x71, 0xed, 0x82, 0xa7, 0xb6, 0x28, + 0x22, 0x32, 0x3a, 0x15, 0x8c, 0xf9, 0x7b, 0x0d, 0xfa, 0xd3, 0x4b, 0x3b, 0x76, 0xbd, 0x60, 0x7e, + 0x16, 0x87, 0x19, 0xa5, 0x79, 0x6a, 0xc7, 0x73, 0x9e, 0xd7, 0x34, 0x09, 0x61, 0x46, 0x9d, 0x9e, + 0x4e, 0x70, 0xff, 0x3a, 0x66, 0x14, 0x7e, 0x0b, 0xfd, 0xe3, 0x24, 0x9d, 0x84, 0x8e, 0x9d, 0x7a, + 0x61, 0x20, 0xb7, 0xaf, 0x22, 0x29, 0x6b, 0x6e, 0x03, 0x87, 0x9c, 0x59, 0xa7, 0xac, 0x21, 0x08, + 0xf5, 0xce, 0x02, 0x49, 0x69, 0x12, 0xa5, 0x80, 0xcd, 0x3f, 0xd5, 0x01, 0xa6, 0xb7, 0x81, 0x23, + 0x1d, 0x75, 0x00, 0x5d, 0x32, 0xf8, 0xd1, 0x35, 0x0f, 0xd2, 0xdc, 0x4d, 0x2a, 0x0a, 0x85, 0x11, + 0xf8, 0x22, 0xca, 0x5d, 0x54, 0xc0, 0x58, 0x56, 0x63, 0xee, 0xf0, 0x20, 0x45, 0x62, 0x9d, 0x88, + 0x25, 0x82, 0x99, 0xd0, 0x5b, 0xd8, 0x49, 0xca, 0xe3, 0x8a, 0x93, 0x2a, 0x38, 0x76, 0x04, 0x86, + 0x0a, 0x9f, 0xa5, 0x9e, 0x4b, 0x01, 0xd0, 0xb1, 0x96, 0xf0, 0x28, 0x8f, 0x8c, 0xc8, 0xe5, 0xb5, + 0x84, 0x3c, 0x15, 0x87, 0xf2, 0x54, 0x98, 0xe4, 0x6d, 0x09, 0x79, 0x77, 0xf1, 0x28, 0x6f, 0xe6, + 0x87, 0xce, 0x95, 0x17, 0xcc, 0xe9, 0x00, 0xda, 0xe4, 0xaa, 0x0a, 0x8e, 0x7d, 0x05, 0x46, 0x16, + 0xc4, 0x3c, 0x09, 0xfd, 0x6b, 0xee, 0xd2, 0x39, 0x26, 0x83, 0x8e, 0x92, 0xf3, 0xea, 0x09, 0x5b, + 0x4b, 0xac, 0xca, 0x09, 0x81, 0x48, 0x1e, 0x79, 0x42, 0x23, 0x80, 0x19, 0x29, 0xf2, 0xe2, 0x36, + 0xe2, 0x83, 0xae, 0x88, 0x9e, 0x12, 0x63, 0xfe, 0x51, 0x83, 0x9e, 0x5a, 0x86, 0x94, 0x02, 0xa9, + 0xad, 0x29, 0x90, 0xba, 0x5a, 0x20, 0xd9, 0xe7, 0x45, 0x61, 0x11, 0x95, 0x82, 0xb4, 0x7d, 0x16, + 0x87, 0x98, 0x87, 0x16, 0x11, 0x8a, 0x5a, 0xf3, 0x05, 0x74, 0x63, 0xec, 0xa5, 0x45, 0x45, 0x2b, + 0x2a, 0x8b, 0x55, 0xa2, 0x2d, 0x95, 0xc7, 0xfc, 0x87, 0x0e, 0x5d, 0x85, 0xb8, 0x74, 0xd2, 0xda, + 0x3b, 0x9e, 0xb4, 0xbe, 0xe6, 0xa4, 0x0f, 0x72, 0x95, 0xb2, 0xd9, 0xa9, 0x17, 0xcb, 0xe0, 0x57, + 0x51, 0x05, 0x47, 0x25, 0xb4, 0x54, 0x14, 0x3b, 0x84, 0x7b, 0x0a, 0xa8, 0x04, 0xd6, 0x5d, 0x34, + 0x3b, 0x06, 0x46, 0xa8, 0xb1, 0x9d, 0x3a, 0x97, 0x2f, 0xa3, 0x73, 0xd2, 0x86, 0xa2, 0xab, 0x6d, + 0xad, 0xa0, 0xb0, 0xef, 0x42, 0x33, 0x49, 0xed, 0x39, 0xa7, 0xc0, 0xda, 0x3e, 0xe9, 0x50, 0x20, + 0x20, 0xc2, 0x12, 0x78, 0xc5, 0xf9, 0xed, 0xb7, 0x38, 0xdf, 0xfc, 0xaf, 0x0e, 0xfd, 0x4a, 0xe3, + 0x58, 0xd5, 0x60, 0xcb, 0x1d, 0xf5, 0x35, 0x3b, 0x1e, 0x40, 0x23, 0x0b, 0x3c, 0x71, 0xd8, 0xdb, + 0x27, 0x3d, 0xa4, 0xbf, 0x0c, 0xbc, 0x14, 0x63, 0xc9, 0x22, 0x8a, 0xa2, 0x53, 0xe3, 0x6d, 0x01, + 0xf1, 0x3d, 0xd8, 0x2d, 0x03, 0xf9, 0xf4, 0x74, 0x32, 0x09, 0x9d, 0xab, 0x27, 0xa7, 0xd2, 0x7b, + 0xab, 0x48, 0x8c, 0x89, 0x76, 0x45, 0x09, 0xf9, 0xb8, 0x26, 0x1a, 0xd6, 0x67, 0xd0, 0x74, 0xb0, + 0x8d, 0x90, 0x97, 0x64, 0x40, 0x29, 0x7d, 0xe5, 0x71, 0xcd, 0x12, 0x74, 0xf6, 0x09, 0x34, 0xdc, + 0x6c, 0x11, 0x49, 0x5f, 0x6d, 0x23, 0x5f, 0x59, 0xd8, 0x1f, 0xd7, 0x2c, 0xa2, 0x22, 0x97, 0x1f, + 0xda, 0xee, 0xa0, 0x53, 0x72, 0x95, 0xf5, 0x1e, 0xb9, 0x90, 0x8a, 0x5c, 0x98, 0x61, 0x94, 0x6d, + 0x92, 0xab, 0x2c, 0x76, 0xc8, 0x85, 0xd4, 0x87, 0x6d, 0x68, 0x25, 0x22, 0x90, 0x7f, 0x0c, 0x3b, + 0x15, 0xef, 0x4f, 0xbc, 0x84, 0x5c, 0x25, 0xc8, 0x03, 0x6d, 0x5d, 0x77, 0xcf, 0xd7, 0x8f, 0x00, + 0xc8, 0x26, 0xd1, 0x70, 0x65, 0xd7, 0xd6, 0xca, 0x8b, 0xd9, 0xc7, 0xd0, 0x41, 0x5b, 0x36, 0x90, + 0xd1, 0x88, 0x75, 0xe4, 0x08, 0x7a, 0xa4, 0xfd, 0xf3, 0xc9, 0x1a, 0x0e, 0x76, 0x02, 0x7b, 0xa2, + 0x81, 0x16, 0x97, 0x59, 0x8f, 0xda, 0x85, 0x48, 0xac, 0x95, 0x34, 0x2c, 0xe8, 0x1c, 0xc5, 0x4d, + 0x9f, 0x4f, 0xf2, 0xae, 0x96, 0xc3, 0xe6, 0x0f, 0xa0, 0x83, 0x3b, 0x8a, 0xed, 0x0e, 0xa1, 0x45, + 0x84, 0xdc, 0x0f, 0x46, 0xe1, 0x4e, 0xa9, 0x90, 0x25, 0xe9, 0xe6, 0x6f, 0xb5, 0xca, 0xed, 0xe4, + 0xbd, 0xab, 0xd5, 0xc1, 0xf2, 0xe5, 0xa6, 0x53, 0xbd, 0xcb, 0x1c, 0x03, 0x50, 0xc1, 0xc9, 0x6f, + 0x32, 0xc5, 0xf1, 0x96, 0x58, 0x4b, 0xe1, 0xc0, 0x83, 0x29, 0xa1, 0x15, 0xae, 0xfd, 0x83, 0x0e, + 0x3d, 0xf5, 0x12, 0xf4, 0x6d, 0xa5, 0x1d, 0x53, 0xee, 0xea, 0x79, 0x66, 0x7c, 0x9a, 0x67, 0x46, + 0xb3, 0x34, 0xa3, 0x8c, 0xa2, 0x32, 0x31, 0x1e, 0xc8, 0xc4, 0x68, 0x11, 0x5b, 0x3f, 0x4f, 0x8c, + 0x9c, 0x4b, 0xe4, 0xc5, 0x03, 0x99, 0x17, 0x5b, 0x25, 0x53, 0x11, 0x52, 0x45, 0x5a, 0x3c, 0x90, + 0x69, 0xd1, 0x2e, 0x99, 0x8a, 0x63, 0x2e, 0xb2, 0x62, 0x0b, 0x9a, 0x74, 0x9c, 0xe6, 0x8f, 0xc0, + 0x50, 0x5d, 0x43, 0x39, 0xf1, 0xa9, 0x24, 0x56, 0x42, 0x41, 0xbd, 0x44, 0xca, 0xb5, 0xaf, 0xa0, + 0x5f, 0x29, 0x2a, 0xd8, 0xe9, 0xbc, 0x64, 0x6c, 0x07, 0x0e, 0xf7, 0x8b, 0x2b, 0xa4, 0x82, 0x51, + 0x82, 0x4c, 0x2f, 0x25, 0x4b, 0x11, 0x95, 0x20, 0x53, 0x2e, 0x82, 0xf5, 0xca, 0x45, 0xf0, 0x5f, + 0x1a, 0xf4, 0xd4, 0x05, 0x78, 0x97, 0x7c, 0x14, 0xc7, 0xe3, 0xd0, 0x15, 0xa7, 0xd9, 0xb4, 0x72, + 0x10, 0x43, 0x1f, 0x3f, 0x7d, 0x3b, 0x49, 0x64, 0x04, 0x16, 0xb0, 0xa4, 0x4d, 0x9d, 0x30, 0xca, + 0x1f, 0x21, 0x05, 0x2c, 0x69, 0x13, 0x7e, 0xcd, 0x7d, 0xd9, 0x6a, 0x0a, 0x18, 0x77, 0x3b, 0xe7, + 0x49, 0x82, 0x61, 0x22, 0x2a, 0x64, 0x0e, 0xe2, 0x2a, 0xcb, 0xbe, 0x19, 0xdb, 0x59, 0xc2, 0xe5, + 0x5d, 0xa5, 0x80, 0xd1, 0x2d, 0xf8, 0x58, 0xb2, 0xe3, 0x30, 0x0b, 0xf2, 0x1b, 0x8a, 0x82, 0x31, + 0x87, 0x30, 0x98, 0xde, 0x78, 0xa9, 0x73, 0x49, 0x51, 0x2c, 0x1a, 0x8f, 0x7c, 0x82, 0x99, 0x27, + 0xb0, 0x2b, 0x5f, 0x99, 0x95, 0x37, 0xf0, 0x77, 0x94, 0x27, 0x66, 0xb7, 0x48, 0x0d, 0xf1, 0xc6, + 0x34, 0x33, 0xd8, 0xab, 0xae, 0x91, 0x0f, 0x90, 0x4d, 0x8b, 0xfe, 0x0f, 0x0f, 0xd3, 0x1b, 0xd8, + 0x79, 0x96, 0xc5, 0xf3, 0xaa, 0xa2, 0x43, 0x68, 0x7b, 0x81, 0xed, 0xa4, 0xde, 0x35, 0x97, 0x01, + 0x51, 0xc0, 0xf4, 0x62, 0xf5, 0xe4, 0xab, 0xba, 0x6e, 0xd1, 0x37, 0xf2, 0x5f, 0x78, 0x3e, 0xa7, + 0xf4, 0x94, 0x27, 0x93, 0xc3, 0xa4, 0x8a, 0xb8, 0x24, 0xc8, 0x07, 0xa2, 0x80, 0xd0, 0x7f, 0xf4, + 0xdc, 0x12, 0x2f, 0xd2, 0x71, 0x18, 0x5c, 0x78, 0xf3, 0xdc, 0x7f, 0x37, 0xf0, 0xd1, 0x0a, 0xda, + 0x5b, 0x5e, 0x64, 0xa5, 0xcd, 0xfa, 0x2a, 0x9b, 0xeb, 0x65, 0x4d, 0x56, 0x86, 0x17, 0x8d, 0xea, + 0xf0, 0xe2, 0xef, 0x1a, 0x0c, 0xe5, 0x29, 0x88, 0xbd, 0xa7, 0xce, 0x25, 0x5f, 0xd8, 0xb9, 0x5f, + 0xee, 0x2b, 0x67, 0x41, 0xb5, 0x44, 0x90, 0xcb, 0x79, 0x03, 0xbd, 0xe5, 0xf5, 0xf2, 0x2d, 0xbf, + 0xf6, 0x20, 0x86, 0xd0, 0x76, 0xed, 0xd4, 0x9e, 0xd9, 0x09, 0xcf, 0xe3, 0x35, 0x87, 0xe9, 0xdd, + 0x66, 0xcf, 0xfc, 0x3c, 0x5a, 0x05, 0x40, 0x92, 0x68, 0x37, 0x19, 0xa9, 0x12, 0x32, 0x7f, 0x0d, + 0x83, 0xc7, 0x76, 0xe0, 0xfa, 0x52, 0xe1, 0xca, 0x28, 0x60, 0x29, 0x76, 0x88, 0xba, 0x41, 0xdd, + 0xfb, 0xd0, 0x99, 0x15, 0x53, 0x18, 0xa1, 0x71, 0x89, 0xc0, 0x15, 0xc9, 0x2b, 0x3f, 0x91, 0x6f, + 0x19, 0xfa, 0x3e, 0xfa, 0x15, 0xb4, 0xc4, 0xa0, 0x84, 0xf5, 0xa1, 0xf3, 0x24, 0xb8, 0xb6, 0x7d, + 0xcf, 0x7d, 0x1a, 0x19, 0x35, 0xd6, 0x86, 0xc6, 0x34, 0x0d, 0x23, 0x43, 0x63, 0x1d, 0x68, 0x3e, + 0xc3, 0x94, 0x32, 0x74, 0x06, 0xd0, 0xc2, 0xaa, 0xb3, 0xe0, 0x46, 0x1d, 0xd1, 0x34, 0x83, 0x32, + 0x1a, 0x88, 0x16, 0xd3, 0x11, 0xa3, 0xc9, 0xb6, 0x01, 0x7e, 0x96, 0xa5, 0xa1, 0x64, 0x6b, 0x1d, + 0xbd, 0x22, 0xb6, 0x39, 0x9e, 0x63, 0x4f, 0xca, 0x27, 0xd8, 0xa8, 0xb1, 0x2d, 0xa8, 0x7f, 0xcd, + 0x6f, 0x0c, 0x8d, 0x75, 0x61, 0xcb, 0xca, 0x02, 0x7c, 0x8c, 0x8a, 0x3d, 0x68, 0x3b, 0xd7, 0xa8, + 0x23, 0x01, 0x95, 0x88, 0xb8, 0x6b, 0x34, 0x58, 0x0f, 0xda, 0x3f, 0x97, 0xaf, 0x4b, 0xa3, 0x89, + 0x24, 0x64, 0xc3, 0x35, 0x2d, 0x24, 0xd1, 0x86, 0x08, 0x6d, 0x1d, 0x3d, 0x85, 0x76, 0xde, 0x24, + 0xd8, 0x3d, 0xe8, 0xca, 0x5d, 0x11, 0x65, 0xd4, 0x50, 0x6d, 0x6a, 0x05, 0x86, 0x86, 0x26, 0x62, + 0xb9, 0x37, 0x74, 0xfc, 0xc2, 0x9a, 0x6e, 0xd4, 0xc9, 0xec, 0xdb, 0xc0, 0x31, 0x1a, 0xc8, 0x48, + 0x49, 0x65, 0xb8, 0x47, 0xe7, 0xb0, 0x25, 0xb3, 0x96, 0x31, 0xd8, 0x96, 0xf2, 0x24, 0xc6, 0xa8, + 0xa1, 0xe7, 0x50, 0x4b, 0xc1, 0xad, 0xa1, 0x07, 0xc8, 0x00, 0x01, 0xeb, 0xa8, 0x82, 0xf0, 0x86, + 0x40, 0xd4, 0x51, 0xbf, 0x3c, 0xf0, 0xd8, 0x2e, 0xdc, 0xcb, 0xbd, 0x22, 0x51, 0x42, 0xe0, 0x19, + 0x4f, 0x05, 0xc2, 0xd0, 0x48, 0x7e, 0x01, 0xea, 0xe8, 0x48, 0x8b, 0x2f, 0xc2, 0x6b, 0x2e, 0x31, + 0xf5, 0xa3, 0x2f, 0xa9, 0x1e, 0x63, 0x64, 0x90, 0xfe, 0x57, 0x1e, 0x0a, 0x41, 0xa7, 0xf2, 0xc8, + 0xb7, 0x1d, 0x6e, 0x68, 0x8a, 0xda, 0x92, 0xd1, 0xd0, 0x4f, 0xfe, 0xba, 0x05, 0x2d, 0x11, 0x74, + 0x6c, 0x0c, 0x3d, 0x75, 0x9e, 0xc8, 0x3e, 0x94, 0x8d, 0xf8, 0xee, 0x84, 0x71, 0x38, 0xa0, 0x56, + 0xba, 0x62, 0x14, 0x65, 0xd6, 0xd8, 0x13, 0xd8, 0xae, 0xce, 0xe6, 0xd8, 0x47, 0xc8, 0xbd, 0x72, + 0xf0, 0x37, 0x1c, 0xae, 0x22, 0x15, 0xa2, 0x1e, 0x41, 0xbf, 0x32, 0x6e, 0x63, 0xb4, 0xef, 0xaa, + 0x09, 0xdc, 0x46, 0x8d, 0x7e, 0x0a, 0x5d, 0x65, 0x86, 0xc5, 0xf6, 0x91, 0x75, 0x79, 0x34, 0x37, + 0xfc, 0x70, 0x09, 0x5f, 0x48, 0xf8, 0x0a, 0xa0, 0x1c, 0x2a, 0xb1, 0x0f, 0x0a, 0x46, 0x35, 0x4d, + 0x87, 0xfb, 0x77, 0xd1, 0xc5, 0xf2, 0x73, 0xd8, 0x59, 0x6a, 0x32, 0xec, 0x3e, 0x39, 0x77, 0x4d, + 0xef, 0xd9, 0x68, 0xcf, 0x18, 0x7a, 0x6a, 0x8f, 0x11, 0xc7, 0xb4, 0xa2, 0x53, 0x09, 0x21, 0xab, + 0xda, 0x91, 0x59, 0x63, 0x3f, 0x01, 0x28, 0x3b, 0x86, 0x30, 0x69, 0xa9, 0x83, 0x6c, 0xd4, 0xe2, + 0x0c, 0x76, 0x94, 0x89, 0xb2, 0xa8, 0xee, 0xc2, 0xb7, 0xcb, 0x83, 0xe6, 0x8d, 0x82, 0x2c, 0x39, + 0xfe, 0x54, 0xdb, 0x84, 0xf0, 0xce, 0xba, 0xce, 0x32, 0xfc, 0x78, 0x0d, 0x55, 0x75, 0x91, 0x3a, + 0xbe, 0x16, 0x2e, 0x5a, 0x31, 0xd0, 0xde, 0xa8, 0xd8, 0x2f, 0xa0, 0x9f, 0x87, 0x26, 0xa5, 0x18, + 0x1b, 0x29, 0xfe, 0x5c, 0xd1, 0x58, 0xde, 0xe2, 0xae, 0xae, 0x28, 0xf0, 0x22, 0x86, 0xc8, 0xbe, + 0x75, 0x15, 0x7f, 0x93, 0xa0, 0x87, 0x83, 0x7f, 0xbe, 0x1e, 0x69, 0xdf, 0xbc, 0x1e, 0x69, 0xff, + 0x79, 0x3d, 0xd2, 0x7e, 0xf7, 0x66, 0x54, 0xfb, 0xe6, 0xcd, 0xa8, 0xf6, 0xef, 0x37, 0xa3, 0xda, + 0xac, 0x45, 0x7f, 0x06, 0xbe, 0xff, 0xbf, 0x00, 0x00, 0x00, 0xff, 0xff, 0x42, 0x0d, 0x57, 0xbf, + 0x2b, 0x18, 0x00, 0x00, } // Reference imports to suppress errors if they are not otherwise used. @@ -2958,6 +3061,7 @@ type WorkerClient interface { // a `set`/`remove` operation should be an one-time operation (only take effect once), // so we use a gRPC method rather than a etcd operation now (no persistent operation state). OperateSchema(ctx context.Context, in *OperateWorkerSchemaRequest, opts ...grpc.CallOption) (*CommonWorkerResponse, error) + HandleError(ctx context.Context, in *HandleWorkerErrorRequest, opts ...grpc.CallOption) (*CommonWorkerResponse, error) } type workerClient struct { @@ -3076,6 +3180,15 @@ func (c *workerClient) OperateSchema(ctx context.Context, in *OperateWorkerSchem return out, nil } +func (c *workerClient) HandleError(ctx context.Context, in *HandleWorkerErrorRequest, opts ...grpc.CallOption) (*CommonWorkerResponse, error) { + out := new(CommonWorkerResponse) + err := c.cc.Invoke(ctx, "/pb.Worker/HandleError", 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) @@ -3097,6 +3210,7 @@ type WorkerServer interface { // a `set`/`remove` operation should be an one-time operation (only take effect once), // so we use a gRPC method rather than a etcd operation now (no persistent operation state). OperateSchema(context.Context, *OperateWorkerSchemaRequest) (*CommonWorkerResponse, error) + HandleError(context.Context, *HandleWorkerErrorRequest) (*CommonWorkerResponse, error) } // UnimplementedWorkerServer can be embedded to have forward compatible implementations. @@ -3139,6 +3253,9 @@ func (*UnimplementedWorkerServer) MigrateRelay(ctx context.Context, req *Migrate func (*UnimplementedWorkerServer) OperateSchema(ctx context.Context, req *OperateWorkerSchemaRequest) (*CommonWorkerResponse, error) { return nil, status.Errorf(codes.Unimplemented, "method OperateSchema not implemented") } +func (*UnimplementedWorkerServer) HandleError(ctx context.Context, req *HandleWorkerErrorRequest) (*CommonWorkerResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method HandleError not implemented") +} func RegisterWorkerServer(s *grpc.Server, srv WorkerServer) { s.RegisterService(&_Worker_serviceDesc, srv) @@ -3360,6 +3477,24 @@ func _Worker_OperateSchema_Handler(srv interface{}, ctx context.Context, dec fun return interceptor(ctx, in, info, handler) } +func _Worker_HandleError_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(HandleWorkerErrorRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(WorkerServer).HandleError(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: "/pb.Worker/HandleError", + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(WorkerServer).HandleError(ctx, req.(*HandleWorkerErrorRequest)) + } + return interceptor(ctx, in, info, handler) +} + var _Worker_serviceDesc = grpc.ServiceDesc{ ServiceName: "pb.Worker", HandlerType: (*WorkerServer)(nil), @@ -3412,6 +3547,10 @@ var _Worker_serviceDesc = grpc.ServiceDesc{ MethodName: "OperateSchema", Handler: _Worker_OperateSchema_Handler, }, + { + MethodName: "HandleError", + Handler: _Worker_HandleError_Handler, + }, }, Streams: []grpc.StreamDesc{}, Metadata: "dmworker.proto", @@ -5384,6 +5523,57 @@ func (m *OperateWorkerSchemaRequest) MarshalToSizedBuffer(dAtA []byte) (int, err return len(dAtA) - i, nil } +func (m *HandleWorkerErrorRequest) 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 *HandleWorkerErrorRequest) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *HandleWorkerErrorRequest) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + _ = i + var l int + _ = l + if len(m.Sqls) > 0 { + for iNdEx := len(m.Sqls) - 1; iNdEx >= 0; iNdEx-- { + i -= len(m.Sqls[iNdEx]) + copy(dAtA[i:], m.Sqls[iNdEx]) + i = encodeVarintDmworker(dAtA, i, uint64(len(m.Sqls[iNdEx]))) + i-- + dAtA[i] = 0x22 + } + } + 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] = 0x1a + } + 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] = 0x12 + } + if m.Op != 0 { + i = encodeVarintDmworker(dAtA, i, uint64(m.Op)) + i-- + dAtA[i] = 0x8 + } + return len(dAtA) - i, nil +} + func encodeVarintDmworker(dAtA []byte, offset int, v uint64) int { offset -= sovDmworker(v) base := offset @@ -6304,6 +6494,32 @@ func (m *OperateWorkerSchemaRequest) Size() (n int) { return n } +func (m *HandleWorkerErrorRequest) 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.Task) + if l > 0 { + n += 1 + l + sovDmworker(uint64(l)) + } + l = len(m.BinlogPos) + if l > 0 { + n += 1 + l + sovDmworker(uint64(l)) + } + if len(m.Sqls) > 0 { + for _, s := range m.Sqls { + l = len(s) + n += 1 + l + sovDmworker(uint64(l)) + } + } + return n +} + func sovDmworker(x uint64) (n int) { return (math_bits.Len64(x|1) + 6) / 7 } @@ -11944,6 +12160,174 @@ func (m *OperateWorkerSchemaRequest) Unmarshal(dAtA []byte) error { } return nil } +func (m *HandleWorkerErrorRequest) 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: HandleWorkerErrorRequest: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: HandleWorkerErrorRequest: 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 |= ErrorOp(b&0x7F) << shift + if b < 0x80 { + break + } + } + 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 ErrIntOverflowDmworker + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLengthDmworker + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthDmworker + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Task = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field BinlogPos", 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.BinlogPos = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 4: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Sqls", 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.Sqls = append(m.Sqls, 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 skipDmworker(dAtA []byte) (n int, err error) { l := len(dAtA) iNdEx := 0 diff --git a/dm/pbmock/dmmaster.go b/dm/pbmock/dmmaster.go index 4e3a0e4250..eab8fdb907 100644 --- a/dm/pbmock/dmmaster.go +++ b/dm/pbmock/dmmaster.go @@ -95,6 +95,26 @@ func (mr *MockMasterClientMockRecorder) GetTaskCfg(arg0, arg1 interface{}, arg2 return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "GetTaskCfg", reflect.TypeOf((*MockMasterClient)(nil).GetTaskCfg), varargs...) } +// HandleError mocks base method +func (m *MockMasterClient) HandleError(arg0 context.Context, arg1 *pb.HandleErrorRequest, arg2 ...grpc.CallOption) (*pb.HandleErrorResponse, error) { + m.ctrl.T.Helper() + varargs := []interface{}{arg0, arg1} + for _, a := range arg2 { + varargs = append(varargs, a) + } + ret := m.ctrl.Call(m, "HandleError", varargs...) + ret0, _ := ret[0].(*pb.HandleErrorResponse) + ret1, _ := ret[1].(error) + return ret0, ret1 +} + +// HandleError indicates an expected call of HandleError +func (mr *MockMasterClientMockRecorder) HandleError(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, "HandleError", reflect.TypeOf((*MockMasterClient)(nil).HandleError), varargs...) +} + // ListMember mocks base method func (m *MockMasterClient) ListMember(arg0 context.Context, arg1 *pb.ListMemberRequest, arg2 ...grpc.CallOption) (*pb.ListMemberResponse, error) { m.ctrl.T.Helper() @@ -543,6 +563,21 @@ func (mr *MockMasterServerMockRecorder) GetTaskCfg(arg0, arg1 interface{}) *gomo return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "GetTaskCfg", reflect.TypeOf((*MockMasterServer)(nil).GetTaskCfg), arg0, arg1) } +// HandleError mocks base method +func (m *MockMasterServer) HandleError(arg0 context.Context, arg1 *pb.HandleErrorRequest) (*pb.HandleErrorResponse, error) { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "HandleError", arg0, arg1) + ret0, _ := ret[0].(*pb.HandleErrorResponse) + ret1, _ := ret[1].(error) + return ret0, ret1 +} + +// HandleError indicates an expected call of HandleError +func (mr *MockMasterServerMockRecorder) HandleError(arg0, arg1 interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "HandleError", reflect.TypeOf((*MockMasterServer)(nil).HandleError), arg0, arg1) +} + // ListMember mocks base method func (m *MockMasterServer) ListMember(arg0 context.Context, arg1 *pb.ListMemberRequest) (*pb.ListMemberResponse, error) { m.ctrl.T.Helper() diff --git a/dm/pbmock/dmworker.go b/dm/pbmock/dmworker.go index ae1114faf5..b046be6591 100644 --- a/dm/pbmock/dmworker.go +++ b/dm/pbmock/dmworker.go @@ -35,6 +35,26 @@ func (m *MockWorkerClient) EXPECT() *MockWorkerClientMockRecorder { return m.recorder } +// HandleError mocks base method +func (m *MockWorkerClient) HandleError(arg0 context.Context, arg1 *pb.HandleWorkerErrorRequest, arg2 ...grpc.CallOption) (*pb.CommonWorkerResponse, error) { + m.ctrl.T.Helper() + varargs := []interface{}{arg0, arg1} + for _, a := range arg2 { + varargs = append(varargs, a) + } + ret := m.ctrl.Call(m, "HandleError", varargs...) + ret0, _ := ret[0].(*pb.CommonWorkerResponse) + ret1, _ := ret[1].(error) + return ret0, ret1 +} + +// HandleError indicates an expected call of HandleError +func (mr *MockWorkerClientMockRecorder) HandleError(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, "HandleError", reflect.TypeOf((*MockWorkerClient)(nil).HandleError), varargs...) +} + // MigrateRelay mocks base method func (m *MockWorkerClient) MigrateRelay(arg0 context.Context, arg1 *pb.MigrateRelayRequest, arg2 ...grpc.CallOption) (*pb.CommonWorkerResponse, error) { m.ctrl.T.Helper() @@ -298,6 +318,21 @@ func (m *MockWorkerServer) EXPECT() *MockWorkerServerMockRecorder { return m.recorder } +// HandleError mocks base method +func (m *MockWorkerServer) HandleError(arg0 context.Context, arg1 *pb.HandleWorkerErrorRequest) (*pb.CommonWorkerResponse, error) { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "HandleError", arg0, arg1) + ret0, _ := ret[0].(*pb.CommonWorkerResponse) + ret1, _ := ret[1].(error) + return ret0, ret1 +} + +// HandleError indicates an expected call of HandleError +func (mr *MockWorkerServerMockRecorder) HandleError(arg0, arg1 interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "HandleError", reflect.TypeOf((*MockWorkerServer)(nil).HandleError), arg0, arg1) +} + // MigrateRelay mocks base method func (m *MockWorkerServer) MigrateRelay(arg0 context.Context, arg1 *pb.MigrateRelayRequest) (*pb.CommonWorkerResponse, error) { m.ctrl.T.Helper() diff --git a/dm/proto/dmmaster.proto b/dm/proto/dmmaster.proto index 23c93d0e00..48dc52da27 100644 --- a/dm/proto/dmmaster.proto +++ b/dm/proto/dmmaster.proto @@ -106,6 +106,13 @@ service Master { get: "/apis/v1alpha1/tasks/{name}" }; } + + rpc HandleError(HandleErrorRequest) returns(HandleErrorResponse) { + option (google.api.http) = { + put: "/apis/v1alpha1/errors" + body: "*" + }; + } } message MigrateWorkerRelayRequest { @@ -439,4 +446,18 @@ message GetTaskCfgResponse { bool result = 1; string msg = 2; string cfg = 3; +} + +message HandleErrorRequest { + ErrorOp op = 1; // operation type + string task = 2; // the task name + repeated string sources = 3; // source ID list + string binlogPos = 4; // binlog-pos (that's file:pos format) + repeated string sqls = 5; // sqls (use for replace) +} + +message HandleErrorResponse { + bool result = 1; + string msg = 2; + repeated CommonWorkerResponse sources = 3; } \ No newline at end of file diff --git a/dm/proto/dmworker.proto b/dm/proto/dmworker.proto index 25b6c584e8..a9fddae400 100644 --- a/dm/proto/dmworker.proto +++ b/dm/proto/dmworker.proto @@ -31,6 +31,8 @@ service Worker { // a `set`/`remove` operation should be an one-time operation (only take effect once), // so we use a gRPC method rather than a etcd operation now (no persistent operation state). rpc OperateSchema(OperateWorkerSchemaRequest) returns(CommonWorkerResponse) {} + + rpc HandleError(HandleWorkerErrorRequest) returns(CommonWorkerResponse) {} } message StartSubTaskRequest { @@ -401,4 +403,17 @@ message OperateWorkerSchemaRequest { string database = 4; // database name string table = 5; // table name string schema = 6; // schema content, a `CREATE TABLE` statement +} + +enum ErrorOp { + Skip = 0; // skip the error event + Replace = 1; // replace the error event with a specified SQL + InvalidErrorOp = 2; +} + +message HandleWorkerErrorRequest { + ErrorOp op = 1; // operation type + string task = 2; // task name + string binlogPos = 3; // binlog-pos (that's file:pos format) + repeated string sqls = 4; // sqls (use for replace) } \ No newline at end of file diff --git a/dm/worker/server.go b/dm/worker/server.go index 7eb85915e2..87fc56551e 100644 --- a/dm/worker/server.go +++ b/dm/worker/server.go @@ -959,3 +959,24 @@ func unifyMasterBinlogPos(resp *pb.QueryStatusResponse, enableGTID bool) { } } } + +// HandleError handle error +func (s *Server) HandleError(ctx context.Context, req *pb.HandleWorkerErrorRequest) (*pb.CommonWorkerResponse, error) { + log.L().Info("", zap.String("request", "HandleError"), zap.Stringer("payload", req)) + + w := s.getWorker(true) + if w == nil { + log.L().Error("fail to call HandleError, because mysql worker has not been started") + return makeCommonWorkerResponse(terror.ErrWorkerNoStart.Generate()), nil + } + + msg, err := w.HandleError(ctx, req) + if err != nil { + return makeCommonWorkerResponse(err), nil + } + return &pb.CommonWorkerResponse{ + Result: true, + Msg: msg, + Worker: s.cfg.Name, + }, nil +} diff --git a/dm/worker/subtask.go b/dm/worker/subtask.go index 80610a7d12..e880ba17c5 100644 --- a/dm/worker/subtask.go +++ b/dm/worker/subtask.go @@ -635,3 +635,21 @@ func (st *SubTask) fail(err error) { }, }) } + +// HandleError handle error for syncer unit +func (st *SubTask) HandleError(ctx context.Context, req *pb.HandleWorkerErrorRequest) (string, error) { + syncUnit, ok := st.currUnit.(*syncer.Syncer) + if !ok { + return "", terror.ErrWorkerOperSyncUnitOnly.Generate(st.currUnit.Type()) + } + + msg, err := syncUnit.HandleError(ctx, req) + if err != nil { + return msg, err + } + + if st.Stage() == pb.Stage_Paused { + err = st.Resume() + } + return msg, err +} diff --git a/dm/worker/worker.go b/dm/worker/worker.go index 4b39b96e8b..a9dcd6048a 100644 --- a/dm/worker/worker.go +++ b/dm/worker/worker.go @@ -811,3 +811,20 @@ func (w *Worker) getAllSubTaskStatus() map[string]*pb.SubTaskStatus { } return result } + +// HandleError handle worker error +func (w *Worker) HandleError(ctx context.Context, req *pb.HandleWorkerErrorRequest) (string, error) { + w.Lock() + defer w.Unlock() + + if w.closed.Get() == closedTrue { + return "", terror.ErrWorkerAlreadyClosed.Generate() + } + + st := w.subTaskHolder.findSubTask(req.Task) + if st == nil { + return "", terror.ErrWorkerSubTaskNotFound.Generate(req.Task) + } + + return st.HandleError(ctx, req) +} diff --git a/errors.toml b/errors.toml index f830c3159d..6ee59ef440 100644 --- a/errors.toml +++ b/errors.toml @@ -130,12 +130,6 @@ description = "" workaround = "" tags = ["internal", "high"] -[error.DM-functional-11017] -message = "" -description = "" -workaround = "Please make sure the args are correct." -tags = ["internal", "low"] - [error.DM-functional-11018] message = "get file statfs" description = "" @@ -718,6 +712,12 @@ description = "" workaround = "Please use `list-member --master` to confirm whether the DM-master cluster is healthy" tags = ["internal", "high"] +[error.DM-functional-11115] +message = "" +description = "" +workaround = "Please make sure the args are correct." +tags = ["internal", "low"] + [error.DM-config-20001] message = "checking item %s is not supported\n%s" description = "" diff --git a/pkg/terror/error_list.go b/pkg/terror/error_list.go index 421d8f4281..d682d1affa 100644 --- a/pkg/terror/error_list.go +++ b/pkg/terror/error_list.go @@ -159,6 +159,8 @@ const ( // pkg/upgrade codeUpgradeVersionEtcdFail + + codeVerifyHandleErrorArgs ) // Config related error code list @@ -605,12 +607,12 @@ var ( ErrMariaDBDomainID = New(codeMariaDBDomainID, ClassFunctional, ScopeInternal, LevelHigh, "%v is not uint32", "") ErrInvalidServerID = New(codeInvalidServerID, ClassFunctional, ScopeInternal, LevelHigh, "invalid server id %s", "") ErrGetSQLModeFromStr = New(codeGetSQLModeFromStr, ClassFunctional, ScopeInternal, LevelHigh, "get sql mode from string literal %s", "") - ErrVerifySQLOperateArgs = New(codeVerifySQLOperateArgs, ClassFunctional, ScopeInternal, LevelLow, "", "Please make sure the args are correct.") ErrStatFileSize = New(codeStatFileSize, ClassFunctional, ScopeInternal, LevelHigh, "get file statfs", "") ErrReaderAlreadyRunning = New(codeReaderAlreadyRunning, ClassFunctional, ScopeInternal, LevelHigh, "binlog reader is already running", "") ErrReaderAlreadyStarted = New(codeReaderAlreadyStarted, ClassFunctional, ScopeInternal, LevelHigh, "stage %s, expect %s, already started", "") ErrReaderStateCannotClose = New(codeReaderStateCannotClose, ClassFunctional, ScopeInternal, LevelHigh, "stage %s, expect %s, can not close", "") ErrReaderShouldStartSync = New(codeReaderShouldStartSync, ClassFunctional, ScopeInternal, LevelHigh, "stage %s, expect %s", "") + ErrVerifyHandleErrorArgs = New(codeVerifyHandleErrorArgs, ClassFunctional, ScopeInternal, LevelLow, "", "Please make sure the args are correct.") // pkg/streamer ErrEmptyRelayDir = New(codeEmptyRelayDir, ClassFunctional, ScopeInternal, LevelHigh, "empty relay dir", "Please check `relay-dir` config in task configuration file.") ErrReadDir = New(codeReadDir, ClassFunctional, ScopeInternal, LevelHigh, "read dir: %s", "") diff --git a/syncer/handle_error.go b/syncer/handle_error.go new file mode 100644 index 0000000000..298143345d --- /dev/null +++ b/syncer/handle_error.go @@ -0,0 +1,25 @@ +// Copyright 2020 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package syncer + +import ( + "context" + + "github.com/pingcap/dm/dm/pb" +) + +// HandleError handle error for syncer +func (s *Syncer) HandleError(ctx context.Context, req *pb.HandleWorkerErrorRequest) (string, error) { + return "", nil +} diff --git a/tests/dmctl_advance/check_list/handle_error.sh b/tests/dmctl_advance/check_list/handle_error.sh new file mode 100644 index 0000000000..6428499c33 --- /dev/null +++ b/tests/dmctl_advance/check_list/handle_error.sh @@ -0,0 +1,27 @@ +#!/bin/bash + +function handle_error_wrong_arg() { + run_dm_ctl $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "handle-error" \ + "handle-error \[-s source ...\] \[-b binlog-pos\] \[replace-sql1;replace-sql2;\] \[flags\]" 1 +} + +function handle_error_invalid_binlogpos() { + binlog_pos="mysql-bin:shoud-bin-digital" + run_dm_ctl $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "handle-error test-task --binlog-pos $binlog_pos skip" \ + "\[.*\], Message: invalid --binlog-pos $binlog_pos in handle-error operation: the pos should be digital" 1 +} + +function handle_error_invalid_sqls() { + sqls="alter table tb add column a int; alter table tb2 b int;" + run_dm_ctl $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "handle-error test-task replace $sqls" \ + "invalid sql" 1 +} + +function handle_error_invalid_op() { + run_dm_ctl $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "handle-error test-task wrong_operation" \ + "invalid operation 'wrong_operation'" 1 +} \ No newline at end of file diff --git a/tests/dmctl_advance/run.sh b/tests/dmctl_advance/run.sh index b777dec292..18f9b3ba02 100755 --- a/tests/dmctl_advance/run.sh +++ b/tests/dmctl_advance/run.sh @@ -23,6 +23,11 @@ function usage_and_arg_test() { query_error_wrong_arg + handle_error_wrong_arg + handle_error_invalid_binlogpos + handle_error_invalid_sqls + handle_error_invalid_op + # TODO: check SQLs error test } diff --git a/tests/dmctl_command/run.sh b/tests/dmctl_command/run.sh index 963f22e407..bca21abb6a 100644 --- a/tests/dmctl_command/run.sh +++ b/tests/dmctl_command/run.sh @@ -6,7 +6,7 @@ cur=$( cd "$( dirname "${BASH_SOURCE[0]}" )" && pwd ) source $cur/../_utils/test_prepare WORK_DIR=$TEST_DIR/$TEST_NAME -help_cnt=39 +help_cnt=40 function run() { # check dmctl alone output From fab914fb33f8848bdfe924f265cd9e78de4d17e9 Mon Sep 17 00:00:00 2001 From: gmhdbjd Date: Fri, 31 Jul 2020 16:17:56 -0600 Subject: [PATCH 03/31] return error --- dm/ctl/master/handle_error.go | 15 +++++++-------- 1 file changed, 7 insertions(+), 8 deletions(-) diff --git a/dm/ctl/master/handle_error.go b/dm/ctl/master/handle_error.go index ef6276cc14..a1ee9e5028 100644 --- a/dm/ctl/master/handle_error.go +++ b/dm/ctl/master/handle_error.go @@ -15,6 +15,7 @@ package master import ( "context" + "errors" "os" "github.com/spf13/cobra" @@ -29,7 +30,7 @@ func NewHandleErrorCmd() *cobra.Command { cmd := &cobra.Command{ Use: "handle-error [-s source ...] [-b binlog-pos] [replace-sql1;replace-sql2;]", Short: "skip/replace the current error event or a specific binlog position (binlog-pos) event", - Run: handleErrorFunc, + RunE: handleErrorFunc, } cmd.Flags().StringP("binlog-pos", "b", "", "position used to match binlog event if matched the handler-error operation will be applied. The format like \"mysql-bin|000001.000003:3270\"") return cmd @@ -47,10 +48,11 @@ func convertOp(t string) pb.ErrorOp { } // handleErrorFunc does handle error request -func handleErrorFunc(cmd *cobra.Command, _ []string) { +func handleErrorFunc(cmd *cobra.Command, _ []string) (err error) { if len(cmd.Flags().Args()) < 2 { cmd.SetOut(os.Stdout) cmd.Usage() + err = errors.New("please check output to see error") return } @@ -67,36 +69,33 @@ func handleErrorFunc(cmd *cobra.Command, _ []string) { case pb.ErrorOp_Replace: if len(cmd.Flags().Args()) <= 2 { common.PrintLines("must specify the replace-sqls for replace operation") + err = errors.New("please check output to see error") return } - var err error sqls, err = common.ExtractSQLsFromArgs(cmd.Flags().Args()[2:]) if err != nil { - common.PrintLines("%v", err) return } default: common.PrintLines("invalid operation '%s', please use `skip` or `relpace`", operation) + err = errors.New("please check output to see error") return } binlogPos, err := cmd.Flags().GetString("binlog-pos") if err != nil { - common.PrintLines("%v", err) return } if len(binlogPos) != 0 { _, err = command.VerifyBinlogPos(binlogPos) if err != nil { - common.PrintLines("%v", err) return } } sources, err := common.GetSourceArgs(cmd) if err != nil { - common.PrintLines("%v", err) return } @@ -112,9 +111,9 @@ func handleErrorFunc(cmd *cobra.Command, _ []string) { Sources: sources, }) if err != nil { - common.PrintLines("can not handle error:\n%v", err) return } common.PrettyPrintResponse(resp) + return } From d1846823d58b1f0fc0b7b2f89544e7c9823d8d0e Mon Sep 17 00:00:00 2001 From: gmhdbjd Date: Fri, 31 Jul 2020 17:58:37 +0800 Subject: [PATCH 04/31] revert remove errors --- _utils/terror_gen/errors_release.txt | 11 +++++- errors.toml | 54 ++++++++++++++++++++++++++++ pkg/terror/error_list.go | 12 ++++++- 3 files changed, 75 insertions(+), 2 deletions(-) diff --git a/_utils/terror_gen/errors_release.txt b/_utils/terror_gen/errors_release.txt index f2875f9afc..2de12ec5a6 100644 --- a/_utils/terror_gen/errors_release.txt +++ b/_utils/terror_gen/errors_release.txt @@ -20,12 +20,12 @@ ErrNotUUIDString,[code=11013:class=functional:scope=internal:level=high], "Messa ErrMariaDBDomainID,[code=11014:class=functional:scope=internal:level=high], "Message: %v is not uint32" ErrInvalidServerID,[code=11015:class=functional:scope=internal:level=high], "Message: invalid server id %s" ErrGetSQLModeFromStr,[code=11016:class=functional:scope=internal:level=high], "Message: get sql mode from string literal %s" +ErrVerifySQLOperateArgs,[code=11017:class=functional:scope=internal:level=low], "Workaround: Please make sure the args are correct." ErrStatFileSize,[code=11018:class=functional:scope=internal:level=high], "Message: get file statfs" ErrReaderAlreadyRunning,[code=11019:class=functional:scope=internal:level=high], "Message: binlog reader is already running" ErrReaderAlreadyStarted,[code=11020:class=functional:scope=internal:level=high], "Message: stage %s, expect %s, already started" ErrReaderStateCannotClose,[code=11021:class=functional:scope=internal:level=high], "Message: stage %s, expect %s, can not close" ErrReaderShouldStartSync,[code=11022:class=functional:scope=internal:level=high], "Message: stage %s, expect %s" -ErrVerifyHandleErrorArgs,[code=11115:class=functional:scope=internal:level=low], "Workaround: Please make sure the args are correct." ErrEmptyRelayDir,[code=11023:class=functional:scope=internal:level=high], "Message: empty relay dir, Workaround: Please check `relay-dir` config in task configuration file." ErrReadDir,[code=11024:class=functional:scope=internal:level=high], "Message: read dir: %s" ErrBaseFileNotFound,[code=11025:class=functional:scope=internal:level=high], "Message: base file %s in directory %s not found" @@ -118,6 +118,7 @@ ErrShardDDLOptimismTrySyncFail,[code=11111:class=functional:scope=internal:level ErrConnInvalidTLSConfig,[code=11112:class=functional:scope=internal:level=medium], "Message: invalid TLS config, Workaround: Please check the `ssl-ca`, `ssl-cert` and `ssl-key` config." ErrConnRegistryTLSConfig,[code=11113:class=functional:scope=internal:level=medium], "Message: fail to registry TLS config" ErrUpgradeVersionEtcdFail,[code=11114:class=functional:scope=internal:level=high], "Message: fail to operate DM cluster version in etcd, Workaround: Please use `list-member --master` to confirm whether the DM-master cluster is healthy" +ErrVerifyHandleErrorArgs,[code=11115:class=functional:scope=internal:level=low], "Workaround: Please make sure the args are correct." ErrConfigCheckItemNotSupport,[code=20001:class=config:scope=internal:level=medium], "Message: checking item %s is not supported\n%s, Workaround: Please check `ignore-checking-items` config in task configuration file, which can be set including `all`/`dump_privilege`/`replication_privilege`/`version`/`binlog_enable`/`binlog_format`/`binlog_row_image`/`table_schema`/`schema_of_shard_tables`/`auto_increment_ID`." ErrConfigTomlTransform,[code=20002:class=config:scope=internal:level=medium], "Message: %s, Workaround: Please check the configuration file has correct TOML format." ErrConfigYamlTransform,[code=20003:class=config:scope=internal:level=medium], "Message: %s, Workaround: Please check the configuration file has correct YAML format." @@ -258,6 +259,10 @@ ErrSyncerUnitDDLExecChanCloseOrBusy,[code=36019:class=sync-unit:scope=internal:l ErrSyncerUnitDDLChanDone,[code=36020:class=sync-unit:scope=internal:level=high], "Message: canceled from external" ErrSyncerUnitDDLChanCanceled,[code=36021:class=sync-unit:scope=internal:level=high], "Message: canceled by Close or Renew" ErrSyncerUnitDDLOnMultipleTable,[code=36022:class=sync-unit:scope=internal:level=high], "Message: ddl on multiple table: %s not supported, Workaround: It is recommended to include only one DDL operation in a statement executed upstream. Please manually handle it using dmctl (skipping the DDL statement or replacing the DDL statement with a specified DDL statement). For details, see https://docs.pingcap.com/tidb-data-migration/stable/skip-or-replace-abnormal-sql-statements" +ErrSyncerUnitInjectDDLOnly,[code=36023:class=sync-unit:scope=internal:level=low], "Message: only support inject DDL for sharding group to be synced currently, but got %s" +ErrSyncerUnitInjectDDLWithoutSchema,[code=36024:class=sync-unit:scope=internal:level=low], "Message: injected DDL %s without schema name not valid" +ErrSyncerUnitNotSupportedOperate,[code=36025:class=sync-unit:scope=internal:level=medium], "Message: op %s not supported" +ErrSyncerUnitNilOperatorReq,[code=36026:class=sync-unit:scope=internal:level=medium], "Message: nil request not valid" ErrSyncerUnitDMLColumnNotMatch,[code=36027:class=sync-unit:scope=internal:level=high], "Message: Column count doesn't match value count: %d (columns) vs %d (values)" ErrSyncerUnitDMLOldNewValueMismatch,[code=36028:class=sync-unit:scope=internal:level=high], "Message: Old value count doesn't match new value count: %d (old) vs %d (new)" ErrSyncerUnitDMLPruneColumnMismatch,[code=36029:class=sync-unit:scope=internal:level=high], "Message: prune DML columns and data mismatch in length: %d (columns) %d (data)" @@ -295,6 +300,9 @@ ErrSyncerUnitGenBAList,[code=36060:class=sync-unit:scope=internal:level=high], " ErrSyncerUnitHandleDDLFailed,[code=36061:class=sync-unit:scope=internal:level=high], "Message: fail to handle ddl job for %s" ErrSyncerShardDDLConflict,[code=36062:class=sync-unit:scope=internal:level=high], "Message: fail to handle shard ddl %v in optimistic mode, because schema conflict detected, Workaround: Please use show-ddl-locks command for more details." ErrSyncerFailpoint,[code=36063:class=sync-unit:scope=internal:level=low], "Message: failpoint specified error" +ErrMasterSQLOpNilRequest,[code=38001:class=dm-master:scope=internal:level=medium], "Message: nil request not valid" +ErrMasterSQLOpNotSupport,[code=38002:class=dm-master:scope=internal:level=medium], "Message: op %s not supported" +ErrMasterSQLOpWithoutSharding,[code=38003:class=dm-master:scope=internal:level=medium], "Message: operate request without --sharding specified not valid" ErrMasterGRPCCreateConn,[code=38004:class=dm-master:scope=internal:level=high], "Message: create grpc connection" ErrMasterGRPCSendOnCloseConn,[code=38005:class=dm-master:scope=internal:level=high], "Message: send request on a closed client" ErrMasterGRPCClientClose,[code=38006:class=dm-master:scope=internal:level=high], "Message: close rpc client" @@ -314,6 +322,7 @@ ErrMasterLockNotFound,[code=38019:class=dm-master:scope=internal:level=high], "M ErrMasterLockIsResolving,[code=38020:class=dm-master:scope=internal:level=high], "Message: lock %s is resolving" ErrMasterWorkerCliNotFound,[code=38021:class=dm-master:scope=internal:level=high], "Message: source %s relevant worker-client not found" ErrMasterWorkerNotWaitLock,[code=38022:class=dm-master:scope=internal:level=high], "Message: worker %s not waiting for DDL lock %s" +ErrMasterHandleSQLReqFail,[code=38023:class=dm-master:scope=internal:level=high], "Message: request DDL lock %s owner %s handle SQLs request %s fail %s" ErrMasterOwnerExecDDL,[code=38024:class=dm-master:scope=internal:level=high], "Message: owner %s ExecuteDDL fail" ErrMasterPartWorkerExecDDLFail,[code=38025:class=dm-master:scope=internal:level=high], "Message: DDL lock %s owner ExecuteDDL successfully, so DDL lock removed. but some dm-workers ExecuteDDL fail, you should to handle dm-worker directly" ErrMasterWorkerExistDDLLock,[code=38026:class=dm-master:scope=internal:level=high], "Message: worker %s exist ddl lock, Workaround: Please unlock ddl lock first." diff --git a/errors.toml b/errors.toml index 6ee59ef440..58d7a9e5cc 100644 --- a/errors.toml +++ b/errors.toml @@ -130,6 +130,12 @@ description = "" workaround = "" tags = ["internal", "high"] +[error.DM-functional-11017] +message = "" +description = "" +workaround = "Please make sure the args are correct." +tags = ["internal", "low"] + [error.DM-functional-11018] message = "get file statfs" description = "" @@ -1558,6 +1564,30 @@ description = "" workaround = "It is recommended to include only one DDL operation in a statement executed upstream. Please manually handle it using dmctl (skipping the DDL statement or replacing the DDL statement with a specified DDL statement). For details, see https://docs.pingcap.com/tidb-data-migration/stable/skip-or-replace-abnormal-sql-statements" tags = ["internal", "high"] +[error.DM-sync-unit-36023] +message = "only support inject DDL for sharding group to be synced currently, but got %s" +description = "" +workaround = "" +tags = ["internal", "low"] + +[error.DM-sync-unit-36024] +message = "injected DDL %s without schema name not valid" +description = "" +workaround = "" +tags = ["internal", "low"] + +[error.DM-sync-unit-36025] +message = "op %s not supported" +description = "" +workaround = "" +tags = ["internal", "medium"] + +[error.DM-sync-unit-36026] +message = "nil request not valid" +description = "" +workaround = "" +tags = ["internal", "medium"] + [error.DM-sync-unit-36027] message = "Column count doesn't match value count: %d (columns) vs %d (values)" description = "" @@ -1780,6 +1810,24 @@ description = "" workaround = "" tags = ["internal", "low"] +[error.DM-dm-master-38001] +message = "nil request not valid" +description = "" +workaround = "" +tags = ["internal", "medium"] + +[error.DM-dm-master-38002] +message = "op %s not supported" +description = "" +workaround = "" +tags = ["internal", "medium"] + +[error.DM-dm-master-38003] +message = "operate request without --sharding specified not valid" +description = "" +workaround = "" +tags = ["internal", "medium"] + [error.DM-dm-master-38004] message = "create grpc connection" description = "" @@ -1894,6 +1942,12 @@ description = "" workaround = "" tags = ["internal", "high"] +[error.DM-dm-master-38023] +message = "request DDL lock %s owner %s handle SQLs request %s fail %s" +description = "" +workaround = "" +tags = ["internal", "high"] + [error.DM-dm-master-38024] message = "owner %s ExecuteDDL fail" description = "" diff --git a/pkg/terror/error_list.go b/pkg/terror/error_list.go index d682d1affa..9700701dad 100644 --- a/pkg/terror/error_list.go +++ b/pkg/terror/error_list.go @@ -607,12 +607,12 @@ var ( ErrMariaDBDomainID = New(codeMariaDBDomainID, ClassFunctional, ScopeInternal, LevelHigh, "%v is not uint32", "") ErrInvalidServerID = New(codeInvalidServerID, ClassFunctional, ScopeInternal, LevelHigh, "invalid server id %s", "") ErrGetSQLModeFromStr = New(codeGetSQLModeFromStr, ClassFunctional, ScopeInternal, LevelHigh, "get sql mode from string literal %s", "") + ErrVerifySQLOperateArgs = New(codeVerifySQLOperateArgs, ClassFunctional, ScopeInternal, LevelLow, "", "Please make sure the args are correct.") ErrStatFileSize = New(codeStatFileSize, ClassFunctional, ScopeInternal, LevelHigh, "get file statfs", "") ErrReaderAlreadyRunning = New(codeReaderAlreadyRunning, ClassFunctional, ScopeInternal, LevelHigh, "binlog reader is already running", "") ErrReaderAlreadyStarted = New(codeReaderAlreadyStarted, ClassFunctional, ScopeInternal, LevelHigh, "stage %s, expect %s, already started", "") ErrReaderStateCannotClose = New(codeReaderStateCannotClose, ClassFunctional, ScopeInternal, LevelHigh, "stage %s, expect %s, can not close", "") ErrReaderShouldStartSync = New(codeReaderShouldStartSync, ClassFunctional, ScopeInternal, LevelHigh, "stage %s, expect %s", "") - ErrVerifyHandleErrorArgs = New(codeVerifyHandleErrorArgs, ClassFunctional, ScopeInternal, LevelLow, "", "Please make sure the args are correct.") // pkg/streamer ErrEmptyRelayDir = New(codeEmptyRelayDir, ClassFunctional, ScopeInternal, LevelHigh, "empty relay dir", "Please check `relay-dir` config in task configuration file.") ErrReadDir = New(codeReadDir, ClassFunctional, ScopeInternal, LevelHigh, "read dir: %s", "") @@ -726,6 +726,8 @@ var ( // pkg/upgrade ErrUpgradeVersionEtcdFail = New(codeUpgradeVersionEtcdFail, ClassFunctional, ScopeInternal, LevelHigh, "fail to operate DM cluster version in etcd", "Please use `list-member --master` to confirm whether the DM-master cluster is healthy") + ErrVerifyHandleErrorArgs = New(codeVerifyHandleErrorArgs, ClassFunctional, ScopeInternal, LevelLow, "", "Please make sure the args are correct.") + // Config related error ErrConfigCheckItemNotSupport = New(codeConfigCheckItemNotSupport, ClassConfig, ScopeInternal, LevelMedium, "checking item %s is not supported\n%s", "Please check `ignore-checking-items` config in task configuration file, which can be set including `all`/`dump_privilege`/`replication_privilege`/`version`/`binlog_enable`/`binlog_format`/`binlog_row_image`/`table_schema`/`schema_of_shard_tables`/`auto_increment_ID`.") ErrConfigTomlTransform = New(codeConfigTomlTransform, ClassConfig, ScopeInternal, LevelMedium, "%s", "Please check the configuration file has correct TOML format.") @@ -884,6 +886,10 @@ var ( ErrSyncerUnitDDLChanDone = New(codeSyncerUnitDDLChanDone, ClassSyncUnit, ScopeInternal, LevelHigh, "canceled from external", "") ErrSyncerUnitDDLChanCanceled = New(codeSyncerUnitDDLChanCanceled, ClassSyncUnit, ScopeInternal, LevelHigh, "canceled by Close or Renew", "") ErrSyncerUnitDDLOnMultipleTable = New(codeSyncerUnitDDLOnMultipleTable, ClassSyncUnit, ScopeInternal, LevelHigh, "ddl on multiple table: %s not supported", "It is recommended to include only one DDL operation in a statement executed upstream. Please manually handle it using dmctl (skipping the DDL statement or replacing the DDL statement with a specified DDL statement). For details, see https://docs.pingcap.com/tidb-data-migration/stable/skip-or-replace-abnormal-sql-statements") + ErrSyncerUnitInjectDDLOnly = New(codeSyncerUnitInjectDDLOnly, ClassSyncUnit, ScopeInternal, LevelLow, "only support inject DDL for sharding group to be synced currently, but got %s", "") + ErrSyncerUnitInjectDDLWithoutSchema = New(codeSyncerUnitInjectDDLWithoutSchema, ClassSyncUnit, ScopeInternal, LevelLow, "injected DDL %s without schema name not valid", "") + ErrSyncerUnitNotSupportedOperate = New(codeSyncerUnitNotSupportedOperate, ClassSyncUnit, ScopeInternal, LevelMedium, "op %s not supported", "") + ErrSyncerUnitNilOperatorReq = New(codeSyncerUnitNilOperatorReq, ClassSyncUnit, ScopeInternal, LevelMedium, "nil request not valid", "") ErrSyncerUnitDMLColumnNotMatch = New(codeSyncerUnitDMLColumnNotMatch, ClassSyncUnit, ScopeInternal, LevelHigh, "Column count doesn't match value count: %d (columns) vs %d (values)", "") ErrSyncerUnitDMLOldNewValueMismatch = New(codeSyncerUnitDMLOldNewValueMismatch, ClassSyncUnit, ScopeInternal, LevelHigh, "Old value count doesn't match new value count: %d (old) vs %d (new)", "") ErrSyncerUnitDMLPruneColumnMismatch = New(codeSyncerUnitDMLPruneColumnMismatch, ClassSyncUnit, ScopeInternal, LevelHigh, "prune DML columns and data mismatch in length: %d (columns) %d (data)", "") @@ -923,6 +929,9 @@ var ( ErrSyncerFailpoint = New(codeSyncerFailpoint, ClassSyncUnit, ScopeInternal, LevelLow, "failpoint specified error", "") // DM-master error + ErrMasterSQLOpNilRequest = New(codeMasterSQLOpNilRequest, ClassDMMaster, ScopeInternal, LevelMedium, "nil request not valid", "") + ErrMasterSQLOpNotSupport = New(codeMasterSQLOpNotSupport, ClassDMMaster, ScopeInternal, LevelMedium, "op %s not supported", "") + ErrMasterSQLOpWithoutSharding = New(codeMasterSQLOpWithoutSharding, ClassDMMaster, ScopeInternal, LevelMedium, "operate request without --sharding specified not valid", "") ErrMasterGRPCCreateConn = New(codeMasterGRPCCreateConn, ClassDMMaster, ScopeInternal, LevelHigh, "create grpc connection", "") ErrMasterGRPCSendOnCloseConn = New(codeMasterGRPCSendOnCloseConn, ClassDMMaster, ScopeInternal, LevelHigh, "send request on a closed client", "") ErrMasterGRPCClientClose = New(codeMasterGRPCClientClose, ClassDMMaster, ScopeInternal, LevelHigh, "close rpc client", "") @@ -942,6 +951,7 @@ var ( ErrMasterLockIsResolving = New(codeMasterLockIsResolving, ClassDMMaster, ScopeInternal, LevelHigh, "lock %s is resolving", "") ErrMasterWorkerCliNotFound = New(codeMasterWorkerCliNotFound, ClassDMMaster, ScopeInternal, LevelHigh, "source %s relevant worker-client not found", "") ErrMasterWorkerNotWaitLock = New(codeMasterWorkerNotWaitLock, ClassDMMaster, ScopeInternal, LevelHigh, "worker %s not waiting for DDL lock %s", "") + ErrMasterHandleSQLReqFail = New(codeMasterHandleSQLReqFail, ClassDMMaster, ScopeInternal, LevelHigh, "request DDL lock %s owner %s handle SQLs request %s fail %s", "") ErrMasterOwnerExecDDL = New(codeMasterOwnerExecDDL, ClassDMMaster, ScopeInternal, LevelHigh, "owner %s ExecuteDDL fail", "") ErrMasterPartWorkerExecDDLFail = New(codeMasterPartWorkerExecDDLFail, ClassDMMaster, ScopeInternal, LevelHigh, "DDL lock %s owner ExecuteDDL successfully, so DDL lock removed. but some dm-workers ExecuteDDL fail, you should to handle dm-worker directly", "") ErrMasterWorkerExistDDLLock = New(codeMasterWorkerExistDDLLock, ClassDMMaster, ScopeInternal, LevelHigh, "worker %s exist ddl lock", "Please unlock ddl lock first.") From f66c7349118ddc766a9d79380e9303217c894224 Mon Sep 17 00:00:00 2001 From: gmhdbjd Date: Sat, 1 Aug 2020 16:28:37 +0800 Subject: [PATCH 05/31] fix fmt --- pkg/terror/error_list.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pkg/terror/error_list.go b/pkg/terror/error_list.go index 9700701dad..7053fc1fb1 100644 --- a/pkg/terror/error_list.go +++ b/pkg/terror/error_list.go @@ -726,7 +726,7 @@ var ( // pkg/upgrade ErrUpgradeVersionEtcdFail = New(codeUpgradeVersionEtcdFail, ClassFunctional, ScopeInternal, LevelHigh, "fail to operate DM cluster version in etcd", "Please use `list-member --master` to confirm whether the DM-master cluster is healthy") - ErrVerifyHandleErrorArgs = New(codeVerifyHandleErrorArgs, ClassFunctional, ScopeInternal, LevelLow, "", "Please make sure the args are correct.") + ErrVerifyHandleErrorArgs = New(codeVerifyHandleErrorArgs, ClassFunctional, ScopeInternal, LevelLow, "", "Please make sure the args are correct.") // Config related error ErrConfigCheckItemNotSupport = New(codeConfigCheckItemNotSupport, ClassConfig, ScopeInternal, LevelMedium, "checking item %s is not supported\n%s", "Please check `ignore-checking-items` config in task configuration file, which can be set including `all`/`dump_privilege`/`replication_privilege`/`version`/`binlog_enable`/`binlog_format`/`binlog_row_image`/`table_schema`/`schema_of_shard_tables`/`auto_increment_ID`.") From 5dae4b9efed3a7a54e0ecf8a04488b3baea957e9 Mon Sep 17 00:00:00 2001 From: gmhdbjd Date: Sat, 1 Aug 2020 17:35:55 +0800 Subject: [PATCH 06/31] record error location --- syncer/syncer.go | 73 +++++++++++++++++++++++++++++++++--------------- 1 file changed, 50 insertions(+), 23 deletions(-) diff --git a/syncer/syncer.go b/syncer/syncer.go index fbc6219013..77c2ed82a6 100644 --- a/syncer/syncer.go +++ b/syncer/syncer.go @@ -176,6 +176,11 @@ type Syncer struct { currentLocation binlog.Location // use to calc remain binlog size } + errLocation struct { + sync.RWMutex + location *binlog.Location // record the error location + } + addJobFunc func(*job) error } @@ -428,6 +433,7 @@ func (s *Syncer) reset() { s.execError.Set(nil) s.resetExecErrors() + s.setErrLocation(nil) switch s.cfg.ShardMode { case config.ShardPessimistic: @@ -942,6 +948,7 @@ func (s *Syncer) syncDDL(tctx *tcontext.Context, queueBucket string, db *DBConn, if err != nil { s.execError.Set(err) if !utils.IsContextCanceledError(err) { + err = s.handleEventError(err, &sqlJob.currentLocation) s.runFatalChan <- unit.NewProcessError(err) } continue @@ -971,17 +978,18 @@ func (s *Syncer) sync(tctx *tcontext.Context, queueBucket string, db *DBConn, jo } } - fatalF := func(err error) { + fatalF := func(err error, affected int) { s.execError.Set(err) if !utils.IsContextCanceledError(err) { + err = s.handleEventError(err, &jobs[affected].currentLocation) s.runFatalChan <- unit.NewProcessError(err) } clearF() } - executeSQLs := func() error { + executeSQLs := func() (error, int) { if len(jobs) == 0 { - return nil + return nil, 0 } queries := make([]string, 0, len(jobs)) args := make([][]interface{}, 0, len(jobs)) @@ -998,10 +1006,11 @@ func (s *Syncer) sync(tctx *tcontext.Context, queueBucket string, db *DBConn, jo errCtx := &ExecErrorContext{err, jobs[affected].currentLocation.Clone(), fmt.Sprintf("%v", jobs)} s.appendExecErrors(errCtx) } - return err + return err, affected } var err error + var affect int for { select { case sqlJob, ok := <-jobChan: @@ -1017,9 +1026,9 @@ func (s *Syncer) sync(tctx *tcontext.Context, queueBucket string, db *DBConn, jo } if idx >= count || sqlJob.tp == flush { - err = executeSQLs() + err, affect = executeSQLs() if err != nil { - fatalF(err) + fatalF(err, affect) continue } clearF() @@ -1027,9 +1036,9 @@ func (s *Syncer) sync(tctx *tcontext.Context, queueBucket string, db *DBConn, jo case <-time.After(waitTime): if len(jobs) > 0 { - err = executeSQLs() + err, affect = executeSQLs() if err != nil { - fatalF(err) + fatalF(err, affect) continue } clearF() @@ -1321,19 +1330,10 @@ func (s *Syncer) Run(ctx context.Context) (err error) { switch ev := e.Event.(type) { case *replication.RotateEvent: err = s.handleRotateEvent(ev, ec) - if err != nil { - return terror.Annotatef(err, "current location %s", currentLocation) - } case *replication.RowsEvent: err = s.handleRowsEvent(ev, ec) - if err != nil { - return terror.Annotatef(err, "current location %s", currentLocation) - } case *replication.QueryEvent: err = s.handleQueryEvent(ev, ec) - if err != nil { - return terror.Annotatef(err, "current location %s", currentLocation) - } case *replication.XIDEvent: if shardingReSync != nil { shardingReSync.currLocation.Position.Pos = e.Header.LogPos @@ -1360,9 +1360,6 @@ func (s *Syncer) Run(ctx context.Context) (err error) { job := newXIDJob(currentLocation, currentLocation, traceID) err = s.addJobFunc(job) - if err != nil { - return terror.Annotatef(err, "current location %s", currentLocation) - } case *replication.GenericEvent: switch e.Header.EventType { case replication.HEARTBEAT_EVENT: @@ -1370,12 +1367,12 @@ func (s *Syncer) Run(ctx context.Context) (err error) { if s.checkpoint.CheckGlobalPoint() { s.tctx.L().Info("meet heartbeat event and then flush jobs") err = s.flushJobs() - if err != nil { - return err - } } } } + if err = s.handleEventError(err, ¤tLocation); err != nil { + return err + } } } @@ -2590,3 +2587,33 @@ func (s *Syncer) ShardDDLInfo() *pessimism.Info { func (s *Syncer) ShardDDLOperation() *pessimism.Operation { return s.pessimist.PendingOperation() } + +func (s *Syncer) setErrLocation(location *binlog.Location) { + s.errLocation.Lock() + defer s.errLocation.Unlock() + + if s.errLocation.location == nil || location == nil { + s.errLocation.location = location + return + } + + // only record the first error location + if binlog.CompareLocation(*location, *s.errLocation.location, s.cfg.EnableGTID) < 0 { + s.errLocation.location = location + } +} + +func (s *Syncer) getErrLocation() *binlog.Location { + s.errLocation.Lock() + defer s.errLocation.Unlock() + return s.errLocation.location +} + +func (s *Syncer) handleEventError(err error, location *binlog.Location) error { + if err == nil { + return nil + } + + s.setErrLocation(location) + return terror.Annotatef(err, "error location %s", location) +} From 540020d46ff74bbce2fd4388202c174262efc33f Mon Sep 17 00:00:00 2001 From: gmhdbjd Date: Mon, 3 Aug 2020 01:54:30 +0800 Subject: [PATCH 07/31] fix test --- syncer/syncer.go | 18 +++++++++++++++--- 1 file changed, 15 insertions(+), 3 deletions(-) diff --git a/syncer/syncer.go b/syncer/syncer.go index 77c2ed82a6..26a880c13e 100644 --- a/syncer/syncer.go +++ b/syncer/syncer.go @@ -1330,10 +1330,19 @@ func (s *Syncer) Run(ctx context.Context) (err error) { switch ev := e.Event.(type) { case *replication.RotateEvent: err = s.handleRotateEvent(ev, ec) + if err = s.handleEventError(err, ¤tLocation); err != nil { + return err + } case *replication.RowsEvent: err = s.handleRowsEvent(ev, ec) + if err = s.handleEventError(err, ¤tLocation); err != nil { + return err + } case *replication.QueryEvent: err = s.handleQueryEvent(ev, ec) + if err = s.handleEventError(err, ¤tLocation); err != nil { + return err + } case *replication.XIDEvent: if shardingReSync != nil { shardingReSync.currLocation.Position.Pos = e.Header.LogPos @@ -1360,6 +1369,9 @@ func (s *Syncer) Run(ctx context.Context) (err error) { job := newXIDJob(currentLocation, currentLocation, traceID) err = s.addJobFunc(job) + if err = s.handleEventError(err, ¤tLocation); err != nil { + return err + } case *replication.GenericEvent: switch e.Header.EventType { case replication.HEARTBEAT_EVENT: @@ -1367,12 +1379,12 @@ func (s *Syncer) Run(ctx context.Context) (err error) { if s.checkpoint.CheckGlobalPoint() { s.tctx.L().Info("meet heartbeat event and then flush jobs") err = s.flushJobs() + if err = s.handleEventError(err, ¤tLocation); err != nil { + return err + } } } } - if err = s.handleEventError(err, ¤tLocation); err != nil { - return err - } } } From cdf10bf9b1ee3e1f1a1e2fa3a20f6d54a0e37e25 Mon Sep 17 00:00:00 2001 From: gmhdbjd Date: Mon, 3 Aug 2020 17:29:23 +0800 Subject: [PATCH 08/31] support handle-error --- _utils/terror_gen/errors_release.txt | 1 + errors.toml | 6 + pkg/binlog/position.go | 29 ++++- pkg/binlog/position_test.go | 4 +- pkg/terror/error_list.go | 2 + syncer/err-operator/operator.go | 162 +++++++++++++++++++++++++++ syncer/handle_error.go | 79 +++++++++++++ syncer/syncer.go | 92 ++++++++++----- 8 files changed, 344 insertions(+), 31 deletions(-) create mode 100644 syncer/err-operator/operator.go diff --git a/_utils/terror_gen/errors_release.txt b/_utils/terror_gen/errors_release.txt index 2de12ec5a6..46b98cc0bf 100644 --- a/_utils/terror_gen/errors_release.txt +++ b/_utils/terror_gen/errors_release.txt @@ -300,6 +300,7 @@ ErrSyncerUnitGenBAList,[code=36060:class=sync-unit:scope=internal:level=high], " ErrSyncerUnitHandleDDLFailed,[code=36061:class=sync-unit:scope=internal:level=high], "Message: fail to handle ddl job for %s" ErrSyncerShardDDLConflict,[code=36062:class=sync-unit:scope=internal:level=high], "Message: fail to handle shard ddl %v in optimistic mode, because schema conflict detected, Workaround: Please use show-ddl-locks command for more details." ErrSyncerFailpoint,[code=36063:class=sync-unit:scope=internal:level=low], "Message: failpoint specified error" +ErrSyncerReplaceEvent,[code=36064:class=sync-unit:scope=internal:level=high] ErrMasterSQLOpNilRequest,[code=38001:class=dm-master:scope=internal:level=medium], "Message: nil request not valid" ErrMasterSQLOpNotSupport,[code=38002:class=dm-master:scope=internal:level=medium], "Message: op %s not supported" ErrMasterSQLOpWithoutSharding,[code=38003:class=dm-master:scope=internal:level=medium], "Message: operate request without --sharding specified not valid" diff --git a/errors.toml b/errors.toml index 58d7a9e5cc..7e7c3f0c2b 100644 --- a/errors.toml +++ b/errors.toml @@ -1810,6 +1810,12 @@ description = "" workaround = "" tags = ["internal", "low"] +[error.DM-sync-unit-36064] +message = "" +description = "" +workaround = "" +tags = ["internal", "high"] + [error.DM-dm-master-38001] message = "nil request not valid" description = "" diff --git a/pkg/binlog/position.go b/pkg/binlog/position.go index c76d47f594..b926694572 100644 --- a/pkg/binlog/position.go +++ b/pkg/binlog/position.go @@ -175,6 +175,8 @@ type Location struct { Position gmysql.Position GTIDSet gtid.Set + + Suffix int // use for replace event } // NewLocation returns a new Location @@ -186,7 +188,10 @@ func NewLocation(flavor string) Location { } func (l Location) String() string { - return fmt.Sprintf("position: %v, gtid-set: %s", l.Position, l.GTIDSetStr()) + if l.Suffix == 0 { + return fmt.Sprintf("position: %v, gtid-set: %s", l.Position, l.GTIDSetStr()) + } + return fmt.Sprintf("position: %v, gtid-set: %s, suffix: %d", l.Position, l.GTIDSetStr(), l.Suffix) } // GTIDSetStr returns gtid set's string @@ -219,6 +224,7 @@ func (l Location) CloneWithFlavor(flavor string) Location { Pos: l.Position.Pos, }, GTIDSet: newGTIDSet, + Suffix: l.Suffix, } } @@ -230,14 +236,21 @@ func CompareLocation(location1, location2 Location, cmpGTID bool) int { if cmpGTID { cmp, canCmp := CompareGTID(location1.GTIDSet, location2.GTIDSet) if canCmp { - return cmp + if cmp != 0 { + return cmp + } + return compareIndex(location1.Suffix, location2.Suffix) } // if can't compare by GTIDSet, then compare by position log.L().Warn("gtidSet can't be compared, will compare by position", zap.Stringer("location1", location1), zap.Stringer("location2", location2)) } - return ComparePosition(location1.Position, location2.Position) + cmp := ComparePosition(location1.Position, location2.Position) + if cmp != 0 { + return cmp + } + return compareIndex(location1.Suffix, location2.Suffix) } // CompareGTID returns: @@ -274,3 +287,13 @@ func CompareGTID(gSet1, gSet2 gtid.Set) (int, bool) { return 0, false } + +func compareIndex(lhs, rhs int) int { + if lhs < rhs { + return -1 + } else if lhs > rhs { + return 1 + } else { + return 0 + } +} diff --git a/pkg/binlog/position_test.go b/pkg/binlog/position_test.go index ad31ae5d69..e15efbd3c8 100644 --- a/pkg/binlog/position_test.go +++ b/pkg/binlog/position_test.go @@ -580,10 +580,10 @@ func (t *testPositionSuite) TestCompareCompareLocation(c *C) { gset2, err := gtid.ParserGTID(cs.flavor, cs.gset2) c.Assert(err, IsNil) - cmpGTID := CompareLocation(Location{cs.pos1, gset1}, Location{cs.pos2, gset2}, true) + cmpGTID := CompareLocation(Location{cs.pos1, gset1, 0}, Location{cs.pos2, gset2, 0}, true) c.Assert(cmpGTID, Equals, cs.cmpGTID) - cmpPos := CompareLocation(Location{cs.pos1, gset1}, Location{cs.pos2, gset2}, false) + cmpPos := CompareLocation(Location{cs.pos1, gset1, 0}, Location{cs.pos2, gset2, 0}, false) c.Assert(cmpPos, Equals, cs.cmpPos) } diff --git a/pkg/terror/error_list.go b/pkg/terror/error_list.go index 7053fc1fb1..604352c941 100644 --- a/pkg/terror/error_list.go +++ b/pkg/terror/error_list.go @@ -378,6 +378,7 @@ const ( codeSyncerUnitHandleDDLFailed codeSyncerShardDDLConflict codeSyncerFailpoint + codeSyncerReplaceEvent ) // DM-master error code @@ -927,6 +928,7 @@ var ( ErrSyncerUnitHandleDDLFailed = New(codeSyncerUnitHandleDDLFailed, ClassSyncUnit, ScopeInternal, LevelHigh, "fail to handle ddl job for %s", "") ErrSyncerShardDDLConflict = New(codeSyncerShardDDLConflict, ClassSyncUnit, ScopeInternal, LevelHigh, "fail to handle shard ddl %v in optimistic mode, because schema conflict detected", "Please use show-ddl-locks command for more details.") ErrSyncerFailpoint = New(codeSyncerFailpoint, ClassSyncUnit, ScopeInternal, LevelLow, "failpoint specified error", "") + ErrSyncerReplaceEvent = New(codeSyncerReplaceEvent, ClassSyncUnit, ScopeInternal, LevelHigh, "", "") // DM-master error ErrMasterSQLOpNilRequest = New(codeMasterSQLOpNilRequest, ClassDMMaster, ScopeInternal, LevelMedium, "nil request not valid", "") diff --git a/syncer/err-operator/operator.go b/syncer/err-operator/operator.go new file mode 100644 index 0000000000..d2e0b65ab6 --- /dev/null +++ b/syncer/err-operator/operator.go @@ -0,0 +1,162 @@ +// Copyright 2020 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package operator + +import ( + "bytes" + "fmt" + "strings" + "sync" + + uuid "github.com/satori/go.uuid" + "github.com/siddontang/go-mysql/replication" + "go.uber.org/zap" + + "github.com/pingcap/dm/dm/pb" + "github.com/pingcap/dm/pkg/binlog" + "github.com/pingcap/dm/pkg/log" + "github.com/pingcap/dm/pkg/terror" +) + +// Operator contains an operation for specified binlog pos +// used by `handle-error` +type Operator struct { + uuid string // add a UUID, make it more friendly to be traced in log + op pb.ErrorOp + events []*replication.BinlogEvent // endLocation -> events +} + +// newOperator creates a new operator with a random UUID +func newOperator(op pb.ErrorOp, events []*replication.BinlogEvent) *Operator { + return &Operator{ + uuid: uuid.NewV4().String(), + op: op, + events: events, + } +} + +func (o *Operator) String() string { + events := make([]string, 0) + for _, e := range o.events { + buf := new(bytes.Buffer) + e.Dump(buf) + events = append(events, buf.String()) + } + return fmt.Sprintf("uuid: %s, op: %s, events: %s", o.uuid, o.op, strings.Join(events, " ")) +} + +// Holder holds error operator +type Holder struct { + mu sync.Mutex + operators map[string]*Operator + replaceEvents map[string][]*replication.BinlogEvent // startLocation -> events +} + +// NewHolder creates a new Holder +func NewHolder() *Holder { + return &Holder{ + operators: make(map[string]*Operator), + replaceEvents: make(map[string][]*replication.BinlogEvent), + } +} + +// Set sets an Operator +func (h *Holder) Set(pos string, op pb.ErrorOp, events []*replication.BinlogEvent) error { + h.mu.Lock() + defer h.mu.Unlock() + + oper := newOperator(op, events) + pre, ok := h.operators[pos] + if ok { + log.L().Warn("overwrite operator", zap.String("position", pos), zap.Stringer("old operator", pre)) + } + h.operators[pos] = oper + log.L().Info("set a new operator", zap.String("position", pos), zap.Stringer("new operator", oper)) + return nil +} + +// GetEvent return a replace binlog event +// for example: +// startLocation endLocation +// event 1 1000, 0 1010, 0 +// event 2 1010, 0 1020, 0 <-- replace it with event a,b,c +// replace event a 1010, 0 1010, 1 +// replace event b 1010, 1 1010, 2 +// replace event c 1010, 2 1020, 0 +// event 3 1020, 0 1030, 0 +func (h *Holder) GetEvent(startLocation *binlog.Location) (*replication.BinlogEvent, error) { + h.mu.Lock() + defer h.mu.Unlock() + + key := startLocation.Position.String() + events, ok := h.replaceEvents[key] + if !ok { + return nil, nil + } + + if len(events) <= startLocation.Suffix { + return nil, terror.ErrSyncerReplaceEvent.New("replace events out of index") + } + + e := events[startLocation.Suffix] + buf := new(bytes.Buffer) + e.Dump(buf) + log.L().Info("get replace event", zap.Stringer("event", buf)) + + return e, nil +} + +// Apply tries to apply operation for event by location +// We use endLocation to set operator for user +// Use startLocation to get replace event +// When meet endLocation first time, copy events to replaceEvents +// Ugly code, but have no better idea now. +func (h *Holder) Apply(startLocation *binlog.Location, endLocation *binlog.Location) (bool, pb.ErrorOp) { + h.mu.Lock() + defer h.mu.Unlock() + + // only apply the origin event + if endLocation.Suffix != 0 { + return false, pb.ErrorOp_InvalidErrorOp + } + + key := endLocation.Position.String() + operator, ok := h.operators[key] + if !ok { + return false, pb.ErrorOp_InvalidErrorOp + } + + if operator.op == pb.ErrorOp_Replace { + if len(operator.events) == 0 { + // this should not happen + return false, pb.ErrorOp_InvalidErrorOp + } + + // set LogPos as start position + for _, ev := range operator.events { + ev.Header.LogPos = startLocation.Position.Pos + } + + // set the last replace event as end position + operator.events[len(operator.events)-1].Header.EventSize = endLocation.Position.Pos - startLocation.Position.Pos + operator.events[len(operator.events)-1].Header.LogPos = endLocation.Position.Pos + + // copy events to replaceEvents + h.replaceEvents[startLocation.Position.String()] = operator.events + } + + log.L().Info("apply a operator", zap.Stringer("operator", operator)) + + return true, operator.op +} diff --git a/syncer/handle_error.go b/syncer/handle_error.go index 298143345d..09fbafd434 100644 --- a/syncer/handle_error.go +++ b/syncer/handle_error.go @@ -15,11 +15,90 @@ package syncer import ( "context" + "fmt" "github.com/pingcap/dm/dm/pb" + parserpkg "github.com/pingcap/dm/pkg/parser" + "github.com/pingcap/dm/pkg/terror" + "github.com/pingcap/parser" + "github.com/pingcap/parser/ast" + tmysql "github.com/pingcap/parser/mysql" + + "github.com/siddontang/go-mysql/replication" ) // HandleError handle error for syncer func (s *Syncer) HandleError(ctx context.Context, req *pb.HandleWorkerErrorRequest) (string, error) { + pos := req.BinlogPos + + if len(pos) == 0 { + location := s.getErrLocation() + if location == nil { + return fmt.Sprintf("source '%s' has no error", s.cfg.SourceID), nil + } + pos = location.Position.String() + } + + events := make([]*replication.BinlogEvent, 0) + var err error + if req.Op == pb.ErrorOp_Replace { + events, err = s.genEvents(req.Sqls) + if err != nil { + return "", err + } + } + + s.errOperatorHolder.Set(pos, req.Op, events) + return "", nil } + +func (s *Syncer) genEvents(sqls []string) ([]*replication.BinlogEvent, error) { + events := make([]*replication.BinlogEvent, 0) + + parser2 := parser.New() + if s.cfg.EnableANSIQuotes { + parser2.SetSQLMode(tmysql.ModeANSIQuotes) + } + + for _, sql := range sqls { + node, err := parser2.ParseOneStmt(sql, "", "") + if err != nil { + return nil, terror.Annotatef(terror.ErrSyncerUnitParseStmt.New(err.Error()), "sql %s", sql) + } + + switch node.(type) { + case ast.DDLNode: + tableNames, err := parserpkg.FetchDDLTableNames("", node) + if err != nil { + return nil, err + } + + schema := tableNames[0].Schema + if len(schema) == 0 { + return nil, terror.ErrSyncerUnitInjectDDLWithoutSchema.Generate(sql) + } + events = append(events, genQueryEvent([]byte(schema), []byte(sql))) + default: + // TODO: support DML + return nil, terror.ErrSyncerReplaceEvent.New("only support replace with DDL currently") + } + } + return events, nil +} + +// genQueryEvent generate QueryEvent with empty EventSize and LogPos +func genQueryEvent(schema, query []byte) *replication.BinlogEvent { + header := &replication.EventHeader{ + EventType: replication.QUERY_EVENT, + } + queryEvent := &replication.QueryEvent{ + Schema: schema, + Query: query, + } + e := &replication.BinlogEvent{ + Header: header, + Event: queryEvent, + } + return e +} diff --git a/syncer/syncer.go b/syncer/syncer.go index 26a880c13e..70a0e6c16d 100644 --- a/syncer/syncer.go +++ b/syncer/syncer.go @@ -57,6 +57,7 @@ import ( "github.com/pingcap/dm/pkg/terror" "github.com/pingcap/dm/pkg/tracing" "github.com/pingcap/dm/pkg/utils" + operator "github.com/pingcap/dm/syncer/err-operator" sm "github.com/pingcap/dm/syncer/safe-mode" "github.com/pingcap/dm/syncer/shardddl" ) @@ -168,6 +169,10 @@ type Syncer struct { readerHub *streamer.ReaderHub + errOperatorHolder *operator.Holder + + firstReplace bool // true if we replace first event by handle-error + // TODO: re-implement tracer flow for binlog event later. tracer *tracing.Tracer @@ -212,6 +217,7 @@ func NewSyncer(cfg *config.SubTaskConfig, etcdClient *clientv3.Client) *Syncer { syncer.setSyncCfg() syncer.binlogType = toBinlogType(cfg.UseRelay) + syncer.errOperatorHolder = operator.NewHolder() syncer.readerHub = streamer.GetReaderHub() if cfg.ShardMode == config.ShardPessimistic { @@ -434,6 +440,7 @@ func (s *Syncer) reset() { s.execError.Set(nil) s.resetExecErrors() s.setErrLocation(nil) + s.firstReplace = false switch s.cfg.ShardMode { case config.ShardPessimistic: @@ -1244,7 +1251,7 @@ func (s *Syncer) Run(ctx context.Context) (err error) { } startTime := time.Now() - e, err = s.streamerController.GetEvent(tctx) + e, err = s.getEvent(tctx, ¤tLocation) if err == context.Canceled { s.tctx.L().Info("binlog replication main routine quit(context canceled)!", zap.Stringer("last location", lastLocation)) @@ -1310,9 +1317,51 @@ func (s *Syncer) Run(ctx context.Context) (err error) { failpoint.Inject("ProcessBinlogSlowDown", nil) s.tctx.L().Debug("receive binlog event", zap.Reflect("header", e.Header)) + + // TODO: support all event + // we calculate startLocation and endLocation(currentLocation) here for Rows/Query event here + var startLocation *binlog.Location + switch e.Event.(type) { + case *replication.RowsEvent, *replication.QueryEvent: + location := currentLocation.Clone() + startLocation = &location + endSuffix := startLocation.Suffix + if s.firstReplace { + endSuffix = 1 + s.firstReplace = false + } else if endSuffix > 0 { + endSuffix++ + } + + currentLocation = binlog.Location{ + Position: mysql.Position{ + Name: lastLocation.Position.Name, + Pos: e.Header.LogPos, + }, + GTIDSet: lastLocation.GTIDSet.Clone(), + Suffix: endSuffix, + } + apply, op := s.errOperatorHolder.Apply(startLocation, ¤tLocation) + if apply { + if op == pb.ErrorOp_Replace { + s.firstReplace = true + // revert currentLocation to startLocation + currentLocation = startLocation.Clone() + } + // skip the event + continue + } + // set endLocation.Suffix=0 of last replace event + if currentLocation.Suffix > 0 && e.Header.EventSize > 0 { + currentLocation.Suffix = 0 + } + default: + } + ec := eventContext{ tctx: tctx, header: e.Header, + startLocation: startLocation, currentLocation: ¤tLocation, lastLocation: &lastLocation, shardingReSync: shardingReSync, @@ -1346,6 +1395,7 @@ func (s *Syncer) Run(ctx context.Context) (err error) { case *replication.XIDEvent: if shardingReSync != nil { shardingReSync.currLocation.Position.Pos = e.Header.LogPos + shardingReSync.currLocation.Suffix = currentLocation.Suffix shardingReSync.currLocation.GTIDSet.Set(ev.GSet) // only need compare binlog position? @@ -1391,6 +1441,7 @@ func (s *Syncer) Run(ctx context.Context) (err error) { type eventContext struct { tctx *tcontext.Context header *replication.EventHeader + startLocation *binlog.Location currentLocation *binlog.Location lastLocation *binlog.Location shardingReSync *ShardingReSync @@ -1450,16 +1501,9 @@ func (s *Syncer) handleRotateEvent(ev *replication.RotateEvent, ec eventContext) func (s *Syncer) handleRowsEvent(ev *replication.RowsEvent, ec eventContext) error { originSchema, originTable := string(ev.Table.Schema), string(ev.Table.Table) schemaName, tableName := s.renameShardingSchema(originSchema, originTable) - *ec.currentLocation = binlog.Location{ - Position: mysql.Position{ - Name: ec.lastLocation.Position.Name, - Pos: ec.header.LogPos, - }, - GTIDSet: ec.lastLocation.GTIDSet.Clone(), - } if ec.shardingReSync != nil { - ec.shardingReSync.currLocation.Position.Pos = ec.header.LogPos + ec.shardingReSync.currLocation = ec.currentLocation.Clone() if binlog.CompareLocation(ec.shardingReSync.currLocation, ec.shardingReSync.latestLocation, s.cfg.EnableGTID) >= 0 { s.tctx.L().Info("re-replicate shard group was completed", zap.String("event", "row"), zap.Stringer("re-shard", ec.shardingReSync)) return ec.closeShardingResync() @@ -1590,13 +1634,6 @@ func (s *Syncer) handleRowsEvent(ev *replication.RowsEvent, ec eventContext) err } func (s *Syncer) handleQueryEvent(ev *replication.QueryEvent, ec eventContext) error { - *ec.currentLocation = binlog.Location{ - Position: mysql.Position{ - Name: ec.lastLocation.Position.Name, - Pos: ec.header.LogPos, - }, - GTIDSet: ec.lastLocation.GTIDSet.Clone(), - } ec.currentLocation.GTIDSet.Set(ev.GSet) sql := strings.TrimSpace(string(ev.Query)) @@ -1619,8 +1656,7 @@ func (s *Syncer) handleQueryEvent(ev *replication.QueryEvent, ec eventContext) e } if ec.shardingReSync != nil { - ec.shardingReSync.currLocation.Position.Pos = ec.header.LogPos - ec.shardingReSync.currLocation.GTIDSet.Set(ev.GSet) + ec.shardingReSync.currLocation = ec.currentLocation.Clone() if binlog.CompareLocation(ec.shardingReSync.currLocation, ec.shardingReSync.latestLocation, s.cfg.EnableGTID) >= 0 { s.tctx.L().Info("re-replicate shard group was completed", zap.String("event", "query"), zap.String("statement", sql), zap.Stringer("re-shard", ec.shardingReSync)) err2 := ec.closeShardingResync() @@ -1638,7 +1674,6 @@ func (s *Syncer) handleQueryEvent(ev *replication.QueryEvent, ec eventContext) e } s.tctx.L().Info("", zap.String("event", "query"), zap.String("statement", sql), zap.String("schema", usedSchema), zap.Stringer("last location", ec.lastLocation), log.WrapStringerField("location", ec.currentLocation)) - lastGTIDSet := ec.lastLocation.GTIDSet.Clone() *ec.lastLocation = ec.currentLocation.Clone() // update lastLocation, because we have checked `isDDL` *ec.latestOp = ddl @@ -1829,13 +1864,7 @@ func (s *Syncer) handleQueryEvent(ev *replication.QueryEvent, ec eventContext) e ) // for sharding DDL, the firstPos should be the `Pos` of the binlog, not the `End_log_pos` // so when restarting before sharding DDLs synced, this binlog can be re-sync again to trigger the TrySync - startLocation := binlog.Location{ - Position: mysql.Position{ - Name: ec.currentLocation.Position.Name, - Pos: ec.currentLocation.Position.Pos - ec.header.EventSize, - }, - GTIDSet: lastGTIDSet, - } + startLocation := ec.startLocation.Clone() source, _ = GenTableID(ddlInfo.tableNames[0][0].Schema, ddlInfo.tableNames[0][0].Name) @@ -2629,3 +2658,14 @@ func (s *Syncer) handleEventError(err error, location *binlog.Location) error { s.setErrLocation(location) return terror.Annotatef(err, "error location %s", location) } + +// getEvent gets an event from streamerController or errOperatorHolder +func (s *Syncer) getEvent(tctx *tcontext.Context, startLocation *binlog.Location) (*replication.BinlogEvent, error) { + // next event is a replace event + if s.firstReplace || startLocation.Suffix > 0 { + log.L().Info("try to get replace event, firstReplace: %s", zap.Stringer("location", startLocation)) + return s.errOperatorHolder.GetEvent(startLocation) + } + + return s.streamerController.GetEvent(tctx) +} From 3e04891bbbb1f7ed70cebb137188842a3d36f26e Mon Sep 17 00:00:00 2001 From: gmhdbjd Date: Tue, 4 Aug 2020 17:30:33 +0800 Subject: [PATCH 09/31] address comment --- dm/master/server.go | 12 ++--- pkg/binlog/position_test.go | 102 +++++++++++++++++++++++++++++++++++- syncer/syncer.go | 2 +- 3 files changed, 106 insertions(+), 10 deletions(-) diff --git a/dm/master/server.go b/dm/master/server.go index 1d83f9abe0..bd86613d07 100644 --- a/dm/master/server.go +++ b/dm/master/server.go @@ -2143,17 +2143,15 @@ func (s *Server) HandleError(ctx context.Context, req *pb.HandleErrorRequest) (* } wg.Wait() - workerRespMap := make(map[string]*pb.CommonWorkerResponse, len(sources)) + workerResps := make([]*pb.CommonWorkerResponse, 0, len(sources)) for len(workerRespCh) > 0 { workerResp := <-workerRespCh - workerRespMap[workerResp.Source] = workerResp + workerResps = append(workerResps, workerResp) } - sort.Strings(sources) - workerResps := make([]*pb.CommonWorkerResponse, 0, len(sources)) - for _, worker := range sources { - workerResps = append(workerResps, workerRespMap[worker]) - } + sort.Slice(workerResps, func(i, j int) bool { + return workerResps[i].Source < workerResps[j].Source + }) return &pb.HandleErrorResponse{ Result: true, diff --git a/pkg/binlog/position_test.go b/pkg/binlog/position_test.go index e15efbd3c8..9adfd0df7f 100644 --- a/pkg/binlog/position_test.go +++ b/pkg/binlog/position_test.go @@ -355,8 +355,10 @@ func (t *testPositionSuite) TestCompareCompareLocation(c *C) { flavor string pos1 gmysql.Position gset1 string + suffix1 int pos2 gmysql.Position gset2 string + suffix2 int cmpGTID int cmpPos int }{ @@ -368,6 +370,7 @@ func (t *testPositionSuite) TestCompareCompareLocation(c *C) { Pos: 123, }, "", + 0, gmysql.Position{ Name: "binlog.00001", Pos: 123, @@ -375,6 +378,7 @@ func (t *testPositionSuite) TestCompareCompareLocation(c *C) { "", 0, 0, + 0, }, { // both gset1 and gset2 is nil, gset1 = gset2, pos1 = pos2 gmysql.MariaDBFlavor, @@ -383,6 +387,7 @@ func (t *testPositionSuite) TestCompareCompareLocation(c *C) { Pos: 123, }, "", + 0, gmysql.Position{ Name: "binlog.00001", Pos: 123, @@ -390,6 +395,7 @@ func (t *testPositionSuite) TestCompareCompareLocation(c *C) { "", 0, 0, + 0, }, { // both gset1 and gset2 is nil, gset1 = gset2, pos1 < pos2 gmysql.MariaDBFlavor, @@ -398,12 +404,14 @@ func (t *testPositionSuite) TestCompareCompareLocation(c *C) { Pos: 123, }, "", + 0, gmysql.Position{ Name: "binlog.00002", Pos: 122, }, "", 0, + 0, -1, }, { // pos1 > pos2, gset is nil @@ -413,12 +421,14 @@ func (t *testPositionSuite) TestCompareCompareLocation(c *C) { Pos: 123, }, "", + 0, gmysql.Position{ Name: "binlog.00002", Pos: 122, }, "", 0, + 0, 1, }, { // gset1 = gset2, pos1 < pos2 @@ -428,12 +438,14 @@ func (t *testPositionSuite) TestCompareCompareLocation(c *C) { Pos: 123, }, "53ea0ed1-9bf8-11e6-8bea-64006a897c73:1-4", + 0, gmysql.Position{ Name: "binlog.00002", Pos: 122, }, "53ea0ed1-9bf8-11e6-8bea-64006a897c73:1-4", 0, + 0, -1, }, { // gset1 < gset2, pos1 < pos2 @@ -443,11 +455,13 @@ func (t *testPositionSuite) TestCompareCompareLocation(c *C) { Pos: 123, }, "53ea0ed1-9bf8-11e6-8bea-64006a897c73:1-2,53ea0ed1-9bf8-11e6-8bea-64006a897c74:1-2", + 0, gmysql.Position{ Name: "binlog.00002", Pos: 124, }, "53ea0ed1-9bf8-11e6-8bea-64006a897c73:1-4,53ea0ed1-9bf8-11e6-8bea-64006a897c74:1-3", + 0, -1, -1, }, { @@ -458,11 +472,13 @@ func (t *testPositionSuite) TestCompareCompareLocation(c *C) { Pos: 123, }, "53ea0ed1-9bf8-11e6-8bea-64006a897c73:1-2,53ea0ed1-9bf8-11e6-8bea-64006a897c74:1-3", + 0, gmysql.Position{ Name: "binlog.00002", Pos: 124, }, "53ea0ed1-9bf8-11e6-8bea-64006a897c73:1-2", + 0, 1, -1, }, { @@ -473,11 +489,13 @@ func (t *testPositionSuite) TestCompareCompareLocation(c *C) { Pos: 123, }, "53ea0ed1-9bf8-11e6-8bea-64006a897c73:1-2,53ea0ed1-9bf8-11e6-8bea-64006a897c74:2-4", + 0, gmysql.Position{ Name: "binlog.00002", Pos: 124, }, "53ea0ed1-9bf8-11e6-8bea-64006a897c73:1-2,53ea0ed1-9bf8-11e6-8bea-64006a897c74:1-3", + 0, -1, -1, }, { @@ -488,12 +506,14 @@ func (t *testPositionSuite) TestCompareCompareLocation(c *C) { Pos: 123, }, "1-1-1,2-2-2", + 0, gmysql.Position{ Name: "binlog.00002", Pos: 122, }, "1-1-1,2-2-2", 0, + 0, -1, }, { // gset1 < gset2, pos1 < pos2 @@ -503,11 +523,13 @@ func (t *testPositionSuite) TestCompareCompareLocation(c *C) { Pos: 123, }, "1-1-1,2-2-2", + 0, gmysql.Position{ Name: "binlog.00002", Pos: 124, }, "1-1-1,2-2-2,3-3-3", + 0, -1, -1, }, { @@ -518,11 +540,13 @@ func (t *testPositionSuite) TestCompareCompareLocation(c *C) { Pos: 123, }, "1-1-1,2-2-3", + 0, gmysql.Position{ Name: "binlog.00002", Pos: 124, }, "1-1-1,2-2-2", + 0, 1, -1, }, { @@ -533,11 +557,13 @@ func (t *testPositionSuite) TestCompareCompareLocation(c *C) { Pos: 123, }, "1-1-1,2-2-2", + 0, gmysql.Position{ Name: "binlog.00002", Pos: 124, }, "2-2-2,3-3-3", + 0, -1, -1, }, { @@ -548,11 +574,13 @@ func (t *testPositionSuite) TestCompareCompareLocation(c *C) { Pos: 123, }, "", + 0, gmysql.Position{ Name: "binlog.00002", Pos: 124, }, "2-2-2,3-3-3", + 0, -1, -1, }, { @@ -563,13 +591,83 @@ func (t *testPositionSuite) TestCompareCompareLocation(c *C) { Pos: 123, }, "", + 0, gmysql.Position{ Name: "binlog.00002", Pos: 124, }, "", 0, + 0, + -1, + }, { + // both gset1 and gset2 is nil, gset1 = gset2, pos1 < pos2 + gmysql.MariaDBFlavor, + gmysql.Position{ + Name: "binlog.00001", + Pos: 123, + }, + "", + 0, + gmysql.Position{ + Name: "binlog.00002", + Pos: 124, + }, + "", + 0, + 0, + -1, + }, { + // gset1 = gset2, pos1 = pos2, suffix1 < suffix2 + gmysql.MariaDBFlavor, + gmysql.Position{ + Name: "binlog.00001", + Pos: 123, + }, + "1-1-1,2-2-2", + 0, + gmysql.Position{ + Name: "binlog.00001", + Pos: 123, + }, + "1-1-1,2-2-2", + 1, + -1, -1, + }, { + // gset1 = gset2, pos1 = pos2, suffix1 = suffix2 + gmysql.MariaDBFlavor, + gmysql.Position{ + Name: "binlog.00001", + Pos: 123, + }, + "1-1-1,2-2-2", + 1, + gmysql.Position{ + Name: "binlog.00001", + Pos: 123, + }, + "1-1-1,2-2-2", + 1, + 0, + 0, + }, { + // gset1 = gset2, pos1 = pos2, suffix1 > suffix2 + gmysql.MariaDBFlavor, + gmysql.Position{ + Name: "binlog.00001", + Pos: 123, + }, + "1-1-1,2-2-2", + 2, + gmysql.Position{ + Name: "binlog.00001", + Pos: 123, + }, + "1-1-1,2-2-2", + 1, + 1, + 1, }, } @@ -580,10 +678,10 @@ func (t *testPositionSuite) TestCompareCompareLocation(c *C) { gset2, err := gtid.ParserGTID(cs.flavor, cs.gset2) c.Assert(err, IsNil) - cmpGTID := CompareLocation(Location{cs.pos1, gset1, 0}, Location{cs.pos2, gset2, 0}, true) + cmpGTID := CompareLocation(Location{cs.pos1, gset1, cs.suffix1}, Location{cs.pos2, gset2, cs.suffix2}, true) c.Assert(cmpGTID, Equals, cs.cmpGTID) - cmpPos := CompareLocation(Location{cs.pos1, gset1, 0}, Location{cs.pos2, gset2, 0}, false) + cmpPos := CompareLocation(Location{cs.pos1, gset1, cs.suffix1}, Location{cs.pos2, gset2, cs.suffix2}, false) c.Assert(cmpPos, Equals, cs.cmpPos) } diff --git a/syncer/syncer.go b/syncer/syncer.go index 70a0e6c16d..7dd571b6b1 100644 --- a/syncer/syncer.go +++ b/syncer/syncer.go @@ -2663,7 +2663,7 @@ func (s *Syncer) handleEventError(err error, location *binlog.Location) error { func (s *Syncer) getEvent(tctx *tcontext.Context, startLocation *binlog.Location) (*replication.BinlogEvent, error) { // next event is a replace event if s.firstReplace || startLocation.Suffix > 0 { - log.L().Info("try to get replace event, firstReplace: %s", zap.Stringer("location", startLocation)) + log.L().Info(fmt.Sprintf("try to get replace event, firstReplace: %v", s.firstReplace), zap.Stringer("location", startLocation)) return s.errOperatorHolder.GetEvent(startLocation) } From 3454208ad2ce54a9883713013ce89ede879ca7bc Mon Sep 17 00:00:00 2001 From: gmhdbjd Date: Wed, 5 Aug 2020 17:13:08 +0800 Subject: [PATCH 10/31] fix bugs --- pkg/binlog/position.go | 5 +++ syncer/checkpoint.go | 3 ++ syncer/err-operator/operator.go | 34 ++++++++------- syncer/handle_error.go | 6 +-- syncer/job.go | 12 ++++-- syncer/job_test.go | 16 ++++---- syncer/optimist.go | 2 +- syncer/syncer.go | 73 +++++++++++++++++---------------- 8 files changed, 82 insertions(+), 69 deletions(-) diff --git a/pkg/binlog/position.go b/pkg/binlog/position.go index b926694572..7221f95999 100644 --- a/pkg/binlog/position.go +++ b/pkg/binlog/position.go @@ -297,3 +297,8 @@ func compareIndex(lhs, rhs int) int { return 0 } } + +// ResetSuffix set suffix to 0 +func (l *Location) ResetSuffix() { + l.Suffix = 0 +} diff --git a/syncer/checkpoint.go b/syncer/checkpoint.go index 84d0b38ac6..9d219fb5d9 100644 --- a/syncer/checkpoint.go +++ b/syncer/checkpoint.go @@ -100,6 +100,9 @@ func (b *binlogPoint) flush() { func (b *binlogPoint) rollback(schemaTracker *schema.Tracker, schema string) (isSchemaChanged bool) { b.Lock() defer b.Unlock() + + // set suffix to 0 when we meet error + b.flushedLocation.ResetSuffix() b.location = b.flushedLocation.Clone() if b.ti == nil { return // for global checkpoint, no need to rollback the schema. diff --git a/syncer/err-operator/operator.go b/syncer/err-operator/operator.go index d2e0b65ab6..084b960608 100644 --- a/syncer/err-operator/operator.go +++ b/syncer/err-operator/operator.go @@ -58,16 +58,14 @@ func (o *Operator) String() string { // Holder holds error operator type Holder struct { - mu sync.Mutex - operators map[string]*Operator - replaceEvents map[string][]*replication.BinlogEvent // startLocation -> events + mu sync.Mutex + operators map[string]*Operator } // NewHolder creates a new Holder func NewHolder() *Holder { return &Holder{ - operators: make(map[string]*Operator), - replaceEvents: make(map[string][]*replication.BinlogEvent), + operators: make(map[string]*Operator), } } @@ -100,16 +98,16 @@ func (h *Holder) GetEvent(startLocation *binlog.Location) (*replication.BinlogEv defer h.mu.Unlock() key := startLocation.Position.String() - events, ok := h.replaceEvents[key] + operator, ok := h.operators[key] if !ok { return nil, nil } - if len(events) <= startLocation.Suffix { + if len(operator.events) <= startLocation.Suffix { return nil, terror.ErrSyncerReplaceEvent.New("replace events out of index") } - e := events[startLocation.Suffix] + e := operator.events[startLocation.Suffix] buf := new(bytes.Buffer) e.Dump(buf) log.L().Info("get replace event", zap.Stringer("event", buf)) @@ -118,11 +116,7 @@ func (h *Holder) GetEvent(startLocation *binlog.Location) (*replication.BinlogEv } // Apply tries to apply operation for event by location -// We use endLocation to set operator for user -// Use startLocation to get replace event -// When meet endLocation first time, copy events to replaceEvents -// Ugly code, but have no better idea now. -func (h *Holder) Apply(startLocation *binlog.Location, endLocation *binlog.Location) (bool, pb.ErrorOp) { +func (h *Holder) Apply(startLocation, endLocation *binlog.Location) (bool, pb.ErrorOp) { h.mu.Lock() defer h.mu.Unlock() @@ -131,7 +125,7 @@ func (h *Holder) Apply(startLocation *binlog.Location, endLocation *binlog.Locat return false, pb.ErrorOp_InvalidErrorOp } - key := endLocation.Position.String() + key := startLocation.Position.String() operator, ok := h.operators[key] if !ok { return false, pb.ErrorOp_InvalidErrorOp @@ -146,17 +140,21 @@ func (h *Holder) Apply(startLocation *binlog.Location, endLocation *binlog.Locat // set LogPos as start position for _, ev := range operator.events { ev.Header.LogPos = startLocation.Position.Pos + if e, ok := ev.Event.(*replication.QueryEvent); ok { + e.GSet = startLocation.GTIDSet.Origin() + } } // set the last replace event as end position operator.events[len(operator.events)-1].Header.EventSize = endLocation.Position.Pos - startLocation.Position.Pos operator.events[len(operator.events)-1].Header.LogPos = endLocation.Position.Pos - - // copy events to replaceEvents - h.replaceEvents[startLocation.Position.String()] = operator.events + e := operator.events[len(operator.events)-1] + if e, ok := e.Event.(*replication.QueryEvent); ok { + e.GSet = endLocation.GTIDSet.Origin() + } } - log.L().Info("apply a operator", zap.Stringer("operator", operator)) + log.L().Info("apply a operator", zap.Stringer("startlocation", startLocation), zap.Stringer("endlocation", endLocation), zap.Stringer("operator", operator)) return true, operator.op } diff --git a/syncer/handle_error.go b/syncer/handle_error.go index 09fbafd434..ece383e146 100644 --- a/syncer/handle_error.go +++ b/syncer/handle_error.go @@ -32,11 +32,11 @@ func (s *Syncer) HandleError(ctx context.Context, req *pb.HandleWorkerErrorReque pos := req.BinlogPos if len(pos) == 0 { - location := s.getErrLocation() - if location == nil { + startLocation := s.getErrLocation() + if startLocation == nil { return fmt.Sprintf("source '%s' has no error", s.cfg.SourceID), nil } - pos = location.Position.String() + pos = startLocation.Position.String() } events := make([]*replication.BinlogEvent, 0) diff --git a/syncer/job.go b/syncer/job.go index 29dec4563e..5309cea280 100644 --- a/syncer/job.go +++ b/syncer/job.go @@ -69,6 +69,7 @@ type job struct { key string retry bool location binlog.Location + startLocation binlog.Location currentLocation binlog.Location // exactly binlog position of current SQL ddls []string traceID string @@ -77,10 +78,10 @@ type job struct { func (j *job) String() string { // only output some important information, maybe useful in execution. - return fmt.Sprintf("tp: %s, sql: %s, args: %v, key: %s, ddls: %s, last_location: %s, current_location: %s", j.tp, j.sql, j.args, j.key, j.ddls, j.location, j.currentLocation) + return fmt.Sprintf("tp: %s, sql: %s, args: %v, key: %s, ddls: %s, last_location: %s, start_location: %s, current_location: %s", j.tp, j.sql, j.args, j.key, j.ddls, j.location, j.startLocation, j.currentLocation) } -func newJob(tp opType, sourceSchema, sourceTable, targetSchema, targetTable, sql string, args []interface{}, key string, location, cmdLocation binlog.Location, traceID string) *job { +func newJob(tp opType, sourceSchema, sourceTable, targetSchema, targetTable, sql string, args []interface{}, key string, location, startLocation, cmdLocation binlog.Location, traceID string) *job { location1 := location.Clone() cmdLocation1 := cmdLocation.Clone() @@ -92,6 +93,7 @@ func newJob(tp opType, sourceSchema, sourceTable, targetSchema, targetTable, sql sql: sql, args: args, key: key, + startLocation: startLocation, location: location1, currentLocation: cmdLocation1, retry: true, @@ -102,7 +104,7 @@ func newJob(tp opType, sourceSchema, sourceTable, targetSchema, targetTable, sql // newDDL job is used to create a new ddl job // when cfg.ShardMode == "", ddlInfo == nil,sourceTbls != nil, we use sourceTbls to record ddl affected tables. // when cfg.ShardMode == ShardOptimistic || ShardPessimistic, ddlInfo != nil, sourceTbls == nil. -func newDDLJob(ddlInfo *shardingDDLInfo, ddls []string, location, cmdLocation binlog.Location, +func newDDLJob(ddlInfo *shardingDDLInfo, ddls []string, location, startLocation, cmdLocation binlog.Location, traceID string, sourceTbls map[string]map[string]struct{}) *job { location1 := location.Clone() cmdLocation1 := cmdLocation.Clone() @@ -111,6 +113,7 @@ func newDDLJob(ddlInfo *shardingDDLInfo, ddls []string, location, cmdLocation bi tp: ddl, ddls: ddls, location: location1, + startLocation: startLocation, currentLocation: cmdLocation1, traceID: traceID, } @@ -135,13 +138,14 @@ func newDDLJob(ddlInfo *shardingDDLInfo, ddls []string, location, cmdLocation bi return j } -func newXIDJob(location, cmdLocation binlog.Location, traceID string) *job { +func newXIDJob(location, startLocation, cmdLocation binlog.Location, traceID string) *job { location1 := location.Clone() cmdLocation1 := cmdLocation.Clone() return &job{ tp: xid, location: location1, + startLocation: startLocation, currentLocation: cmdLocation1, traceID: traceID, } diff --git a/syncer/job_test.go b/syncer/job_test.go index 55af3ff695..37c7c6d230 100644 --- a/syncer/job_test.go +++ b/syncer/job_test.go @@ -87,20 +87,20 @@ func (t *testJobSuite) TestJob(c *C) { jobStr string }{ { - newJob(insert, "test", "t1", "test", "t1", "insert into test.t1 values(?)", []interface{}{1}, "1", binlog.NewLocation(""), binlog.NewLocation(""), ""), - "tp: insert, sql: insert into test.t1 values(?), args: [1], key: 1, ddls: [], last_location: position: (, 4), gtid-set: , current_location: position: (, 4), gtid-set: ", + newJob(insert, "test", "t1", "test", "t1", "insert into test.t1 values(?)", []interface{}{1}, "1", binlog.NewLocation(""), binlog.NewLocation(""), binlog.NewLocation(""), ""), + "tp: insert, sql: insert into test.t1 values(?), args: [1], key: 1, ddls: [], last_location: position: (, 4), gtid-set: , start_location: position: (, 4), gtid-set: ,current_location: position: (, 4), gtid-set: ", }, { - newDDLJob(ddlInfo, []string{"create database test"}, binlog.NewLocation(""), binlog.NewLocation(""), "", nil), - "tp: ddl, sql: , args: [], key: , ddls: [create database test], last_location: position: (, 4), gtid-set: , current_location: position: (, 4), gtid-set: ", + newDDLJob(ddlInfo, []string{"create database test"}, binlog.NewLocation(""), binlog.NewLocation(""), binlog.NewLocation(""), "", nil), + "tp: ddl, sql: , args: [], key: , ddls: [create database test], last_location: position: (, 4), gtid-set: , start_location: position: (, 4), gtid-set: , current_location: position: (, 4), gtid-set: ", }, { - newXIDJob(binlog.NewLocation(""), binlog.NewLocation(""), ""), - "tp: xid, sql: , args: [], key: , ddls: [], last_location: position: (, 4), gtid-set: , current_location: position: (, 4), gtid-set: ", + newXIDJob(binlog.NewLocation(""), binlog.NewLocation(""), binlog.NewLocation(""), ""), + "tp: xid, sql: , args: [], key: , ddls: [], last_location: position: (, 4), gtid-set: , start_location: position: (, 4), gtid-set: , current_location: position: (, 4), gtid-set: ", }, { newFlushJob(), - "tp: flush, sql: , args: [], key: , ddls: [], last_location: position: (, 0), gtid-set: , current_location: position: (, 0), gtid-set: ", + "tp: flush, sql: , args: [], key: , ddls: [], last_location: position: (, 0), gtid-set: , start_location: position: (, 4), gtid-set: , current_location: position: (, 0), gtid-set: ", }, { newSkipJob(binlog.NewLocation("")), - "tp: skip, sql: , args: [], key: , ddls: [], last_location: position: (, 4), gtid-set: , current_location: position: (, 0), gtid-set: ", + "tp: skip, sql: , args: [], key: , ddls: [], last_location: position: (, 4), gtid-set: , start_location: position: (, 4), gtid-set: , current_location: position: (, 0), gtid-set: ", }, } diff --git a/syncer/optimist.go b/syncer/optimist.go index fc32baf4d0..f25fdc31ee 100644 --- a/syncer/optimist.go +++ b/syncer/optimist.go @@ -196,7 +196,7 @@ func (s *Syncer) handleQueryEventOptimistic( tableNames: needTrackDDLs[0].tableNames, stmt: needTrackDDLs[0].stmt, } - job := newDDLJob(ddlInfo, needHandleDDLs, *ec.lastLocation, *ec.currentLocation, *ec.traceID, nil) + job := newDDLJob(ddlInfo, needHandleDDLs, *ec.lastLocation, *ec.startLocation, *ec.currentLocation, *ec.traceID, nil) err = s.addJobFunc(job) if err != nil { return err diff --git a/syncer/syncer.go b/syncer/syncer.go index 7dd571b6b1..a8f9b5fa84 100644 --- a/syncer/syncer.go +++ b/syncer/syncer.go @@ -183,7 +183,8 @@ type Syncer struct { errLocation struct { sync.RWMutex - location *binlog.Location // record the error location + startLocation *binlog.Location + endLocation *binlog.Location } addJobFunc func(*job) error @@ -439,7 +440,7 @@ func (s *Syncer) reset() { s.execError.Set(nil) s.resetExecErrors() - s.setErrLocation(nil) + s.setErrLocation(nil, nil) s.firstReplace = false switch s.cfg.ShardMode { @@ -955,7 +956,7 @@ func (s *Syncer) syncDDL(tctx *tcontext.Context, queueBucket string, db *DBConn, if err != nil { s.execError.Set(err) if !utils.IsContextCanceledError(err) { - err = s.handleEventError(err, &sqlJob.currentLocation) + err = s.handleEventError(err, &sqlJob.startLocation, &sqlJob.currentLocation) s.runFatalChan <- unit.NewProcessError(err) } continue @@ -988,7 +989,7 @@ func (s *Syncer) sync(tctx *tcontext.Context, queueBucket string, db *DBConn, jo fatalF := func(err error, affected int) { s.execError.Set(err) if !utils.IsContextCanceledError(err) { - err = s.handleEventError(err, &jobs[affected].currentLocation) + err = s.handleEventError(err, &jobs[affected].startLocation, &jobs[affected].currentLocation) s.runFatalChan <- unit.NewProcessError(err) } clearF() @@ -1102,6 +1103,7 @@ func (s *Syncer) Run(ctx context.Context) (err error) { // we use lastPos to update global checkpoint and table checkpoint var ( currentLocation = s.checkpoint.GlobalPoint() // also init to global checkpoint + startLocation = s.checkpoint.GlobalPoint() lastLocation = s.checkpoint.GlobalPoint() ) s.tctx.L().Info("replicate binlog from checkpoint", zap.Stringer("checkpoint", lastLocation)) @@ -1245,7 +1247,6 @@ func (s *Syncer) Run(ctx context.Context) (err error) { var e *replication.BinlogEvent - // we only inject sqls in global streaming to avoid DDL position confusion if shardingReSync == nil { latestOp = null } @@ -1318,13 +1319,11 @@ func (s *Syncer) Run(ctx context.Context) (err error) { s.tctx.L().Debug("receive binlog event", zap.Reflect("header", e.Header)) + startLocation = currentLocation.Clone() // TODO: support all event - // we calculate startLocation and endLocation(currentLocation) here for Rows/Query event here - var startLocation *binlog.Location + // we calculate endLocation(currentLocation) for Rows/Query event here switch e.Event.(type) { case *replication.RowsEvent, *replication.QueryEvent: - location := currentLocation.Clone() - startLocation = &location endSuffix := startLocation.Suffix if s.firstReplace { endSuffix = 1 @@ -1341,7 +1340,11 @@ func (s *Syncer) Run(ctx context.Context) (err error) { GTIDSet: lastLocation.GTIDSet.Clone(), Suffix: endSuffix, } - apply, op := s.errOperatorHolder.Apply(startLocation, ¤tLocation) + if ev, ok := e.Event.(*replication.QueryEvent); ok { + currentLocation.GTIDSet.Set(ev.GSet) + } + + apply, op := s.errOperatorHolder.Apply(&startLocation, ¤tLocation) if apply { if op == pb.ErrorOp_Replace { s.firstReplace = true @@ -1361,7 +1364,7 @@ func (s *Syncer) Run(ctx context.Context) (err error) { ec := eventContext{ tctx: tctx, header: e.Header, - startLocation: startLocation, + startLocation: &startLocation, currentLocation: ¤tLocation, lastLocation: &lastLocation, shardingReSync: shardingReSync, @@ -1379,17 +1382,17 @@ func (s *Syncer) Run(ctx context.Context) (err error) { switch ev := e.Event.(type) { case *replication.RotateEvent: err = s.handleRotateEvent(ev, ec) - if err = s.handleEventError(err, ¤tLocation); err != nil { + if err = s.handleEventError(err, &startLocation, ¤tLocation); err != nil { return err } case *replication.RowsEvent: err = s.handleRowsEvent(ev, ec) - if err = s.handleEventError(err, ¤tLocation); err != nil { + if err = s.handleEventError(err, &startLocation, ¤tLocation); err != nil { return err } case *replication.QueryEvent: err = s.handleQueryEvent(ev, ec) - if err = s.handleEventError(err, ¤tLocation); err != nil { + if err = s.handleEventError(err, &startLocation, ¤tLocation); err != nil { return err } case *replication.XIDEvent: @@ -1417,9 +1420,9 @@ func (s *Syncer) Run(ctx context.Context) (err error) { lastLocation.Position.Pos = e.Header.LogPos // update lastPos lastLocation.GTIDSet.Set(ev.GSet) - job := newXIDJob(currentLocation, currentLocation, traceID) + job := newXIDJob(currentLocation, startLocation, currentLocation, traceID) err = s.addJobFunc(job) - if err = s.handleEventError(err, ¤tLocation); err != nil { + if err = s.handleEventError(err, &startLocation, ¤tLocation); err != nil { return err } case *replication.GenericEvent: @@ -1429,7 +1432,7 @@ func (s *Syncer) Run(ctx context.Context) (err error) { if s.checkpoint.CheckGlobalPoint() { s.tctx.L().Info("meet heartbeat event and then flush jobs") err = s.flushJobs() - if err = s.handleEventError(err, ¤tLocation); err != nil { + if err = s.handleEventError(err, &startLocation, ¤tLocation); err != nil { return err } } @@ -1624,7 +1627,7 @@ func (s *Syncer) handleRowsEvent(ev *replication.RowsEvent, ec eventContext) err if keys != nil { key = keys[i] } - err = s.commitJob(*ec.latestOp, originSchema, originTable, schemaName, tableName, sqls[i], arg, key, true, *ec.lastLocation, *ec.currentLocation, *ec.traceID) + err = s.commitJob(*ec.latestOp, originSchema, originTable, schemaName, tableName, sqls[i], arg, key, true, *ec.lastLocation, *ec.startLocation, *ec.currentLocation, *ec.traceID) if err != nil { return err } @@ -1634,8 +1637,6 @@ func (s *Syncer) handleRowsEvent(ev *replication.RowsEvent, ec eventContext) err } func (s *Syncer) handleQueryEvent(ev *replication.QueryEvent, ec eventContext) error { - ec.currentLocation.GTIDSet.Set(ev.GSet) - sql := strings.TrimSpace(string(ev.Query)) usedSchema := string(ev.Schema) parseResult, err := s.parseDDLSQL(sql, ec.parser2, usedSchema) @@ -1822,7 +1823,7 @@ func (s *Syncer) handleQueryEvent(ev *replication.QueryEvent, ec eventContext) e } }) - job := newDDLJob(nil, needHandleDDLs, *ec.lastLocation, *ec.currentLocation, *ec.traceID, sourceTbls) + job := newDDLJob(nil, needHandleDDLs, *ec.lastLocation, *ec.startLocation, *ec.currentLocation, *ec.traceID, sourceTbls) err = s.addJobFunc(job) if err != nil { return err @@ -2007,7 +2008,7 @@ func (s *Syncer) handleQueryEvent(ev *replication.QueryEvent, ec eventContext) e } }) - job := newDDLJob(ddlInfo, needHandleDDLs, *ec.lastLocation, *ec.currentLocation, *ec.traceID, nil) + job := newDDLJob(ddlInfo, needHandleDDLs, *ec.lastLocation, *ec.startLocation, *ec.currentLocation, *ec.traceID, nil) err = s.addJobFunc(job) if err != nil { return err @@ -2105,7 +2106,7 @@ func (s *Syncer) trackDDL(usedSchema string, sql string, tableNames [][]*filter. return nil } -func (s *Syncer) commitJob(tp opType, sourceSchema, sourceTable, targetSchema, targetTable, sql string, args []interface{}, keys []string, retry bool, location, cmdLocation binlog.Location, traceID string) error { +func (s *Syncer) commitJob(tp opType, sourceSchema, sourceTable, targetSchema, targetTable, sql string, args []interface{}, keys []string, retry bool, location, startLocation, cmdLocation binlog.Location, traceID string) error { startTime := time.Now() key, err := s.resolveCasuality(keys) if err != nil { @@ -2114,7 +2115,7 @@ func (s *Syncer) commitJob(tp opType, sourceSchema, sourceTable, targetSchema, t s.tctx.L().Debug("key for keys", zap.String("key", key), zap.Strings("keys", keys)) conflictDetectDurationHistogram.WithLabelValues(s.cfg.Name, s.cfg.SourceID).Observe(time.Since(startTime).Seconds()) - job := newJob(tp, sourceSchema, sourceTable, targetSchema, targetTable, sql, args, key, location, cmdLocation, traceID) + job := newJob(tp, sourceSchema, sourceTable, targetSchema, targetTable, sql, args, key, location, startLocation, cmdLocation, traceID) return s.addJobFunc(job) } @@ -2629,34 +2630,36 @@ func (s *Syncer) ShardDDLOperation() *pessimism.Operation { return s.pessimist.PendingOperation() } -func (s *Syncer) setErrLocation(location *binlog.Location) { +func (s *Syncer) setErrLocation(startLocation, endLocation *binlog.Location) { s.errLocation.Lock() defer s.errLocation.Unlock() - if s.errLocation.location == nil || location == nil { - s.errLocation.location = location - return + if s.errLocation.startLocation == nil || startLocation == nil { + s.errLocation.startLocation = startLocation + } else if binlog.CompareLocation(*startLocation, *s.errLocation.startLocation, s.cfg.EnableGTID) < 0 { + s.errLocation.startLocation = startLocation } - // only record the first error location - if binlog.CompareLocation(*location, *s.errLocation.location, s.cfg.EnableGTID) < 0 { - s.errLocation.location = location + if s.errLocation.endLocation == nil || endLocation == nil { + s.errLocation.endLocation = endLocation + } else if binlog.CompareLocation(*endLocation, *s.errLocation.endLocation, s.cfg.EnableGTID) < 0 { + s.errLocation.endLocation = endLocation } } func (s *Syncer) getErrLocation() *binlog.Location { s.errLocation.Lock() defer s.errLocation.Unlock() - return s.errLocation.location + return s.errLocation.startLocation } -func (s *Syncer) handleEventError(err error, location *binlog.Location) error { +func (s *Syncer) handleEventError(err error, startLocation, endLocation *binlog.Location) error { if err == nil { return nil } - s.setErrLocation(location) - return terror.Annotatef(err, "error location %s", location) + s.setErrLocation(startLocation, endLocation) + return terror.Annotatef(err, "[startLocation: %s, endLocation: %s]", startLocation, endLocation) } // getEvent gets an event from streamerController or errOperatorHolder From 694055bdc47ded2c471a50389fc54f08a892f082 Mon Sep 17 00:00:00 2001 From: gmhdbjd Date: Wed, 5 Aug 2020 17:13:57 +0800 Subject: [PATCH 11/31] add integration test --- tests/handle_error/conf/diff_config.toml | 55 +++ tests/handle_error/conf/dm-master.toml | 6 + tests/handle_error/conf/dm-worker1.toml | 2 + tests/handle_error/conf/dm-worker2.toml | 2 + .../conf/double-source-optimistic.yaml | 60 +++ .../conf/double-source-pessimistic.yaml | 60 +++ tests/handle_error/conf/no-sharding.yaml | 45 ++ tests/handle_error/conf/source1.yaml | 11 + tests/handle_error/conf/source2.yaml | 11 + tests/handle_error/lib.sh | 40 ++ tests/handle_error/run.sh | 384 ++++++++++++++++++ 11 files changed, 676 insertions(+) create mode 100644 tests/handle_error/conf/diff_config.toml create mode 100644 tests/handle_error/conf/dm-master.toml create mode 100644 tests/handle_error/conf/dm-worker1.toml create mode 100644 tests/handle_error/conf/dm-worker2.toml create mode 100644 tests/handle_error/conf/double-source-optimistic.yaml create mode 100644 tests/handle_error/conf/double-source-pessimistic.yaml create mode 100644 tests/handle_error/conf/no-sharding.yaml create mode 100644 tests/handle_error/conf/source1.yaml create mode 100644 tests/handle_error/conf/source2.yaml create mode 100644 tests/handle_error/lib.sh create mode 100644 tests/handle_error/run.sh diff --git a/tests/handle_error/conf/diff_config.toml b/tests/handle_error/conf/diff_config.toml new file mode 100644 index 0000000000..c3eed9887b --- /dev/null +++ b/tests/handle_error/conf/diff_config.toml @@ -0,0 +1,55 @@ +# diff Configuration. + +log-level = "info" + +chunk-size = 1000 + +check-thread-count = 4 + +sample-percent = 100 + +use-rowid = false + +use-checksum = true + +fix-sql-file = "fix.sql" + +# tables need to check. +[[check-tables]] +schema = "handle_error" +tables = ["tb"] + +[[table-config]] +schema = "handle_error" +table = "tb" +is-sharding = true + + [[table-config.source-tables]] + instance-id = "source-1" + schema = "handle_error" + table = "~tb*" + + [[table-config.source-tables]] + instance-id = "source-2" + schema = "handle_error" + table = "~tb*" + +[[source-db]] +host = "127.0.0.1" +port = 3306 +user = "root" +password = "123456" +instance-id = "source-1" + +[[source-db]] +host = "127.0.0.1" +port = 3307 +user = "root" +password = "123456" +instance-id = "source-2" + +[target-db] +host = "127.0.0.1" +port = 4000 +user = "test" +password = "123456" diff --git a/tests/handle_error/conf/dm-master.toml b/tests/handle_error/conf/dm-master.toml new file mode 100644 index 0000000000..aa24d7d7d3 --- /dev/null +++ b/tests/handle_error/conf/dm-master.toml @@ -0,0 +1,6 @@ +# Master Configuration. +name = "master1" +master-addr = ":8261" +advertise-addr = "127.0.0.1:8261" +peer-urls = "127.0.0.1:8291" +initial-cluster = "master1=http://127.0.0.1:8291" diff --git a/tests/handle_error/conf/dm-worker1.toml b/tests/handle_error/conf/dm-worker1.toml new file mode 100644 index 0000000000..6f1d1b5344 --- /dev/null +++ b/tests/handle_error/conf/dm-worker1.toml @@ -0,0 +1,2 @@ +name = "worker1" +join = "127.0.0.1:8261" \ No newline at end of file diff --git a/tests/handle_error/conf/dm-worker2.toml b/tests/handle_error/conf/dm-worker2.toml new file mode 100644 index 0000000000..8394916268 --- /dev/null +++ b/tests/handle_error/conf/dm-worker2.toml @@ -0,0 +1,2 @@ +name = "worker2" +join = "127.0.0.1:8261" \ No newline at end of file diff --git a/tests/handle_error/conf/double-source-optimistic.yaml b/tests/handle_error/conf/double-source-optimistic.yaml new file mode 100644 index 0000000000..d1b6dda25e --- /dev/null +++ b/tests/handle_error/conf/double-source-optimistic.yaml @@ -0,0 +1,60 @@ +--- +name: test +task-mode: all +is-sharding: true +shard-mode: "optimistic" +meta-schema: "dm_meta" +timezone: "Asia/Shanghai" + +target-database: + host: "127.0.0.1" + port: 4000 + user: "test" + password: "/Q7B9DizNLLTTfiZHv9WoEAKamfpIUs=" + +mysql-instances: + - source-id: "mysql-replica-01" + block-allow-list: "instance" + route-rules: ["sharding-table-rules1","sharding-table-rules2"] + mydumper-config-name: "global" + loader-config-name: "global" + syncer-config-name: "global" + - source-id: "mysql-replica-02" + block-allow-list: "instance" + route-rules: ["sharding-table-rules1","sharding-table-rules2"] + mydumper-config-name: "global" + loader-config-name: "global" + syncer-config-name: "global" + +block-allow-list: + instance: + do-dbs: ["handle_error"] + +routes: + sharding-table-rules1: + schema-pattern: "handle_error" + target-schema: "handle_error" + table-pattern: "tb*" + target-table: "tb" + sharding-table-rules2: + schema-pattern: "handle_error" + target-schema: "handle_error" + table-pattern: "ta*" + target-table: "ta" + +mydumpers: + global: + threads: 4 + chunk-filesize: 64 + skip-tz-utc: true + extra-args: "" + +loaders: + global: + pool-size: 16 + dir: "./dumped_data" + +syncers: + global: + worker-count: 16 + batch: 100 diff --git a/tests/handle_error/conf/double-source-pessimistic.yaml b/tests/handle_error/conf/double-source-pessimistic.yaml new file mode 100644 index 0000000000..739ac8581c --- /dev/null +++ b/tests/handle_error/conf/double-source-pessimistic.yaml @@ -0,0 +1,60 @@ +--- +name: test +task-mode: all +is-sharding: true +shard-mode: "pessimistic" +meta-schema: "dm_meta" +timezone: "Asia/Shanghai" + +target-database: + host: "127.0.0.1" + port: 4000 + user: "test" + password: "/Q7B9DizNLLTTfiZHv9WoEAKamfpIUs=" + +mysql-instances: + - source-id: "mysql-replica-01" + block-allow-list: "instance" + route-rules: ["sharding-table-rules1","sharding-table-rules2"] + mydumper-config-name: "global" + loader-config-name: "global" + syncer-config-name: "global" + - source-id: "mysql-replica-02" + block-allow-list: "instance" + route-rules: ["sharding-table-rules1","sharding-table-rules2"] + mydumper-config-name: "global" + loader-config-name: "global" + syncer-config-name: "global" + +block-allow-list: + instance: + do-dbs: ["handle_error"] + +routes: + sharding-table-rules1: + schema-pattern: "handle_error" + target-schema: "handle_error" + table-pattern: "tb*" + target-table: "tb" + sharding-table-rules2: + schema-pattern: "handle_error" + target-schema: "handle_error" + table-pattern: "ta*" + target-table: "ta" + +mydumpers: + global: + threads: 4 + chunk-filesize: 64 + skip-tz-utc: true + extra-args: "" + +loaders: + global: + pool-size: 16 + dir: "./dumped_data" + +syncers: + global: + worker-count: 16 + batch: 100 diff --git a/tests/handle_error/conf/no-sharding.yaml b/tests/handle_error/conf/no-sharding.yaml new file mode 100644 index 0000000000..d2116075a0 --- /dev/null +++ b/tests/handle_error/conf/no-sharding.yaml @@ -0,0 +1,45 @@ +--- +name: test +task-mode: all +is-sharding: false +meta-schema: "dm_meta" +timezone: "Asia/Shanghai" + +target-database: + host: "127.0.0.1" + port: 4000 + user: "test" + password: "/Q7B9DizNLLTTfiZHv9WoEAKamfpIUs=" + +mysql-instances: + - source-id: "mysql-replica-01" + block-allow-list: "instance" + mydumper-config-name: "global" + loader-config-name: "global" + syncer-config-name: "global" + - source-id: "mysql-replica-02" + block-allow-list: "instance" + mydumper-config-name: "global" + loader-config-name: "global" + syncer-config-name: "global" + +block-allow-list: + instance: + do-dbs: ["handle_error"] + +mydumpers: + global: + threads: 4 + chunk-filesize: 64 + skip-tz-utc: true + extra-args: "" + +loaders: + global: + pool-size: 16 + dir: "./dumped_data" + +syncers: + global: + worker-count: 16 + batch: 100 diff --git a/tests/handle_error/conf/source1.yaml b/tests/handle_error/conf/source1.yaml new file mode 100644 index 0000000000..175e07df7a --- /dev/null +++ b/tests/handle_error/conf/source1.yaml @@ -0,0 +1,11 @@ +source-id: mysql-replica-01 +flavor: '' +enable-gtid: false +enable-relay: false +from: + host: 127.0.0.1 + user: root + password: /Q7B9DizNLLTTfiZHv9WoEAKamfpIUs= + port: 3306 +checker: + check-enable: false diff --git a/tests/handle_error/conf/source2.yaml b/tests/handle_error/conf/source2.yaml new file mode 100644 index 0000000000..8850ed1849 --- /dev/null +++ b/tests/handle_error/conf/source2.yaml @@ -0,0 +1,11 @@ +source-id: mysql-replica-02 +flavor: '' +enable-gtid: true +enable-relay: false +from: + host: 127.0.0.1 + user: root + password: /Q7B9DizNLLTTfiZHv9WoEAKamfpIUs= + port: 3307 +checker: + check-enable: false \ No newline at end of file diff --git a/tests/handle_error/lib.sh b/tests/handle_error/lib.sh new file mode 100644 index 0000000000..d73bce4974 --- /dev/null +++ b/tests/handle_error/lib.sh @@ -0,0 +1,40 @@ +#!/bin/bash + +set -eu + +db="handle_error" +tb1="tb1" +tb2="tb2" +tb="tb" +ta1="ta1" +ta2="ta2" +ta="ta" + +function init_database() { + run_sql_both_source "drop database if exists ${db};" + run_sql_both_source "create database if not exists ${db};" +} + +function extract() { + str="$1" + source=${str:0:1} + database=${str:1:1} + table=${str:2:1} +} + +function init_table() { + for i in $@; do + extract $i + run_sql_source${source} "create table ${db}.tb${table} (id int);" + done +} + +function clean_table() { + run_sql_both_source "drop table if exists ${db}.${tb1};" + run_sql_both_source "drop table if exists ${db}.${tb2};" + run_sql_both_source "drop table if exists ${db}.${ta1};" + run_sql_both_source "drop table if exists ${db}.${ta2};" + run_sql_tidb "drop table if exists ${db}.${tb};" + run_sql_tidb "drop table if exists ${db}.${ta};" + run_sql_tidb "drop database if exists dm_meta;" +} \ No newline at end of file diff --git a/tests/handle_error/run.sh b/tests/handle_error/run.sh new file mode 100644 index 0000000000..45e1b300df --- /dev/null +++ b/tests/handle_error/run.sh @@ -0,0 +1,384 @@ +#!/bin/bash + +set -eu + +cur=$( cd "$( dirname "${BASH_SOURCE[0]}" )" && pwd ) +source $cur/../_utils/test_prepare +WORK_DIR=$TEST_DIR/$TEST_NAME +source $cur/lib.sh + +# skip modify column, two sources, no sharding +function DM_SKIP_ERROR_CASE() { + run_sql_source1 "insert into ${db}.${tb1} values(1);" + run_sql_source2 "insert into ${db}.${tb2} values(2);" + run_sql_source1 "alter table ${db}.${tb1} add column new_col1 int;" + run_sql_source2 "alter table ${db}.${tb2} add column new_col1 int;" + run_sql_source1 "insert into ${db}.${tb1} values(3,3);" + run_sql_source2 "insert into ${db}.${tb2} values(4,4);" + + # not support in TiDB + run_sql_source1 "alter table ${db}.${tb1} modify id varchar(10);" + run_sql_source2 "alter table ${db}.${tb2} modify id varchar(10);" + run_sql_source1 "insert into ${db}.${tb1} values('aaa',5);" + run_sql_source2 "insert into ${db}.${tb2} values('bbb',6);" + + run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "query-status test" \ + "Unsupported modify column" 2 + + # begin to handle error + # skip all sources + run_dm_ctl $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "handle-error test skip" \ + "\"result\": true" 3 + + # insert fail + run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "query-status test" \ + "Error .*: Incorrect int value" 2 + + # skip one source + run_dm_ctl $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "handle-error test skip -s mysql-replica-01" \ + "\"result\": true" 2 + + run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "query-status test" \ + "\"stage\": \"Running\"" 1 \ + "\"stage\": \"Paused\"" 1 + + # skip all sources + run_dm_ctl $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "handle-error test skip" \ + "\"result\": true" 3 \ + "\"source 'mysql-replica-01' has no error\"" 1 + + run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "query-status test" \ + "\"stage\": \"Running\"" 2 + + # '111' -> 111, '222' -> 222, no error + run_sql_source1 "insert into ${db}.${tb1} values('111',7)" + run_sql_source2 "insert into ${db}.${tb2} values('222',8)" + run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "query-status test" \ + "\"stage\": \"Running\"" 2 + + run_sql_tidb_with_retry "select count(1) from ${db}.${tb1} where id=111;" "count(1): 1" + run_sql_tidb_with_retry "select count(1) from ${db}.${tb2} where id=222;" "count(1): 1" +} + +function DM_SKIP_ERROR() { + run_case SKIP_ERROR "no-sharding" "init_table 111 212" "clean_table" "" +} + +# skip modify column, two sources, 4 tables, sharding +# source1: tb1 first ddl -> tb1 second ddl -> tb2 first ddl -> tb2 second ddl +# source2: tb1 first ddl -> tb2 first ddl -> tb1 second ddl -> tb2 second ddl +function DM_SKIP_ERROR_SHARDING_CASE() { + run_sql_source1 "insert into ${db}.${tb1} values(1);" + run_sql_source1 "insert into ${db}.${tb2} values(2);" + run_sql_source2 "insert into ${db}.${tb1} values(3);" + run_sql_source2 "insert into ${db}.${tb2} values(4);" + + # 11/21 first ddl + run_sql_source1 "alter table ${db}.${tb1} CHARACTER SET latin1 COLLATE latin1_danish_ci;" + run_sql_source2 "alter table ${db}.${tb1} CHARACTER SET latin1 COLLATE latin1_danish_ci;" + # 11 second ddl + run_sql_source1 "alter table ${db}.${tb1} add column new_col1 varchar(10);" + run_sql_source1 "insert into ${db}.${tb1} values(5,'aaa');" + # 12/22 first ddl + run_sql_source1 "alter table ${db}.${tb2} CHARACTER SET latin1 COLLATE latin1_danish_ci;" + run_sql_source2 "alter table ${db}.${tb2} CHARACTER SET latin1 COLLATE latin1_danish_ci;" + # 21 second ddl + run_sql_source2 "alter table ${db}.${tb1} add column new_col1 varchar(10);" + run_sql_source2 "insert into ${db}.${tb1} values(6,'bbb');" + # 12/22 second ddl + run_sql_source1 "alter table ${db}.${tb2} add column new_col1 varchar(10);" + run_sql_source1 "insert into ${db}.${tb2} values(7,'ccc');" + run_sql_source2 "alter table ${db}.${tb2} add column new_col1 varchar(10);" + run_sql_source2 "insert into ${db}.${tb2} values(8,'ddd');" + + # begin to handle error + # 11/21 first ddl: unsupport error + run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "query-status test" \ + "Unsupported modify change collate from latin1_bin to latin1_danish_ci" 2 + + # skip 11/21 first ddl + run_dm_ctl $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "handle-error test skip" \ + "\"result\": true" 3 + + if [[ "$1" = "pessimistic" ]]; then + # 11 second ddl bypass, 12 first ddl: detect conflict + # 22 first ddl: unsupport error + run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "query-status test" \ + "detect inconsistent DDL sequence from source" 1 \ + "Unsupported modify change collate from latin1_bin to latin1_danish_ci" 1 + else + # 12/22 first ddl: unsupport error + run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "query-status test" \ + "Unsupported modify change collate from latin1_bin to latin1_danish_ci" 2 + fi + + run_dm_ctl $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "handle-error test skip -s mysql-replica-01,mysql-replica-02" \ + "\"result\": true" 3 + + run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "query-status test" \ + "\"stage\": \"Running\"" 2 + + run_sql_tidb_with_retry "select count(1) from ${db}.${tb}" "count(1): 8" +} + +function DM_SKIP_ERROR_SHARDING() { + run_case SKIP_ERROR_SHARDING "double-source-pessimistic" "init_table 111 112 211 212" "clean_table" "pessimistic" + run_case SKIP_ERROR_SHARDING "double-source-optimistic" "init_table 111 112 211 212" "clean_table" "optimistic" +} + +# replace add foreign key with database name +# two source, no sharding +function DM_REPLACE_ERROR_CASE() { + run_sql_source1 "insert into ${db}.${tb2} values(1,1);" + run_sql_source1 "insert into ${db}.${tb1} values(2,1);" + + # error in TiDB + run_sql_source1 "alter table ${db}.${tb1} add constraint fk foreign key (b) references ${tb2}(a);" + run_sql_source1 "insert into ${db}.${tb2} values(3,3);" + run_sql_source1 "insert into ${db}.${tb1} values(4,3);" + + run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "query-status test" \ + "No database selected" 1 + + # replace sql + run_dm_ctl $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "handle-error test -s mysql-replica-01 replace alter table ${db}.${tb1} add constraint fk foreign key (b) references ${tb2}(a);" \ + "\"result\": true" 2 + + run_sql_source1 "insert into ${db}.${tb2} values(5,5);" + run_sql_source1 "insert into ${db}.${tb1} values(6,5);" + +} + +function DM_REPLACE_ERROR() { + run_case REPLACE_ERROR "no-sharding" \ + "run_sql_source1 \"create table ${db}.${tb2} (a int unique, b int);\"; \ + run_sql_source1 \"create table ${db}.${tb1} (a int unique, b int);\"" \ + "clean_table" "" +} + +# replace add column without unique, add foreign key with database name +# two source, 4 tables +# source1: tb1 first ddl -> tb1 second ddl -> tb2 first ddl -> tb2 second ddl +# source2: tb1 first ddl -> tb2 first ddl -> tb1 second ddl -> tb2 second ddl +function DM_REPLACE_ERROR_SHARDING_CASE() { + run_sql_source1 "insert into ${db}.${ta1} values(1,1),(3,3),(5,5),(7,7);" + run_sql_source2 "insert into ${db}.${ta1} values(2,2),(4,4),(6,6),(8,8);" + + # 11/21 first ddl + run_sql_source1 "alter table ${db}.${tb1} add column c int unique;" + run_sql_source2 "alter table ${db}.${tb1} add column c int unique;" + # 11 second ddl + run_sql_source1 "alter table ${db}.${tb1} add constraint foreign key (c) references ${ta1}(a);" + run_sql_source1 "insert into ${db}.${tb1} values(1,1,1);" + # 12/22 first ddl + run_sql_source1 "alter table ${db}.${tb2} add column c int unique;" + run_sql_source2 "alter table ${db}.${tb2} add column c int unique;" + # 21 second ddl + run_sql_source2 "alter table ${db}.${tb1} add constraint foreign key (c) references ${ta1}(a);" + run_sql_source2 "insert into ${db}.${tb1} values(2,2,2);" + # 12/22 second ddl + run_sql_source1 "alter table ${db}.${tb2} add constraint foreign key (c) references ${ta1}(a);" + run_sql_source1 "insert into ${db}.${tb2} values(3,3,3);" + run_sql_source2 "alter table ${db}.${tb2} add constraint foreign key (c) references ${ta1}(a);" + run_sql_source2 "insert into ${db}.${tb2} values(4,4,4);" + + # 11/21 first ddl: unsupport error + run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "query-status test" \ + "unsupported add column .* constraint UNIQUE KEY" 2 + + # begin to handle error + # replace 11/21 first ddl without unique + run_dm_ctl $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "handle-error test replace alter table ${db}.${tb1} add column c int;" \ + "\"result\": true" 3 + + if [[ "$1" = "pessimistic" ]]; then + # 11 second ddl bypass, 12 first ddl detect conflict + # 22 first ddl: detect conflict + run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "query-status test" \ + "detect inconsistent DDL sequence from source" 2 + + # replace 12,22 first ddl without unique + run_dm_ctl $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "handle-error test -s mysql-replica-01,mysql-replica-02 replace alter table ${db}.${tb2} add column c int;" \ + "\"result\": true" 3 + + # 11/21 second ddl: no database selected + run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "query-status test" \ + "No database selected" 2 + + # replace 11/21 second ddl with database name + run_dm_ctl $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "handle-error test replace alter table ${db}.${tb1} add constraint foreign key (c) references ${db}.${ta1}(a);" \ + "\"result\": true" 3 + + # 12/22 second ddl: detect conflict + run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "query-status test" \ + "detect inconsistent DDL sequence from source" 2 + + # replace 12/22 second ddl with database name one by one + run_dm_ctl $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "handle-error test -s mysql-replica-01 replace alter table ${db}.${tb2} add constraint foreign key (c) references ${db}.${ta1}(a);" \ + "\"result\": true" 2 + run_dm_ctl $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "handle-error test -s mysql-replica-02 replace alter table ${db}.${tb2} add constraint foreign key (c) references ${db}.${ta1}(a);" \ + "\"result\": true" 2 + else + # 11 second ddl: no database selected, 22 first ddl: unsupport error + run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "query-status test" \ + "unsupported add column .* constraint UNIQUE KEY" 1 \ + "No database selected" 1 + + # replace 11 second ddl + run_dm_ctl $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "handle-error test -s mysql-replica-01 replace alter table ${db}.${tb1} add constraint foreign key (c) references ${db}.${ta1}(a);" \ + "\"result\": true" 2 + + # replace 22 first ddl + run_dm_ctl $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "handle-error test -s mysql-replica-02 replace alter table ${db}.${tb2} add column c int;" \ + "\"result\": true" 2 + + # 12 first ddl: unsupport error, 21 second ddl: no database selected + run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "query-status test" \ + "unsupported add column .* constraint UNIQUE KEY" 1 \ + "No database selected" 1 + + # replace 12 first ddl + run_dm_ctl $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "handle-error test -s mysql-replica-01 replace alter table ${db}.${tb2} add column c int;" \ + "\"result\": true" 2 + + # replace 21 second ddl + run_dm_ctl $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "handle-error test -s mysql-replica-02 replace alter table ${db}.${tb1} add constraint foreign key (c) references ${db}.${ta1}(a);" \ + "\"result\": true" 2 + + # 12 first ddl, 22 second ddl: no database selected + run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "query-status test" \ + "No database selected" 2 + + # replace 12/22 second ddl with database name + run_dm_ctl $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "handle-error test replace alter table ${db}.${tb2} add constraint foreign key (c) references ${db}.${ta1}(a);" \ + "\"result\": true" 3 + + fi + + run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "query-status test" \ + "\"stage\": \"Running\"" 2 \ + + run_sql_tidb_with_retry "select count(1) from ${db}.${tb}" "count(1): 4" +} + +function DM_REPLACE_ERROR_SHARDING() { + run_case REPLACE_ERROR_SHARDING "double-source-pessimistic" \ + "run_sql_source1 \"create table ${db}.${tb1} (a int, b int);\"; \ + run_sql_source1 \"create table ${db}.${tb2} (a int, b int);\"; \ + run_sql_source2 \"create table ${db}.${tb1} (a int, b int);\"; \ + run_sql_source2 \"create table ${db}.${tb2} (a int, b int);\"; \ + run_sql_source1 \"create table ${db}.${ta1} (a int unique, b int);\"; \ + run_sql_source2 \"create table ${db}.${ta1} (a int unique, b int);\"" \ + "clean_table" "pessimistic" + + run_case REPLACE_ERROR_SHARDING "double-source-optimistic" \ + "run_sql_source1 \"create table ${db}.${tb1} (a int, b int);\"; \ + run_sql_source1 \"create table ${db}.${tb2} (a int, b int);\"; \ + run_sql_source2 \"create table ${db}.${tb1} (a int, b int);\"; \ + run_sql_source2 \"create table ${db}.${tb2} (a int, b int);\"; \ + run_sql_source1 \"create table ${db}.${ta1} (a int unique, b int);\"; \ + run_sql_source2 \"create table ${db}.${ta1} (a int unique, b int);\"" \ + "clean_table" "optimistic" +} + +# test handle_error fail on second replace ddl +# two sources, two tables +function DM_REPLACE_ERROR_MULTIPLE_CASE() { + run_sql_source1 "alter table ${db}.${tb1} add column a int unique, add column b int unique;" + run_sql_source2 "alter table ${db}.${tb1} add column a int unique, add column b int unique;" + run_sql_source1 "insert into ${db}.${tb1} values(1,1,1);" + run_sql_source2 "insert into ${db}.${tb1} values(2,2,2);" + + # 11, 21 unspported error + run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "query-status test" \ + "unsupported add column 'a' constraint UNIQUE KEY" 2 + + # begin to handle error + # replace 11/21 ddl, wrong on second replace ddl + run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "handle-error test replace \"alter table ${db}.${tb1} add column a int; alter table ${db}.${tb1} add column b int unique;\"" \ + "\"result\": true" 3 + + run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "query-status test" \ + "unsupported add column 'b' constraint UNIQUE KEY" 2 + + # now we change the second replace ddl, but first replace ddl will error because it has been executed in TiDB ... + run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "handle-error test replace \"alter table ${db}.${tb1} add column a int; alter table ${db}.${tb1} add column b int;\"" \ + "\"result\": true" 3 + + # 11, 21 first replace ddl error + run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "query-status test" \ + "Duplicate column name 'a'" 2 + + # now we only replace with ddl2 + run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "handle-error test replace \"alter table ${db}.${tb1} add column b int;\"" \ + "\"result\": true" 3 + + run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "query-status test" \ + "\"stage\": \"Running\"" 2 \ + + run_sql_tidb_with_retry "select count(1) from ${db}.${tb};" "count(1): 2" +} + +function DM_REPLACE_ERROR_MULTIPLE() { + run_case REPLACE_ERROR_MULTIPLE "double-source-pessimistic" "init_table 111 211" "clean_table" "pessimistic" + run_case REPLACE_ERROR_MULTIPLE "double-source-optimistic" "init_table 111 211" "clean_table" "optimistic" +} + +function run() { + init_cluster + init_database + # DM_SKIP_ERROR + # DM_SKIP_ERROR_SHARDING + # DM_REPLACE_ERROR + DM_REPLACE_ERROR_SHARDING + # DM_REPLACE_ERROR_MULTIPLE +} + +cleanup_data $db +# also cleanup dm processes in case of last run failed +cleanup_process $* +run $* +cleanup_process $* + +echo "[$(date)] <<<<<< test case $TEST_NAME success! >>>>>>" From 562b80a14d1178fbe1b1cb306b58668c988e6d78 Mon Sep 17 00:00:00 2001 From: gmhdbjd Date: Wed, 5 Aug 2020 17:55:18 +0800 Subject: [PATCH 12/31] minor update --- syncer/err-operator/operator.go | 14 +++++++------- syncer/syncer.go | 2 -- 2 files changed, 7 insertions(+), 9 deletions(-) diff --git a/syncer/err-operator/operator.go b/syncer/err-operator/operator.go index 084b960608..609556ac65 100644 --- a/syncer/err-operator/operator.go +++ b/syncer/err-operator/operator.go @@ -86,13 +86,13 @@ func (h *Holder) Set(pos string, op pb.ErrorOp, events []*replication.BinlogEven // GetEvent return a replace binlog event // for example: -// startLocation endLocation -// event 1 1000, 0 1010, 0 -// event 2 1010, 0 1020, 0 <-- replace it with event a,b,c -// replace event a 1010, 0 1010, 1 -// replace event b 1010, 1 1010, 2 -// replace event c 1010, 2 1020, 0 -// event 3 1020, 0 1030, 0 +// startLocation endLocation +// event 1 1000, 0 1010, 0 +// event 2 1010, 0 1020, 0 <-- replace it with event a,b,c +// replace event a 1010, 0 1010, 1 +// replace event b 1010, 1 1010, 2 +// replace event c 1010, 2 1020, 0 +// event 3 1020, 0 1030, 0 func (h *Holder) GetEvent(startLocation *binlog.Location) (*replication.BinlogEvent, error) { h.mu.Lock() defer h.mu.Unlock() diff --git a/syncer/syncer.go b/syncer/syncer.go index a8f9b5fa84..376c21e014 100644 --- a/syncer/syncer.go +++ b/syncer/syncer.go @@ -102,7 +102,6 @@ type Syncer struct { sgk *ShardingGroupKeeper // keeper to keep all sharding (sub) group in this syncer pessimist *shardddl.Pessimist // shard DDL pessimist optimist *shardddl.Optimist // shard DDL optimist - injectEventCh chan *replication.BinlogEvent // extra binlog event chan, used to inject binlog event into the main for loop binlogType BinlogType streamerController *StreamerController @@ -207,7 +206,6 @@ func NewSyncer(cfg *config.SubTaskConfig, etcdClient *clientv3.Client) *Syncer { syncer.count.Set(0) syncer.c = newCausality() syncer.done = nil - syncer.injectEventCh = make(chan *replication.BinlogEvent) syncer.tracer = tracing.GetTracer() syncer.setTimezone() syncer.addJobFunc = syncer.addJob From 229f82509b472e33d1908472a006ff3ac79bd21a Mon Sep 17 00:00:00 2001 From: gmhdbjd Date: Wed, 5 Aug 2020 18:01:05 +0800 Subject: [PATCH 13/31] fix ut --- syncer/job_test.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/syncer/job_test.go b/syncer/job_test.go index 37c7c6d230..ef73fc05b2 100644 --- a/syncer/job_test.go +++ b/syncer/job_test.go @@ -88,7 +88,7 @@ func (t *testJobSuite) TestJob(c *C) { }{ { newJob(insert, "test", "t1", "test", "t1", "insert into test.t1 values(?)", []interface{}{1}, "1", binlog.NewLocation(""), binlog.NewLocation(""), binlog.NewLocation(""), ""), - "tp: insert, sql: insert into test.t1 values(?), args: [1], key: 1, ddls: [], last_location: position: (, 4), gtid-set: , start_location: position: (, 4), gtid-set: ,current_location: position: (, 4), gtid-set: ", + "tp: insert, sql: insert into test.t1 values(?), args: [1], key: 1, ddls: [], last_location: position: (, 4), gtid-set: , start_location: position: (, 4), gtid-set: , current_location: position: (, 4), gtid-set: ", }, { newDDLJob(ddlInfo, []string{"create database test"}, binlog.NewLocation(""), binlog.NewLocation(""), binlog.NewLocation(""), "", nil), "tp: ddl, sql: , args: [], key: , ddls: [create database test], last_location: position: (, 4), gtid-set: , start_location: position: (, 4), gtid-set: , current_location: position: (, 4), gtid-set: ", From 84561a44d8441b6ba304e7e7d8844d70df2d2fc6 Mon Sep 17 00:00:00 2001 From: GMHDBJD <35025882+GMHDBJD@users.noreply.github.com> Date: Wed, 5 Aug 2020 18:03:20 +0800 Subject: [PATCH 14/31] format comment --- syncer/err-operator/operator.go | 14 +++++++------- 1 file changed, 7 insertions(+), 7 deletions(-) diff --git a/syncer/err-operator/operator.go b/syncer/err-operator/operator.go index 609556ac65..01eff7ecba 100644 --- a/syncer/err-operator/operator.go +++ b/syncer/err-operator/operator.go @@ -86,13 +86,13 @@ func (h *Holder) Set(pos string, op pb.ErrorOp, events []*replication.BinlogEven // GetEvent return a replace binlog event // for example: -// startLocation endLocation -// event 1 1000, 0 1010, 0 -// event 2 1010, 0 1020, 0 <-- replace it with event a,b,c -// replace event a 1010, 0 1010, 1 -// replace event b 1010, 1 1010, 2 -// replace event c 1010, 2 1020, 0 -// event 3 1020, 0 1030, 0 +// startLocation endLocation +// event 1 1000, 0 1010, 0 +// event 2 1010, 0 1020, 0 <--replace it with event a,b,c +// replace event a 1010, 0 1010, 1 +// replace event b 1010, 1 1010, 2 +// replace event c 1010, 2 1020, 0 +// event 3 1020, 0 1030, 0 func (h *Holder) GetEvent(startLocation *binlog.Location) (*replication.BinlogEvent, error) { h.mu.Lock() defer h.mu.Unlock() From 1de3af6c9107934ae2f459ddeb9ebbed876fa5af Mon Sep 17 00:00:00 2001 From: gmhdbjd Date: Wed, 5 Aug 2020 19:00:55 +0800 Subject: [PATCH 15/31] minor fix --- syncer/job_test.go | 4 ++-- syncer/syncer.go | 6 +++--- tests/others_integration.txt | 1 + 3 files changed, 6 insertions(+), 5 deletions(-) diff --git a/syncer/job_test.go b/syncer/job_test.go index ef73fc05b2..5c2c7befa9 100644 --- a/syncer/job_test.go +++ b/syncer/job_test.go @@ -97,10 +97,10 @@ func (t *testJobSuite) TestJob(c *C) { "tp: xid, sql: , args: [], key: , ddls: [], last_location: position: (, 4), gtid-set: , start_location: position: (, 4), gtid-set: , current_location: position: (, 4), gtid-set: ", }, { newFlushJob(), - "tp: flush, sql: , args: [], key: , ddls: [], last_location: position: (, 0), gtid-set: , start_location: position: (, 4), gtid-set: , current_location: position: (, 0), gtid-set: ", + "tp: flush, sql: , args: [], key: , ddls: [], last_location: position: (, 0), gtid-set: , start_location: position: (, 0), gtid-set: , current_location: position: (, 0), gtid-set: ", }, { newSkipJob(binlog.NewLocation("")), - "tp: skip, sql: , args: [], key: , ddls: [], last_location: position: (, 4), gtid-set: , start_location: position: (, 4), gtid-set: , current_location: position: (, 0), gtid-set: ", + "tp: skip, sql: , args: [], key: , ddls: [], last_location: position: (, 4), gtid-set: , start_location: position: (, 0), gtid-set: , current_location: position: (, 0), gtid-set: ", }, } diff --git a/syncer/syncer.go b/syncer/syncer.go index 376c21e014..2420ad5f1f 100644 --- a/syncer/syncer.go +++ b/syncer/syncer.go @@ -99,9 +99,9 @@ type Syncer struct { cfg *config.SubTaskConfig syncCfg replication.BinlogSyncerConfig - sgk *ShardingGroupKeeper // keeper to keep all sharding (sub) group in this syncer - pessimist *shardddl.Pessimist // shard DDL pessimist - optimist *shardddl.Optimist // shard DDL optimist + sgk *ShardingGroupKeeper // keeper to keep all sharding (sub) group in this syncer + pessimist *shardddl.Pessimist // shard DDL pessimist + optimist *shardddl.Optimist // shard DDL optimist binlogType BinlogType streamerController *StreamerController diff --git a/tests/others_integration.txt b/tests/others_integration.txt index 27610d24b4..498ad843cc 100644 --- a/tests/others_integration.txt +++ b/tests/others_integration.txt @@ -8,3 +8,4 @@ dm_syncer sequence_sharding_optimistic sequence_sharding_removemeta drop_column_with_index +handle_error From 72ab1d08b04f2e7cfde404fd210922da0df1cb17 Mon Sep 17 00:00:00 2001 From: gmhdbjd Date: Thu, 6 Aug 2020 11:31:22 +0800 Subject: [PATCH 16/31] add handle-error revert --- _utils/terror_gen/errors_release.txt | 1 + dm/ctl/master/handle_error.go | 12 +- dm/pb/dmworker.pb.go | 295 ++++++++++++++------------- dm/proto/dmworker.proto | 3 +- errors.toml | 6 + pkg/terror/error_list.go | 2 + syncer/err-operator/operator.go | 10 +- syncer/handle_error.go | 5 +- tests/handle_error/run.sh | 16 +- 9 files changed, 192 insertions(+), 158 deletions(-) diff --git a/_utils/terror_gen/errors_release.txt b/_utils/terror_gen/errors_release.txt index 8aed3129a1..6877812c5d 100644 --- a/_utils/terror_gen/errors_release.txt +++ b/_utils/terror_gen/errors_release.txt @@ -303,6 +303,7 @@ ErrSyncerUnitHandleDDLFailed,[code=36061:class=sync-unit:scope=internal:level=hi ErrSyncerShardDDLConflict,[code=36062:class=sync-unit:scope=internal:level=high], "Message: fail to handle shard ddl %v in optimistic mode, because schema conflict detected, Workaround: Please use show-ddl-locks command for more details." ErrSyncerFailpoint,[code=36063:class=sync-unit:scope=internal:level=low], "Message: failpoint specified error" ErrSyncerReplaceEvent,[code=36064:class=sync-unit:scope=internal:level=high] +ErrSyncerOperatorNotExist,[code=36065:class=sync-unit:scope=internal:level=low], "Message: error operator not exist, position: %s" ErrMasterSQLOpNilRequest,[code=38001:class=dm-master:scope=internal:level=medium], "Message: nil request not valid" ErrMasterSQLOpNotSupport,[code=38002:class=dm-master:scope=internal:level=medium], "Message: op %s not supported" ErrMasterSQLOpWithoutSharding,[code=38003:class=dm-master:scope=internal:level=medium], "Message: operate request without --sharding specified not valid" diff --git a/dm/ctl/master/handle_error.go b/dm/ctl/master/handle_error.go index a1ee9e5028..fd0097d6f3 100644 --- a/dm/ctl/master/handle_error.go +++ b/dm/ctl/master/handle_error.go @@ -28,8 +28,8 @@ import ( // NewHandleErrorCmd creates a HandleError command func NewHandleErrorCmd() *cobra.Command { cmd := &cobra.Command{ - Use: "handle-error [-s source ...] [-b binlog-pos] [replace-sql1;replace-sql2;]", - Short: "skip/replace the current error event or a specific binlog position (binlog-pos) event", + Use: "handle-error [-s source ...] [-b binlog-pos] [replace-sql1;replace-sql2;]", + Short: "skip/replace/revert the current error event or a specific binlog position (binlog-pos) event", RunE: handleErrorFunc, } cmd.Flags().StringP("binlog-pos", "b", "", "position used to match binlog event if matched the handler-error operation will be applied. The format like \"mysql-bin|000001.000003:3270\"") @@ -42,6 +42,8 @@ func convertOp(t string) pb.ErrorOp { return pb.ErrorOp_Skip case "replace": return pb.ErrorOp_Replace + case "revert": + return pb.ErrorOp_Revert default: return pb.ErrorOp_InvalidErrorOp } @@ -62,9 +64,9 @@ func handleErrorFunc(cmd *cobra.Command, _ []string) (err error) { op := convertOp(operation) switch op { - case pb.ErrorOp_Skip: + case pb.ErrorOp_Skip, pb.ErrorOp_Revert: if len(cmd.Flags().Args()) > 2 { - common.PrintLines("replace-sqls will be ignored for 'skip' operation") + common.PrintLines("replace-sqls will be ignored for 'skip/revert' operation") } case pb.ErrorOp_Replace: if len(cmd.Flags().Args()) <= 2 { @@ -78,7 +80,7 @@ func handleErrorFunc(cmd *cobra.Command, _ []string) (err error) { return } default: - common.PrintLines("invalid operation '%s', please use `skip` or `relpace`", operation) + common.PrintLines("invalid operation '%s', please use `skip`, `replace`, or `revert`", operation) err = errors.New("please check output to see error") return } diff --git a/dm/pb/dmworker.pb.go b/dm/pb/dmworker.pb.go index c42d1710bb..abf01e4e1d 100644 --- a/dm/pb/dmworker.pb.go +++ b/dm/pb/dmworker.pb.go @@ -266,19 +266,22 @@ type ErrorOp int32 const ( ErrorOp_Skip ErrorOp = 0 ErrorOp_Replace ErrorOp = 1 - ErrorOp_InvalidErrorOp ErrorOp = 2 + ErrorOp_Revert ErrorOp = 2 + ErrorOp_InvalidErrorOp ErrorOp = 3 ) var ErrorOp_name = map[int32]string{ 0: "Skip", 1: "Replace", - 2: "InvalidErrorOp", + 2: "Revert", + 3: "InvalidErrorOp", } var ErrorOp_value = map[string]int32{ "Skip": 0, "Replace": 1, - "InvalidErrorOp": 2, + "Revert": 2, + "InvalidErrorOp": 3, } func (x ErrorOp) String() string { @@ -3100,149 +3103,149 @@ func init() { func init() { proto.RegisterFile("dmworker.proto", fileDescriptor_51a1b9e17fd67b10) } var fileDescriptor_51a1b9e17fd67b10 = []byte{ - // 2265 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xb4, 0x19, 0xcb, 0x6e, 0xdc, 0xc8, - 0x71, 0xc8, 0x79, 0x68, 0xa6, 0x66, 0x46, 0xa6, 0x5a, 0x5e, 0xef, 0x78, 0xe2, 0x9d, 0x08, 0xf4, - 0x62, 0x57, 0xab, 0x83, 0x10, 0x2b, 0x0e, 0x36, 0x58, 0x60, 0x93, 0x8d, 0x25, 0xad, 0xec, 0x44, - 0x5a, 0xdb, 0x1c, 0x7b, 0x73, 0x0c, 0x38, 0x64, 0x6b, 0x44, 0x88, 0x43, 0xd2, 0x7c, 0x48, 0x98, - 0x43, 0x7e, 0x21, 0xc9, 0x25, 0x87, 0x00, 0xb9, 0x05, 0x41, 0x6e, 0x41, 0xbe, 0x22, 0xc9, 0x71, - 0x91, 0x53, 0x8e, 0x81, 0xfd, 0x0d, 0xb9, 0xe5, 0x10, 0x54, 0x75, 0x93, 0x6c, 0x6a, 0x1e, 0x5e, - 0x03, 0xc9, 0x8d, 0xf5, 0xe8, 0xea, 0xaa, 0xea, 0x7a, 0x75, 0x13, 0x36, 0xdd, 0xd9, 0x75, 0x18, - 0x5f, 0xf2, 0x78, 0x3f, 0x8a, 0xc3, 0x34, 0x64, 0x7a, 0x34, 0x31, 0x3f, 0x81, 0xed, 0x71, 0x6a, - 0xc7, 0xe9, 0x38, 0x9b, 0xbc, 0xb0, 0x93, 0x4b, 0x8b, 0xbf, 0xca, 0x78, 0x92, 0x32, 0x06, 0x8d, - 0xd4, 0x4e, 0x2e, 0x07, 0xda, 0x8e, 0xb6, 0xdb, 0xb1, 0xe8, 0xdb, 0xdc, 0x07, 0xf6, 0x32, 0x72, - 0xed, 0x94, 0x5b, 0xdc, 0xb7, 0xe7, 0x39, 0xe7, 0x00, 0x36, 0x9c, 0x30, 0x48, 0x79, 0x90, 0x4a, - 0xe6, 0x1c, 0x34, 0xc7, 0xb0, 0x7d, 0xe6, 0x4d, 0xe3, 0x9b, 0x0b, 0x46, 0x00, 0x8f, 0xbc, 0xc0, - 0x0f, 0xa7, 0x5f, 0xd9, 0x33, 0x2e, 0xd7, 0x28, 0x18, 0x76, 0x0f, 0x3a, 0x02, 0x7a, 0x16, 0x26, - 0x03, 0x7d, 0x47, 0xdb, 0xed, 0x5b, 0x25, 0xc2, 0x3c, 0x81, 0xf7, 0x9e, 0x46, 0x1c, 0x85, 0xde, - 0xd0, 0x78, 0x08, 0x7a, 0x18, 0x91, 0xb8, 0xcd, 0x03, 0xd8, 0x8f, 0x26, 0xfb, 0x48, 0x7c, 0x1a, - 0x59, 0x7a, 0x18, 0xa1, 0x35, 0x01, 0x6e, 0xa6, 0x0b, 0x6b, 0xf0, 0xdb, 0xbc, 0x82, 0x3b, 0x37, - 0x05, 0x25, 0x51, 0x18, 0x24, 0x7c, 0xad, 0xa4, 0x3b, 0xd0, 0x8a, 0x79, 0x92, 0xf9, 0x29, 0xc9, - 0x6a, 0x5b, 0x12, 0x42, 0x7c, 0x12, 0x66, 0xb1, 0xc3, 0x07, 0x75, 0xda, 0x43, 0x42, 0xcc, 0x80, - 0xfa, 0x2c, 0x99, 0x0e, 0x1a, 0x84, 0xc4, 0x4f, 0x73, 0x0f, 0x6e, 0x0b, 0x2f, 0x7e, 0x0b, 0x8f, - 0xef, 0x02, 0x7b, 0x9e, 0xf1, 0x78, 0x3e, 0x4e, 0xed, 0x34, 0x4b, 0x14, 0xce, 0xa0, 0x74, 0x9d, - 0xb0, 0xe6, 0x63, 0xd8, 0x22, 0xce, 0xe3, 0x38, 0x0e, 0xe3, 0x75, 0x8c, 0x11, 0xdc, 0x3e, 0x0c, - 0x67, 0xb3, 0x30, 0xf8, 0x39, 0x45, 0x42, 0x61, 0x74, 0x69, 0x98, 0x56, 0x31, 0x4c, 0x1a, 0xa0, - 0x17, 0x06, 0xac, 0x34, 0xf5, 0x0e, 0xb4, 0x44, 0x74, 0x49, 0x6b, 0x25, 0x64, 0xfe, 0x59, 0x83, - 0xed, 0x8a, 0x15, 0xef, 0xbc, 0xe3, 0x43, 0xe8, 0x89, 0x3d, 0x84, 0x04, 0xda, 0xb7, 0x7b, 0x60, - 0xe0, 0xd1, 0x8c, 0x15, 0xbc, 0x55, 0xe1, 0x62, 0x9f, 0x42, 0x3f, 0x11, 0x2e, 0x96, 0xcb, 0x1a, - 0x3b, 0xf5, 0xdd, 0xee, 0xc1, 0x16, 0x2d, 0x53, 0x09, 0x56, 0x95, 0xcf, 0xfc, 0x93, 0x26, 0xdd, - 0x2e, 0x9d, 0xf9, 0xce, 0xfa, 0x3e, 0x80, 0xae, 0xd0, 0x8b, 0x04, 0x48, 0x75, 0x6f, 0x95, 0xea, - 0x0a, 0xb9, 0x2a, 0x0f, 0x99, 0x28, 0x94, 0x10, 0x6b, 0x84, 0xae, 0x86, 0xa2, 0xab, 0x58, 0x54, - 0xe1, 0x32, 0xff, 0xa8, 0x41, 0xf7, 0xf0, 0x82, 0x3b, 0x52, 0x73, 0x54, 0x31, 0xb2, 0x93, 0x84, - 0xbb, 0xb9, 0x8a, 0x02, 0x62, 0xb7, 0xa1, 0x99, 0x86, 0xa9, 0xed, 0x93, 0x92, 0x4d, 0x4b, 0x00, - 0x98, 0x88, 0x49, 0xe6, 0x38, 0x3c, 0x49, 0xce, 0x33, 0x9f, 0xb4, 0x6c, 0x5a, 0x0a, 0x06, 0xa5, - 0x9d, 0xdb, 0x9e, 0xcf, 0x5d, 0x3a, 0xd0, 0xa6, 0x25, 0x21, 0xcc, 0xf8, 0x6b, 0x3b, 0x0e, 0xbc, - 0x60, 0x3a, 0x68, 0x12, 0x21, 0x07, 0x71, 0x85, 0xcb, 0x53, 0xdb, 0xf3, 0x07, 0xad, 0x1d, 0x6d, - 0xb7, 0x67, 0x49, 0xc8, 0xec, 0x01, 0x1c, 0x65, 0xb3, 0x48, 0xfa, 0xf7, 0x57, 0x1a, 0xc0, 0x69, - 0x68, 0xbb, 0x52, 0xe9, 0x0f, 0xa1, 0x7f, 0xee, 0x05, 0x5e, 0x72, 0xc1, 0xdd, 0x47, 0xf3, 0x94, - 0x27, 0xa4, 0x7b, 0xdd, 0xaa, 0x22, 0x51, 0x59, 0xd2, 0x5a, 0xb0, 0xe8, 0xc4, 0xa2, 0x60, 0xd8, - 0x10, 0xda, 0x51, 0x1c, 0x4e, 0x63, 0x9e, 0x24, 0x32, 0x2e, 0x0b, 0x18, 0xd7, 0xce, 0x78, 0x6a, - 0x8b, 0x22, 0x22, 0xa3, 0x53, 0xc1, 0x98, 0xbf, 0xd5, 0xa0, 0x3f, 0xbe, 0xb0, 0x63, 0xd7, 0x0b, - 0xa6, 0x27, 0x71, 0x98, 0x51, 0x9a, 0xa7, 0x76, 0x3c, 0xe5, 0x79, 0x4d, 0x93, 0x10, 0x66, 0xd4, - 0xd1, 0xd1, 0x29, 0xee, 0x5f, 0xc7, 0x8c, 0xc2, 0x6f, 0xa1, 0x7f, 0x9c, 0xa4, 0xa7, 0xa1, 0x63, - 0xa7, 0x5e, 0x18, 0xc8, 0xed, 0xab, 0x48, 0xca, 0x9a, 0x79, 0xe0, 0x90, 0x33, 0xeb, 0x94, 0x35, - 0x04, 0xa1, 0xde, 0x59, 0x20, 0x29, 0x4d, 0xa2, 0x14, 0xb0, 0xf9, 0x87, 0x3a, 0xc0, 0x78, 0x1e, - 0x38, 0xd2, 0x51, 0x3b, 0xd0, 0x25, 0x83, 0x8f, 0xaf, 0x78, 0x90, 0xe6, 0x6e, 0x52, 0x51, 0x28, - 0x8c, 0xc0, 0x17, 0x51, 0xee, 0xa2, 0x02, 0xc6, 0xb2, 0x1a, 0x73, 0x87, 0x07, 0x29, 0x12, 0xeb, - 0x44, 0x2c, 0x11, 0xcc, 0x84, 0xde, 0xcc, 0x4e, 0x52, 0x1e, 0x57, 0x9c, 0x54, 0xc1, 0xb1, 0x3d, - 0x30, 0x54, 0xf8, 0x24, 0xf5, 0x5c, 0x0a, 0x80, 0x8e, 0xb5, 0x80, 0x47, 0x79, 0x64, 0x44, 0x2e, - 0xaf, 0x25, 0xe4, 0xa9, 0x38, 0x94, 0xa7, 0xc2, 0x24, 0x6f, 0x43, 0xc8, 0xbb, 0x89, 0x47, 0x79, - 0x13, 0x3f, 0x74, 0x2e, 0xbd, 0x60, 0x4a, 0x07, 0xd0, 0x26, 0x57, 0x55, 0x70, 0xec, 0x73, 0x30, - 0xb2, 0x20, 0xe6, 0x49, 0xe8, 0x5f, 0x71, 0x97, 0xce, 0x31, 0x19, 0x74, 0x94, 0x9c, 0x57, 0x4f, - 0xd8, 0x5a, 0x60, 0x55, 0x4e, 0x08, 0x44, 0xf2, 0xc8, 0x13, 0x1a, 0x01, 0x4c, 0x48, 0x91, 0x17, - 0xf3, 0x88, 0x0f, 0xba, 0x22, 0x7a, 0x4a, 0x8c, 0xf9, 0x7b, 0x0d, 0x7a, 0x6a, 0x19, 0x52, 0x0a, - 0xa4, 0xb6, 0xa2, 0x40, 0xea, 0x6a, 0x81, 0x64, 0x9f, 0x14, 0x85, 0x45, 0x54, 0x0a, 0xd2, 0xf6, - 0x59, 0x1c, 0x62, 0x1e, 0x5a, 0x44, 0x28, 0x6a, 0xcd, 0x03, 0xe8, 0xc6, 0xd8, 0x4b, 0x8b, 0x8a, - 0x56, 0x54, 0x16, 0xab, 0x44, 0x5b, 0x2a, 0x8f, 0xf9, 0x37, 0x1d, 0xba, 0x0a, 0x71, 0xe1, 0xa4, - 0xb5, 0x6f, 0x79, 0xd2, 0xfa, 0x8a, 0x93, 0xde, 0xc9, 0x55, 0xca, 0x26, 0x47, 0x5e, 0x2c, 0x83, - 0x5f, 0x45, 0x15, 0x1c, 0x95, 0xd0, 0x52, 0x51, 0x6c, 0x17, 0x6e, 0x29, 0xa0, 0x12, 0x58, 0x37, - 0xd1, 0x6c, 0x1f, 0x18, 0xa1, 0x0e, 0xed, 0xd4, 0xb9, 0x78, 0x19, 0x9d, 0x91, 0x36, 0x14, 0x5d, - 0x6d, 0x6b, 0x09, 0x85, 0x7d, 0x17, 0x9a, 0x49, 0x6a, 0x4f, 0x39, 0x05, 0xd6, 0xe6, 0x41, 0x87, - 0x02, 0x01, 0x11, 0x96, 0xc0, 0x2b, 0xce, 0x6f, 0xbf, 0xc5, 0xf9, 0xe6, 0x7f, 0x74, 0xe8, 0x57, - 0x1a, 0xc7, 0xb2, 0x06, 0x5b, 0xee, 0xa8, 0xaf, 0xd8, 0x71, 0x07, 0x1a, 0x59, 0xe0, 0x89, 0xc3, - 0xde, 0x3c, 0xe8, 0x21, 0xfd, 0x65, 0xe0, 0xa5, 0x18, 0x4b, 0x16, 0x51, 0x14, 0x9d, 0x1a, 0x6f, - 0x0b, 0x88, 0xef, 0xc1, 0x76, 0x19, 0xc8, 0x47, 0x47, 0xa7, 0xa7, 0xa1, 0x73, 0xf9, 0xe4, 0x48, - 0x7a, 0x6f, 0x19, 0x89, 0x31, 0xd1, 0xae, 0x28, 0x21, 0x1f, 0xd7, 0x44, 0xc3, 0xfa, 0x18, 0x9a, - 0x0e, 0xb6, 0x11, 0xf2, 0x92, 0x0c, 0x28, 0xa5, 0xaf, 0x3c, 0xae, 0x59, 0x82, 0xce, 0x3e, 0x84, - 0x86, 0x9b, 0xcd, 0x22, 0xe9, 0xab, 0x4d, 0xe4, 0x2b, 0x0b, 0xfb, 0xe3, 0x9a, 0x45, 0x54, 0xe4, - 0xf2, 0x43, 0xdb, 0x1d, 0x74, 0x4a, 0xae, 0xb2, 0xde, 0x23, 0x17, 0x52, 0x91, 0x0b, 0x33, 0x8c, - 0xb2, 0x4d, 0x72, 0x95, 0xc5, 0x0e, 0xb9, 0x90, 0xfa, 0xa8, 0x0d, 0xad, 0x44, 0x04, 0xf2, 0x8f, - 0x60, 0xab, 0xe2, 0xfd, 0x53, 0x2f, 0x21, 0x57, 0x09, 0xf2, 0x40, 0x5b, 0xd5, 0xdd, 0xf3, 0xf5, - 0x23, 0x00, 0xb2, 0x49, 0x34, 0x5c, 0xd9, 0xb5, 0xb5, 0x72, 0x30, 0xfb, 0x00, 0x3a, 0x68, 0xcb, - 0x1a, 0x32, 0x1a, 0xb1, 0x8a, 0x1c, 0x41, 0x8f, 0xb4, 0x7f, 0x7e, 0xba, 0x82, 0x83, 0x1d, 0xc0, - 0x6d, 0xd1, 0x40, 0x8b, 0x61, 0xd6, 0xa3, 0x76, 0x21, 0x12, 0x6b, 0x29, 0x0d, 0x0b, 0x3a, 0x47, - 0x71, 0xe3, 0xe7, 0xa7, 0x79, 0x57, 0xcb, 0x61, 0xf3, 0x07, 0xd0, 0xc1, 0x1d, 0xc5, 0x76, 0xbb, - 0xd0, 0x22, 0x42, 0xee, 0x07, 0xa3, 0x70, 0xa7, 0x54, 0xc8, 0x92, 0x74, 0xf3, 0xd7, 0x5a, 0x65, - 0x3a, 0x79, 0xe7, 0x6a, 0xb5, 0xb3, 0x38, 0xdc, 0x74, 0xaa, 0xb3, 0xcc, 0x3e, 0x00, 0x15, 0x9c, - 0x7c, 0x92, 0x29, 0x8e, 0xb7, 0xc4, 0x5a, 0x0a, 0x07, 0x1e, 0x4c, 0x09, 0x2d, 0x71, 0xed, 0xef, - 0x74, 0xe8, 0xa9, 0x43, 0xd0, 0xff, 0x2b, 0xed, 0x98, 0x32, 0xab, 0xe7, 0x99, 0xf1, 0x51, 0x9e, - 0x19, 0xcd, 0xd2, 0x8c, 0x32, 0x8a, 0xca, 0xc4, 0xb8, 0x2f, 0x13, 0xa3, 0x45, 0x6c, 0xfd, 0x3c, - 0x31, 0x72, 0x2e, 0x91, 0x17, 0xf7, 0x65, 0x5e, 0x6c, 0x94, 0x4c, 0x45, 0x48, 0x15, 0x69, 0x71, - 0x5f, 0xa6, 0x45, 0xbb, 0x64, 0x2a, 0x8e, 0xb9, 0xc8, 0x8a, 0x0d, 0x68, 0xd2, 0x71, 0x9a, 0x9f, - 0x81, 0xa1, 0xba, 0x86, 0x72, 0xe2, 0x23, 0x49, 0xac, 0x84, 0x82, 0x3a, 0x44, 0xca, 0xb5, 0xaf, - 0xa0, 0x5f, 0x29, 0x2a, 0xd8, 0xe9, 0xbc, 0xe4, 0xd0, 0x0e, 0x1c, 0xee, 0x17, 0x23, 0xa4, 0x82, - 0x51, 0x82, 0x4c, 0x2f, 0x25, 0x4b, 0x11, 0x95, 0x20, 0x53, 0x06, 0xc1, 0x7a, 0x65, 0x10, 0xfc, - 0x87, 0x06, 0x3d, 0x75, 0x01, 0xce, 0x92, 0xc7, 0x71, 0x7c, 0x18, 0xba, 0xe2, 0x34, 0x9b, 0x56, - 0x0e, 0x62, 0xe8, 0xe3, 0xa7, 0x6f, 0x27, 0x89, 0x8c, 0xc0, 0x02, 0x96, 0xb4, 0xb1, 0x13, 0x46, - 0xf9, 0x25, 0xa4, 0x80, 0x25, 0xed, 0x94, 0x5f, 0x71, 0x5f, 0xb6, 0x9a, 0x02, 0xc6, 0xdd, 0xce, - 0x78, 0x92, 0x60, 0x98, 0x88, 0x0a, 0x99, 0x83, 0xb8, 0xca, 0xb2, 0xaf, 0x0f, 0xed, 0x2c, 0xe1, - 0x72, 0x56, 0x29, 0x60, 0x74, 0x0b, 0x5e, 0x96, 0xec, 0x38, 0xcc, 0x82, 0x7c, 0x42, 0x51, 0x30, - 0xe6, 0x10, 0x06, 0xe3, 0x6b, 0x2f, 0x75, 0x2e, 0x28, 0x8a, 0x45, 0xe3, 0x91, 0x57, 0x30, 0xf3, - 0x00, 0xb6, 0xe5, 0x2d, 0xb3, 0x72, 0x07, 0xfe, 0x8e, 0x72, 0xc5, 0xec, 0x16, 0xa9, 0x21, 0xee, - 0x98, 0x66, 0x06, 0xb7, 0xab, 0x6b, 0xe4, 0x05, 0x64, 0xdd, 0xa2, 0xff, 0xc1, 0xc5, 0xf4, 0x1a, - 0xb6, 0x9e, 0x65, 0xf1, 0xb4, 0xaa, 0xe8, 0x10, 0xda, 0x5e, 0x60, 0x3b, 0xa9, 0x77, 0xc5, 0x65, - 0x40, 0x14, 0x30, 0xdd, 0x58, 0x3d, 0x79, 0xab, 0xae, 0x5b, 0xf4, 0x8d, 0xfc, 0xe7, 0x9e, 0xcf, - 0x29, 0x3d, 0xe5, 0xc9, 0xe4, 0x30, 0xa9, 0x22, 0x86, 0x04, 0x79, 0x41, 0x14, 0x10, 0xfa, 0x8f, - 0xae, 0x5b, 0xe2, 0x46, 0x7a, 0x18, 0x06, 0xe7, 0xde, 0x34, 0xf7, 0xdf, 0x35, 0xdc, 0x5d, 0x42, - 0x7b, 0xcb, 0x8d, 0xac, 0xb4, 0x59, 0x5f, 0x66, 0x73, 0xbd, 0xac, 0xc9, 0xca, 0xe3, 0x45, 0xa3, - 0xfa, 0x78, 0xf1, 0x17, 0x0d, 0x86, 0xf2, 0x14, 0xc4, 0xde, 0x63, 0xe7, 0x82, 0xcf, 0xec, 0xdc, - 0x2f, 0xf7, 0x94, 0xb3, 0xa0, 0x5a, 0x22, 0xc8, 0xe5, 0x7b, 0x03, 0xdd, 0xe5, 0xf5, 0xf2, 0x2e, - 0xbf, 0xf2, 0x20, 0x86, 0xd0, 0x76, 0xed, 0xd4, 0x9e, 0xd8, 0x09, 0xcf, 0xe3, 0x35, 0x87, 0xe9, - 0xde, 0x66, 0x4f, 0xfc, 0x3c, 0x5a, 0x05, 0x40, 0x92, 0x68, 0x37, 0x19, 0xa9, 0x12, 0x32, 0x53, - 0xe8, 0x7f, 0xfd, 0x40, 0x26, 0xfa, 0x19, 0x4f, 0xed, 0xb5, 0x0f, 0x19, 0x6f, 0xad, 0x97, 0x79, - 0x91, 0xad, 0x2b, 0x45, 0x36, 0xb7, 0xab, 0x41, 0x49, 0x2d, 0xde, 0x28, 0x1e, 0x16, 0xd1, 0xfa, - 0xf5, 0x03, 0xdc, 0x75, 0xa5, 0x87, 0x04, 0x59, 0xc6, 0xf8, 0x5f, 0xb5, 0xe2, 0x1d, 0x27, 0x5f, - 0xf6, 0xce, 0xd7, 0xec, 0x4f, 0xa1, 0x81, 0x97, 0xb8, 0x41, 0x9d, 0x8a, 0xd1, 0x7d, 0xdc, 0x63, - 0xa9, 0xc8, 0x7d, 0x04, 0x8e, 0x83, 0x34, 0x9e, 0x5b, 0xb4, 0x60, 0xf8, 0x53, 0xe8, 0x14, 0x28, - 0x94, 0x7b, 0xc9, 0xe7, 0x79, 0xbf, 0xb9, 0xe4, 0x73, 0x9c, 0x86, 0xae, 0x6c, 0x3f, 0x13, 0xae, - 0x91, 0x23, 0x45, 0xc5, 0xb1, 0x96, 0xa0, 0x7f, 0xa6, 0xff, 0x50, 0x33, 0x7f, 0x09, 0x83, 0xc7, - 0x76, 0xe0, 0xfa, 0x32, 0x4a, 0x2a, 0xef, 0x2f, 0x0b, 0x09, 0x4b, 0xd4, 0x35, 0x31, 0x72, 0x0f, - 0x3a, 0x93, 0xe2, 0xe9, 0x4b, 0x38, 0xbe, 0x44, 0xe0, 0x8a, 0xe4, 0x95, 0x9f, 0xc8, 0x0b, 0x24, - 0x7d, 0xef, 0xfd, 0x02, 0x5a, 0xe2, 0x50, 0x59, 0x1f, 0x3a, 0x4f, 0x82, 0x2b, 0xdb, 0xf7, 0xdc, - 0xa7, 0x91, 0x51, 0x63, 0x6d, 0x68, 0x8c, 0xd3, 0x30, 0x32, 0x34, 0xd6, 0x81, 0xe6, 0x33, 0xac, - 0x63, 0x86, 0xce, 0x00, 0x5a, 0x58, 0xea, 0x67, 0xdc, 0xa8, 0x23, 0x9a, 0x1e, 0xfe, 0x8c, 0x06, - 0xa2, 0xc5, 0x93, 0x94, 0xd1, 0x64, 0x9b, 0x00, 0x3f, 0xc9, 0xd2, 0x50, 0xb2, 0xb5, 0xf6, 0x5e, - 0x11, 0xdb, 0x14, 0x93, 0xa7, 0x27, 0xe5, 0x13, 0x6c, 0xd4, 0xd8, 0x06, 0xd4, 0xbf, 0xe2, 0xd7, - 0x86, 0xc6, 0xba, 0xb0, 0x61, 0x65, 0x41, 0xe0, 0x05, 0x53, 0xb1, 0x07, 0x6d, 0xe7, 0x1a, 0x75, - 0x24, 0xa0, 0x12, 0x11, 0x77, 0x8d, 0x06, 0xeb, 0x41, 0xfb, 0x4b, 0x79, 0xa5, 0x37, 0x9a, 0x48, - 0x42, 0x36, 0x5c, 0xd3, 0x42, 0x12, 0x6d, 0x88, 0xd0, 0xc6, 0xde, 0x53, 0x68, 0xe7, 0x9d, 0x99, - 0xdd, 0x82, 0xae, 0xdc, 0x15, 0x51, 0x46, 0x0d, 0xd5, 0xa6, 0xfe, 0x6b, 0x68, 0x68, 0x22, 0xf6, - 0x58, 0x43, 0xc7, 0x2f, 0x6c, 0xa4, 0x46, 0x9d, 0xcc, 0x9e, 0x07, 0x8e, 0xd1, 0x40, 0x46, 0xaa, - 0x64, 0x86, 0xbb, 0x77, 0x06, 0x1b, 0xb2, 0x54, 0x32, 0x06, 0x9b, 0x52, 0x9e, 0xc4, 0x18, 0x35, - 0xf4, 0x1c, 0x6a, 0x29, 0xb8, 0x35, 0xf4, 0x00, 0x19, 0x20, 0x60, 0x1d, 0x55, 0x10, 0xde, 0x10, - 0x88, 0x3a, 0xea, 0x97, 0x67, 0x3b, 0xdb, 0x86, 0x5b, 0xb9, 0x57, 0x24, 0x4a, 0x08, 0x3c, 0xe1, - 0xa9, 0x40, 0x18, 0x1a, 0xc9, 0x2f, 0x40, 0x1d, 0x1d, 0x69, 0xf1, 0x59, 0x78, 0xc5, 0x25, 0xa6, - 0xbe, 0xf7, 0x05, 0xb4, 0xf3, 0xe4, 0x50, 0x04, 0xe6, 0xa8, 0x42, 0xa0, 0x40, 0x18, 0x5a, 0x29, - 0x41, 0x62, 0xf4, 0xbd, 0x87, 0xd4, 0x46, 0x31, 0xb6, 0xc8, 0x03, 0x97, 0x1e, 0xae, 0xc2, 0x63, - 0xe1, 0x91, 0x6f, 0x3b, 0xdc, 0xd0, 0x14, 0xc3, 0x25, 0xa3, 0xa1, 0x1f, 0xfc, 0x7b, 0x03, 0x5a, - 0x22, 0x6c, 0xd9, 0x21, 0xf4, 0xd4, 0x67, 0x60, 0xf6, 0xbe, 0xac, 0x07, 0x37, 0x1f, 0x86, 0x87, - 0x03, 0x9a, 0x80, 0x96, 0xbc, 0x20, 0x9a, 0x35, 0xf6, 0x04, 0x36, 0xab, 0x4f, 0xaa, 0xec, 0xae, - 0x92, 0x94, 0x37, 0x04, 0x0d, 0x97, 0x91, 0x0a, 0x51, 0xc7, 0xd0, 0xaf, 0xbc, 0x92, 0x32, 0xda, - 0x77, 0xd9, 0xc3, 0xe9, 0x5a, 0x8d, 0xbe, 0x80, 0xae, 0xf2, 0xf4, 0xc8, 0xee, 0x20, 0xeb, 0xe2, - 0x8b, 0xea, 0xf0, 0xfd, 0x05, 0x7c, 0x21, 0xe1, 0x73, 0x80, 0xf2, 0x2d, 0x90, 0xbd, 0x57, 0x30, - 0xaa, 0x89, 0x3e, 0xbc, 0x73, 0x13, 0x5d, 0x2c, 0x3f, 0x83, 0xad, 0x85, 0xd9, 0x80, 0xdd, 0x23, - 0xe7, 0xae, 0x18, 0x19, 0xd6, 0xda, 0x73, 0x08, 0x3d, 0x75, 0x34, 0x10, 0xc7, 0xb4, 0x64, 0xc0, - 0x10, 0x42, 0x96, 0x4d, 0x11, 0x66, 0x8d, 0xfd, 0x18, 0xa0, 0x6c, 0xf4, 0xc2, 0xa4, 0x85, 0xc6, - 0xbf, 0x56, 0x8b, 0x13, 0xd8, 0x52, 0x7e, 0x04, 0x88, 0xa6, 0x2c, 0x7c, 0xbb, 0xf8, 0x7f, 0x60, - 0xad, 0x20, 0x4b, 0xbe, 0x5a, 0xab, 0xdd, 0x5d, 0x78, 0x67, 0xd5, 0x40, 0x30, 0xfc, 0x60, 0x05, - 0x55, 0x75, 0x91, 0xfa, 0xd7, 0x41, 0xb8, 0x68, 0xc9, 0x7f, 0x88, 0xb5, 0x8a, 0xfd, 0x0c, 0xfa, - 0x79, 0x68, 0x52, 0x92, 0xb2, 0x91, 0xe2, 0xcf, 0x25, 0xf3, 0xc0, 0x5a, 0x61, 0x5f, 0x16, 0xc2, - 0x44, 0xbe, 0xb2, 0xc1, 0x92, 0x56, 0x25, 0xc4, 0xdc, 0x5d, 0xd9, 0xc4, 0xc8, 0xed, 0x5d, 0xd1, - 0x6a, 0x44, 0x2c, 0x92, 0x9f, 0x56, 0xf5, 0x9e, 0x75, 0x0a, 0x3d, 0x1a, 0xfc, 0xfd, 0xf5, 0x48, - 0xfb, 0xe6, 0xf5, 0x48, 0xfb, 0xd7, 0xeb, 0x91, 0xf6, 0x9b, 0x37, 0xa3, 0xda, 0x37, 0x6f, 0x46, - 0xb5, 0x7f, 0xbe, 0x19, 0xd5, 0x26, 0x2d, 0xfa, 0x31, 0xf4, 0xfd, 0xff, 0x06, 0x00, 0x00, 0xff, - 0xff, 0x35, 0x89, 0x24, 0x7f, 0x2a, 0x1a, 0x00, 0x00, + // 2272 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xb4, 0x19, 0x4d, 0x6f, 0xdb, 0xc8, + 0x55, 0x24, 0x25, 0x59, 0x7a, 0x92, 0x1c, 0x7a, 0x9c, 0xcd, 0x2a, 0x6a, 0x56, 0x35, 0x98, 0xc5, + 0xae, 0xd7, 0x07, 0xa3, 0x71, 0xb7, 0xd8, 0x62, 0x81, 0x6d, 0xd3, 0xd8, 0x5e, 0x27, 0xad, 0xbd, + 0x49, 0xa8, 0x64, 0x7b, 0x2c, 0x28, 0x6a, 0x2c, 0x13, 0xa6, 0x48, 0x86, 0x1f, 0x36, 0x74, 0xe8, + 0x5f, 0x68, 0x7b, 0xe9, 0xa1, 0x40, 0x6f, 0x45, 0xd1, 0x5b, 0xd1, 0x5f, 0xd1, 0xf6, 0xb8, 0xe8, + 0xa9, 0xc7, 0x22, 0xf9, 0x0d, 0xbd, 0xf5, 0x50, 0xbc, 0x37, 0x43, 0x72, 0x68, 0x7d, 0x64, 0x03, + 0xb4, 0x37, 0xbe, 0x8f, 0x79, 0xf3, 0xde, 0x9b, 0xf7, 0x35, 0x43, 0xd8, 0x9c, 0xcc, 0xae, 0xc3, + 0xf8, 0x92, 0xc7, 0xfb, 0x51, 0x1c, 0xa6, 0x21, 0xd3, 0xa3, 0xb1, 0xf5, 0x09, 0x6c, 0x8f, 0x52, + 0x27, 0x4e, 0x47, 0xd9, 0xf8, 0x85, 0x93, 0x5c, 0xda, 0xfc, 0x55, 0xc6, 0x93, 0x94, 0x31, 0xa8, + 0xa7, 0x4e, 0x72, 0xd9, 0xd7, 0x76, 0xb4, 0xdd, 0xb6, 0x4d, 0xdf, 0xd6, 0x3e, 0xb0, 0x97, 0xd1, + 0xc4, 0x49, 0xb9, 0xcd, 0x7d, 0x67, 0x9e, 0x73, 0xf6, 0x61, 0xc3, 0x0d, 0x83, 0x94, 0x07, 0xa9, + 0x64, 0xce, 0x41, 0x6b, 0x04, 0xdb, 0x67, 0xde, 0x34, 0xbe, 0xb9, 0x60, 0x08, 0xf0, 0xc8, 0x0b, + 0xfc, 0x70, 0xfa, 0x95, 0x33, 0xe3, 0x72, 0x8d, 0x82, 0x61, 0xf7, 0xa0, 0x2d, 0xa0, 0x67, 0x61, + 0xd2, 0xd7, 0x77, 0xb4, 0xdd, 0x9e, 0x5d, 0x22, 0xac, 0x13, 0x78, 0xef, 0x69, 0xc4, 0x51, 0xe8, + 0x0d, 0x8d, 0x07, 0xa0, 0x87, 0x11, 0x89, 0xdb, 0x3c, 0x80, 0xfd, 0x68, 0xbc, 0x8f, 0xc4, 0xa7, + 0x91, 0xad, 0x87, 0x11, 0x5a, 0x13, 0xe0, 0x66, 0xba, 0xb0, 0x06, 0xbf, 0xad, 0x2b, 0xb8, 0x73, + 0x53, 0x50, 0x12, 0x85, 0x41, 0xc2, 0xd7, 0x4a, 0xba, 0x03, 0xcd, 0x98, 0x27, 0x99, 0x9f, 0x92, + 0xac, 0x96, 0x2d, 0x21, 0xc4, 0x27, 0x61, 0x16, 0xbb, 0xbc, 0x6f, 0xd0, 0x1e, 0x12, 0x62, 0x26, + 0x18, 0xb3, 0x64, 0xda, 0xaf, 0x13, 0x12, 0x3f, 0xad, 0x3d, 0xb8, 0x2d, 0xbc, 0xf8, 0x2d, 0x3c, + 0xbe, 0x0b, 0xec, 0x79, 0xc6, 0xe3, 0xf9, 0x28, 0x75, 0xd2, 0x2c, 0x51, 0x38, 0x83, 0xd2, 0x75, + 0xc2, 0x9a, 0x8f, 0x61, 0x8b, 0x38, 0x8f, 0xe3, 0x38, 0x8c, 0xd7, 0x31, 0x46, 0x70, 0xfb, 0x30, + 0x9c, 0xcd, 0xc2, 0xe0, 0xe7, 0x14, 0x09, 0x85, 0xd1, 0xa5, 0x61, 0x5a, 0xc5, 0x30, 0x69, 0x80, + 0x5e, 0x18, 0xb0, 0xd2, 0xd4, 0x3b, 0xd0, 0x14, 0xd1, 0x25, 0xad, 0x95, 0x90, 0xf5, 0x67, 0x0d, + 0xb6, 0x2b, 0x56, 0xbc, 0xf3, 0x8e, 0x9f, 0x42, 0x57, 0xec, 0x21, 0x24, 0xd0, 0xbe, 0x9d, 0x03, + 0x13, 0x8f, 0x66, 0xa4, 0xe0, 0xed, 0x0a, 0x17, 0xfb, 0x0c, 0x7a, 0x89, 0x70, 0xb1, 0x5c, 0x56, + 0xdf, 0x31, 0x76, 0x3b, 0x07, 0x5b, 0xb4, 0x4c, 0x25, 0xd8, 0x55, 0x3e, 0xeb, 0x4f, 0x9a, 0x74, + 0xbb, 0x74, 0xe6, 0x3b, 0xeb, 0xfb, 0x00, 0x3a, 0x42, 0x2f, 0x12, 0x20, 0xd5, 0xbd, 0x55, 0xaa, + 0x2b, 0xe4, 0xaa, 0x3c, 0x64, 0xa2, 0x50, 0x42, 0xac, 0x11, 0xba, 0x9a, 0x8a, 0xae, 0x62, 0x51, + 0x85, 0xcb, 0xfa, 0xa3, 0x06, 0x9d, 0xc3, 0x0b, 0xee, 0x4a, 0xcd, 0x51, 0xc5, 0xc8, 0x49, 0x12, + 0x3e, 0xc9, 0x55, 0x14, 0x10, 0xbb, 0x0d, 0x8d, 0x34, 0x4c, 0x1d, 0x9f, 0x94, 0x6c, 0xd8, 0x02, + 0xc0, 0x44, 0x4c, 0x32, 0xd7, 0xe5, 0x49, 0x72, 0x9e, 0xf9, 0xa4, 0x65, 0xc3, 0x56, 0x30, 0x28, + 0xed, 0xdc, 0xf1, 0x7c, 0x3e, 0xa1, 0x03, 0x6d, 0xd8, 0x12, 0xc2, 0x8c, 0xbf, 0x76, 0xe2, 0xc0, + 0x0b, 0xa6, 0xfd, 0x06, 0x11, 0x72, 0x10, 0x57, 0x4c, 0x78, 0xea, 0x78, 0x7e, 0xbf, 0xb9, 0xa3, + 0xed, 0x76, 0x6d, 0x09, 0x59, 0x5d, 0x80, 0xa3, 0x6c, 0x16, 0x49, 0xff, 0xfe, 0x4a, 0x03, 0x38, + 0x0d, 0x9d, 0x89, 0x54, 0xfa, 0x43, 0xe8, 0x9d, 0x7b, 0x81, 0x97, 0x5c, 0xf0, 0xc9, 0xa3, 0x79, + 0xca, 0x13, 0xd2, 0xdd, 0xb0, 0xab, 0x48, 0x54, 0x96, 0xb4, 0x16, 0x2c, 0x3a, 0xb1, 0x28, 0x18, + 0x36, 0x80, 0x56, 0x14, 0x87, 0xd3, 0x98, 0x27, 0x89, 0x8c, 0xcb, 0x02, 0xc6, 0xb5, 0x33, 0x9e, + 0x3a, 0xa2, 0x88, 0xc8, 0xe8, 0x54, 0x30, 0xd6, 0x6f, 0x35, 0xe8, 0x8d, 0x2e, 0x9c, 0x78, 0xe2, + 0x05, 0xd3, 0x93, 0x38, 0xcc, 0x28, 0xcd, 0x53, 0x27, 0x9e, 0xf2, 0xbc, 0xa6, 0x49, 0x08, 0x33, + 0xea, 0xe8, 0xe8, 0x14, 0xf7, 0x37, 0x30, 0xa3, 0xf0, 0x5b, 0xe8, 0x1f, 0x27, 0xe9, 0x69, 0xe8, + 0x3a, 0xa9, 0x17, 0x06, 0x72, 0xfb, 0x2a, 0x92, 0xb2, 0x66, 0x1e, 0xb8, 0xe4, 0x4c, 0x83, 0xb2, + 0x86, 0x20, 0xd4, 0x3b, 0x0b, 0x24, 0xa5, 0x41, 0x94, 0x02, 0xb6, 0xfe, 0x60, 0x00, 0x8c, 0xe6, + 0x81, 0x2b, 0x1d, 0xb5, 0x03, 0x1d, 0x32, 0xf8, 0xf8, 0x8a, 0x07, 0x69, 0xee, 0x26, 0x15, 0x85, + 0xc2, 0x08, 0x7c, 0x11, 0xe5, 0x2e, 0x2a, 0x60, 0x2c, 0xab, 0x31, 0x77, 0x79, 0x90, 0x22, 0xd1, + 0x20, 0x62, 0x89, 0x60, 0x16, 0x74, 0x67, 0x4e, 0x92, 0xf2, 0xb8, 0xe2, 0xa4, 0x0a, 0x8e, 0xed, + 0x81, 0xa9, 0xc2, 0x27, 0xa9, 0x37, 0xa1, 0x00, 0x68, 0xdb, 0x0b, 0x78, 0x94, 0x47, 0x46, 0xe4, + 0xf2, 0x9a, 0x42, 0x9e, 0x8a, 0x43, 0x79, 0x2a, 0x4c, 0xf2, 0x36, 0x84, 0xbc, 0x9b, 0x78, 0x94, + 0x37, 0xf6, 0x43, 0xf7, 0xd2, 0x0b, 0xa6, 0x74, 0x00, 0x2d, 0x72, 0x55, 0x05, 0xc7, 0xbe, 0x00, + 0x33, 0x0b, 0x62, 0x9e, 0x84, 0xfe, 0x15, 0x9f, 0xd0, 0x39, 0x26, 0xfd, 0xb6, 0x92, 0xf3, 0xea, + 0x09, 0xdb, 0x0b, 0xac, 0xca, 0x09, 0x81, 0x48, 0x1e, 0x79, 0x42, 0x43, 0x80, 0x31, 0x29, 0xf2, + 0x62, 0x1e, 0xf1, 0x7e, 0x47, 0x44, 0x4f, 0x89, 0xb1, 0x7e, 0xaf, 0x41, 0x57, 0x2d, 0x43, 0x4a, + 0x81, 0xd4, 0x56, 0x14, 0x48, 0x5d, 0x2d, 0x90, 0xec, 0x93, 0xa2, 0xb0, 0x88, 0x4a, 0x41, 0xda, + 0x3e, 0x8b, 0x43, 0xcc, 0x43, 0x9b, 0x08, 0x45, 0xad, 0x79, 0x00, 0x9d, 0x18, 0x7b, 0x69, 0x51, + 0xd1, 0x8a, 0xca, 0x62, 0x97, 0x68, 0x5b, 0xe5, 0xb1, 0xfe, 0xa6, 0x43, 0x47, 0x21, 0x2e, 0x9c, + 0xb4, 0xf6, 0x2d, 0x4f, 0x5a, 0x5f, 0x71, 0xd2, 0x3b, 0xb9, 0x4a, 0xd9, 0xf8, 0xc8, 0x8b, 0x65, + 0xf0, 0xab, 0xa8, 0x82, 0xa3, 0x12, 0x5a, 0x2a, 0x8a, 0xed, 0xc2, 0x2d, 0x05, 0x54, 0x02, 0xeb, + 0x26, 0x9a, 0xed, 0x03, 0x23, 0xd4, 0xa1, 0x93, 0xba, 0x17, 0x2f, 0xa3, 0x33, 0xd2, 0x86, 0xa2, + 0xab, 0x65, 0x2f, 0xa1, 0xb0, 0xef, 0x42, 0x23, 0x49, 0x9d, 0x29, 0xa7, 0xc0, 0xda, 0x3c, 0x68, + 0x53, 0x20, 0x20, 0xc2, 0x16, 0x78, 0xc5, 0xf9, 0xad, 0xb7, 0x38, 0xdf, 0xfa, 0x8f, 0x0e, 0xbd, + 0x4a, 0xe3, 0x58, 0xd6, 0x60, 0xcb, 0x1d, 0xf5, 0x15, 0x3b, 0xee, 0x40, 0x3d, 0x0b, 0x3c, 0x71, + 0xd8, 0x9b, 0x07, 0x5d, 0xa4, 0xbf, 0x0c, 0xbc, 0x14, 0x63, 0xc9, 0x26, 0x8a, 0xa2, 0x53, 0xfd, + 0x6d, 0x01, 0xf1, 0x3d, 0xd8, 0x2e, 0x03, 0xf9, 0xe8, 0xe8, 0xf4, 0x34, 0x74, 0x2f, 0x9f, 0x1c, + 0x49, 0xef, 0x2d, 0x23, 0x31, 0x26, 0xda, 0x15, 0x25, 0xe4, 0xe3, 0x9a, 0x68, 0x58, 0x1f, 0x43, + 0xc3, 0xc5, 0x36, 0x42, 0x5e, 0x92, 0x01, 0xa5, 0xf4, 0x95, 0xc7, 0x35, 0x5b, 0xd0, 0xd9, 0x87, + 0x50, 0x9f, 0x64, 0xb3, 0x48, 0xfa, 0x6a, 0x13, 0xf9, 0xca, 0xc2, 0xfe, 0xb8, 0x66, 0x13, 0x15, + 0xb9, 0xfc, 0xd0, 0x99, 0xf4, 0xdb, 0x25, 0x57, 0x59, 0xef, 0x91, 0x0b, 0xa9, 0xc8, 0x85, 0x19, + 0x46, 0xd9, 0x26, 0xb9, 0xca, 0x62, 0x87, 0x5c, 0x48, 0x7d, 0xd4, 0x82, 0x66, 0x22, 0x02, 0xf9, + 0x47, 0xb0, 0x55, 0xf1, 0xfe, 0xa9, 0x97, 0x90, 0xab, 0x04, 0xb9, 0xaf, 0xad, 0xea, 0xee, 0xf9, + 0xfa, 0x21, 0x00, 0xd9, 0x24, 0x1a, 0xae, 0xec, 0xda, 0x5a, 0x39, 0x98, 0x7d, 0x00, 0x6d, 0xb4, + 0x65, 0x0d, 0x19, 0x8d, 0x58, 0x45, 0x8e, 0xa0, 0x4b, 0xda, 0x3f, 0x3f, 0x5d, 0xc1, 0xc1, 0x0e, + 0xe0, 0xb6, 0x68, 0xa0, 0xc5, 0x30, 0xeb, 0x51, 0xbb, 0x10, 0x89, 0xb5, 0x94, 0x86, 0x05, 0x9d, + 0xa3, 0xb8, 0xd1, 0xf3, 0xd3, 0xbc, 0xab, 0xe5, 0xb0, 0xf5, 0x03, 0x68, 0xe3, 0x8e, 0x62, 0xbb, + 0x5d, 0x68, 0x12, 0x21, 0xf7, 0x83, 0x59, 0xb8, 0x53, 0x2a, 0x64, 0x4b, 0xba, 0xf5, 0x6b, 0xad, + 0x32, 0x9d, 0xbc, 0x73, 0xb5, 0xda, 0x59, 0x1c, 0x6e, 0xda, 0xd5, 0x59, 0x66, 0x1f, 0x80, 0x0a, + 0x4e, 0x3e, 0xc9, 0x14, 0xc7, 0x5b, 0x62, 0x6d, 0x85, 0x03, 0x0f, 0xa6, 0x84, 0x96, 0xb8, 0xf6, + 0x77, 0x3a, 0x74, 0xd5, 0x21, 0xe8, 0xff, 0x95, 0x76, 0x4c, 0x99, 0xd5, 0xf3, 0xcc, 0xf8, 0x28, + 0xcf, 0x8c, 0x46, 0x69, 0x46, 0x19, 0x45, 0x65, 0x62, 0xdc, 0x97, 0x89, 0xd1, 0x24, 0xb6, 0x5e, + 0x9e, 0x18, 0x39, 0x97, 0xc8, 0x8b, 0xfb, 0x32, 0x2f, 0x36, 0x4a, 0xa6, 0x22, 0xa4, 0x8a, 0xb4, + 0xb8, 0x2f, 0xd3, 0xa2, 0x55, 0x32, 0x15, 0xc7, 0x5c, 0x64, 0xc5, 0x06, 0x34, 0xe8, 0x38, 0xad, + 0xcf, 0xc1, 0x54, 0x5d, 0x43, 0x39, 0xf1, 0x91, 0x24, 0x56, 0x42, 0x41, 0x1d, 0x22, 0xe5, 0xda, + 0x57, 0xd0, 0xab, 0x14, 0x15, 0xec, 0x74, 0x5e, 0x72, 0xe8, 0x04, 0x2e, 0xf7, 0x8b, 0x11, 0x52, + 0xc1, 0x28, 0x41, 0xa6, 0x97, 0x92, 0xa5, 0x88, 0x4a, 0x90, 0x29, 0x83, 0xa0, 0x51, 0x19, 0x04, + 0xff, 0xa1, 0x41, 0x57, 0x5d, 0x80, 0xb3, 0xe4, 0x71, 0x1c, 0x1f, 0x86, 0x13, 0x71, 0x9a, 0x0d, + 0x3b, 0x07, 0x31, 0xf4, 0xf1, 0xd3, 0x77, 0x92, 0x44, 0x46, 0x60, 0x01, 0x4b, 0xda, 0xc8, 0x0d, + 0xa3, 0xfc, 0x12, 0x52, 0xc0, 0x92, 0x76, 0xca, 0xaf, 0xb8, 0x2f, 0x5b, 0x4d, 0x01, 0xe3, 0x6e, + 0x67, 0x3c, 0x49, 0x30, 0x4c, 0x44, 0x85, 0xcc, 0x41, 0x5c, 0x65, 0x3b, 0xd7, 0x87, 0x4e, 0x96, + 0x70, 0x39, 0xab, 0x14, 0x30, 0xba, 0x05, 0x2f, 0x4b, 0x4e, 0x1c, 0x66, 0x41, 0x3e, 0xa1, 0x28, + 0x18, 0x6b, 0x00, 0xfd, 0xd1, 0xb5, 0x97, 0xba, 0x17, 0x14, 0xc5, 0xa2, 0xf1, 0xc8, 0x2b, 0x98, + 0x75, 0x00, 0xdb, 0xf2, 0x96, 0x59, 0xb9, 0x03, 0x7f, 0x47, 0xb9, 0x62, 0x76, 0x8a, 0xd4, 0x10, + 0x77, 0x4c, 0x2b, 0x83, 0xdb, 0xd5, 0x35, 0xf2, 0x02, 0xb2, 0x6e, 0xd1, 0xff, 0xe0, 0x62, 0x7a, + 0x0d, 0x5b, 0xcf, 0xb2, 0x78, 0x5a, 0x55, 0x74, 0x00, 0x2d, 0x2f, 0x70, 0xdc, 0xd4, 0xbb, 0xe2, + 0x32, 0x20, 0x0a, 0x98, 0x6e, 0xac, 0x9e, 0xbc, 0x55, 0x1b, 0x36, 0x7d, 0x23, 0xff, 0xb9, 0xe7, + 0x73, 0x4a, 0x4f, 0x79, 0x32, 0x39, 0x4c, 0xaa, 0x88, 0x21, 0x41, 0x5e, 0x10, 0x05, 0x84, 0xfe, + 0xa3, 0xeb, 0x96, 0xb8, 0x91, 0x1e, 0x86, 0xc1, 0xb9, 0x37, 0xcd, 0xfd, 0x77, 0x0d, 0x77, 0x97, + 0xd0, 0xde, 0x72, 0x23, 0x2b, 0x6d, 0xd6, 0x97, 0xd9, 0x6c, 0x94, 0x35, 0x59, 0x79, 0xbc, 0xa8, + 0x57, 0x1f, 0x2f, 0xfe, 0xa2, 0xc1, 0x40, 0x9e, 0x82, 0xd8, 0x7b, 0xe4, 0x5e, 0xf0, 0x99, 0x93, + 0xfb, 0xe5, 0x9e, 0x72, 0x16, 0x54, 0x4b, 0x04, 0xb9, 0x7c, 0x6f, 0xa0, 0xbb, 0xbc, 0x5e, 0xde, + 0xe5, 0x57, 0x1e, 0xc4, 0x00, 0x5a, 0x13, 0x27, 0x75, 0xc6, 0x4e, 0xc2, 0xf3, 0x78, 0xcd, 0x61, + 0xba, 0xb7, 0x39, 0x63, 0x3f, 0x8f, 0x56, 0x01, 0x90, 0x24, 0xda, 0x4d, 0x46, 0xaa, 0x84, 0xac, + 0x14, 0x7a, 0x5f, 0x3f, 0x90, 0x89, 0x7e, 0xc6, 0x53, 0x67, 0xed, 0x43, 0xc6, 0x5b, 0xeb, 0x65, + 0x5e, 0x64, 0x0d, 0xa5, 0xc8, 0xe6, 0x76, 0xd5, 0x29, 0xa9, 0xc5, 0x1b, 0xc5, 0xa7, 0x45, 0xb4, + 0x7e, 0xfd, 0x00, 0x77, 0x5d, 0xe9, 0x21, 0x41, 0x96, 0x31, 0xfe, 0x57, 0xad, 0x78, 0xc7, 0xc9, + 0x97, 0xbd, 0xf3, 0x35, 0xfb, 0x33, 0xa8, 0xe3, 0x25, 0xae, 0x6f, 0x50, 0x31, 0xba, 0x8f, 0x7b, + 0x2c, 0x15, 0xb9, 0x8f, 0xc0, 0x71, 0x90, 0xc6, 0x73, 0x9b, 0x16, 0x0c, 0x7e, 0x0a, 0xed, 0x02, + 0x85, 0x72, 0x2f, 0xf9, 0x3c, 0xef, 0x37, 0x97, 0x7c, 0x8e, 0xd3, 0xd0, 0x95, 0xe3, 0x67, 0xc2, + 0x35, 0x72, 0xa4, 0xa8, 0x38, 0xd6, 0x16, 0xf4, 0xcf, 0xf5, 0x1f, 0x6a, 0xd6, 0x2f, 0xa1, 0xff, + 0xd8, 0x09, 0x26, 0xbe, 0x8c, 0x92, 0xca, 0xfb, 0xcb, 0x42, 0xc2, 0x12, 0x75, 0x4d, 0x8c, 0xdc, + 0x83, 0xf6, 0xb8, 0x78, 0xfa, 0x12, 0x8e, 0x2f, 0x11, 0xb8, 0x22, 0x79, 0xe5, 0x27, 0xf2, 0x02, + 0x49, 0xdf, 0x7b, 0xbf, 0x80, 0xa6, 0x38, 0x54, 0xd6, 0x83, 0xf6, 0x93, 0xe0, 0xca, 0xf1, 0xbd, + 0xc9, 0xd3, 0xc8, 0xac, 0xb1, 0x16, 0xd4, 0x47, 0x69, 0x18, 0x99, 0x1a, 0x6b, 0x43, 0xe3, 0x19, + 0xd6, 0x31, 0x53, 0x67, 0x00, 0x4d, 0x2c, 0xf5, 0x33, 0x6e, 0x1a, 0x88, 0xa6, 0x87, 0x3f, 0xb3, + 0x8e, 0x68, 0xf1, 0x24, 0x65, 0x36, 0xd8, 0x26, 0xc0, 0x4f, 0xb2, 0x34, 0x94, 0x6c, 0xcd, 0xbd, + 0x57, 0xc4, 0x36, 0xc5, 0xe4, 0xe9, 0x4a, 0xf9, 0x04, 0x9b, 0x35, 0xb6, 0x01, 0xc6, 0x57, 0xfc, + 0xda, 0xd4, 0x58, 0x07, 0x36, 0xec, 0x2c, 0x08, 0xbc, 0x60, 0x2a, 0xf6, 0xa0, 0xed, 0x26, 0xa6, + 0x81, 0x04, 0x54, 0x22, 0xe2, 0x13, 0xb3, 0xce, 0xba, 0xd0, 0xfa, 0x52, 0x5e, 0xe9, 0xcd, 0x06, + 0x92, 0x90, 0x0d, 0xd7, 0x34, 0x91, 0x44, 0x1b, 0x22, 0xb4, 0xb1, 0xf7, 0x14, 0x5a, 0x79, 0x67, + 0x66, 0xb7, 0xa0, 0x23, 0x77, 0x45, 0x94, 0x59, 0x43, 0xb5, 0xa9, 0xff, 0x9a, 0x1a, 0x9a, 0x88, + 0x3d, 0xd6, 0xd4, 0xf1, 0x0b, 0x1b, 0xa9, 0x69, 0x90, 0xd9, 0xf3, 0xc0, 0x35, 0xeb, 0xc8, 0x48, + 0x95, 0xcc, 0x9c, 0xec, 0x9d, 0xc1, 0x86, 0x2c, 0x95, 0x8c, 0xc1, 0xa6, 0x94, 0x27, 0x31, 0x66, + 0x0d, 0x3d, 0x87, 0x5a, 0x0a, 0x6e, 0x0d, 0x3d, 0x40, 0x06, 0x08, 0x58, 0x47, 0x15, 0x84, 0x37, + 0x04, 0xc2, 0x40, 0xfd, 0xf2, 0x6c, 0x67, 0xdb, 0x70, 0x2b, 0xf7, 0x8a, 0x44, 0x09, 0x81, 0x27, + 0x3c, 0x15, 0x08, 0x53, 0x23, 0xf9, 0x05, 0xa8, 0xa3, 0x23, 0x6d, 0x3e, 0x0b, 0xaf, 0xb8, 0xc4, + 0x18, 0x7b, 0x0f, 0xa1, 0x95, 0x27, 0x87, 0x22, 0x30, 0x47, 0x15, 0x02, 0x05, 0xc2, 0xd4, 0x4a, + 0x09, 0x12, 0xa3, 0xef, 0x3d, 0xa4, 0x36, 0x8a, 0xb1, 0x45, 0x1e, 0xb8, 0xf4, 0x70, 0x15, 0x1e, + 0x0b, 0x8f, 0x7c, 0xc7, 0xe5, 0xa6, 0x26, 0x8e, 0xfe, 0x8a, 0xc7, 0xa9, 0xa9, 0x2b, 0x4e, 0x90, + 0x8b, 0x4c, 0xe3, 0xe0, 0xdf, 0x1b, 0xd0, 0x14, 0x21, 0xcc, 0x0e, 0xa1, 0xab, 0x3e, 0x09, 0xb3, + 0xf7, 0x65, 0x6d, 0xb8, 0xf9, 0x48, 0x3c, 0xe8, 0xd3, 0x34, 0xb4, 0xe4, 0x35, 0xd1, 0xaa, 0xb1, + 0x27, 0xb0, 0x59, 0x7d, 0x5e, 0x65, 0x77, 0x95, 0x04, 0xbd, 0x21, 0x68, 0xb0, 0x8c, 0x54, 0x88, + 0x3a, 0x86, 0x5e, 0xe5, 0xc5, 0x94, 0xd1, 0xbe, 0xcb, 0x1e, 0x51, 0xd7, 0x6a, 0xf4, 0x10, 0x3a, + 0xca, 0x33, 0x24, 0xbb, 0x83, 0xac, 0x8b, 0xaf, 0xab, 0x83, 0xf7, 0x17, 0xf0, 0x85, 0x84, 0x2f, + 0x00, 0xca, 0x77, 0x41, 0xf6, 0x5e, 0xc1, 0xa8, 0x26, 0xfd, 0xe0, 0xce, 0x4d, 0x74, 0xb1, 0xfc, + 0x0c, 0xb6, 0x16, 0xe6, 0x04, 0x76, 0x8f, 0x9c, 0xbb, 0x62, 0x7c, 0x58, 0x6b, 0xcf, 0x21, 0x74, + 0xd5, 0x31, 0x41, 0x1c, 0xd3, 0x92, 0x61, 0x43, 0x08, 0x59, 0x36, 0x51, 0x58, 0x35, 0xf6, 0x63, + 0x80, 0xb2, 0xe9, 0x0b, 0x93, 0x16, 0x86, 0x80, 0xb5, 0x5a, 0x9c, 0xc0, 0x96, 0xf2, 0x53, 0x40, + 0x34, 0x68, 0xe1, 0xdb, 0xc5, 0x7f, 0x05, 0x6b, 0x05, 0xd9, 0xf2, 0x05, 0x5b, 0xed, 0xf4, 0xc2, + 0x3b, 0xab, 0x86, 0x83, 0xc1, 0x07, 0x2b, 0xa8, 0xaa, 0x8b, 0xd4, 0x3f, 0x10, 0xc2, 0x45, 0x4b, + 0xfe, 0x49, 0xac, 0x55, 0xec, 0x67, 0xd0, 0xcb, 0x43, 0x93, 0x12, 0x96, 0x0d, 0x15, 0x7f, 0x2e, + 0x99, 0x0d, 0xd6, 0x0a, 0xfb, 0xb2, 0x10, 0x26, 0x72, 0x97, 0xf5, 0x97, 0xb4, 0x2d, 0x21, 0xe6, + 0xee, 0xca, 0x86, 0x46, 0x6e, 0xef, 0x88, 0xb6, 0x23, 0x62, 0x91, 0xfc, 0xb4, 0xaa, 0x0f, 0xad, + 0x53, 0xe8, 0x51, 0xff, 0xef, 0xaf, 0x87, 0xda, 0x37, 0xaf, 0x87, 0xda, 0xbf, 0x5e, 0x0f, 0xb5, + 0xdf, 0xbc, 0x19, 0xd6, 0xbe, 0x79, 0x33, 0xac, 0xfd, 0xf3, 0xcd, 0xb0, 0x36, 0x6e, 0xd2, 0x4f, + 0xa2, 0xef, 0xff, 0x37, 0x00, 0x00, 0xff, 0xff, 0xff, 0x04, 0x0c, 0xd2, 0x36, 0x1a, 0x00, 0x00, } // Reference imports to suppress errors if they are not otherwise used. diff --git a/dm/proto/dmworker.proto b/dm/proto/dmworker.proto index ac8601bd3a..3788fa3dbf 100644 --- a/dm/proto/dmworker.proto +++ b/dm/proto/dmworker.proto @@ -434,7 +434,8 @@ message OperateV1MetaResponse { enum ErrorOp { Skip = 0; // skip the error event Replace = 1; // replace the error event with a specified SQL - InvalidErrorOp = 2; + Revert = 2; // remove the error operator + InvalidErrorOp = 3; } message HandleWorkerErrorRequest { diff --git a/errors.toml b/errors.toml index 5f883d1498..19b0566f8a 100644 --- a/errors.toml +++ b/errors.toml @@ -1828,6 +1828,12 @@ description = "" workaround = "" tags = ["internal", "high"] +[error.DM-sync-unit-36065] +message = "error operator not exist, position: %s" +description = "" +workaround = "" +tags = ["internal", "low"] + [error.DM-dm-master-38001] message = "nil request not valid" description = "" diff --git a/pkg/terror/error_list.go b/pkg/terror/error_list.go index ece182aed6..df9dc66a39 100644 --- a/pkg/terror/error_list.go +++ b/pkg/terror/error_list.go @@ -386,6 +386,7 @@ const ( codeSyncerShardDDLConflict codeSyncerFailpoint codeSyncerReplaceEvent + codeSyncerOperatorNotExist ) // DM-master error code @@ -943,6 +944,7 @@ var ( ErrSyncerShardDDLConflict = New(codeSyncerShardDDLConflict, ClassSyncUnit, ScopeInternal, LevelHigh, "fail to handle shard ddl %v in optimistic mode, because schema conflict detected", "Please use show-ddl-locks command for more details.") ErrSyncerFailpoint = New(codeSyncerFailpoint, ClassSyncUnit, ScopeInternal, LevelLow, "failpoint specified error", "") ErrSyncerReplaceEvent = New(codeSyncerReplaceEvent, ClassSyncUnit, ScopeInternal, LevelHigh, "", "") + ErrSyncerOperatorNotExist = New(codeSyncerOperatorNotExist, ClassSyncUnit, ScopeInternal, LevelLow, "error operator not exist, position: %s", "") // DM-master error ErrMasterSQLOpNilRequest = New(codeMasterSQLOpNilRequest, ClassDMMaster, ScopeInternal, LevelMedium, "nil request not valid", "") diff --git a/syncer/err-operator/operator.go b/syncer/err-operator/operator.go index 01eff7ecba..8e7ade76ba 100644 --- a/syncer/err-operator/operator.go +++ b/syncer/err-operator/operator.go @@ -74,6 +74,14 @@ func (h *Holder) Set(pos string, op pb.ErrorOp, events []*replication.BinlogEven h.mu.Lock() defer h.mu.Unlock() + if op == pb.ErrorOp_Revert { + if _, ok := h.operators[pos]; !ok { + return terror.ErrSyncerOperatorNotExist.Generate(pos) + } + delete(h.operators, pos) + return nil + } + oper := newOperator(op, events) pre, ok := h.operators[pos] if ok { @@ -104,7 +112,7 @@ func (h *Holder) GetEvent(startLocation *binlog.Location) (*replication.BinlogEv } if len(operator.events) <= startLocation.Suffix { - return nil, terror.ErrSyncerReplaceEvent.New("replace events out of index") + return nil, terror.ErrSyncerReplaceEvent.Generatef("replace events out of range, index: %d, total: %d", startLocation.Suffix, len(operator.events)) } e := operator.events[startLocation.Suffix] diff --git a/syncer/handle_error.go b/syncer/handle_error.go index ece383e146..8b3b869142 100644 --- a/syncer/handle_error.go +++ b/syncer/handle_error.go @@ -48,7 +48,10 @@ func (s *Syncer) HandleError(ctx context.Context, req *pb.HandleWorkerErrorReque } } - s.errOperatorHolder.Set(pos, req.Op, events) + err = s.errOperatorHolder.Set(pos, req.Op, events) + if err != nil { + return "", err + } return "", nil } diff --git a/tests/handle_error/run.sh b/tests/handle_error/run.sh index 45e1b300df..76d10c0be7 100644 --- a/tests/handle_error/run.sh +++ b/tests/handle_error/run.sh @@ -348,6 +348,14 @@ function DM_REPLACE_ERROR_MULTIPLE_CASE() { "query-status test" \ "Duplicate column name 'a'" 2 + # test handle-error revert + run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "handle-error test revert" \ + "\"result\": true" 3 + run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "query-status test" \ + "unsupported add column 'a' constraint UNIQUE KEY" 2 + # now we only replace with ddl2 run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" \ "handle-error test replace \"alter table ${db}.${tb1} add column b int;\"" \ @@ -368,11 +376,11 @@ function DM_REPLACE_ERROR_MULTIPLE() { function run() { init_cluster init_database - # DM_SKIP_ERROR - # DM_SKIP_ERROR_SHARDING - # DM_REPLACE_ERROR + DM_SKIP_ERROR + DM_SKIP_ERROR_SHARDING + DM_REPLACE_ERROR DM_REPLACE_ERROR_SHARDING - # DM_REPLACE_ERROR_MULTIPLE + DM_REPLACE_ERROR_MULTIPLE } cleanup_data $db From b1c7d8c944bff03f02706be5c8c2815b2654b8a3 Mon Sep 17 00:00:00 2001 From: gmhdbjd Date: Thu, 6 Aug 2020 14:00:58 +0800 Subject: [PATCH 17/31] remove outdated operator --- _utils/terror_gen/errors_release.txt | 1 + dm/ctl/master/handle_error.go | 2 +- errors.toml | 6 +++++ pkg/binlog/position.go | 25 +++++++++++++++++++ pkg/terror/error_list.go | 8 +++--- syncer/err-operator/operator.go | 15 +++++++++++ syncer/handle_error.go | 6 +++++ syncer/syncer.go | 2 +- .../dmctl_advance/check_list/handle_error.sh | 2 +- 9 files changed, 61 insertions(+), 6 deletions(-) diff --git a/_utils/terror_gen/errors_release.txt b/_utils/terror_gen/errors_release.txt index b84c017628..98bfbe90bc 100644 --- a/_utils/terror_gen/errors_release.txt +++ b/_utils/terror_gen/errors_release.txt @@ -160,6 +160,7 @@ ErrConfigMissingForBound,[code=20036:class=config:scope=internal:level=high], "M ErrBinlogExtractPosition,[code=22001:class=binlog-op:scope=internal:level=high] ErrBinlogInvalidFilename,[code=22002:class=binlog-op:scope=internal:level=high], "Message: invalid binlog filename" ErrBinlogParsePosFromStr,[code=22003:class=binlog-op:scope=internal:level=high] +ErrBinlogParsePosFromPosStr,[code=22004:class=binlog-op:scope=internal:level=high] ErrCheckpointInvalidTaskMode,[code=24001:class=checkpoint:scope=internal:level=medium], "Message: invalid task mode: %s" ErrCheckpointSaveInvalidPos,[code=24002:class=checkpoint:scope=internal:level=high], "Message: save point %s is older than current location %s" ErrCheckpointInvalidTableFile,[code=24003:class=checkpoint:scope=internal:level=medium], "Message: invalid db table sql file - %s" diff --git a/dm/ctl/master/handle_error.go b/dm/ctl/master/handle_error.go index fd0097d6f3..f88c5c1f6c 100644 --- a/dm/ctl/master/handle_error.go +++ b/dm/ctl/master/handle_error.go @@ -28,7 +28,7 @@ import ( // NewHandleErrorCmd creates a HandleError command func NewHandleErrorCmd() *cobra.Command { cmd := &cobra.Command{ - Use: "handle-error [-s source ...] [-b binlog-pos] [replace-sql1;replace-sql2;]", + Use: "handle-error [-s source ...] [-b binlog-pos] [replace-sql1;replace-sql2;]", Short: "skip/replace/revert the current error event or a specific binlog position (binlog-pos) event", RunE: handleErrorFunc, } diff --git a/errors.toml b/errors.toml index 6a9d1de7ae..c6c05ca27e 100644 --- a/errors.toml +++ b/errors.toml @@ -970,6 +970,12 @@ description = "" workaround = "" tags = ["internal", "high"] +[error.DM-binlog-op-22004] +message = "" +description = "" +workaround = "" +tags = ["internal", "high"] + [error.DM-checkpoint-24001] message = "invalid task mode: %s" description = "" diff --git a/pkg/binlog/position.go b/pkg/binlog/position.go index 7221f95999..bc0ae73172 100644 --- a/pkg/binlog/position.go +++ b/pkg/binlog/position.go @@ -59,6 +59,31 @@ func PositionFromStr(s string) (gmysql.Position, error) { }, nil } +func trimBrackets(s string) string { + if len(s) > 2 && s[0] == '(' && s[len(s)-1] == ')' { + return s[1 : len(s)-1] + } + return s +} + +// PositionFromPosStr constructs a mysql.Position from a string representation like `(mysql-bin.000001, 2345)` +func PositionFromPosStr(str string) (gmysql.Position, error) { + s := trimBrackets(str) + parsed := strings.Split(s, ", ") + if len(parsed) != 2 { + return gmysql.Position{}, terror.ErrBinlogParsePosFromStr.Generatef("invalid binlog pos, position string %s", str) + } + pos, err := strconv.ParseUint(parsed[1], 10, 32) + if err != nil { + return gmysql.Position{}, terror.ErrBinlogParsePosFromStr.Generatef("the pos should be digital, position string %s", str) + } + + return gmysql.Position{ + Name: parsed[0], + Pos: uint32(pos), + }, nil +} + // RealMySQLPos parses a relay position and returns a mysql position and whether error occurs // if parsed successfully and `UUIDSuffix` exists, sets position Name to // `originalPos.NamePrefix + binlogFilenameSep + originalPos.NameSuffix`. diff --git a/pkg/terror/error_list.go b/pkg/terror/error_list.go index d5cd2c201a..e674fbc18d 100644 --- a/pkg/terror/error_list.go +++ b/pkg/terror/error_list.go @@ -215,6 +215,7 @@ const ( codeBinlogExtractPosition ErrCode = iota + 22001 codeBinlogInvalidFilename codeBinlogParsePosFromStr + codeBinlogParsePosFromPosStr ) // Checkpoint error code @@ -784,9 +785,10 @@ var ( ErrConfigMissingForBound = New(codeConfigMissingForBound, ClassConfig, ScopeInternal, LevelHigh, "source bound %s doesn't have related source config in etcd", "") // Binlog operation error - ErrBinlogExtractPosition = New(codeBinlogExtractPosition, ClassBinlogOp, ScopeInternal, LevelHigh, "", "") - ErrBinlogInvalidFilename = New(codeBinlogInvalidFilename, ClassBinlogOp, ScopeInternal, LevelHigh, "invalid binlog filename", "") - ErrBinlogParsePosFromStr = New(codeBinlogParsePosFromStr, ClassBinlogOp, ScopeInternal, LevelHigh, "", "") + ErrBinlogExtractPosition = New(codeBinlogExtractPosition, ClassBinlogOp, ScopeInternal, LevelHigh, "", "") + ErrBinlogInvalidFilename = New(codeBinlogInvalidFilename, ClassBinlogOp, ScopeInternal, LevelHigh, "invalid binlog filename", "") + ErrBinlogParsePosFromStr = New(codeBinlogParsePosFromStr, ClassBinlogOp, ScopeInternal, LevelHigh, "", "") + ErrBinlogParsePosFromPosStr = New(codeBinlogParsePosFromPosStr, ClassBinlogOp, ScopeInternal, LevelHigh, "", "") // Checkpoint error ErrCheckpointInvalidTaskMode = New(codeCheckpointInvalidTaskMode, ClassCheckpoint, ScopeInternal, LevelMedium, "invalid task mode: %s", "") diff --git a/syncer/err-operator/operator.go b/syncer/err-operator/operator.go index 8e7ade76ba..5be8d86c27 100644 --- a/syncer/err-operator/operator.go +++ b/syncer/err-operator/operator.go @@ -166,3 +166,18 @@ func (h *Holder) Apply(startLocation, endLocation *binlog.Location) (bool, pb.Er return true, operator.op } + +// RemoveOutdated remove outdated operator +func (h *Holder) RemoveOutdated(flushLocation binlog.Location) error { + for pos := range h.operators { + position, err := binlog.PositionFromPosStr(pos) + if err != nil { + // should not happen + return err + } + if binlog.ComparePosition(position, flushLocation.Position) == -1 { + delete(h.operators, pos) + } + } + return nil +} diff --git a/syncer/handle_error.go b/syncer/handle_error.go index 8b3b869142..f5c378be26 100644 --- a/syncer/handle_error.go +++ b/syncer/handle_error.go @@ -53,6 +53,12 @@ func (s *Syncer) HandleError(ctx context.Context, req *pb.HandleWorkerErrorReque return "", err } + // remove outdated operators when add operator + err = s.errOperatorHolder.RemoveOutdated(s.checkpoint.FlushedGlobalPoint()) + if err != nil { + return "", err + } + return "", nil } diff --git a/syncer/syncer.go b/syncer/syncer.go index 2420ad5f1f..9d16e757b8 100644 --- a/syncer/syncer.go +++ b/syncer/syncer.go @@ -2657,7 +2657,7 @@ func (s *Syncer) handleEventError(err error, startLocation, endLocation *binlog. } s.setErrLocation(startLocation, endLocation) - return terror.Annotatef(err, "[startLocation: %s, endLocation: %s]", startLocation, endLocation) + return terror.Annotatef(err, "startLocation: [%s], endLocation: [%s]", startLocation, endLocation) } // getEvent gets an event from streamerController or errOperatorHolder diff --git a/tests/dmctl_advance/check_list/handle_error.sh b/tests/dmctl_advance/check_list/handle_error.sh index 6428499c33..ed902e60f2 100644 --- a/tests/dmctl_advance/check_list/handle_error.sh +++ b/tests/dmctl_advance/check_list/handle_error.sh @@ -3,7 +3,7 @@ function handle_error_wrong_arg() { run_dm_ctl $WORK_DIR "127.0.0.1:$MASTER_PORT" \ "handle-error" \ - "handle-error \[-s source ...\] \[-b binlog-pos\] \[replace-sql1;replace-sql2;\] \[flags\]" 1 + "handle-error \[-s source ...\] \[-b binlog-pos\] \[replace-sql1;replace-sql2;\] \[flags\]" 1 } function handle_error_invalid_binlogpos() { From d31fb71cc36a5d8b71854835a9a806126d2b8b03 Mon Sep 17 00:00:00 2001 From: gmhdbjd Date: Thu, 6 Aug 2020 17:00:30 +0800 Subject: [PATCH 18/31] add operator test --- syncer/err-operator/operator.go | 8 +- syncer/err-operator/operator_test.go | 145 +++++++++++++++++++++++++++ syncer/handle_error.go | 7 ++ 3 files changed, 158 insertions(+), 2 deletions(-) create mode 100644 syncer/err-operator/operator_test.go diff --git a/syncer/err-operator/operator.go b/syncer/err-operator/operator.go index 5be8d86c27..531d8403a5 100644 --- a/syncer/err-operator/operator.go +++ b/syncer/err-operator/operator.go @@ -149,7 +149,9 @@ func (h *Holder) Apply(startLocation, endLocation *binlog.Location) (bool, pb.Er for _, ev := range operator.events { ev.Header.LogPos = startLocation.Position.Pos if e, ok := ev.Event.(*replication.QueryEvent); ok { - e.GSet = startLocation.GTIDSet.Origin() + if startLocation.GTIDSet != nil { + e.GSet = startLocation.GTIDSet.Origin() + } } } @@ -158,7 +160,9 @@ func (h *Holder) Apply(startLocation, endLocation *binlog.Location) (bool, pb.Er operator.events[len(operator.events)-1].Header.LogPos = endLocation.Position.Pos e := operator.events[len(operator.events)-1] if e, ok := e.Event.(*replication.QueryEvent); ok { - e.GSet = endLocation.GTIDSet.Origin() + if endLocation.GTIDSet != nil { + e.GSet = endLocation.GTIDSet.Origin() + } } } diff --git a/syncer/err-operator/operator_test.go b/syncer/err-operator/operator_test.go new file mode 100644 index 0000000000..a0e5eb03f4 --- /dev/null +++ b/syncer/err-operator/operator_test.go @@ -0,0 +1,145 @@ +// Copyright 2020 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package operator + +import ( + "testing" + + . "github.com/pingcap/check" + "github.com/siddontang/go-mysql/mysql" + "github.com/siddontang/go-mysql/replication" + + "github.com/pingcap/dm/dm/pb" + "github.com/pingcap/dm/pkg/binlog" +) + +var _ = Suite(&testOperatorSuite{}) + +func TestSuite(t *testing.T) { + TestingT(t) +} + +type testOperatorSuite struct { +} + +func (o *testOperatorSuite) TestOperator(c *C) { + h := NewHolder() + + startLocation := binlog.Location{ + Position: mysql.Position{ + Name: "mysql-bin.000001", + Pos: 233, + }} + endLocation := binlog.Location{ + Position: mysql.Position{ + Name: "mysql-bin.000001", + Pos: 250, + }, + } + nextLocation := binlog.Location{ + Position: mysql.Position{ + Name: "mysql-bin.000001", + Pos: 300, + }, + } + + sql1 := "alter table tb add column a int" + event1 := &replication.BinlogEvent{ + Header: &replication.EventHeader{ + EventType: replication.QUERY_EVENT, + }, + Event: &replication.QueryEvent{ + Schema: []byte("db"), + Query: []byte(sql1), + }, + } + sql2 := "alter table tb add column b int" + event2 := &replication.BinlogEvent{ + Header: &replication.EventHeader{ + EventType: replication.QUERY_EVENT, + }, + Event: &replication.QueryEvent{ + Schema: []byte("db"), + Query: []byte(sql2), + }, + } + + // revert not exist operator + err := h.Set(startLocation.Position.String(), pb.ErrorOp_Revert, nil) + c.Assert(err, NotNil) + + // skip event + err = h.Set(startLocation.Position.String(), pb.ErrorOp_Skip, nil) + c.Assert(err, IsNil) + apply, op := h.Apply(&startLocation, &endLocation) + c.Assert(apply, IsTrue) + c.Assert(op, Equals, pb.ErrorOp_Skip) + + // overwrite operator + err = h.Set(startLocation.Position.String(), pb.ErrorOp_Replace, []*replication.BinlogEvent{event1, event2}) + apply, op = h.Apply(&startLocation, &endLocation) + c.Assert(apply, IsTrue) + c.Assert(op, Equals, pb.ErrorOp_Replace) + + // test GetEvent + // get by endLocation + e, err := h.GetEvent(&endLocation) + c.Assert(e, IsNil) + c.Assert(err, IsNil) + // get first event + e, err = h.GetEvent(&startLocation) + c.Assert(err, IsNil) + c.Assert(e.Header.LogPos, Equals, startLocation.Position.Pos) + c.Assert(e.Header.EventSize, Equals, uint32(0)) + c.Assert(e.Event, Equals, event1.Event) + // get second event + startLocation.Suffix++ + e, err = h.GetEvent(&startLocation) + c.Assert(e.Header.LogPos, Equals, endLocation.Position.Pos) + c.Assert(e.Header.EventSize, Equals, endLocation.Position.Pos-startLocation.Position.Pos) + c.Assert(e.Event, Equals, event2.Event) + // get third event, out of index + startLocation.Suffix++ + e, err = h.GetEvent(&startLocation) + c.Assert(err, NotNil) + + // revert exist operator + err = h.Set(startLocation.Position.String(), pb.ErrorOp_Revert, nil) + apply, op = h.Apply(&startLocation, &endLocation) + c.Assert(apply, IsFalse) + c.Assert(op, Equals, pb.ErrorOp_InvalidErrorOp) + + // add two operators + err = h.Set(startLocation.Position.String(), pb.ErrorOp_Replace, []*replication.BinlogEvent{event1, event2}) + c.Assert(err, IsNil) + err = h.Set(endLocation.Position.String(), pb.ErrorOp_Replace, []*replication.BinlogEvent{event1, event2}) + c.Assert(err, IsNil) + + // test removeOutdated + flushLocation := startLocation + h.RemoveOutdated(flushLocation) + apply, op = h.Apply(&startLocation, &endLocation) + c.Assert(apply, IsTrue) + c.Assert(op, Equals, pb.ErrorOp_Replace) + + flushLocation = endLocation + h.RemoveOutdated(flushLocation) + apply, op = h.Apply(&startLocation, &endLocation) + c.Assert(apply, IsFalse) + c.Assert(op, Equals, pb.ErrorOp_InvalidErrorOp) + + apply, op = h.Apply(&endLocation, &nextLocation) + c.Assert(apply, IsTrue) + c.Assert(op, Equals, pb.ErrorOp_Replace) +} diff --git a/syncer/handle_error.go b/syncer/handle_error.go index f5c378be26..e33e6c7c95 100644 --- a/syncer/handle_error.go +++ b/syncer/handle_error.go @@ -17,6 +17,7 @@ import ( "context" "fmt" + "github.com/pingcap/dm/dm/command" "github.com/pingcap/dm/dm/pb" parserpkg "github.com/pingcap/dm/pkg/parser" "github.com/pingcap/dm/pkg/terror" @@ -37,6 +38,12 @@ func (s *Syncer) HandleError(ctx context.Context, req *pb.HandleWorkerErrorReque return fmt.Sprintf("source '%s' has no error", s.cfg.SourceID), nil } pos = startLocation.Position.String() + } else { + startLocation, err := command.VerifyBinlogPos(pos) + if err != nil { + return "", err + } + pos = startLocation.String() } events := make([]*replication.BinlogEvent, 0) From 934faab134e400f47fdc1c99900356123026135f Mon Sep 17 00:00:00 2001 From: gmhdbjd Date: Thu, 6 Aug 2020 18:20:43 +0800 Subject: [PATCH 19/31] fix test --- tests/handle_error/run.sh | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/handle_error/run.sh b/tests/handle_error/run.sh index 76d10c0be7..488461f2e4 100644 --- a/tests/handle_error/run.sh +++ b/tests/handle_error/run.sh @@ -163,6 +163,8 @@ function DM_REPLACE_ERROR_CASE() { run_sql_source1 "insert into ${db}.${tb2} values(5,5);" run_sql_source1 "insert into ${db}.${tb1} values(6,5);" + run_sql_tidb_with_retry "select count(1) from ${db}.${tb1};" "count(1): 4" + run_sql_tidb_with_retry "select count(1) from ${db}.${tb2};" "count(1): 4" } function DM_REPLACE_ERROR() { From 5290ea298bb0085939dc8a700106743b43e828f1 Mon Sep 17 00:00:00 2001 From: gmhdbjd Date: Fri, 7 Aug 2020 11:59:01 +0800 Subject: [PATCH 20/31] address comment --- dm/ctl/master/handle_error.go | 4 ++-- syncer/err-operator/operator.go | 4 ++-- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/dm/ctl/master/handle_error.go b/dm/ctl/master/handle_error.go index f88c5c1f6c..d2fffc5d0f 100644 --- a/dm/ctl/master/handle_error.go +++ b/dm/ctl/master/handle_error.go @@ -28,7 +28,7 @@ import ( // NewHandleErrorCmd creates a HandleError command func NewHandleErrorCmd() *cobra.Command { cmd := &cobra.Command{ - Use: "handle-error [-s source ...] [-b binlog-pos] [replace-sql1;replace-sql2;]", + Use: "handle-error [-s source ...] [-b binlog-pos] [replace-sql1;replace-sql2;]", Short: "skip/replace/revert the current error event or a specific binlog position (binlog-pos) event", RunE: handleErrorFunc, } @@ -58,7 +58,7 @@ func handleErrorFunc(cmd *cobra.Command, _ []string) (err error) { return } - taskName := cmd.Flags().Arg(0) + taskName := common.GetTaskNameFromArgOrFile(cmd.Flags().Arg(0)) operation := cmd.Flags().Arg(1) var sqls []string diff --git a/syncer/err-operator/operator.go b/syncer/err-operator/operator.go index 531d8403a5..dbc47060a2 100644 --- a/syncer/err-operator/operator.go +++ b/syncer/err-operator/operator.go @@ -156,9 +156,9 @@ func (h *Holder) Apply(startLocation, endLocation *binlog.Location) (bool, pb.Er } // set the last replace event as end position - operator.events[len(operator.events)-1].Header.EventSize = endLocation.Position.Pos - startLocation.Position.Pos - operator.events[len(operator.events)-1].Header.LogPos = endLocation.Position.Pos e := operator.events[len(operator.events)-1] + e.Header.EventSize = endLocation.Position.Pos - startLocation.Position.Pos + e.Header.LogPos = endLocation.Position.Pos if e, ok := e.Event.(*replication.QueryEvent); ok { if endLocation.GTIDSet != nil { e.GSet = endLocation.GTIDSet.Origin() From 7673386e90c9023e99d4b470f25a98454dac0166 Mon Sep 17 00:00:00 2001 From: gmhdbjd Date: Fri, 7 Aug 2020 13:54:14 +0800 Subject: [PATCH 21/31] fix test --- tests/dmctl_advance/check_list/handle_error.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/dmctl_advance/check_list/handle_error.sh b/tests/dmctl_advance/check_list/handle_error.sh index ed902e60f2..b6c56bbeda 100644 --- a/tests/dmctl_advance/check_list/handle_error.sh +++ b/tests/dmctl_advance/check_list/handle_error.sh @@ -3,7 +3,7 @@ function handle_error_wrong_arg() { run_dm_ctl $WORK_DIR "127.0.0.1:$MASTER_PORT" \ "handle-error" \ - "handle-error \[-s source ...\] \[-b binlog-pos\] \[replace-sql1;replace-sql2;\] \[flags\]" 1 + "handle-error \[-s source ...\] \[-b binlog-pos\] \[replace-sql1;replace-sql2;\] \[flags\]" 1 } function handle_error_invalid_binlogpos() { From 817354a84f230e7cbd4d74d4b5a9607c5f3ad0c9 Mon Sep 17 00:00:00 2001 From: gmhdbjd Date: Tue, 11 Aug 2020 11:56:22 +0800 Subject: [PATCH 22/31] address comment --- _utils/terror_gen/errors_release.txt | 1 - dm/ctl/master/handle_error.go | 6 ++- dm/pb/dmmaster.pb.go | 2 +- dm/pb/dmworker.pb.go | 66 ++++++++++++++-------------- dm/proto/dmworker.proto | 8 ++-- errors.toml | 6 --- pkg/terror/error_list.go | 8 ++-- 7 files changed, 45 insertions(+), 52 deletions(-) diff --git a/_utils/terror_gen/errors_release.txt b/_utils/terror_gen/errors_release.txt index 1792e48b1f..bd6e39a1ee 100644 --- a/_utils/terror_gen/errors_release.txt +++ b/_utils/terror_gen/errors_release.txt @@ -160,7 +160,6 @@ ErrConfigMissingForBound,[code=20036:class=config:scope=internal:level=high], "M ErrBinlogExtractPosition,[code=22001:class=binlog-op:scope=internal:level=high] ErrBinlogInvalidFilename,[code=22002:class=binlog-op:scope=internal:level=high], "Message: invalid binlog filename" ErrBinlogParsePosFromStr,[code=22003:class=binlog-op:scope=internal:level=high] -ErrBinlogParsePosFromPosStr,[code=22004:class=binlog-op:scope=internal:level=high] ErrCheckpointInvalidTaskMode,[code=24001:class=checkpoint:scope=internal:level=medium], "Message: invalid task mode: %s" ErrCheckpointSaveInvalidPos,[code=24002:class=checkpoint:scope=internal:level=high], "Message: save point %s is older than current location %s" ErrCheckpointInvalidTableFile,[code=24003:class=checkpoint:scope=internal:level=medium], "Message: invalid db table sql file - %s" diff --git a/dm/ctl/master/handle_error.go b/dm/ctl/master/handle_error.go index d2fffc5d0f..7ad89f6cb7 100644 --- a/dm/ctl/master/handle_error.go +++ b/dm/ctl/master/handle_error.go @@ -66,7 +66,9 @@ func handleErrorFunc(cmd *cobra.Command, _ []string) (err error) { switch op { case pb.ErrorOp_Skip, pb.ErrorOp_Revert: if len(cmd.Flags().Args()) > 2 { - common.PrintLines("replace-sqls will be ignored for 'skip/revert' operation") + common.PrintLines("replace-sqls can not be used for 'skip/revert' operation") + err = errors.New("please check output to see error") + return } case pb.ErrorOp_Replace: if len(cmd.Flags().Args()) <= 2 { @@ -80,7 +82,7 @@ func handleErrorFunc(cmd *cobra.Command, _ []string) (err error) { return } default: - common.PrintLines("invalid operation '%s', please use `skip`, `replace`, or `revert`", operation) + common.PrintLines("invalid operation '%s', please use `skip`, `replace` or `revert`", operation) err = errors.New("please check output to see error") return } diff --git a/dm/pb/dmmaster.pb.go b/dm/pb/dmmaster.pb.go index 6b2b47e0ea..9b1b517b5c 100644 --- a/dm/pb/dmmaster.pb.go +++ b/dm/pb/dmmaster.pb.go @@ -3051,7 +3051,7 @@ func (m *HandleErrorRequest) GetOp() ErrorOp { if m != nil { return m.Op } - return ErrorOp_Skip + return ErrorOp_InvalidErrorOp } func (m *HandleErrorRequest) GetTask() string { diff --git a/dm/pb/dmworker.pb.go b/dm/pb/dmworker.pb.go index abf01e4e1d..a4bd837d40 100644 --- a/dm/pb/dmworker.pb.go +++ b/dm/pb/dmworker.pb.go @@ -264,24 +264,24 @@ func (V1MetaOp) EnumDescriptor() ([]byte, []int) { type ErrorOp int32 const ( - ErrorOp_Skip ErrorOp = 0 - ErrorOp_Replace ErrorOp = 1 - ErrorOp_Revert ErrorOp = 2 - ErrorOp_InvalidErrorOp ErrorOp = 3 + ErrorOp_InvalidErrorOp ErrorOp = 0 + ErrorOp_Skip ErrorOp = 1 + ErrorOp_Replace ErrorOp = 2 + ErrorOp_Revert ErrorOp = 3 ) var ErrorOp_name = map[int32]string{ - 0: "Skip", - 1: "Replace", - 2: "Revert", - 3: "InvalidErrorOp", + 0: "InvalidErrorOp", + 1: "Skip", + 2: "Replace", + 3: "Revert", } var ErrorOp_value = map[string]int32{ - "Skip": 0, - "Replace": 1, - "Revert": 2, - "InvalidErrorOp": 3, + "InvalidErrorOp": 0, + "Skip": 1, + "Replace": 2, + "Revert": 3, } func (x ErrorOp) String() string { @@ -3023,7 +3023,7 @@ func (m *HandleWorkerErrorRequest) GetOp() ErrorOp { if m != nil { return m.Op } - return ErrorOp_Skip + return ErrorOp_InvalidErrorOp } func (m *HandleWorkerErrorRequest) GetTask() string { @@ -3103,7 +3103,7 @@ func init() { func init() { proto.RegisterFile("dmworker.proto", fileDescriptor_51a1b9e17fd67b10) } var fileDescriptor_51a1b9e17fd67b10 = []byte{ - // 2272 bytes of a gzipped FileDescriptorProto + // 2268 bytes of a gzipped FileDescriptorProto 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xb4, 0x19, 0x4d, 0x6f, 0xdb, 0xc8, 0x55, 0x24, 0x25, 0x59, 0x7a, 0x92, 0x1c, 0x7a, 0x9c, 0xcd, 0x2a, 0x6a, 0x56, 0x35, 0x98, 0xc5, 0xae, 0xd7, 0x07, 0xa3, 0x71, 0xb7, 0xd8, 0x62, 0x81, 0x6d, 0xd3, 0xd8, 0x5e, 0x27, 0xad, 0xbd, @@ -3227,25 +3227,25 @@ var fileDescriptor_51a1b9e17fd67b10 = []byte{ 0x04, 0xc2, 0x40, 0xfd, 0xf2, 0x6c, 0x67, 0xdb, 0x70, 0x2b, 0xf7, 0x8a, 0x44, 0x09, 0x81, 0x27, 0x3c, 0x15, 0x08, 0x53, 0x23, 0xf9, 0x05, 0xa8, 0xa3, 0x23, 0x6d, 0x3e, 0x0b, 0xaf, 0xb8, 0xc4, 0x18, 0x7b, 0x0f, 0xa1, 0x95, 0x27, 0x87, 0x22, 0x30, 0x47, 0x15, 0x02, 0x05, 0xc2, 0xd4, 0x4a, - 0x09, 0x12, 0xa3, 0xef, 0x3d, 0xa4, 0x36, 0x8a, 0xb1, 0x45, 0x1e, 0xb8, 0xf4, 0x70, 0x15, 0x1e, - 0x0b, 0x8f, 0x7c, 0xc7, 0xe5, 0xa6, 0x26, 0x8e, 0xfe, 0x8a, 0xc7, 0xa9, 0xa9, 0x2b, 0x4e, 0x90, - 0x8b, 0x4c, 0xe3, 0xe0, 0xdf, 0x1b, 0xd0, 0x14, 0x21, 0xcc, 0x0e, 0xa1, 0xab, 0x3e, 0x09, 0xb3, - 0xf7, 0x65, 0x6d, 0xb8, 0xf9, 0x48, 0x3c, 0xe8, 0xd3, 0x34, 0xb4, 0xe4, 0x35, 0xd1, 0xaa, 0xb1, - 0x27, 0xb0, 0x59, 0x7d, 0x5e, 0x65, 0x77, 0x95, 0x04, 0xbd, 0x21, 0x68, 0xb0, 0x8c, 0x54, 0x88, - 0x3a, 0x86, 0x5e, 0xe5, 0xc5, 0x94, 0xd1, 0xbe, 0xcb, 0x1e, 0x51, 0xd7, 0x6a, 0xf4, 0x10, 0x3a, - 0xca, 0x33, 0x24, 0xbb, 0x83, 0xac, 0x8b, 0xaf, 0xab, 0x83, 0xf7, 0x17, 0xf0, 0x85, 0x84, 0x2f, - 0x00, 0xca, 0x77, 0x41, 0xf6, 0x5e, 0xc1, 0xa8, 0x26, 0xfd, 0xe0, 0xce, 0x4d, 0x74, 0xb1, 0xfc, - 0x0c, 0xb6, 0x16, 0xe6, 0x04, 0x76, 0x8f, 0x9c, 0xbb, 0x62, 0x7c, 0x58, 0x6b, 0xcf, 0x21, 0x74, - 0xd5, 0x31, 0x41, 0x1c, 0xd3, 0x92, 0x61, 0x43, 0x08, 0x59, 0x36, 0x51, 0x58, 0x35, 0xf6, 0x63, - 0x80, 0xb2, 0xe9, 0x0b, 0x93, 0x16, 0x86, 0x80, 0xb5, 0x5a, 0x9c, 0xc0, 0x96, 0xf2, 0x53, 0x40, - 0x34, 0x68, 0xe1, 0xdb, 0xc5, 0x7f, 0x05, 0x6b, 0x05, 0xd9, 0xf2, 0x05, 0x5b, 0xed, 0xf4, 0xc2, - 0x3b, 0xab, 0x86, 0x83, 0xc1, 0x07, 0x2b, 0xa8, 0xaa, 0x8b, 0xd4, 0x3f, 0x10, 0xc2, 0x45, 0x4b, - 0xfe, 0x49, 0xac, 0x55, 0xec, 0x67, 0xd0, 0xcb, 0x43, 0x93, 0x12, 0x96, 0x0d, 0x15, 0x7f, 0x2e, - 0x99, 0x0d, 0xd6, 0x0a, 0xfb, 0xb2, 0x10, 0x26, 0x72, 0x97, 0xf5, 0x97, 0xb4, 0x2d, 0x21, 0xe6, - 0xee, 0xca, 0x86, 0x46, 0x6e, 0xef, 0x88, 0xb6, 0x23, 0x62, 0x91, 0xfc, 0xb4, 0xaa, 0x0f, 0xad, - 0x53, 0xe8, 0x51, 0xff, 0xef, 0xaf, 0x87, 0xda, 0x37, 0xaf, 0x87, 0xda, 0xbf, 0x5e, 0x0f, 0xb5, - 0xdf, 0xbc, 0x19, 0xd6, 0xbe, 0x79, 0x33, 0xac, 0xfd, 0xf3, 0xcd, 0xb0, 0x36, 0x6e, 0xd2, 0x4f, - 0xa2, 0xef, 0xff, 0x37, 0x00, 0x00, 0xff, 0xff, 0xff, 0x04, 0x0c, 0xd2, 0x36, 0x1a, 0x00, 0x00, + 0x09, 0x12, 0xa3, 0xef, 0x3d, 0xa4, 0x36, 0x8a, 0xb1, 0xa5, 0x58, 0x28, 0x31, 0x32, 0x18, 0x2e, + 0xbd, 0x48, 0x1e, 0x15, 0x8f, 0x7c, 0xc7, 0x2d, 0xc2, 0xe1, 0x8a, 0xc7, 0xa9, 0x69, 0x1c, 0xfc, + 0x7b, 0x03, 0x9a, 0x22, 0x84, 0xd9, 0x21, 0x74, 0xd5, 0x27, 0x61, 0xf6, 0xbe, 0xac, 0x0d, 0x37, + 0x1f, 0x89, 0x07, 0x7d, 0x9a, 0x86, 0x96, 0xbc, 0x26, 0x5a, 0x35, 0xf6, 0x04, 0x36, 0xab, 0xcf, + 0xab, 0xec, 0xae, 0x92, 0xa0, 0x37, 0x04, 0x0d, 0x96, 0x91, 0x0a, 0x51, 0xc7, 0xd0, 0xab, 0xbc, + 0x98, 0x32, 0xda, 0x77, 0xd9, 0x23, 0xea, 0x5a, 0x8d, 0x1e, 0x42, 0x47, 0x79, 0x86, 0x64, 0x77, + 0x90, 0x75, 0xf1, 0x75, 0x75, 0xf0, 0xfe, 0x02, 0xbe, 0x90, 0xf0, 0x05, 0x40, 0xf9, 0x2e, 0xc8, + 0xde, 0x2b, 0x18, 0xd5, 0xa4, 0x1f, 0xdc, 0xb9, 0x89, 0x2e, 0x96, 0x9f, 0xc1, 0xd6, 0xc2, 0x9c, + 0xc0, 0xee, 0x91, 0x73, 0x57, 0x8c, 0x0f, 0x6b, 0xed, 0x39, 0x84, 0xae, 0x3a, 0x26, 0x88, 0x63, + 0x5a, 0x32, 0x6c, 0x08, 0x21, 0xcb, 0x26, 0x0a, 0xab, 0xc6, 0x7e, 0x0c, 0x50, 0x36, 0x7d, 0x61, + 0xd2, 0xc2, 0x10, 0xb0, 0x56, 0x8b, 0x13, 0xd8, 0x52, 0x7e, 0x0a, 0x88, 0x06, 0x2d, 0x7c, 0xbb, + 0xf8, 0xaf, 0x60, 0xad, 0x20, 0x5b, 0xbe, 0x60, 0xab, 0x9d, 0x5e, 0x78, 0x67, 0xd5, 0x70, 0x30, + 0xf8, 0x60, 0x05, 0x55, 0x75, 0x91, 0xfa, 0x07, 0x42, 0xb8, 0x68, 0xc9, 0x3f, 0x89, 0xb5, 0x8a, + 0xfd, 0x0c, 0x7a, 0x79, 0x68, 0x52, 0xc2, 0xb2, 0xa1, 0xe2, 0xcf, 0x25, 0xb3, 0xc1, 0x5a, 0x61, + 0x5f, 0x16, 0xc2, 0x44, 0xee, 0xb2, 0xfe, 0x92, 0xb6, 0x25, 0xc4, 0xdc, 0x5d, 0xd9, 0xd0, 0xc8, + 0xed, 0x1d, 0xd1, 0x76, 0x44, 0x2c, 0x92, 0x9f, 0x56, 0xf5, 0xa1, 0x75, 0x0a, 0x3d, 0xea, 0xff, + 0xfd, 0xf5, 0x50, 0xfb, 0xe6, 0xf5, 0x50, 0xfb, 0xd7, 0xeb, 0xa1, 0xf6, 0x9b, 0x37, 0xc3, 0xda, + 0x37, 0x6f, 0x86, 0xb5, 0x7f, 0xbe, 0x19, 0xd6, 0xc6, 0x4d, 0xfa, 0x49, 0xf4, 0xfd, 0xff, 0x06, + 0x00, 0x00, 0xff, 0xff, 0x26, 0x56, 0x1b, 0xdb, 0x36, 0x1a, 0x00, 0x00, } // Reference imports to suppress errors if they are not otherwise used. diff --git a/dm/proto/dmworker.proto b/dm/proto/dmworker.proto index 3788fa3dbf..b20f9d5896 100644 --- a/dm/proto/dmworker.proto +++ b/dm/proto/dmworker.proto @@ -432,10 +432,10 @@ message OperateV1MetaResponse { } enum ErrorOp { - Skip = 0; // skip the error event - Replace = 1; // replace the error event with a specified SQL - Revert = 2; // remove the error operator - InvalidErrorOp = 3; + InvalidErrorOp = 0; + Skip = 1; // skip the error event + Replace = 2; // replace the error event with a specified SQL + Revert = 3; // remove the error operator } message HandleWorkerErrorRequest { diff --git a/errors.toml b/errors.toml index bb0d3e7a7a..cfb1733598 100644 --- a/errors.toml +++ b/errors.toml @@ -970,12 +970,6 @@ description = "" workaround = "" tags = ["internal", "high"] -[error.DM-binlog-op-22004] -message = "" -description = "" -workaround = "" -tags = ["internal", "high"] - [error.DM-checkpoint-24001] message = "invalid task mode: %s" description = "" diff --git a/pkg/terror/error_list.go b/pkg/terror/error_list.go index bb67747d60..dd3e3bdda8 100644 --- a/pkg/terror/error_list.go +++ b/pkg/terror/error_list.go @@ -215,7 +215,6 @@ const ( codeBinlogExtractPosition ErrCode = iota + 22001 codeBinlogInvalidFilename codeBinlogParsePosFromStr - codeBinlogParsePosFromPosStr ) // Checkpoint error code @@ -785,10 +784,9 @@ var ( ErrConfigMissingForBound = New(codeConfigMissingForBound, ClassConfig, ScopeInternal, LevelHigh, "source bound %s doesn't have related source config in etcd", "") // Binlog operation error - ErrBinlogExtractPosition = New(codeBinlogExtractPosition, ClassBinlogOp, ScopeInternal, LevelHigh, "", "") - ErrBinlogInvalidFilename = New(codeBinlogInvalidFilename, ClassBinlogOp, ScopeInternal, LevelHigh, "invalid binlog filename", "") - ErrBinlogParsePosFromStr = New(codeBinlogParsePosFromStr, ClassBinlogOp, ScopeInternal, LevelHigh, "", "") - ErrBinlogParsePosFromPosStr = New(codeBinlogParsePosFromPosStr, ClassBinlogOp, ScopeInternal, LevelHigh, "", "") + ErrBinlogExtractPosition = New(codeBinlogExtractPosition, ClassBinlogOp, ScopeInternal, LevelHigh, "", "") + ErrBinlogInvalidFilename = New(codeBinlogInvalidFilename, ClassBinlogOp, ScopeInternal, LevelHigh, "invalid binlog filename", "") + ErrBinlogParsePosFromStr = New(codeBinlogParsePosFromStr, ClassBinlogOp, ScopeInternal, LevelHigh, "", "") // Checkpoint error ErrCheckpointInvalidTaskMode = New(codeCheckpointInvalidTaskMode, ClassCheckpoint, ScopeInternal, LevelMedium, "invalid task mode: %s", "") From 283cb32a082c14a7ace45ba9d19d2f06a513a6b8 Mon Sep 17 00:00:00 2001 From: gmhdbjd Date: Tue, 11 Aug 2020 14:45:12 +0800 Subject: [PATCH 23/31] address comment and fix test --- syncer/err-operator/operator.go | 16 +++++++++------- syncer/err-operator/operator_test.go | 4 +++- syncer/syncer.go | 2 +- ...rding.yaml => double-source-no-sharding.yaml} | 1 + .../conf/double-source-optimistic.yaml | 1 + .../conf/double-source-pessimistic.yaml | 1 + tests/handle_error/run.sh | 4 ++-- 7 files changed, 18 insertions(+), 11 deletions(-) rename tests/handle_error/conf/{no-sharding.yaml => double-source-no-sharding.yaml} (95%) diff --git a/syncer/err-operator/operator.go b/syncer/err-operator/operator.go index dbc47060a2..2cb5ed1e2b 100644 --- a/syncer/err-operator/operator.go +++ b/syncer/err-operator/operator.go @@ -34,7 +34,7 @@ import ( type Operator struct { uuid string // add a UUID, make it more friendly to be traced in log op pb.ErrorOp - events []*replication.BinlogEvent // endLocation -> events + events []*replication.BinlogEvent // startLocation -> events } // newOperator creates a new operator with a random UUID @@ -53,19 +53,21 @@ func (o *Operator) String() string { e.Dump(buf) events = append(events, buf.String()) } - return fmt.Sprintf("uuid: %s, op: %s, events: %s", o.uuid, o.op, strings.Join(events, " ")) + return fmt.Sprintf("uuid: %s, op: %s, events: %s", o.uuid, o.op, strings.Join(events, "\n")) } // Holder holds error operator type Holder struct { mu sync.Mutex operators map[string]*Operator + logger log.Logger } // NewHolder creates a new Holder -func NewHolder() *Holder { +func NewHolder(pLogger *log.Logger) *Holder { return &Holder{ operators: make(map[string]*Operator), + logger: pLogger.WithFields(zap.String("component", "error operator holder")), } } @@ -85,10 +87,10 @@ func (h *Holder) Set(pos string, op pb.ErrorOp, events []*replication.BinlogEven oper := newOperator(op, events) pre, ok := h.operators[pos] if ok { - log.L().Warn("overwrite operator", zap.String("position", pos), zap.Stringer("old operator", pre)) + h.logger.Warn("overwrite operator", zap.String("position", pos), zap.Stringer("old operator", pre)) } h.operators[pos] = oper - log.L().Info("set a new operator", zap.String("position", pos), zap.Stringer("new operator", oper)) + h.logger.Info("set a new operator", zap.String("position", pos), zap.Stringer("new operator", oper)) return nil } @@ -118,7 +120,7 @@ func (h *Holder) GetEvent(startLocation *binlog.Location) (*replication.BinlogEv e := operator.events[startLocation.Suffix] buf := new(bytes.Buffer) e.Dump(buf) - log.L().Info("get replace event", zap.Stringer("event", buf)) + h.logger.Info("get replace event", zap.Stringer("event", buf)) return e, nil } @@ -166,7 +168,7 @@ func (h *Holder) Apply(startLocation, endLocation *binlog.Location) (bool, pb.Er } } - log.L().Info("apply a operator", zap.Stringer("startlocation", startLocation), zap.Stringer("endlocation", endLocation), zap.Stringer("operator", operator)) + h.logger.Info("apply a operator", zap.Stringer("startlocation", startLocation), zap.Stringer("endlocation", endLocation), zap.Stringer("operator", operator)) return true, operator.op } diff --git a/syncer/err-operator/operator_test.go b/syncer/err-operator/operator_test.go index a0e5eb03f4..15d6985ce4 100644 --- a/syncer/err-operator/operator_test.go +++ b/syncer/err-operator/operator_test.go @@ -22,6 +22,7 @@ import ( "github.com/pingcap/dm/dm/pb" "github.com/pingcap/dm/pkg/binlog" + "github.com/pingcap/dm/pkg/log" ) var _ = Suite(&testOperatorSuite{}) @@ -34,7 +35,8 @@ type testOperatorSuite struct { } func (o *testOperatorSuite) TestOperator(c *C) { - h := NewHolder() + logger := log.L() + h := NewHolder(&logger) startLocation := binlog.Location{ Position: mysql.Position{ diff --git a/syncer/syncer.go b/syncer/syncer.go index 9d16e757b8..3e06a88b7b 100644 --- a/syncer/syncer.go +++ b/syncer/syncer.go @@ -216,7 +216,7 @@ func NewSyncer(cfg *config.SubTaskConfig, etcdClient *clientv3.Client) *Syncer { syncer.setSyncCfg() syncer.binlogType = toBinlogType(cfg.UseRelay) - syncer.errOperatorHolder = operator.NewHolder() + syncer.errOperatorHolder = operator.NewHolder(&logger) syncer.readerHub = streamer.GetReaderHub() if cfg.ShardMode == config.ShardPessimistic { diff --git a/tests/handle_error/conf/no-sharding.yaml b/tests/handle_error/conf/double-source-no-sharding.yaml similarity index 95% rename from tests/handle_error/conf/no-sharding.yaml rename to tests/handle_error/conf/double-source-no-sharding.yaml index d2116075a0..38b6b4f1b0 100644 --- a/tests/handle_error/conf/no-sharding.yaml +++ b/tests/handle_error/conf/double-source-no-sharding.yaml @@ -4,6 +4,7 @@ task-mode: all is-sharding: false meta-schema: "dm_meta" timezone: "Asia/Shanghai" +ignore-checking-items: ["table_schema"] target-database: host: "127.0.0.1" diff --git a/tests/handle_error/conf/double-source-optimistic.yaml b/tests/handle_error/conf/double-source-optimistic.yaml index d1b6dda25e..8490b8ea2e 100644 --- a/tests/handle_error/conf/double-source-optimistic.yaml +++ b/tests/handle_error/conf/double-source-optimistic.yaml @@ -5,6 +5,7 @@ is-sharding: true shard-mode: "optimistic" meta-schema: "dm_meta" timezone: "Asia/Shanghai" +ignore-checking-items: ["table_schema"] target-database: host: "127.0.0.1" diff --git a/tests/handle_error/conf/double-source-pessimistic.yaml b/tests/handle_error/conf/double-source-pessimistic.yaml index 739ac8581c..e58949f193 100644 --- a/tests/handle_error/conf/double-source-pessimistic.yaml +++ b/tests/handle_error/conf/double-source-pessimistic.yaml @@ -5,6 +5,7 @@ is-sharding: true shard-mode: "pessimistic" meta-schema: "dm_meta" timezone: "Asia/Shanghai" +ignore-checking-items: ["table_schema"] target-database: host: "127.0.0.1" diff --git a/tests/handle_error/run.sh b/tests/handle_error/run.sh index 488461f2e4..5143a134cb 100644 --- a/tests/handle_error/run.sh +++ b/tests/handle_error/run.sh @@ -69,7 +69,7 @@ function DM_SKIP_ERROR_CASE() { } function DM_SKIP_ERROR() { - run_case SKIP_ERROR "no-sharding" "init_table 111 212" "clean_table" "" + run_case SKIP_ERROR "double-source-no-sharding" "init_table 111 212" "clean_table" "" } # skip modify column, two sources, 4 tables, sharding @@ -168,7 +168,7 @@ function DM_REPLACE_ERROR_CASE() { } function DM_REPLACE_ERROR() { - run_case REPLACE_ERROR "no-sharding" \ + run_case REPLACE_ERROR "double-source-no-sharding" \ "run_sql_source1 \"create table ${db}.${tb2} (a int unique, b int);\"; \ run_sql_source1 \"create table ${db}.${tb1} (a int unique, b int);\"" \ "clean_table" "" From 4c6d0ee782d9001c3098e260a5226b661a889b20 Mon Sep 17 00:00:00 2001 From: gmhdbjd Date: Tue, 11 Aug 2020 20:21:09 +0800 Subject: [PATCH 24/31] address some comment --- syncer/err-operator/operator.go | 2 ++ syncer/err-operator/operator_test.go | 6 +++++- syncer/syncer.go | 12 ++++++------ tests/handle_error/lib.sh | 3 +-- tests/handle_error/run.sh | 12 ++++++------ 5 files changed, 20 insertions(+), 15 deletions(-) diff --git a/syncer/err-operator/operator.go b/syncer/err-operator/operator.go index 2cb5ed1e2b..91eaecad38 100644 --- a/syncer/err-operator/operator.go +++ b/syncer/err-operator/operator.go @@ -175,6 +175,8 @@ func (h *Holder) Apply(startLocation, endLocation *binlog.Location) (bool, pb.Er // RemoveOutdated remove outdated operator func (h *Holder) RemoveOutdated(flushLocation binlog.Location) error { + h.mu.Lock() + defer h.mu.Unlock() for pos := range h.operators { position, err := binlog.PositionFromPosStr(pos) if err != nil { diff --git a/syncer/err-operator/operator_test.go b/syncer/err-operator/operator_test.go index 15d6985ce4..9e6548b956 100644 --- a/syncer/err-operator/operator_test.go +++ b/syncer/err-operator/operator_test.go @@ -23,6 +23,7 @@ import ( "github.com/pingcap/dm/dm/pb" "github.com/pingcap/dm/pkg/binlog" "github.com/pingcap/dm/pkg/log" + "github.com/pingcap/dm/pkg/terror" ) var _ = Suite(&testOperatorSuite{}) @@ -79,7 +80,7 @@ func (o *testOperatorSuite) TestOperator(c *C) { // revert not exist operator err := h.Set(startLocation.Position.String(), pb.ErrorOp_Revert, nil) - c.Assert(err, NotNil) + c.Assert(terror.ErrSyncerOperatorNotExist.Equal(err), IsTrue) // skip event err = h.Set(startLocation.Position.String(), pb.ErrorOp_Skip, nil) @@ -90,6 +91,7 @@ func (o *testOperatorSuite) TestOperator(c *C) { // overwrite operator err = h.Set(startLocation.Position.String(), pb.ErrorOp_Replace, []*replication.BinlogEvent{event1, event2}) + c.Assert(err, IsNil) apply, op = h.Apply(&startLocation, &endLocation) c.Assert(apply, IsTrue) c.Assert(op, Equals, pb.ErrorOp_Replace) @@ -108,6 +110,7 @@ func (o *testOperatorSuite) TestOperator(c *C) { // get second event startLocation.Suffix++ e, err = h.GetEvent(&startLocation) + c.Assert(err, IsNil) c.Assert(e.Header.LogPos, Equals, endLocation.Position.Pos) c.Assert(e.Header.EventSize, Equals, endLocation.Position.Pos-startLocation.Position.Pos) c.Assert(e.Event, Equals, event2.Event) @@ -118,6 +121,7 @@ func (o *testOperatorSuite) TestOperator(c *C) { // revert exist operator err = h.Set(startLocation.Position.String(), pb.ErrorOp_Revert, nil) + c.Assert(err, IsNil) apply, op = h.Apply(&startLocation, &endLocation) c.Assert(apply, IsFalse) c.Assert(op, Equals, pb.ErrorOp_InvalidErrorOp) diff --git a/syncer/syncer.go b/syncer/syncer.go index 41afc43eb9..ae450718fd 100644 --- a/syncer/syncer.go +++ b/syncer/syncer.go @@ -1003,9 +1003,9 @@ func (s *Syncer) sync(tctx *tcontext.Context, queueBucket string, db *DBConn, jo clearF() } - executeSQLs := func() (error, int) { + executeSQLs := func() (int, error) { if len(jobs) == 0 { - return nil, 0 + return 0, nil } queries := make([]string, 0, len(jobs)) args := make([][]interface{}, 0, len(jobs)) @@ -1022,7 +1022,7 @@ func (s *Syncer) sync(tctx *tcontext.Context, queueBucket string, db *DBConn, jo errCtx := &ExecErrorContext{err, jobs[affected].currentLocation.Clone(), fmt.Sprintf("%v", jobs)} s.appendExecErrors(errCtx) } - return err, affected + return affected, err } var err error @@ -1042,7 +1042,7 @@ func (s *Syncer) sync(tctx *tcontext.Context, queueBucket string, db *DBConn, jo } if idx >= count || sqlJob.tp == flush { - err, affect = executeSQLs() + affect, err = executeSQLs() if err != nil { fatalF(err, affect) continue @@ -1052,7 +1052,7 @@ func (s *Syncer) sync(tctx *tcontext.Context, queueBucket string, db *DBConn, jo case <-time.After(waitTime): if len(jobs) > 0 { - err, affect = executeSQLs() + affect, err = executeSQLs() if err != nil { fatalF(err, affect) continue @@ -2675,7 +2675,7 @@ func (s *Syncer) handleEventError(err error, startLocation, endLocation *binlog. func (s *Syncer) getEvent(tctx *tcontext.Context, startLocation *binlog.Location) (*replication.BinlogEvent, error) { // next event is a replace event if s.firstReplace || startLocation.Suffix > 0 { - log.L().Info(fmt.Sprintf("try to get replace event, firstReplace: %v", s.firstReplace), zap.Stringer("location", startLocation)) + s.tctx.L().Info(fmt.Sprintf("try to get replace event, firstReplace: %v", s.firstReplace), zap.Stringer("location", startLocation)) return s.errOperatorHolder.GetEvent(startLocation) } diff --git a/tests/handle_error/lib.sh b/tests/handle_error/lib.sh index d73bce4974..5d1fcdf90c 100644 --- a/tests/handle_error/lib.sh +++ b/tests/handle_error/lib.sh @@ -18,8 +18,7 @@ function init_database() { function extract() { str="$1" source=${str:0:1} - database=${str:1:1} - table=${str:2:1} + table=${str:1:1} } function init_table() { diff --git a/tests/handle_error/run.sh b/tests/handle_error/run.sh index 5143a134cb..60b381e5a4 100644 --- a/tests/handle_error/run.sh +++ b/tests/handle_error/run.sh @@ -57,7 +57,7 @@ function DM_SKIP_ERROR_CASE() { "query-status test" \ "\"stage\": \"Running\"" 2 - # '111' -> 111, '222' -> 222, no error + # '11' -> 11, '22' -> 22, no error run_sql_source1 "insert into ${db}.${tb1} values('111',7)" run_sql_source2 "insert into ${db}.${tb2} values('222',8)" run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" \ @@ -69,7 +69,7 @@ function DM_SKIP_ERROR_CASE() { } function DM_SKIP_ERROR() { - run_case SKIP_ERROR "double-source-no-sharding" "init_table 111 212" "clean_table" "" + run_case SKIP_ERROR "double-source-no-sharding" "init_table 11 22" "clean_table" "" } # skip modify column, two sources, 4 tables, sharding @@ -136,8 +136,8 @@ function DM_SKIP_ERROR_SHARDING_CASE() { } function DM_SKIP_ERROR_SHARDING() { - run_case SKIP_ERROR_SHARDING "double-source-pessimistic" "init_table 111 112 211 212" "clean_table" "pessimistic" - run_case SKIP_ERROR_SHARDING "double-source-optimistic" "init_table 111 112 211 212" "clean_table" "optimistic" + run_case SKIP_ERROR_SHARDING "double-source-pessimistic" "init_table 11 12 21 22" "clean_table" "pessimistic" + run_case SKIP_ERROR_SHARDING "double-source-optimistic" "init_table 11 12 21 22" "clean_table" "optimistic" } # replace add foreign key with database name @@ -371,8 +371,8 @@ function DM_REPLACE_ERROR_MULTIPLE_CASE() { } function DM_REPLACE_ERROR_MULTIPLE() { - run_case REPLACE_ERROR_MULTIPLE "double-source-pessimistic" "init_table 111 211" "clean_table" "pessimistic" - run_case REPLACE_ERROR_MULTIPLE "double-source-optimistic" "init_table 111 211" "clean_table" "optimistic" + run_case REPLACE_ERROR_MULTIPLE "double-source-pessimistic" "init_table 11 21" "clean_table" "pessimistic" + run_case REPLACE_ERROR_MULTIPLE "double-source-optimistic" "init_table 11 21" "clean_table" "optimistic" } function run() { From 1d07b9c0da2275073de0d0569147185f0ef190ae Mon Sep 17 00:00:00 2001 From: gmhdbjd Date: Tue, 11 Aug 2020 20:46:16 +0800 Subject: [PATCH 25/31] remove msg --- dm/worker/server.go | 3 +-- dm/worker/subtask.go | 10 +++++----- dm/worker/worker.go | 6 +++--- syncer/handle_error.go | 14 +++++++------- 4 files changed, 16 insertions(+), 17 deletions(-) diff --git a/dm/worker/server.go b/dm/worker/server.go index fcf1b0a958..0c5dcab182 100644 --- a/dm/worker/server.go +++ b/dm/worker/server.go @@ -973,13 +973,12 @@ func (s *Server) HandleError(ctx context.Context, req *pb.HandleWorkerErrorReque return makeCommonWorkerResponse(terror.ErrWorkerNoStart.Generate()), nil } - msg, err := w.HandleError(ctx, req) + err := w.HandleError(ctx, req) if err != nil { return makeCommonWorkerResponse(err), nil } return &pb.CommonWorkerResponse{ Result: true, - Msg: msg, Worker: s.cfg.Name, }, nil } diff --git a/dm/worker/subtask.go b/dm/worker/subtask.go index e880ba17c5..97ec9654e9 100644 --- a/dm/worker/subtask.go +++ b/dm/worker/subtask.go @@ -637,19 +637,19 @@ func (st *SubTask) fail(err error) { } // HandleError handle error for syncer unit -func (st *SubTask) HandleError(ctx context.Context, req *pb.HandleWorkerErrorRequest) (string, error) { +func (st *SubTask) HandleError(ctx context.Context, req *pb.HandleWorkerErrorRequest) error { syncUnit, ok := st.currUnit.(*syncer.Syncer) if !ok { - return "", terror.ErrWorkerOperSyncUnitOnly.Generate(st.currUnit.Type()) + return terror.ErrWorkerOperSyncUnitOnly.Generate(st.currUnit.Type()) } - msg, err := syncUnit.HandleError(ctx, req) + err := syncUnit.HandleError(ctx, req) if err != nil { - return msg, err + return err } if st.Stage() == pb.Stage_Paused { err = st.Resume() } - return msg, err + return err } diff --git a/dm/worker/worker.go b/dm/worker/worker.go index a9dcd6048a..154c0cb6bd 100644 --- a/dm/worker/worker.go +++ b/dm/worker/worker.go @@ -813,17 +813,17 @@ func (w *Worker) getAllSubTaskStatus() map[string]*pb.SubTaskStatus { } // HandleError handle worker error -func (w *Worker) HandleError(ctx context.Context, req *pb.HandleWorkerErrorRequest) (string, error) { +func (w *Worker) HandleError(ctx context.Context, req *pb.HandleWorkerErrorRequest) error { w.Lock() defer w.Unlock() if w.closed.Get() == closedTrue { - return "", terror.ErrWorkerAlreadyClosed.Generate() + return terror.ErrWorkerAlreadyClosed.Generate() } st := w.subTaskHolder.findSubTask(req.Task) if st == nil { - return "", terror.ErrWorkerSubTaskNotFound.Generate(req.Task) + return terror.ErrWorkerSubTaskNotFound.Generate(req.Task) } return st.HandleError(ctx, req) diff --git a/syncer/handle_error.go b/syncer/handle_error.go index e33e6c7c95..64d7480810 100644 --- a/syncer/handle_error.go +++ b/syncer/handle_error.go @@ -29,19 +29,19 @@ import ( ) // HandleError handle error for syncer -func (s *Syncer) HandleError(ctx context.Context, req *pb.HandleWorkerErrorRequest) (string, error) { +func (s *Syncer) HandleError(ctx context.Context, req *pb.HandleWorkerErrorRequest) error { pos := req.BinlogPos if len(pos) == 0 { startLocation := s.getErrLocation() if startLocation == nil { - return fmt.Sprintf("source '%s' has no error", s.cfg.SourceID), nil + return fmt.Errorf("source '%s' has no error", s.cfg.SourceID) } pos = startLocation.Position.String() } else { startLocation, err := command.VerifyBinlogPos(pos) if err != nil { - return "", err + return err } pos = startLocation.String() } @@ -51,22 +51,22 @@ func (s *Syncer) HandleError(ctx context.Context, req *pb.HandleWorkerErrorReque if req.Op == pb.ErrorOp_Replace { events, err = s.genEvents(req.Sqls) if err != nil { - return "", err + return err } } err = s.errOperatorHolder.Set(pos, req.Op, events) if err != nil { - return "", err + return err } // remove outdated operators when add operator err = s.errOperatorHolder.RemoveOutdated(s.checkpoint.FlushedGlobalPoint()) if err != nil { - return "", err + return err } - return "", nil + return nil } func (s *Syncer) genEvents(sqls []string) ([]*replication.BinlogEvent, error) { From 9ce4c9b924e2697bf1b0e79be77a9aa55ba2d925 Mon Sep 17 00:00:00 2001 From: gmhdbjd Date: Tue, 11 Aug 2020 21:16:07 +0800 Subject: [PATCH 26/31] address comment --- syncer/handle_error.go | 1 + syncer/job.go | 6 +++--- 2 files changed, 4 insertions(+), 3 deletions(-) diff --git a/syncer/handle_error.go b/syncer/handle_error.go index 64d7480810..646bd4600b 100644 --- a/syncer/handle_error.go +++ b/syncer/handle_error.go @@ -21,6 +21,7 @@ import ( "github.com/pingcap/dm/dm/pb" parserpkg "github.com/pingcap/dm/pkg/parser" "github.com/pingcap/dm/pkg/terror" + "github.com/pingcap/parser" "github.com/pingcap/parser/ast" tmysql "github.com/pingcap/parser/mysql" diff --git a/syncer/job.go b/syncer/job.go index 5309cea280..73a69fe469 100644 --- a/syncer/job.go +++ b/syncer/job.go @@ -68,9 +68,9 @@ type job struct { args []interface{} key string retry bool - location binlog.Location - startLocation binlog.Location - currentLocation binlog.Location // exactly binlog position of current SQL + location binlog.Location // location of last received (ROTATE / QUERY / XID) event, for global/table checkpoint + startLocation binlog.Location // start location of the sql in binlog, for handle_error + currentLocation binlog.Location // end location of the sql in binlog, for user to skip sql manually by changing checkpoint ddls []string traceID string traceGID string From 92d933c96808f1d60596f9ddc0119d4159e097a2 Mon Sep 17 00:00:00 2001 From: gmhdbjd Date: Wed, 12 Aug 2020 10:33:00 +0800 Subject: [PATCH 27/31] address comment --- _utils/terror_gen/errors_release.txt | 1 + errors.toml | 6 ++++++ pkg/terror/error_list.go | 2 ++ syncer/err-operator/operator.go | 3 ++- tests/handle_error/run.sh | 2 +- 5 files changed, 12 insertions(+), 2 deletions(-) diff --git a/_utils/terror_gen/errors_release.txt b/_utils/terror_gen/errors_release.txt index e7f69ddf08..f5482cb0fa 100644 --- a/_utils/terror_gen/errors_release.txt +++ b/_utils/terror_gen/errors_release.txt @@ -305,6 +305,7 @@ ErrSyncerShardDDLConflict,[code=36062:class=sync-unit:scope=internal:level=high] ErrSyncerFailpoint,[code=36063:class=sync-unit:scope=internal:level=low], "Message: failpoint specified error" ErrSyncerReplaceEvent,[code=36064:class=sync-unit:scope=internal:level=high] ErrSyncerOperatorNotExist,[code=36065:class=sync-unit:scope=internal:level=low], "Message: error operator not exist, position: %s" +ErrSyncerReplaceEventNotExist,[code=36066:class=sync-unit:scope=internal:level=high], "Message: replace event not exist, location: %s" ErrMasterSQLOpNilRequest,[code=38001:class=dm-master:scope=internal:level=medium], "Message: nil request not valid" ErrMasterSQLOpNotSupport,[code=38002:class=dm-master:scope=internal:level=medium], "Message: op %s not supported" ErrMasterSQLOpWithoutSharding,[code=38003:class=dm-master:scope=internal:level=medium], "Message: operate request without --sharding specified not valid" diff --git a/errors.toml b/errors.toml index c98fc84a23..6155637452 100644 --- a/errors.toml +++ b/errors.toml @@ -1840,6 +1840,12 @@ description = "" workaround = "" tags = ["internal", "low"] +[error.DM-sync-unit-36066] +message = "replace event not exist, location: %s" +description = "" +workaround = "" +tags = ["internal", "high"] + [error.DM-dm-master-38001] message = "nil request not valid" description = "" diff --git a/pkg/terror/error_list.go b/pkg/terror/error_list.go index 57233a8c70..60de188f6f 100644 --- a/pkg/terror/error_list.go +++ b/pkg/terror/error_list.go @@ -388,6 +388,7 @@ const ( codeSyncerFailpoint codeSyncerReplaceEvent codeSyncerOperatorNotExist + codeSyncerReplcaeEventNotExist ) // DM-master error code @@ -948,6 +949,7 @@ var ( ErrSyncerFailpoint = New(codeSyncerFailpoint, ClassSyncUnit, ScopeInternal, LevelLow, "failpoint specified error", "") ErrSyncerReplaceEvent = New(codeSyncerReplaceEvent, ClassSyncUnit, ScopeInternal, LevelHigh, "", "") ErrSyncerOperatorNotExist = New(codeSyncerOperatorNotExist, ClassSyncUnit, ScopeInternal, LevelLow, "error operator not exist, position: %s", "") + ErrSyncerReplaceEventNotExist = New(codeSyncerReplcaeEventNotExist, ClassSyncUnit, ScopeInternal, LevelHigh, "replace event not exist, location: %s", "") // DM-master error ErrMasterSQLOpNilRequest = New(codeMasterSQLOpNilRequest, ClassDMMaster, ScopeInternal, LevelMedium, "nil request not valid", "") diff --git a/syncer/err-operator/operator.go b/syncer/err-operator/operator.go index 91eaecad38..5674157ec4 100644 --- a/syncer/err-operator/operator.go +++ b/syncer/err-operator/operator.go @@ -110,7 +110,7 @@ func (h *Holder) GetEvent(startLocation *binlog.Location) (*replication.BinlogEv key := startLocation.Position.String() operator, ok := h.operators[key] if !ok { - return nil, nil + return nil, terror.ErrSyncerReplaceEventNotExist.Generate(startLocation) } if len(operator.events) <= startLocation.Suffix { @@ -184,6 +184,7 @@ func (h *Holder) RemoveOutdated(flushLocation binlog.Location) error { return err } if binlog.ComparePosition(position, flushLocation.Position) == -1 { + h.logger.Info("remove a outdated operator", zap.Stringer("position", position), zap.Stringer("flush postion", flushLocation.Position), zap.Stringer("operator", h.operators[pos])) delete(h.operators, pos) } } diff --git a/tests/handle_error/run.sh b/tests/handle_error/run.sh index 60b381e5a4..171fbeeb6d 100644 --- a/tests/handle_error/run.sh +++ b/tests/handle_error/run.sh @@ -50,7 +50,7 @@ function DM_SKIP_ERROR_CASE() { # skip all sources run_dm_ctl $WORK_DIR "127.0.0.1:$MASTER_PORT" \ "handle-error test skip" \ - "\"result\": true" 3 \ + "\"result\": true" 2 \ "\"source 'mysql-replica-01' has no error\"" 1 run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" \ From 0018b5d60e9ff60d46a2e99cb25da7046c3a9f80 Mon Sep 17 00:00:00 2001 From: gmhdbjd Date: Wed, 12 Aug 2020 11:19:36 +0800 Subject: [PATCH 28/31] address comment --- syncer/err-operator/operator.go | 5 ----- syncer/syncer.go | 35 +++++++++++++++++---------------- 2 files changed, 18 insertions(+), 22 deletions(-) diff --git a/syncer/err-operator/operator.go b/syncer/err-operator/operator.go index 5674157ec4..fba7b28525 100644 --- a/syncer/err-operator/operator.go +++ b/syncer/err-operator/operator.go @@ -130,11 +130,6 @@ func (h *Holder) Apply(startLocation, endLocation *binlog.Location) (bool, pb.Er h.mu.Lock() defer h.mu.Unlock() - // only apply the origin event - if endLocation.Suffix != 0 { - return false, pb.ErrorOp_InvalidErrorOp - } - key := startLocation.Position.String() operator, ok := h.operators[key] if !ok { diff --git a/syncer/syncer.go b/syncer/syncer.go index ae450718fd..8b01e84246 100644 --- a/syncer/syncer.go +++ b/syncer/syncer.go @@ -170,7 +170,7 @@ type Syncer struct { errOperatorHolder *operator.Holder - firstReplace bool // true if we replace first event by handle-error + inReplaceErr bool // true if we are in replace events by handle-error // TODO: re-implement tracer flow for binlog event later. tracer *tracing.Tracer @@ -439,7 +439,7 @@ func (s *Syncer) reset() { s.execError.Set(nil) s.resetExecErrors() s.setErrLocation(nil, nil) - s.firstReplace = false + s.inReplaceErr = false switch s.cfg.ShardMode { case config.ShardPessimistic: @@ -1218,6 +1218,7 @@ func (s *Syncer) Run(ctx context.Context) (err error) { currentLocation = savedGlobalLastLocation.Clone() lastLocation = savedGlobalLastLocation.Clone() // restore global last pos } + s.inReplaceErr = (currentLocation.Suffix != 0) err3 := s.streamerController.RedirectStreamer(s.tctx, currentLocation.Clone()) if err3 != nil { @@ -1242,6 +1243,7 @@ func (s *Syncer) Run(ctx context.Context) (err error) { lastLocation = shardingReSync.currLocation.Clone() currentLocation = shardingReSync.currLocation.Clone() + s.inReplaceErr = (currentLocation.Suffix != 0) err = s.streamerController.RedirectStreamer(s.tctx, shardingReSync.currLocation.Clone()) if err != nil { return err @@ -1333,13 +1335,9 @@ func (s *Syncer) Run(ctx context.Context) (err error) { switch e.Event.(type) { case *replication.RowsEvent, *replication.QueryEvent: endSuffix := startLocation.Suffix - if s.firstReplace { - endSuffix = 1 - s.firstReplace = false - } else if endSuffix > 0 { + if s.inReplaceErr { endSuffix++ } - currentLocation = binlog.Location{ Position: mysql.Position{ Name: lastLocation.Position.Name, @@ -1352,19 +1350,22 @@ func (s *Syncer) Run(ctx context.Context) (err error) { currentLocation.GTIDSet.Set(ev.GSet) } - apply, op := s.errOperatorHolder.Apply(&startLocation, ¤tLocation) - if apply { - if op == pb.ErrorOp_Replace { - s.firstReplace = true - // revert currentLocation to startLocation - currentLocation = startLocation.Clone() + if !s.inReplaceErr { + apply, op := s.errOperatorHolder.Apply(&startLocation, ¤tLocation) + if apply { + if op == pb.ErrorOp_Replace { + s.inReplaceErr = true + // revert currentLocation to startLocation + currentLocation = startLocation.Clone() + } + // skip the event + continue } - // skip the event - continue } // set endLocation.Suffix=0 of last replace event if currentLocation.Suffix > 0 && e.Header.EventSize > 0 { currentLocation.Suffix = 0 + s.inReplaceErr = false } default: } @@ -2674,8 +2675,8 @@ func (s *Syncer) handleEventError(err error, startLocation, endLocation *binlog. // getEvent gets an event from streamerController or errOperatorHolder func (s *Syncer) getEvent(tctx *tcontext.Context, startLocation *binlog.Location) (*replication.BinlogEvent, error) { // next event is a replace event - if s.firstReplace || startLocation.Suffix > 0 { - s.tctx.L().Info(fmt.Sprintf("try to get replace event, firstReplace: %v", s.firstReplace), zap.Stringer("location", startLocation)) + if s.inReplaceErr { + s.tctx.L().Info(fmt.Sprintf("try to get replace event"), zap.Stringer("location", startLocation)) return s.errOperatorHolder.GetEvent(startLocation) } From b8403ec4f08ba777782fee1ac46169ff2f284728 Mon Sep 17 00:00:00 2001 From: gmhdbjd Date: Wed, 12 Aug 2020 11:47:03 +0800 Subject: [PATCH 29/31] update ut --- syncer/err-operator/operator_test.go | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/syncer/err-operator/operator_test.go b/syncer/err-operator/operator_test.go index 9e6548b956..b3890082eb 100644 --- a/syncer/err-operator/operator_test.go +++ b/syncer/err-operator/operator_test.go @@ -100,7 +100,7 @@ func (o *testOperatorSuite) TestOperator(c *C) { // get by endLocation e, err := h.GetEvent(&endLocation) c.Assert(e, IsNil) - c.Assert(err, IsNil) + c.Assert(terror.ErrSyncerReplaceEventNotExist.Equal(err), IsTrue) // get first event e, err = h.GetEvent(&startLocation) c.Assert(err, IsNil) @@ -117,7 +117,7 @@ func (o *testOperatorSuite) TestOperator(c *C) { // get third event, out of index startLocation.Suffix++ e, err = h.GetEvent(&startLocation) - c.Assert(err, NotNil) + c.Assert(terror.ErrSyncerReplaceEvent.Equal(err), IsTrue) // revert exist operator err = h.Set(startLocation.Position.String(), pb.ErrorOp_Revert, nil) From eaef36c6498d76ce5d522a33d77106925b144903 Mon Sep 17 00:00:00 2001 From: gmhdbjd Date: Thu, 13 Aug 2020 14:04:25 +0800 Subject: [PATCH 30/31] address comment --- _utils/terror_gen/errors_release.txt | 2 +- pkg/binlog/event/util.go | 8 ++++---- pkg/binlog/position.go | 9 +++++++++ pkg/terror/error_list.go | 4 +++- syncer/syncer.go | 25 ++++++++++++++----------- 5 files changed, 31 insertions(+), 17 deletions(-) diff --git a/_utils/terror_gen/errors_release.txt b/_utils/terror_gen/errors_release.txt index 57ce215f6b..17816584dc 100644 --- a/_utils/terror_gen/errors_release.txt +++ b/_utils/terror_gen/errors_release.txt @@ -100,7 +100,6 @@ ErrBinlogDecodeMySQLGTIDSet,[code=11093:class=functional:scope=internal:level=hi ErrBinlogNeedMariaDBGTIDSet,[code=11094:class=functional:scope=internal:level=high], "Message: the event should be a MariadbGTIDListEvent, but got %T" ErrBinlogParseMariaDBGTIDSet,[code=11095:class=functional:scope=internal:level=high], "Message: parse MariaDB GTID set" ErrBinlogMariaDBAddGTIDSet,[code=11096:class=functional:scope=internal:level=high], "Message: add set %v to GTID set" -ErrBinlogStatusVarsParse,[code=11117:class=functional:scope=internal:level=medium], "Message: fail to parse binglog status_vars: %v, offset: %d" ErrTracingEventDataNotValid,[code=11097:class=functional:scope=internal:level=high], "Message: invalid event data for type: %s" ErrTracingUploadData,[code=11098:class=functional:scope=internal:level=high], "Message: upload event" ErrTracingEventTypeNotValid,[code=11099:class=functional:scope=internal:level=high], "Message: invalid event type %s, will not process" @@ -121,6 +120,7 @@ ErrConnRegistryTLSConfig,[code=11113:class=functional:scope=internal:level=mediu ErrUpgradeVersionEtcdFail,[code=11114:class=functional:scope=internal:level=high], "Message: fail to operate DM cluster version in etcd, Workaround: Please use `list-member --master` to confirm whether the DM-master cluster is healthy" ErrInvalidV1WorkerMetaPath,[code=11115:class=functional:scope=internal:level=medium], "Message: %s is an invalid v1.0.x DM-worker meta path, Workaround: Please check no `meta-dir` set for v1.0.x DM-worker." ErrFailUpdateV1DBSchema,[code=11116:class=functional:scope=internal:level=medium], "Message: fail to upgrade v1.0.x DB schema, Workaround: Please confirm that you have not violated any restrictions in the upgrade documentation." +ErrBinlogStatusVarsParse,[code=11117:class=functional:scope=internal:level=medium], "Message: fail to parse binglog status_vars: %v, offset: %d" ErrVerifyHandleErrorArgs,[code=11118:class=functional:scope=internal:level=low], "Workaround: Please make sure the args are correct." ErrConfigCheckItemNotSupport,[code=20001:class=config:scope=internal:level=medium], "Message: checking item %s is not supported\n%s, Workaround: Please check `ignore-checking-items` config in task configuration file, which can be set including `all`/`dump_privilege`/`replication_privilege`/`version`/`binlog_enable`/`binlog_format`/`binlog_row_image`/`table_schema`/`schema_of_shard_tables`/`auto_increment_ID`." ErrConfigTomlTransform,[code=20002:class=config:scope=internal:level=medium], "Message: %s, Workaround: Please check the configuration file has correct TOML format." diff --git a/pkg/binlog/event/util.go b/pkg/binlog/event/util.go index 7df2a89dff..3f0665a77b 100644 --- a/pkg/binlog/event/util.go +++ b/pkg/binlog/event/util.go @@ -375,8 +375,8 @@ func statusVarsToKV(statusVars []byte) (map[byte][]byte, error) { if length, ok := statusVarsFixedLength[key]; ok { value = make([]byte, length) - n, err := r.Read(value) - if err != nil || n != length { + n, err2 := r.Read(value) + if err2 != nil || n != length { return generateError(io.EOF) } @@ -392,8 +392,8 @@ func statusVarsToKV(statusVars []byte) (map[byte][]byte, error) { return generateError(err) } - b, err := r.ReadByte() - if err != nil { + b, err2 := r.ReadByte() + if err2 != nil { return generateError(err) } value = append(value, b) diff --git a/pkg/binlog/position.go b/pkg/binlog/position.go index bc0ae73172..3c6bf029e5 100644 --- a/pkg/binlog/position.go +++ b/pkg/binlog/position.go @@ -234,6 +234,15 @@ func (l Location) Clone() Location { return l.CloneWithFlavor("") } +// ClonePtr clones a same Location pointer +func (l *Location) ClonePtr() *Location { + if l == nil { + return nil + } + newLocation := l.Clone() + return &newLocation +} + // CloneWithFlavor clones the location, and if the GTIDSet is nil, will create a GTIDSet with specified flavor. func (l Location) CloneWithFlavor(flavor string) Location { var newGTIDSet gtid.Set diff --git a/pkg/terror/error_list.go b/pkg/terror/error_list.go index c4b454271e..a3ce57a9c3 100644 --- a/pkg/terror/error_list.go +++ b/pkg/terror/error_list.go @@ -705,7 +705,6 @@ var ( ErrBinlogNeedMariaDBGTIDSet = New(codeBinlogNeedMariaDBGTIDSet, ClassFunctional, ScopeInternal, LevelHigh, "the event should be a MariadbGTIDListEvent, but got %T", "") ErrBinlogParseMariaDBGTIDSet = New(codeBinlogParseMariaDBGTIDSet, ClassFunctional, ScopeInternal, LevelHigh, "parse MariaDB GTID set", "") ErrBinlogMariaDBAddGTIDSet = New(codeBinlogMariaDBAddGTIDSet, ClassFunctional, ScopeInternal, LevelHigh, "add set %v to GTID set", "") - ErrBinlogStatusVarsParse = New(codeBinlogStatusVarsParse, ClassFunctional, ScopeInternal, LevelMedium, "fail to parse binglog status_vars: %v, offset: %d", "") // pkg/tracing ErrTracingEventDataNotValid = New(codeTracingEventDataNotValid, ClassFunctional, ScopeInternal, LevelHigh, "invalid event data for type: %s", "") @@ -748,6 +747,9 @@ var ( // pkg/v1dbschema ErrFailUpdateV1DBSchema = New(codeFailUpdateV1DBSchema, ClassFunctional, ScopeInternal, LevelMedium, "fail to upgrade v1.0.x DB schema", "Please confirm that you have not violated any restrictions in the upgrade documentation.") + // pkg/binlog + ErrBinlogStatusVarsParse = New(codeBinlogStatusVarsParse, ClassFunctional, ScopeInternal, LevelMedium, "fail to parse binglog status_vars: %v, offset: %d", "") + // Functional error ErrVerifyHandleErrorArgs = New(codeVerifyHandleErrorArgs, ClassFunctional, ScopeInternal, LevelLow, "", "Please make sure the args are correct.") diff --git a/syncer/syncer.go b/syncer/syncer.go index 8b01e84246..0408c4798b 100644 --- a/syncer/syncer.go +++ b/syncer/syncer.go @@ -994,7 +994,7 @@ func (s *Syncer) sync(tctx *tcontext.Context, queueBucket string, db *DBConn, jo } } - fatalF := func(err error, affected int) { + fatalF := func(affected int, err error) { s.execError.Set(err) if !utils.IsContextCanceledError(err) { err = s.handleEventError(err, &jobs[affected].startLocation, &jobs[affected].currentLocation) @@ -1044,7 +1044,7 @@ func (s *Syncer) sync(tctx *tcontext.Context, queueBucket string, db *DBConn, jo if idx >= count || sqlJob.tp == flush { affect, err = executeSQLs() if err != nil { - fatalF(err, affect) + fatalF(affect, err) continue } clearF() @@ -1054,7 +1054,7 @@ func (s *Syncer) sync(tctx *tcontext.Context, queueBucket string, db *DBConn, jo if len(jobs) > 0 { affect, err = executeSQLs() if err != nil { - fatalF(err, affect) + fatalF(affect, err) continue } clearF() @@ -2644,16 +2644,19 @@ func (s *Syncer) setErrLocation(startLocation, endLocation *binlog.Location) { s.errLocation.Lock() defer s.errLocation.Unlock() - if s.errLocation.startLocation == nil || startLocation == nil { - s.errLocation.startLocation = startLocation - } else if binlog.CompareLocation(*startLocation, *s.errLocation.startLocation, s.cfg.EnableGTID) < 0 { - s.errLocation.startLocation = startLocation + cloneStartLocation := startLocation.ClonePtr() + cloneEndLocation := endLocation.ClonePtr() + + if s.errLocation.startLocation == nil || cloneStartLocation == nil { + s.errLocation.startLocation = cloneStartLocation + } else if binlog.CompareLocation(*cloneStartLocation, *s.errLocation.startLocation, s.cfg.EnableGTID) < 0 { + s.errLocation.startLocation = cloneStartLocation } - if s.errLocation.endLocation == nil || endLocation == nil { - s.errLocation.endLocation = endLocation - } else if binlog.CompareLocation(*endLocation, *s.errLocation.endLocation, s.cfg.EnableGTID) < 0 { - s.errLocation.endLocation = endLocation + if s.errLocation.endLocation == nil || cloneEndLocation == nil { + s.errLocation.endLocation = cloneEndLocation + } else if binlog.CompareLocation(*cloneEndLocation, *s.errLocation.endLocation, s.cfg.EnableGTID) < 0 { + s.errLocation.endLocation = cloneEndLocation } } From ccb3d91b93d5b470dffaf177dfc035f9f16f1eaa Mon Sep 17 00:00:00 2001 From: gmhdbjd Date: Thu, 13 Aug 2020 14:24:34 +0800 Subject: [PATCH 31/31] better variable and func name --- syncer/err-operator/operator.go | 6 +++--- syncer/err-operator/operator_test.go | 12 ++++++------ syncer/syncer.go | 22 ++++++++++++---------- 3 files changed, 21 insertions(+), 19 deletions(-) diff --git a/syncer/err-operator/operator.go b/syncer/err-operator/operator.go index fba7b28525..5d49736de9 100644 --- a/syncer/err-operator/operator.go +++ b/syncer/err-operator/operator.go @@ -125,8 +125,8 @@ func (h *Holder) GetEvent(startLocation *binlog.Location) (*replication.BinlogEv return e, nil } -// Apply tries to apply operation for event by location -func (h *Holder) Apply(startLocation, endLocation *binlog.Location) (bool, pb.ErrorOp) { +// MatchAndApply tries to match operation for event by location and apply it on replace events +func (h *Holder) MatchAndApply(startLocation, endLocation *binlog.Location) (bool, pb.ErrorOp) { h.mu.Lock() defer h.mu.Unlock() @@ -163,7 +163,7 @@ func (h *Holder) Apply(startLocation, endLocation *binlog.Location) (bool, pb.Er } } - h.logger.Info("apply a operator", zap.Stringer("startlocation", startLocation), zap.Stringer("endlocation", endLocation), zap.Stringer("operator", operator)) + h.logger.Info("match and apply a operator", zap.Stringer("startlocation", startLocation), zap.Stringer("endlocation", endLocation), zap.Stringer("operator", operator)) return true, operator.op } diff --git a/syncer/err-operator/operator_test.go b/syncer/err-operator/operator_test.go index b3890082eb..4e8c6d4518 100644 --- a/syncer/err-operator/operator_test.go +++ b/syncer/err-operator/operator_test.go @@ -85,14 +85,14 @@ func (o *testOperatorSuite) TestOperator(c *C) { // skip event err = h.Set(startLocation.Position.String(), pb.ErrorOp_Skip, nil) c.Assert(err, IsNil) - apply, op := h.Apply(&startLocation, &endLocation) + apply, op := h.MatchAndApply(&startLocation, &endLocation) c.Assert(apply, IsTrue) c.Assert(op, Equals, pb.ErrorOp_Skip) // overwrite operator err = h.Set(startLocation.Position.String(), pb.ErrorOp_Replace, []*replication.BinlogEvent{event1, event2}) c.Assert(err, IsNil) - apply, op = h.Apply(&startLocation, &endLocation) + apply, op = h.MatchAndApply(&startLocation, &endLocation) c.Assert(apply, IsTrue) c.Assert(op, Equals, pb.ErrorOp_Replace) @@ -122,7 +122,7 @@ func (o *testOperatorSuite) TestOperator(c *C) { // revert exist operator err = h.Set(startLocation.Position.String(), pb.ErrorOp_Revert, nil) c.Assert(err, IsNil) - apply, op = h.Apply(&startLocation, &endLocation) + apply, op = h.MatchAndApply(&startLocation, &endLocation) c.Assert(apply, IsFalse) c.Assert(op, Equals, pb.ErrorOp_InvalidErrorOp) @@ -135,17 +135,17 @@ func (o *testOperatorSuite) TestOperator(c *C) { // test removeOutdated flushLocation := startLocation h.RemoveOutdated(flushLocation) - apply, op = h.Apply(&startLocation, &endLocation) + apply, op = h.MatchAndApply(&startLocation, &endLocation) c.Assert(apply, IsTrue) c.Assert(op, Equals, pb.ErrorOp_Replace) flushLocation = endLocation h.RemoveOutdated(flushLocation) - apply, op = h.Apply(&startLocation, &endLocation) + apply, op = h.MatchAndApply(&startLocation, &endLocation) c.Assert(apply, IsFalse) c.Assert(op, Equals, pb.ErrorOp_InvalidErrorOp) - apply, op = h.Apply(&endLocation, &nextLocation) + apply, op = h.MatchAndApply(&endLocation, &nextLocation) c.Assert(apply, IsTrue) c.Assert(op, Equals, pb.ErrorOp_Replace) } diff --git a/syncer/syncer.go b/syncer/syncer.go index 0408c4798b..07301733d7 100644 --- a/syncer/syncer.go +++ b/syncer/syncer.go @@ -170,7 +170,7 @@ type Syncer struct { errOperatorHolder *operator.Holder - inReplaceErr bool // true if we are in replace events by handle-error + isReplacingErr bool // true if we are in replace events by handle-error // TODO: re-implement tracer flow for binlog event later. tracer *tracing.Tracer @@ -439,7 +439,7 @@ func (s *Syncer) reset() { s.execError.Set(nil) s.resetExecErrors() s.setErrLocation(nil, nil) - s.inReplaceErr = false + s.isReplacingErr = false switch s.cfg.ShardMode { case config.ShardPessimistic: @@ -1218,7 +1218,8 @@ func (s *Syncer) Run(ctx context.Context) (err error) { currentLocation = savedGlobalLastLocation.Clone() lastLocation = savedGlobalLastLocation.Clone() // restore global last pos } - s.inReplaceErr = (currentLocation.Suffix != 0) + // if suffix>0, we are replacing error + s.isReplacingErr = (currentLocation.Suffix != 0) err3 := s.streamerController.RedirectStreamer(s.tctx, currentLocation.Clone()) if err3 != nil { @@ -1243,7 +1244,8 @@ func (s *Syncer) Run(ctx context.Context) (err error) { lastLocation = shardingReSync.currLocation.Clone() currentLocation = shardingReSync.currLocation.Clone() - s.inReplaceErr = (currentLocation.Suffix != 0) + // if suffix>0, we are replacing error + s.isReplacingErr = (currentLocation.Suffix != 0) err = s.streamerController.RedirectStreamer(s.tctx, shardingReSync.currLocation.Clone()) if err != nil { return err @@ -1335,7 +1337,7 @@ func (s *Syncer) Run(ctx context.Context) (err error) { switch e.Event.(type) { case *replication.RowsEvent, *replication.QueryEvent: endSuffix := startLocation.Suffix - if s.inReplaceErr { + if s.isReplacingErr { endSuffix++ } currentLocation = binlog.Location{ @@ -1350,11 +1352,11 @@ func (s *Syncer) Run(ctx context.Context) (err error) { currentLocation.GTIDSet.Set(ev.GSet) } - if !s.inReplaceErr { - apply, op := s.errOperatorHolder.Apply(&startLocation, ¤tLocation) + if !s.isReplacingErr { + apply, op := s.errOperatorHolder.MatchAndApply(&startLocation, ¤tLocation) if apply { if op == pb.ErrorOp_Replace { - s.inReplaceErr = true + s.isReplacingErr = true // revert currentLocation to startLocation currentLocation = startLocation.Clone() } @@ -1365,7 +1367,7 @@ func (s *Syncer) Run(ctx context.Context) (err error) { // set endLocation.Suffix=0 of last replace event if currentLocation.Suffix > 0 && e.Header.EventSize > 0 { currentLocation.Suffix = 0 - s.inReplaceErr = false + s.isReplacingErr = false } default: } @@ -2678,7 +2680,7 @@ func (s *Syncer) handleEventError(err error, startLocation, endLocation *binlog. // getEvent gets an event from streamerController or errOperatorHolder func (s *Syncer) getEvent(tctx *tcontext.Context, startLocation *binlog.Location) (*replication.BinlogEvent, error) { // next event is a replace event - if s.inReplaceErr { + if s.isReplacingErr { s.tctx.L().Info(fmt.Sprintf("try to get replace event"), zap.Stringer("location", startLocation)) return s.errOperatorHolder.GetEvent(startLocation) }