diff --git a/dm/_utils/terror_gen/errors_release.txt b/dm/_utils/terror_gen/errors_release.txt index 1d732811c0b..fda955674ea 100644 --- a/dm/_utils/terror_gen/errors_release.txt +++ b/dm/_utils/terror_gen/errors_release.txt @@ -184,6 +184,8 @@ ErrOpenAPITaskConfigNotExist,[code=20051:class=config:scope=internal:level=low], ErrConfigCollationCompatibleNotSupport,[code=20052:class=config:scope=internal:level=medium], "Message: collation compatible %s not supported, Workaround: Please check the `collation_compatible` config in task configuration file, which can be set to `loose`/`strict`." ErrConfigInvalidLoadMode,[code=20053:class=config:scope=internal:level=medium], "Message: invalid load mode '%s', Workaround: Please choose a valid value in ['sql', 'loader']" ErrConfigInvalidDuplicateResolution,[code=20054:class=config:scope=internal:level=medium], "Message: invalid load on-duplicate '%s', Workaround: Please choose a valid value in ['replace', 'error', 'ignore']" +ErrConfigValidationMode,[code=20055:class=config:scope=internal:level=high], "Message: invalid validation mode, Workaround: Please check `validation-mode` config in task configuration file." +ErrContinuousValidatorCfgNotFound,[code=20056:class=config:scope=internal:level=medium], "Message: mysql-instance(%d)'s continuous validator config %s not exist, Workaround: Please check the `continuous-validator-config-name` config in task configuration file." 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] diff --git a/dm/dm/common/common.go b/dm/dm/common/common.go index ab4cecfd67a..ecce1e8f4e8 100644 --- a/dm/dm/common/common.go +++ b/dm/dm/common/common.go @@ -60,6 +60,9 @@ var ( // StageSubTaskKeyAdapter is used to store the running stage of the subtask. // k/v: Encode(source-id, task-name) -> the running stage of the subtask. StageSubTaskKeyAdapter KeyAdapter = keyHexEncoderDecoder("/dm-master/stage/subtask/") + // StageValidatorKeyAdapter is used to store the running stage of the validator. + // k/v: Encode(source-id, task-name) -> the running stage of the validator. + StageValidatorKeyAdapter KeyAdapter = keyHexEncoderDecoder("/dm-master/stage/validator/") // ShardDDLPessimismInfoKeyAdapter is used to store shard DDL info in pessimistic model. // k/v: Encode(task-name, source-id) -> shard DDL info. @@ -112,7 +115,7 @@ func keyAdapterKeysLen(s KeyAdapter) int { WorkerKeepAliveKeyAdapter, StageRelayKeyAdapter, UpstreamLastBoundWorkerKeyAdapter, UpstreamRelayWorkerKeyAdapter, OpenAPITaskTemplateKeyAdapter: return 1 - case UpstreamSubTaskKeyAdapter, StageSubTaskKeyAdapter, + case UpstreamSubTaskKeyAdapter, StageSubTaskKeyAdapter, StageValidatorKeyAdapter, ShardDDLPessimismInfoKeyAdapter, ShardDDLPessimismOperationKeyAdapter, ShardDDLOptimismSourceTablesKeyAdapter, LoadTaskKeyAdapter, TaskCliArgsKeyAdapter: return 2 diff --git a/dm/dm/config/subtask.go b/dm/dm/config/subtask.go index 4c63b5d2f33..3bfc2813a5d 100644 --- a/dm/dm/config/subtask.go +++ b/dm/dm/config/subtask.go @@ -240,6 +240,7 @@ type SubTaskConfig struct { MydumperConfig // Mydumper configuration LoaderConfig // Loader configuration SyncerConfig // Syncer configuration + ValidatorCfg ValidatorConfig // compatible with standalone dm unit LogLevel string `toml:"log-level" json:"log-level"` @@ -442,6 +443,9 @@ func (c *SubTaskConfig) Adjust(verifyDecryptPassword bool) error { if err := c.LoaderConfig.adjust(); err != nil { return err } + if err := c.ValidatorCfg.adjust(); err != nil { + return err + } // TODO: check every member // TODO: since we checked here, we could remove other terror like ErrSyncerUnitGenBAList diff --git a/dm/dm/config/subtask_test.go b/dm/dm/config/subtask_test.go index 37f4ca64eaa..46e68d16e47 100644 --- a/dm/dm/config/subtask_test.go +++ b/dm/dm/config/subtask_test.go @@ -18,6 +18,8 @@ import ( . "github.com/pingcap/check" "github.com/pingcap/tidb-tools/pkg/filter" + + "github.com/pingcap/tiflow/dm/pkg/terror" ) func (t *testConfig) TestSubTask(c *C) { @@ -69,6 +71,14 @@ func (t *testConfig) TestSubTask(c *C) { err = cfg.Adjust(true) c.Assert(err, IsNil) + + cfg.ValidatorCfg = ValidatorConfig{Mode: ValidationFast} + err = cfg.Adjust(true) + c.Assert(err, IsNil) + + cfg.ValidatorCfg = ValidatorConfig{Mode: "invalid-mode"} + err = cfg.Adjust(true) + c.Assert(terror.ErrConfigValidationMode.Equal(err), IsTrue) } func (t *testConfig) TestSubTaskAdjustFail(c *C) { diff --git a/dm/dm/config/task.go b/dm/dm/config/task.go index 0ce58d97d55..0a1601aa153 100644 --- a/dm/dm/config/task.go +++ b/dm/dm/config/task.go @@ -56,6 +56,12 @@ const ( StrictCollationCompatible = "strict" ) +const ( + ValidationNone = "none" + ValidationFast = "fast" + ValidationFull = "full" +) + // default config item values. var ( // TaskConfig. @@ -136,6 +142,9 @@ type MySQLInstance struct { Syncer *SyncerConfig `yaml:"syncer"` // SyncerThread is alias for WorkerCount in SyncerConfig, and its priority is higher than WorkerCount SyncerThread int `yaml:"syncer-thread"` + + ContinuousValidatorConfigName string `yaml:"continuous-validator-config-name"` + ContinuousValidator ValidatorConfig `yaml:"-"` } // VerifyAndAdjust does verification on configs, and adjust some configs. @@ -327,6 +336,26 @@ func (m *SyncerConfig) UnmarshalYAML(unmarshal func(interface{}) error) error { return nil } +type ValidatorConfig struct { + Mode string `yaml:"mode" toml:"mode" json:"mode"` +} + +func (v *ValidatorConfig) adjust() error { + if v.Mode == "" { + v.Mode = ValidationNone + } + if v.Mode != ValidationNone && v.Mode != ValidationFast && v.Mode != ValidationFull { + return terror.ErrConfigValidationMode + } + return nil +} + +func defaultValidatorConfig() ValidatorConfig { + return ValidatorConfig{ + Mode: ValidationNone, + } +} + // TaskConfig is the configuration for Task. type TaskConfig struct { *flag.FlagSet `yaml:"-" toml:"-" json:"-"` @@ -377,9 +406,10 @@ type TaskConfig struct { BWList map[string]*filter.Rules `yaml:"black-white-list" toml:"black-white-list" json:"black-white-list"` BAList map[string]*filter.Rules `yaml:"block-allow-list" toml:"block-allow-list" json:"block-allow-list"` - Mydumpers map[string]*MydumperConfig `yaml:"mydumpers" toml:"mydumpers" json:"mydumpers"` - Loaders map[string]*LoaderConfig `yaml:"loaders" toml:"loaders" json:"loaders"` - Syncers map[string]*SyncerConfig `yaml:"syncers" toml:"syncers" json:"syncers"` + Mydumpers map[string]*MydumperConfig `yaml:"mydumpers" toml:"mydumpers" json:"mydumpers"` + Loaders map[string]*LoaderConfig `yaml:"loaders" toml:"loaders" json:"loaders"` + Syncers map[string]*SyncerConfig `yaml:"syncers" toml:"syncers" json:"syncers"` + Validators map[string]*ValidatorConfig `yaml:"validators" toml:"validators" json:"validators"` CleanDumpFile bool `yaml:"clean-dump-file" toml:"clean-dump-file" json:"clean-dump-file"` // deprecated @@ -413,6 +443,7 @@ func NewTaskConfig() *TaskConfig { Mydumpers: make(map[string]*MydumperConfig), Loaders: make(map[string]*LoaderConfig), Syncers: make(map[string]*SyncerConfig), + Validators: make(map[string]*ValidatorConfig), CleanDumpFile: true, CollationCompatible: defaultCollationCompatible, } @@ -470,7 +501,7 @@ func (c *TaskConfig) RawDecode(data string) error { } // find unused items in config. -var configRefPrefixes = []string{"RouteRules", "FilterRules", "ColumnMappingRules", "Mydumper", "Loader", "Syncer", "ExprFilter"} +var configRefPrefixes = []string{"RouteRules", "FilterRules", "ColumnMappingRules", "Mydumper", "Loader", "Syncer", "ExprFilter", "Validator"} const ( routeRulesIdx = iota @@ -480,6 +511,7 @@ const ( loaderIdx syncerIdx exprFilterIdx + validatorIdx ) // adjust adjusts and verifies config. @@ -562,6 +594,12 @@ func (c *TaskConfig) adjust() error { } } + for _, validatorCfg := range c.Validators { + if err := validatorCfg.adjust(); err != nil { + return err + } + } + instanceIDs := make(map[string]int) // source-id -> instance-index globalConfigReferCount := map[string]int{} duplicateErrorStrings := make([]string, 0) @@ -684,6 +722,16 @@ func (c *TaskConfig) adjust() error { inst.Syncer.WorkerCount = inst.SyncerThread } + inst.ContinuousValidator = defaultValidatorConfig() + if inst.ContinuousValidatorConfigName != "" { + rule, ok := c.Validators[inst.ContinuousValidatorConfigName] + if !ok { + return terror.ErrContinuousValidatorCfgNotFound.Generate(i, inst.ContinuousValidatorConfigName) + } + globalConfigReferCount[configRefPrefixes[validatorIdx]+inst.ContinuousValidatorConfigName]++ + inst.ContinuousValidator = *rule + } + // for backward compatible, set global config `ansi-quotes: true` if any syncer is true if inst.Syncer.EnableANSIQuotes { log.L().Warn("DM could discover proper ANSI_QUOTES, `enable-ansi-quotes` is no longer take effect") @@ -755,6 +803,11 @@ func (c *TaskConfig) adjust() error { unusedConfigs = append(unusedConfigs, exprFilter) } } + for key := range c.Validators { + if globalConfigReferCount[configRefPrefixes[validatorIdx]+key] == 0 { + unusedConfigs = append(unusedConfigs, key) + } + } if len(unusedConfigs) != 0 { sort.Strings(unusedConfigs) diff --git a/dm/dm/config/task_converters.go b/dm/dm/config/task_converters.go index 99c3c8ef7a9..a5ab3610dea 100644 --- a/dm/dm/config/task_converters.go +++ b/dm/dm/config/task_converters.go @@ -93,6 +93,7 @@ func TaskConfigToSubTaskConfigs(c *TaskConfig, sources map[string]DBConfig) ([]* cfg.MydumperConfig = *inst.Mydumper cfg.LoaderConfig = *inst.Loader cfg.SyncerConfig = *inst.Syncer + cfg.ValidatorCfg = inst.ContinuousValidator cfg.CleanDumpFile = c.CleanDumpFile @@ -205,6 +206,7 @@ func OpenAPITaskToSubTaskConfigs(task *openapi.Task, toDBCfg *DBConfig, sourceCf subTaskCfg.SyncerConfig.Batch = *incrCfg.ReplBatch } } + subTaskCfg.ValidatorCfg = defaultValidatorConfig() // set route,blockAllowList,filter config doCnt := len(tableMigrateRuleMap[sourceCfg.SourceName]) doDBs := make([]string, doCnt) @@ -302,6 +304,7 @@ func SubTaskConfigsToTaskConfig(stCfgs ...*SubTaskConfig) *TaskConfig { c.Syncers = make(map[string]*SyncerConfig) c.ExprFilter = make(map[string]*ExpressionFilter) c.Experimental = stCfg0.Experimental + c.Validators = make(map[string]*ValidatorConfig) baListMap := make(map[string]string, len(stCfgs)) routeMap := make(map[string]string, len(stCfgs)) @@ -311,8 +314,9 @@ func SubTaskConfigsToTaskConfig(stCfgs ...*SubTaskConfig) *TaskConfig { syncMap := make(map[string]string, len(stCfgs)) cmMap := make(map[string]string, len(stCfgs)) exprFilterMap := make(map[string]string, len(stCfgs)) - var baListIdx, routeIdx, filterIdx, dumpIdx, loadIdx, syncIdx, cmIdx, efIdx int - var baListName, routeName, filterName, dumpName, loadName, syncName, cmName, efName string + validatorMap := make(map[string]string, len(stCfgs)) + var baListIdx, routeIdx, filterIdx, dumpIdx, loadIdx, syncIdx, validateIdx, cmIdx, efIdx int + var baListName, routeName, filterName, dumpName, loadName, syncName, validateName, cmName, efName string // NOTE: // - we choose to ref global configs for instances now. @@ -354,6 +358,9 @@ func SubTaskConfigsToTaskConfig(stCfgs ...*SubTaskConfig) *TaskConfig { c.ExprFilter[efName] = f } + validateName, validateIdx = getGenerateName(stCfg.ValidatorCfg, validateIdx, "validator", validatorMap) + c.Validators[validateName] = &stCfg.ValidatorCfg + cmNames := make([]string, 0, len(stCfg.ColumnMappingRules)) for _, rule := range stCfg.ColumnMappingRules { cmName, cmIdx = getGenerateName(rule, cmIdx, "cm", cmMap) @@ -362,16 +369,17 @@ func SubTaskConfigsToTaskConfig(stCfgs ...*SubTaskConfig) *TaskConfig { } c.MySQLInstances = append(c.MySQLInstances, &MySQLInstance{ - SourceID: stCfg.SourceID, - Meta: stCfg.Meta, - FilterRules: filterNames, - ColumnMappingRules: cmNames, - RouteRules: routeNames, - BAListName: baListName, - MydumperConfigName: dumpName, - LoaderConfigName: loadName, - SyncerConfigName: syncName, - ExpressionFilters: exprFilterNames, + SourceID: stCfg.SourceID, + Meta: stCfg.Meta, + FilterRules: filterNames, + ColumnMappingRules: cmNames, + RouteRules: routeNames, + BAListName: baListName, + MydumperConfigName: dumpName, + LoaderConfigName: loadName, + SyncerConfigName: syncName, + ExpressionFilters: exprFilterNames, + ContinuousValidatorConfigName: validateName, }) } if c.CollationCompatible == "" { diff --git a/dm/dm/config/task_test.go b/dm/dm/config/task_test.go index 71571d8c455..8a1de3f019f 100644 --- a/dm/dm/config/task_test.go +++ b/dm/dm/config/task_test.go @@ -623,6 +623,7 @@ func (t *testConfig) TestGenAndFromSubTaskConfigs(c *C) { Table: "tbl", DeleteValueExpr: "state = 1", } + validatorCfg = ValidatorConfig{Mode: ValidationNone} source1DBCfg = DBConfig{ Host: "127.0.0.1", Port: 3306, @@ -705,6 +706,7 @@ func (t *testConfig) TestGenAndFromSubTaskConfigs(c *C) { EnableGTID: true, SafeMode: true, }, + ValidatorCfg: validatorCfg, CleanDumpFile: true, EnableANSIQuotes: true, } @@ -723,6 +725,7 @@ func (t *testConfig) TestGenAndFromSubTaskConfigs(c *C) { stCfg2.BAList = &baList2 stCfg2.RouteRules = []*router.TableRule{&routeRule4, &routeRule1, &routeRule2} stCfg2.ExprFilter = []*ExpressionFilter{&exprFilter1} + stCfg2.ValidatorCfg.Mode = ValidationFast cfg := SubTaskConfigsToTaskConfig(stCfg1, stCfg2) @@ -757,6 +760,8 @@ func (t *testConfig) TestGenAndFromSubTaskConfigs(c *C) { SyncerConfigName: "sync-01", Syncer: nil, SyncerThread: 0, + + ContinuousValidatorConfigName: "validator-01", }, { SourceID: source2, @@ -776,6 +781,8 @@ func (t *testConfig) TestGenAndFromSubTaskConfigs(c *C) { Syncer: nil, SyncerThread: 0, ExpressionFilters: []string{"expr-filter-01"}, + + ContinuousValidatorConfigName: "validator-02", }, }, OnlineDDL: onlineDDL, @@ -807,6 +814,12 @@ func (t *testConfig) TestGenAndFromSubTaskConfigs(c *C) { ExprFilter: map[string]*ExpressionFilter{ "expr-filter-01": &exprFilter1, }, + Validators: map[string]*ValidatorConfig{ + "validator-01": &validatorCfg, + "validator-02": { + Mode: ValidationFast, + }, + }, CleanDumpFile: stCfg1.CleanDumpFile, } cfg2.Experimental.AsyncCheckpointFlush = true @@ -1031,7 +1044,7 @@ func (t *testConfig) TestTaskConfigForDowngrade(c *C) { // make sure all new field were added cfgReflect := reflect.Indirect(reflect.ValueOf(cfg)) cfgForDowngradeReflect := reflect.Indirect(reflect.ValueOf(cfgForDowngrade)) - c.Assert(cfgReflect.NumField(), Equals, cfgForDowngradeReflect.NumField()+3) // without flag, collation_compatible and experimental + c.Assert(cfgReflect.NumField(), Equals, cfgForDowngradeReflect.NumField()+4) // without flag, collation_compatible, experimental, validator // make sure all field were copied cfgForClone := &TaskConfigForDowngrade{} diff --git a/dm/dm/master/scheduler/scheduler.go b/dm/dm/master/scheduler/scheduler.go index c0784c57392..435b4d1b145 100644 --- a/dm/dm/master/scheduler/scheduler.go +++ b/dm/dm/master/scheduler/scheduler.go @@ -175,6 +175,17 @@ type Scheduler struct { // - stop-relay relayWorkers map[string]map[string]struct{} + // expectant validator stages, task name -> source ID -> stage. + // add: + // - on subtask start with validator mode not none + // - start validator manually + // - recover from etcd + // update + // - update stage by user request + // delete: + // - when subtask is removed by user request + expectValidatorStages sync.Map + // workers in load stage // task -> source -> worker loadTasks map[string]map[string]string @@ -897,6 +908,7 @@ func (s *Scheduler) AddSubTasks(latched bool, cfgs ...config.SubTaskConfig) erro // 2. construct `Running` stages when adding. newCfgs := make([]config.SubTaskConfig, 0, len(cfgs)-len(existSources)) newStages := make([]ha.Stage, 0, cap(newCfgs)) + validatorStages := make([]ha.Stage, 0, cap(newCfgs)) unbounds := make([]string, 0) for _, cfg := range cfgs { if _, ok := existSourcesM[cfg.SourceID]; ok { @@ -904,6 +916,9 @@ func (s *Scheduler) AddSubTasks(latched bool, cfgs ...config.SubTaskConfig) erro } newCfgs = append(newCfgs, cfg) newStages = append(newStages, ha.NewSubTaskStage(pb.Stage_Running, cfg.SourceID, cfg.Name)) + if cfg.ValidatorCfg.Mode != config.ValidationNone { + validatorStages = append(validatorStages, ha.NewValidatorStage(pb.Stage_Running, cfg.SourceID, cfg.Name)) + } if _, ok := s.bounds[cfg.SourceID]; !ok { unbounds = append(unbounds, cfg.SourceID) } @@ -915,7 +930,7 @@ func (s *Scheduler) AddSubTasks(latched bool, cfgs ...config.SubTaskConfig) erro } // 4. put the configs and stages into etcd. - _, err := ha.PutSubTaskCfgStage(s.etcdCli, newCfgs, newStages) + _, err := ha.PutSubTaskCfgStage(s.etcdCli, newCfgs, newStages, validatorStages) if err != nil { return err } @@ -931,6 +946,11 @@ func (s *Scheduler) AddSubTasks(latched bool, cfgs ...config.SubTaskConfig) erro m := v.(map[string]ha.Stage) m[stage.Source] = stage } + for _, stage := range validatorStages { + v, _ := s.expectValidatorStages.LoadOrStore(stage.Task, map[string]ha.Stage{}) + m := v.(map[string]ha.Stage) + m[stage.Source] = stage + } return nil } @@ -958,11 +978,17 @@ func (s *Scheduler) RemoveSubTasks(task string, sources ...string) error { return terror.ErrSchedulerSubTaskOpTaskNotExist.Generate(task) } + var validatorStageM map[string]ha.Stage + if validatorStageV, ok := s.expectValidatorStages.Load(task); ok { + validatorStageM = validatorStageV.(map[string]ha.Stage) + } + var ( stagesM = stagesMapV.(map[string]ha.Stage) cfgsM = cfgsMapV.(map[string]config.SubTaskConfig) notExistSourcesM = make(map[string]struct{}) stages = make([]ha.Stage, 0, len(sources)) + validatorStages = make([]ha.Stage, 0, len(sources)) cfgs = make([]config.SubTaskConfig, 0, len(sources)) ) for _, source := range sources { @@ -971,6 +997,9 @@ func (s *Scheduler) RemoveSubTasks(task string, sources ...string) error { } else { stages = append(stages, stage) } + if stage, ok := validatorStageM[source]; ok { + validatorStages = append(validatorStages, stage) + } if cfg, ok := cfgsM[source]; ok { cfgs = append(cfgs, cfg) } @@ -982,7 +1011,7 @@ func (s *Scheduler) RemoveSubTasks(task string, sources ...string) error { } // 2. delete the configs and the stages. - _, err = ha.DeleteSubTaskCfgStage(s.etcdCli, cfgs, stages) + _, err = ha.DeleteSubTaskCfgStage(s.etcdCli, cfgs, stages, validatorStages) if err != nil { return err } @@ -1000,6 +1029,12 @@ func (s *Scheduler) RemoveSubTasks(task string, sources ...string) error { if len(stagesM) == 0 { s.expectSubTaskStages.Delete(task) } + for _, stage := range validatorStages { + delete(validatorStageM, stage.Source) + } + if len(validatorStageM) == 0 { + s.expectValidatorStages.Delete(task) + } return nil } @@ -1661,6 +1696,10 @@ func (s *Scheduler) recoverSubTasks() error { if err != nil { return err } + validatorStageMM, _, err := ha.GetAllValidatorStage(s.etcdCli) + if err != nil { + return err + } // recover in-memory data. for source, cfgM := range cfgMM { @@ -1677,6 +1716,13 @@ func (s *Scheduler) recoverSubTasks() error { m[source] = stage } } + for source, stageM := range validatorStageMM { + for task, stage := range stageM { + v, _ := s.expectValidatorStages.LoadOrStore(task, map[string]ha.Stage{}) + m := v.(map[string]ha.Stage) + m[source] = stage + } + } return nil } diff --git a/dm/dm/master/scheduler/scheduler_test.go b/dm/dm/master/scheduler/scheduler_test.go index e9eb1627e6d..0f775c816b4 100644 --- a/dm/dm/master/scheduler/scheduler_test.go +++ b/dm/dm/master/scheduler/scheduler_test.go @@ -131,6 +131,7 @@ func (t *testScheduler) testSchedulerProgress(c *C, restart int) { c.Assert(subtaskCfg21.Adjust(true), IsNil) subtaskCfg22 := subtaskCfg21 subtaskCfg22.SourceID = sourceID2 + subtaskCfg22.ValidatorCfg = config.ValidatorConfig{Mode: config.ValidationFast} c.Assert(subtaskCfg22.Adjust(true), IsNil) // not started scheduler can't do anything. @@ -425,6 +426,7 @@ func (t *testScheduler) testSchedulerProgress(c *C, restart int) { t.subTaskCfgExist(c, s, subtaskCfg22) t.subTaskStageMatch(c, s, taskName2, sourceID1, pb.Stage_Running) t.subTaskStageMatch(c, s, taskName2, sourceID2, pb.Stage_Running) + t.validatorStageMatch(c, s, taskName2, sourceID2, pb.Stage_Running) rebuildScheduler(ctx) // CASE 4.4.2 fail to stop any task. @@ -471,6 +473,7 @@ func (t *testScheduler) testSchedulerProgress(c *C, restart int) { t.subTaskCfgNotExist(c, s, taskName2, sourceID2) t.subTaskStageMatch(c, s, taskName2, sourceID1, pb.Stage_InvalidStage) t.subTaskStageMatch(c, s, taskName2, sourceID2, pb.Stage_InvalidStage) + t.validatorStageMatch(c, s, taskName2, sourceID2, pb.Stage_InvalidStage) rebuildScheduler(ctx) // CASE 4.7: remove source2. @@ -768,6 +771,29 @@ func (t *testScheduler) subTaskStageMatch(c *C, s *Scheduler, task, source strin } } +func (t *testScheduler) validatorStageMatch(c *C, s *Scheduler, task, source string, expectStage pb.Stage) { + stage := ha.NewValidatorStage(expectStage, source, task) + var m map[string]ha.Stage + if v, ok := s.expectValidatorStages.Load(task); ok { + m = v.(map[string]ha.Stage) + } + if expectStage == pb.Stage_InvalidStage { + _, ok := m[source] + c.Assert(ok, IsFalse) + } else { + stageDeepEqualExcludeRev(c, m[source], stage) + } + stageM, _, err := ha.GetValidatorStage(etcdTestCli, source, task, 0) + c.Assert(err, IsNil) + switch expectStage { + case pb.Stage_Running, pb.Stage_Stopped: + c.Assert(stageM, HasLen, 1) + stageDeepEqualExcludeRev(c, stageM[task], stage) + default: + c.Assert(stageM, HasLen, 0) + } +} + func (t *testScheduler) TestRestartScheduler(c *C) { defer clearTestInfoOperation(c) diff --git a/dm/dm/pb/dmworker.pb.go b/dm/dm/pb/dmworker.pb.go index fe9b587c7a4..f52701d0f71 100644 --- a/dm/dm/pb/dmworker.pb.go +++ b/dm/dm/pb/dmworker.pb.go @@ -307,6 +307,34 @@ func (ErrorOp) EnumDescriptor() ([]byte, []int) { return fileDescriptor_51a1b9e17fd67b10, []int{6} } +type ValidatorOp int32 + +const ( + ValidatorOp_InvalidValidatorOp ValidatorOp = 0 + ValidatorOp_StartValidator ValidatorOp = 1 + ValidatorOp_StopValidator ValidatorOp = 2 +) + +var ValidatorOp_name = map[int32]string{ + 0: "InvalidValidatorOp", + 1: "StartValidator", + 2: "StopValidator", +} + +var ValidatorOp_value = map[string]int32{ + "InvalidValidatorOp": 0, + "StartValidator": 1, + "StopValidator": 2, +} + +func (x ValidatorOp) String() string { + return proto.EnumName(ValidatorOp_name, int32(x)) +} + +func (ValidatorOp) EnumDescriptor() ([]byte, []int) { + return fileDescriptor_51a1b9e17fd67b10, []int{7} +} + type QueryStatusRequest struct { Name string `protobuf:"bytes,1,opt,name=name,proto3" json:"name,omitempty"` } @@ -2567,6 +2595,7 @@ func init() { proto.RegisterEnum("pb.SchemaOp", SchemaOp_name, SchemaOp_value) proto.RegisterEnum("pb.V1MetaOp", V1MetaOp_name, V1MetaOp_value) proto.RegisterEnum("pb.ErrorOp", ErrorOp_name, ErrorOp_value) + proto.RegisterEnum("pb.ValidatorOp", ValidatorOp_name, ValidatorOp_value) proto.RegisterType((*QueryStatusRequest)(nil), "pb.QueryStatusRequest") proto.RegisterType((*CommonWorkerResponse)(nil), "pb.CommonWorkerResponse") proto.RegisterType((*QueryStatusResponse)(nil), "pb.QueryStatusResponse") @@ -2604,140 +2633,142 @@ func init() { func init() { proto.RegisterFile("dmworker.proto", fileDescriptor_51a1b9e17fd67b10) } var fileDescriptor_51a1b9e17fd67b10 = []byte{ - // 2124 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xb4, 0x58, 0x4f, 0x73, 0xdc, 0x4a, - 0x11, 0x5f, 0xad, 0x76, 0xd7, 0xbb, 0xbd, 0x6b, 0x47, 0x99, 0x24, 0x8f, 0xc5, 0x84, 0xc5, 0xa5, - 0xbc, 0x0a, 0xc6, 0x45, 0xb9, 0x5e, 0xcc, 0xa3, 0x1e, 0xf5, 0xaa, 0x80, 0x47, 0xec, 0x3c, 0x27, - 0xe0, 0xe0, 0x44, 0x76, 0xc2, 0x91, 0x92, 0xa5, 0xf1, 0x5a, 0x58, 0x2b, 0x29, 0x9a, 0x91, 0x5d, - 0x7b, 0xa0, 0xf8, 0x08, 0x70, 0xe1, 0x00, 0xc5, 0x95, 0xeb, 0x3b, 0xf2, 0x11, 0x80, 0x63, 0x8a, - 0x2a, 0xaa, 0x38, 0x52, 0xc9, 0xd7, 0xe0, 0x40, 0x75, 0xcf, 0x48, 0x1a, 0xd9, 0xbb, 0x09, 0x39, - 0x70, 0x53, 0xff, 0xba, 0xa7, 0xbb, 0xa7, 0xa7, 0xff, 0xcc, 0x08, 0xd6, 0xc2, 0xd9, 0x65, 0x9a, - 0x9f, 0xf3, 0x7c, 0x3b, 0xcb, 0x53, 0x99, 0xb2, 0x76, 0x76, 0xe2, 0x6e, 0x02, 0x7b, 0x5e, 0xf0, - 0x7c, 0x7e, 0x24, 0x7d, 0x59, 0x08, 0x8f, 0xbf, 0x2a, 0xb8, 0x90, 0x8c, 0x41, 0x27, 0xf1, 0x67, - 0x7c, 0x6c, 0x6d, 0x58, 0x9b, 0x03, 0x8f, 0xbe, 0xdd, 0x0c, 0x6e, 0xef, 0xa6, 0xb3, 0x59, 0x9a, - 0xfc, 0x82, 0x74, 0x78, 0x5c, 0x64, 0x69, 0x22, 0x38, 0xfb, 0x08, 0x7a, 0x39, 0x17, 0x45, 0x2c, - 0x49, 0xba, 0xef, 0x69, 0x8a, 0x39, 0x60, 0xcf, 0xc4, 0x74, 0xdc, 0x26, 0x15, 0xf8, 0x89, 0x92, - 0x22, 0x2d, 0xf2, 0x80, 0x8f, 0x6d, 0x02, 0x35, 0x85, 0xb8, 0xf2, 0x6b, 0xdc, 0x51, 0xb8, 0xa2, - 0xdc, 0xaf, 0x2c, 0xb8, 0xd5, 0x70, 0xee, 0x83, 0x2d, 0x7e, 0x0a, 0x23, 0x65, 0x43, 0x69, 0x20, - 0xbb, 0xc3, 0x1d, 0x67, 0x3b, 0x3b, 0xd9, 0x3e, 0x32, 0x70, 0xaf, 0x21, 0xc5, 0x3e, 0x83, 0x55, - 0x51, 0x9c, 0x1c, 0xfb, 0xe2, 0x5c, 0x2f, 0xeb, 0x6c, 0xd8, 0x9b, 0xc3, 0x9d, 0x9b, 0xb4, 0xcc, - 0x64, 0x78, 0x4d, 0x39, 0xf7, 0xcf, 0x16, 0x0c, 0x77, 0xcf, 0x78, 0xa0, 0x69, 0x74, 0x34, 0xf3, - 0x85, 0xe0, 0x61, 0xe9, 0xa8, 0xa2, 0xd8, 0x6d, 0xe8, 0xca, 0x54, 0xfa, 0x31, 0xb9, 0xda, 0xf5, - 0x14, 0xc1, 0x26, 0x00, 0xa2, 0x08, 0x02, 0x2e, 0xc4, 0x69, 0x11, 0x93, 0xab, 0x5d, 0xcf, 0x40, - 0x50, 0xdb, 0xa9, 0x1f, 0xc5, 0x3c, 0xa4, 0x30, 0x75, 0x3d, 0x4d, 0xb1, 0x31, 0xac, 0x5c, 0xfa, - 0x79, 0x12, 0x25, 0xd3, 0x71, 0x97, 0x18, 0x25, 0x89, 0x2b, 0x42, 0x2e, 0xfd, 0x28, 0x1e, 0xf7, - 0x36, 0xac, 0xcd, 0x91, 0xa7, 0x29, 0xf7, 0xb5, 0x05, 0xb0, 0x57, 0xcc, 0x32, 0xed, 0xe6, 0x06, - 0x0c, 0xc9, 0x83, 0x63, 0xff, 0x24, 0xe6, 0x82, 0x7c, 0xb5, 0x3d, 0x13, 0x62, 0x9b, 0x70, 0x23, - 0x48, 0x67, 0x59, 0xcc, 0x25, 0x0f, 0xb5, 0x14, 0xba, 0x6e, 0x79, 0x57, 0x61, 0xf6, 0x31, 0xac, - 0x9e, 0x46, 0x49, 0x24, 0xce, 0x78, 0xf8, 0x70, 0x2e, 0xb9, 0x0a, 0xb9, 0xe5, 0x35, 0x41, 0xe6, - 0xc2, 0xa8, 0x04, 0xbc, 0xf4, 0x52, 0xd0, 0x86, 0x2c, 0xaf, 0x81, 0xb1, 0xef, 0xc2, 0x4d, 0x2e, - 0x64, 0x34, 0xf3, 0x25, 0x3f, 0x46, 0x57, 0x48, 0xb0, 0x4b, 0x82, 0xd7, 0x19, 0xee, 0x5f, 0x2c, - 0x80, 0x83, 0xd4, 0x0f, 0xf5, 0x96, 0xae, 0xb9, 0xa1, 0x36, 0x75, 0xc5, 0x8d, 0x09, 0x00, 0xed, - 0x52, 0x89, 0xb4, 0x49, 0xc4, 0x40, 0xd8, 0x3a, 0xf4, 0xb3, 0x3c, 0x9d, 0xe6, 0x5c, 0x08, 0x9d, - 0xb2, 0x15, 0x8d, 0x6b, 0x67, 0x5c, 0xfa, 0x0f, 0xa3, 0x24, 0x4e, 0xa7, 0x3a, 0x71, 0x0d, 0x84, - 0xdd, 0x87, 0xb5, 0x9a, 0xda, 0x3f, 0x7e, 0xb2, 0x47, 0xbe, 0x0f, 0xbc, 0x2b, 0xa8, 0xfb, 0x7b, - 0x0b, 0x56, 0x8f, 0xce, 0xfc, 0x3c, 0x8c, 0x92, 0xe9, 0x7e, 0x9e, 0x16, 0x19, 0x9e, 0x9a, 0xf4, - 0xf3, 0x29, 0x97, 0xba, 0xfc, 0x34, 0x85, 0x45, 0xb9, 0xb7, 0x77, 0x80, 0x7e, 0xda, 0x58, 0x94, - 0xf8, 0xad, 0xf6, 0x99, 0x0b, 0x79, 0x90, 0x06, 0xbe, 0x8c, 0xd2, 0x44, 0xbb, 0xd9, 0x04, 0xa9, - 0xf0, 0xe6, 0x49, 0x40, 0x99, 0x63, 0x53, 0xe1, 0x11, 0x85, 0xfb, 0x2b, 0x12, 0xcd, 0xe9, 0x12, - 0xa7, 0xa2, 0xdd, 0x7f, 0xda, 0x00, 0x47, 0xf3, 0x24, 0xb8, 0x92, 0x23, 0x8f, 0x2e, 0x78, 0x22, - 0x9b, 0x39, 0xa2, 0x20, 0x54, 0xa6, 0x52, 0x26, 0x2b, 0x43, 0x59, 0xd1, 0xec, 0x2e, 0x0c, 0x72, - 0x1e, 0xf0, 0x44, 0x22, 0xd3, 0x26, 0x66, 0x0d, 0x60, 0x36, 0xcc, 0x7c, 0x21, 0x79, 0xde, 0x08, - 0x66, 0x03, 0x63, 0x5b, 0xe0, 0x98, 0xf4, 0xbe, 0x8c, 0x42, 0x1d, 0xd0, 0x6b, 0x38, 0xea, 0xa3, - 0x4d, 0x94, 0xfa, 0x7a, 0x4a, 0x9f, 0x89, 0xa1, 0x3e, 0x93, 0x26, 0x7d, 0x2b, 0x4a, 0xdf, 0x55, - 0x1c, 0xf5, 0x9d, 0xc4, 0x69, 0x70, 0x1e, 0x25, 0x53, 0x3a, 0x80, 0x3e, 0x85, 0xaa, 0x81, 0xb1, - 0x1f, 0x82, 0x53, 0x24, 0x39, 0x17, 0x69, 0x7c, 0xc1, 0x43, 0x3a, 0x47, 0x31, 0x1e, 0x18, 0x6d, - 0xc3, 0x3c, 0x61, 0xef, 0x9a, 0xa8, 0x71, 0x42, 0xa0, 0x3a, 0x85, 0x3e, 0xa1, 0x09, 0xc0, 0x09, - 0x39, 0x72, 0x3c, 0xcf, 0xf8, 0x78, 0xa8, 0xb2, 0xac, 0x46, 0xd8, 0x27, 0x70, 0x4b, 0xf0, 0x20, - 0x4d, 0x42, 0xf1, 0x90, 0x9f, 0x45, 0x49, 0xf8, 0x94, 0x62, 0x31, 0x1e, 0x51, 0x88, 0x17, 0xb1, - 0xdc, 0x3f, 0x59, 0x30, 0x32, 0x7b, 0x9f, 0xd1, 0x95, 0xad, 0x25, 0x5d, 0xb9, 0x6d, 0x76, 0x65, - 0xf6, 0x9d, 0xaa, 0xfb, 0xaa, 0x6e, 0x4a, 0xfb, 0x7b, 0x96, 0xa7, 0xd8, 0xa6, 0x3c, 0x62, 0x54, - 0x0d, 0xf9, 0x01, 0x0c, 0x73, 0x1e, 0xfb, 0xf3, 0xaa, 0x8d, 0xa2, 0xfc, 0x0d, 0x94, 0xf7, 0x6a, - 0xd8, 0x33, 0x65, 0xdc, 0xbf, 0xb5, 0x61, 0x68, 0x30, 0xaf, 0xe5, 0x86, 0xf5, 0x3f, 0xe6, 0x46, - 0x7b, 0x49, 0x6e, 0x6c, 0x94, 0x2e, 0x15, 0x27, 0x7b, 0x51, 0xae, 0xcb, 0xc5, 0x84, 0x2a, 0x89, - 0x46, 0x32, 0x9a, 0x10, 0x76, 0x43, 0x83, 0x34, 0x52, 0xf1, 0x2a, 0xcc, 0xb6, 0x81, 0x11, 0xb4, - 0xeb, 0xcb, 0xe0, 0xec, 0x45, 0xa6, 0x4f, 0xa7, 0x47, 0x47, 0xbc, 0x80, 0xc3, 0xbe, 0x05, 0x5d, - 0x21, 0xfd, 0x29, 0xa7, 0x54, 0x5c, 0xdb, 0x19, 0x50, 0xea, 0x20, 0xe0, 0x29, 0xdc, 0x08, 0x7e, - 0xff, 0x3d, 0xc1, 0x77, 0xff, 0xd3, 0x86, 0xd5, 0xc6, 0xb4, 0x5a, 0x34, 0xd5, 0x6b, 0x8b, 0xed, - 0x25, 0x16, 0x37, 0xa0, 0x53, 0x24, 0x91, 0x3a, 0xec, 0xb5, 0x9d, 0x11, 0xf2, 0x5f, 0x24, 0x91, - 0xc4, 0xec, 0xf3, 0x88, 0x63, 0xf8, 0xd4, 0x79, 0x5f, 0x42, 0x7c, 0x02, 0xb7, 0xea, 0xd4, 0xdf, - 0xdb, 0x3b, 0x38, 0x48, 0x83, 0xf3, 0xaa, 0x33, 0x2e, 0x62, 0x31, 0xa6, 0x66, 0x3a, 0x95, 0xf0, - 0xe3, 0x96, 0x9a, 0xea, 0xdf, 0x86, 0x6e, 0x80, 0x53, 0x96, 0xa2, 0xa4, 0x13, 0xca, 0x18, 0xbb, - 0x8f, 0x5b, 0x9e, 0xe2, 0xb3, 0x8f, 0xa1, 0x13, 0x16, 0xb3, 0x4c, 0xc7, 0x6a, 0x0d, 0xe5, 0xea, - 0xb1, 0xf7, 0xb8, 0xe5, 0x11, 0x17, 0xa5, 0xe2, 0xd4, 0x0f, 0xc7, 0x83, 0x5a, 0xaa, 0x9e, 0x24, - 0x28, 0x85, 0x5c, 0x94, 0xc2, 0x9a, 0xa4, 0xfa, 0xd4, 0x52, 0x75, 0x7b, 0x44, 0x29, 0xe4, 0x3e, - 0xec, 0x43, 0x4f, 0xa8, 0x44, 0xfe, 0x11, 0xdc, 0x6c, 0x44, 0xff, 0x20, 0x12, 0x14, 0x2a, 0xc5, - 0x1e, 0x5b, 0xcb, 0xae, 0x14, 0xe5, 0xfa, 0x09, 0x00, 0xed, 0xe9, 0x51, 0x9e, 0xa7, 0x79, 0x79, - 0xb5, 0xb1, 0xaa, 0xab, 0x8d, 0xfb, 0x4d, 0x18, 0xe0, 0x5e, 0xde, 0xc1, 0xc6, 0x4d, 0x2c, 0x63, - 0x67, 0x30, 0x22, 0xef, 0x9f, 0x1f, 0x2c, 0x91, 0x60, 0x3b, 0x70, 0x5b, 0xdd, 0x2f, 0x54, 0x3a, - 0x3f, 0x4b, 0x45, 0x44, 0x03, 0x46, 0x15, 0xd6, 0x42, 0x1e, 0x8e, 0x00, 0x8e, 0xea, 0x8e, 0x9e, - 0x1f, 0x94, 0xf3, 0xb2, 0xa4, 0xdd, 0xef, 0xc3, 0x00, 0x2d, 0x2a, 0x73, 0x9b, 0xd0, 0x23, 0x46, - 0x19, 0x07, 0xa7, 0x0a, 0xa7, 0x76, 0xc8, 0xd3, 0x7c, 0xf7, 0xb7, 0x16, 0x0c, 0x55, 0xbb, 0x52, - 0x2b, 0x3f, 0xb4, 0x5b, 0x6d, 0x34, 0x96, 0x97, 0xf5, 0x6e, 0x6a, 0xdc, 0x06, 0xa0, 0x86, 0xa3, - 0x04, 0x3a, 0xf5, 0xf1, 0xd6, 0xa8, 0x67, 0x48, 0xe0, 0xc1, 0xd4, 0xd4, 0x82, 0xd0, 0xfe, 0xa1, - 0x0d, 0x23, 0x7d, 0xa4, 0x4a, 0xe4, 0xff, 0x54, 0x76, 0xba, 0x32, 0x3a, 0x66, 0x65, 0xdc, 0x2f, - 0x2b, 0xa3, 0x5b, 0x6f, 0xa3, 0xce, 0xa2, 0xba, 0x30, 0xee, 0xe9, 0xc2, 0xe8, 0x91, 0xd8, 0x6a, - 0x59, 0x18, 0xa5, 0x94, 0xaa, 0x8b, 0x7b, 0xba, 0x2e, 0x56, 0x6a, 0xa1, 0x2a, 0xa5, 0xaa, 0xb2, - 0xb8, 0xa7, 0xcb, 0xa2, 0x5f, 0x0b, 0x55, 0xc7, 0x5c, 0x55, 0xc5, 0x0a, 0x74, 0xe9, 0x38, 0xdd, - 0xcf, 0xc1, 0x31, 0x43, 0x43, 0x35, 0x71, 0x5f, 0x33, 0x1b, 0xa9, 0x60, 0x08, 0x79, 0x7a, 0xed, - 0x2b, 0x58, 0x6d, 0x34, 0x15, 0x9c, 0x8d, 0x91, 0xd8, 0xf5, 0x93, 0x80, 0xc7, 0xd5, 0x0d, 0xdb, - 0x40, 0x8c, 0x24, 0x6b, 0xd7, 0x9a, 0xb5, 0x8a, 0x46, 0x92, 0x19, 0xf7, 0x64, 0xbb, 0x71, 0x4f, - 0xfe, 0x87, 0x05, 0x23, 0x73, 0x01, 0x5e, 0xb5, 0x1f, 0xe5, 0xf9, 0x6e, 0x1a, 0xaa, 0xd3, 0xec, - 0x7a, 0x25, 0x89, 0xa9, 0x8f, 0x9f, 0xb1, 0x2f, 0x84, 0xce, 0xc0, 0x8a, 0xd6, 0xbc, 0xa3, 0x20, - 0xcd, 0xca, 0x97, 0x4f, 0x45, 0x6b, 0xde, 0x01, 0xbf, 0xe0, 0xb1, 0x1e, 0x35, 0x15, 0x8d, 0xd6, - 0x9e, 0x72, 0x21, 0x30, 0x4d, 0x54, 0x87, 0x2c, 0x49, 0x5c, 0xe5, 0xf9, 0x97, 0xbb, 0x7e, 0x21, - 0xb8, 0xbe, 0xdd, 0x54, 0x34, 0x86, 0x05, 0x5f, 0x68, 0x7e, 0x9e, 0x16, 0x49, 0x79, 0xa7, 0x31, - 0x10, 0xf7, 0x12, 0x6e, 0x3e, 0x2b, 0xf2, 0x29, 0xa7, 0x24, 0x2e, 0x1f, 0x7c, 0xeb, 0xd0, 0x8f, - 0x12, 0x3f, 0x90, 0xd1, 0x05, 0xd7, 0x91, 0xac, 0x68, 0xcc, 0x5f, 0x19, 0xcd, 0xb8, 0xbe, 0xd4, - 0xd1, 0x37, 0xca, 0x9f, 0x46, 0x31, 0xa7, 0xbc, 0xd6, 0x5b, 0x2a, 0x69, 0x2a, 0x51, 0x35, 0x5d, - 0xf5, 0x73, 0x4e, 0x51, 0xee, 0x1f, 0xdb, 0xb0, 0x7e, 0x98, 0xf1, 0xdc, 0x97, 0x5c, 0x3d, 0x21, - 0x8f, 0x82, 0x33, 0x3e, 0xf3, 0x4b, 0x17, 0xee, 0x42, 0x3b, 0xcd, 0xc8, 0xb8, 0xce, 0x77, 0xc5, - 0x3e, 0xcc, 0xbc, 0x76, 0x9a, 0x91, 0x13, 0xbe, 0x38, 0xd7, 0xb1, 0xa5, 0xef, 0xa5, 0xef, 0xc9, - 0x75, 0xe8, 0x87, 0xbe, 0xf4, 0x4f, 0x7c, 0xc1, 0xcb, 0x98, 0x96, 0x34, 0x3d, 0xbd, 0xf0, 0xa5, - 0xa2, 0x23, 0xaa, 0x08, 0xd2, 0x44, 0xd6, 0x74, 0x34, 0x35, 0x85, 0xd2, 0xa7, 0x71, 0x21, 0xce, - 0x28, 0x8c, 0x7d, 0x4f, 0x11, 0xe8, 0x4b, 0x95, 0xf3, 0x7d, 0x95, 0xe2, 0x18, 0xf5, 0xd3, 0x3c, - 0x9d, 0xa9, 0xc6, 0x42, 0xa3, 0xa4, 0xef, 0x19, 0x48, 0xc9, 0x3f, 0x56, 0x17, 0x7b, 0xa8, 0xf9, - 0x0a, 0x71, 0x25, 0xac, 0xbe, 0x7c, 0xa0, 0xd3, 0xfe, 0x29, 0x97, 0x3e, 0x5b, 0x37, 0xc2, 0x01, - 0x18, 0x0e, 0xe4, 0xe8, 0x60, 0xbc, 0xb7, 0x7b, 0x94, 0x2d, 0xc7, 0x36, 0x5a, 0x4e, 0x19, 0xc1, - 0x0e, 0xa5, 0x38, 0x7d, 0xbb, 0x9f, 0xc2, 0x6d, 0x7d, 0x22, 0x2f, 0x1f, 0xa0, 0xd5, 0xa5, 0x67, - 0xa1, 0xd8, 0xca, 0xbc, 0xfb, 0x57, 0x0b, 0xee, 0x5c, 0x59, 0xf6, 0xc1, 0x2f, 0xf3, 0xcf, 0xa0, - 0x83, 0x0f, 0xa1, 0xb1, 0x4d, 0xa5, 0x79, 0x0f, 0x6d, 0x2c, 0x54, 0xb9, 0x8d, 0xc4, 0xa3, 0x44, - 0xe6, 0x73, 0x8f, 0x16, 0xac, 0xff, 0x14, 0x06, 0x15, 0x84, 0x7a, 0xcf, 0xf9, 0xbc, 0xec, 0xbe, - 0xe7, 0x7c, 0x8e, 0x77, 0x83, 0x0b, 0x3f, 0x2e, 0x54, 0x68, 0xf4, 0x80, 0x6d, 0x04, 0xd6, 0x53, - 0xfc, 0xcf, 0xdb, 0x3f, 0xb0, 0xdc, 0x5f, 0xc3, 0xf8, 0xb1, 0x9f, 0x84, 0xb1, 0xce, 0x47, 0xd5, - 0x14, 0x74, 0x08, 0xbe, 0x61, 0x84, 0x60, 0x88, 0x5a, 0x88, 0xfb, 0x8e, 0x6c, 0xbc, 0x0b, 0x83, - 0x93, 0x72, 0x1c, 0xea, 0xc0, 0xd7, 0x00, 0xe5, 0xcc, 0xab, 0x58, 0xe8, 0x07, 0x18, 0x7d, 0xbb, - 0x77, 0xe0, 0xd6, 0x3e, 0x97, 0xca, 0xf6, 0xee, 0xe9, 0x54, 0x5b, 0x76, 0x37, 0xe1, 0x76, 0x13, - 0xd6, 0xc1, 0x75, 0xc0, 0x0e, 0x4e, 0xab, 0x51, 0x13, 0x9c, 0x4e, 0xb7, 0x7e, 0x09, 0x3d, 0x95, - 0x15, 0x6c, 0x15, 0x06, 0x4f, 0x92, 0x0b, 0x3f, 0x8e, 0xc2, 0xc3, 0xcc, 0x69, 0xb1, 0x3e, 0x74, - 0x8e, 0x64, 0x9a, 0x39, 0x16, 0x1b, 0x40, 0xf7, 0x19, 0xb6, 0x05, 0xa7, 0xcd, 0x00, 0x7a, 0xd8, - 0x39, 0x67, 0xdc, 0xb1, 0x11, 0x3e, 0x92, 0x7e, 0x2e, 0x9d, 0x0e, 0xc2, 0x2f, 0xb2, 0xd0, 0x97, - 0xdc, 0xe9, 0xb2, 0x35, 0x80, 0x9f, 0x14, 0x32, 0xd5, 0x62, 0xbd, 0xad, 0xdf, 0x90, 0xd8, 0x14, - 0x6d, 0x8f, 0xb4, 0x7e, 0xa2, 0x9d, 0x16, 0x5b, 0x01, 0xfb, 0xe7, 0xfc, 0xd2, 0xb1, 0xd8, 0x10, - 0x56, 0xbc, 0x22, 0x49, 0xa2, 0x64, 0xaa, 0x6c, 0x90, 0xb9, 0xd0, 0xb1, 0x91, 0x81, 0x4e, 0x64, - 0x3c, 0x74, 0x3a, 0x6c, 0x04, 0xfd, 0x2f, 0xf5, 0xdb, 0xdb, 0xe9, 0x22, 0x0b, 0xc5, 0x70, 0x4d, - 0x0f, 0x59, 0x64, 0x10, 0xa9, 0x15, 0xa4, 0x68, 0x15, 0x52, 0xfd, 0xad, 0x43, 0xe8, 0x97, 0x63, - 0x8f, 0xdd, 0x80, 0xa1, 0xf6, 0x01, 0x21, 0xa7, 0x85, 0x9b, 0xa0, 0xe1, 0xe6, 0x58, 0xb8, 0x61, - 0x1c, 0x60, 0x4e, 0x1b, 0xbf, 0x70, 0x4a, 0x39, 0x36, 0x05, 0x61, 0x9e, 0x04, 0x4e, 0x07, 0x05, - 0xa9, 0xdb, 0x39, 0xe1, 0xd6, 0x53, 0x58, 0xa1, 0xcf, 0x43, 0x3c, 0xc4, 0x35, 0xad, 0x4f, 0x23, - 0x4e, 0x0b, 0xe3, 0x88, 0xd6, 0x95, 0xb4, 0x85, 0xf1, 0xa0, 0xed, 0x28, 0xba, 0x8d, 0x2e, 0xa8, - 0xd8, 0x28, 0xc0, 0xde, 0x4a, 0xa0, 0x5f, 0xb6, 0x29, 0x76, 0x0b, 0x6e, 0x94, 0x31, 0xd2, 0x90, - 0x52, 0xb8, 0xcf, 0xa5, 0x02, 0x1c, 0x8b, 0xf4, 0x57, 0x64, 0x1b, 0xc3, 0xea, 0xf1, 0x59, 0x7a, - 0xc1, 0x35, 0x62, 0xa3, 0x45, 0x9c, 0x8a, 0x9a, 0xee, 0xe0, 0x02, 0xa4, 0xe9, 0xef, 0x8a, 0xd3, - 0xdd, 0xfa, 0x02, 0xfa, 0x65, 0x29, 0x1a, 0xf6, 0x4a, 0xa8, 0xb2, 0xa7, 0x00, 0xc7, 0xaa, 0x0d, - 0x68, 0xa4, 0xbd, 0xf5, 0x92, 0x46, 0x18, 0x66, 0xb2, 0x11, 0x00, 0x8d, 0xe8, 0xcc, 0x39, 0x8f, - 0x32, 0x7d, 0xae, 0x3c, 0x8b, 0xfd, 0xa0, 0xca, 0x9d, 0x0b, 0x9e, 0x4b, 0xc7, 0xc6, 0xef, 0x27, - 0xc9, 0xaf, 0x78, 0x80, 0xc9, 0x83, 0xd1, 0x8e, 0x84, 0x74, 0xba, 0x3b, 0x5f, 0xd9, 0xd0, 0x53, - 0x39, 0xcb, 0xbe, 0x80, 0xa1, 0xf1, 0xdb, 0x8e, 0x7d, 0x84, 0xd5, 0x73, 0xfd, 0x27, 0xe3, 0xfa, - 0xd7, 0xae, 0xe1, 0x2a, 0xd1, 0xdd, 0x16, 0xfb, 0x31, 0x40, 0x3d, 0xa3, 0xd8, 0x1d, 0x1a, 0xdc, - 0x57, 0x67, 0xd6, 0xfa, 0x98, 0x6e, 0x37, 0x0b, 0x7e, 0x49, 0xba, 0x2d, 0xf6, 0x33, 0x58, 0xd5, - 0xed, 0x44, 0x45, 0x92, 0x4d, 0x8c, 0x0e, 0xb3, 0x60, 0xfa, 0xbc, 0x53, 0xd9, 0x97, 0x95, 0x32, - 0x15, 0x45, 0x36, 0x5e, 0xd0, 0xae, 0x94, 0x9a, 0xaf, 0x2f, 0x6d, 0x64, 0x6e, 0x8b, 0xed, 0xc3, - 0x50, 0xb5, 0x1b, 0x75, 0x99, 0xb8, 0x8b, 0xb2, 0xcb, 0xfa, 0xcf, 0x3b, 0x1d, 0xda, 0x85, 0x91, - 0xd9, 0x21, 0x18, 0x45, 0x72, 0x41, 0x2b, 0x51, 0x4a, 0x16, 0x35, 0x13, 0xb7, 0xf5, 0x70, 0xfc, - 0xf7, 0x37, 0x13, 0xeb, 0xf5, 0x9b, 0x89, 0xf5, 0xef, 0x37, 0x13, 0xeb, 0x77, 0x6f, 0x27, 0xad, - 0xd7, 0x6f, 0x27, 0xad, 0x7f, 0xbd, 0x9d, 0xb4, 0x4e, 0x7a, 0xf4, 0x7b, 0xf8, 0x7b, 0xff, 0x0d, - 0x00, 0x00, 0xff, 0xff, 0xa7, 0x6e, 0x71, 0xea, 0x30, 0x16, 0x00, 0x00, + // 2154 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xb4, 0x58, 0x4f, 0x6f, 0x1c, 0x4b, + 0x11, 0xdf, 0xd9, 0x7f, 0xde, 0xad, 0x5d, 0x3b, 0x93, 0x4e, 0x5e, 0x58, 0x4c, 0x58, 0xac, 0xc9, + 0x53, 0x30, 0x16, 0xb2, 0x5e, 0xcc, 0x43, 0x0f, 0x3d, 0x09, 0x78, 0xc4, 0xce, 0x73, 0x02, 0x0e, + 0x4e, 0xc6, 0x4e, 0x38, 0xa2, 0xf6, 0x4c, 0x7b, 0x3d, 0x78, 0x76, 0x66, 0x32, 0xdd, 0x63, 0xcb, + 0x07, 0xc4, 0x47, 0x80, 0x0b, 0x07, 0x10, 0x57, 0xae, 0xef, 0xc8, 0x47, 0x00, 0x8e, 0x11, 0x12, + 0x12, 0x47, 0x94, 0x7c, 0x0d, 0x0e, 0xa8, 0xaa, 0x7b, 0x66, 0x7a, 0xec, 0x75, 0x42, 0x0e, 0xdc, + 0xa6, 0x7e, 0x55, 0x5d, 0x55, 0x5d, 0x5d, 0x7f, 0xba, 0x07, 0x56, 0xc2, 0xf9, 0x79, 0x9a, 0x9f, + 0x8a, 0x7c, 0x33, 0xcb, 0x53, 0x95, 0xb2, 0x76, 0x76, 0xe4, 0xad, 0x03, 0x7b, 0x5e, 0x88, 0xfc, + 0xe2, 0x40, 0x71, 0x55, 0x48, 0x5f, 0xbc, 0x2a, 0x84, 0x54, 0x8c, 0x41, 0x37, 0xe1, 0x73, 0x31, + 0x71, 0xd6, 0x9c, 0xf5, 0xa1, 0x4f, 0xdf, 0x5e, 0x06, 0xb7, 0xb7, 0xd3, 0xf9, 0x3c, 0x4d, 0x7e, + 0x41, 0x3a, 0x7c, 0x21, 0xb3, 0x34, 0x91, 0x82, 0xdd, 0x81, 0x7e, 0x2e, 0x64, 0x11, 0x2b, 0x92, + 0x1e, 0xf8, 0x86, 0x62, 0x2e, 0x74, 0xe6, 0x72, 0x36, 0x69, 0x93, 0x0a, 0xfc, 0x44, 0x49, 0x99, + 0x16, 0x79, 0x20, 0x26, 0x1d, 0x02, 0x0d, 0x85, 0xb8, 0xf6, 0x6b, 0xd2, 0xd5, 0xb8, 0xa6, 0xbc, + 0xaf, 0x1c, 0xb8, 0xd5, 0x70, 0xee, 0x83, 0x2d, 0x7e, 0x0a, 0x63, 0x6d, 0x43, 0x6b, 0x20, 0xbb, + 0xa3, 0x2d, 0x77, 0x33, 0x3b, 0xda, 0x3c, 0xb0, 0x70, 0xbf, 0x21, 0xc5, 0x3e, 0x83, 0x65, 0x59, + 0x1c, 0x1d, 0x72, 0x79, 0x6a, 0x96, 0x75, 0xd7, 0x3a, 0xeb, 0xa3, 0xad, 0x9b, 0xb4, 0xcc, 0x66, + 0xf8, 0x4d, 0x39, 0xef, 0xcf, 0x0e, 0x8c, 0xb6, 0x4f, 0x44, 0x60, 0x68, 0x74, 0x34, 0xe3, 0x52, + 0x8a, 0xb0, 0x74, 0x54, 0x53, 0xec, 0x36, 0xf4, 0x54, 0xaa, 0x78, 0x4c, 0xae, 0xf6, 0x7c, 0x4d, + 0xb0, 0x29, 0x80, 0x2c, 0x82, 0x40, 0x48, 0x79, 0x5c, 0xc4, 0xe4, 0x6a, 0xcf, 0xb7, 0x10, 0xd4, + 0x76, 0xcc, 0xa3, 0x58, 0x84, 0x14, 0xa6, 0x9e, 0x6f, 0x28, 0x36, 0x81, 0xa5, 0x73, 0x9e, 0x27, + 0x51, 0x32, 0x9b, 0xf4, 0x88, 0x51, 0x92, 0xb8, 0x22, 0x14, 0x8a, 0x47, 0xf1, 0xa4, 0xbf, 0xe6, + 0xac, 0x8f, 0x7d, 0x43, 0x79, 0xaf, 0x1d, 0x80, 0x9d, 0x62, 0x9e, 0x19, 0x37, 0xd7, 0x60, 0x44, + 0x1e, 0x1c, 0xf2, 0xa3, 0x58, 0x48, 0xf2, 0xb5, 0xe3, 0xdb, 0x10, 0x5b, 0x87, 0x1b, 0x41, 0x3a, + 0xcf, 0x62, 0xa1, 0x44, 0x68, 0xa4, 0xd0, 0x75, 0xc7, 0xbf, 0x0c, 0xb3, 0x8f, 0x61, 0xf9, 0x38, + 0x4a, 0x22, 0x79, 0x22, 0xc2, 0x87, 0x17, 0x4a, 0xe8, 0x90, 0x3b, 0x7e, 0x13, 0x64, 0x1e, 0x8c, + 0x4b, 0xc0, 0x4f, 0xcf, 0x25, 0x6d, 0xc8, 0xf1, 0x1b, 0x18, 0xfb, 0x2e, 0xdc, 0x14, 0x52, 0x45, + 0x73, 0xae, 0xc4, 0x21, 0xba, 0x42, 0x82, 0x3d, 0x12, 0xbc, 0xca, 0xf0, 0xfe, 0xe2, 0x00, 0xec, + 0xa5, 0x3c, 0x34, 0x5b, 0xba, 0xe2, 0x86, 0xde, 0xd4, 0x25, 0x37, 0xa6, 0x00, 0xb4, 0x4b, 0x2d, + 0xd2, 0x26, 0x11, 0x0b, 0x61, 0xab, 0x30, 0xc8, 0xf2, 0x74, 0x96, 0x0b, 0x29, 0x4d, 0xca, 0x56, + 0x34, 0xae, 0x9d, 0x0b, 0xc5, 0x1f, 0x46, 0x49, 0x9c, 0xce, 0x4c, 0xe2, 0x5a, 0x08, 0xbb, 0x0f, + 0x2b, 0x35, 0xb5, 0x7b, 0xf8, 0x64, 0x87, 0x7c, 0x1f, 0xfa, 0x97, 0x50, 0xef, 0xf7, 0x0e, 0x2c, + 0x1f, 0x9c, 0xf0, 0x3c, 0x8c, 0x92, 0xd9, 0x6e, 0x9e, 0x16, 0x19, 0x9e, 0x9a, 0xe2, 0xf9, 0x4c, + 0x28, 0x53, 0x7e, 0x86, 0xc2, 0xa2, 0xdc, 0xd9, 0xd9, 0x43, 0x3f, 0x3b, 0x58, 0x94, 0xf8, 0xad, + 0xf7, 0x99, 0x4b, 0xb5, 0x97, 0x06, 0x5c, 0x45, 0x69, 0x62, 0xdc, 0x6c, 0x82, 0x54, 0x78, 0x17, + 0x49, 0x40, 0x99, 0xd3, 0xa1, 0xc2, 0x23, 0x0a, 0xf7, 0x57, 0x24, 0x86, 0xd3, 0x23, 0x4e, 0x45, + 0x7b, 0xff, 0xec, 0x00, 0x1c, 0x5c, 0x24, 0xc1, 0xa5, 0x1c, 0x79, 0x74, 0x26, 0x12, 0xd5, 0xcc, + 0x11, 0x0d, 0xa1, 0x32, 0x9d, 0x32, 0x59, 0x19, 0xca, 0x8a, 0x66, 0x77, 0x61, 0x98, 0x8b, 0x40, + 0x24, 0x0a, 0x99, 0x1d, 0x62, 0xd6, 0x00, 0x66, 0xc3, 0x9c, 0x4b, 0x25, 0xf2, 0x46, 0x30, 0x1b, + 0x18, 0xdb, 0x00, 0xd7, 0xa6, 0x77, 0x55, 0x14, 0x9a, 0x80, 0x5e, 0xc1, 0x51, 0x1f, 0x6d, 0xa2, + 0xd4, 0xd7, 0xd7, 0xfa, 0x6c, 0x0c, 0xf5, 0xd9, 0x34, 0xe9, 0x5b, 0xd2, 0xfa, 0x2e, 0xe3, 0xa8, + 0xef, 0x28, 0x4e, 0x83, 0xd3, 0x28, 0x99, 0xd1, 0x01, 0x0c, 0x28, 0x54, 0x0d, 0x8c, 0xfd, 0x10, + 0xdc, 0x22, 0xc9, 0x85, 0x4c, 0xe3, 0x33, 0x11, 0xd2, 0x39, 0xca, 0xc9, 0xd0, 0x6a, 0x1b, 0xf6, + 0x09, 0xfb, 0x57, 0x44, 0xad, 0x13, 0x02, 0xdd, 0x29, 0xcc, 0x09, 0x4d, 0x01, 0x8e, 0xc8, 0x91, + 0xc3, 0x8b, 0x4c, 0x4c, 0x46, 0x3a, 0xcb, 0x6a, 0x84, 0x7d, 0x02, 0xb7, 0xa4, 0x08, 0xd2, 0x24, + 0x94, 0x0f, 0xc5, 0x49, 0x94, 0x84, 0x4f, 0x29, 0x16, 0x93, 0x31, 0x85, 0x78, 0x11, 0xcb, 0xfb, + 0x93, 0x03, 0x63, 0xbb, 0xf7, 0x59, 0x5d, 0xd9, 0xb9, 0xa6, 0x2b, 0xb7, 0xed, 0xae, 0xcc, 0xbe, + 0x53, 0x75, 0x5f, 0xdd, 0x4d, 0x69, 0x7f, 0xcf, 0xf2, 0x14, 0xdb, 0x94, 0x4f, 0x8c, 0xaa, 0x21, + 0x3f, 0x80, 0x51, 0x2e, 0x62, 0x7e, 0x51, 0xb5, 0x51, 0x94, 0xbf, 0x81, 0xf2, 0x7e, 0x0d, 0xfb, + 0xb6, 0x8c, 0xf7, 0xb7, 0x36, 0x8c, 0x2c, 0xe6, 0x95, 0xdc, 0x70, 0xfe, 0xc7, 0xdc, 0x68, 0x5f, + 0x93, 0x1b, 0x6b, 0xa5, 0x4b, 0xc5, 0xd1, 0x4e, 0x94, 0x9b, 0x72, 0xb1, 0xa1, 0x4a, 0xa2, 0x91, + 0x8c, 0x36, 0x84, 0xdd, 0xd0, 0x22, 0xad, 0x54, 0xbc, 0x0c, 0xb3, 0x4d, 0x60, 0x04, 0x6d, 0x73, + 0x15, 0x9c, 0xbc, 0xc8, 0xcc, 0xe9, 0xf4, 0xe9, 0x88, 0x17, 0x70, 0xd8, 0xb7, 0xa0, 0x27, 0x15, + 0x9f, 0x09, 0x4a, 0xc5, 0x95, 0xad, 0x21, 0xa5, 0x0e, 0x02, 0xbe, 0xc6, 0xad, 0xe0, 0x0f, 0xde, + 0x13, 0x7c, 0xef, 0x3f, 0x6d, 0x58, 0x6e, 0x4c, 0xab, 0x45, 0x53, 0xbd, 0xb6, 0xd8, 0xbe, 0xc6, + 0xe2, 0x1a, 0x74, 0x8b, 0x24, 0xd2, 0x87, 0xbd, 0xb2, 0x35, 0x46, 0xfe, 0x8b, 0x24, 0x52, 0x98, + 0x7d, 0x3e, 0x71, 0x2c, 0x9f, 0xba, 0xef, 0x4b, 0x88, 0x4f, 0xe0, 0x56, 0x9d, 0xfa, 0x3b, 0x3b, + 0x7b, 0x7b, 0x69, 0x70, 0x5a, 0x75, 0xc6, 0x45, 0x2c, 0xc6, 0xf4, 0x4c, 0xa7, 0x12, 0x7e, 0xdc, + 0xd2, 0x53, 0xfd, 0xdb, 0xd0, 0x0b, 0x70, 0xca, 0x52, 0x94, 0x4c, 0x42, 0x59, 0x63, 0xf7, 0x71, + 0xcb, 0xd7, 0x7c, 0xf6, 0x31, 0x74, 0xc3, 0x62, 0x9e, 0x99, 0x58, 0xad, 0xa0, 0x5c, 0x3d, 0xf6, + 0x1e, 0xb7, 0x7c, 0xe2, 0xa2, 0x54, 0x9c, 0xf2, 0x70, 0x32, 0xac, 0xa5, 0xea, 0x49, 0x82, 0x52, + 0xc8, 0x45, 0x29, 0xac, 0x49, 0xaa, 0x4f, 0x23, 0x55, 0xb7, 0x47, 0x94, 0x42, 0xee, 0xc3, 0x01, + 0xf4, 0xa5, 0x4e, 0xe4, 0x1f, 0xc1, 0xcd, 0x46, 0xf4, 0xf7, 0x22, 0x49, 0xa1, 0xd2, 0xec, 0x89, + 0x73, 0xdd, 0x95, 0xa2, 0x5c, 0x3f, 0x05, 0xa0, 0x3d, 0x3d, 0xca, 0xf3, 0x34, 0x2f, 0xaf, 0x36, + 0x4e, 0x75, 0xb5, 0xf1, 0xbe, 0x09, 0x43, 0xdc, 0xcb, 0x3b, 0xd8, 0xb8, 0x89, 0xeb, 0xd8, 0x19, + 0x8c, 0xc9, 0xfb, 0xe7, 0x7b, 0xd7, 0x48, 0xb0, 0x2d, 0xb8, 0xad, 0xef, 0x17, 0x3a, 0x9d, 0x9f, + 0xa5, 0x32, 0xa2, 0x01, 0xa3, 0x0b, 0x6b, 0x21, 0x0f, 0x47, 0x80, 0x40, 0x75, 0x07, 0xcf, 0xf7, + 0xca, 0x79, 0x59, 0xd2, 0xde, 0xf7, 0x61, 0x88, 0x16, 0xb5, 0xb9, 0x75, 0xe8, 0x13, 0xa3, 0x8c, + 0x83, 0x5b, 0x85, 0xd3, 0x38, 0xe4, 0x1b, 0xbe, 0xf7, 0x5b, 0x07, 0x46, 0xba, 0x5d, 0xe9, 0x95, + 0x1f, 0xda, 0xad, 0xd6, 0x1a, 0xcb, 0xcb, 0x7a, 0xb7, 0x35, 0x6e, 0x02, 0x50, 0xc3, 0xd1, 0x02, + 0xdd, 0xfa, 0x78, 0x6b, 0xd4, 0xb7, 0x24, 0xf0, 0x60, 0x6a, 0x6a, 0x41, 0x68, 0xff, 0xd0, 0x86, + 0xb1, 0x39, 0x52, 0x2d, 0xf2, 0x7f, 0x2a, 0x3b, 0x53, 0x19, 0x5d, 0xbb, 0x32, 0xee, 0x97, 0x95, + 0xd1, 0xab, 0xb7, 0x51, 0x67, 0x51, 0x5d, 0x18, 0xf7, 0x4c, 0x61, 0xf4, 0x49, 0x6c, 0xb9, 0x2c, + 0x8c, 0x52, 0x4a, 0xd7, 0xc5, 0x3d, 0x53, 0x17, 0x4b, 0xb5, 0x50, 0x95, 0x52, 0x55, 0x59, 0xdc, + 0x33, 0x65, 0x31, 0xa8, 0x85, 0xaa, 0x63, 0xae, 0xaa, 0x62, 0x09, 0x7a, 0x74, 0x9c, 0xde, 0xe7, + 0xe0, 0xda, 0xa1, 0xa1, 0x9a, 0xb8, 0x6f, 0x98, 0x8d, 0x54, 0xb0, 0x84, 0x7c, 0xb3, 0xf6, 0x15, + 0x2c, 0x37, 0x9a, 0x0a, 0xce, 0xc6, 0x48, 0x6e, 0xf3, 0x24, 0x10, 0x71, 0x75, 0xc3, 0xb6, 0x10, + 0x2b, 0xc9, 0xda, 0xb5, 0x66, 0xa3, 0xa2, 0x91, 0x64, 0xd6, 0x3d, 0xb9, 0xd3, 0xb8, 0x27, 0xff, + 0xc3, 0x81, 0xb1, 0xbd, 0x00, 0xaf, 0xda, 0x8f, 0xf2, 0x7c, 0x3b, 0x0d, 0xf5, 0x69, 0xf6, 0xfc, + 0x92, 0xc4, 0xd4, 0xc7, 0xcf, 0x98, 0x4b, 0x69, 0x32, 0xb0, 0xa2, 0x0d, 0xef, 0x20, 0x48, 0xb3, + 0xf2, 0xe5, 0x53, 0xd1, 0x86, 0xb7, 0x27, 0xce, 0x44, 0x6c, 0x46, 0x4d, 0x45, 0xa3, 0xb5, 0xa7, + 0x42, 0x4a, 0x4c, 0x13, 0xdd, 0x21, 0x4b, 0x12, 0x57, 0xf9, 0xfc, 0x7c, 0x9b, 0x17, 0x52, 0x98, + 0xdb, 0x4d, 0x45, 0x63, 0x58, 0xf0, 0x85, 0xc6, 0xf3, 0xb4, 0x48, 0xca, 0x3b, 0x8d, 0x85, 0x78, + 0xe7, 0x70, 0xf3, 0x59, 0x91, 0xcf, 0x04, 0x25, 0x71, 0xf9, 0xe0, 0x5b, 0x85, 0x41, 0x94, 0xf0, + 0x40, 0x45, 0x67, 0xc2, 0x44, 0xb2, 0xa2, 0x31, 0x7f, 0x55, 0x34, 0x17, 0xe6, 0x52, 0x47, 0xdf, + 0x28, 0x7f, 0x1c, 0xc5, 0x82, 0xf2, 0xda, 0x6c, 0xa9, 0xa4, 0xa9, 0x44, 0xf5, 0x74, 0x35, 0xcf, + 0x39, 0x4d, 0x79, 0x7f, 0x6c, 0xc3, 0xea, 0x7e, 0x26, 0x72, 0xae, 0x84, 0x7e, 0x42, 0x1e, 0x04, + 0x27, 0x62, 0xce, 0x4b, 0x17, 0xee, 0x42, 0x3b, 0xcd, 0xc8, 0xb8, 0xc9, 0x77, 0xcd, 0xde, 0xcf, + 0xfc, 0x76, 0x9a, 0x91, 0x13, 0x5c, 0x9e, 0x9a, 0xd8, 0xd2, 0xf7, 0xb5, 0xef, 0xc9, 0x55, 0x18, + 0x84, 0x5c, 0xf1, 0x23, 0x2e, 0x45, 0x19, 0xd3, 0x92, 0xa6, 0xa7, 0x17, 0xbe, 0x54, 0x4c, 0x44, + 0x35, 0x41, 0x9a, 0xc8, 0x9a, 0x89, 0xa6, 0xa1, 0x50, 0xfa, 0x38, 0x2e, 0xe4, 0x09, 0x85, 0x71, + 0xe0, 0x6b, 0x02, 0x7d, 0xa9, 0x72, 0x7e, 0xa0, 0x53, 0x1c, 0xa3, 0x7e, 0x9c, 0xa7, 0x73, 0xdd, + 0x58, 0x68, 0x94, 0x0c, 0x7c, 0x0b, 0x29, 0xf9, 0x87, 0xfa, 0x62, 0x0f, 0x35, 0x5f, 0x23, 0x9e, + 0x82, 0xe5, 0x97, 0x0f, 0x4c, 0xda, 0x3f, 0x15, 0x8a, 0xb3, 0x55, 0x2b, 0x1c, 0x80, 0xe1, 0x40, + 0x8e, 0x09, 0xc6, 0x7b, 0xbb, 0x47, 0xd9, 0x72, 0x3a, 0x56, 0xcb, 0x29, 0x23, 0xd8, 0xa5, 0x14, + 0xa7, 0x6f, 0xef, 0x53, 0xb8, 0x6d, 0x4e, 0xe4, 0xe5, 0x03, 0xb4, 0x7a, 0xed, 0x59, 0x68, 0xb6, + 0x36, 0xef, 0xfd, 0xd5, 0x81, 0x8f, 0x2e, 0x2d, 0xfb, 0xe0, 0x97, 0xf9, 0x67, 0xd0, 0xc5, 0x87, + 0xd0, 0xa4, 0x43, 0xa5, 0x79, 0x0f, 0x6d, 0x2c, 0x54, 0xb9, 0x89, 0xc4, 0xa3, 0x44, 0xe5, 0x17, + 0x3e, 0x2d, 0x58, 0xfd, 0x29, 0x0c, 0x2b, 0x08, 0xf5, 0x9e, 0x8a, 0x8b, 0xb2, 0xfb, 0x9e, 0x8a, + 0x0b, 0xbc, 0x1b, 0x9c, 0xf1, 0xb8, 0xd0, 0xa1, 0x31, 0x03, 0xb6, 0x11, 0x58, 0x5f, 0xf3, 0x3f, + 0x6f, 0xff, 0xc0, 0xf1, 0x7e, 0x0d, 0x93, 0xc7, 0x3c, 0x09, 0x63, 0x93, 0x8f, 0xba, 0x29, 0x98, + 0x10, 0x7c, 0xc3, 0x0a, 0xc1, 0x08, 0xb5, 0x10, 0xf7, 0x1d, 0xd9, 0x78, 0x17, 0x86, 0x47, 0xe5, + 0x38, 0x34, 0x81, 0xaf, 0x01, 0xca, 0x99, 0x57, 0xb1, 0x34, 0x0f, 0x30, 0xfa, 0xf6, 0x3e, 0x82, + 0x5b, 0xbb, 0x42, 0x69, 0xdb, 0xdb, 0xc7, 0x33, 0x63, 0xd9, 0x5b, 0x87, 0xdb, 0x4d, 0xd8, 0x04, + 0xd7, 0x85, 0x4e, 0x70, 0x5c, 0x8d, 0x9a, 0xe0, 0x78, 0xb6, 0xf1, 0x4b, 0xe8, 0xeb, 0xac, 0x60, + 0xcb, 0x30, 0x7c, 0x92, 0x9c, 0xf1, 0x38, 0x0a, 0xf7, 0x33, 0xb7, 0xc5, 0x06, 0xd0, 0x3d, 0x50, + 0x69, 0xe6, 0x3a, 0x6c, 0x08, 0xbd, 0x67, 0xd8, 0x16, 0xdc, 0x36, 0x03, 0xe8, 0x63, 0xe7, 0x9c, + 0x0b, 0xb7, 0x83, 0xf0, 0x81, 0xe2, 0xb9, 0x72, 0xbb, 0x08, 0xbf, 0xc8, 0x42, 0xae, 0x84, 0xdb, + 0x63, 0x2b, 0x00, 0x3f, 0x29, 0x54, 0x6a, 0xc4, 0xfa, 0x1b, 0xbf, 0x21, 0xb1, 0x19, 0xda, 0x1e, + 0x1b, 0xfd, 0x44, 0xbb, 0x2d, 0xb6, 0x04, 0x9d, 0x9f, 0x8b, 0x73, 0xd7, 0x61, 0x23, 0x58, 0xf2, + 0x8b, 0x24, 0x89, 0x92, 0x99, 0xb6, 0x41, 0xe6, 0x42, 0xb7, 0x83, 0x0c, 0x74, 0x22, 0x13, 0xa1, + 0xdb, 0x65, 0x63, 0x18, 0x7c, 0x69, 0xde, 0xde, 0x6e, 0x0f, 0x59, 0x28, 0x86, 0x6b, 0xfa, 0xc8, + 0x22, 0x83, 0x48, 0x2d, 0x21, 0x45, 0xab, 0x90, 0x1a, 0x6c, 0xec, 0xc3, 0xa0, 0x1c, 0x7b, 0xec, + 0x06, 0x8c, 0x8c, 0x0f, 0x08, 0xb9, 0x2d, 0xdc, 0x04, 0x0d, 0x37, 0xd7, 0xc1, 0x0d, 0xe3, 0x00, + 0x73, 0xdb, 0xf8, 0x85, 0x53, 0xca, 0xed, 0x50, 0x10, 0x2e, 0x92, 0xc0, 0xed, 0xa2, 0x20, 0x75, + 0x3b, 0x37, 0xdc, 0x78, 0x0a, 0x4b, 0xf4, 0xb9, 0x8f, 0x87, 0xb8, 0x62, 0xf4, 0x19, 0xc4, 0x6d, + 0x61, 0x1c, 0xd1, 0xba, 0x96, 0x76, 0x30, 0x1e, 0xb4, 0x1d, 0x4d, 0xb7, 0xd1, 0x05, 0x1d, 0x1b, + 0x0d, 0x74, 0x36, 0x12, 0x18, 0x94, 0x6d, 0x8a, 0xdd, 0x82, 0x1b, 0x65, 0x8c, 0x0c, 0xa4, 0x15, + 0xee, 0x0a, 0xa5, 0x01, 0xd7, 0x21, 0xfd, 0x15, 0xd9, 0xc6, 0xb0, 0xfa, 0x62, 0x9e, 0x9e, 0x09, + 0x83, 0x74, 0xd0, 0x22, 0x4e, 0x45, 0x43, 0x77, 0x71, 0x01, 0xd2, 0xf4, 0x77, 0xc5, 0xed, 0x6d, + 0x7c, 0x01, 0x83, 0xb2, 0x14, 0x2d, 0x7b, 0x25, 0x54, 0xd9, 0xd3, 0x80, 0xeb, 0xd4, 0x06, 0x0c, + 0xd2, 0xde, 0x78, 0x49, 0x23, 0x0c, 0x33, 0xd9, 0x0a, 0x80, 0x41, 0x4c, 0xe6, 0x9c, 0x46, 0x99, + 0x39, 0x57, 0x91, 0xc5, 0x3c, 0xa8, 0x72, 0xe7, 0x4c, 0xe4, 0xca, 0xed, 0xe0, 0xf7, 0x93, 0xe4, + 0x57, 0x22, 0xc0, 0xe4, 0xc1, 0x68, 0x47, 0x52, 0xb9, 0xbd, 0x8d, 0x3d, 0x18, 0xbd, 0x44, 0x55, + 0x5c, 0x91, 0xee, 0x3b, 0xc0, 0x4a, 0xe7, 0x6a, 0xd4, 0x6d, 0xa1, 0x4d, 0x4a, 0xbc, 0x0a, 0x75, + 0x1d, 0x76, 0x13, 0x96, 0x31, 0xe8, 0x35, 0xd4, 0xde, 0xfa, 0xaa, 0x03, 0x7d, 0x5d, 0x01, 0xec, + 0x0b, 0x18, 0x59, 0x3f, 0x01, 0xd9, 0x1d, 0xac, 0xc5, 0xab, 0xbf, 0x2c, 0x57, 0xbf, 0x76, 0x05, + 0xd7, 0x65, 0xe3, 0xb5, 0xd8, 0x8f, 0x01, 0xea, 0x89, 0xc7, 0x3e, 0xa2, 0x6b, 0xc0, 0xe5, 0x09, + 0xb8, 0x3a, 0xa1, 0xbb, 0xd2, 0x82, 0x1f, 0x9c, 0x5e, 0x8b, 0xfd, 0x0c, 0x96, 0x4d, 0x73, 0xd2, + 0xe7, 0xc2, 0xa6, 0x56, 0xbf, 0x5a, 0x30, 0xcb, 0xde, 0xa9, 0xec, 0xcb, 0x4a, 0x99, 0x3e, 0x13, + 0x36, 0x59, 0xd0, 0xfc, 0xb4, 0x9a, 0xaf, 0x5f, 0xdb, 0x16, 0xbd, 0x16, 0xdb, 0x85, 0x91, 0x6e, + 0x5e, 0xfa, 0x6a, 0x72, 0x17, 0x65, 0xaf, 0xeb, 0x66, 0xef, 0x74, 0x68, 0x1b, 0xc6, 0x76, 0xbf, + 0x61, 0x14, 0xc9, 0x05, 0x8d, 0x49, 0x2b, 0x59, 0xd4, 0x9a, 0xbc, 0xd6, 0xc3, 0xc9, 0xdf, 0xdf, + 0x4c, 0x9d, 0xd7, 0x6f, 0xa6, 0xce, 0xbf, 0xdf, 0x4c, 0x9d, 0xdf, 0xbd, 0x9d, 0xb6, 0x5e, 0xbf, + 0x9d, 0xb6, 0xfe, 0xf5, 0x76, 0xda, 0x3a, 0xea, 0xd3, 0xcf, 0xe6, 0xef, 0xfd, 0x37, 0x00, 0x00, + 0xff, 0xff, 0xf3, 0x6f, 0xde, 0xe1, 0x7e, 0x16, 0x00, 0x00, } // Reference imports to suppress errors if they are not otherwise used. diff --git a/dm/dm/proto/dmworker.proto b/dm/dm/proto/dmworker.proto index 99cc56e0ef4..4f54b1d76cf 100644 --- a/dm/dm/proto/dmworker.proto +++ b/dm/dm/proto/dmworker.proto @@ -383,4 +383,10 @@ message GetWorkerCfgRequest { message GetWorkerCfgResponse { string cfg = 1; -} \ No newline at end of file +} + +enum ValidatorOp { + InvalidValidatorOp = 0; + StartValidator = 1; + StopValidator = 2; +} diff --git a/dm/dm/worker/server_test.go b/dm/dm/worker/server_test.go index f7c007983da..6220ef3f45c 100644 --- a/dm/dm/worker/server_test.go +++ b/dm/dm/worker/server_test.go @@ -171,8 +171,7 @@ func (t *testServer) TestServer(c *C) { subtaskCfg.MydumperPath = mydumperPath sourceCfg := loadSourceConfigWithoutPassword(c) - _, err = ha.PutSubTaskCfgStage(s.etcdClient, []config.SubTaskConfig{subtaskCfg}, - []ha.Stage{ha.NewSubTaskStage(pb.Stage_Running, sourceCfg.SourceID, subtaskCfg.Name)}) + _, err = ha.PutSubTaskCfgStage(s.etcdClient, []config.SubTaskConfig{subtaskCfg}, []ha.Stage{ha.NewSubTaskStage(pb.Stage_Running, sourceCfg.SourceID, subtaskCfg.Name)}, nil) c.Assert(err, IsNil) c.Assert(utils.WaitSomething(30, 100*time.Millisecond, func() bool { diff --git a/dm/dm/worker/source_worker.go b/dm/dm/worker/source_worker.go index f8dbb5b2c10..b322602bc2e 100644 --- a/dm/dm/worker/source_worker.go +++ b/dm/dm/worker/source_worker.go @@ -321,7 +321,8 @@ func (w *SourceWorker) EnableRelay(startBySourceCfg bool) (err error) { w.relayCtx, w.relayCancel = context.WithCancel(w.ctx) // 1. adjust relay starting position, to the earliest of subtasks var subTaskCfgs map[string]config.SubTaskConfig - _, subTaskCfgs, _, err = w.fetchSubTasksAndAdjust() + //nolint:dogsled + _, _, subTaskCfgs, _, err = w.fetchSubTasksAndAdjust() if err != nil { return err } @@ -455,7 +456,7 @@ func (w *SourceWorker) EnableHandleSubtasks() error { // we get the newest subtask stages directly which will omit the subtask stage PUT/DELETE event // because triggering these events is useless now - subTaskStages, subTaskCfgM, revSubTask, err := w.fetchSubTasksAndAdjust() + subTaskStages, validatorStages, subTaskCfgM, revSubTask, err := w.fetchSubTasksAndAdjust() if err != nil { return err } @@ -467,10 +468,14 @@ func (w *SourceWorker) EnableHandleSubtasks() error { if expectStage.IsDeleted { continue } + validatorStage := pb.Stage_InvalidStage + if s, ok := validatorStages[subTaskCfg.Name]; ok { + validatorStage = s.Expect + } w.l.Info("start to create subtask", zap.String("sourceID", subTaskCfg.SourceID), zap.String("task", subTaskCfg.Name)) // "for range" of a map will use same value address, so we'd better not pass value address to other function clone := subTaskCfg - if err2 := w.StartSubTask(&clone, expectStage.Expect, false); err2 != nil { + if err2 := w.StartSubTask(&clone, expectStage.Expect, validatorStage, false); err2 != nil { w.subTaskHolder.closeAllSubTasks() return err2 } @@ -483,6 +488,13 @@ func (w *SourceWorker) EnableHandleSubtasks() error { //nolint:errcheck w.observeSubtaskStage(w.subTaskCtx, w.etcdClient, revSubTask) }() + w.subTaskWg.Add(1) + go func() { + defer w.subTaskWg.Done() + // TODO: handle fatal error from observeValidatorStage + //nolint:errcheck + w.observeValidatorStage(w.subTaskCtx, revSubTask) + }() w.subTaskEnabled.Store(true) w.l.Info("handling subtask enabled") @@ -510,23 +522,23 @@ func (w *SourceWorker) DisableHandleSubtasks() { // fetchSubTasksAndAdjust gets source's subtask stages and configs, adjust some values by worker's config and status // source **must not be empty** -// return map{task name -> subtask stage}, map{task name -> subtask config}, revision, error. -func (w *SourceWorker) fetchSubTasksAndAdjust() (map[string]ha.Stage, map[string]config.SubTaskConfig, int64, error) { +// return map{task name -> subtask stage}, map{task name -> validator stage}, map{task name -> subtask config}, revision, error. +func (w *SourceWorker) fetchSubTasksAndAdjust() (map[string]ha.Stage, map[string]ha.Stage, map[string]config.SubTaskConfig, int64, error) { // we get the newest subtask stages directly which will omit the subtask stage PUT/DELETE event // because triggering these events is useless now - subTaskStages, subTaskCfgM, revSubTask, err := ha.GetSubTaskStageConfig(w.etcdClient, w.cfg.SourceID) + subTaskStages, validatorStages, subTaskCfgM, revSubTask, err := ha.GetSubTaskStageConfig(w.etcdClient, w.cfg.SourceID) if err != nil { - return nil, nil, 0, err + return nil, nil, nil, 0, err } if err = copyConfigFromSourceForEach(subTaskCfgM, w.cfg, w.relayEnabled.Load()); err != nil { - return nil, nil, 0, err + return nil, nil, nil, 0, err } - return subTaskStages, subTaskCfgM, revSubTask, nil + return subTaskStages, validatorStages, subTaskCfgM, revSubTask, nil } // StartSubTask creates a subtask and run it. -func (w *SourceWorker) StartSubTask(cfg *config.SubTaskConfig, expectStage pb.Stage, needLock bool) error { +func (w *SourceWorker) StartSubTask(cfg *config.SubTaskConfig, expectStage, validatorStage pb.Stage, needLock bool) error { if needLock { w.Lock() defer w.Unlock() @@ -563,7 +575,7 @@ func (w *SourceWorker) StartSubTask(cfg *config.SubTaskConfig, expectStage pb.St } w.l.Info("subtask created", zap.Stringer("config", cfg2)) - st.Run(expectStage, w.getRelayWithoutLock()) + st.Run(expectStage, validatorStage, w.getRelayWithoutLock()) return nil } @@ -662,7 +674,7 @@ func (w *SourceWorker) QueryStatus(ctx context.Context, name string) ([]*pb.SubT } func (w *SourceWorker) resetSubtaskStage() (int64, error) { - subTaskStages, subTaskCfgm, revSubTask, err := w.fetchSubTasksAndAdjust() + subTaskStages, _, subTaskCfgm, revSubTask, err := w.fetchSubTasksAndAdjust() if err != nil { return 0, err } @@ -787,7 +799,13 @@ func (w *SourceWorker) operateSubTaskStage(stage ha.Stage, subTaskCfg config.Sub if st := w.subTaskHolder.findSubTask(stage.Task); st == nil { // create the subtask for expected running and paused stage. log.L().Info("start to create subtask", zap.String("sourceID", subTaskCfg.SourceID), zap.String("task", subTaskCfg.Name)) - err := w.StartSubTask(&subTaskCfg, stage.Expect, true) + + expectValidatorStage, err := getExpectValidatorStage(subTaskCfg.ValidatorCfg, w.etcdClient, stage.Source, stage.Task, stage.Revision) + if err != nil { + return opErrTypeBeforeOp, terror.Annotate(err, "fail to get validator stage from etcd") + } + + err = w.StartSubTask(&subTaskCfg, stage.Expect, expectValidatorStage, true) return opErrTypeBeforeOp, err } if stage.Expect == pb.Stage_Running { @@ -957,7 +975,7 @@ func (w *SourceWorker) PurgeRelay(ctx context.Context, req *pb.PurgeRelayRequest uuid := w.relayHolder.Status(nil).RelaySubDir - _, subTaskCfgs, _, err := w.fetchSubTasksAndAdjust() + _, _, subTaskCfgs, _, err := w.fetchSubTasksAndAdjust() if err != nil { return err } @@ -1097,3 +1115,149 @@ func (w *SourceWorker) HandleError(ctx context.Context, req *pb.HandleWorkerErro return st.HandleError(ctx, req, w.getRelayWithoutLock()) } + +func (w *SourceWorker) observeValidatorStage(ctx context.Context, lastUsedRev int64) error { + var wg sync.WaitGroup + + startRevision := lastUsedRev + 1 + for { + stageCh := make(chan ha.Stage, 10) + errCh := make(chan error, 10) + wg.Add(1) + watchCtx, watchCancel := context.WithCancel(ctx) + go func() { + defer func() { + close(stageCh) + close(errCh) + wg.Done() + }() + ha.WatchValidatorStage(watchCtx, w.etcdClient, w.cfg.SourceID, startRevision, stageCh, errCh) + }() + err := w.handleValidatorStage(watchCtx, stageCh, errCh) + watchCancel() + wg.Wait() + + if etcdutil.IsRetryableError(err) { + startRevision = 0 + retryNum := 1 + for startRevision == 0 { + select { + case <-ctx.Done(): + return nil + case <-time.After(500 * time.Millisecond): + startRevision, err = w.getCurrentValidatorRevision(w.cfg.SourceID) + if err != nil { + log.L().Error("reset validator stage failed, will retry later", zap.Error(err), zap.Int("retryNum", retryNum)) + } + } + retryNum++ + } + } else { + if err != nil { + log.L().Error("observe validator stage failed, quit now", zap.Error(err)) + } else { + log.L().Info("observe validator stage will quit now") + } + return err + } + } +} + +func (w *SourceWorker) handleValidatorStage(ctx context.Context, stageCh chan ha.Stage, errCh chan error) error { + closed := false + for { + select { + case <-ctx.Done(): + closed = true + case stage, ok := <-stageCh: + if !ok { + closed = true + break + } + log.L().Info("receive validator stage change", zap.Stringer("stage", stage), zap.Bool("is deleted", stage.IsDeleted)) + err := w.operateValidatorStage(stage) + if err != nil { + opType := w.getValidatorOp(stage) + opErrCounter.WithLabelValues(w.name, opType).Inc() + log.L().Error("fail to operate validator stage", zap.Stringer("stage", stage), zap.Bool("is deleted", stage.IsDeleted), zap.Error(err)) + if etcdutil.IsRetryableError(err) { + return err + } + } + case err, ok := <-errCh: + if !ok { + closed = true + break + } + // TODO: deal with err + log.L().Error("watch validator stage received an error", zap.Error(err)) + if etcdutil.IsRetryableError(err) { + return err + } + } + if closed { + log.L().Info("worker is closed, handle validator stage will quit now") + return nil + } + } +} + +func (w *SourceWorker) getCurrentValidatorRevision(source string) (int64, error) { + _, rev, err := ha.GetValidatorStage(w.etcdClient, source, "", 0) + if err != nil { + return 0, err + } + return rev, nil +} + +func (w *SourceWorker) getValidatorOp(stage ha.Stage) string { + if stage.IsDeleted { + return "validator-delete" + } + if stage.Expect == pb.Stage_Running { + return pb.ValidatorOp_StartValidator.String() + } else if stage.Expect == pb.Stage_Stopped { + return pb.ValidatorOp_StopValidator.String() + } + // should not happen + return "" +} + +func (w *SourceWorker) operateValidatorStage(stage ha.Stage) error { + // if the key it's deleted, the subtask is deleted too, let subtask clean it up. + if stage.IsDeleted { + return nil + } + + subtask := w.subTaskHolder.findSubTask(stage.Task) + if subtask == nil { + // when a new subtask start with validator, both subtask and validator stage observer will observe it, + // if validator observe it first, we may not have the subtask. + log.L().Info("cannot find subtask. maybe it's a new task, let subtask stage observer handles it") + return nil + } + + // stage of validator can only be Running or Stopped + switch stage.Expect { + case pb.Stage_Stopped: + subtask.StopValidator() + case pb.Stage_Running: + // validator's config is stored with subtask config, we need to update subtask config as validator may start + // on the fly. + subTaskCfg, _, err := ha.GetSubTaskCfg(w.etcdClient, stage.Source, stage.Task, stage.Revision) + if err != nil { + return err + } + if _, ok := subTaskCfg[stage.Task]; !ok { + log.L().Error("failed to get subtask config", zap.Reflect("stage", stage)) + return errors.New("failed to get subtask config") + } + + subtask.SetCfg(subTaskCfg[stage.Task]) + subtask.StartValidator(stage.Expect) + default: + // should not happen + log.L().Warn("invalid validator stage", zap.Reflect("stage", stage)) + } + return nil +} diff --git a/dm/dm/worker/source_worker_test.go b/dm/dm/worker/source_worker_test.go index 5dd29bce6c9..1ae4eaa476a 100644 --- a/dm/dm/worker/source_worker_test.go +++ b/dm/dm/worker/source_worker_test.go @@ -34,6 +34,7 @@ import ( "github.com/pingcap/tiflow/dm/pkg/log" "github.com/pingcap/tiflow/dm/pkg/utils" "github.com/pingcap/tiflow/dm/relay" + "github.com/pingcap/tiflow/dm/syncer" ) var emptyWorkerStatusInfoJSONLength = 25 @@ -99,7 +100,7 @@ func (t *testServer) testWorker(c *C) { c.Assert(w.StartSubTask(&config.SubTaskConfig{ Name: "testStartTask", - }, pb.Stage_Running, true), IsNil) + }, pb.Stage_Running, pb.Stage_Stopped, true), IsNil) task := w.subTaskHolder.findSubTask("testStartTask") c.Assert(task, NotNil) c.Assert(task.Result().String(), Matches, ".*worker already closed.*") @@ -197,7 +198,7 @@ func (t *testServer2) TestTaskAutoResume(c *C) { c.Assert(err, IsNil) subtaskCfg.Mode = "full" subtaskCfg.Timezone = "UTC" - c.Assert(s.getWorker(true).StartSubTask(&subtaskCfg, pb.Stage_Running, true), IsNil) + c.Assert(s.getWorker(true).StartSubTask(&subtaskCfg, pb.Stage_Running, pb.Stage_Stopped, true), IsNil) // check task in paused state c.Assert(utils.WaitSomething(100, 100*time.Millisecond, func() bool { @@ -396,8 +397,7 @@ func (t *testWorkerFunctionalities) testEnableHandleSubtasks(c *C, w *SourceWork c.Assert(w.EnableHandleSubtasks(), IsNil) c.Assert(w.subTaskEnabled.Load(), IsTrue) - _, err := ha.PutSubTaskCfgStage(etcdCli, []config.SubTaskConfig{subtaskCfg}, - []ha.Stage{ha.NewSubTaskStage(pb.Stage_Running, sourceCfg.SourceID, subtaskCfg.Name)}) + _, err := ha.PutSubTaskCfgStage(etcdCli, []config.SubTaskConfig{subtaskCfg}, []ha.Stage{ha.NewSubTaskStage(pb.Stage_Running, sourceCfg.SourceID, subtaskCfg.Name)}, nil) c.Assert(err, IsNil) c.Assert(utils.WaitSomething(30, 100*time.Millisecond, func() bool { return w.subTaskHolder.findSubTask(subtaskCfg.Name) != nil @@ -480,14 +480,13 @@ func (t *testWorkerEtcdCompact) TestWatchSubtaskStageEtcdCompact(c *C) { c.Assert(err, IsNil) subtaskCfg.MydumperPath = mydumperPath - _, err = ha.PutSubTaskCfgStage(etcdCli, []config.SubTaskConfig{subtaskCfg}, - []ha.Stage{ha.NewSubTaskStage(pb.Stage_Running, sourceCfg.SourceID, subtaskCfg.Name)}) + _, err = ha.PutSubTaskCfgStage(etcdCli, []config.SubTaskConfig{subtaskCfg}, []ha.Stage{ha.NewSubTaskStage(pb.Stage_Running, sourceCfg.SourceID, subtaskCfg.Name)}, nil) c.Assert(err, IsNil) rev, err := ha.DeleteSubTaskCfgStage(etcdCli, []config.SubTaskConfig{subtaskCfg}, - []ha.Stage{ha.NewSubTaskStage(pb.Stage_Stopped, sourceCfg.SourceID, subtaskCfg.Name)}) + []ha.Stage{ha.NewSubTaskStage(pb.Stage_Stopped, sourceCfg.SourceID, subtaskCfg.Name)}, nil) c.Assert(err, IsNil) // step 2.1: start a subtask manually - c.Assert(w.StartSubTask(&subtaskCfg, pb.Stage_Running, true), IsNil) + c.Assert(w.StartSubTask(&subtaskCfg, pb.Stage_Running, pb.Stage_Stopped, true), IsNil) // step 3: trigger etcd compaction and check whether we can receive it through watcher _, err = etcdCli.Compact(ctx, rev) c.Assert(err, IsNil) @@ -515,8 +514,7 @@ func (t *testWorkerEtcdCompact) TestWatchSubtaskStageEtcdCompact(c *C) { return w.subTaskHolder.findSubTask(subtaskCfg.Name) == nil }), IsTrue) // step 4.2: add a new subtask stage, worker should receive and start it - _, err = ha.PutSubTaskCfgStage(etcdCli, []config.SubTaskConfig{subtaskCfg}, - []ha.Stage{ha.NewSubTaskStage(pb.Stage_Running, sourceCfg.SourceID, subtaskCfg.Name)}) + _, err = ha.PutSubTaskCfgStage(etcdCli, []config.SubTaskConfig{subtaskCfg}, []ha.Stage{ha.NewSubTaskStage(pb.Stage_Running, sourceCfg.SourceID, subtaskCfg.Name)}, nil) c.Assert(err, IsNil) c.Assert(utils.WaitSomething(30, 100*time.Millisecond, func() bool { return w.subTaskHolder.findSubTask(subtaskCfg.Name) != nil @@ -553,6 +551,128 @@ func (t *testWorkerEtcdCompact) TestWatchSubtaskStageEtcdCompact(c *C) { wg.Wait() } +func (t *testWorkerEtcdCompact) TestWatchValidatorStageEtcdCompact(c *C) { + var ( + masterAddr = tempurl.Alloc()[len("http://"):] + keepAliveTTL = int64(1) + startRev = int64(1) + ) + + etcdDir := c.MkDir() + ETCD, err := createMockETCD(etcdDir, "http://"+masterAddr) + c.Assert(err, IsNil) + defer ETCD.Close() + cfg := NewConfig() + c.Assert(cfg.Parse([]string{"-config=./dm-worker.toml"}), IsNil) + cfg.Join = masterAddr + cfg.KeepAliveTTL = keepAliveTTL + cfg.RelayKeepAliveTTL = keepAliveTTL + + etcdCli, err := clientv3.New(clientv3.Config{ + Endpoints: GetJoinURLs(cfg.Join), + DialTimeout: dialTimeout, + DialKeepAliveTime: keepaliveTime, + DialKeepAliveTimeout: keepaliveTimeout, + }) + c.Assert(err, IsNil) + sourceCfg := loadSourceConfigWithoutPassword(c) + sourceCfg.From = config.GetDBConfigForTest() + sourceCfg.EnableRelay = false + + // + // step 1: start worker + w, err := NewSourceWorker(sourceCfg, etcdCli, "", "") + c.Assert(err, IsNil) + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + defer w.Close() + go func() { + w.Start() + }() + c.Assert(utils.WaitSomething(50, 100*time.Millisecond, func() bool { + return !w.closed.Load() + }), IsTrue) + + // + // step 2: Put a subtask config and subtask stage to this source, then delete it + subtaskCfg := config.SubTaskConfig{} + err = subtaskCfg.DecodeFile(subtaskSampleFile, true) + c.Assert(err, IsNil) + subtaskCfg.MydumperPath = mydumperPath + subtaskCfg.ValidatorCfg = config.ValidatorConfig{Mode: config.ValidationNone} + + // increase revision + _, err = etcdCli.Put(context.Background(), "/dummy-key", "value") + c.Assert(err, IsNil) + rev, err := ha.PutSubTaskCfgStage(etcdCli, []config.SubTaskConfig{subtaskCfg}, []ha.Stage{ha.NewSubTaskStage(pb.Stage_Running, sourceCfg.SourceID, subtaskCfg.Name)}, nil) + c.Assert(err, IsNil) + + // + // step 2.1: start a subtask manually + c.Assert(w.StartSubTask(&subtaskCfg, pb.Stage_Running, pb.Stage_Stopped, true), IsNil) + + // + // step 3: trigger etcd compaction and check whether we can receive it through watcher + _, err = etcdCli.Compact(ctx, rev) + c.Assert(err, IsNil) + subTaskStageCh := make(chan ha.Stage, 10) + subTaskErrCh := make(chan error, 10) + ctxForWatch, cancelFunc := context.WithCancel(ctx) + ha.WatchValidatorStage(ctxForWatch, etcdCli, sourceCfg.SourceID, startRev, subTaskStageCh, subTaskErrCh) + select { + case err = <-subTaskErrCh: + c.Assert(err, Equals, etcdErrCompacted) + case <-time.After(300 * time.Millisecond): + c.Fatal("fail to get etcd error compacted") + } + cancelFunc() + + // + // step 4: watch subtask stage from startRev + subTask := w.subTaskHolder.findSubTask(subtaskCfg.Name) + getValidator := func() *syncer.DataValidator { + subTask.RLock() + defer subTask.RUnlock() + return subTask.validator + } + c.Assert(subTask, NotNil) + c.Assert(getValidator(), IsNil) + var wg sync.WaitGroup + ctx1, cancel1 := context.WithCancel(ctx) + wg.Add(1) + go func() { + defer wg.Done() + c.Assert(w.observeValidatorStage(ctx1, startRev), IsNil) + }() + time.Sleep(time.Second) + + subtaskCfg.ValidatorCfg = config.ValidatorConfig{Mode: config.ValidationFast} + unitBakup := subTask.units[len(subTask.units)-1] + subTask.units[len(subTask.units)-1] = &syncer.Syncer{} // validator need a Syncer, not a mocked unit + validatorStage := ha.NewValidatorStage(pb.Stage_Running, subtaskCfg.SourceID, subtaskCfg.Name) + _, err = ha.PutSubTaskCfgStage(etcdCli, []config.SubTaskConfig{subtaskCfg}, nil, []ha.Stage{validatorStage}) + c.Assert(err, IsNil) + + // validator created + c.Assert(utils.WaitSomething(30, 100*time.Millisecond, func() bool { + return getValidator() != nil + }), IsTrue) + + subTask.units[len(subTask.units)-1] = unitBakup // restore unit + cancel1() + wg.Wait() + + // test operate validator + err = w.operateValidatorStage(ha.Stage{IsDeleted: true}) + c.Assert(err, IsNil) + err = w.operateValidatorStage(ha.Stage{Expect: pb.Stage_Running, Task: "not-exist"}) + c.Assert(err, IsNil) + err = w.operateValidatorStage(ha.Stage{Expect: pb.Stage_Running, Task: subtaskCfg.Name}) + c.Assert(err, ErrorMatches, ".*failed to get subtask config.*") + err = w.operateValidatorStage(ha.Stage{Expect: pb.Stage_Running, Source: subtaskCfg.SourceID, Task: subtaskCfg.Name}) + c.Assert(err, IsNil) +} + func (t *testWorkerEtcdCompact) TestWatchRelayStageEtcdCompact(c *C) { var ( masterAddr = tempurl.Alloc()[len("http://"):] diff --git a/dm/dm/worker/subtask.go b/dm/dm/worker/subtask.go index ca04012059a..078902ad978 100644 --- a/dm/dm/worker/subtask.go +++ b/dm/dm/worker/subtask.go @@ -117,6 +117,8 @@ type SubTask struct { etcdClient *clientv3.Client workerName string + + validator *syncer.DataValidator } // NewSubTask is subtask initializer @@ -213,7 +215,7 @@ func (st *SubTask) initUnits(relay relay.Process) error { // Run runs the sub task. // TODO: check concurrent problems. -func (st *SubTask) Run(expectStage pb.Stage, relay relay.Process) { +func (st *SubTask) Run(expectStage pb.Stage, expectValidatorStage pb.Stage, relay relay.Process) { if st.Stage() == pb.Stage_Finished || st.Stage() == pb.Stage_Running { st.l.Warn("prepare to run a subtask with invalid stage", zap.Stringer("current stage", st.Stage()), @@ -227,6 +229,8 @@ func (st *SubTask) Run(expectStage pb.Stage, relay relay.Process) { return } + st.StartValidator(expectValidatorStage) + if expectStage == pb.Stage_Running { st.run() } else { @@ -257,6 +261,42 @@ func (st *SubTask) run() { go cu.Process(ctx, pr) } +func (st *SubTask) StartValidator(expect pb.Stage) { + // when validator mode=none + if expect == pb.Stage_InvalidStage { + return + } + st.Lock() + defer st.Unlock() + if st.cfg.ValidatorCfg.Mode != config.ValidationFast && st.cfg.ValidatorCfg.Mode != config.ValidationFull { + return + } + var syncerObj *syncer.Syncer + var ok bool + for _, u := range st.units { + if syncerObj, ok = u.(*syncer.Syncer); ok { + break + } + } + if syncerObj == nil { + st.l.Warn("cannot start validator without syncer") + return + } + + if st.validator == nil { + st.validator = syncer.NewContinuousDataValidator(st.cfg, syncerObj) + } + st.validator.Start(expect) +} + +func (st *SubTask) StopValidator() { + st.Lock() + if st.validator != nil { + st.validator.Stop() + } + st.Unlock() +} + func (st *SubTask) setCurrCtx(ctx context.Context, cancel context.CancelFunc) { st.Lock() // call previous cancel func for safety @@ -492,7 +532,11 @@ func (st *SubTask) Close() { } st.closeUnits() // close all un-closed units - updateTaskMetric(st.cfg.Name, st.cfg.SourceID, pb.Stage_Stopped, st.workerName) + + cfg := st.getCfg() + updateTaskMetric(cfg.Name, cfg.SourceID, pb.Stage_Stopped, st.workerName) + + st.StopValidator() } // Pause pauses a running sub task or a sub task paused by error. @@ -515,7 +559,11 @@ func (st *SubTask) Pause() error { // TODO: similar to Run, refactor later. func (st *SubTask) Resume(relay relay.Process) error { if !st.initialized.Load() { - st.Run(pb.Stage_Running, relay) + expectValidatorStage, err := getExpectValidatorStage(st.cfg.ValidatorCfg, st.etcdClient, st.cfg.SourceID, st.cfg.Name, 0) + if err != nil { + return terror.Annotate(err, "fail to get validator stage from etcd") + } + st.Run(pb.Stage_Running, expectValidatorStage, relay) return nil } @@ -652,8 +700,9 @@ func (st *SubTask) unitTransWaitCondition(subTaskCtx context.Context) error { loadStatus := pu.Status(nil).(*pb.LoadStatus) - if st.cfg.EnableGTID { - gset1, err = gtid.ParserGTID(st.cfg.Flavor, loadStatus.MetaBinlogGTID) + cfg := st.getCfg() + if cfg.EnableGTID { + gset1, err = gtid.ParserGTID(cfg.Flavor, loadStatus.MetaBinlogGTID) if err != nil { return terror.WithClass(err, terror.ClassDMWorker) } @@ -667,8 +716,8 @@ func (st *SubTask) unitTransWaitCondition(subTaskCtx context.Context) error { for { relayStatus := hub.w.relayHolder.Status(nil) - if st.cfg.EnableGTID { - gset2, err = gtid.ParserGTID(st.cfg.Flavor, relayStatus.RelayBinlogGtid) + if cfg.EnableGTID { + gset2, err = gtid.ParserGTID(cfg.Flavor, relayStatus.RelayBinlogGtid) if err != nil { return terror.WithClass(err, terror.ClassDMWorker) } @@ -689,11 +738,11 @@ func (st *SubTask) unitTransWaitCondition(subTaskCtx context.Context) error { } } - st.l.Debug("wait relay to catchup", zap.Bool("enableGTID", st.cfg.EnableGTID), zap.Stringer("load end position", pos1), zap.String("load end gtid", loadStatus.MetaBinlogGTID), zap.Stringer("relay position", pos2), zap.String("relay gtid", relayStatus.RelayBinlogGtid)) + st.l.Debug("wait relay to catchup", zap.Bool("enableGTID", cfg.EnableGTID), zap.Stringer("load end position", pos1), zap.String("load end gtid", loadStatus.MetaBinlogGTID), zap.Stringer("relay position", pos2), zap.String("relay gtid", relayStatus.RelayBinlogGtid)) select { case <-ctxWait.Done(): - if st.cfg.EnableGTID { + if cfg.EnableGTID { return terror.ErrWorkerWaitRelayCatchupTimeout.Generate(waitRelayCatchupTimeout, loadStatus.MetaBinlogGTID, relayStatus.RelayBinlogGtid) } return terror.ErrWorkerWaitRelayCatchupTimeout.Generate(waitRelayCatchupTimeout, pos1, pos2) @@ -733,6 +782,28 @@ func (st *SubTask) HandleError(ctx context.Context, req *pb.HandleWorkerErrorReq return msg, err } +func (st *SubTask) getCfg() *config.SubTaskConfig { + st.RLock() + defer st.RUnlock() + return st.cfg +} + +func (st *SubTask) SetCfg(subTaskConfig config.SubTaskConfig) { + st.Lock() + st.cfg = &subTaskConfig + st.Unlock() +} + +func (st *SubTask) getValidatorStage() pb.Stage { + st.RLock() + defer st.RUnlock() + + if st.validator != nil { + return st.validator.Stage() + } + return pb.Stage_InvalidStage +} + func updateTaskMetric(task, sourceID string, stage pb.Stage, workerName string) { if stage == pb.Stage_Stopped || stage == pb.Stage_Finished { taskState.DeleteAllAboutLabels(prometheus.Labels{"task": task, "source_id": sourceID}) diff --git a/dm/dm/worker/subtask_holder.go b/dm/dm/worker/subtask_holder.go index 888b756f894..27f512d62ed 100644 --- a/dm/dm/worker/subtask_holder.go +++ b/dm/dm/worker/subtask_holder.go @@ -59,7 +59,7 @@ func (h *subTaskHolder) resetAllSubTasks(relay relay.Process) { // TODO: make a st.reset st.ctx, st.cancel = context.WithCancel(context.Background()) st.cfg.UseRelay = relay != nil - st.Run(stage, relay) + st.Run(stage, st.getValidatorStage(), relay) } } diff --git a/dm/dm/worker/subtask_test.go b/dm/dm/worker/subtask_test.go index 41e7a5a2152..7e4d142dd43 100644 --- a/dm/dm/worker/subtask_test.go +++ b/dm/dm/worker/subtask_test.go @@ -180,7 +180,7 @@ func (t *testSubTask) TestSubTaskNormalUsage(c *C) { createUnits = func(cfg *config.SubTaskConfig, etcdClient *clientv3.Client, worker string, relay relay.Process) []unit.Unit { return nil } - st.Run(pb.Stage_Running, nil) + st.Run(pb.Stage_Running, pb.Stage_Running, nil) c.Assert(st.Stage(), Equals, pb.Stage_Paused) c.Assert(strings.Contains(st.Result().Errors[0].String(), "has no dm units for mode"), IsTrue) @@ -190,7 +190,7 @@ func (t *testSubTask) TestSubTaskNormalUsage(c *C) { return []unit.Unit{mockDumper, mockLoader} } - st.Run(pb.Stage_Running, nil) + st.Run(pb.Stage_Running, pb.Stage_Running, nil) c.Assert(st.Stage(), Equals, pb.Stage_Running) c.Assert(st.CurrUnit(), Equals, mockDumper) c.Assert(st.Result(), IsNil) @@ -301,7 +301,7 @@ func (t *testSubTask) TestPauseAndResumeSubtask(c *C) { return []unit.Unit{mockDumper, mockLoader} } - st.Run(pb.Stage_Running, nil) + st.Run(pb.Stage_Running, pb.Stage_Running, nil) c.Assert(st.Stage(), Equals, pb.Stage_Running) c.Assert(st.CurrUnit(), Equals, mockDumper) c.Assert(st.Result(), IsNil) @@ -401,7 +401,7 @@ func (t *testSubTask) TestPauseAndResumeSubtask(c *C) { c.Assert(st.Stage(), Equals, pb.Stage_Finished) c.Assert(st.Result().Errors, HasLen, 0) - st.Run(pb.Stage_Finished, nil) + st.Run(pb.Stage_Finished, pb.Stage_Stopped, nil) c.Assert(st.CurrUnit(), Equals, mockLoader) c.Assert(st.Stage(), Equals, pb.Stage_Finished) c.Assert(st.Result().Errors, HasLen, 0) @@ -409,9 +409,11 @@ func (t *testSubTask) TestPauseAndResumeSubtask(c *C) { func (t *testSubTask) TestSubtaskWithStage(c *C) { cfg := &config.SubTaskConfig{ - Name: "testSubtaskScene", - Mode: config.ModeFull, + SourceID: "source", + Name: "testSubtaskScene", + Mode: config.ModeFull, } + c.Assert(cfg.Adjust(false), IsNil) st := NewSubTaskWithStage(cfg, pb.Stage_Paused, nil, "worker") c.Assert(st.Stage(), DeepEquals, pb.Stage_Paused) @@ -450,7 +452,7 @@ func (t *testSubTask) TestSubtaskWithStage(c *C) { return []unit.Unit{mockDumper, mockLoader} } - st.Run(pb.Stage_Finished, nil) + st.Run(pb.Stage_Finished, pb.Stage_Stopped, nil) c.Assert(st.Stage(), Equals, pb.Stage_Finished) c.Assert(st.CurrUnit(), Equals, nil) c.Assert(st.Result(), IsNil) diff --git a/dm/dm/worker/task_checker_test.go b/dm/dm/worker/task_checker_test.go index 5a58a16a5eb..c15abf97091 100644 --- a/dm/dm/worker/task_checker_test.go +++ b/dm/dm/worker/task_checker_test.go @@ -107,10 +107,11 @@ func (s *testTaskCheckerSuite) TestCheck(c *check.C) { rtsc.w = w st := &SubTask{ - cfg: &config.SubTaskConfig{Name: taskName}, + cfg: &config.SubTaskConfig{SourceID: "source", Name: taskName}, stage: pb.Stage_Running, l: log.With(zap.String("subtask", taskName)), } + c.Assert(st.cfg.Adjust(false), check.IsNil) rtsc.w.subTaskHolder.recordSubTask(st) rtsc.check() bf, ok := rtsc.bc.backoffs[taskName] @@ -241,7 +242,7 @@ func (s *testTaskCheckerSuite) TestCheckTaskIndependent(c *check.C) { // test backoff strategies of different tasks do not affect each other st1 = &SubTask{ - cfg: &config.SubTaskConfig{Name: task1}, + cfg: &config.SubTaskConfig{SourceID: "source", Name: task1}, stage: pb.Stage_Paused, result: &pb.ProcessResult{ IsCanceled: false, @@ -249,9 +250,10 @@ func (s *testTaskCheckerSuite) TestCheckTaskIndependent(c *check.C) { }, l: log.With(zap.String("subtask", task1)), } + c.Assert(st1.cfg.Adjust(false), check.IsNil) rtsc.w.subTaskHolder.recordSubTask(st1) st2 = &SubTask{ - cfg: &config.SubTaskConfig{Name: task2}, + cfg: &config.SubTaskConfig{SourceID: "source", Name: task2}, stage: pb.Stage_Paused, result: &pb.ProcessResult{ IsCanceled: false, @@ -259,6 +261,7 @@ func (s *testTaskCheckerSuite) TestCheckTaskIndependent(c *check.C) { }, l: log.With(zap.String("subtask", task2)), } + c.Assert(st2.cfg.Adjust(false), check.IsNil) rtsc.w.subTaskHolder.recordSubTask(st2) task1LatestResumeTime = rtsc.bc.latestResumeTime[task1] diff --git a/dm/dm/worker/utils.go b/dm/dm/worker/utils.go new file mode 100644 index 00000000000..0b4aad256b8 --- /dev/null +++ b/dm/dm/worker/utils.go @@ -0,0 +1,37 @@ +// Copyright 2022 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 worker + +import ( + "go.etcd.io/etcd/clientv3" + + "github.com/pingcap/tiflow/dm/dm/config" + "github.com/pingcap/tiflow/dm/dm/pb" + "github.com/pingcap/tiflow/dm/pkg/ha" +) + +func getExpectValidatorStage(cfg config.ValidatorConfig, etcdClient *clientv3.Client, source, task string, revision int64) (pb.Stage, error) { + // for subtask with validation mode=none, there is no validator stage, set to invalid + expectedValidatorStage := pb.Stage_InvalidStage + if cfg.Mode != config.ValidationNone { + validatorStageM, _, err := ha.GetValidatorStage(etcdClient, source, task, revision) + if err != nil { + return expectedValidatorStage, err + } + if s, ok := validatorStageM[task]; ok { + expectedValidatorStage = s.Expect + } + } + return expectedValidatorStage, nil +} diff --git a/dm/dm/worker/utils_test.go b/dm/dm/worker/utils_test.go new file mode 100644 index 00000000000..0a63f9eb3ff --- /dev/null +++ b/dm/dm/worker/utils_test.go @@ -0,0 +1,63 @@ +// Copyright 2022 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 worker + +import ( + "testing" + + "github.com/stretchr/testify/require" + "go.etcd.io/etcd/integration" + + "github.com/pingcap/tiflow/dm/dm/config" + "github.com/pingcap/tiflow/dm/dm/pb" + "github.com/pingcap/tiflow/dm/pkg/ha" +) + +func TestGetExpectValidatorStage(t *testing.T) { + mockCluster := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1}) + defer mockCluster.Terminate(t) + + etcdTestCli := mockCluster.RandClient() + defer func() { + require.Nil(t, ha.ClearTestInfoOperation(etcdTestCli)) + }() + cfg := config.SubTaskConfig{} + require.Nil(t, cfg.DecodeFile("subtask.toml", true)) + source := cfg.SourceID + task := cfg.Name + stage := ha.NewSubTaskStage(pb.Stage_Running, source, task) + + validatorStage, err := getExpectValidatorStage(cfg.ValidatorCfg, etcdTestCli, source, task, 0) + require.Nil(t, err) + require.Equal(t, pb.Stage_InvalidStage, validatorStage) + + cfg.ValidatorCfg.Mode = config.ValidationFast + validatorStage, err = getExpectValidatorStage(cfg.ValidatorCfg, etcdTestCli, source, task, 0) + require.Nil(t, err) + require.Equal(t, pb.Stage_InvalidStage, validatorStage) + + // put subtask config and stage at the same time + rev, err := ha.PutSubTaskCfgStage(etcdTestCli, []config.SubTaskConfig{cfg}, []ha.Stage{stage}, []ha.Stage{stage}) + require.Nil(t, err) + + validatorStage, err = getExpectValidatorStage(cfg.ValidatorCfg, etcdTestCli, source, task, 0) + require.Nil(t, err) + require.Equal(t, pb.Stage_Running, validatorStage) + validatorStage, err = getExpectValidatorStage(cfg.ValidatorCfg, etcdTestCli, source, task+"not exist", 0) + require.Nil(t, err) + require.Equal(t, pb.Stage_InvalidStage, validatorStage) + + _, err = getExpectValidatorStage(cfg.ValidatorCfg, etcdTestCli, source, task+"not exist", rev+1) + require.NotNil(t, err) +} diff --git a/dm/errors.toml b/dm/errors.toml index beee203ee50..9e2376a51f7 100644 --- a/dm/errors.toml +++ b/dm/errors.toml @@ -1114,6 +1114,18 @@ description = "" workaround = "Please choose a valid value in ['replace', 'error', 'ignore']" tags = ["internal", "medium"] +[error.DM-config-20055] +message = "invalid validation mode" +description = "" +workaround = "Please check `validation-mode` config in task configuration file." +tags = ["internal", "high"] + +[error.DM-config-20056] +message = "mysql-instance(%d)'s continuous validator config %s not exist" +description = "" +workaround = "Please check the `continuous-validator-config-name` config in task configuration file." +tags = ["internal", "medium"] + [error.DM-binlog-op-22001] message = "" description = "" diff --git a/dm/pkg/ha/ops.go b/dm/pkg/ha/ops.go index 07caa515631..b6e2d2eb519 100644 --- a/dm/pkg/ha/ops.go +++ b/dm/pkg/ha/ops.go @@ -85,25 +85,26 @@ func DeleteSourceCfgRelayStageSourceBound(cli *clientv3.Client, source, worker s // - subtask config. // - subtask stage. // NOTE: golang can't use two `...` in the func, so use `[]` instead. -func PutSubTaskCfgStage(cli *clientv3.Client, cfgs []config.SubTaskConfig, stages []Stage) (int64, error) { - return opSubTaskCfgStage(cli, mvccpb.PUT, cfgs, stages) +func PutSubTaskCfgStage(cli *clientv3.Client, cfgs []config.SubTaskConfig, stages []Stage, validatorStages []Stage) (int64, error) { + return operateSubtask(cli, mvccpb.PUT, cfgs, stages, validatorStages) } // DeleteSubTaskCfgStage deletes the following data in one txn. // - subtask config. // - subtask stage. // NOTE: golang can't use two `...` in the func, so use `[]` instead. -func DeleteSubTaskCfgStage(cli *clientv3.Client, cfgs []config.SubTaskConfig, stages []Stage) (int64, error) { - return opSubTaskCfgStage(cli, mvccpb.DELETE, cfgs, stages) +func DeleteSubTaskCfgStage(cli *clientv3.Client, cfgs []config.SubTaskConfig, stages []Stage, validatorStages []Stage) (int64, error) { + return operateSubtask(cli, mvccpb.DELETE, cfgs, stages, validatorStages) } -// opSubTaskCfgStage puts/deletes for subtask config and stage in one txn. -func opSubTaskCfgStage(cli *clientv3.Client, evType mvccpb.Event_EventType, - cfgs []config.SubTaskConfig, stages []Stage) (int64, error) { +// operateSubtask puts/deletes KVs for the subtask in one txn. +func operateSubtask(cli *clientv3.Client, evType mvccpb.Event_EventType, cfgs []config.SubTaskConfig, stages []Stage, + validatorStages []Stage) (int64, error) { var ( - ops1 []clientv3.Op - ops2 []clientv3.Op - err error + ops1 []clientv3.Op + ops2 []clientv3.Op + validatorOps []clientv3.Op + err error ) switch evType { case mvccpb.PUT: @@ -115,14 +116,20 @@ func opSubTaskCfgStage(cli *clientv3.Client, evType mvccpb.Event_EventType, if err != nil { return 0, err } + validatorOps, err = putValidatorStageOps(validatorStages...) + if err != nil { + return 0, err + } case mvccpb.DELETE: ops1 = deleteSubTaskCfgOp(cfgs...) ops2 = deleteSubTaskStageOp(stages...) + validatorOps = deleteValidatorStageOps(validatorStages...) } - ops := make([]clientv3.Op, 0, len(ops1)+len(ops2)) + ops := make([]clientv3.Op, 0, 2*len(cfgs)+len(stages)) ops = append(ops, ops1...) ops = append(ops, ops2...) + ops = append(ops, validatorOps...) _, rev, err := etcdutil.DoOpsInOneTxnWithRetry(cli, ops...) return rev, err } diff --git a/dm/pkg/ha/ops_test.go b/dm/pkg/ha/ops_test.go index d0730879f85..a4295a807a4 100644 --- a/dm/pkg/ha/ops_test.go +++ b/dm/pkg/ha/ops_test.go @@ -24,14 +24,15 @@ func (t *testForEtcd) TestOpsEtcd(c *C) { defer clearTestInfoOperation(c) var ( - source = "mysql-replica-1" - worker = "dm-worker-1" - task1 = "task-1" - task2 = "task-2" - relayStage = NewRelayStage(pb.Stage_Running, source) - subtaskStage1 = NewSubTaskStage(pb.Stage_Running, source, task1) - subtaskStage2 = NewSubTaskStage(pb.Stage_Running, source, task2) - bound = NewSourceBound(source, worker) + source = "mysql-replica-1" + worker = "dm-worker-1" + task1 = "task-1" + task2 = "task-2" + relayStage = NewRelayStage(pb.Stage_Running, source) + subtaskStage1 = NewSubTaskStage(pb.Stage_Running, source, task1) + subtaskStage2 = NewSubTaskStage(pb.Stage_Running, source, task2) + validatorStage = NewSubTaskStage(pb.Stage_Running, source, task2) + bound = NewSourceBound(source, worker) emptyStage Stage subtaskCfg1 config.SubTaskConfig @@ -95,7 +96,7 @@ func (t *testForEtcd) TestOpsEtcd(c *C) { c.Assert(scm2, HasLen, 0) // put subtask config and subtask stage. - rev6, err := PutSubTaskCfgStage(etcdTestCli, []config.SubTaskConfig{subtaskCfg1, subtaskCfg2}, []Stage{subtaskStage1, subtaskStage2}) + rev6, err := PutSubTaskCfgStage(etcdTestCli, []config.SubTaskConfig{subtaskCfg1, subtaskCfg2}, []Stage{subtaskStage1, subtaskStage2}, []Stage{validatorStage}) c.Assert(err, IsNil) c.Assert(rev6, Greater, rev5) @@ -114,9 +115,22 @@ func (t *testForEtcd) TestOpsEtcd(c *C) { subtaskStage2.Revision = rev6 c.Assert(stsm[task1], DeepEquals, subtaskStage1) c.Assert(stsm[task2], DeepEquals, subtaskStage2) + validatorStages, rev7, err := GetValidatorStage(etcdTestCli, source, "", rev6) + c.Assert(err, IsNil) + c.Assert(rev7, Equals, rev6) + c.Assert(validatorStages, HasLen, 1) + validatorStage.Revision = rev6 + c.Assert(validatorStages[task2], DeepEquals, validatorStage) + // get with task name + validatorStages, rev7, err = GetValidatorStage(etcdTestCli, source, task2, rev6) + c.Assert(err, IsNil) + c.Assert(rev7, Equals, rev6) + c.Assert(validatorStages, HasLen, 1) + validatorStage.Revision = rev6 + c.Assert(validatorStages[task2], DeepEquals, validatorStage) // delete them. - rev8, err := DeleteSubTaskCfgStage(etcdTestCli, []config.SubTaskConfig{subtaskCfg1, subtaskCfg2}, []Stage{subtaskStage1, subtaskStage2}) + rev8, err := DeleteSubTaskCfgStage(etcdTestCli, []config.SubTaskConfig{subtaskCfg1, subtaskCfg2}, []Stage{subtaskStage1, subtaskStage2}, []Stage{validatorStage}) c.Assert(err, IsNil) c.Assert(rev8, Greater, rev7) @@ -129,4 +143,8 @@ func (t *testForEtcd) TestOpsEtcd(c *C) { c.Assert(err, IsNil) c.Assert(rev9, Equals, rev8) c.Assert(stsm, HasLen, 0) + validatorStages, rev9, err = GetValidatorStage(etcdTestCli, source, "", 0) + c.Assert(err, IsNil) + c.Assert(rev9, Equals, rev8) + c.Assert(validatorStages, HasLen, 0) } diff --git a/dm/pkg/ha/source.go b/dm/pkg/ha/source.go index 5224e5d04bb..0901362c2c5 100644 --- a/dm/pkg/ha/source.go +++ b/dm/pkg/ha/source.go @@ -134,8 +134,10 @@ func ClearTestInfoOperation(cli *clientv3.Client) error { clearRelayStage := clientv3.OpDelete(common.StageRelayKeyAdapter.Path(), clientv3.WithPrefix()) clearRelayConfig := clientv3.OpDelete(common.UpstreamRelayWorkerKeyAdapter.Path(), clientv3.WithPrefix()) clearSubTaskStage := clientv3.OpDelete(common.StageSubTaskKeyAdapter.Path(), clientv3.WithPrefix()) + clearValidatorStage := clientv3.OpDelete(common.StageValidatorKeyAdapter.Path(), clientv3.WithPrefix()) clearLoadTasks := clientv3.OpDelete(common.LoadTaskKeyAdapter.Path(), clientv3.WithPrefix()) _, _, err := etcdutil.DoOpsInOneTxnWithRetry(cli, clearSource, clearSubTask, clearWorkerInfo, clearBound, - clearLastBound, clearWorkerKeepAlive, clearRelayStage, clearRelayConfig, clearSubTaskStage, clearLoadTasks) + clearLastBound, clearWorkerKeepAlive, clearRelayStage, clearRelayConfig, clearSubTaskStage, clearValidatorStage, + clearLoadTasks) return err } diff --git a/dm/pkg/ha/stage.go b/dm/pkg/ha/stage.go index 781de9667bf..0cee6ce77ce 100644 --- a/dm/pkg/ha/stage.go +++ b/dm/pkg/ha/stage.go @@ -52,6 +52,10 @@ func NewSubTaskStage(expect pb.Stage, source, task string) Stage { return newStage(expect, source, task) } +func NewValidatorStage(expect pb.Stage, source, task string) Stage { + return newStage(expect, source, task) +} + // newStage creates a new Stage instance. func newStage(expect pb.Stage, source, task string) Stage { return Stage{ @@ -172,6 +176,10 @@ func GetAllRelayStage(cli *clientv3.Client) (map[string]Stage, int64, error) { // if task name is "", it will return all subtasks' stage as a map{task-name: stage} for the source. // if task name is given, it will return a map{task-name: stage} whose length is 1. func GetSubTaskStage(cli *clientv3.Client, source, task string) (map[string]Stage, int64, error) { + return getStageByKey(cli, common.StageSubTaskKeyAdapter, source, task, 0) +} + +func getStageByKey(cli *clientv3.Client, key common.KeyAdapter, source, task string, revision int64) (map[string]Stage, int64, error) { ctx, cancel := context.WithTimeout(cli.Ctx(), etcdutil.DefaultRequestTimeout) defer cancel() @@ -179,18 +187,23 @@ func GetSubTaskStage(cli *clientv3.Client, source, task string) (map[string]Stag stm = make(map[string]Stage) resp *clientv3.GetResponse err error + opts = make([]clientv3.OpOption, 0) ) + if revision > 0 { + opts = append(opts, clientv3.WithRev(revision)) + } if task != "" { - resp, err = cli.Get(ctx, common.StageSubTaskKeyAdapter.Encode(source, task)) + resp, err = cli.Get(ctx, key.Encode(source, task), opts...) } else { - resp, err = cli.Get(ctx, common.StageSubTaskKeyAdapter.Encode(source), clientv3.WithPrefix()) + opts = append(opts, clientv3.WithPrefix()) + resp, err = cli.Get(ctx, key.Encode(source), opts...) } if err != nil { return stm, 0, err } - stages, err := subTaskStageFromResp(source, task, resp) + stages, err := getStagesFromResp(source, task, resp) if err != nil { return stm, 0, err } @@ -199,18 +212,26 @@ func GetSubTaskStage(cli *clientv3.Client, source, task string) (map[string]Stag return stm, resp.Header.Revision, nil } +func GetValidatorStage(cli *clientv3.Client, source, task string, revision int64) (map[string]Stage, int64, error) { + return getStageByKey(cli, common.StageValidatorKeyAdapter, source, task, revision) +} + // GetAllSubTaskStage gets all subtask stages. // k/v: source ID -> task name -> subtask stage. func GetAllSubTaskStage(cli *clientv3.Client) (map[string]map[string]Stage, int64, error) { + return getAllStagesInner(cli, common.StageSubTaskKeyAdapter) +} + +func getAllStagesInner(cli *clientv3.Client, key common.KeyAdapter) (map[string]map[string]Stage, int64, error) { ctx, cancel := context.WithTimeout(cli.Ctx(), etcdutil.DefaultRequestTimeout) defer cancel() - resp, err := cli.Get(ctx, common.StageSubTaskKeyAdapter.Path(), clientv3.WithPrefix()) + resp, err := cli.Get(ctx, key.Path(), clientv3.WithPrefix()) if err != nil { return nil, 0, err } - stages, err := subTaskStageFromResp("", "", resp) + stages, err := getStagesFromResp("", "", resp) if err != nil { return nil, 0, err } @@ -218,34 +239,48 @@ func GetAllSubTaskStage(cli *clientv3.Client) (map[string]map[string]Stage, int6 return stages, resp.Header.Revision, nil } +func GetAllValidatorStage(cli *clientv3.Client) (map[string]map[string]Stage, int64, error) { + return getAllStagesInner(cli, common.StageValidatorKeyAdapter) +} + // GetSubTaskStageConfig gets source's subtask stages and configs at the same time // source **must not be empty** -// return map{task name -> subtask stage}, map{task name -> subtask config}, revision, error. -func GetSubTaskStageConfig(cli *clientv3.Client, source string) (map[string]Stage, map[string]config.SubTaskConfig, int64, error) { +// return map{task name -> subtask stage}, map{task name -> validator stage}, map{task name -> subtask config}, revision, error. +func GetSubTaskStageConfig(cli *clientv3.Client, source string) (map[string]Stage, map[string]Stage, map[string]config.SubTaskConfig, int64, error) { var ( - stm = make(map[string]Stage) - scm = make(map[string]config.SubTaskConfig) + stm = make(map[string]Stage) + validatorStageMap = make(map[string]Stage) + scm = make(map[string]config.SubTaskConfig) ) - txnResp, rev, err := etcdutil.DoOpsInOneTxnWithRetry(cli, clientv3.OpGet(common.StageSubTaskKeyAdapter.Encode(source), clientv3.WithPrefix()), + txnResp, rev, err := etcdutil.DoOpsInOneTxnWithRetry(cli, + clientv3.OpGet(common.StageSubTaskKeyAdapter.Encode(source), clientv3.WithPrefix()), + clientv3.OpGet(common.StageValidatorKeyAdapter.Encode(source), clientv3.WithPrefix()), clientv3.OpGet(common.UpstreamSubTaskKeyAdapter.Encode(source), clientv3.WithPrefix())) if err != nil { - return stm, scm, 0, err + return stm, validatorStageMap, scm, 0, err } stageResp := txnResp.Responses[0].GetResponseRange() - stages, err := subTaskStageFromResp(source, "", (*clientv3.GetResponse)(stageResp)) + stages, err := getStagesFromResp(source, "", (*clientv3.GetResponse)(stageResp)) if err != nil { - return stm, scm, 0, err + return stm, validatorStageMap, scm, 0, err } stm = stages[source] - cfgResp := txnResp.Responses[1].GetResponseRange() + validatorStageResp := txnResp.Responses[1].GetResponseRange() + validatorStages, err := getStagesFromResp(source, "", (*clientv3.GetResponse)(validatorStageResp)) + if err != nil { + return stm, validatorStageMap, scm, 0, err + } + validatorStageMap = validatorStages[source] + + cfgResp := txnResp.Responses[2].GetResponseRange() cfgs, err := subTaskCfgFromResp(source, "", (*clientv3.GetResponse)(cfgResp)) if err != nil { - return stm, scm, 0, err + return stm, validatorStageMap, scm, 0, err } scm = cfgs[source] - return stm, scm, rev, err + return stm, validatorStageMap, scm, rev, err } // WatchRelayStage watches PUT & DELETE operations for the relay stage. @@ -268,6 +303,14 @@ func WatchSubTaskStage(ctx context.Context, cli *clientv3.Client, watchStage(ctx, ch, subTaskStageFromKey, outCh, errCh) } +func WatchValidatorStage(ctx context.Context, cli *clientv3.Client, + source string, rev int64, outCh chan<- Stage, errCh chan<- error) { + wCtx, cancel := context.WithCancel(ctx) + defer cancel() + ch := cli.Watch(wCtx, common.StageValidatorKeyAdapter.Encode(source), clientv3.WithPrefix(), clientv3.WithRev(rev)) + watchStage(ctx, ch, validatorStageFromKey, outCh, errCh) +} + // DeleteSubTaskStage deletes the subtask stage. func DeleteSubTaskStage(cli *clientv3.Client, stages ...Stage) (int64, error) { ops := deleteSubTaskStageOp(stages...) @@ -298,7 +341,18 @@ func subTaskStageFromKey(key string) (Stage, error) { return stage, nil } -func subTaskStageFromResp(source, task string, resp *clientv3.GetResponse) (map[string]map[string]Stage, error) { +func validatorStageFromKey(key string) (Stage, error) { + var stage Stage + ks, err := common.StageValidatorKeyAdapter.Decode(key) + if err != nil { + return stage, err + } + stage.Source = ks[0] + stage.Task = ks[1] + return stage, nil +} + +func getStagesFromResp(source, task string, resp *clientv3.GetResponse) (map[string]map[string]Stage, error) { stages := make(map[string]map[string]Stage) if source != "" { stages[source] = make(map[string]Stage) // avoid stages[source] is nil diff --git a/dm/pkg/ha/stage_test.go b/dm/pkg/ha/stage_test.go index a92645bf530..946dea993fb 100644 --- a/dm/pkg/ha/stage_test.go +++ b/dm/pkg/ha/stage_test.go @@ -224,23 +224,26 @@ func (t *testForEtcd) TestGetSubTaskStageConfigEtcd(c *C) { stage := NewSubTaskStage(pb.Stage_Running, source, task) // no subtask stage and config - stm, scm, rev1, err := GetSubTaskStageConfig(etcdTestCli, source) + stm, validatorM, scm, rev1, err := GetSubTaskStageConfig(etcdTestCli, source) c.Assert(err, IsNil) c.Assert(rev1, Greater, int64(0)) c.Assert(stm, HasLen, 0) + c.Assert(validatorM, HasLen, 0) c.Assert(scm, HasLen, 0) // put subtask config and stage at the same time - rev2, err := PutSubTaskCfgStage(etcdTestCli, []config.SubTaskConfig{cfg}, []Stage{stage}) + rev2, err := PutSubTaskCfgStage(etcdTestCli, []config.SubTaskConfig{cfg}, []Stage{stage}, []Stage{stage}) c.Assert(err, IsNil) c.Assert(rev2, Greater, rev1) // get subtask config and stage at the same time - stm, scm, rev3, err := GetSubTaskStageConfig(etcdTestCli, source) + stm, validatorM, scm, rev3, err := GetSubTaskStageConfig(etcdTestCli, source) c.Assert(err, IsNil) c.Assert(rev3, Equals, rev2) c.Assert(stm, HasLen, 1) + c.Assert(validatorM, HasLen, 1) stage.Revision = rev2 c.Assert(stm[task], DeepEquals, stage) + c.Assert(validatorM[task], DeepEquals, stage) c.Assert(scm[task], DeepEquals, cfg) } diff --git a/dm/pkg/ha/subtask.go b/dm/pkg/ha/subtask.go index 660a487bd87..80b3bdfd60e 100644 --- a/dm/pkg/ha/subtask.go +++ b/dm/pkg/ha/subtask.go @@ -98,6 +98,28 @@ func deleteSubTaskCfgOp(cfgs ...config.SubTaskConfig) []clientv3.Op { return ops } +func putValidatorStageOps(stages ...Stage) ([]clientv3.Op, error) { + ops := make([]clientv3.Op, 0, len(stages)) + for _, stage := range stages { + key := common.StageValidatorKeyAdapter.Encode(stage.Source, stage.Task) + value, err := stage.toJSON() + if err != nil { + return nil, err + } + ops = append(ops, clientv3.OpPut(key, value)) + } + return ops, nil +} + +func deleteValidatorStageOps(stages ...Stage) []clientv3.Op { + ops := make([]clientv3.Op, 0, len(stages)) + for _, stage := range stages { + key := common.StageValidatorKeyAdapter.Encode(stage.Source, stage.Task) + ops = append(ops, clientv3.OpDelete(key)) + } + return ops +} + func subTaskCfgFromResp(source, task string, resp *clientv3.GetResponse) (map[string]map[string]config.SubTaskConfig, error) { cfgs := make(map[string]map[string]config.SubTaskConfig) if source != "" { diff --git a/dm/pkg/ha/subtask_test.go b/dm/pkg/ha/subtask_test.go index eb1e9f567e3..5dc537bc131 100644 --- a/dm/pkg/ha/subtask_test.go +++ b/dm/pkg/ha/subtask_test.go @@ -47,7 +47,7 @@ func (t *testForEtcd) TestSubTaskEtcd(c *C) { c.Assert(tsm1, HasLen, 0) // put subtask configs. - rev2, err := PutSubTaskCfgStage(etcdTestCli, []config.SubTaskConfig{cfg1, cfg2}, []Stage{}) + rev2, err := PutSubTaskCfgStage(etcdTestCli, []config.SubTaskConfig{cfg1, cfg2}, []Stage{}, nil) c.Assert(err, IsNil) c.Assert(rev2, Greater, rev1) @@ -92,14 +92,14 @@ func (t *testForEtcd) TestSubTaskEtcd(c *C) { c.Assert(tsm4, HasLen, 0) // put subtask config. - rev6, err := PutSubTaskCfgStage(etcdTestCli, []config.SubTaskConfig{cfg1}, []Stage{}) + rev6, err := PutSubTaskCfgStage(etcdTestCli, []config.SubTaskConfig{cfg1}, []Stage{}, nil) c.Assert(err, IsNil) c.Assert(rev6, Greater, int64(0)) // update subtask config. cfg3 := cfg1 cfg3.SourceID = "testForRevision" - rev7, err := PutSubTaskCfgStage(etcdTestCli, []config.SubTaskConfig{cfg3}, []Stage{}) + rev7, err := PutSubTaskCfgStage(etcdTestCli, []config.SubTaskConfig{cfg3}, []Stage{}, nil) c.Assert(err, IsNil) c.Assert(rev7, Greater, rev6) diff --git a/dm/pkg/terror/error_list.go b/dm/pkg/terror/error_list.go index 04490f95306..7737bfb9374 100644 --- a/dm/pkg/terror/error_list.go +++ b/dm/pkg/terror/error_list.go @@ -249,6 +249,8 @@ const ( codeCollationCompatibleNotSupport codeConfigInvalidLoadMode codeConfigInvalidLoadDuplicateResolution + codeConfigValidationMode + codeContinuousValidatorCfgNotFound ) // Binlog operation error code list. @@ -910,6 +912,8 @@ var ( ErrConfigCollationCompatibleNotSupport = New(codeCollationCompatibleNotSupport, ClassConfig, ScopeInternal, LevelMedium, "collation compatible %s not supported", "Please check the `collation_compatible` config in task configuration file, which can be set to `loose`/`strict`.") ErrConfigInvalidLoadMode = New(codeConfigInvalidLoadMode, ClassConfig, ScopeInternal, LevelMedium, "invalid load mode '%s'", "Please choose a valid value in ['sql', 'loader']") ErrConfigInvalidDuplicateResolution = New(codeConfigInvalidLoadDuplicateResolution, ClassConfig, ScopeInternal, LevelMedium, "invalid load on-duplicate '%s'", "Please choose a valid value in ['replace', 'error', 'ignore']") + ErrConfigValidationMode = New(codeConfigValidationMode, ClassConfig, ScopeInternal, LevelHigh, "invalid validation mode", "Please check `validation-mode` config in task configuration file.") + ErrContinuousValidatorCfgNotFound = New(codeContinuousValidatorCfgNotFound, ClassConfig, ScopeInternal, LevelMedium, "mysql-instance(%d)'s continuous validator config %s not exist", "Please check the `continuous-validator-config-name` config in task configuration file.") // Binlog operation error. ErrBinlogExtractPosition = New(codeBinlogExtractPosition, ClassBinlogOp, ScopeInternal, LevelHigh, "", "") diff --git a/dm/syncer/data_validator.go b/dm/syncer/data_validator.go new file mode 100644 index 00000000000..1eb368e9bac --- /dev/null +++ b/dm/syncer/data_validator.go @@ -0,0 +1,200 @@ +// Copyright 2022 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package syncer + +import ( + "context" + "sync" + "time" + + "github.com/go-mysql-org/go-mysql/replication" + "go.uber.org/zap" + + "github.com/pingcap/tiflow/dm/dm/config" + "github.com/pingcap/tiflow/dm/dm/pb" + "github.com/pingcap/tiflow/dm/dm/unit" + "github.com/pingcap/tiflow/dm/pkg/conn" + tcontext "github.com/pingcap/tiflow/dm/pkg/context" + "github.com/pingcap/tiflow/dm/pkg/log" + "github.com/pingcap/tiflow/dm/pkg/terror" + "github.com/pingcap/tiflow/dm/pkg/utils" + "github.com/pingcap/tiflow/dm/syncer/dbconn" +) + +// DataValidator +// validator can be start when there's syncer unit in the subtask and validation mode is not none, +// it's terminated when the subtask is terminated. +// stage of validator is independent of subtask, pause/resume subtask doesn't affect the stage of validator. +// +// validator can be in running or stopped stage +// - in running when it's started with subtask or started later on the fly. +// - in stopped when validation stop is executed. +type DataValidator struct { + sync.RWMutex + cfg *config.SubTaskConfig + syncer *Syncer + + stage pb.Stage + wg sync.WaitGroup + ctx context.Context + cancel context.CancelFunc + + L log.Logger + fromDB *conn.BaseDB + timezone *time.Location + syncCfg replication.BinlogSyncerConfig + streamerController *StreamerController + + result pb.ProcessResult +} + +func NewContinuousDataValidator(cfg *config.SubTaskConfig, syncerObj *Syncer) *DataValidator { + c := &DataValidator{ + cfg: cfg, + syncer: syncerObj, + stage: pb.Stage_Stopped, + } + c.L = log.With(zap.String("task", cfg.Name), zap.String("unit", "continuous validator")) + return c +} + +func (v *DataValidator) initialize() error { + newCtx, cancelFunc := context.WithTimeout(v.ctx, unit.DefaultInitTimeout) + defer cancelFunc() + tctx := tcontext.NewContext(newCtx, v.L) + + var err error + defer func() { + if err != nil && v.fromDB != nil { + v.fromDB.Close() + } + }() + + dbCfg := v.cfg.From + dbCfg.RawDBCfg = config.DefaultRawDBConfig().SetReadTimeout(maxDMLConnectionTimeout) + v.fromDB, err = dbconn.CreateBaseDB(&dbCfg) + if err != nil { + return err + } + + v.timezone, err = str2TimezoneOrFromDB(tctx, v.cfg.Timezone, &v.cfg.To) + if err != nil { + return err + } + + v.syncCfg, err = subtaskCfg2BinlogSyncerCfg(v.cfg, v.timezone) + if err != nil { + return err + } + + v.streamerController = NewStreamerController(v.syncCfg, v.cfg.EnableGTID, &dbconn.UpStreamConn{BaseDB: v.fromDB}, v.cfg.RelayDir, v.timezone, nil) + + return nil +} + +func (v *DataValidator) Start(expect pb.Stage) { + v.Lock() + defer v.Unlock() + + if v.stage == pb.Stage_Running { + v.L.Info("already started") + return + } + + v.ctx, v.cancel = context.WithCancel(context.Background()) + + if err := v.initialize(); err != nil { + v.fillResult(err, false) + return + } + + if expect != pb.Stage_Running { + return + } + + v.wg.Add(1) + go func() { + defer v.wg.Done() + v.doValidate() + }() + + v.stage = pb.Stage_Running +} + +func (v *DataValidator) fillResult(err error, needLock bool) { + if needLock { + v.Lock() + defer v.Unlock() + } + + var errs []*pb.ProcessError + if utils.IsContextCanceledError(err) { + v.L.Info("filter out context cancelled error", log.ShortError(err)) + } else { + errs = append(errs, unit.NewProcessError(err)) + } + + isCanceled := false + select { + case <-v.ctx.Done(): + isCanceled = true + default: + } + + v.result = pb.ProcessResult{ + IsCanceled: isCanceled, + Errors: errs, + } +} + +func (v *DataValidator) doValidate() { + tctx := tcontext.NewContext(v.ctx, v.L) + err := v.streamerController.Start(tctx, lastLocation) + if err != nil { + v.fillResult(terror.Annotate(err, "fail to restart streamer controller"), true) + return + } + + v.L.Info("start continuous validation") +} + +func (v *DataValidator) Stop() { + v.Lock() + defer v.Unlock() + if v.stage != pb.Stage_Running { + v.L.Warn("not started") + return + } + + v.streamerController.Close() + v.fromDB.Close() + + if v.cancel != nil { + v.cancel() + } + v.wg.Wait() + v.stage = pb.Stage_Stopped +} + +func (v *DataValidator) Started() bool { + v.RLock() + defer v.RUnlock() + return v.stage == pb.Stage_Running +} + +func (v *DataValidator) Stage() pb.Stage { + v.RLock() + defer v.RUnlock() + return v.stage +} diff --git a/dm/syncer/streamer_controller.go b/dm/syncer/streamer_controller.go index 4bcaf28244f..6b0f9aa6705 100644 --- a/dm/syncer/streamer_controller.go +++ b/dm/syncer/streamer_controller.go @@ -339,7 +339,7 @@ func (c *StreamerController) ReopenWithRetry(tctx *tcontext.Context, location bi } // Close closes streamer. -func (c *StreamerController) Close(tctx *tcontext.Context) { +func (c *StreamerController) Close() { c.Lock() c.close() c.Unlock() diff --git a/dm/syncer/syncer.go b/dm/syncer/syncer.go index 6ba6cf5e487..68eb7ef9ba7 100644 --- a/dm/syncer/syncer.go +++ b/dm/syncer/syncer.go @@ -16,7 +16,6 @@ package syncer import ( "bytes" "context" - "crypto/tls" "fmt" "math" "os" @@ -36,7 +35,6 @@ import ( "github.com/pingcap/tidb-tools/pkg/dbutil" "github.com/pingcap/tidb-tools/pkg/filter" router "github.com/pingcap/tidb-tools/pkg/table-router" - toolutils "github.com/pingcap/tidb-tools/pkg/utils" "github.com/pingcap/tidb/parser" "github.com/pingcap/tidb/parser/ast" "github.com/pingcap/tidb/parser/format" @@ -51,7 +49,6 @@ import ( "github.com/pingcap/tiflow/dm/dm/pb" "github.com/pingcap/tiflow/dm/dm/unit" "github.com/pingcap/tiflow/dm/pkg/binlog" - "github.com/pingcap/tiflow/dm/pkg/binlog/common" "github.com/pingcap/tiflow/dm/pkg/binlog/event" "github.com/pingcap/tiflow/dm/pkg/binlog/reader" "github.com/pingcap/tiflow/dm/pkg/conn" @@ -318,11 +315,12 @@ func (s *Syncer) Init(ctx context.Context) (err error) { }() tctx := s.tctx.WithContext(ctx) - if err = s.setTimezone(ctx); err != nil { + s.timezone, err = str2TimezoneOrFromDB(tctx, s.cfg.Timezone, &s.cfg.To) + if err != nil { return } - err = s.setSyncCfg() + s.syncCfg, err = subtaskCfg2BinlogSyncerCfg(s.cfg, s.timezone) if err != nil { return err } @@ -560,7 +558,7 @@ func (s *Syncer) IsFreshTask(ctx context.Context) (bool, error) { func (s *Syncer) reset() { if s.streamerController != nil { - s.streamerController.Close(s.tctx) + s.streamerController.Close() } // create new job chans s.newJobChans() @@ -1544,7 +1542,7 @@ func (s *Syncer) Run(ctx context.Context) (err error) { failpoint.Inject("AdjustGTIDExit", func() { tctx.L().Warn("exit triggered", zap.String("failpoint", "AdjustGTIDExit")) - s.streamerController.Close(tctx) + s.streamerController.Close() utils.OsExit(1) }) @@ -3392,7 +3390,7 @@ func (s *Syncer) stopSync() { // when resuming, re-create s.syncer if s.streamerController != nil { - s.streamerController.Close(s.tctx) + s.streamerController.Close() } } @@ -3527,7 +3525,8 @@ func (s *Syncer) Update(ctx context.Context, cfg *config.SubTaskConfig) error { // update timezone if s.timezone == nil { - return s.setTimezone(ctx) + s.timezone, err = str2TimezoneOrFromDB(s.tctx.WithContext(ctx), s.cfg.Timezone, &s.cfg.To) + return err } return nil } @@ -3565,7 +3564,7 @@ func (s *Syncer) UpdateFromConfig(cfg *config.SubTaskConfig) error { return err } - err = s.setSyncCfg() + s.syncCfg, err = subtaskCfg2BinlogSyncerCfg(s.cfg, s.timezone) if err != nil { return err } @@ -3576,59 +3575,6 @@ func (s *Syncer) UpdateFromConfig(cfg *config.SubTaskConfig) error { return nil } -func (s *Syncer) setTimezone(ctx context.Context) error { - tz := s.cfg.Timezone - var err error - if len(tz) == 0 { - tz, err = conn.FetchTimeZoneSetting(ctx, &s.cfg.To) - if err != nil { - return err - } - } - loc, err := utils.ParseTimeZone(tz) - if err != nil { - return err - } - s.tctx.L().Info("use timezone", zap.String("location", loc.String())) - s.timezone = loc - return nil -} - -func (s *Syncer) setSyncCfg() error { - var tlsConfig *tls.Config - var err error - if s.cfg.From.Security != nil { - if loadErr := s.cfg.From.Security.LoadTLSContent(); loadErr != nil { - return terror.ErrCtlLoadTLSCfg.Delegate(loadErr) - } - tlsConfig, err = toolutils.ToTLSConfigWithVerifyByRawbytes(s.cfg.From.Security.SSLCABytes, - s.cfg.From.Security.SSLCertBytes, s.cfg.From.Security.SSLKEYBytes, s.cfg.From.Security.CertAllowedCN) - if err != nil { - return terror.ErrConnInvalidTLSConfig.Delegate(err) - } - if tlsConfig != nil { - tlsConfig.InsecureSkipVerify = true - } - } - - syncCfg := replication.BinlogSyncerConfig{ - ServerID: s.cfg.ServerID, - Flavor: s.cfg.Flavor, - Host: s.cfg.From.Host, - Port: uint16(s.cfg.From.Port), - User: s.cfg.From.User, - Password: s.cfg.From.Password, - TimestampStringLocation: s.timezone, - TLSConfig: tlsConfig, - } - // when retry count > 1, go-mysql will retry sync from the previous GTID set in GTID mode, - // which may get duplicate binlog event after retry success. so just set retry count = 1, and task - // will exit when meet error, and then auto resume by DM itself. - common.SetDefaultReplicationCfg(&syncCfg, 1) - s.syncCfg = syncCfg - return nil -} - // ShardDDLOperation returns the current pending to handle shard DDL lock operation. func (s *Syncer) ShardDDLOperation() *pessimism.Operation { return s.pessimist.PendingOperation() @@ -3708,7 +3654,7 @@ func (s *Syncer) adjustGlobalPointGTID(tctx *tcontext.Context) (bool, error) { if err != nil { return false, err } - defer streamerController.Close(tctx) + defer streamerController.Close() gs, err := reader.GetGTIDsForPosFromStreamer(tctx.Context(), streamerController.streamer, endPos) if err != nil { diff --git a/dm/syncer/util.go b/dm/syncer/util.go index 384dc5d0b7a..7b27b71af40 100644 --- a/dm/syncer/util.go +++ b/dm/syncer/util.go @@ -14,18 +14,25 @@ package syncer import ( + "crypto/tls" "fmt" + "time" + "github.com/go-mysql-org/go-mysql/replication" "github.com/pingcap/tidb-tools/pkg/filter" + toolutils "github.com/pingcap/tidb-tools/pkg/utils" "github.com/pingcap/tidb/br/pkg/version" "github.com/pingcap/tidb/dumpling/export" dlog "github.com/pingcap/tidb/dumpling/log" "github.com/pingcap/tidb/parser/ast" "go.uber.org/zap" + "github.com/pingcap/tiflow/dm/dm/config" + "github.com/pingcap/tiflow/dm/pkg/binlog/common" "github.com/pingcap/tiflow/dm/pkg/conn" tcontext "github.com/pingcap/tiflow/dm/pkg/context" "github.com/pingcap/tiflow/dm/pkg/terror" + "github.com/pingcap/tiflow/dm/pkg/utils" "github.com/pingcap/tiflow/dm/relay" ) @@ -118,3 +125,53 @@ func printServerVersion(tctx *tcontext.Context, db *conn.BaseDB, scope string) { } version.ParseServerInfo(versionInfo) } + +func str2TimezoneOrFromDB(tctx *tcontext.Context, tzStr string, dbCfg *config.DBConfig) (*time.Location, error) { + var err error + if len(tzStr) == 0 { + tzStr, err = conn.FetchTimeZoneSetting(tctx.Ctx, dbCfg) + if err != nil { + return nil, err + } + } + loc, err := utils.ParseTimeZone(tzStr) + if err != nil { + return nil, err + } + tctx.L().Info("use timezone", zap.String("location", loc.String())) + return loc, nil +} + +func subtaskCfg2BinlogSyncerCfg(cfg *config.SubTaskConfig, timezone *time.Location) (replication.BinlogSyncerConfig, error) { + var tlsConfig *tls.Config + var err error + if cfg.From.Security != nil { + if loadErr := cfg.From.Security.LoadTLSContent(); loadErr != nil { + return replication.BinlogSyncerConfig{}, terror.ErrCtlLoadTLSCfg.Delegate(loadErr) + } + tlsConfig, err = toolutils.ToTLSConfigWithVerifyByRawbytes(cfg.From.Security.SSLCABytes, + cfg.From.Security.SSLCertBytes, cfg.From.Security.SSLKEYBytes, cfg.From.Security.CertAllowedCN) + if err != nil { + return replication.BinlogSyncerConfig{}, terror.ErrConnInvalidTLSConfig.Delegate(err) + } + if tlsConfig != nil { + tlsConfig.InsecureSkipVerify = true + } + } + + syncCfg := replication.BinlogSyncerConfig{ + ServerID: cfg.ServerID, + Flavor: cfg.Flavor, + Host: cfg.From.Host, + Port: uint16(cfg.From.Port), + User: cfg.From.User, + Password: cfg.From.Password, + TimestampStringLocation: timezone, + TLSConfig: tlsConfig, + } + // when retry count > 1, go-mysql will retry sync from the previous GTID set in GTID mode, + // which may get duplicate binlog event after retry success. so just set retry count = 1, and task + // will exit when meet error, and then auto resume by DM itself. + common.SetDefaultReplicationCfg(&syncCfg, 1) + return syncCfg, nil +} diff --git a/dm/tests/dmctl_basic/conf/get_task.yaml b/dm/tests/dmctl_basic/conf/get_task.yaml index 30aa6e566a5..3a431dbdf37 100644 --- a/dm/tests/dmctl_basic/conf/get_task.yaml +++ b/dm/tests/dmctl_basic/conf/get_task.yaml @@ -41,6 +41,7 @@ mysql-instances: syncer-config-name: sync-01 syncer: null syncer-thread: 0 + continuous-validator-config-name: validator-01 - source-id: mysql-replica-02 meta: null filter-rules: @@ -62,6 +63,7 @@ mysql-instances: syncer-config-name: sync-01 syncer: null syncer-thread: 0 + continuous-validator-config-name: validator-01 online-ddl: false shadow-table-rules: [] trash-table-rules: [] @@ -159,6 +161,9 @@ syncers: disable-detect: false safe-mode: false enable-ansi-quotes: false +validators: + validator-01: + mode: none clean-dump-file: true ansi-quotes: false remove-meta: false diff --git a/dm/tests/import_v10x/conf/task.yaml b/dm/tests/import_v10x/conf/task.yaml index a8756979ed2..10e8f111563 100644 --- a/dm/tests/import_v10x/conf/task.yaml +++ b/dm/tests/import_v10x/conf/task.yaml @@ -40,6 +40,7 @@ mysql-instances: syncer-config-name: sync-01 syncer: null syncer-thread: 0 + continuous-validator-config-name: validator-01 - source-id: mysql-replica-02 meta: null filter-rules: [] @@ -57,6 +58,7 @@ mysql-instances: syncer-config-name: sync-02 syncer: null syncer-thread: 0 + continuous-validator-config-name: validator-01 online-ddl: false shadow-table-rules: [] trash-table-rules: [] @@ -118,6 +120,9 @@ syncers: disable-detect: false safe-mode: false enable-ansi-quotes: false +validators: + validator-01: + mode: none clean-dump-file: false ansi-quotes: false remove-meta: false