From ba8b2c566e1d5758fa978ac7fb72c2797f00f6d2 Mon Sep 17 00:00:00 2001 From: gmhdbjd Date: Tue, 9 Mar 2021 21:13:52 +0800 Subject: [PATCH 01/17] only save table and joined info if no conflict --- pkg/shardddl/optimism/lock.go | 7 +++++-- pkg/shardddl/optimism/lock_test.go | 12 ++++++------ 2 files changed, 11 insertions(+), 8 deletions(-) diff --git a/pkg/shardddl/optimism/lock.go b/pkg/shardddl/optimism/lock.go index e537591f04..7490f73d89 100644 --- a/pkg/shardddl/optimism/lock.go +++ b/pkg/shardddl/optimism/lock.go @@ -185,8 +185,9 @@ func (l *Lock) TrySync(callerSource, callerSchema, callerTable string, nextTable = schemacmp.Encode(newTI) // special case: check whether DDLs making the schema become part of larger and another part of smaller. if _, err = prevTable.Compare(nextTable); err != nil { - return emptyDDLs, terror.ErrShardDDLOptimismTrySyncFail.Delegate( + err = terror.ErrShardDDLOptimismTrySyncFail.Delegate( err, l.ID, fmt.Sprintf("there will be conflicts if DDLs %s are applied to the downstream. old table info: %s, new table info: %s", ddls, prevTable, nextTable)) + return emptyDDLs, err } // special case: if the DDL does not affect the schema at all, assume it is @@ -211,6 +212,7 @@ func (l *Lock) TrySync(callerSource, callerSchema, callerTable string, // resolving conflict in non-intrusive mode. log.L().Warn("resolving conflict", zap.String("lock", l.ID), zap.String("source", callerSource), zap.String("schema", callerSchema), zap.String("table", callerTable), zap.Stringer("joined-from", oldJoined), zap.Stringer("joined-to", newJoined), zap.Strings("ddls", ddls)) + err = nil return ddls, nil } if cmp != 0 { @@ -267,8 +269,9 @@ func (l *Lock) TrySync(callerSource, callerSchema, callerTable string, // compare the current table's info with joined info. cmp, err = nextTable.Compare(newJoined) if err != nil { - return emptyDDLs, terror.ErrShardDDLOptimismTrySyncFail.Delegate( + err = terror.ErrShardDDLOptimismTrySyncFail.Delegate( err, l.ID, "can't compare table info (new table info) %s with (new joined table info) %s", nextTable, newJoined) // NOTE: this should not happen. + return emptyDDLs, err } if cmp < 0 { // no need to replicate DDLs, because has a larger joined schema (in the downstream). diff --git a/pkg/shardddl/optimism/lock_test.go b/pkg/shardddl/optimism/lock_test.go index 624c9cc9ad..2a97ea7120 100644 --- a/pkg/shardddl/optimism/lock_test.go +++ b/pkg/shardddl/optimism/lock_test.go @@ -933,12 +933,12 @@ func (t *testLock) TestLockTrySyncConflictIntrusive(c *C) { DDLs1 = []string{"ALTER TABLE bar ADD COLUMN c1 TEXT"} DDLs2 = []string{"ALTER TABLE bar ADD COLUMN c1 DATETIME", "ALTER TABLE bar ADD COLUMN c2 INT"} DDLs3 = []string{"ALTER TABLE bar DROP COLUMN c2"} - DDLs4 = []string{"ALTER TABLE bar DROP COLUMN c1"} - ti0 = createTableInfo(c, p, se, tblID, `CREATE TABLE bar (id INT PRIMARY KEY)`) - ti1 = createTableInfo(c, p, se, tblID, `CREATE TABLE bar (id INT PRIMARY KEY, c1 TEXT)`) - ti2 = createTableInfo(c, p, se, tblID, `CREATE TABLE bar (id INT PRIMARY KEY, c1 DATETIME, c2 INT)`) - ti3 = createTableInfo(c, p, se, tblID, `CREATE TABLE bar (id INT PRIMARY KEY, c1 DATETIME)`) - ti4 = ti0 + DDLs4 = []string{"ALTER TABLE bar DROP COLUMN c1"} + ti0 = createTableInfo(c, p, se, tblID, `CREATE TABLE bar (id INT PRIMARY KEY)`) + ti1 = createTableInfo(c, p, se, tblID, `CREATE TABLE bar (id INT PRIMARY KEY, c1 TEXT)`) + ti2 = createTableInfo(c, p, se, tblID, `CREATE TABLE bar (id INT PRIMARY KEY, c1 DATETIME, c2 INT)`) + ti3 = createTableInfo(c, p, se, tblID, `CREATE TABLE bar (id INT PRIMARY KEY, c1 DATETIME)`) + ti4 = ti0 DDLs5 = []string{"ALTER TABLE bar ADD COLUMN c2 TEXT"} DDLs6 = []string{"ALTER TABLE bar ADD COLUMN c2 DATETIME", "ALTER TABLE bar ADD COLUMN c3 INT"} From d3822f8b9cff26776bb4000f9ba70826a7de7618 Mon Sep 17 00:00:00 2001 From: gmhdbjd Date: Tue, 9 Mar 2021 15:55:15 +0800 Subject: [PATCH 02/17] flush checkpoint after set schema --- syncer/checkpoint.go | 40 ++++++++++ syncer/schema.go | 11 ++- tests/shardddl1/conf/source1.yaml | 4 +- tests/shardddl1/conf/source2.yaml | 2 + tests/shardddl1/run.sh | 119 ++++++++++++++++++++++++++++++ 5 files changed, 172 insertions(+), 4 deletions(-) diff --git a/syncer/checkpoint.go b/syncer/checkpoint.go index ce87c1ba93..dac3179171 100644 --- a/syncer/checkpoint.go +++ b/syncer/checkpoint.go @@ -211,6 +211,9 @@ type CheckPoint interface { // corresponding to Meta.Flush FlushPointsExcept(tctx *tcontext.Context, exceptTables [][]string, extraSQLs []string, extraArgs [][]interface{}) error + // FlushPointWithTableInfo flushed the table point with given table info + FlushPointWithTableInfo(tctx *tcontext.Context, schema string, table string, ti *model.TableInfo) error + // GlobalPoint returns the global binlog stream's checkpoint // corresponding to Meta.Pos and Meta.GTID GlobalPoint() binlog.Location @@ -553,6 +556,43 @@ func (cp *RemoteCheckPoint) FlushPointsExcept(tctx *tcontext.Context, exceptTabl return nil } +// FlushPointWithTableInfo implements CheckPoint.FlushPointWithTableInfo +func (cp *RemoteCheckPoint) FlushPointWithTableInfo(tctx *tcontext.Context, schema string, table string, ti *model.TableInfo) error { + cp.RLock() + defer cp.RUnlock() + + sqls := make([]string, 0, 10) + args := make([][]interface{}, 0, 10) + + point := cp.points[schema][table] + + tiBytes, err := json.Marshal(ti) + if err != nil { + return terror.ErrSchemaTrackerCannotSerialize.Delegate(err, schema, table) + } + + location := point.MySQLLocation() + sql2, arg := cp.genUpdateSQL(schema, table, location, nil, tiBytes, false) + sqls = append(sqls, sql2) + args = append(args, arg) + + // use a new context apart from syncer, to make sure when syncer call `cancel` checkpoint could update + tctx2, cancel := tctx.WithContext(context.Background()).WithTimeout(maxDMLConnectionDuration) + defer cancel() + _, err = cp.dbConn.executeSQL(tctx2, sqls, args...) + if err != nil { + return err + } + + err = point.save(point.location, ti) + if err != nil { + return err + } + point.flush() + + return nil +} + // GlobalPoint implements CheckPoint.GlobalPoint func (cp *RemoteCheckPoint) GlobalPoint() binlog.Location { cp.RLock() diff --git a/syncer/schema.go b/syncer/schema.go index 4afc77d3ec..6e1e5e1422 100644 --- a/syncer/schema.go +++ b/syncer/schema.go @@ -22,6 +22,8 @@ import ( "github.com/pingcap/parser/model" "github.com/pingcap/dm/dm/pb" + tcontext "github.com/pingcap/dm/pkg/context" + "github.com/pingcap/dm/pkg/log" "github.com/pingcap/dm/pkg/schema" "github.com/pingcap/dm/pkg/terror" ) @@ -74,7 +76,14 @@ func (s *Syncer) OperateSchema(ctx context.Context, req *pb.OperateWorkerSchemaR if err != nil { return "", terror.ErrSchemaTrackerCannotCreateTable.Delegate(err, req.Database, req.Table) } - return "", nil + + ti, err := s.schemaTracker.GetTable(req.Database, req.Table) + if err != nil { + return "", err + } + + err = s.checkpoint.FlushPointWithTableInfo(tcontext.NewContext(ctx, log.L()), req.Database, req.Table, ti) + return "", err case pb.SchemaOp_RemoveSchema: // we only drop the schema in the schema-tracker now, // so if we drop the schema and continue to replicate any DDL/DML, it will try to get schema from downstream again. diff --git a/tests/shardddl1/conf/source1.yaml b/tests/shardddl1/conf/source1.yaml index 01660cf685..175e07df7a 100644 --- a/tests/shardddl1/conf/source1.yaml +++ b/tests/shardddl1/conf/source1.yaml @@ -8,6 +8,4 @@ from: password: /Q7B9DizNLLTTfiZHv9WoEAKamfpIUs= port: 3306 checker: - check-enable: true - backoff-rollback: 5m - backoff-max: 5m + check-enable: false diff --git a/tests/shardddl1/conf/source2.yaml b/tests/shardddl1/conf/source2.yaml index 31bd5de10e..e6508fd9d4 100644 --- a/tests/shardddl1/conf/source2.yaml +++ b/tests/shardddl1/conf/source2.yaml @@ -7,3 +7,5 @@ from: user: root password: /Q7B9DizNLLTTfiZHv9WoEAKamfpIUs= port: 3307 +checker: + check-enable: false diff --git a/tests/shardddl1/run.sh b/tests/shardddl1/run.sh index 413e11bb51..6492e551e8 100644 --- a/tests/shardddl1/run.sh +++ b/tests/shardddl1/run.sh @@ -464,6 +464,124 @@ function DM_035() { run_case 035 "double-source-optimistic" "init_table 111 211 212" "clean_table" "optimistic" } + +function DM_RENAME_COLUMN_OPTIMISTIC_CASE() { + run_sql_source1 "insert into ${shardddl1}.${tb1} values(1,'aaa');" + run_sql_source2 "insert into ${shardddl1}.${tb1} values(2,'bbb');" + run_sql_source2 "insert into ${shardddl1}.${tb2} values(3,'ccc');" + + run_sql_source1 "alter table ${shardddl1}.${tb1} change a c int;" + run_sql_source1 "insert into ${shardddl1}.${tb1} values(4,'ddd');" + run_sql_source2 "insert into ${shardddl1}.${tb1} values(5,'eee');" + run_sql_source2 "insert into ${shardddl1}.${tb2} values(6,'fff');" + + run_sql_source2 "alter table ${shardddl1}.${tb1} change a c int;" + run_sql_source1 "insert into ${shardddl1}.${tb1} values(7,'ggg');" + run_sql_source2 "insert into ${shardddl1}.${tb1} values(8,'hhh');" + run_sql_source2 "insert into ${shardddl1}.${tb2} values(9,'iii');" + + run_sql_source2 "alter table ${shardddl1}.${tb2} change a c int;" + run_sql_source1 "insert into ${shardddl1}.${tb1} values(10,'jjj');" + run_sql_source2 "insert into ${shardddl1}.${tb1} values(11,'kkk');" + run_sql_source2 "insert into ${shardddl1}.${tb2} values(12,'lll');" + + run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "query-status test" \ + "because schema conflict detected" 2 + + # first, execute sql in downstream TiDB + run_sql_tidb "alter table ${shardddl}.${tb} change a c int;" + + # second, skip the unsupported ddl + run_dm_ctl $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "handle-error test skip" \ + "\"result\": true" 3 + + # dmls fail + run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "query-status test" \ + "Paused" 2 + #"Error 1054: Unknown column 'a' in 'field list'" 2 // may more than 2 dml error + + # third, set schema to be same with upstream + # TODO: support set schema automatically base on upstream schema + echo 'CREATE TABLE `tb1` ( `c` int NOT NULL, `b` varchar(10) DEFAULT NULL, PRIMARY KEY (`c`)) ENGINE=InnoDB DEFAULT CHARSET=latin1' > ${WORK_DIR}/schema1.sql + run_dm_ctl $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "operate-schema set -s mysql-replica-01 test -d ${shardddl1} -t ${tb1} ${WORK_DIR}/schema1.sql" \ + "\"result\": true" 2 + run_dm_ctl $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "operate-schema set -s mysql-replica-02 test -d ${shardddl1} -t ${tb1} ${WORK_DIR}/schema1.sql" \ + "\"result\": true" 2 + + # fourth, resume-task + run_dm_ctl $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "resume-task test" \ + "\"result\": true" 3 + + # WARN: if it's sequence_sharding, the other tables will not be fixed + # source2.table2's dml fails + run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "query-status test" \ + "Error 1054: Unknown column 'a' in 'field list'" 1 + + # WARN: set schema of source2.table2 + # Actually it should be tb2(a,b), dml is {a: 9, b: 'iii'} + # Now we set it to tb2(c,b), dml become {c: 9, b: 'iii'} + # This only work for a "rename ddl" + echo 'CREATE TABLE `tb2` ( `c` int NOT NULL, `b` varchar(10) DEFAULT NULL, PRIMARY KEY (`c`)) ENGINE=InnoDB DEFAULT CHARSET=latin1' > ${WORK_DIR}/schema2.sql + run_dm_ctl $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "operate-schema set -s mysql-replica-02 test -d ${shardddl1} -t ${tb2} ${WORK_DIR}/schema2.sql" \ + "\"result\": true" 2 + + run_dm_ctl $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "resume-task test -s mysql-replica-02" \ + "\"result\": true" 2 + + # source2.table2's ddl fails + # Unknown column 'a' in 'tb2' + run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "query-status test" \ + "Unknown column 'a' in 'tb2'" 1 + + # skip source2.table2's ddl + run_dm_ctl $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "handle-error test skip -s mysql-replica-02" \ + "\"result\": true" 2 + + check_sync_diff $WORK_DIR $cur/conf/diff_config.toml + + run_sql_source1 "alter table ${shardddl1}.${tb1} drop column b;" + run_sql_source1 "insert into ${shardddl1}.${tb1} values(13);" + run_sql_source2 "insert into ${shardddl1}.${tb1} values(14,'eee');" + run_sql_source2 "insert into ${shardddl1}.${tb2} values(15,'fff');" + + run_sql_source2 "alter table ${shardddl1}.${tb1} drop column b;" + run_sql_source1 "insert into ${shardddl1}.${tb1} values(16);" + run_sql_source2 "insert into ${shardddl1}.${tb1} values(17);" + run_sql_source2 "insert into ${shardddl1}.${tb2} values(18,'iii');" + + run_sql_source2 "alter table ${shardddl1}.${tb2} drop column b;" + run_sql_source1 "insert into ${shardddl1}.${tb1} values(19);" + run_sql_source2 "insert into ${shardddl1}.${tb1} values(20);" + run_sql_source2 "insert into ${shardddl1}.${tb2} values(21);" + + check_sync_diff $WORK_DIR $cur/conf/diff_config.toml + + run_dm_ctl $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "query-status test" \ + "\"result\": true" 3 +} + +# workaround of rename column in optimistic mode currently until we support it +# maybe also work for some other unsupported ddls in optimistic mode +function DM_RENAME_COLUMN_OPTIMISTIC() { + run_case RENAME_COLUMN_OPTIMISTIC "double-source-optimistic" \ + "run_sql_source1 \"create table ${shardddl1}.${tb1} (a int primary key, b varchar(10)) DEFAULT CHARSET=latin1;\"; \ + run_sql_source2 \"create table ${shardddl1}.${tb1} (a int primary key, b varchar(10)) DEFAULT CHARSET=latin1;\"; \ + run_sql_source2 \"create table ${shardddl1}.${tb2} (a int primary key, b varchar(10)) DEFAULT CHARSET=latin1;\"" \ + "clean_table" "optimistic" +} + function run() { init_cluster init_database @@ -477,6 +595,7 @@ function run() { DM_${i} sleep 1 done + DM_RENAME_COLUMN_OPTIMISTIC } cleanup_data $shardddl From 1d599b38660e5d83860f5f80451721f8bddb3e31 Mon Sep 17 00:00:00 2001 From: gmhdbjd Date: Wed, 10 Mar 2021 14:35:37 +0800 Subject: [PATCH 03/17] temp commit --- dm/master/shardddl/optimist.go | 10 ++++-- pkg/shardddl/optimism/info.go | 3 ++ pkg/shardddl/optimism/keeper.go | 2 +- pkg/shardddl/optimism/lock.go | 17 ++++++---- syncer/schema.go | 17 +++++++++- tests/shardddl1/run.sh | 59 +++++++++++++++++++-------------- 6 files changed, 74 insertions(+), 34 deletions(-) diff --git a/dm/master/shardddl/optimist.go b/dm/master/shardddl/optimist.go index b63da694ea..c5ad3c7b07 100644 --- a/dm/master/shardddl/optimist.go +++ b/dm/master/shardddl/optimist.go @@ -391,7 +391,7 @@ func (o *Optimist) handleInfo(ctx context.Context, infoCh <-chan optimism.Info) if !ok { return } - o.logger.Info("receive a shard DDL info", zap.Stringer("info", info), zap.Bool("is deleted", info.IsDeleted)) + o.logger.Info("receive a shard DDL info", zap.Stringer("info", info), zap.Bool("is deleted", info.IsDeleted), zap.Bool("ignore conflict", info.IgnoreConflict)) // avoid new ddl added while previous ddl resolved and remove lock // change lock granularity if needed @@ -499,7 +499,9 @@ func (o *Optimist) handleOperationPut(ctx context.Context, opCh <-chan optimism. func (o *Optimist) handleLock(info optimism.Info, tts []optimism.TargetTable, skipDone bool) error { lockID, newDDLs, err := o.lk.TrySync(info, tts) var cfStage = optimism.ConflictNone - if err != nil { + if info.IgnoreConflict { + o.logger.Warn("handle lock in ignore conflict mode", zap.String("lock", lockID), zap.Stringer("info", info)) + } else if err != nil { cfStage = optimism.ConflictDetected // we treat any errors returned from `TrySync` as conflict detected now. o.logger.Warn("error occur when trying to sync for shard DDL info, this often means shard DDL conflict detected", zap.String("lock", lockID), zap.Stringer("info", info), zap.Bool("is deleted", info.IsDeleted), log.ShortError(err)) @@ -536,6 +538,10 @@ func (o *Optimist) handleLock(info optimism.Info, tts []optimism.TargetTable, sk return nil } + if info.IgnoreConflict { + return nil + } + op := optimism.NewOperation(lockID, lock.Task, info.Source, info.UpSchema, info.UpTable, newDDLs, cfStage, false) rev, succ, err := optimism.PutOperation(o.cli, skipDone, op) if err != nil { diff --git a/pkg/shardddl/optimism/info.go b/pkg/shardddl/optimism/info.go index 31254b02bb..38c11918d6 100644 --- a/pkg/shardddl/optimism/info.go +++ b/pkg/shardddl/optimism/info.go @@ -53,6 +53,9 @@ type Info struct { // only set it when get/watch from etcd Version int64 `json:"-"` + + // use to resolve conflict + IgnoreConflict bool `json:"ignore-conflict"` } // NewInfo creates a new Info instance. diff --git a/pkg/shardddl/optimism/keeper.go b/pkg/shardddl/optimism/keeper.go index 0a30aa70a6..099b444590 100644 --- a/pkg/shardddl/optimism/keeper.go +++ b/pkg/shardddl/optimism/keeper.go @@ -50,7 +50,7 @@ func (lk *LockKeeper) TrySync(info Info, tts []TargetTable) (string, []string, e l = lk.locks[lockID] } - newDDLs, err := l.TrySync(info.Source, info.UpSchema, info.UpTable, info.DDLs, info.TableInfosAfter, tts, info.Version) + newDDLs, err := l.TrySync(info.Source, info.UpSchema, info.UpTable, info.DDLs, info.TableInfosAfter, tts, info.Version, info.IgnoreConflict) return lockID, newDDLs, err } diff --git a/pkg/shardddl/optimism/lock.go b/pkg/shardddl/optimism/lock.go index 7490f73d89..85fad901a8 100644 --- a/pkg/shardddl/optimism/lock.go +++ b/pkg/shardddl/optimism/lock.go @@ -92,7 +92,7 @@ func NewLock(ID, task, downSchema, downTable string, ti *model.TableInfo, tts [] // for non-intrusive, a broadcast mechanism needed to notify conflict tables after the conflict has resolved, or even a block mechanism needed. // for intrusive, a DML prune or transform mechanism needed for two different schemas (before and after the conflict resolved). func (l *Lock) TrySync(callerSource, callerSchema, callerTable string, - ddls []string, newTIs []*model.TableInfo, tts []TargetTable, infoVersion int64) (newDDLs []string, err error) { + ddls []string, newTIs []*model.TableInfo, tts []TargetTable, infoVersion int64, ignoreConflict bool) (newDDLs []string, err error) { l.mu.Lock() defer func() { if len(newDDLs) > 0 { @@ -137,7 +137,7 @@ func (l *Lock) TrySync(callerSource, callerSchema, callerTable string, } // should not happen - if len(ddls) != len(newTIs) || len(newTIs) == 0 { + if !ignoreConflict && (len(ddls) != len(newTIs) || len(newTIs) == 0) { return ddls, terror.ErrMasterInconsistentOptimisticDDLsAndInfo.Generate(len(ddls), len(newTIs)) } @@ -157,6 +157,15 @@ func (l *Lock) TrySync(callerSource, callerSchema, callerTable string, oldJoined := l.joined lastTableInfo := schemacmp.Encode(newTIs[len(newTIs)-1]) + + defer func() { + if ignoreConflict || err == nil { + log.L().Info("update table info", zap.String("lock", l.ID), zap.String("source", callerSource), zap.String("schema", callerSchema), zap.String("table", callerTable), + zap.Stringer("from", prevTable), zap.Stringer("to", lastTableInfo), zap.Strings("ddls", ddls)) + l.tables[callerSource][callerSchema][callerTable] = lastTableInfo + } + }() + lastJoined, err := joinTable(lastTableInfo) if err != nil { return emptyDDLs, err @@ -165,10 +174,6 @@ func (l *Lock) TrySync(callerSource, callerSchema, callerTable string, defer func() { // only update table info and joined info if no error if err == nil { - // update table info and joined info base on the last new table info - log.L().Info("update table info", zap.String("lock", l.ID), zap.String("source", callerSource), zap.String("schema", callerSchema), zap.String("table", callerTable), - zap.Stringer("from", l.tables[callerSource][callerSchema][callerTable]), zap.Stringer("to", lastTableInfo), zap.Strings("ddls", ddls)) - l.tables[callerSource][callerSchema][callerTable] = lastTableInfo // update the current joined table info, it should be logged in `if cmp != 0` block below. l.joined = lastJoined } diff --git a/syncer/schema.go b/syncer/schema.go index 6e1e5e1422..ca37270c83 100644 --- a/syncer/schema.go +++ b/syncer/schema.go @@ -21,6 +21,7 @@ import ( "github.com/pingcap/parser/format" "github.com/pingcap/parser/model" + "github.com/pingcap/dm/dm/config" "github.com/pingcap/dm/dm/pb" tcontext "github.com/pingcap/dm/pkg/context" "github.com/pingcap/dm/pkg/log" @@ -83,7 +84,21 @@ func (s *Syncer) OperateSchema(ctx context.Context, req *pb.OperateWorkerSchemaR } err = s.checkpoint.FlushPointWithTableInfo(tcontext.NewContext(ctx, log.L()), req.Database, req.Table, ti) - return "", err + if err != nil { + return "", err + } + + if s.cfg.ShardMode == config.ShardOptimistic { + downSchema, downTable := s.renameShardingSchema(req.Database, req.Table) + info := s.optimist.ConstructInfo(req.Database, req.Table, downSchema, downTable, []string{""}, nil, []*model.TableInfo{ti}) + info.IgnoreConflict = true + log.L().Info("resolve conflict with operateschema") + _, err = s.optimist.PutInfo(info) + if err != nil { + return "", err + } + } + case pb.SchemaOp_RemoveSchema: // we only drop the schema in the schema-tracker now, // so if we drop the schema and continue to replicate any DDL/DML, it will try to get schema from downstream again. diff --git a/tests/shardddl1/run.sh b/tests/shardddl1/run.sh index 6492e551e8..7bd5c1b307 100644 --- a/tests/shardddl1/run.sh +++ b/tests/shardddl1/run.sh @@ -527,7 +527,7 @@ function DM_RENAME_COLUMN_OPTIMISTIC_CASE() { # WARN: set schema of source2.table2 # Actually it should be tb2(a,b), dml is {a: 9, b: 'iii'} # Now we set it to tb2(c,b), dml become {c: 9, b: 'iii'} - # This only work for a "rename ddl" + # This may only work for a "rename ddl" echo 'CREATE TABLE `tb2` ( `c` int NOT NULL, `b` varchar(10) DEFAULT NULL, PRIMARY KEY (`c`)) ENGINE=InnoDB DEFAULT CHARSET=latin1' > ${WORK_DIR}/schema2.sql run_dm_ctl $WORK_DIR "127.0.0.1:$MASTER_PORT" \ "operate-schema set -s mysql-replica-02 test -d ${shardddl1} -t ${tb2} ${WORK_DIR}/schema2.sql" \ @@ -549,27 +549,38 @@ function DM_RENAME_COLUMN_OPTIMISTIC_CASE() { "\"result\": true" 2 check_sync_diff $WORK_DIR $cur/conf/diff_config.toml + run_dm_ctl $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "query-status test" \ + "\"result\": true" 3 + run_dm_ctl $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "show-ddl-locks" \ + "no DDL lock exists" 1 - run_sql_source1 "alter table ${shardddl1}.${tb1} drop column b;" - run_sql_source1 "insert into ${shardddl1}.${tb1} values(13);" - run_sql_source2 "insert into ${shardddl1}.${tb1} values(14,'eee');" - run_sql_source2 "insert into ${shardddl1}.${tb2} values(15,'fff');" + # now, it works as normal + run_sql_source1 "alter table ${shardddl1}.${tb1} add column d int;" + run_sql_source1 "insert into ${shardddl1}.${tb1} values(13,'mmm',13);" + run_sql_source2 "insert into ${shardddl1}.${tb1} values(14,'nnn');" + run_sql_source2 "insert into ${shardddl1}.${tb2} values(15,'ooo');" - run_sql_source2 "alter table ${shardddl1}.${tb1} drop column b;" - run_sql_source1 "insert into ${shardddl1}.${tb1} values(16);" - run_sql_source2 "insert into ${shardddl1}.${tb1} values(17);" - run_sql_source2 "insert into ${shardddl1}.${tb2} values(18,'iii');" + run_sql_source2 "alter table ${shardddl1}.${tb1} add column d int;" + run_sql_source1 "insert into ${shardddl1}.${tb1} values(16,'ppp',16);" + run_sql_source2 "insert into ${shardddl1}.${tb1} values(17,'qqq',17);" + run_sql_source2 "insert into ${shardddl1}.${tb2} values(18,'rrr');" - run_sql_source2 "alter table ${shardddl1}.${tb2} drop column b;" - run_sql_source1 "insert into ${shardddl1}.${tb1} values(19);" - run_sql_source2 "insert into ${shardddl1}.${tb1} values(20);" - run_sql_source2 "insert into ${shardddl1}.${tb2} values(21);" + run_sql_source2 "alter table ${shardddl1}.${tb2} add column d int;" + run_sql_source1 "insert into ${shardddl1}.${tb1} values(19,'sss',19);" + run_sql_source2 "insert into ${shardddl1}.${tb1} values(20,'ttt',20);" + run_sql_source2 "insert into ${shardddl1}.${tb2} values(21,'uuu',21);" check_sync_diff $WORK_DIR $cur/conf/diff_config.toml - run_dm_ctl $WORK_DIR "127.0.0.1:$MASTER_PORT" \ "query-status test" \ "\"result\": true" 3 + run_dm_ctl $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "show-ddl-locks" \ + "no DDL lock exists" 1 + + read v1 } # workaround of rename column in optimistic mode currently until we support it @@ -585,16 +596,16 @@ function DM_RENAME_COLUMN_OPTIMISTIC() { function run() { init_cluster init_database - start=1 - end=35 - except=(024 025 029) - for i in $(seq -f "%03g" ${start} ${end}); do - if [[ ${except[@]} =~ $i ]]; then - continue - fi - DM_${i} - sleep 1 - done +# start=1 +# end=35 +# except=(024 025 029) +# for i in $(seq -f "%03g" ${start} ${end}); do +# if [[ ${except[@]} =~ $i ]]; then +# continue +# fi +# DM_${i} +# sleep 1 +# done DM_RENAME_COLUMN_OPTIMISTIC } From 2bdcc4dee805c7b48ef59b6465ff4c8516b474c3 Mon Sep 17 00:00:00 2001 From: gmhdbjd Date: Fri, 12 Mar 2021 13:46:55 +0800 Subject: [PATCH 04/17] temp commit --- pkg/shardddl/optimism/info_test.go | 2 +- pkg/shardddl/optimism/keeper.go | 2 +- pkg/shardddl/optimism/lock.go | 16 +- pkg/shardddl/optimism/lock_test.go | 259 +++++++++++++++++++++-------- tests/shardddl1/run.sh | 29 ++-- 5 files changed, 220 insertions(+), 88 deletions(-) diff --git a/pkg/shardddl/optimism/info_test.go b/pkg/shardddl/optimism/info_test.go index 37d9dadb06..12b824c5bb 100644 --- a/pkg/shardddl/optimism/info_test.go +++ b/pkg/shardddl/optimism/info_test.go @@ -77,7 +77,7 @@ func (t *testForEtcd) TestInfoJSON(c *C) { j, err := i1.toJSON() c.Assert(err, IsNil) - c.Assert(j, Equals, `{"task":"test","source":"mysql-replica-1","up-schema":"db-1","up-table":"tbl-1","down-schema":"db","down-table":"tbl","ddls":["ALTER TABLE tbl ADD COLUMN c1 INT","ALTER TABLE tbl ADD COLUMN c2 INT"],"table-info-before":null,"table-info-after":null}`) + c.Assert(j, Equals, `{"task":"test","source":"mysql-replica-1","up-schema":"db-1","up-table":"tbl-1","down-schema":"db","down-table":"tbl","ddls":["ALTER TABLE tbl ADD COLUMN c1 INT","ALTER TABLE tbl ADD COLUMN c2 INT"],"table-info-before":null,"table-info-after":null,"ignore-conflict":false}`) c.Assert(j, Equals, i1.String()) i2, err := infoFromJSON(j) diff --git a/pkg/shardddl/optimism/keeper.go b/pkg/shardddl/optimism/keeper.go index 099b444590..5c6a9ec67e 100644 --- a/pkg/shardddl/optimism/keeper.go +++ b/pkg/shardddl/optimism/keeper.go @@ -50,7 +50,7 @@ func (lk *LockKeeper) TrySync(info Info, tts []TargetTable) (string, []string, e l = lk.locks[lockID] } - newDDLs, err := l.TrySync(info.Source, info.UpSchema, info.UpTable, info.DDLs, info.TableInfosAfter, tts, info.Version, info.IgnoreConflict) + newDDLs, err := l.TrySync(info, tts) return lockID, newDDLs, err } diff --git a/pkg/shardddl/optimism/lock.go b/pkg/shardddl/optimism/lock.go index 85fad901a8..e677534adb 100644 --- a/pkg/shardddl/optimism/lock.go +++ b/pkg/shardddl/optimism/lock.go @@ -91,8 +91,16 @@ func NewLock(ID, task, downSchema, downTable string, ti *model.TableInfo, tts [] // TODO: but both of these modes are difficult to be implemented in DM-worker now, try to do that later. // for non-intrusive, a broadcast mechanism needed to notify conflict tables after the conflict has resolved, or even a block mechanism needed. // for intrusive, a DML prune or transform mechanism needed for two different schemas (before and after the conflict resolved). -func (l *Lock) TrySync(callerSource, callerSchema, callerTable string, - ddls []string, newTIs []*model.TableInfo, tts []TargetTable, infoVersion int64, ignoreConflict bool) (newDDLs []string, err error) { +func (l *Lock) TrySync(info Info, tts []TargetTable) (newDDLs []string, err error) { + var ( + callerSource = info.Source + callerSchema = info.UpSchema + callerTable = info.UpTable + ddls = info.DDLs + newTIs = info.TableInfosAfter + infoVersion = info.Version + ignoreConflict = info.IgnoreConflict + ) l.mu.Lock() defer func() { if len(newDDLs) > 0 { @@ -137,7 +145,7 @@ func (l *Lock) TrySync(callerSource, callerSchema, callerTable string, } // should not happen - if !ignoreConflict && (len(ddls) != len(newTIs) || len(newTIs) == 0) { + if len(ddls) != len(newTIs) || len(newTIs) == 0 { return ddls, terror.ErrMasterInconsistentOptimisticDDLsAndInfo.Generate(len(ddls), len(newTIs)) } @@ -159,6 +167,7 @@ func (l *Lock) TrySync(callerSource, callerSchema, callerTable string, lastTableInfo := schemacmp.Encode(newTIs[len(newTIs)-1]) defer func() { + // only update table info if no error or ignore conflict if ignoreConflict || err == nil { log.L().Info("update table info", zap.String("lock", l.ID), zap.String("source", callerSource), zap.String("schema", callerSchema), zap.String("table", callerTable), zap.Stringer("from", prevTable), zap.Stringer("to", lastTableInfo), zap.Strings("ddls", ddls)) @@ -172,7 +181,6 @@ func (l *Lock) TrySync(callerSource, callerSchema, callerTable string, } defer func() { - // only update table info and joined info if no error if err == nil { // update the current joined table info, it should be logged in `if cmp != 0` block below. l.joined = lastJoined diff --git a/pkg/shardddl/optimism/lock_test.go b/pkg/shardddl/optimism/lock_test.go index 2a97ea7120..be244861fd 100644 --- a/pkg/shardddl/optimism/lock_test.go +++ b/pkg/shardddl/optimism/lock_test.go @@ -65,7 +65,8 @@ func (t *testLock) TestLockTrySyncNormal(c *C) { newTargetTable(task, sources[1], downSchema, downTable, tables), } - l = NewLock(ID, task, downSchema, downTable, ti0, tts) + l = NewLock(ID, task, downSchema, downTable, ti0, tts) + info = NewInfo("", "", "", "", "", "", []string{}, nil, nil) vers = map[string]map[string]map[string]int64{ sources[0]: { @@ -101,7 +102,9 @@ func (t *testLock) TestLockTrySyncNormal(c *C) { for _, db := range dbs { for _, tbl := range tbls { vers[source][db][tbl]++ - DDLs, err := l.TrySync(source, db, tbl, DDLs1, []*model.TableInfo{ti1}, tts, vers[source][db][tbl]) + info = NewInfo(task, source, db, tbl, downSchema, downTable, DDLs1, nil, []*model.TableInfo{ti1}) + info.Version = vers[source][db][tbl] + DDLs, err := l.TrySync(info, tts) c.Assert(err, IsNil) c.Assert(DDLs, DeepEquals, DDLs1) c.Assert(l.versions, DeepEquals, vers) @@ -120,7 +123,9 @@ func (t *testLock) TestLockTrySyncNormal(c *C) { // CASE: TrySync again after synced is idempotent. vers[sources[0]][dbs[0]][tbls[0]]++ - DDLs, err := l.TrySync(sources[0], dbs[0], tbls[0], DDLs1, []*model.TableInfo{ti1}, tts, vers[sources[0]][dbs[0]][tbls[0]]) + info = NewInfo(task, sources[0], dbs[0], tbls[0], downSchema, downTable, DDLs1, nil, []*model.TableInfo{ti1}) + info.Version = vers[sources[0]][dbs[0]][tbls[0]] + DDLs, err := l.TrySync(info, tts) c.Assert(err, IsNil) c.Assert(DDLs, DeepEquals, DDLs1) c.Assert(l.versions, DeepEquals, vers) @@ -130,7 +135,9 @@ func (t *testLock) TestLockTrySyncNormal(c *C) { // CASE: need to add more than one DDL to reach the desired schema (schema become larger). // add two columns for one table. vers[sources[0]][dbs[0]][tbls[0]]++ - DDLs, err = l.TrySync(sources[0], dbs[0], tbls[0], DDLs2, []*model.TableInfo{ti2_1, ti2}, tts, vers[sources[0]][dbs[0]][tbls[0]]) + info = NewInfo(task, sources[0], dbs[0], tbls[0], downSchema, downTable, DDLs2, nil, []*model.TableInfo{ti2_1, ti2}) + info.Version = vers[sources[0]][dbs[0]][tbls[0]] + DDLs, err = l.TrySync(info, tts) c.Assert(err, IsNil) c.Assert(DDLs, DeepEquals, DDLs2) c.Assert(l.versions, DeepEquals, vers) @@ -140,7 +147,9 @@ func (t *testLock) TestLockTrySyncNormal(c *C) { // TrySync again is idempotent (more than one DDL). vers[sources[0]][dbs[0]][tbls[0]]++ - DDLs, err = l.TrySync(sources[0], dbs[0], tbls[0], DDLs2, []*model.TableInfo{ti2_1, ti2}, tts, vers[sources[0]][dbs[0]][tbls[0]]) + info = NewInfo(task, sources[0], dbs[0], tbls[0], downSchema, downTable, DDLs2, nil, []*model.TableInfo{ti2_1, ti2}) + info.Version = vers[sources[0]][dbs[0]][tbls[0]] + DDLs, err = l.TrySync(info, tts) c.Assert(err, IsNil) c.Assert(DDLs, DeepEquals, DDLs2) c.Assert(l.versions, DeepEquals, vers) @@ -150,7 +159,9 @@ func (t *testLock) TestLockTrySyncNormal(c *C) { // add only the first column for another table. vers[sources[0]][dbs[0]][tbls[1]]++ - DDLs, err = l.TrySync(sources[0], dbs[0], tbls[1], DDLs2[0:1], []*model.TableInfo{ti2_1}, tts, vers[sources[0]][dbs[0]][tbls[1]]) // use ti2_1 info + info = NewInfo(task, sources[0], dbs[0], tbls[1], downSchema, downTable, DDLs2[0:1], nil, []*model.TableInfo{ti2_1}) + info.Version = vers[sources[0]][dbs[0]][tbls[1]] + DDLs, err = l.TrySync(info, tts) c.Assert(err, IsNil) c.Assert(DDLs, DeepEquals, DDLs2[0:1]) c.Assert(l.versions, DeepEquals, vers) @@ -167,7 +178,9 @@ func (t *testLock) TestLockTrySyncNormal(c *C) { // TrySync again (only the first DDL). vers[sources[0]][dbs[0]][tbls[1]]++ - DDLs, err = l.TrySync(sources[0], dbs[0], tbls[1], DDLs2[0:1], []*model.TableInfo{ti2_1}, tts, vers[sources[0]][dbs[0]][tbls[1]]) + info = NewInfo(task, sources[0], dbs[0], tbls[1], downSchema, downTable, DDLs2[0:1], nil, []*model.TableInfo{ti2_1}) + info.Version = vers[sources[0]][dbs[0]][tbls[1]] + DDLs, err = l.TrySync(info, tts) c.Assert(err, IsNil) c.Assert(DDLs, DeepEquals, []string{}) // NOTE: special case, joined has larger schema. c.Assert(l.versions, DeepEquals, vers) @@ -176,7 +189,9 @@ func (t *testLock) TestLockTrySyncNormal(c *C) { // add the second column for another table. vers[sources[0]][dbs[0]][tbls[1]]++ - DDLs, err = l.TrySync(sources[0], dbs[0], tbls[1], DDLs2[1:2], []*model.TableInfo{ti2}, tts, vers[sources[0]][dbs[0]][tbls[1]]) // use ti2 info. + info = NewInfo(task, sources[0], dbs[0], tbls[1], downSchema, downTable, DDLs2[1:2], nil, []*model.TableInfo{ti2}) + info.Version = vers[sources[0]][dbs[0]][tbls[1]] + DDLs, err = l.TrySync(info, tts) c.Assert(err, IsNil) c.Assert(DDLs, DeepEquals, DDLs2[1:2]) c.Assert(l.versions, DeepEquals, vers) @@ -192,7 +207,9 @@ func (t *testLock) TestLockTrySyncNormal(c *C) { // Try again (for the second DDL). vers[sources[0]][dbs[0]][tbls[1]]++ - DDLs, err = l.TrySync(sources[0], dbs[0], tbls[1], DDLs2[1:2], []*model.TableInfo{ti2}, tts, vers[sources[0]][dbs[0]][tbls[1]]) + info = NewInfo(task, sources[0], dbs[0], tbls[1], downSchema, downTable, DDLs2[1:2], nil, []*model.TableInfo{ti2}) + info.Version = vers[sources[0]][dbs[0]][tbls[1]] + DDLs, err = l.TrySync(info, tts) c.Assert(err, IsNil) c.Assert(DDLs, DeepEquals, DDLs2[1:2]) c.Assert(l.versions, DeepEquals, vers) @@ -231,7 +248,9 @@ func (t *testLock) TestLockTrySyncNormal(c *C) { for _, tbl := range tbls { syncedCount++ vers[source][db][tbl]++ - DDLs, err = l.TrySync(source, db, tbl, DDLs3, []*model.TableInfo{ti3}, tts, vers[source][db][tbl]) + info = NewInfo(task, source, db, tbl, downSchema, downTable, DDLs3, nil, []*model.TableInfo{ti3}) + info.Version = vers[source][db][tbl] + DDLs, err = l.TrySync(info, tts) c.Assert(err, IsNil) c.Assert(l.versions, DeepEquals, vers) synced, remain = l.IsSynced() @@ -254,7 +273,9 @@ func (t *testLock) TestLockTrySyncNormal(c *C) { // CASE: need to drop more than one DDL to reach the desired schema (schema become smaller). // drop two columns for one table. vers[sources[0]][dbs[0]][tbls[0]]++ - DDLs, err = l.TrySync(sources[0], dbs[0], tbls[0], DDLs4, []*model.TableInfo{ti4_1, ti4}, tts, vers[sources[0]][dbs[0]][tbls[0]]) + info = NewInfo(task, sources[0], dbs[0], tbls[0], downSchema, downTable, DDLs4, nil, []*model.TableInfo{ti4_1, ti4}) + info.Version = vers[sources[0]][dbs[0]][tbls[0]] + DDLs, err = l.TrySync(info, tts) c.Assert(err, IsNil) c.Assert(DDLs, DeepEquals, []string{}) c.Assert(l.versions, DeepEquals, vers) @@ -264,7 +285,9 @@ func (t *testLock) TestLockTrySyncNormal(c *C) { // TrySync again is idempotent. vers[sources[0]][dbs[0]][tbls[0]]++ - DDLs, err = l.TrySync(sources[0], dbs[0], tbls[0], DDLs4, []*model.TableInfo{ti4_1, ti4}, tts, vers[sources[0]][dbs[0]][tbls[0]]) + info = NewInfo(task, sources[0], dbs[0], tbls[0], downSchema, downTable, DDLs4, nil, []*model.TableInfo{ti4_1, ti4}) + info.Version = vers[sources[0]][dbs[0]][tbls[0]] + DDLs, err = l.TrySync(info, tts) c.Assert(err, IsNil) c.Assert(DDLs, DeepEquals, DDLs4[:1]) c.Assert(l.versions, DeepEquals, vers) @@ -274,7 +297,9 @@ func (t *testLock) TestLockTrySyncNormal(c *C) { // drop only the first column for another table. vers[sources[0]][dbs[0]][tbls[1]]++ - DDLs, err = l.TrySync(sources[0], dbs[0], tbls[1], DDLs4[0:1], []*model.TableInfo{ti4_1}, tts, vers[sources[0]][dbs[0]][tbls[1]]) + info = NewInfo(task, sources[0], dbs[0], tbls[1], downSchema, downTable, DDLs4[0:1], nil, []*model.TableInfo{ti4_1}) + info.Version = vers[sources[0]][dbs[0]][tbls[1]] + DDLs, err = l.TrySync(info, tts) c.Assert(err, IsNil) c.Assert(DDLs, DeepEquals, []string{}) c.Assert(l.versions, DeepEquals, vers) @@ -287,14 +312,18 @@ func (t *testLock) TestLockTrySyncNormal(c *C) { // TrySync again (only the first DDL). vers[sources[0]][dbs[0]][tbls[1]]++ - DDLs, err = l.TrySync(sources[0], dbs[0], tbls[1], DDLs4[0:1], []*model.TableInfo{ti4_1}, tts, vers[sources[0]][dbs[0]][tbls[1]]) + info = NewInfo(task, sources[0], dbs[0], tbls[1], downSchema, downTable, DDLs4[0:1], nil, []*model.TableInfo{ti4_1}) + info.Version = vers[sources[0]][dbs[0]][tbls[1]] + DDLs, err = l.TrySync(info, tts) c.Assert(err, IsNil) c.Assert(DDLs, DeepEquals, []string{}) c.Assert(l.versions, DeepEquals, vers) // drop the second column for another table. vers[sources[0]][dbs[0]][tbls[1]]++ - DDLs, err = l.TrySync(sources[0], dbs[0], tbls[1], DDLs4[1:2], []*model.TableInfo{ti4}, tts, vers[sources[0]][dbs[0]][tbls[1]]) // use ti4 info. + info = NewInfo(task, sources[0], dbs[0], tbls[1], downSchema, downTable, DDLs4[1:2], nil, []*model.TableInfo{ti4}) + info.Version = vers[sources[0]][dbs[0]][tbls[1]] + DDLs, err = l.TrySync(info, tts) c.Assert(err, IsNil) c.Assert(DDLs, DeepEquals, []string{}) c.Assert(l.versions, DeepEquals, vers) @@ -307,7 +336,9 @@ func (t *testLock) TestLockTrySyncNormal(c *C) { // TrySync again (for the second DDL). vers[sources[0]][dbs[0]][tbls[1]]++ - DDLs, err = l.TrySync(sources[0], dbs[0], tbls[1], DDLs4[1:2], []*model.TableInfo{ti4}, tts, vers[sources[0]][dbs[0]][tbls[1]]) + info = NewInfo(task, sources[0], dbs[0], tbls[1], downSchema, downTable, DDLs4[1:2], nil, []*model.TableInfo{ti4}) + info.Version = vers[sources[0]][dbs[0]][tbls[1]] + DDLs, err = l.TrySync(info, tts) c.Assert(err, IsNil) c.Assert(DDLs, DeepEquals, []string{}) c.Assert(l.versions, DeepEquals, vers) @@ -318,7 +349,9 @@ func (t *testLock) TestLockTrySyncNormal(c *C) { for schema, tables := range schemaTables { for table, synced2 := range tables { if synced2 { // do not `TrySync` again for previous two (un-synced now). - DDLs, err = l.TrySync(source, schema, table, DDLs4, []*model.TableInfo{ti4_1, ti4}, tts, vers[source][schema][table]) + info = NewInfo(task, source, schema, table, downSchema, downTable, DDLs4, nil, []*model.TableInfo{ti4_1, ti4}) + info.Version = vers[source][schema][table] + DDLs, err = l.TrySync(info, tts) c.Assert(err, IsNil) c.Assert(l.versions, DeepEquals, vers) remain-- @@ -375,7 +408,9 @@ func (t *testLock) TestLockTrySyncIndex(c *C) { // try sync for one table, `DROP INDEX` returned directly (to make schema become more compatible). // `DROP INDEX` is handled like `ADD COLUMN`. vers[source][db][tbls[0]]++ - DDLs, err := l.TrySync(source, db, tbls[0], DDLs1, []*model.TableInfo{ti1}, tts, vers[source][db][tbls[0]]) + info := NewInfo(task, source, db, tbls[0], downSchema, downTable, DDLs1, nil, []*model.TableInfo{ti1}) + info.Version = vers[source][db][tbls[0]] + DDLs, err := l.TrySync(info, tts) c.Assert(err, IsNil) c.Assert(DDLs, DeepEquals, DDLs1) c.Assert(l.versions, DeepEquals, vers) @@ -386,7 +421,9 @@ func (t *testLock) TestLockTrySyncIndex(c *C) { // try sync for another table, also got `DROP INDEX` now. vers[source][db][tbls[1]]++ - DDLs, err = l.TrySync(source, db, tbls[1], DDLs1, []*model.TableInfo{ti1}, tts, vers[source][db][tbls[1]]) + info = NewInfo(task, source, db, tbls[1], downSchema, downTable, DDLs1, nil, []*model.TableInfo{ti1}) + info.Version = vers[source][db][tbls[1]] + DDLs, err = l.TrySync(info, tts) c.Assert(err, IsNil) c.Assert(DDLs, DeepEquals, DDLs1) c.Assert(l.versions, DeepEquals, vers) @@ -395,7 +432,9 @@ func (t *testLock) TestLockTrySyncIndex(c *C) { // try sync for one table, `ADD INDEX` not returned directly (to keep the schema more compatible). // `ADD INDEX` is handled like `DROP COLUMN`. vers[source][db][tbls[0]]++ - DDLs, err = l.TrySync(source, db, tbls[0], DDLs2, []*model.TableInfo{ti2}, tts, vers[source][db][tbls[0]]) + info = NewInfo(task, source, db, tbls[0], downSchema, downTable, DDLs2, nil, []*model.TableInfo{ti2}) + info.Version = vers[source][db][tbls[0]] + DDLs, err = l.TrySync(info, tts) c.Assert(err, IsNil) c.Assert(DDLs, DeepEquals, []string{}) // no DDLs returned c.Assert(l.versions, DeepEquals, vers) @@ -406,7 +445,9 @@ func (t *testLock) TestLockTrySyncIndex(c *C) { // try sync for another table, got `ADD INDEX` now. vers[source][db][tbls[1]]++ - DDLs, err = l.TrySync(source, db, tbls[1], DDLs2, []*model.TableInfo{ti2}, tts, vers[source][db][tbls[1]]) + info = NewInfo(task, source, db, tbls[1], downSchema, downTable, DDLs2, nil, []*model.TableInfo{ti2}) + info.Version = vers[source][db][tbls[1]] + DDLs, err = l.TrySync(info, tts) c.Assert(err, IsNil) c.Assert(DDLs, DeepEquals, DDLs2) c.Assert(l.versions, DeepEquals, vers) @@ -454,28 +495,36 @@ func (t *testLock) TestLockTrySyncNullNotNull(c *C) { for i := 0; i < 2; i++ { // two round // try sync for one table, from `NULL` to `NOT NULL`, no DDLs returned. vers[source][db][tbls[0]]++ - DDLs, err := l.TrySync(source, db, tbls[0], DDLs1, []*model.TableInfo{ti1}, tts, vers[source][db][tbls[0]]) + info := NewInfo(task, source, db, tbls[0], downSchema, downTable, DDLs1, nil, []*model.TableInfo{ti1}) + info.Version = vers[source][db][tbls[0]] + DDLs, err := l.TrySync(info, tts) c.Assert(err, IsNil) c.Assert(DDLs, DeepEquals, []string{}) c.Assert(l.versions, DeepEquals, vers) // try sync for another table, DDLs returned. vers[source][db][tbls[1]]++ - DDLs, err = l.TrySync(source, db, tbls[1], DDLs1, []*model.TableInfo{ti1}, tts, vers[source][db][tbls[1]]) + info = NewInfo(task, source, db, tbls[1], downSchema, downTable, DDLs1, nil, []*model.TableInfo{ti1}) + info.Version = vers[source][db][tbls[1]] + DDLs, err = l.TrySync(info, tts) c.Assert(err, IsNil) c.Assert(DDLs, DeepEquals, DDLs1) c.Assert(l.versions, DeepEquals, vers) // try sync for one table, from `NOT NULL` to `NULL`, DDLs returned. vers[source][db][tbls[0]]++ - DDLs, err = l.TrySync(source, db, tbls[0], DDLs2, []*model.TableInfo{ti2}, tts, vers[source][db][tbls[0]]) + info = NewInfo(task, source, db, tbls[0], downSchema, downTable, DDLs2, nil, []*model.TableInfo{ti2}) + info.Version = vers[source][db][tbls[0]] + DDLs, err = l.TrySync(info, tts) c.Assert(err, IsNil) c.Assert(DDLs, DeepEquals, DDLs2) c.Assert(l.versions, DeepEquals, vers) // try sync for another table, from `NOT NULL` to `NULL`, DDLs, returned. vers[source][db][tbls[1]]++ - DDLs, err = l.TrySync(source, db, tbls[1], DDLs2, []*model.TableInfo{ti2}, tts, vers[source][db][tbls[1]]) + info = NewInfo(task, source, db, tbls[1], downSchema, downTable, DDLs2, nil, []*model.TableInfo{ti2}) + info.Version = vers[source][db][tbls[1]] + DDLs, err = l.TrySync(info, tts) c.Assert(err, IsNil) c.Assert(DDLs, DeepEquals, DDLs2) c.Assert(l.versions, DeepEquals, vers) @@ -520,14 +569,18 @@ func (t *testLock) TestLockTrySyncIntBigint(c *C) { // try sync for one table, from `INT` to `BIGINT`, DDLs returned. vers[source][db][tbls[0]]++ - DDLs, err := l.TrySync(source, db, tbls[0], DDLs1, []*model.TableInfo{ti1}, tts, vers[source][db][tbls[0]]) + info := NewInfo(task, source, db, tbls[0], downSchema, downTable, DDLs1, nil, []*model.TableInfo{ti1}) + info.Version = vers[source][db][tbls[0]] + DDLs, err := l.TrySync(info, tts) c.Assert(err, IsNil) c.Assert(DDLs, DeepEquals, DDLs1) c.Assert(l.versions, DeepEquals, vers) // try sync for another table, DDLs returned. vers[source][db][tbls[1]]++ - DDLs, err = l.TrySync(source, db, tbls[1], DDLs1, []*model.TableInfo{ti1}, tts, vers[source][db][tbls[1]]) + info = NewInfo(task, source, db, tbls[1], downSchema, downTable, DDLs1, nil, []*model.TableInfo{ti1}) + info.Version = vers[source][db][tbls[1]] + DDLs, err = l.TrySync(info, tts) c.Assert(err, IsNil) c.Assert(DDLs, DeepEquals, DDLs1) c.Assert(l.versions, DeepEquals, vers) @@ -571,7 +624,9 @@ func (t *testLock) TestLockTrySyncNoDiff(c *C) { // try sync for one table. vers[source][db][tbls[0]]++ - DDLs, err := l.TrySync(source, db, tbls[0], DDLs1, []*model.TableInfo{ti1}, tts, vers[source][db][tbls[0]]) + info := NewInfo(task, source, db, tbls[0], downSchema, downTable, DDLs1, nil, []*model.TableInfo{ti1}) + info.Version = vers[source][db][tbls[0]] + DDLs, err := l.TrySync(info, tts) c.Assert(terror.ErrShardDDLOptimismTrySyncFail.Equal(err), IsTrue) c.Assert(DDLs, DeepEquals, []string{}) c.Assert(l.versions, DeepEquals, vers) @@ -615,7 +670,9 @@ func (t *testLock) TestLockTrySyncNewTable(c *C) { // TrySync for a new table as the caller. vers[source2][db2][tbl2]++ - DDLs, err := l.TrySync(source2, db2, tbl2, DDLs1, []*model.TableInfo{ti1}, tts, vers[source2][db2][tbl2]) + info := NewInfo(task, source2, db2, tbl2, downSchema, downTable, DDLs1, nil, []*model.TableInfo{ti1}) + info.Version = vers[source2][db2][tbl2] + DDLs, err := l.TrySync(info, tts) c.Assert(err, IsNil) c.Assert(DDLs, DeepEquals, DDLs1) c.Assert(l.versions, DeepEquals, vers) @@ -639,7 +696,9 @@ func (t *testLock) TestLockTrySyncNewTable(c *C) { vers[source2][db2][tbl1] = 0 vers[source1][db1][tbl1]++ - DDLs, err = l.TrySync(source1, db1, tbl1, DDLs1, []*model.TableInfo{ti1}, tts, vers[source1][db1][tbl1]) + info = NewInfo(task, source1, db1, tbl1, downSchema, downTable, DDLs1, nil, []*model.TableInfo{ti1}) + info.Version = vers[source1][db1][tbl1] + DDLs, err = l.TrySync(info, tts) c.Assert(err, IsNil) c.Assert(DDLs, DeepEquals, DDLs1) c.Assert(l.versions, DeepEquals, vers) @@ -707,7 +766,9 @@ func (t *testLock) TestLockTrySyncRevert(c *C) { // CASE: revert for single DDL. // TrySync for one table. vers[source][db][tbls[0]]++ - DDLs, err := l.TrySync(source, db, tbls[0], DDLs1, []*model.TableInfo{ti1}, tts, vers[source][db][tbls[0]]) + info := NewInfo(task, source, db, tbls[0], downSchema, downTable, DDLs1, nil, []*model.TableInfo{ti1}) + info.Version = vers[source][db][tbls[0]] + DDLs, err := l.TrySync(info, tts) c.Assert(err, IsNil) c.Assert(DDLs, DeepEquals, DDLs1) c.Assert(l.versions, DeepEquals, vers) @@ -723,7 +784,9 @@ func (t *testLock) TestLockTrySyncRevert(c *C) { // revert for the table, become synced again. vers[source][db][tbls[0]]++ - DDLs, err = l.TrySync(source, db, tbls[0], DDLs2, []*model.TableInfo{ti2}, tts, vers[source][db][tbls[0]]) + info = NewInfo(task, source, db, tbls[0], downSchema, downTable, DDLs2, nil, []*model.TableInfo{ti2}) + info.Version = vers[source][db][tbls[0]] + DDLs, err = l.TrySync(info, tts) c.Assert(err, IsNil) c.Assert(DDLs, DeepEquals, DDLs2) c.Assert(l.versions, DeepEquals, vers) @@ -733,7 +796,9 @@ func (t *testLock) TestLockTrySyncRevert(c *C) { // CASE: revert for multiple DDLs. // TrySync for one table. vers[source][db][tbls[0]]++ - DDLs, err = l.TrySync(source, db, tbls[0], DDLs3, []*model.TableInfo{ti4, ti3}, tts, vers[source][db][tbls[0]]) + info = NewInfo(task, source, db, tbls[0], downSchema, downTable, DDLs3, nil, []*model.TableInfo{ti4, ti3}) + info.Version = vers[source][db][tbls[0]] + DDLs, err = l.TrySync(info, tts) c.Assert(err, IsNil) c.Assert(DDLs, DeepEquals, DDLs3) c.Assert(l.versions, DeepEquals, vers) @@ -749,7 +814,9 @@ func (t *testLock) TestLockTrySyncRevert(c *C) { // revert part of the DDLs. vers[source][db][tbls[0]]++ - DDLs, err = l.TrySync(source, db, tbls[0], DDLs4, []*model.TableInfo{ti4}, tts, vers[source][db][tbls[0]]) + info = NewInfo(task, source, db, tbls[0], downSchema, downTable, DDLs4, nil, []*model.TableInfo{ti4}) + info.Version = vers[source][db][tbls[0]] + DDLs, err = l.TrySync(info, tts) c.Assert(err, IsNil) c.Assert(DDLs, DeepEquals, DDLs4) c.Assert(l.versions, DeepEquals, vers) @@ -764,7 +831,9 @@ func (t *testLock) TestLockTrySyncRevert(c *C) { // revert the reset part of the DDLs. vers[source][db][tbls[0]]++ - DDLs, err = l.TrySync(source, db, tbls[0], DDLs5, []*model.TableInfo{ti5}, tts, vers[source][db][tbls[0]]) + info = NewInfo(task, source, db, tbls[0], downSchema, downTable, DDLs5, nil, []*model.TableInfo{ti5}) + info.Version = vers[source][db][tbls[0]] + DDLs, err = l.TrySync(info, tts) c.Assert(err, IsNil) c.Assert(DDLs, DeepEquals, DDLs5) c.Assert(l.versions, DeepEquals, vers) @@ -774,7 +843,9 @@ func (t *testLock) TestLockTrySyncRevert(c *C) { // CASE: revert part of multiple DDLs. // TrySync for one table. vers[source][db][tbls[0]]++ - DDLs, err = l.TrySync(source, db, tbls[0], DDLs6, []*model.TableInfo{ti7, ti6}, tts, vers[source][db][tbls[0]]) + info = NewInfo(task, source, db, tbls[0], downSchema, downTable, DDLs6, nil, []*model.TableInfo{ti7, ti6}) + info.Version = vers[source][db][tbls[0]] + DDLs, err = l.TrySync(info, tts) c.Assert(err, IsNil) c.Assert(DDLs, DeepEquals, DDLs6) c.Assert(l.versions, DeepEquals, vers) @@ -789,7 +860,9 @@ func (t *testLock) TestLockTrySyncRevert(c *C) { // revert part of the DDLs. vers[source][db][tbls[0]]++ - DDLs, err = l.TrySync(source, db, tbls[0], DDLs7, []*model.TableInfo{ti7}, tts, vers[source][db][tbls[0]]) + info = NewInfo(task, source, db, tbls[0], downSchema, downTable, DDLs7, nil, []*model.TableInfo{ti7}) + info.Version = vers[source][db][tbls[0]] + DDLs, err = l.TrySync(info, tts) c.Assert(err, IsNil) c.Assert(DDLs, DeepEquals, DDLs7) c.Assert(l.versions, DeepEquals, vers) @@ -804,7 +877,9 @@ func (t *testLock) TestLockTrySyncRevert(c *C) { // TrySync for another table. vers[source][db][tbls[1]]++ - DDLs, err = l.TrySync(source, db, tbls[1], DDLs8, []*model.TableInfo{ti8}, tts, vers[source][db][tbls[1]]) + info = NewInfo(task, source, db, tbls[1], downSchema, downTable, DDLs8, nil, []*model.TableInfo{ti8}) + info.Version = vers[source][db][tbls[1]] + DDLs, err = l.TrySync(info, tts) c.Assert(err, IsNil) c.Assert(DDLs, DeepEquals, DDLs8) c.Assert(l.versions, DeepEquals, vers) @@ -853,7 +928,9 @@ func (t *testLock) TestLockTrySyncConflictNonIntrusive(c *C) { // TrySync for the first table, construct the joined schema. vers[source][db][tbls[0]]++ - DDLs, err := l.TrySync(source, db, tbls[0], DDLs1, []*model.TableInfo{ti1}, tts, vers[source][db][tbls[0]]) + info := NewInfo(task, source, db, tbls[0], downSchema, downTable, DDLs1, nil, []*model.TableInfo{ti1}) + info.Version = vers[source][db][tbls[0]] + DDLs, err := l.TrySync(info, tts) c.Assert(err, IsNil) c.Assert(DDLs, DeepEquals, DDLs1) c.Assert(l.versions, DeepEquals, vers) @@ -869,7 +946,9 @@ func (t *testLock) TestLockTrySyncConflictNonIntrusive(c *C) { // TrySync for the second table with another schema (add two columns, one of them will cause conflict). vers[source][db][tbls[1]]++ - DDLs, err = l.TrySync(source, db, tbls[1], DDLs2, []*model.TableInfo{ti2_1, ti2}, tts, vers[source][db][tbls[1]]) + info = NewInfo(task, source, db, tbls[1], downSchema, downTable, DDLs2, nil, []*model.TableInfo{ti2_1, ti2}) + info.Version = vers[source][db][tbls[1]] + DDLs, err = l.TrySync(info, tts) c.Assert(terror.ErrShardDDLOptimismTrySyncFail.Equal(err), IsTrue) c.Assert(DDLs, DeepEquals, []string{}) c.Assert(l.versions, DeepEquals, vers) @@ -882,7 +961,9 @@ func (t *testLock) TestLockTrySyncConflictNonIntrusive(c *C) { // TrySync for the first table to resolve the conflict. vers[source][db][tbls[0]]++ - DDLs, err = l.TrySync(source, db, tbls[0], DDLs3, []*model.TableInfo{ti3}, tts, vers[source][db][tbls[0]]) + info = NewInfo(task, source, db, tbls[0], downSchema, downTable, DDLs3, nil, []*model.TableInfo{ti3}) + info.Version = vers[source][db][tbls[0]] + DDLs, err = l.TrySync(info, tts) c.Assert(err, IsNil) c.Assert(DDLs, DeepEquals, DDLs3) c.Assert(l.versions, DeepEquals, vers) @@ -898,7 +979,9 @@ func (t *testLock) TestLockTrySyncConflictNonIntrusive(c *C) { // TrySync for the second table, succeed now vers[source][db][tbls[1]]++ - DDLs, err = l.TrySync(source, db, tbls[1], DDLs2, []*model.TableInfo{ti2_1, ti2}, tts, vers[source][db][tbls[1]]) + info = NewInfo(task, source, db, tbls[1], downSchema, downTable, DDLs2, nil, []*model.TableInfo{ti2_1, ti2}) + info.Version = vers[source][db][tbls[1]] + DDLs, err = l.TrySync(info, tts) c.Assert(err, IsNil) c.Assert(DDLs, DeepEquals, DDLs2) c.Assert(l.versions, DeepEquals, vers) @@ -910,7 +993,9 @@ func (t *testLock) TestLockTrySyncConflictNonIntrusive(c *C) { // TrySync for the first table. vers[source][db][tbls[0]]++ - DDLs, err = l.TrySync(source, db, tbls[0], DDLs4, []*model.TableInfo{ti4_1, ti4}, tts, vers[source][db][tbls[0]]) + info = NewInfo(task, source, db, tbls[0], downSchema, downTable, DDLs4, nil, []*model.TableInfo{ti4_1, ti4}) + info.Version = vers[source][db][tbls[0]] + DDLs, err = l.TrySync(info, tts) c.Assert(err, IsNil) c.Assert(DDLs, DeepEquals, DDLs4) c.Assert(l.versions, DeepEquals, vers) @@ -969,7 +1054,9 @@ func (t *testLock) TestLockTrySyncConflictIntrusive(c *C) { // CASE: conflict happen, revert all changes to resolve the conflict. // TrySync for the first table, construct the joined schema. vers[source][db][tbls[0]]++ - DDLs, err := l.TrySync(source, db, tbls[0], DDLs1, []*model.TableInfo{ti1}, tts, vers[source][db][tbls[0]]) + info := NewInfo(task, source, db, tbls[0], downSchema, downTable, DDLs1, nil, []*model.TableInfo{ti1}) + info.Version = vers[source][db][tbls[0]] + DDLs, err := l.TrySync(info, tts) c.Assert(err, IsNil) c.Assert(DDLs, DeepEquals, DDLs1) c.Assert(l.versions, DeepEquals, vers) @@ -985,7 +1072,9 @@ func (t *testLock) TestLockTrySyncConflictIntrusive(c *C) { // TrySync for the second table with another schema (add two columns, one of them will cause conflict). vers[source][db][tbls[1]]++ - DDLs, err = l.TrySync(source, db, tbls[1], DDLs2, []*model.TableInfo{ti3, ti2}, tts, vers[source][db][tbls[1]]) + info = NewInfo(task, source, db, tbls[1], downSchema, downTable, DDLs2, nil, []*model.TableInfo{ti3, ti2}) + info.Version = vers[source][db][tbls[1]] + DDLs, err = l.TrySync(info, tts) c.Assert(terror.ErrShardDDLOptimismTrySyncFail.Equal(err), IsTrue) c.Assert(DDLs, DeepEquals, []string{}) c.Assert(l.versions, DeepEquals, vers) @@ -998,7 +1087,9 @@ func (t *testLock) TestLockTrySyncConflictIntrusive(c *C) { // TrySync again. vers[source][db][tbls[1]]++ - DDLs, err = l.TrySync(source, db, tbls[1], DDLs2, []*model.TableInfo{ti3, ti2}, tts, vers[source][db][tbls[1]]) + info = NewInfo(task, source, db, tbls[1], downSchema, downTable, DDLs2, nil, []*model.TableInfo{ti3, ti2}) + info.Version = vers[source][db][tbls[1]] + DDLs, err = l.TrySync(info, tts) c.Assert(terror.ErrShardDDLOptimismTrySyncFail.Equal(err), IsTrue) c.Assert(DDLs, DeepEquals, []string{}) c.Assert(l.versions, DeepEquals, vers) @@ -1020,7 +1111,9 @@ func (t *testLock) TestLockTrySyncConflictIntrusive(c *C) { // TrySync for the second table to drop the conflict column, the conflict should be resolved. vers[source][db][tbls[1]]++ - DDLs, err = l.TrySync(source, db, tbls[1], DDLs4, []*model.TableInfo{ti4}, tts, vers[source][db][tbls[1]]) + info = NewInfo(task, source, db, tbls[1], downSchema, downTable, DDLs4, nil, []*model.TableInfo{ti4}) + info.Version = vers[source][db][tbls[1]] + DDLs, err = l.TrySync(info, tts) c.Assert(err, IsNil) c.Assert(DDLs, DeepEquals, []string{}) c.Assert(l.versions, DeepEquals, vers) @@ -1032,7 +1125,9 @@ func (t *testLock) TestLockTrySyncConflictIntrusive(c *C) { // TrySync for the second table as we did for the first table, the lock should be synced. vers[source][db][tbls[1]]++ - DDLs, err = l.TrySync(source, db, tbls[1], DDLs1, []*model.TableInfo{ti1}, tts, vers[source][db][tbls[1]]) + info = NewInfo(task, source, db, tbls[1], downSchema, downTable, DDLs1, nil, []*model.TableInfo{ti1}) + info.Version = vers[source][db][tbls[1]] + DDLs, err = l.TrySync(info, tts) c.Assert(err, IsNil) c.Assert(DDLs, DeepEquals, DDLs) c.Assert(l.versions, DeepEquals, vers) @@ -1045,7 +1140,9 @@ func (t *testLock) TestLockTrySyncConflictIntrusive(c *C) { // CASE: conflict happen, revert part of changes to resolve the conflict. // TrySync for the first table, construct the joined schema. vers[source][db][tbls[0]]++ - DDLs, err = l.TrySync(source, db, tbls[0], DDLs5, []*model.TableInfo{ti5}, tts, vers[source][db][tbls[0]]) + info = NewInfo(task, source, db, tbls[0], downSchema, downTable, DDLs5, nil, []*model.TableInfo{ti5}) + info.Version = vers[source][db][tbls[0]] + DDLs, err = l.TrySync(info, tts) c.Assert(err, IsNil) c.Assert(DDLs, DeepEquals, DDLs5) c.Assert(l.versions, DeepEquals, vers) @@ -1061,7 +1158,9 @@ func (t *testLock) TestLockTrySyncConflictIntrusive(c *C) { // TrySync for the second table with another schema (add two columns, one of them will cause conflict). vers[source][db][tbls[1]]++ - DDLs, err = l.TrySync(source, db, tbls[1], DDLs6, []*model.TableInfo{ti6_1, ti6}, tts, vers[source][db][tbls[1]]) + info = NewInfo(task, source, db, tbls[1], downSchema, downTable, DDLs6, nil, []*model.TableInfo{ti6_1, ti6}) + info.Version = vers[source][db][tbls[1]] + DDLs, err = l.TrySync(info, tts) c.Assert(terror.ErrShardDDLOptimismTrySyncFail.Equal(err), IsTrue) c.Assert(DDLs, DeepEquals, []string{}) cmp, err = l.tables[source][db][tbls[1]].Compare(l.Joined()) @@ -1074,7 +1173,9 @@ func (t *testLock) TestLockTrySyncConflictIntrusive(c *C) { // TrySync for the second table to drop the conflict column, the conflict should be resolved. // but both of tables are not synced now. vers[source][db][tbls[1]]++ - DDLs, err = l.TrySync(source, db, tbls[1], DDLs7, []*model.TableInfo{ti7}, tts, vers[source][db][tbls[1]]) + info = NewInfo(task, source, db, tbls[1], downSchema, downTable, DDLs7, nil, []*model.TableInfo{ti7}) + info.Version = vers[source][db][tbls[1]] + DDLs, err = l.TrySync(info, tts) c.Assert(err, IsNil) c.Assert(DDLs, DeepEquals, DDLs7) // special case: these DDLs should not be replicated to the downstream. c.Assert(l.versions, DeepEquals, vers) @@ -1090,7 +1191,9 @@ func (t *testLock) TestLockTrySyncConflictIntrusive(c *C) { // TrySync for the first table to become synced. vers[source][db][tbls[0]]++ - DDLs, err = l.TrySync(source, db, tbls[0], DDLs8_1, []*model.TableInfo{ti8}, tts, vers[source][db][tbls[0]]) + info = NewInfo(task, source, db, tbls[0], downSchema, downTable, DDLs8_1, nil, []*model.TableInfo{ti8}) + info.Version = vers[source][db][tbls[0]] + DDLs, err = l.TrySync(info, tts) c.Assert(err, IsNil) c.Assert(DDLs, DeepEquals, DDLs8_1) ready = l.Ready() @@ -1098,7 +1201,9 @@ func (t *testLock) TestLockTrySyncConflictIntrusive(c *C) { // TrySync for the second table to become synced. vers[source][db][tbls[1]]++ - DDLs, err = l.TrySync(source, db, tbls[1], DDLs8_2, []*model.TableInfo{ti8}, tts, vers[source][db][tbls[1]]) + info = NewInfo(task, source, db, tbls[1], downSchema, downTable, DDLs8_2, nil, []*model.TableInfo{ti8}) + info.Version = vers[source][db][tbls[1]] + DDLs, err = l.TrySync(info, tts) c.Assert(err, IsNil) c.Assert(DDLs, DeepEquals, DDLs8_2) ready = l.Ready() @@ -1163,12 +1268,16 @@ func (t *testLock) TestLockTrySyncMultipleChangeDDL(c *C) { // inconsistent ddls and table infos vers[sources[0]][dbs[0]][tbls[0]]++ - DDLs, err := l.TrySync(sources[0], dbs[0], tbls[0], DDLs1[:1], []*model.TableInfo{ti1_1, ti1}, tts, vers[sources[0]][dbs[0]][tbls[0]]) + info := NewInfo(task, sources[0], dbs[0], tbls[0], downSchema, downTable, DDLs1[:1], nil, []*model.TableInfo{ti1_1, ti1}) + info.Version = vers[sources[0]][dbs[0]][tbls[0]] + DDLs, err := l.TrySync(info, tts) c.Assert(DDLs, DeepEquals, DDLs1[:1]) c.Assert(terror.ErrMasterInconsistentOptimisticDDLsAndInfo.Equal(err), IsTrue) vers[sources[0]][dbs[0]][tbls[0]]++ - DDLs, err = l.TrySync(sources[0], dbs[0], tbls[0], DDLs1, []*model.TableInfo{ti1}, tts, vers[sources[0]][dbs[0]][tbls[0]]) + info = NewInfo(task, sources[0], dbs[0], tbls[0], downSchema, downTable, DDLs1, nil, []*model.TableInfo{ti1}) + info.Version = vers[sources[0]][dbs[0]][tbls[0]] + DDLs, err = l.TrySync(info, tts) c.Assert(DDLs, DeepEquals, DDLs1) c.Assert(terror.ErrMasterInconsistentOptimisticDDLsAndInfo.Equal(err), IsTrue) @@ -1191,7 +1300,9 @@ func (t *testLock) TestLockTrySyncMultipleChangeDDL(c *C) { for _, db := range dbs { for _, tbl := range tbls { vers[source][db][tbl]++ - DDLs, err = l.TrySync(source, db, tbl, DDLs1, []*model.TableInfo{ti1_1, ti1}, tts, vers[source][db][tbl]) + info = NewInfo(task, source, db, tbl, downSchema, downTable, DDLs1, nil, []*model.TableInfo{ti1_1, ti1}) + info.Version = vers[source][db][tbl] + DDLs, err = l.TrySync(info, tts) c.Assert(err, IsNil) c.Assert(DDLs, DeepEquals, resultDDLs1[source][db][tbl]) c.Assert(l.versions, DeepEquals, vers) @@ -1210,7 +1321,9 @@ func (t *testLock) TestLockTrySyncMultipleChangeDDL(c *C) { // CASE: TrySync again after synced is idempotent. // both ddl will sync again vers[sources[0]][dbs[0]][tbls[0]]++ - DDLs, err = l.TrySync(sources[0], dbs[0], tbls[0], DDLs1, []*model.TableInfo{ti1_1, ti1}, tts, vers[sources[0]][dbs[0]][tbls[0]]) + info = NewInfo(task, sources[0], dbs[0], tbls[0], downSchema, downTable, DDLs1, nil, []*model.TableInfo{ti1_1, ti1}) + info.Version = vers[sources[0]][dbs[0]][tbls[0]] + DDLs, err = l.TrySync(info, tts) c.Assert(err, IsNil) c.Assert(DDLs, DeepEquals, DDLs1) c.Assert(l.versions, DeepEquals, vers) @@ -1233,7 +1346,9 @@ func (t *testLock) TestLockTrySyncMultipleChangeDDL(c *C) { for _, db := range dbs { for _, tbl := range tbls { vers[source][db][tbl]++ - DDLs, err = l.TrySync(source, db, tbl, DDLs2, []*model.TableInfo{ti2_1, ti2}, tts, vers[source][db][tbl]) + info = NewInfo(task, source, db, tbl, downSchema, downTable, DDLs2, nil, []*model.TableInfo{ti2_1, ti2}) + info.Version = vers[source][db][tbl] + DDLs, err = l.TrySync(info, tts) c.Assert(err, IsNil) c.Assert(DDLs, DeepEquals, resultDDLs2[source][db][tbl]) c.Assert(l.versions, DeepEquals, vers) @@ -1252,7 +1367,9 @@ func (t *testLock) TestLockTrySyncMultipleChangeDDL(c *C) { // CASE: TrySync again after synced is idempotent. // only the second ddl(ADD COLUMN) will sync, the first one(DROP COLUMN) will not sync since oldJoined==newJoined vers[sources[0]][dbs[0]][tbls[0]]++ - DDLs, err = l.TrySync(sources[0], dbs[0], tbls[0], DDLs2, []*model.TableInfo{ti2_1, ti2}, tts, vers[sources[0]][dbs[0]][tbls[0]]) + info = NewInfo(task, sources[0], dbs[0], tbls[0], downSchema, downTable, DDLs2, nil, []*model.TableInfo{ti2_1, ti2}) + info.Version = vers[sources[0]][dbs[0]][tbls[0]] + DDLs, err = l.TrySync(info, tts) c.Assert(err, IsNil) c.Assert(DDLs, DeepEquals, DDLs2[1:]) c.Assert(l.versions, DeepEquals, vers) @@ -1297,7 +1414,9 @@ func (t *testLock) TestTryRemoveTable(c *C) { // CASE: remove a table as normal. // TrySync for the first table. vers[source][db][tbl1]++ - DDLs, err := l.TrySync(source, db, tbl1, DDLs1, []*model.TableInfo{ti1}, tts, vers[source][db][tbl1]) + info := NewInfo(task, source, db, tbl1, downSchema, downTable, DDLs1, nil, []*model.TableInfo{ti1}) + info.Version = vers[source][db][tbl1] + DDLs, err := l.TrySync(info, tts) c.Assert(err, IsNil) c.Assert(DDLs, DeepEquals, DDLs1) c.Assert(l.versions, DeepEquals, vers) @@ -1321,7 +1440,9 @@ func (t *testLock) TestTryRemoveTable(c *C) { // CASE: remove a table will not rebuild joined schema now. // TrySync to add the second back. vers[source][db][tbl2] = 1 - DDLs, err = l.TrySync(source, db, tbl2, DDLs2, []*model.TableInfo{ti2}, tts, vers[source][db][tbl1]) + info = NewInfo(task, source, db, tbl2, downSchema, downTable, DDLs2, nil, []*model.TableInfo{ti2}) + info.Version = vers[source][db][tbl2] + DDLs, err = l.TrySync(info, tts) c.Assert(err, IsNil) c.Assert(DDLs, DeepEquals, DDLs2) c.Assert(l.versions, DeepEquals, vers) @@ -1386,7 +1507,9 @@ func (t *testLock) TestLockTryMarkDone(c *C) { // TrySync for the first table, no table has done the DDLs operation. vers[source][db][tbls[0]]++ - DDLs, err := l.TrySync(source, db, tbls[0], DDLs1, []*model.TableInfo{ti1}, tts, vers[source][db][tbls[0]]) + info := NewInfo(task, source, db, tbls[0], downSchema, downTable, DDLs1, nil, []*model.TableInfo{ti1}) + info.Version = vers[source][db][tbls[0]] + DDLs, err := l.TrySync(info, tts) c.Assert(err, IsNil) c.Assert(DDLs, DeepEquals, DDLs1) c.Assert(l.versions, DeepEquals, vers) @@ -1401,7 +1524,9 @@ func (t *testLock) TestLockTryMarkDone(c *C) { // TrySync for the second table, the joined schema become larger. vers[source][db][tbls[1]]++ - DDLs, err = l.TrySync(source, db, tbls[1], DDLs2, []*model.TableInfo{ti1, ti2}, tts, vers[source][db][tbls[1]]) + info = NewInfo(task, source, db, tbls[1], downSchema, downTable, DDLs2, nil, []*model.TableInfo{ti1, ti2}) + info.Version = vers[source][db][tbls[1]] + DDLs, err = l.TrySync(info, tts) c.Assert(err, IsNil) c.Assert(DDLs, DeepEquals, DDLs2) c.Assert(l.versions, DeepEquals, vers) @@ -1420,7 +1545,9 @@ func (t *testLock) TestLockTryMarkDone(c *C) { // TrySync for the first table, all tables become synced. vers[source][db][tbls[0]]++ - DDLs, err = l.TrySync(source, db, tbls[0], DDLs3, []*model.TableInfo{ti3}, tts, vers[source][db][tbls[0]]) + info = NewInfo(task, source, db, tbls[0], downSchema, downTable, DDLs3, nil, []*model.TableInfo{ti3}) + info.Version = vers[source][db][tbls[0]] + DDLs, err = l.TrySync(info, tts) c.Assert(err, IsNil) c.Assert(DDLs, DeepEquals, DDLs3) c.Assert(l.versions, DeepEquals, vers) @@ -1533,7 +1660,9 @@ func (t *testLock) trySyncForAllTablesLarger(c *C, l *Lock, for source, schemaTables := range l.Ready() { for schema, tables := range schemaTables { for table := range tables { - DDLs2, err := l.TrySync(source, schema, table, DDLs, tis, tts, vers[source][schema][table]) + info := NewInfo(l.Task, source, schema, table, l.DownSchema, l.DownTable, DDLs, nil, tis) + info.Version = vers[source][schema][table] + DDLs2, err := l.TrySync(info, tts) c.Assert(err, IsNil) c.Assert(DDLs2, DeepEquals, resultDDLs[source][schema][table]) } diff --git a/tests/shardddl1/run.sh b/tests/shardddl1/run.sh index 7bd5c1b307..ad052bb65e 100644 --- a/tests/shardddl1/run.sh +++ b/tests/shardddl1/run.sh @@ -552,9 +552,6 @@ function DM_RENAME_COLUMN_OPTIMISTIC_CASE() { run_dm_ctl $WORK_DIR "127.0.0.1:$MASTER_PORT" \ "query-status test" \ "\"result\": true" 3 - run_dm_ctl $WORK_DIR "127.0.0.1:$MASTER_PORT" \ - "show-ddl-locks" \ - "no DDL lock exists" 1 # now, it works as normal run_sql_source1 "alter table ${shardddl1}.${tb1} add column d int;" @@ -573,14 +570,12 @@ function DM_RENAME_COLUMN_OPTIMISTIC_CASE() { run_sql_source2 "insert into ${shardddl1}.${tb2} values(21,'uuu',21);" check_sync_diff $WORK_DIR $cur/conf/diff_config.toml - run_dm_ctl $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" \ "query-status test" \ "\"result\": true" 3 - run_dm_ctl $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" \ "show-ddl-locks" \ "no DDL lock exists" 1 - - read v1 } # workaround of rename column in optimistic mode currently until we support it @@ -596,16 +591,16 @@ function DM_RENAME_COLUMN_OPTIMISTIC() { function run() { init_cluster init_database -# start=1 -# end=35 -# except=(024 025 029) -# for i in $(seq -f "%03g" ${start} ${end}); do -# if [[ ${except[@]} =~ $i ]]; then -# continue -# fi -# DM_${i} -# sleep 1 -# done + start=1 + end=35 + except=(024 025 029) + for i in $(seq -f "%03g" ${start} ${end}); do + if [[ ${except[@]} =~ $i ]]; then + continue + fi + DM_${i} + sleep 1 + done DM_RENAME_COLUMN_OPTIMISTIC } From c331f8349b0c7ccaf65bd849c022b8e1beb1e9a2 Mon Sep 17 00:00:00 2001 From: gmhdbjd Date: Fri, 12 Mar 2021 13:47:28 +0800 Subject: [PATCH 05/17] change func args --- dm/ctl/master/operate_schema.go | 21 +- dm/master/server.go | 2 + dm/pb/dmmaster.pb.go | 321 +++++++++++++++++----------- dm/pb/dmworker.pb.go | 327 ++++++++++++++++++----------- dm/proto/dmmaster.proto | 2 + dm/proto/dmworker.proto | 2 + pkg/shardddl/optimism/lock_test.go | 32 ++- syncer/schema.go | 22 +- tests/shardddl1/run.sh | 26 +-- 9 files changed, 484 insertions(+), 271 deletions(-) diff --git a/dm/ctl/master/operate_schema.go b/dm/ctl/master/operate_schema.go index 49c4fd30f6..ba99e84145 100644 --- a/dm/ctl/master/operate_schema.go +++ b/dm/ctl/master/operate_schema.go @@ -27,12 +27,14 @@ import ( // NewOperateSchemaCmd creates a OperateSchema command. func NewOperateSchemaCmd() *cobra.Command { cmd := &cobra.Command{ - Use: "operate-schema <-s source ...> <-d database> <-t table> [schema-file]", + Use: "operate-schema <-s source ...> <-d database> <-t table> [schema-file] [--flush] [--sync]", Short: "`get`/`set`/`remove` the schema for an upstream table.", RunE: operateSchemaCmd, } cmd.Flags().StringP("database", "d", "", "database name of the table") cmd.Flags().StringP("table", "t", "", "table name") + cmd.Flags().Bool("flush", false, "flush the table info and checkpoint imediately") + cmd.Flags().Bool("sync", false, "sync the table info to master to resolve shard ddl lock, only for optimistic mode now.") return cmd } @@ -109,6 +111,21 @@ func operateSchemaCmd(cmd *cobra.Command, _ []string) (err error) { return } + flush, err := cmd.Flags().GetBool("flush") + if err != nil { + return + } + if flush && op != pb.SchemaOp_SetSchema { + err = errors.New("--flush flag is only used to set schema") + } + sync, err := cmd.Flags().GetBool("sync") + if err != nil { + return + } + if sync && op != pb.SchemaOp_SetSchema { + err = errors.New("--sync flag is only used to set schema") + } + ctx, cancel := context.WithCancel(context.Background()) defer cancel() @@ -123,6 +140,8 @@ func operateSchemaCmd(cmd *cobra.Command, _ []string) (err error) { Database: database, Table: table, Schema: string(schemaContent), + Flush: flush, + Sync: sync, }, &resp, ) diff --git a/dm/master/server.go b/dm/master/server.go index 4645349842..984107e8ee 100644 --- a/dm/master/server.go +++ b/dm/master/server.go @@ -1737,6 +1737,8 @@ func (s *Server) OperateSchema(ctx context.Context, req *pb.OperateSchemaRequest Database: req.Database, Table: req.Table, Schema: req.Schema, + Flush: req.Flush, + Sync: req.Sync, }, } diff --git a/dm/pb/dmmaster.pb.go b/dm/pb/dmmaster.pb.go index 91233745d7..96310a2138 100644 --- a/dm/pb/dmmaster.pb.go +++ b/dm/pb/dmmaster.pb.go @@ -2259,6 +2259,8 @@ type OperateSchemaRequest struct { Database string `protobuf:"bytes,4,opt,name=database,proto3" json:"database,omitempty"` Table string `protobuf:"bytes,5,opt,name=table,proto3" json:"table,omitempty"` Schema string `protobuf:"bytes,6,opt,name=schema,proto3" json:"schema,omitempty"` + Flush bool `protobuf:"varint,7,opt,name=flush,proto3" json:"flush,omitempty"` + Sync bool `protobuf:"varint,8,opt,name=sync,proto3" json:"sync,omitempty"` } func (m *OperateSchemaRequest) Reset() { *m = OperateSchemaRequest{} } @@ -2336,6 +2338,20 @@ func (m *OperateSchemaRequest) GetSchema() string { return "" } +func (m *OperateSchemaRequest) GetFlush() bool { + if m != nil { + return m.Flush + } + return false +} + +func (m *OperateSchemaRequest) GetSync() bool { + if m != nil { + return m.Sync + } + return false +} + type OperateSchemaResponse struct { Result bool `protobuf:"varint,1,opt,name=result,proto3" json:"result,omitempty"` Msg string `protobuf:"bytes,2,opt,name=msg,proto3" json:"msg,omitempty"` @@ -2883,125 +2899,126 @@ func init() { func init() { proto.RegisterFile("dmmaster.proto", fileDescriptor_f9bef11f2a341f03) } var fileDescriptor_f9bef11f2a341f03 = []byte{ - // 1873 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xbc, 0x59, 0x4f, 0x6f, 0xdb, 0xc8, - 0x15, 0x17, 0x25, 0xc5, 0x96, 0x9f, 0x6c, 0xaf, 0x3c, 0xb6, 0x65, 0x86, 0xf1, 0x2a, 0xde, 0xe9, - 0x6e, 0x60, 0x18, 0x45, 0x8c, 0xb8, 0x3d, 0x2d, 0xb0, 0x05, 0x36, 0x52, 0x36, 0x6b, 0x54, 0xa9, - 0xb7, 0x74, 0x82, 0x76, 0x51, 0xa0, 0x58, 0x8a, 0x1a, 0xc9, 0x84, 0x29, 0x92, 0x21, 0x29, 0xbb, - 0x46, 0xb0, 0x97, 0x7e, 0x80, 0xfe, 0x41, 0x0f, 0x7b, 0xec, 0xa1, 0xd7, 0x9e, 0xfa, 0x29, 0x7a, - 0x5c, 0xa0, 0x97, 0x1e, 0x8b, 0xa4, 0x1f, 0xa4, 0x98, 0x37, 0x43, 0x72, 0xf8, 0x47, 0x6e, 0x15, - 0xa0, 0xbe, 0xcd, 0x9b, 0x37, 0x7a, 0xef, 0xf7, 0xfe, 0xcc, 0x9b, 0xf7, 0x28, 0xd8, 0x1c, 0xcf, - 0x66, 0x56, 0x14, 0xb3, 0xf0, 0x71, 0x10, 0xfa, 0xb1, 0x4f, 0xea, 0xc1, 0xc8, 0xd8, 0x1c, 0xcf, - 0xae, 0xfd, 0xf0, 0x32, 0xd9, 0x33, 0xf6, 0xa7, 0xbe, 0x3f, 0x75, 0xd9, 0xb1, 0x15, 0x38, 0xc7, - 0x96, 0xe7, 0xf9, 0xb1, 0x15, 0x3b, 0xbe, 0x17, 0x09, 0x2e, 0xfd, 0x06, 0x3a, 0xe7, 0xb1, 0x15, - 0xc6, 0x2f, 0xad, 0xe8, 0xd2, 0x64, 0xaf, 0xe7, 0x2c, 0x8a, 0x09, 0x81, 0x66, 0x6c, 0x45, 0x97, - 0xba, 0x76, 0xa0, 0x1d, 0xae, 0x99, 0xb8, 0x26, 0x3a, 0xac, 0x46, 0xfe, 0x3c, 0xb4, 0x59, 0xa4, - 0xd7, 0x0f, 0x1a, 0x87, 0x6b, 0x66, 0x42, 0x92, 0x1e, 0x40, 0xc8, 0x66, 0xfe, 0x15, 0x7b, 0xc1, - 0x62, 0x4b, 0x6f, 0x1c, 0x68, 0x87, 0x2d, 0x53, 0xd9, 0xa1, 0xaf, 0x61, 0x4b, 0xd1, 0x10, 0x05, - 0xbe, 0x17, 0x31, 0xd2, 0x85, 0x95, 0x90, 0x45, 0x73, 0x37, 0x46, 0x25, 0x2d, 0x53, 0x52, 0xa4, - 0x03, 0x8d, 0x59, 0x34, 0xd5, 0xeb, 0xa8, 0x99, 0x2f, 0xc9, 0x49, 0xa6, 0xb8, 0x71, 0xd0, 0x38, - 0x6c, 0x9f, 0xe8, 0x8f, 0x83, 0xd1, 0xe3, 0xbe, 0x3f, 0x9b, 0xf9, 0xde, 0x2f, 0xd0, 0xce, 0x44, - 0x68, 0x0a, 0x89, 0xfe, 0x1a, 0xc8, 0x59, 0xc0, 0x42, 0x2b, 0x66, 0xaa, 0x59, 0x06, 0xd4, 0xfd, - 0x00, 0xf5, 0x6d, 0x9e, 0x00, 0x17, 0xc2, 0x99, 0x67, 0x81, 0x59, 0xf7, 0x03, 0x6e, 0xb2, 0x67, - 0xcd, 0x98, 0x54, 0x8c, 0x6b, 0xd5, 0xe4, 0x46, 0xce, 0x64, 0xfa, 0x7b, 0x0d, 0xb6, 0x73, 0x0a, - 0xa4, 0x55, 0xb7, 0x69, 0xc8, 0x2c, 0xae, 0x57, 0x59, 0xdc, 0xa8, 0xb4, 0xb8, 0xf9, 0xbf, 0x5a, - 0xfc, 0x39, 0x6c, 0xbd, 0x0a, 0xc6, 0x05, 0x83, 0x97, 0x8a, 0x23, 0x0d, 0x81, 0xa8, 0x22, 0xee, - 0x24, 0x50, 0x5f, 0x40, 0xf7, 0xe7, 0x73, 0x16, 0xde, 0x9c, 0xc7, 0x56, 0x3c, 0x8f, 0x86, 0x4e, - 0x14, 0x2b, 0xd8, 0x31, 0x20, 0x5a, 0x75, 0x40, 0x0a, 0xd8, 0xaf, 0x60, 0xaf, 0x24, 0x67, 0x69, - 0x03, 0x9e, 0x14, 0x0d, 0xd8, 0xe3, 0x06, 0x28, 0x72, 0xcb, 0xf8, 0xfb, 0xb0, 0x7d, 0x7e, 0xe1, - 0x5f, 0x0f, 0x06, 0xc3, 0xa1, 0x6f, 0x5f, 0x46, 0xef, 0xe7, 0xf8, 0x3f, 0x6b, 0xb0, 0x2a, 0x25, - 0x90, 0x4d, 0xa8, 0x9f, 0x0e, 0xe4, 0xef, 0xea, 0xa7, 0x83, 0x54, 0x52, 0x5d, 0x91, 0x44, 0xa0, - 0x39, 0xf3, 0xc7, 0x4c, 0xa6, 0x0c, 0xae, 0xc9, 0x0e, 0xdc, 0xf3, 0xaf, 0x3d, 0x16, 0xea, 0x4d, - 0xdc, 0x14, 0x04, 0x3f, 0x39, 0x18, 0x0c, 0x23, 0xfd, 0x1e, 0x2a, 0xc4, 0x35, 0xf7, 0x47, 0x74, - 0xe3, 0xd9, 0x6c, 0xac, 0xaf, 0xe0, 0xae, 0xa4, 0x88, 0x01, 0xad, 0xb9, 0x27, 0x39, 0xab, 0xc8, - 0x49, 0x69, 0x6a, 0xc3, 0x4e, 0xde, 0xcc, 0xa5, 0x7d, 0xfb, 0x11, 0xdc, 0x73, 0xf9, 0x4f, 0xa5, - 0x67, 0xdb, 0xdc, 0xb3, 0x52, 0x9c, 0x29, 0x38, 0xd4, 0x85, 0x9d, 0x57, 0x1e, 0x5f, 0x26, 0xfb, - 0xd2, 0x99, 0x45, 0x97, 0x50, 0x58, 0x0f, 0x59, 0xe0, 0x5a, 0x36, 0x3b, 0x43, 0x8b, 0x85, 0x96, - 0xdc, 0x1e, 0x39, 0x80, 0xf6, 0xc4, 0x0f, 0x6d, 0x66, 0x62, 0x19, 0x92, 0x45, 0x49, 0xdd, 0xa2, - 0x9f, 0xc3, 0x6e, 0x41, 0xdb, 0xb2, 0x36, 0x51, 0x13, 0xee, 0xcb, 0x22, 0x90, 0xa4, 0xb7, 0x6b, - 0xdd, 0x24, 0xa8, 0x1f, 0x28, 0xa5, 0x00, 0xad, 0x45, 0xae, 0xac, 0x05, 0x8b, 0x73, 0xe1, 0x3b, - 0x0d, 0x8c, 0x2a, 0xa1, 0x12, 0xdc, 0xad, 0x52, 0xff, 0xbf, 0x15, 0xe6, 0x3b, 0x0d, 0xf6, 0xbe, - 0x9a, 0x87, 0xd3, 0x2a, 0x63, 0x15, 0x7b, 0xb4, 0xfc, 0xe3, 0x60, 0x40, 0xcb, 0xf1, 0x2c, 0x3b, - 0x76, 0xae, 0x98, 0x44, 0x95, 0xd2, 0x98, 0xdb, 0xce, 0x4c, 0x44, 0xa7, 0x61, 0xe2, 0x9a, 0x9f, - 0x9f, 0x38, 0x2e, 0xc3, 0xab, 0x2f, 0x52, 0x39, 0xa5, 0x31, 0x73, 0xe7, 0xa3, 0x81, 0x13, 0xea, - 0xf7, 0x90, 0x23, 0x29, 0xfa, 0x1b, 0xd0, 0xcb, 0xc0, 0xee, 0xa4, 0x7c, 0x3d, 0x82, 0x4e, 0xff, - 0x82, 0xd9, 0x97, 0xff, 0xa5, 0xe8, 0xd2, 0xcf, 0x60, 0x4b, 0x39, 0xb7, 0x74, 0xa2, 0x5d, 0xc0, - 0x8e, 0xcc, 0x89, 0x73, 0x54, 0x9c, 0xa8, 0xda, 0x57, 0xb2, 0x61, 0x9d, 0xa3, 0x15, 0xec, 0x2c, - 0x1d, 0x6c, 0xdf, 0x9b, 0x38, 0x53, 0x99, 0x63, 0x92, 0xe2, 0x2e, 0x16, 0xf8, 0x4f, 0x07, 0xf2, - 0x5d, 0x4b, 0x69, 0x3a, 0x87, 0xdd, 0x82, 0xa6, 0x3b, 0xf1, 0xe3, 0x33, 0xd8, 0x35, 0xd9, 0xd4, - 0xe1, 0x8d, 0x4c, 0x72, 0xe4, 0xd6, 0x57, 0xc0, 0x1a, 0x8f, 0x43, 0x16, 0x45, 0x52, 0x6d, 0x42, - 0xd2, 0xa7, 0xd0, 0x2d, 0x8a, 0x59, 0xda, 0xd7, 0x3f, 0x81, 0x9d, 0xb3, 0xc9, 0xc4, 0x75, 0x3c, - 0xf6, 0x82, 0xcd, 0x46, 0x39, 0x24, 0xf1, 0x4d, 0x90, 0x22, 0xe1, 0xeb, 0xaa, 0xa6, 0x81, 0xd7, - 0x95, 0xc2, 0xef, 0x97, 0x86, 0xf0, 0xe3, 0x34, 0xdc, 0x43, 0x66, 0x8d, 0x33, 0x08, 0xa5, 0x70, - 0x0b, 0xb6, 0x08, 0x37, 0x2a, 0xce, 0xff, 0x6a, 0x69, 0xc5, 0xbf, 0xd3, 0x00, 0x5e, 0x60, 0x3b, - 0x79, 0xea, 0x4d, 0xfc, 0x4a, 0xe7, 0x1b, 0xd0, 0x9a, 0xa1, 0x5d, 0xa7, 0x03, 0xfc, 0x65, 0xd3, - 0x4c, 0x69, 0xfe, 0x06, 0x59, 0xae, 0x93, 0x96, 0x5b, 0x41, 0xf0, 0x5f, 0x04, 0x8c, 0x85, 0xaf, - 0xcc, 0xa1, 0x28, 0x36, 0x6b, 0x66, 0x4a, 0xf3, 0xd6, 0xd1, 0x76, 0x1d, 0xe6, 0xc5, 0xc8, 0x15, - 0xaf, 0x94, 0xb2, 0x43, 0x47, 0x00, 0x22, 0x90, 0x0b, 0xf1, 0x10, 0x68, 0xf2, 0xe8, 0x27, 0x21, - 0xe0, 0x6b, 0x8e, 0x23, 0x8a, 0xad, 0x69, 0xf2, 0x40, 0x0a, 0x02, 0xab, 0x07, 0xa6, 0x9b, 0xac, - 0x2b, 0x92, 0xa2, 0x43, 0xe8, 0xf0, 0x7e, 0x41, 0x38, 0x4d, 0xc4, 0x2c, 0x71, 0x8d, 0x96, 0x65, - 0x75, 0x55, 0x7f, 0x98, 0xe8, 0x6e, 0x64, 0xba, 0xe9, 0xcf, 0x84, 0x34, 0xe1, 0xc5, 0x85, 0xd2, - 0x0e, 0x61, 0x55, 0xb4, 0xed, 0xa2, 0xfe, 0xb7, 0x4f, 0x36, 0x79, 0x38, 0x33, 0xd7, 0x9b, 0x09, - 0x3b, 0x91, 0x27, 0xbc, 0x70, 0x9b, 0x3c, 0xd1, 0xf2, 0xe7, 0xe4, 0x65, 0xae, 0x33, 0x13, 0x36, - 0xfd, 0x8b, 0x06, 0xab, 0x42, 0x4c, 0x44, 0x1e, 0xc3, 0x8a, 0x8b, 0x56, 0xa3, 0xa8, 0xf6, 0xc9, - 0x0e, 0xe6, 0x54, 0xc1, 0x17, 0x5f, 0xd6, 0x4c, 0x79, 0x8a, 0x9f, 0x17, 0xb0, 0xd0, 0x0b, 0xca, - 0x79, 0xd5, 0x5a, 0x7e, 0x5e, 0x9c, 0xe2, 0xe7, 0x85, 0x5a, 0xf4, 0x90, 0x72, 0x5e, 0xb5, 0x86, - 0x9f, 0x17, 0xa7, 0x9e, 0xb6, 0x60, 0x45, 0xe4, 0x12, 0x1f, 0x19, 0x50, 0x6e, 0xee, 0x06, 0x76, - 0x73, 0x70, 0x5b, 0x29, 0xac, 0x6e, 0x0e, 0x56, 0x2b, 0x55, 0xdf, 0xcd, 0xa9, 0x6f, 0x25, 0x6a, - 0x78, 0x7a, 0xf0, 0xf0, 0x25, 0xd9, 0x28, 0x08, 0xca, 0x80, 0xa8, 0x2a, 0x97, 0x2e, 0x7b, 0x9f, - 0xc0, 0xaa, 0x00, 0x9f, 0x6b, 0x71, 0xa4, 0xab, 0xcd, 0x84, 0x47, 0xff, 0xaa, 0x65, 0xb5, 0xdc, - 0xbe, 0x60, 0x33, 0x6b, 0x71, 0x2d, 0x47, 0x76, 0x36, 0x9e, 0x94, 0xda, 0xc0, 0x85, 0xe3, 0x09, - 0xbf, 0x72, 0x63, 0x2b, 0xb6, 0x46, 0x56, 0x94, 0x3e, 0xa2, 0x09, 0xcd, 0xad, 0x8f, 0xad, 0x91, - 0xcb, 0xe4, 0x1b, 0x2a, 0x08, 0xbc, 0x1c, 0xa8, 0x4f, 0x5f, 0x91, 0x97, 0x03, 0x29, 0xf5, 0x3d, - 0x90, 0x68, 0xef, 0xe4, 0x3d, 0x38, 0x82, 0x9d, 0xe7, 0x2c, 0x3e, 0x9f, 0x8f, 0xf8, 0x83, 0xd9, - 0x9f, 0x4c, 0x6f, 0x79, 0x0e, 0xe8, 0x2b, 0xd8, 0x2d, 0x9c, 0x5d, 0x1a, 0x22, 0x81, 0xa6, 0x3d, - 0x99, 0x26, 0x6e, 0xc4, 0x35, 0x1d, 0xc0, 0xc6, 0x73, 0x16, 0x2b, 0xba, 0x1f, 0x2a, 0x0f, 0x80, - 0x6c, 0xbe, 0xfa, 0x93, 0xe9, 0xcb, 0x9b, 0x80, 0xdd, 0xf2, 0x1a, 0x0c, 0x61, 0x33, 0x91, 0xb2, - 0x34, 0xaa, 0x0e, 0x34, 0xec, 0x49, 0xda, 0xb6, 0xd9, 0x93, 0x29, 0xdd, 0x85, 0xed, 0xe7, 0x4c, - 0xde, 0xb6, 0x0c, 0x19, 0x3d, 0x44, 0x6f, 0x29, 0xdb, 0x52, 0x95, 0x14, 0xa0, 0x65, 0x02, 0xfe, - 0xa8, 0x01, 0xf9, 0xd2, 0xf2, 0xc6, 0x2e, 0x7b, 0x16, 0x86, 0x7e, 0xb8, 0xb0, 0x57, 0x45, 0xee, - 0x7b, 0xa5, 0xde, 0x3e, 0xac, 0x8d, 0x1c, 0xcf, 0xf5, 0xa7, 0x5f, 0xf9, 0x91, 0xcc, 0xbd, 0x6c, - 0x83, 0xcb, 0x8a, 0x5e, 0xbb, 0xe9, 0x3c, 0xc2, 0xd7, 0x34, 0x82, 0xed, 0x1c, 0xa4, 0xbb, 0x48, - 0xb0, 0xa3, 0x11, 0xb4, 0x92, 0x5e, 0x89, 0x6c, 0xc3, 0x07, 0xa7, 0xde, 0x95, 0xe5, 0x3a, 0xe3, - 0x64, 0xab, 0x53, 0x23, 0x1f, 0x40, 0x1b, 0x3f, 0x5a, 0x88, 0xad, 0x8e, 0x46, 0x3a, 0xb0, 0x2e, - 0xa6, 0x63, 0xb9, 0x53, 0x27, 0x9b, 0x00, 0xe7, 0xb1, 0x1f, 0x48, 0xba, 0x81, 0xf4, 0x85, 0x7f, - 0x2d, 0xe9, 0xe6, 0xd1, 0x4f, 0xa1, 0x95, 0x3c, 0xd0, 0x8a, 0x8e, 0x64, 0xab, 0x53, 0x23, 0x5b, - 0xb0, 0xf1, 0xec, 0xca, 0xb1, 0xe3, 0x74, 0x4b, 0x23, 0x7b, 0xb0, 0xdd, 0xb7, 0x3c, 0x9b, 0xb9, - 0x79, 0x46, 0xfd, 0xe8, 0x97, 0xb0, 0x2a, 0xb3, 0x8d, 0x43, 0x93, 0xb2, 0x38, 0xd9, 0xa9, 0x91, - 0x75, 0x68, 0xf1, 0xdc, 0x47, 0x4a, 0xe3, 0x30, 0x44, 0x2a, 0x20, 0x8d, 0x30, 0x85, 0x17, 0x90, - 0x16, 0x30, 0x11, 0x22, 0xd2, 0xcd, 0x93, 0xbf, 0x6d, 0xc0, 0x8a, 0xf8, 0x01, 0xf9, 0x1a, 0xd6, - 0xd2, 0x2f, 0x35, 0x04, 0xab, 0x75, 0xf1, 0xd3, 0x90, 0xb1, 0x5b, 0xd8, 0x15, 0x8e, 0xa5, 0x0f, - 0x7f, 0xfb, 0x8f, 0x7f, 0xff, 0xa9, 0x7e, 0x9f, 0xee, 0x1c, 0x5b, 0x81, 0x13, 0x1d, 0x5f, 0x3d, - 0xb1, 0xdc, 0xe0, 0xc2, 0x7a, 0x72, 0xcc, 0xb3, 0x25, 0xfa, 0x54, 0x3b, 0x22, 0x13, 0x68, 0x2b, - 0x1f, 0x4c, 0x48, 0x97, 0x8b, 0x29, 0x7f, 0xa2, 0x31, 0xf6, 0x4a, 0xfb, 0x52, 0xc1, 0x23, 0x54, - 0x70, 0x60, 0x3c, 0xa8, 0x52, 0x70, 0xfc, 0x86, 0x5f, 0xb6, 0x6f, 0xb9, 0x9e, 0xcf, 0x00, 0xb2, - 0x8f, 0x18, 0x04, 0xd1, 0x96, 0xbe, 0x8b, 0x18, 0xdd, 0xe2, 0xb6, 0x54, 0x52, 0x23, 0x2e, 0xb4, - 0x95, 0x79, 0x9f, 0x18, 0x85, 0x0f, 0x00, 0xca, 0x07, 0x0a, 0xe3, 0x41, 0x25, 0x4f, 0x4a, 0xfa, - 0x18, 0xe1, 0xf6, 0xc8, 0x7e, 0x01, 0x6e, 0x84, 0x47, 0x25, 0x5e, 0xd2, 0x87, 0x75, 0x75, 0xac, - 0x26, 0x68, 0x7d, 0xc5, 0xf7, 0x04, 0x43, 0x2f, 0x33, 0x52, 0xc8, 0x5f, 0xc0, 0x46, 0x6e, 0x90, - 0x25, 0x78, 0xb8, 0x6a, 0x92, 0x36, 0xee, 0x57, 0x70, 0x52, 0x39, 0x5f, 0x43, 0xb7, 0x3c, 0x78, - 0xa2, 0x17, 0x3f, 0x54, 0x82, 0x52, 0x1e, 0xfe, 0x8c, 0xde, 0x22, 0x76, 0x2a, 0xfa, 0x0c, 0x3a, - 0xc5, 0x01, 0x8d, 0xa0, 0xfb, 0x16, 0xcc, 0x93, 0xc6, 0x7e, 0x35, 0x33, 0x15, 0xf8, 0x29, 0xac, - 0xa5, 0xf3, 0x94, 0x48, 0xd4, 0xe2, 0x18, 0x26, 0x12, 0xb5, 0x34, 0x74, 0xd1, 0x1a, 0x99, 0xc2, - 0x46, 0x6e, 0xc4, 0x11, 0xfe, 0xaa, 0x9a, 0xaf, 0x84, 0xbf, 0x2a, 0xe7, 0x21, 0xfa, 0x11, 0x06, - 0xf8, 0x81, 0xd1, 0x2d, 0x06, 0x58, 0x14, 0x18, 0x9e, 0x8a, 0xa7, 0xb0, 0x99, 0x9f, 0x46, 0xc8, - 0x7d, 0x31, 0xb1, 0x57, 0x0c, 0x3a, 0x86, 0x51, 0xc5, 0x4a, 0x31, 0x87, 0xb0, 0x91, 0x1b, 0x2a, - 0x24, 0xe6, 0x8a, 0x39, 0x45, 0x62, 0xae, 0x9a, 0x40, 0xe8, 0x0f, 0x11, 0xf3, 0xa3, 0xa3, 0x8f, - 0x0b, 0x98, 0x65, 0x6f, 0x72, 0xfc, 0x86, 0x3f, 0x63, 0xdf, 0x26, 0xc9, 0x79, 0x99, 0xfa, 0x49, - 0x94, 0xa1, 0x9c, 0x9f, 0x72, 0x83, 0x49, 0xce, 0x4f, 0xf9, 0xe1, 0x83, 0x7e, 0x82, 0x3a, 0x1f, - 0x1a, 0x46, 0x41, 0xa7, 0xe8, 0xdd, 0x8e, 0xdf, 0xf8, 0x01, 0x5e, 0xdb, 0x5f, 0x01, 0x64, 0xdd, - 0x97, 0xb8, 0xb6, 0xa5, 0x06, 0x50, 0x5c, 0xdb, 0x72, 0x93, 0x46, 0x7b, 0xa8, 0x43, 0x27, 0xdd, - 0x6a, 0xbb, 0xc8, 0x24, 0x8b, 0x38, 0x36, 0x31, 0xf9, 0x88, 0xab, 0x5d, 0x58, 0x3e, 0xe2, 0xb9, - 0x8e, 0x87, 0x1e, 0xa0, 0x16, 0xc3, 0xd8, 0x2d, 0x46, 0x1c, 0x8f, 0x71, 0x23, 0x5c, 0x6c, 0x19, - 0xb2, 0x4e, 0x44, 0xe8, 0xa9, 0x6a, 0x64, 0x84, 0x9e, 0xca, 0xb6, 0x25, 0xa9, 0x74, 0xa4, 0x57, - 0xd4, 0x33, 0x1f, 0xa9, 0xc5, 0x8e, 0xbc, 0x84, 0x15, 0xd1, 0x5a, 0x90, 0x2d, 0x29, 0x4c, 0x91, - 0x4f, 0xd4, 0x2d, 0x29, 0xf8, 0x07, 0x28, 0xf8, 0x43, 0x72, 0x5b, 0x09, 0x25, 0xdf, 0x40, 0x5b, - 0x79, 0x8d, 0x45, 0x9d, 0x2e, 0x77, 0x0c, 0xa2, 0x4e, 0x57, 0x3c, 0xdb, 0x0b, 0xbd, 0xc4, 0xf8, - 0x29, 0xbc, 0x16, 0x7d, 0x58, 0x57, 0xbb, 0x15, 0x51, 0xf4, 0x2a, 0xda, 0x1a, 0x43, 0x2f, 0x33, - 0x92, 0x0b, 0xf1, 0x54, 0xff, 0xfb, 0xdb, 0x9e, 0xf6, 0xfd, 0xdb, 0x9e, 0xf6, 0xaf, 0xb7, 0x3d, - 0xed, 0x0f, 0xef, 0x7a, 0xb5, 0xef, 0xdf, 0xf5, 0x6a, 0xff, 0x7c, 0xd7, 0xab, 0x8d, 0x56, 0xf0, - 0x6f, 0x8d, 0x1f, 0xfd, 0x27, 0x00, 0x00, 0xff, 0xff, 0xf5, 0xb6, 0xf4, 0x66, 0x1a, 0x19, 0x00, - 0x00, + // 1893 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xbc, 0x59, 0x5f, 0x6f, 0xdb, 0xc8, + 0x11, 0x17, 0x25, 0xc5, 0x96, 0x47, 0xb6, 0x4f, 0x5e, 0xdb, 0x32, 0xc3, 0xf8, 0x14, 0xdf, 0xf6, + 0x2e, 0x30, 0x8c, 0x22, 0x46, 0xdc, 0x3e, 0x1d, 0x70, 0x05, 0x2e, 0x52, 0x2e, 0x67, 0x54, 0xa9, + 0xaf, 0x74, 0x82, 0xf6, 0x50, 0xa0, 0x38, 0x8a, 0x5a, 0xc9, 0x84, 0x29, 0x92, 0x21, 0x29, 0xbb, + 0x46, 0x70, 0x2f, 0xfd, 0x00, 0xfd, 0x83, 0x3e, 0xdc, 0x63, 0x1f, 0xfa, 0x0d, 0xfa, 0x29, 0xfa, + 0x78, 0x40, 0x81, 0xa2, 0x8f, 0x45, 0xd2, 0x0f, 0x52, 0xec, 0xec, 0x92, 0x5c, 0xfe, 0x91, 0x5b, + 0x05, 0xa8, 0xdf, 0x76, 0x76, 0x56, 0x33, 0xbf, 0xf9, 0xb3, 0xb3, 0x33, 0x14, 0x6c, 0x8e, 0x67, + 0x33, 0x2b, 0x8a, 0x59, 0xf8, 0x38, 0x08, 0xfd, 0xd8, 0x27, 0xf5, 0x60, 0x64, 0x6c, 0x8e, 0x67, + 0xd7, 0x7e, 0x78, 0x99, 0xec, 0x19, 0xfb, 0x53, 0xdf, 0x9f, 0xba, 0xec, 0xd8, 0x0a, 0x9c, 0x63, + 0xcb, 0xf3, 0xfc, 0xd8, 0x8a, 0x1d, 0xdf, 0x8b, 0x04, 0x97, 0x7e, 0x03, 0x9d, 0xf3, 0xd8, 0x0a, + 0xe3, 0x97, 0x56, 0x74, 0x69, 0xb2, 0xd7, 0x73, 0x16, 0xc5, 0x84, 0x40, 0x33, 0xb6, 0xa2, 0x4b, + 0x5d, 0x3b, 0xd0, 0x0e, 0xd7, 0x4c, 0x5c, 0x13, 0x1d, 0x56, 0x23, 0x7f, 0x1e, 0xda, 0x2c, 0xd2, + 0xeb, 0x07, 0x8d, 0xc3, 0x35, 0x33, 0x21, 0x49, 0x0f, 0x20, 0x64, 0x33, 0xff, 0x8a, 0xbd, 0x60, + 0xb1, 0xa5, 0x37, 0x0e, 0xb4, 0xc3, 0x96, 0xa9, 0xec, 0xd0, 0xd7, 0xb0, 0xa5, 0x68, 0x88, 0x02, + 0xdf, 0x8b, 0x18, 0xe9, 0xc2, 0x4a, 0xc8, 0xa2, 0xb9, 0x1b, 0xa3, 0x92, 0x96, 0x29, 0x29, 0xd2, + 0x81, 0xc6, 0x2c, 0x9a, 0xea, 0x75, 0xd4, 0xcc, 0x97, 0xe4, 0x24, 0x53, 0xdc, 0x38, 0x68, 0x1c, + 0xb6, 0x4f, 0xf4, 0xc7, 0xc1, 0xe8, 0x71, 0xdf, 0x9f, 0xcd, 0x7c, 0xef, 0x17, 0x68, 0x67, 0x22, + 0x34, 0x85, 0x44, 0x7f, 0x0d, 0xe4, 0x2c, 0x60, 0xa1, 0x15, 0x33, 0xd5, 0x2c, 0x03, 0xea, 0x7e, + 0x80, 0xfa, 0x36, 0x4f, 0x80, 0x0b, 0xe1, 0xcc, 0xb3, 0xc0, 0xac, 0xfb, 0x01, 0x37, 0xd9, 0xb3, + 0x66, 0x4c, 0x2a, 0xc6, 0xb5, 0x6a, 0x72, 0x23, 0x67, 0x32, 0xfd, 0xbd, 0x06, 0xdb, 0x39, 0x05, + 0xd2, 0xaa, 0xdb, 0x34, 0x64, 0x16, 0xd7, 0xab, 0x2c, 0x6e, 0x54, 0x5a, 0xdc, 0xfc, 0x5f, 0x2d, + 0xfe, 0x1c, 0xb6, 0x5e, 0x05, 0xe3, 0x82, 0xc1, 0x4b, 0xc5, 0x91, 0x86, 0x40, 0x54, 0x11, 0x77, + 0x12, 0xa8, 0x2f, 0xa0, 0xfb, 0xf3, 0x39, 0x0b, 0x6f, 0xce, 0x63, 0x2b, 0x9e, 0x47, 0x43, 0x27, + 0x8a, 0x15, 0xec, 0x18, 0x10, 0xad, 0x3a, 0x20, 0x05, 0xec, 0x57, 0xb0, 0x57, 0x92, 0xb3, 0xb4, + 0x01, 0x4f, 0x8a, 0x06, 0xec, 0x71, 0x03, 0x14, 0xb9, 0x65, 0xfc, 0x7d, 0xd8, 0x3e, 0xbf, 0xf0, + 0xaf, 0x07, 0x83, 0xe1, 0xd0, 0xb7, 0x2f, 0xa3, 0xf7, 0x73, 0xfc, 0x9f, 0x35, 0x58, 0x95, 0x12, + 0xc8, 0x26, 0xd4, 0x4f, 0x07, 0xf2, 0x77, 0xf5, 0xd3, 0x41, 0x2a, 0xa9, 0xae, 0x48, 0x22, 0xd0, + 0x9c, 0xf9, 0x63, 0x26, 0x53, 0x06, 0xd7, 0x64, 0x07, 0xee, 0xf9, 0xd7, 0x1e, 0x0b, 0xf5, 0x26, + 0x6e, 0x0a, 0x82, 0x9f, 0x1c, 0x0c, 0x86, 0x91, 0x7e, 0x0f, 0x15, 0xe2, 0x9a, 0xfb, 0x23, 0xba, + 0xf1, 0x6c, 0x36, 0xd6, 0x57, 0x70, 0x57, 0x52, 0xc4, 0x80, 0xd6, 0xdc, 0x93, 0x9c, 0x55, 0xe4, + 0xa4, 0x34, 0xb5, 0x61, 0x27, 0x6f, 0xe6, 0xd2, 0xbe, 0xfd, 0x08, 0xee, 0xb9, 0xfc, 0xa7, 0xd2, + 0xb3, 0x6d, 0xee, 0x59, 0x29, 0xce, 0x14, 0x1c, 0xea, 0xc2, 0xce, 0x2b, 0x8f, 0x2f, 0x93, 0x7d, + 0xe9, 0xcc, 0xa2, 0x4b, 0x28, 0xac, 0x87, 0x2c, 0x70, 0x2d, 0x9b, 0x9d, 0xa1, 0xc5, 0x42, 0x4b, + 0x6e, 0x8f, 0x1c, 0x40, 0x7b, 0xe2, 0x87, 0x36, 0x33, 0xb1, 0x0c, 0xc9, 0xa2, 0xa4, 0x6e, 0xd1, + 0xcf, 0x61, 0xb7, 0xa0, 0x6d, 0x59, 0x9b, 0xa8, 0x09, 0xf7, 0x65, 0x11, 0x48, 0xd2, 0xdb, 0xb5, + 0x6e, 0x12, 0xd4, 0x0f, 0x94, 0x52, 0x80, 0xd6, 0x22, 0x57, 0xd6, 0x82, 0xc5, 0xb9, 0xf0, 0x9d, + 0x06, 0x46, 0x95, 0x50, 0x09, 0xee, 0x56, 0xa9, 0xff, 0xdf, 0x0a, 0xf3, 0x9d, 0x06, 0x7b, 0x5f, + 0xcd, 0xc3, 0x69, 0x95, 0xb1, 0x8a, 0x3d, 0x5a, 0xfe, 0x71, 0x30, 0xa0, 0xe5, 0x78, 0x96, 0x1d, + 0x3b, 0x57, 0x4c, 0xa2, 0x4a, 0x69, 0xcc, 0x6d, 0x67, 0x26, 0xa2, 0xd3, 0x30, 0x71, 0xcd, 0xcf, + 0x4f, 0x1c, 0x97, 0xe1, 0xd5, 0x17, 0xa9, 0x9c, 0xd2, 0x98, 0xb9, 0xf3, 0xd1, 0xc0, 0x09, 0xf5, + 0x7b, 0xc8, 0x91, 0x14, 0xfd, 0x0d, 0xe8, 0x65, 0x60, 0x77, 0x52, 0xbe, 0x1e, 0x41, 0xa7, 0x7f, + 0xc1, 0xec, 0xcb, 0xff, 0x52, 0x74, 0xe9, 0x67, 0xb0, 0xa5, 0x9c, 0x5b, 0x3a, 0xd1, 0x2e, 0x60, + 0x47, 0xe6, 0xc4, 0x39, 0x2a, 0x4e, 0x54, 0xed, 0x2b, 0xd9, 0xb0, 0xce, 0xd1, 0x0a, 0x76, 0x96, + 0x0e, 0xb6, 0xef, 0x4d, 0x9c, 0xa9, 0xcc, 0x31, 0x49, 0x71, 0x17, 0x0b, 0xfc, 0xa7, 0x03, 0xf9, + 0xae, 0xa5, 0x34, 0x9d, 0xc3, 0x6e, 0x41, 0xd3, 0x9d, 0xf8, 0xf1, 0x19, 0xec, 0x9a, 0x6c, 0xea, + 0xf0, 0x46, 0x26, 0x39, 0x72, 0xeb, 0x2b, 0x60, 0x8d, 0xc7, 0x21, 0x8b, 0x22, 0xa9, 0x36, 0x21, + 0xe9, 0x53, 0xe8, 0x16, 0xc5, 0x2c, 0xed, 0xeb, 0x9f, 0xc0, 0xce, 0xd9, 0x64, 0xe2, 0x3a, 0x1e, + 0x7b, 0xc1, 0x66, 0xa3, 0x1c, 0x92, 0xf8, 0x26, 0x48, 0x91, 0xf0, 0x75, 0x55, 0xd3, 0xc0, 0xeb, + 0x4a, 0xe1, 0xf7, 0x4b, 0x43, 0xf8, 0x71, 0x1a, 0xee, 0x21, 0xb3, 0xc6, 0x19, 0x84, 0x52, 0xb8, + 0x05, 0x5b, 0x84, 0x1b, 0x15, 0xe7, 0x7f, 0xb5, 0xb4, 0xe2, 0xdf, 0x69, 0x00, 0x2f, 0xb0, 0x9d, + 0x3c, 0xf5, 0x26, 0x7e, 0xa5, 0xf3, 0x0d, 0x68, 0xcd, 0xd0, 0xae, 0xd3, 0x01, 0xfe, 0xb2, 0x69, + 0xa6, 0x34, 0x7f, 0x83, 0x2c, 0xd7, 0x49, 0xcb, 0xad, 0x20, 0xf8, 0x2f, 0x02, 0xc6, 0xc2, 0x57, + 0xe6, 0x50, 0x14, 0x9b, 0x35, 0x33, 0xa5, 0x79, 0xeb, 0x68, 0xbb, 0x0e, 0xf3, 0x62, 0xe4, 0x8a, + 0x57, 0x4a, 0xd9, 0xa1, 0x23, 0x00, 0x11, 0xc8, 0x85, 0x78, 0x08, 0x34, 0x79, 0xf4, 0x93, 0x10, + 0xf0, 0x35, 0xc7, 0x11, 0xc5, 0xd6, 0x34, 0x79, 0x20, 0x05, 0x81, 0xd5, 0x03, 0xd3, 0x4d, 0xd6, + 0x15, 0x49, 0xd1, 0x21, 0x74, 0x78, 0xbf, 0x20, 0x9c, 0x26, 0x62, 0x96, 0xb8, 0x46, 0xcb, 0xb2, + 0xba, 0xaa, 0x3f, 0x4c, 0x74, 0x37, 0x32, 0xdd, 0xf4, 0x67, 0x42, 0x9a, 0xf0, 0xe2, 0x42, 0x69, + 0x87, 0xb0, 0x2a, 0xda, 0x76, 0x51, 0xff, 0xdb, 0x27, 0x9b, 0x3c, 0x9c, 0x99, 0xeb, 0xcd, 0x84, + 0x9d, 0xc8, 0x13, 0x5e, 0xb8, 0x4d, 0x9e, 0x68, 0xf9, 0x73, 0xf2, 0x32, 0xd7, 0x99, 0x09, 0x9b, + 0xfe, 0x45, 0x83, 0x55, 0x21, 0x26, 0x22, 0x8f, 0x61, 0xc5, 0x45, 0xab, 0x51, 0x54, 0xfb, 0x64, + 0x07, 0x73, 0xaa, 0xe0, 0x8b, 0x2f, 0x6b, 0xa6, 0x3c, 0xc5, 0xcf, 0x0b, 0x58, 0xe8, 0x05, 0xe5, + 0xbc, 0x6a, 0x2d, 0x3f, 0x2f, 0x4e, 0xf1, 0xf3, 0x42, 0x2d, 0x7a, 0x48, 0x39, 0xaf, 0x5a, 0xc3, + 0xcf, 0x8b, 0x53, 0x4f, 0x5b, 0xb0, 0x22, 0x72, 0x89, 0x8f, 0x0c, 0x28, 0x37, 0x77, 0x03, 0xbb, + 0x39, 0xb8, 0xad, 0x14, 0x56, 0x37, 0x07, 0xab, 0x95, 0xaa, 0xef, 0xe6, 0xd4, 0xb7, 0x12, 0x35, + 0x3c, 0x3d, 0x78, 0xf8, 0x92, 0x6c, 0x14, 0x04, 0x65, 0x40, 0x54, 0x95, 0x4b, 0x97, 0xbd, 0x4f, + 0x60, 0x55, 0x80, 0xcf, 0xb5, 0x38, 0xd2, 0xd5, 0x66, 0xc2, 0xa3, 0xff, 0xd0, 0xb2, 0x5a, 0x6e, + 0x5f, 0xb0, 0x99, 0xb5, 0xb8, 0x96, 0x23, 0x3b, 0x1b, 0x4f, 0x4a, 0x6d, 0xe0, 0xc2, 0xf1, 0x84, + 0x5f, 0xb9, 0xb1, 0x15, 0x5b, 0x23, 0x2b, 0x4a, 0x1f, 0xd1, 0x84, 0xe6, 0xd6, 0xc7, 0xd6, 0xc8, + 0x65, 0xf2, 0x0d, 0x15, 0x04, 0x5e, 0x0e, 0xd4, 0xa7, 0xaf, 0xc8, 0xcb, 0x81, 0x14, 0x3f, 0x3d, + 0x71, 0xe7, 0xd1, 0x85, 0xbe, 0x2a, 0xae, 0x34, 0x12, 0x1c, 0x0d, 0x6f, 0x0c, 0xf5, 0x16, 0x6e, + 0xe2, 0x5a, 0x7d, 0x39, 0xa4, 0x5d, 0x77, 0xf2, 0x72, 0x1c, 0xc1, 0xce, 0x73, 0x16, 0x9f, 0xcf, + 0x47, 0xfc, 0x69, 0xed, 0x4f, 0xa6, 0xb7, 0x3c, 0x1c, 0xf4, 0x15, 0xec, 0x16, 0xce, 0x2e, 0x0d, + 0x91, 0x40, 0xd3, 0x9e, 0x4c, 0x13, 0x87, 0xe3, 0x9a, 0x0e, 0x60, 0xe3, 0x39, 0x8b, 0x15, 0xdd, + 0x0f, 0x95, 0xa7, 0x42, 0xb6, 0x69, 0xfd, 0xc9, 0xf4, 0xe5, 0x4d, 0xc0, 0x6e, 0x79, 0x37, 0x86, + 0xb0, 0x99, 0x48, 0x59, 0x1a, 0x55, 0x07, 0x1a, 0xf6, 0x24, 0x6d, 0xf0, 0xec, 0xc9, 0x94, 0xee, + 0xc2, 0xf6, 0x73, 0x26, 0xef, 0x65, 0x86, 0x8c, 0x1e, 0xa2, 0xb7, 0x94, 0x6d, 0xa9, 0x4a, 0x0a, + 0xd0, 0x32, 0x01, 0x7f, 0xd4, 0x80, 0x7c, 0x69, 0x79, 0x63, 0x97, 0x3d, 0x0b, 0x43, 0x3f, 0x5c, + 0xd8, 0xd5, 0x22, 0xf7, 0xbd, 0x92, 0x74, 0x1f, 0xd6, 0x46, 0x8e, 0xe7, 0xfa, 0xd3, 0xaf, 0xfc, + 0x48, 0x66, 0x69, 0xb6, 0x81, 0x29, 0xf6, 0xda, 0x4d, 0x27, 0x17, 0xbe, 0xa6, 0x11, 0x6c, 0xe7, + 0x20, 0xdd, 0x45, 0x82, 0x1d, 0x8d, 0xa0, 0x95, 0x74, 0x55, 0x64, 0x1b, 0x3e, 0x38, 0xf5, 0xae, + 0x2c, 0xd7, 0x19, 0x27, 0x5b, 0x9d, 0x1a, 0xf9, 0x00, 0xda, 0xf8, 0x79, 0x43, 0x6c, 0x75, 0x34, + 0xd2, 0x81, 0x75, 0x31, 0x47, 0xcb, 0x9d, 0x3a, 0xd9, 0x04, 0x38, 0x8f, 0xfd, 0x40, 0xd2, 0x0d, + 0xa4, 0x2f, 0xfc, 0x6b, 0x49, 0x37, 0x8f, 0x7e, 0x0a, 0xad, 0xe4, 0x29, 0x57, 0x74, 0x24, 0x5b, + 0x9d, 0x1a, 0xd9, 0x82, 0x8d, 0x67, 0x57, 0x8e, 0x1d, 0xa7, 0x5b, 0x1a, 0xd9, 0x83, 0xed, 0xbe, + 0xe5, 0xd9, 0xcc, 0xcd, 0x33, 0xea, 0x47, 0xbf, 0x84, 0x55, 0x99, 0x6d, 0x1c, 0x9a, 0x94, 0xc5, + 0xc9, 0x4e, 0x8d, 0xac, 0x43, 0x8b, 0xe7, 0x3e, 0x52, 0x1a, 0x87, 0x21, 0x52, 0x01, 0x69, 0x84, + 0x29, 0xbc, 0x80, 0xb4, 0x80, 0x89, 0x10, 0x91, 0x6e, 0x9e, 0xfc, 0x75, 0x03, 0x56, 0xc4, 0x0f, + 0xc8, 0xd7, 0xb0, 0x96, 0x7e, 0xd3, 0x21, 0x58, 0xd7, 0x8b, 0x1f, 0x91, 0x8c, 0xdd, 0xc2, 0xae, + 0x70, 0x2c, 0x7d, 0xf8, 0xdb, 0xbf, 0xff, 0xfb, 0x4f, 0xf5, 0xfb, 0x74, 0xe7, 0xd8, 0x0a, 0x9c, + 0xe8, 0xf8, 0xea, 0x89, 0xe5, 0x06, 0x17, 0xd6, 0x93, 0x63, 0x9e, 0x2d, 0xd1, 0xa7, 0xda, 0x11, + 0x99, 0x40, 0x5b, 0xf9, 0xb4, 0x42, 0xba, 0x5c, 0x4c, 0xf9, 0x63, 0x8e, 0xb1, 0x57, 0xda, 0x97, + 0x0a, 0x1e, 0xa1, 0x82, 0x03, 0xe3, 0x41, 0x95, 0x82, 0xe3, 0x37, 0xfc, 0xb2, 0x7d, 0xcb, 0xf5, + 0x7c, 0x06, 0x90, 0x7d, 0xee, 0x20, 0x88, 0xb6, 0xf4, 0x05, 0xc5, 0xe8, 0x16, 0xb7, 0xa5, 0x92, + 0x1a, 0x71, 0xa1, 0xad, 0x7c, 0x19, 0x20, 0x46, 0xe1, 0x53, 0x81, 0xf2, 0x29, 0xc3, 0x78, 0x50, + 0xc9, 0x93, 0x92, 0x3e, 0x46, 0xb8, 0x3d, 0xb2, 0x5f, 0x80, 0x1b, 0xe1, 0x51, 0x89, 0x97, 0xf4, + 0x61, 0x5d, 0x1d, 0xc0, 0x09, 0x5a, 0x5f, 0xf1, 0xe5, 0xc1, 0xd0, 0xcb, 0x8c, 0x14, 0xf2, 0x17, + 0xb0, 0x91, 0x1b, 0x79, 0x09, 0x1e, 0xae, 0x9a, 0xb9, 0x8d, 0xfb, 0x15, 0x9c, 0x54, 0xce, 0xd7, + 0xd0, 0x2d, 0x8f, 0xa8, 0xe8, 0xc5, 0x0f, 0x95, 0xa0, 0x94, 0xc7, 0x44, 0xa3, 0xb7, 0x88, 0x9d, + 0x8a, 0x3e, 0x83, 0x4e, 0x71, 0x94, 0x23, 0xe8, 0xbe, 0x05, 0x93, 0xa7, 0xb1, 0x5f, 0xcd, 0x4c, + 0x05, 0x7e, 0x0a, 0x6b, 0xe9, 0xe4, 0x25, 0x12, 0xb5, 0x38, 0xb0, 0x89, 0x44, 0x2d, 0x8d, 0x67, + 0xb4, 0x46, 0xa6, 0xb0, 0x91, 0x1b, 0x86, 0x84, 0xbf, 0xaa, 0x26, 0x31, 0xe1, 0xaf, 0xca, 0xc9, + 0x89, 0x7e, 0x84, 0x01, 0x7e, 0x60, 0x74, 0x8b, 0x01, 0x16, 0x05, 0x86, 0xa7, 0xe2, 0x29, 0x6c, + 0xe6, 0xe7, 0x16, 0x72, 0x5f, 0xcc, 0xf6, 0x15, 0x23, 0x91, 0x61, 0x54, 0xb1, 0x52, 0xcc, 0x21, + 0x6c, 0xe4, 0xc6, 0x0f, 0x89, 0xb9, 0x62, 0xa2, 0x91, 0x98, 0xab, 0x66, 0x15, 0xfa, 0x43, 0xc4, + 0xfc, 0xe8, 0xe8, 0xe3, 0x02, 0x66, 0xd9, 0xc5, 0x1c, 0xbf, 0xe1, 0xcf, 0xd8, 0xb7, 0x49, 0x72, + 0x5e, 0xa6, 0x7e, 0x12, 0x65, 0x28, 0xe7, 0xa7, 0xdc, 0x08, 0x93, 0xf3, 0x53, 0x7e, 0x4c, 0xa1, + 0x9f, 0xa0, 0xce, 0x87, 0x86, 0x51, 0xd0, 0x29, 0xba, 0xbc, 0xe3, 0x37, 0x7e, 0x80, 0xd7, 0xf6, + 0x57, 0x00, 0x59, 0x9f, 0x26, 0xae, 0x6d, 0xa9, 0x55, 0x14, 0xd7, 0xb6, 0xdc, 0xce, 0xd1, 0x1e, + 0xea, 0xd0, 0x49, 0xb7, 0xda, 0x2e, 0x32, 0xc9, 0x22, 0x2e, 0xfa, 0x9f, 0x5c, 0xc4, 0xd5, 0x7e, + 0x2d, 0x1f, 0xf1, 0x5c, 0xc7, 0x43, 0x0f, 0x50, 0x8b, 0x61, 0xec, 0x16, 0x23, 0x8e, 0xc7, 0xb8, + 0x11, 0x2e, 0xb6, 0x0c, 0x59, 0x27, 0x22, 0xf4, 0x54, 0x35, 0x32, 0x42, 0x4f, 0x65, 0xdb, 0x92, + 0x54, 0x3a, 0xd2, 0x2b, 0xea, 0x99, 0x8f, 0xd4, 0x62, 0x47, 0x5e, 0xc2, 0x8a, 0x68, 0x2d, 0xc8, + 0x96, 0x14, 0xa6, 0xc8, 0x27, 0xea, 0x96, 0x14, 0xfc, 0x03, 0x14, 0xfc, 0x21, 0xb9, 0xad, 0x84, + 0x92, 0x6f, 0xa0, 0xad, 0xbc, 0xc6, 0xa2, 0x4e, 0x97, 0x3b, 0x06, 0x51, 0xa7, 0x2b, 0x9e, 0xed, + 0x85, 0x5e, 0x62, 0xfc, 0x14, 0x5e, 0x8b, 0x3e, 0xac, 0xab, 0xdd, 0x8a, 0x28, 0x7a, 0x15, 0x6d, + 0x8d, 0xa1, 0x97, 0x19, 0xc9, 0x85, 0x78, 0xaa, 0xff, 0xed, 0x6d, 0x4f, 0xfb, 0xfe, 0x6d, 0x4f, + 0xfb, 0xd7, 0xdb, 0x9e, 0xf6, 0x87, 0x77, 0xbd, 0xda, 0xf7, 0xef, 0x7a, 0xb5, 0x7f, 0xbe, 0xeb, + 0xd5, 0x46, 0x2b, 0xf8, 0x07, 0xc8, 0x8f, 0xfe, 0x13, 0x00, 0x00, 0xff, 0xff, 0xc7, 0xd9, 0x1d, + 0x47, 0x44, 0x19, 0x00, 0x00, } // Reference imports to suppress errors if they are not otherwise used. @@ -5488,6 +5505,26 @@ func (m *OperateSchemaRequest) MarshalToSizedBuffer(dAtA []byte) (int, error) { _ = i var l int _ = l + if m.Sync { + i-- + if m.Sync { + dAtA[i] = 1 + } else { + dAtA[i] = 0 + } + i-- + dAtA[i] = 0x40 + } + if m.Flush { + i-- + if m.Flush { + dAtA[i] = 1 + } else { + dAtA[i] = 0 + } + i-- + dAtA[i] = 0x38 + } if len(m.Schema) > 0 { i -= len(m.Schema) copy(dAtA[i:], m.Schema) @@ -6722,6 +6759,12 @@ func (m *OperateSchemaRequest) Size() (n int) { if l > 0 { n += 1 + l + sovDmmaster(uint64(l)) } + if m.Flush { + n += 2 + } + if m.Sync { + n += 2 + } return n } @@ -11838,6 +11881,46 @@ func (m *OperateSchemaRequest) Unmarshal(dAtA []byte) error { } m.Schema = string(dAtA[iNdEx:postIndex]) iNdEx = postIndex + case 7: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Flush", wireType) + } + var v int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowDmmaster + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + m.Flush = bool(v != 0) + case 8: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Sync", wireType) + } + var v int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowDmmaster + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + m.Sync = bool(v != 0) default: iNdEx = preIndex skippy, err := skipDmmaster(dAtA[iNdEx:]) diff --git a/dm/pb/dmworker.pb.go b/dm/pb/dmworker.pb.go index 4d199df92e..988326f1cf 100644 --- a/dm/pb/dmworker.pb.go +++ b/dm/pb/dmworker.pb.go @@ -2066,6 +2066,8 @@ type OperateWorkerSchemaRequest struct { Database string `protobuf:"bytes,4,opt,name=database,proto3" json:"database,omitempty"` Table string `protobuf:"bytes,5,opt,name=table,proto3" json:"table,omitempty"` Schema string `protobuf:"bytes,6,opt,name=schema,proto3" json:"schema,omitempty"` + Flush bool `protobuf:"varint,7,opt,name=flush,proto3" json:"flush,omitempty"` + Sync bool `protobuf:"varint,8,opt,name=sync,proto3" json:"sync,omitempty"` } func (m *OperateWorkerSchemaRequest) Reset() { *m = OperateWorkerSchemaRequest{} } @@ -2143,6 +2145,20 @@ func (m *OperateWorkerSchemaRequest) GetSchema() string { return "" } +func (m *OperateWorkerSchemaRequest) GetFlush() bool { + if m != nil { + return m.Flush + } + return false +} + +func (m *OperateWorkerSchemaRequest) GetSync() bool { + if m != nil { + return m.Sync + } + return false +} + // copied `TaskMeta` from release-1.0 branch. type V1SubTaskMeta struct { Op TaskOp `protobuf:"varint,1,opt,name=op,proto3,enum=pb.TaskOp" json:"op,omitempty"` @@ -2509,130 +2525,131 @@ func init() { func init() { proto.RegisterFile("dmworker.proto", fileDescriptor_51a1b9e17fd67b10) } var fileDescriptor_51a1b9e17fd67b10 = []byte{ - // 1962 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, 0x38, 0x8f, 0xc5, 0x84, 0xc5, 0xa5, + // 1980 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xb4, 0x58, 0x41, 0x73, 0xdc, 0x4a, + 0x11, 0x5e, 0xad, 0x76, 0xd7, 0xbb, 0xbd, 0x6b, 0x47, 0x99, 0x38, 0x8f, 0xc5, 0x84, 0xc5, 0xa5, 0xbc, 0x0a, 0xc6, 0x07, 0x17, 0x31, 0x8f, 0x7a, 0xd4, 0xab, 0x02, 0x42, 0xec, 0x3c, 0xe7, 0x81, 0x83, 0x13, 0x39, 0x79, 0x1c, 0xa9, 0x59, 0x69, 0xbc, 0x56, 0x59, 0x2b, 0x29, 0xd2, 0xc8, 0xae, - 0x3d, 0xf0, 0x19, 0xe0, 0xc2, 0x81, 0x2a, 0x6e, 0x14, 0xd7, 0x57, 0x9c, 0xf8, 0x08, 0xc0, 0xf1, - 0x15, 0x27, 0x8e, 0x54, 0xf2, 0x35, 0x38, 0x50, 0xdd, 0x33, 0x92, 0x46, 0xf6, 0x6e, 0x42, 0x0e, - 0xdc, 0xa6, 0x7f, 0xdd, 0xd3, 0xdd, 0xd3, 0xd3, 0x7f, 0x34, 0x82, 0x8d, 0x60, 0x7e, 0x95, 0x64, - 0x17, 0x22, 0xdb, 0x4b, 0xb3, 0x44, 0x26, 0xac, 0x9d, 0x4e, 0xdd, 0x1d, 0x60, 0x2f, 0x0a, 0x91, - 0x2d, 0x4e, 0x25, 0x97, 0x45, 0xee, 0x89, 0xd7, 0x85, 0xc8, 0x25, 0x63, 0xd0, 0x89, 0xf9, 0x5c, - 0x8c, 0xad, 0x6d, 0x6b, 0x67, 0xe0, 0xd1, 0xda, 0x4d, 0x61, 0xf3, 0x20, 0x99, 0xcf, 0x93, 0xf8, - 0x57, 0xa4, 0xc3, 0x13, 0x79, 0x9a, 0xc4, 0xb9, 0x60, 0x1f, 0x41, 0x2f, 0x13, 0x79, 0x11, 0x49, - 0x92, 0xee, 0x7b, 0x9a, 0x62, 0x0e, 0xd8, 0xf3, 0x7c, 0x36, 0x6e, 0x93, 0x0a, 0x5c, 0xa2, 0x64, - 0x9e, 0x14, 0x99, 0x2f, 0xc6, 0x36, 0x81, 0x9a, 0x42, 0x5c, 0xf9, 0x35, 0xee, 0x28, 0x5c, 0x51, - 0xee, 0x57, 0x16, 0xdc, 0x69, 0x38, 0xf7, 0xc1, 0x16, 0x3f, 0x81, 0x91, 0xb2, 0xa1, 0x34, 0x90, - 0xdd, 0xe1, 0xbe, 0xb3, 0x97, 0x4e, 0xf7, 0x4e, 0x0d, 0xdc, 0x6b, 0x48, 0xb1, 0x4f, 0x61, 0x3d, - 0x2f, 0xa6, 0x2f, 0x79, 0x7e, 0xa1, 0xb7, 0x75, 0xb6, 0xed, 0x9d, 0xe1, 0xfe, 0x6d, 0xda, 0x66, - 0x32, 0xbc, 0xa6, 0x9c, 0xfb, 0x67, 0x0b, 0x86, 0x07, 0xe7, 0xc2, 0xd7, 0x34, 0x3a, 0x9a, 0xf2, - 0x3c, 0x17, 0x41, 0xe9, 0xa8, 0xa2, 0xd8, 0x26, 0x74, 0x65, 0x22, 0x79, 0x44, 0xae, 0x76, 0x3d, - 0x45, 0xb0, 0x09, 0x40, 0x5e, 0xf8, 0xbe, 0xc8, 0xf3, 0xb3, 0x22, 0x22, 0x57, 0xbb, 0x9e, 0x81, - 0xa0, 0xb6, 0x33, 0x1e, 0x46, 0x22, 0xa0, 0x30, 0x75, 0x3d, 0x4d, 0xb1, 0x31, 0xac, 0x5d, 0xf1, - 0x2c, 0x0e, 0xe3, 0xd9, 0xb8, 0x4b, 0x8c, 0x92, 0xc4, 0x1d, 0x81, 0x90, 0x3c, 0x8c, 0xc6, 0xbd, - 0x6d, 0x6b, 0x67, 0xe4, 0x69, 0xca, 0x1d, 0x01, 0x1c, 0x16, 0xf3, 0x54, 0x7b, 0xfd, 0x57, 0x0b, - 0xe0, 0x38, 0xe1, 0x81, 0x76, 0xfa, 0x63, 0x58, 0x3f, 0x0b, 0xe3, 0x30, 0x3f, 0x17, 0xc1, 0xe3, - 0x85, 0x14, 0x39, 0xf9, 0x6e, 0x7b, 0x4d, 0x10, 0x9d, 0x25, 0xaf, 0x95, 0x48, 0x9b, 0x44, 0x0c, - 0x84, 0x6d, 0x41, 0x3f, 0xcd, 0x92, 0x59, 0x26, 0xf2, 0x5c, 0xdf, 0x76, 0x45, 0xe3, 0xde, 0xb9, - 0x90, 0xfc, 0x71, 0x18, 0x47, 0xc9, 0x4c, 0xdf, 0xb9, 0x81, 0xb0, 0x07, 0xb0, 0x51, 0x53, 0x47, - 0x2f, 0xbf, 0x38, 0xa4, 0x73, 0x0d, 0xbc, 0x6b, 0xa8, 0xfb, 0x7b, 0x0b, 0xd6, 0x4f, 0xcf, 0x79, - 0x16, 0x84, 0xf1, 0xec, 0x28, 0x4b, 0x8a, 0x14, 0x0f, 0x2c, 0x79, 0x36, 0x13, 0x52, 0x67, 0xae, - 0xa6, 0x30, 0x9f, 0x0f, 0x0f, 0x8f, 0xd1, 0x4f, 0x1b, 0xf3, 0x19, 0xd7, 0xea, 0x9c, 0x59, 0x2e, - 0x8f, 0x13, 0x9f, 0xcb, 0x30, 0x89, 0xb5, 0x9b, 0x4d, 0x90, 0x72, 0x76, 0x11, 0xfb, 0x14, 0x74, - 0x9b, 0x72, 0x96, 0x28, 0x3c, 0x5f, 0x11, 0x6b, 0x4e, 0x97, 0x38, 0x15, 0xed, 0xfe, 0xc9, 0x06, - 0x38, 0x5d, 0xc4, 0xbe, 0x0e, 0xe8, 0x36, 0x0c, 0x29, 0x30, 0x4f, 0x2e, 0x45, 0x2c, 0xcb, 0x70, - 0x9a, 0x10, 0x2a, 0x23, 0xf2, 0x65, 0x5a, 0x86, 0xb2, 0xa2, 0xd9, 0x3d, 0x18, 0x64, 0xc2, 0x17, - 0xb1, 0x44, 0xa6, 0x4d, 0xcc, 0x1a, 0x60, 0x2e, 0x8c, 0xe6, 0x3c, 0x97, 0x22, 0x6b, 0x04, 0xb3, - 0x81, 0xb1, 0x5d, 0x70, 0x4c, 0xfa, 0x48, 0x86, 0x81, 0x0e, 0xe8, 0x0d, 0x1c, 0xf5, 0xd1, 0x21, - 0x4a, 0x7d, 0x3d, 0xa5, 0xcf, 0xc4, 0x50, 0x9f, 0x49, 0x93, 0xbe, 0x35, 0xa5, 0xef, 0x3a, 0x8e, - 0xfa, 0xa6, 0x51, 0xe2, 0x5f, 0x84, 0xf1, 0x8c, 0x2e, 0xa0, 0x4f, 0xa1, 0x6a, 0x60, 0xec, 0xc7, - 0xe0, 0x14, 0x71, 0x26, 0xf2, 0x24, 0xba, 0x14, 0x01, 0xdd, 0x63, 0x3e, 0x1e, 0x18, 0x15, 0x67, - 0xde, 0xb0, 0x77, 0x43, 0xd4, 0xb8, 0x21, 0x50, 0x45, 0xa6, 0x6f, 0x68, 0x02, 0x30, 0x25, 0x47, - 0x5e, 0x2e, 0x52, 0x31, 0x1e, 0xaa, 0x2c, 0xab, 0x11, 0xf7, 0x8f, 0x16, 0x8c, 0xcc, 0x26, 0x60, - 0xb4, 0x27, 0x6b, 0x45, 0x7b, 0x6a, 0x9b, 0xed, 0x89, 0x7d, 0xaf, 0x6a, 0x43, 0xaa, 0xad, 0x90, - 0xb7, 0xcf, 0xb3, 0x04, 0xeb, 0xd5, 0x23, 0x46, 0xd5, 0x99, 0x1e, 0xc2, 0x30, 0x13, 0x11, 0x5f, - 0x54, 0xfd, 0x04, 0xe5, 0x6f, 0xa1, 0xbc, 0x57, 0xc3, 0x9e, 0x29, 0xe3, 0xfe, 0xbd, 0x0d, 0x43, - 0x83, 0x79, 0xe3, 0xa6, 0xad, 0xff, 0xf1, 0xa6, 0xdb, 0x2b, 0x6e, 0x7a, 0xbb, 0x74, 0xa9, 0x98, - 0x1e, 0x86, 0x99, 0x4e, 0x7e, 0x13, 0xaa, 0x24, 0x1a, 0xa9, 0x65, 0x42, 0x6c, 0x07, 0x6e, 0x19, - 0xa4, 0x91, 0x58, 0xd7, 0x61, 0xb6, 0x07, 0x8c, 0xa0, 0x03, 0x2e, 0xfd, 0xf3, 0x57, 0xe9, 0x33, - 0xf2, 0x86, 0xb2, 0xab, 0xef, 0x2d, 0xe1, 0xb0, 0xef, 0x40, 0x37, 0x97, 0x7c, 0x26, 0x28, 0xb1, - 0x36, 0xf6, 0x07, 0x94, 0x08, 0x08, 0x78, 0x0a, 0x37, 0x82, 0xdf, 0x7f, 0x4f, 0xf0, 0xdd, 0xff, - 0xb4, 0x61, 0xbd, 0xd1, 0xb6, 0x97, 0x8d, 0xb7, 0xda, 0x62, 0x7b, 0x85, 0xc5, 0x6d, 0xe8, 0x14, - 0x71, 0xa8, 0x2e, 0x7b, 0x63, 0x7f, 0x84, 0xfc, 0x57, 0x71, 0x28, 0x31, 0x97, 0x3c, 0xe2, 0x18, - 0x3e, 0x75, 0xde, 0x97, 0x10, 0xdf, 0x87, 0x3b, 0x75, 0x22, 0x1f, 0x1e, 0x1e, 0x1f, 0x27, 0xfe, - 0x45, 0xd5, 0xe7, 0x96, 0xb1, 0x18, 0x53, 0xc3, 0x8d, 0x0a, 0xf2, 0x69, 0x4b, 0x8d, 0xb7, 0xef, - 0x42, 0xd7, 0xc7, 0x71, 0x43, 0x51, 0xd2, 0x09, 0x65, 0xcc, 0x9f, 0xa7, 0x2d, 0x4f, 0xf1, 0xd9, - 0xc7, 0xd0, 0x09, 0x8a, 0x79, 0xaa, 0x63, 0xb5, 0x81, 0x72, 0xf5, 0x00, 0x78, 0xda, 0xf2, 0x88, - 0x8b, 0x52, 0x51, 0xc2, 0x83, 0xf1, 0xa0, 0x96, 0xaa, 0xe7, 0x02, 0x4a, 0x21, 0x17, 0xa5, 0xb0, - 0xc2, 0xa8, 0xda, 0xb4, 0x54, 0xdd, 0xec, 0x50, 0x0a, 0xb9, 0x8f, 0xfb, 0xd0, 0xcb, 0x55, 0x22, - 0xff, 0x04, 0x6e, 0x37, 0xa2, 0x7f, 0x1c, 0xe6, 0x14, 0x2a, 0xc5, 0x1e, 0x5b, 0xab, 0x66, 0x6b, - 0xb9, 0x7f, 0x02, 0x40, 0x67, 0x7a, 0x92, 0x65, 0x49, 0x56, 0xce, 0x78, 0xab, 0x9a, 0xf1, 0xee, - 0xb7, 0x61, 0x80, 0x67, 0x79, 0x07, 0x1b, 0x0f, 0xb1, 0x8a, 0x9d, 0xc2, 0x88, 0xbc, 0x7f, 0x71, - 0xbc, 0x42, 0x82, 0xed, 0xc3, 0xa6, 0x1a, 0xb4, 0x2a, 0x9d, 0x9f, 0x27, 0x79, 0x48, 0xe3, 0x42, - 0x15, 0xd6, 0x52, 0x1e, 0x36, 0x74, 0x81, 0xea, 0x4e, 0x5f, 0x1c, 0x97, 0xd3, 0xaf, 0xa4, 0xdd, - 0x1f, 0xc2, 0x00, 0x2d, 0x2a, 0x73, 0x3b, 0xd0, 0x23, 0x46, 0x19, 0x07, 0xa7, 0x0a, 0xa7, 0x76, - 0xc8, 0xd3, 0x7c, 0xf7, 0xb7, 0x16, 0x0c, 0x55, 0xbb, 0x52, 0x3b, 0x3f, 0xb4, 0x5b, 0x6d, 0x37, - 0xb6, 0x97, 0xf5, 0x6e, 0x6a, 0xdc, 0x03, 0xa0, 0x86, 0xa3, 0x04, 0x3a, 0xf5, 0xf5, 0xd6, 0xa8, - 0x67, 0x48, 0xe0, 0xc5, 0xd4, 0xd4, 0x92, 0xd0, 0xfe, 0xa1, 0x0d, 0x23, 0x7d, 0xa5, 0x4a, 0xe4, - 0xff, 0x54, 0x76, 0xba, 0x32, 0x3a, 0x66, 0x65, 0x3c, 0x28, 0x2b, 0xa3, 0x5b, 0x1f, 0xa3, 0xce, - 0xa2, 0xba, 0x30, 0xee, 0xeb, 0xc2, 0xe8, 0x91, 0xd8, 0x7a, 0x59, 0x18, 0xa5, 0x94, 0xaa, 0x8b, - 0xfb, 0xba, 0x2e, 0xd6, 0x6a, 0xa1, 0x2a, 0xa5, 0xaa, 0xb2, 0xb8, 0xaf, 0xcb, 0xa2, 0x5f, 0x0b, - 0x55, 0xd7, 0x5c, 0x55, 0xc5, 0x1a, 0x74, 0xe9, 0x3a, 0xdd, 0xcf, 0xc0, 0x31, 0x43, 0x43, 0x35, - 0xf1, 0x40, 0x33, 0x1b, 0xa9, 0x60, 0x08, 0x79, 0x7a, 0xef, 0x6b, 0x58, 0x6f, 0x34, 0x15, 0x9c, - 0x74, 0x61, 0x7e, 0xc0, 0x63, 0x5f, 0x44, 0xd5, 0xa7, 0xa6, 0x81, 0x18, 0x49, 0xd6, 0xae, 0x35, - 0x6b, 0x15, 0x8d, 0x24, 0x33, 0x3e, 0x18, 0xed, 0xc6, 0x07, 0xe3, 0x3f, 0x2d, 0x18, 0x99, 0x1b, - 0xf0, 0x9b, 0xf3, 0x49, 0x96, 0x1d, 0x24, 0x81, 0xba, 0xcd, 0xae, 0x57, 0x92, 0x98, 0xfa, 0xb8, - 0x8c, 0x78, 0x9e, 0xeb, 0x0c, 0xac, 0x68, 0xcd, 0x3b, 0xf5, 0x93, 0xb4, 0x7c, 0x02, 0x54, 0xb4, - 0xe6, 0x1d, 0x8b, 0x4b, 0x11, 0xe9, 0x51, 0x53, 0xd1, 0x68, 0xed, 0x99, 0xc8, 0x73, 0x4c, 0x13, - 0xd5, 0x21, 0x4b, 0x12, 0x77, 0x79, 0xfc, 0xea, 0x80, 0x17, 0xb9, 0xd0, 0xdf, 0x2a, 0x15, 0x8d, - 0x61, 0xc1, 0xa7, 0x0a, 0xcf, 0x92, 0x22, 0x2e, 0xbf, 0x50, 0x0c, 0xc4, 0xbd, 0x82, 0xdb, 0xcf, - 0x8b, 0x6c, 0x26, 0x28, 0x89, 0xcb, 0x97, 0xcf, 0x16, 0xf4, 0xc3, 0x98, 0xfb, 0x32, 0xbc, 0x14, - 0x3a, 0x92, 0x15, 0x8d, 0xf9, 0x2b, 0xc3, 0xb9, 0xd0, 0x9f, 0x68, 0xb4, 0x46, 0xf9, 0xb3, 0x30, - 0x12, 0x94, 0xd7, 0xfa, 0x48, 0x25, 0x4d, 0x25, 0xaa, 0xa6, 0xab, 0x7e, 0xd7, 0x28, 0xca, 0xfd, - 0x8b, 0x05, 0x5b, 0x27, 0xa9, 0xc8, 0xb8, 0x14, 0xea, 0x2d, 0x75, 0xea, 0x9f, 0x8b, 0x39, 0x2f, - 0x5d, 0xb8, 0x07, 0xed, 0x24, 0x25, 0xe3, 0x3a, 0xdf, 0x15, 0xfb, 0x24, 0xf5, 0xda, 0x49, 0x4a, - 0x4e, 0xf0, 0xfc, 0x42, 0xc7, 0x96, 0xd6, 0x2b, 0x1f, 0x56, 0x5b, 0xd0, 0x0f, 0xb8, 0xe4, 0x53, - 0x9e, 0x8b, 0x32, 0xa6, 0x25, 0x4d, 0x6f, 0x10, 0x3e, 0x8d, 0xca, 0x88, 0x2a, 0x82, 0x34, 0x91, - 0x35, 0x1d, 0x4d, 0x4d, 0xb9, 0x12, 0xd6, 0xbf, 0x7c, 0xa8, 0x93, 0xf1, 0x99, 0x90, 0x9c, 0x6d, - 0x19, 0x4e, 0x02, 0x3a, 0x89, 0x1c, 0xed, 0xe2, 0x7b, 0x6b, 0xba, 0x6c, 0x04, 0xb6, 0xd1, 0x08, - 0xca, 0x73, 0x75, 0x28, 0xf1, 0x68, 0xed, 0x7e, 0x02, 0x9b, 0x3a, 0x4e, 0x5f, 0x3e, 0x44, 0xab, - 0x2b, 0x23, 0xa4, 0xd8, 0xca, 0xbc, 0xfb, 0x37, 0x0b, 0xee, 0x5e, 0xdb, 0xf6, 0xc1, 0x0f, 0xc7, - 0x4f, 0xa1, 0x83, 0x8f, 0x8d, 0xb1, 0x4d, 0x05, 0x73, 0x1f, 0x6d, 0x2c, 0x55, 0xb9, 0x87, 0xc4, - 0x93, 0x58, 0x66, 0x0b, 0x8f, 0x36, 0x6c, 0xfd, 0x1c, 0x06, 0x15, 0x84, 0x7a, 0x2f, 0xc4, 0xa2, - 0xec, 0x89, 0x17, 0x62, 0x81, 0x13, 0xfb, 0x92, 0x47, 0x85, 0x0a, 0x8d, 0x1e, 0x7b, 0x8d, 0xc0, - 0x7a, 0x8a, 0xff, 0x59, 0xfb, 0x47, 0x96, 0xfb, 0x1b, 0x18, 0x3f, 0xe5, 0x71, 0x10, 0xe9, 0x2c, - 0x51, 0xa5, 0xaa, 0x43, 0xf0, 0x2d, 0x23, 0x04, 0x43, 0xd4, 0x42, 0xdc, 0x77, 0xe4, 0xc8, 0x3d, - 0x18, 0x4c, 0xcb, 0x21, 0xa5, 0x03, 0x5f, 0x03, 0xb8, 0x23, 0x7f, 0x1d, 0xe5, 0xfa, 0x91, 0x43, - 0x6b, 0xf7, 0x2e, 0xdc, 0x39, 0x12, 0x52, 0xd9, 0x3e, 0x38, 0x9b, 0x69, 0xcb, 0xee, 0x0e, 0x6c, - 0x36, 0x61, 0x1d, 0x5c, 0x07, 0x6c, 0xff, 0xac, 0x1a, 0x00, 0xfe, 0xd9, 0x6c, 0xf7, 0xd7, 0xd0, - 0x53, 0x59, 0xc1, 0xd6, 0x61, 0xf0, 0x45, 0x7c, 0xc9, 0xa3, 0x30, 0x38, 0x49, 0x9d, 0x16, 0xeb, - 0x43, 0xe7, 0x54, 0x26, 0xa9, 0x63, 0xb1, 0x01, 0x74, 0x9f, 0x63, 0xb1, 0x3a, 0x6d, 0x06, 0xd0, - 0xc3, 0x7e, 0x36, 0x17, 0x8e, 0x8d, 0xf0, 0xa9, 0xe4, 0x99, 0x74, 0x3a, 0x08, 0xbf, 0x4a, 0x03, - 0x2e, 0x85, 0xd3, 0x65, 0x1b, 0x00, 0x3f, 0x2b, 0x64, 0xa2, 0xc5, 0x7a, 0xbb, 0xaf, 0x49, 0x6c, - 0x86, 0xb6, 0x47, 0x5a, 0x3f, 0xd1, 0x4e, 0x8b, 0xad, 0x81, 0xfd, 0x4b, 0x71, 0xe5, 0x58, 0x6c, - 0x08, 0x6b, 0x5e, 0x11, 0xe3, 0x73, 0x58, 0xd9, 0x20, 0x73, 0x81, 0x63, 0x23, 0x03, 0x9d, 0x48, - 0x45, 0xe0, 0x74, 0xd8, 0x08, 0xfa, 0x9f, 0xeb, 0xf7, 0xad, 0xd3, 0x45, 0x16, 0x8a, 0xe1, 0x9e, - 0x1e, 0xb2, 0xc8, 0x20, 0x52, 0x6b, 0xbb, 0x27, 0xd0, 0x2f, 0xc7, 0x0f, 0xbb, 0x05, 0x43, 0x6d, - 0x15, 0x21, 0xa7, 0x85, 0x6e, 0xd3, 0x90, 0x71, 0x2c, 0x3c, 0x22, 0x0e, 0x12, 0xa7, 0x8d, 0x2b, - 0x9c, 0x16, 0x8e, 0x4d, 0xc7, 0x5e, 0xc4, 0xbe, 0xd3, 0x41, 0x41, 0xea, 0x3a, 0x4e, 0xb0, 0xfb, - 0x0c, 0xd6, 0x68, 0x79, 0x82, 0xd7, 0xb6, 0xa1, 0xf5, 0x69, 0xc4, 0x69, 0x61, 0xe4, 0xd0, 0x4b, - 0x25, 0x6d, 0x61, 0x04, 0xe8, 0x00, 0x8a, 0x6e, 0xa3, 0x0b, 0x2a, 0x1a, 0x0a, 0xb0, 0xd1, 0xbf, - 0xb2, 0x5d, 0xb0, 0x3b, 0x70, 0xab, 0x8c, 0x8a, 0x86, 0x94, 0xc2, 0x23, 0x21, 0x15, 0xe0, 0x58, - 0xa4, 0xbf, 0x22, 0xdb, 0x18, 0x48, 0x4f, 0xcc, 0x93, 0x4b, 0xa1, 0x11, 0x7b, 0xf7, 0x11, 0xf4, - 0xcb, 0xea, 0x32, 0x14, 0x96, 0x50, 0xa5, 0x50, 0x01, 0x8e, 0x55, 0x6b, 0xd0, 0x48, 0x7b, 0xf7, - 0x11, 0xcd, 0x0a, 0x4c, 0x4e, 0xe3, 0x84, 0x1a, 0xd1, 0xc9, 0x70, 0x11, 0xa6, 0xfa, 0xaa, 0x44, - 0x1a, 0x71, 0xbf, 0x4a, 0x87, 0x4b, 0x91, 0x49, 0xc7, 0xde, 0xff, 0xca, 0x86, 0x9e, 0x4a, 0x38, - 0xf6, 0x08, 0x86, 0xc6, 0x2f, 0x21, 0xf6, 0x11, 0xa6, 0xfe, 0xcd, 0x1f, 0x58, 0x5b, 0xdf, 0xb8, - 0x81, 0xab, 0x2c, 0x75, 0x5b, 0xec, 0xa7, 0x00, 0x75, 0xdb, 0x67, 0x77, 0x69, 0x16, 0x5e, 0x1f, - 0x03, 0x5b, 0x63, 0xfa, 0x60, 0x58, 0xf2, 0xbb, 0xcb, 0x6d, 0xb1, 0x5f, 0xc0, 0xba, 0xee, 0x05, - 0x2a, 0x48, 0x6c, 0x62, 0xb4, 0x87, 0x25, 0x0d, 0xfd, 0x9d, 0xca, 0x3e, 0xaf, 0x94, 0xa9, 0x78, - 0xb1, 0xf1, 0x92, 0x5e, 0xa3, 0xd4, 0x7c, 0x73, 0x65, 0x17, 0x72, 0x5b, 0xec, 0x08, 0x86, 0xaa, - 0x57, 0xa8, 0xf9, 0x7c, 0x0f, 0x65, 0x57, 0x35, 0x8f, 0x77, 0x3a, 0x74, 0x00, 0x23, 0xb3, 0xbc, - 0x19, 0x45, 0x72, 0x49, 0x1f, 0x50, 0x4a, 0x96, 0x75, 0x02, 0xb7, 0xf5, 0x78, 0xfc, 0x8f, 0x37, - 0x13, 0xeb, 0xeb, 0x37, 0x13, 0xeb, 0xdf, 0x6f, 0x26, 0xd6, 0xef, 0xde, 0x4e, 0x5a, 0x5f, 0xbf, - 0x9d, 0xb4, 0xfe, 0xf5, 0x76, 0xd2, 0x9a, 0xf6, 0xe8, 0xd7, 0xe3, 0x0f, 0xfe, 0x1b, 0x00, 0x00, - 0xff, 0xff, 0x8b, 0xc0, 0x89, 0xd7, 0x8c, 0x14, 0x00, 0x00, + 0x3d, 0xf0, 0x1b, 0xe0, 0xc2, 0x81, 0x2a, 0x6e, 0x14, 0xd7, 0x77, 0xe4, 0x27, 0x00, 0xc7, 0x57, + 0x9c, 0x28, 0x4e, 0x54, 0xf2, 0x37, 0x38, 0x50, 0xdd, 0x33, 0x92, 0x46, 0xf6, 0x6e, 0x42, 0x0e, + 0xdc, 0xd4, 0x5f, 0xf7, 0xf4, 0xf4, 0xf4, 0x7c, 0xdd, 0xbd, 0xb3, 0xb0, 0x11, 0xcc, 0xaf, 0x92, + 0xec, 0x42, 0x64, 0x7b, 0x69, 0x96, 0xc8, 0x84, 0xb5, 0xd3, 0xa9, 0xbb, 0x03, 0xec, 0x45, 0x21, + 0xb2, 0xc5, 0xa9, 0xe4, 0xb2, 0xc8, 0x3d, 0xf1, 0xba, 0x10, 0xb9, 0x64, 0x0c, 0x3a, 0x31, 0x9f, + 0x8b, 0xb1, 0xb5, 0x6d, 0xed, 0x0c, 0x3c, 0xfa, 0x76, 0x53, 0xd8, 0x3c, 0x48, 0xe6, 0xf3, 0x24, + 0xfe, 0x15, 0xf9, 0xf0, 0x44, 0x9e, 0x26, 0x71, 0x2e, 0xd8, 0x47, 0xd0, 0xcb, 0x44, 0x5e, 0x44, + 0x92, 0xac, 0xfb, 0x9e, 0x96, 0x98, 0x03, 0xf6, 0x3c, 0x9f, 0x8d, 0xdb, 0xe4, 0x02, 0x3f, 0xd1, + 0x32, 0x4f, 0x8a, 0xcc, 0x17, 0x63, 0x9b, 0x40, 0x2d, 0x21, 0xae, 0xe2, 0x1a, 0x77, 0x14, 0xae, + 0x24, 0xf7, 0x2b, 0x0b, 0xee, 0x34, 0x82, 0xfb, 0xe0, 0x1d, 0x3f, 0x81, 0x91, 0xda, 0x43, 0x79, + 0xa0, 0x7d, 0x87, 0xfb, 0xce, 0x5e, 0x3a, 0xdd, 0x3b, 0x35, 0x70, 0xaf, 0x61, 0xc5, 0x3e, 0x85, + 0xf5, 0xbc, 0x98, 0xbe, 0xe4, 0xf9, 0x85, 0x5e, 0xd6, 0xd9, 0xb6, 0x77, 0x86, 0xfb, 0xb7, 0x69, + 0x99, 0xa9, 0xf0, 0x9a, 0x76, 0xee, 0x9f, 0x2d, 0x18, 0x1e, 0x9c, 0x0b, 0x5f, 0xcb, 0x18, 0x68, + 0xca, 0xf3, 0x5c, 0x04, 0x65, 0xa0, 0x4a, 0x62, 0x9b, 0xd0, 0x95, 0x89, 0xe4, 0x11, 0x85, 0xda, + 0xf5, 0x94, 0xc0, 0x26, 0x00, 0x79, 0xe1, 0xfb, 0x22, 0xcf, 0xcf, 0x8a, 0x88, 0x42, 0xed, 0x7a, + 0x06, 0x82, 0xde, 0xce, 0x78, 0x18, 0x89, 0x80, 0xd2, 0xd4, 0xf5, 0xb4, 0xc4, 0xc6, 0xb0, 0x76, + 0xc5, 0xb3, 0x38, 0x8c, 0x67, 0xe3, 0x2e, 0x29, 0x4a, 0x11, 0x57, 0x04, 0x42, 0xf2, 0x30, 0x1a, + 0xf7, 0xb6, 0xad, 0x9d, 0x91, 0xa7, 0x25, 0x77, 0x04, 0x70, 0x58, 0xcc, 0x53, 0x1d, 0xf5, 0x5f, + 0x2c, 0x80, 0xe3, 0x84, 0x07, 0x3a, 0xe8, 0x8f, 0x61, 0xfd, 0x2c, 0x8c, 0xc3, 0xfc, 0x5c, 0x04, + 0x8f, 0x17, 0x52, 0xe4, 0x14, 0xbb, 0xed, 0x35, 0x41, 0x0c, 0x96, 0xa2, 0x56, 0x26, 0x6d, 0x32, + 0x31, 0x10, 0xb6, 0x05, 0xfd, 0x34, 0x4b, 0x66, 0x99, 0xc8, 0x73, 0x7d, 0xdb, 0x95, 0x8c, 0x6b, + 0xe7, 0x42, 0xf2, 0xc7, 0x61, 0x1c, 0x25, 0x33, 0x7d, 0xe7, 0x06, 0xc2, 0x1e, 0xc0, 0x46, 0x2d, + 0x1d, 0xbd, 0xfc, 0xe2, 0x90, 0xce, 0x35, 0xf0, 0xae, 0xa1, 0xee, 0xef, 0x2d, 0x58, 0x3f, 0x3d, + 0xe7, 0x59, 0x10, 0xc6, 0xb3, 0xa3, 0x2c, 0x29, 0x52, 0x3c, 0xb0, 0xe4, 0xd9, 0x4c, 0x48, 0xcd, + 0x5c, 0x2d, 0x21, 0x9f, 0x0f, 0x0f, 0x8f, 0x31, 0x4e, 0x1b, 0xf9, 0x8c, 0xdf, 0xea, 0x9c, 0x59, + 0x2e, 0x8f, 0x13, 0x9f, 0xcb, 0x30, 0x89, 0x75, 0x98, 0x4d, 0x90, 0x38, 0xbb, 0x88, 0x7d, 0x4a, + 0xba, 0x4d, 0x9c, 0x25, 0x09, 0xcf, 0x57, 0xc4, 0x5a, 0xd3, 0x25, 0x4d, 0x25, 0xbb, 0x7f, 0xb2, + 0x01, 0x4e, 0x17, 0xb1, 0xaf, 0x13, 0xba, 0x0d, 0x43, 0x4a, 0xcc, 0x93, 0x4b, 0x11, 0xcb, 0x32, + 0x9d, 0x26, 0x84, 0xce, 0x48, 0x7c, 0x99, 0x96, 0xa9, 0xac, 0x64, 0x76, 0x0f, 0x06, 0x99, 0xf0, + 0x45, 0x2c, 0x51, 0x69, 0x93, 0xb2, 0x06, 0x98, 0x0b, 0xa3, 0x39, 0xcf, 0xa5, 0xc8, 0x1a, 0xc9, + 0x6c, 0x60, 0x6c, 0x17, 0x1c, 0x53, 0x3e, 0x92, 0x61, 0xa0, 0x13, 0x7a, 0x03, 0x47, 0x7f, 0x74, + 0x88, 0xd2, 0x5f, 0x4f, 0xf9, 0x33, 0x31, 0xf4, 0x67, 0xca, 0xe4, 0x6f, 0x4d, 0xf9, 0xbb, 0x8e, + 0xa3, 0xbf, 0x69, 0x94, 0xf8, 0x17, 0x61, 0x3c, 0xa3, 0x0b, 0xe8, 0x53, 0xaa, 0x1a, 0x18, 0xfb, + 0x31, 0x38, 0x45, 0x9c, 0x89, 0x3c, 0x89, 0x2e, 0x45, 0x40, 0xf7, 0x98, 0x8f, 0x07, 0x46, 0xc5, + 0x99, 0x37, 0xec, 0xdd, 0x30, 0x35, 0x6e, 0x08, 0x54, 0x91, 0xe9, 0x1b, 0x9a, 0x00, 0x4c, 0x29, + 0x90, 0x97, 0x8b, 0x54, 0x8c, 0x87, 0x8a, 0x65, 0x35, 0xe2, 0xfe, 0xd1, 0x82, 0x91, 0xd9, 0x04, + 0x8c, 0xf6, 0x64, 0xad, 0x68, 0x4f, 0x6d, 0xb3, 0x3d, 0xb1, 0xef, 0x55, 0x6d, 0x48, 0xb5, 0x15, + 0x8a, 0xf6, 0x79, 0x96, 0x60, 0xbd, 0x7a, 0xa4, 0xa8, 0x3a, 0xd3, 0x43, 0x18, 0x66, 0x22, 0xe2, + 0x8b, 0xaa, 0x9f, 0xa0, 0xfd, 0x2d, 0xb4, 0xf7, 0x6a, 0xd8, 0x33, 0x6d, 0xdc, 0xbf, 0xb5, 0x61, + 0x68, 0x28, 0x6f, 0xdc, 0xb4, 0xf5, 0x3f, 0xde, 0x74, 0x7b, 0xc5, 0x4d, 0x6f, 0x97, 0x21, 0x15, + 0xd3, 0xc3, 0x30, 0xd3, 0xe4, 0x37, 0xa1, 0xca, 0xa2, 0x41, 0x2d, 0x13, 0x62, 0x3b, 0x70, 0xcb, + 0x10, 0x0d, 0x62, 0x5d, 0x87, 0xd9, 0x1e, 0x30, 0x82, 0x0e, 0xb8, 0xf4, 0xcf, 0x5f, 0xa5, 0xcf, + 0x28, 0x1a, 0x62, 0x57, 0xdf, 0x5b, 0xa2, 0x61, 0xdf, 0x81, 0x6e, 0x2e, 0xf9, 0x4c, 0x10, 0xb1, + 0x36, 0xf6, 0x07, 0x44, 0x04, 0x04, 0x3c, 0x85, 0x1b, 0xc9, 0xef, 0xbf, 0x27, 0xf9, 0xee, 0x7f, + 0xda, 0xb0, 0xde, 0x68, 0xdb, 0xcb, 0xc6, 0x5b, 0xbd, 0x63, 0x7b, 0xc5, 0x8e, 0xdb, 0xd0, 0x29, + 0xe2, 0x50, 0x5d, 0xf6, 0xc6, 0xfe, 0x08, 0xf5, 0xaf, 0xe2, 0x50, 0x22, 0x97, 0x3c, 0xd2, 0x18, + 0x31, 0x75, 0xde, 0x47, 0x88, 0xef, 0xc3, 0x9d, 0x9a, 0xc8, 0x87, 0x87, 0xc7, 0xc7, 0x89, 0x7f, + 0x51, 0xf5, 0xb9, 0x65, 0x2a, 0xc6, 0xd4, 0x70, 0xa3, 0x82, 0x7c, 0xda, 0x52, 0xe3, 0xed, 0xbb, + 0xd0, 0xf5, 0x71, 0xdc, 0x50, 0x96, 0x34, 0xa1, 0x8c, 0xf9, 0xf3, 0xb4, 0xe5, 0x29, 0x3d, 0xfb, + 0x18, 0x3a, 0x41, 0x31, 0x4f, 0x75, 0xae, 0x36, 0xd0, 0xae, 0x1e, 0x00, 0x4f, 0x5b, 0x1e, 0x69, + 0xd1, 0x2a, 0x4a, 0x78, 0x30, 0x1e, 0xd4, 0x56, 0xf5, 0x5c, 0x40, 0x2b, 0xd4, 0xa2, 0x15, 0x56, + 0x18, 0x55, 0x9b, 0xb6, 0xaa, 0x9b, 0x1d, 0x5a, 0xa1, 0xf6, 0x71, 0x1f, 0x7a, 0xb9, 0x22, 0xf2, + 0x4f, 0xe0, 0x76, 0x23, 0xfb, 0xc7, 0x61, 0x4e, 0xa9, 0x52, 0xea, 0xb1, 0xb5, 0x6a, 0xb6, 0x96, + 0xeb, 0x27, 0x00, 0x74, 0xa6, 0x27, 0x59, 0x96, 0x64, 0xe5, 0x8c, 0xb7, 0xaa, 0x19, 0xef, 0x7e, + 0x1b, 0x06, 0x78, 0x96, 0x77, 0xa8, 0xf1, 0x10, 0xab, 0xd4, 0x29, 0x8c, 0x28, 0xfa, 0x17, 0xc7, + 0x2b, 0x2c, 0xd8, 0x3e, 0x6c, 0xaa, 0x41, 0xab, 0xe8, 0xfc, 0x3c, 0xc9, 0x43, 0x1a, 0x17, 0xaa, + 0xb0, 0x96, 0xea, 0xb0, 0xa1, 0x0b, 0x74, 0x77, 0xfa, 0xe2, 0xb8, 0x9c, 0x7e, 0xa5, 0xec, 0xfe, + 0x10, 0x06, 0xb8, 0xa3, 0xda, 0x6e, 0x07, 0x7a, 0xa4, 0x28, 0xf3, 0xe0, 0x54, 0xe9, 0xd4, 0x01, + 0x79, 0x5a, 0xef, 0xfe, 0xd6, 0x82, 0xa1, 0x6a, 0x57, 0x6a, 0xe5, 0x87, 0x76, 0xab, 0xed, 0xc6, + 0xf2, 0xb2, 0xde, 0x4d, 0x8f, 0x7b, 0x00, 0xd4, 0x70, 0x94, 0x41, 0xa7, 0xbe, 0xde, 0x1a, 0xf5, + 0x0c, 0x0b, 0xbc, 0x98, 0x5a, 0x5a, 0x92, 0xda, 0x3f, 0xb4, 0x61, 0xa4, 0xaf, 0x54, 0x99, 0xfc, + 0x9f, 0xca, 0x4e, 0x57, 0x46, 0xc7, 0xac, 0x8c, 0x07, 0x65, 0x65, 0x74, 0xeb, 0x63, 0xd4, 0x2c, + 0xaa, 0x0b, 0xe3, 0xbe, 0x2e, 0x8c, 0x1e, 0x99, 0xad, 0x97, 0x85, 0x51, 0x5a, 0xa9, 0xba, 0xb8, + 0xaf, 0xeb, 0x62, 0xad, 0x36, 0xaa, 0x28, 0x55, 0x95, 0xc5, 0x7d, 0x5d, 0x16, 0xfd, 0xda, 0xa8, + 0xba, 0xe6, 0xaa, 0x2a, 0xd6, 0xa0, 0x4b, 0xd7, 0xe9, 0x7e, 0x06, 0x8e, 0x99, 0x1a, 0xaa, 0x89, + 0x07, 0x5a, 0xd9, 0xa0, 0x82, 0x61, 0xe4, 0xe9, 0xb5, 0xaf, 0x61, 0xbd, 0xd1, 0x54, 0x70, 0xd2, + 0x85, 0xf9, 0x01, 0x8f, 0x7d, 0x11, 0x55, 0x3f, 0x35, 0x0d, 0xc4, 0x20, 0x59, 0xbb, 0xf6, 0xac, + 0x5d, 0x34, 0x48, 0x66, 0xfc, 0x60, 0xb4, 0x1b, 0x3f, 0x18, 0xff, 0x61, 0xc1, 0xc8, 0x5c, 0x80, + 0xbf, 0x39, 0x9f, 0x64, 0xd9, 0x41, 0x12, 0xa8, 0xdb, 0xec, 0x7a, 0xa5, 0x88, 0xd4, 0xc7, 0xcf, + 0x88, 0xe7, 0xb9, 0x66, 0x60, 0x25, 0x6b, 0xdd, 0xa9, 0x9f, 0xa4, 0xe5, 0x13, 0xa0, 0x92, 0xb5, + 0xee, 0x58, 0x5c, 0x8a, 0x48, 0x8f, 0x9a, 0x4a, 0xc6, 0xdd, 0x9e, 0x89, 0x3c, 0x47, 0x9a, 0xa8, + 0x0e, 0x59, 0x8a, 0xb8, 0xca, 0xe3, 0x57, 0x07, 0xbc, 0xc8, 0x85, 0xfe, 0xad, 0x52, 0xc9, 0x98, + 0x16, 0x7c, 0xaa, 0xf0, 0x2c, 0x29, 0xe2, 0xf2, 0x17, 0x8a, 0x81, 0xb8, 0x57, 0x70, 0xfb, 0x79, + 0x91, 0xcd, 0x04, 0x91, 0xb8, 0x7c, 0xf9, 0x6c, 0x41, 0x3f, 0x8c, 0xb9, 0x2f, 0xc3, 0x4b, 0xa1, + 0x33, 0x59, 0xc9, 0xc8, 0x5f, 0x19, 0xce, 0x85, 0xfe, 0x89, 0x46, 0xdf, 0x68, 0x7f, 0x16, 0x46, + 0x82, 0x78, 0xad, 0x8f, 0x54, 0xca, 0x54, 0xa2, 0x6a, 0xba, 0xea, 0x77, 0x8d, 0x92, 0xdc, 0x7f, + 0x59, 0xb0, 0x75, 0x92, 0x8a, 0x8c, 0x4b, 0xa1, 0xde, 0x52, 0xa7, 0xfe, 0xb9, 0x98, 0xf3, 0x32, + 0x84, 0x7b, 0xd0, 0x4e, 0x52, 0xda, 0x5c, 0xf3, 0x5d, 0xa9, 0x4f, 0x52, 0xaf, 0x9d, 0xa4, 0x14, + 0x04, 0xcf, 0x2f, 0x74, 0x6e, 0xe9, 0x7b, 0xe5, 0xc3, 0x6a, 0x0b, 0xfa, 0x01, 0x97, 0x7c, 0xca, + 0x73, 0x51, 0xe6, 0xb4, 0x94, 0xe9, 0x0d, 0xc2, 0xa7, 0x51, 0x99, 0x51, 0x25, 0x90, 0x27, 0xda, + 0x4d, 0x67, 0x53, 0x4b, 0x68, 0x7d, 0x16, 0x15, 0xf9, 0x39, 0xa5, 0xb1, 0xef, 0x29, 0x01, 0x63, + 0xa9, 0x38, 0xdf, 0x57, 0x14, 0x77, 0x25, 0xac, 0x7f, 0xf9, 0x50, 0xd3, 0xf6, 0x99, 0x90, 0x9c, + 0x6d, 0x19, 0xc7, 0x01, 0x3c, 0x0e, 0x6a, 0xf4, 0x61, 0xde, 0x5b, 0xfd, 0x65, 0xcb, 0xb0, 0x8d, + 0x96, 0x51, 0x66, 0xa0, 0x43, 0x14, 0xa5, 0x6f, 0xf7, 0x13, 0xd8, 0xd4, 0x19, 0xfd, 0xf2, 0x21, + 0xee, 0xba, 0x32, 0x97, 0x4a, 0xad, 0xb6, 0x77, 0xff, 0x6a, 0xc1, 0xdd, 0x6b, 0xcb, 0x3e, 0xf8, + 0x89, 0xf9, 0x29, 0x74, 0xf0, 0x59, 0x32, 0xb6, 0xa9, 0xb4, 0xee, 0xe3, 0x1e, 0x4b, 0x5d, 0xee, + 0xa1, 0xf0, 0x24, 0x96, 0xd9, 0xc2, 0xa3, 0x05, 0x5b, 0x3f, 0x87, 0x41, 0x05, 0xa1, 0xdf, 0x0b, + 0xb1, 0x28, 0xbb, 0xe7, 0x85, 0x58, 0xe0, 0x6c, 0xbf, 0xe4, 0x51, 0xa1, 0x52, 0xa3, 0x07, 0x64, + 0x23, 0xb1, 0x9e, 0xd2, 0x7f, 0xd6, 0xfe, 0x91, 0xe5, 0xfe, 0x06, 0xc6, 0x4f, 0x79, 0x1c, 0x44, + 0x9a, 0x4f, 0xaa, 0xa8, 0x75, 0x0a, 0xbe, 0x65, 0xa4, 0x60, 0x88, 0x5e, 0x48, 0xfb, 0x0e, 0x36, + 0xdd, 0x83, 0xc1, 0xb4, 0x1c, 0x67, 0x3a, 0xf1, 0x35, 0x40, 0x77, 0xfe, 0x3a, 0xca, 0xf5, 0x73, + 0x88, 0xbe, 0xdd, 0xbb, 0x70, 0xe7, 0x48, 0x48, 0xb5, 0xf7, 0xc1, 0xd9, 0x4c, 0xef, 0xec, 0xee, + 0xc0, 0x66, 0x13, 0xd6, 0xc9, 0x75, 0xc0, 0xf6, 0xcf, 0xaa, 0x51, 0xe1, 0x9f, 0xcd, 0x76, 0x7f, + 0x0d, 0x3d, 0xc5, 0x0a, 0xb6, 0x0e, 0x83, 0x2f, 0xe2, 0x4b, 0x1e, 0x85, 0xc1, 0x49, 0xea, 0xb4, + 0x58, 0x1f, 0x3a, 0xa7, 0x32, 0x49, 0x1d, 0x8b, 0x0d, 0xa0, 0xfb, 0x1c, 0xcb, 0xda, 0x69, 0x33, + 0x80, 0x1e, 0x76, 0xbe, 0xb9, 0x70, 0x6c, 0x84, 0x4f, 0x25, 0xcf, 0xa4, 0xd3, 0x41, 0xf8, 0x55, + 0x1a, 0x70, 0x29, 0x9c, 0x2e, 0xdb, 0x00, 0xf8, 0x59, 0x21, 0x13, 0x6d, 0xd6, 0xdb, 0x7d, 0x4d, + 0x66, 0x33, 0xdc, 0x7b, 0xa4, 0xfd, 0x93, 0xec, 0xb4, 0xd8, 0x1a, 0xd8, 0xbf, 0x14, 0x57, 0x8e, + 0xc5, 0x86, 0xb0, 0xe6, 0x15, 0x31, 0x3e, 0x9c, 0xd5, 0x1e, 0xb4, 0x5d, 0xe0, 0xd8, 0xa8, 0xc0, + 0x20, 0x52, 0x11, 0x38, 0x1d, 0x36, 0x82, 0xfe, 0xe7, 0xfa, 0x25, 0xec, 0x74, 0x51, 0x85, 0x66, + 0xb8, 0xa6, 0x87, 0x2a, 0xda, 0x10, 0xa5, 0xb5, 0xdd, 0x13, 0xe8, 0x97, 0x83, 0x8a, 0xdd, 0x82, + 0xa1, 0xde, 0x15, 0x21, 0xa7, 0x85, 0x61, 0xd3, 0x38, 0x72, 0x2c, 0x3c, 0x22, 0x8e, 0x1c, 0xa7, + 0x8d, 0x5f, 0x38, 0x57, 0x1c, 0x9b, 0x8e, 0xbd, 0x88, 0x7d, 0xa7, 0x83, 0x86, 0xd4, 0x9f, 0x9c, + 0x60, 0xf7, 0x19, 0xac, 0xd1, 0xe7, 0x09, 0x5e, 0xdb, 0x86, 0xf6, 0xa7, 0x11, 0xa7, 0x85, 0x99, + 0xc3, 0x28, 0x95, 0xb5, 0x85, 0x19, 0xa0, 0x03, 0x28, 0xb9, 0x8d, 0x21, 0xa8, 0x6c, 0x28, 0xc0, + 0xc6, 0xf8, 0xca, 0xc6, 0xc2, 0xee, 0xc0, 0xad, 0x32, 0x2b, 0x1a, 0x52, 0x0e, 0x8f, 0x84, 0x54, + 0x80, 0x63, 0x91, 0xff, 0x4a, 0x6c, 0x63, 0x22, 0x3d, 0x31, 0x4f, 0x2e, 0x85, 0x46, 0xec, 0xdd, + 0x47, 0xd0, 0x2f, 0xab, 0xcb, 0x70, 0x58, 0x42, 0x95, 0x43, 0x05, 0x38, 0x56, 0xed, 0x41, 0x23, + 0xed, 0xdd, 0x47, 0x34, 0x55, 0x90, 0x9c, 0xc6, 0x09, 0x35, 0xa2, 0xc9, 0x70, 0x11, 0xa6, 0xfa, + 0xaa, 0x44, 0x1a, 0x71, 0xbf, 0xa2, 0xc3, 0xa5, 0xc8, 0xa4, 0x63, 0xef, 0x7f, 0x65, 0x43, 0x4f, + 0x11, 0x8e, 0x3d, 0x82, 0xa1, 0xf1, 0xe7, 0x11, 0xfb, 0x08, 0xa9, 0x7f, 0xf3, 0xaf, 0xae, 0xad, + 0x6f, 0xdc, 0xc0, 0x15, 0x4b, 0xdd, 0x16, 0xfb, 0x29, 0x40, 0x3d, 0x20, 0xd8, 0x5d, 0x9a, 0x9a, + 0xd7, 0x07, 0xc6, 0xd6, 0x98, 0x7e, 0x5a, 0x2c, 0xf9, 0x63, 0xcc, 0x6d, 0xb1, 0x5f, 0xc0, 0xba, + 0xee, 0x05, 0x2a, 0x49, 0x6c, 0x62, 0xb4, 0x87, 0x25, 0xad, 0xff, 0x9d, 0xce, 0x3e, 0xaf, 0x9c, + 0xa9, 0x7c, 0xb1, 0xf1, 0x92, 0x5e, 0xa3, 0xdc, 0x7c, 0x73, 0x65, 0x17, 0x72, 0x5b, 0xec, 0x08, + 0x86, 0xaa, 0x57, 0xa8, 0x49, 0x7e, 0x0f, 0x6d, 0x57, 0x35, 0x8f, 0x77, 0x06, 0x74, 0x00, 0x23, + 0xb3, 0xbc, 0x19, 0x65, 0x72, 0x49, 0x1f, 0x50, 0x4e, 0x96, 0x75, 0x02, 0xb7, 0xf5, 0x78, 0xfc, + 0xf7, 0x37, 0x13, 0xeb, 0xeb, 0x37, 0x13, 0xeb, 0xdf, 0x6f, 0x26, 0xd6, 0xef, 0xde, 0x4e, 0x5a, + 0x5f, 0xbf, 0x9d, 0xb4, 0xfe, 0xf9, 0x76, 0xd2, 0x9a, 0xf6, 0xe8, 0x4f, 0xca, 0x1f, 0xfc, 0x37, + 0x00, 0x00, 0xff, 0xff, 0xd6, 0xde, 0xda, 0x8c, 0xb6, 0x14, 0x00, 0x00, } // Reference imports to suppress errors if they are not otherwise used. @@ -4346,6 +4363,26 @@ func (m *OperateWorkerSchemaRequest) MarshalToSizedBuffer(dAtA []byte) (int, err _ = i var l int _ = l + if m.Sync { + i-- + if m.Sync { + dAtA[i] = 1 + } else { + dAtA[i] = 0 + } + i-- + dAtA[i] = 0x40 + } + if m.Flush { + i-- + if m.Flush { + dAtA[i] = 1 + } else { + dAtA[i] = 0 + } + i-- + dAtA[i] = 0x38 + } if len(m.Schema) > 0 { i -= len(m.Schema) copy(dAtA[i:], m.Schema) @@ -5345,6 +5382,12 @@ func (m *OperateWorkerSchemaRequest) Size() (n int) { if l > 0 { n += 1 + l + sovDmworker(uint64(l)) } + if m.Flush { + n += 2 + } + if m.Sync { + n += 2 + } return n } @@ -9726,6 +9769,46 @@ func (m *OperateWorkerSchemaRequest) Unmarshal(dAtA []byte) error { } m.Schema = string(dAtA[iNdEx:postIndex]) iNdEx = postIndex + case 7: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Flush", wireType) + } + var v int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowDmworker + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + m.Flush = bool(v != 0) + case 8: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Sync", wireType) + } + var v int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowDmworker + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + m.Sync = bool(v != 0) default: iNdEx = preIndex skippy, err := skipDmworker(dAtA[iNdEx:]) diff --git a/dm/proto/dmmaster.proto b/dm/proto/dmmaster.proto index 6b2a7746ea..dbbd265093 100644 --- a/dm/proto/dmmaster.proto +++ b/dm/proto/dmmaster.proto @@ -366,6 +366,8 @@ message OperateSchemaRequest { string database = 4; // database name string table = 5; // table name string schema = 6; // schema content, a `CREATE TABLE` statement + bool flush = 7; // flush table info and checkpoint + bool sync = 8; // sync the table info to master } message OperateSchemaResponse { diff --git a/dm/proto/dmworker.proto b/dm/proto/dmworker.proto index f9965b3df5..3817ca05f9 100644 --- a/dm/proto/dmworker.proto +++ b/dm/proto/dmworker.proto @@ -323,6 +323,8 @@ message OperateWorkerSchemaRequest { string database = 4; // database name string table = 5; // table name string schema = 6; // schema content, a `CREATE TABLE` statement + bool flush = 7; // flush table info and checkpoint + bool sync = 8; // sync the table info to master } // copied `TaskMeta` from release-1.0 branch. diff --git a/pkg/shardddl/optimism/lock_test.go b/pkg/shardddl/optimism/lock_test.go index be244861fd..86760731fd 100644 --- a/pkg/shardddl/optimism/lock_test.go +++ b/pkg/shardddl/optimism/lock_test.go @@ -1018,12 +1018,12 @@ func (t *testLock) TestLockTrySyncConflictIntrusive(c *C) { DDLs1 = []string{"ALTER TABLE bar ADD COLUMN c1 TEXT"} DDLs2 = []string{"ALTER TABLE bar ADD COLUMN c1 DATETIME", "ALTER TABLE bar ADD COLUMN c2 INT"} DDLs3 = []string{"ALTER TABLE bar DROP COLUMN c2"} - DDLs4 = []string{"ALTER TABLE bar DROP COLUMN c1"} - ti0 = createTableInfo(c, p, se, tblID, `CREATE TABLE bar (id INT PRIMARY KEY)`) - ti1 = createTableInfo(c, p, se, tblID, `CREATE TABLE bar (id INT PRIMARY KEY, c1 TEXT)`) - ti2 = createTableInfo(c, p, se, tblID, `CREATE TABLE bar (id INT PRIMARY KEY, c1 DATETIME, c2 INT)`) - ti3 = createTableInfo(c, p, se, tblID, `CREATE TABLE bar (id INT PRIMARY KEY, c1 DATETIME)`) - ti4 = ti0 + DDLs4 = []string{"ALTER TABLE bar DROP COLUMN c1"} + ti0 = createTableInfo(c, p, se, tblID, `CREATE TABLE bar (id INT PRIMARY KEY)`) + ti1 = createTableInfo(c, p, se, tblID, `CREATE TABLE bar (id INT PRIMARY KEY, c1 TEXT)`) + ti2 = createTableInfo(c, p, se, tblID, `CREATE TABLE bar (id INT PRIMARY KEY, c1 DATETIME, c2 INT)`) + ti3 = createTableInfo(c, p, se, tblID, `CREATE TABLE bar (id INT PRIMARY KEY, c1 DATETIME)`) + ti4 = ti0 DDLs5 = []string{"ALTER TABLE bar ADD COLUMN c2 TEXT"} DDLs6 = []string{"ALTER TABLE bar ADD COLUMN c2 DATETIME", "ALTER TABLE bar ADD COLUMN c3 INT"} @@ -1099,7 +1099,9 @@ func (t *testLock) TestLockTrySyncConflictIntrusive(c *C) { // TrySync for the second table to drop the non-conflict column, the conflict should still exist. vers[source][db][tbls[1]]++ - DDLs, err = l.TrySync(source, db, tbls[1], DDLs3, []*model.TableInfo{ti3}, tts, vers[source][db][tbls[1]]) + info = NewInfo(task, source, db, tbls[1], downSchema, downTable, DDLs3, nil, []*model.TableInfo{ti3}) + info.Version = vers[source][db][tbls[1]] + DDLs, err = l.TrySync(info, tts) c.Assert(terror.ErrShardDDLOptimismTrySyncFail.Equal(err), IsTrue) c.Assert(DDLs, DeepEquals, []string{}) c.Assert(l.versions, DeepEquals, vers) @@ -1621,7 +1623,9 @@ func (t *testLock) TestAddDifferentFieldLenColumns(c *C) { // TrySync for the first table, no table has done the DDLs operation. vers[source][db][tbls[0]]++ - DDLs, err := l.TrySync(source, db, tbls[0], DDLs1, []*model.TableInfo{ti1}, tts, vers[source][db][tbls[0]]) + info := NewInfo(task, source, db, tbls[0], downSchema, downTable, DDLs1, nil, []*model.TableInfo{ti1}) + info.Version = vers[source][db][tbls[0]] + DDLs, err := l.TrySync(info, tts) c.Assert(err, IsNil) c.Assert(DDLs, DeepEquals, DDLs1) c.Assert(l.versions, DeepEquals, vers) @@ -1630,7 +1634,9 @@ func (t *testLock) TestAddDifferentFieldLenColumns(c *C) { // TrySync for the second table, add a table with a larger field length vers[source][db][tbls[1]]++ - DDLs, err = l.TrySync(source, db, tbls[1], DDLs2, []*model.TableInfo{ti2}, tts, vers[source][db][tbls[1]]) + info = NewInfo(task, source, db, tbls[1], downSchema, downTable, DDLs2, nil, []*model.TableInfo{ti2}) + info.Version = vers[source][db][tbls[0]] + DDLs, err = l.TrySync(info, tts) c.Assert(err, ErrorMatches, ".*add columns with different field lengths.*") c.Assert(DDLs, DeepEquals, DDLs2) c.Assert(l.versions, DeepEquals, vers) @@ -1640,7 +1646,9 @@ func (t *testLock) TestAddDifferentFieldLenColumns(c *C) { // TrySync for the first table, no table has done the DDLs operation. vers[source][db][tbls[0]]-- - DDLs, err = l.TrySync(source, db, tbls[1], DDLs2, []*model.TableInfo{ti2}, tts, vers[source][db][tbls[1]]) + info = NewInfo(task, source, db, tbls[1], downSchema, downTable, DDLs2, nil, []*model.TableInfo{ti2}) + info.Version = vers[source][db][tbls[1]] + DDLs, err = l.TrySync(info, tts) c.Assert(err, IsNil) c.Assert(DDLs, DeepEquals, DDLs2) c.Assert(l.versions, DeepEquals, vers) @@ -1649,7 +1657,9 @@ func (t *testLock) TestAddDifferentFieldLenColumns(c *C) { // TrySync for the second table, add a table with a smaller field length vers[source][db][tbls[0]]++ - DDLs, err = l.TrySync(source, db, tbls[0], DDLs1, []*model.TableInfo{ti1}, tts, vers[source][db][tbls[0]]) + info = NewInfo(task, source, db, tbls[0], downSchema, downTable, DDLs1, nil, []*model.TableInfo{ti1}) + info.Version = vers[source][db][tbls[0]] + DDLs, err = l.TrySync(info, tts) c.Assert(err, ErrorMatches, ".*add columns with different field lengths.*") c.Assert(DDLs, DeepEquals, DDLs1) c.Assert(l.versions, DeepEquals, vers) diff --git a/syncer/schema.go b/syncer/schema.go index ca37270c83..1ff981fa46 100644 --- a/syncer/schema.go +++ b/syncer/schema.go @@ -20,6 +20,7 @@ import ( "github.com/pingcap/parser/ast" "github.com/pingcap/parser/format" "github.com/pingcap/parser/model" + "go.uber.org/zap" "github.com/pingcap/dm/dm/config" "github.com/pingcap/dm/dm/pb" @@ -78,21 +79,32 @@ func (s *Syncer) OperateSchema(ctx context.Context, req *pb.OperateWorkerSchemaR return "", terror.ErrSchemaTrackerCannotCreateTable.Delegate(err, req.Database, req.Table) } + if !req.Flush && !req.Sync { + break + } + ti, err := s.schemaTracker.GetTable(req.Database, req.Table) if err != nil { return "", err } - err = s.checkpoint.FlushPointWithTableInfo(tcontext.NewContext(ctx, log.L()), req.Database, req.Table, ti) - if err != nil { - return "", err + if req.Flush { + log.L().Info("flush table info", zap.String("table info", newSQL)) + err = s.checkpoint.FlushPointWithTableInfo(tcontext.NewContext(ctx, log.L()), req.Database, req.Table, ti) + if err != nil { + return "", err + } } - if s.cfg.ShardMode == config.ShardOptimistic { + if req.Sync { + if s.cfg.ShardMode != config.ShardOptimistic { + log.L().Warn("ignore --sync flag", zap.String("shard mode", s.cfg.ShardMode)) + break + } downSchema, downTable := s.renameShardingSchema(req.Database, req.Table) info := s.optimist.ConstructInfo(req.Database, req.Table, downSchema, downTable, []string{""}, nil, []*model.TableInfo{ti}) info.IgnoreConflict = true - log.L().Info("resolve conflict with operateschema") + log.L().Info("sync info with operateschema", zap.Stringer("info", info)) _, err = s.optimist.PutInfo(info) if err != nil { return "", err diff --git a/tests/shardddl1/run.sh b/tests/shardddl1/run.sh index ad052bb65e..0f4497ee39 100644 --- a/tests/shardddl1/run.sh +++ b/tests/shardddl1/run.sh @@ -507,10 +507,10 @@ function DM_RENAME_COLUMN_OPTIMISTIC_CASE() { # TODO: support set schema automatically base on upstream schema echo 'CREATE TABLE `tb1` ( `c` int NOT NULL, `b` varchar(10) DEFAULT NULL, PRIMARY KEY (`c`)) ENGINE=InnoDB DEFAULT CHARSET=latin1' > ${WORK_DIR}/schema1.sql run_dm_ctl $WORK_DIR "127.0.0.1:$MASTER_PORT" \ - "operate-schema set -s mysql-replica-01 test -d ${shardddl1} -t ${tb1} ${WORK_DIR}/schema1.sql" \ + "operate-schema set -s mysql-replica-01 test -d ${shardddl1} -t ${tb1} ${WORK_DIR}/schema1.sql --flush --sync" \ "\"result\": true" 2 run_dm_ctl $WORK_DIR "127.0.0.1:$MASTER_PORT" \ - "operate-schema set -s mysql-replica-02 test -d ${shardddl1} -t ${tb1} ${WORK_DIR}/schema1.sql" \ + "operate-schema set -s mysql-replica-02 test -d ${shardddl1} -t ${tb1} ${WORK_DIR}/schema1.sql --flush --sync" \ "\"result\": true" 2 # fourth, resume-task @@ -530,7 +530,7 @@ function DM_RENAME_COLUMN_OPTIMISTIC_CASE() { # This may only work for a "rename ddl" echo 'CREATE TABLE `tb2` ( `c` int NOT NULL, `b` varchar(10) DEFAULT NULL, PRIMARY KEY (`c`)) ENGINE=InnoDB DEFAULT CHARSET=latin1' > ${WORK_DIR}/schema2.sql run_dm_ctl $WORK_DIR "127.0.0.1:$MASTER_PORT" \ - "operate-schema set -s mysql-replica-02 test -d ${shardddl1} -t ${tb2} ${WORK_DIR}/schema2.sql" \ + "operate-schema set -s mysql-replica-02 test -d ${shardddl1} -t ${tb2} ${WORK_DIR}/schema2.sql --flush --sync" \ "\"result\": true" 2 run_dm_ctl $WORK_DIR "127.0.0.1:$MASTER_PORT" \ @@ -591,16 +591,16 @@ function DM_RENAME_COLUMN_OPTIMISTIC() { function run() { init_cluster init_database - start=1 - end=35 - except=(024 025 029) - for i in $(seq -f "%03g" ${start} ${end}); do - if [[ ${except[@]} =~ $i ]]; then - continue - fi - DM_${i} - sleep 1 - done +# start=1 +# end=35 +# except=(024 025 029) +# for i in $(seq -f "%03g" ${start} ${end}); do +# if [[ ${except[@]} =~ $i ]]; then +# continue +# fi +# DM_${i} +# sleep 1 +# done DM_RENAME_COLUMN_OPTIMISTIC } From 0fe81219c521483c2cfcc73dfb75e702379c897e Mon Sep 17 00:00:00 2001 From: gmhdbjd Date: Fri, 12 Mar 2021 13:47:40 +0800 Subject: [PATCH 06/17] add flag for command --- dm/ctl/master/operate_schema.go | 2 +- dm/master/shardddl/optimist.go | 2 +- go.sum | 245 -------------------------------- pkg/shardddl/optimism/lock.go | 9 +- tests/shardddl1/run.sh | 20 +-- 5 files changed, 15 insertions(+), 263 deletions(-) diff --git a/dm/ctl/master/operate_schema.go b/dm/ctl/master/operate_schema.go index ba99e84145..698c2ca290 100644 --- a/dm/ctl/master/operate_schema.go +++ b/dm/ctl/master/operate_schema.go @@ -34,7 +34,7 @@ func NewOperateSchemaCmd() *cobra.Command { cmd.Flags().StringP("database", "d", "", "database name of the table") cmd.Flags().StringP("table", "t", "", "table name") cmd.Flags().Bool("flush", false, "flush the table info and checkpoint imediately") - cmd.Flags().Bool("sync", false, "sync the table info to master to resolve shard ddl lock, only for optimistic mode now.") + cmd.Flags().Bool("sync", false, "sync the table info to master to resolve shard ddl lock, only for optimistic mode now") return cmd } diff --git a/dm/master/shardddl/optimist.go b/dm/master/shardddl/optimist.go index c5ad3c7b07..7cd7eca1c5 100644 --- a/dm/master/shardddl/optimist.go +++ b/dm/master/shardddl/optimist.go @@ -391,7 +391,7 @@ func (o *Optimist) handleInfo(ctx context.Context, infoCh <-chan optimism.Info) if !ok { return } - o.logger.Info("receive a shard DDL info", zap.Stringer("info", info), zap.Bool("is deleted", info.IsDeleted), zap.Bool("ignore conflict", info.IgnoreConflict)) + o.logger.Info("receive a shard DDL info", zap.Stringer("info", info), zap.Bool("is deleted", info.IsDeleted)) // avoid new ddl added while previous ddl resolved and remove lock // change lock granularity if needed diff --git a/go.sum b/go.sum index de74ec2645..01dc426947 100644 --- a/go.sum +++ b/go.sum @@ -7,31 +7,26 @@ cloud.google.com/go v0.44.2/go.mod h1:60680Gw3Yr4ikxnPRS/oxxkBccT6SA1yMk63TGekxK cloud.google.com/go v0.45.1/go.mod h1:RpBamKRgapWJb87xiFSdk4g1CME7QZg3uwTez+TSTjc= cloud.google.com/go v0.46.3/go.mod h1:a6bKKbmY7er1mI7TEI4lsAkts/mkhTSZK8w33B4RAg0= cloud.google.com/go v0.49.0/go.mod h1:hGvAdzcWNbyuxS3nWhD7H2cIJxjRRTRLQVB0bdputVY= -cloud.google.com/go v0.50.0 h1:0E3eE8MX426vUOs7aHfI7aN1BrIzzzf4ccKCSfSjGmc= cloud.google.com/go v0.50.0/go.mod h1:r9sluTvynVuxRIOHXQEHMFffphuXHOMZMycpNR5e6To= cloud.google.com/go v0.51.0/go.mod h1:hWtGJ6gnXH+KgDv+V0zFGDvpi07n3z8ZNj3T1RW0Gcw= cloud.google.com/go v0.52.0/go.mod h1:pXajvRH/6o3+F9jDHZWQ5PbGhn+o8w9qiu/CffaVdO4= cloud.google.com/go v0.53.0 h1:MZQCQQaRwOrAcuKjiHWHrgKykt4fZyuwF2dtiG3fGW8= cloud.google.com/go v0.53.0/go.mod h1:fp/UouUEsRkN6ryDKNW/Upv/JBKnv6WDthjR6+vze6M= cloud.google.com/go/bigquery v1.0.1/go.mod h1:i/xbL2UlR5RvWAURpBYZTtm/cXjCha9lbfbpx4poX+o= -cloud.google.com/go/bigquery v1.3.0 h1:sAbMqjY1PEQKZBWfbu6Y6bsupJ9c4QdHnzg/VvYTLcE= cloud.google.com/go/bigquery v1.3.0/go.mod h1:PjpwJnslEMmckchkHFfq+HTD2DmtT67aNFKH1/VBDHE= cloud.google.com/go/bigquery v1.4.0 h1:xE3CPsOgttP4ACBePh79zTKALtXwn/Edhcr16R5hMWU= cloud.google.com/go/bigquery v1.4.0/go.mod h1:S8dzgnTigyfTmLBfrtrhyYhwRxG72rYxvftPBK2Dvzc= cloud.google.com/go/bigtable v1.2.0/go.mod h1:JcVAOl45lrTmQfLj7T6TxyMzIN/3FGGcFm+2xVAli2o= -cloud.google.com/go/datastore v1.0.0 h1:Kt+gOPPp2LEPWp8CSfxhsM8ik9CcyE/gYu+0r+RnZvM= cloud.google.com/go/datastore v1.0.0/go.mod h1:LXYbyblFSglQ5pkeyhO+Qmw7ukd3C+pD7TKLgZqpHYE= cloud.google.com/go/datastore v1.1.0 h1:/May9ojXjRkPBNVrq+oWLqmWCkr4OU5uRY29bu0mRyQ= cloud.google.com/go/datastore v1.1.0/go.mod h1:umbIZjpQpHh4hmRpGhH4tLFup+FVzqBi1b3c64qFpCk= cloud.google.com/go/pubsub v1.0.1/go.mod h1:R0Gpsv3s54REJCy4fxDixWD93lHJMoZTyQ2kNxGRt3I= -cloud.google.com/go/pubsub v1.1.0 h1:9/vpR43S4aJaROxqQHQ3nH9lfyKKV0dC3vOmnw8ebQQ= cloud.google.com/go/pubsub v1.1.0/go.mod h1:EwwdRX2sKPjnvnqCa270oGRyludottCI76h+R3AArQw= cloud.google.com/go/pubsub v1.2.0 h1:Lpy6hKgdcl7a3WGSfJIFmxmcdjSpP6OmBEfcOv1Y680= cloud.google.com/go/pubsub v1.2.0/go.mod h1:jhfEVHT8odbXTkndysNHCcx0awwzvfOlguIAii9o8iA= cloud.google.com/go/spanner v1.1.0/go.mod h1:TzTaF9l2ZY2CIetNvVpUu6ZQy8YEOtzB6ICa5EwYjL0= cloud.google.com/go/storage v1.0.0/go.mod h1:IhtSnM/ZTZV8YYJWCY8RULGVqBDmpoyjwiyrjsg+URw= cloud.google.com/go/storage v1.4.0/go.mod h1:ZusYJWlOshgSBGbt6K3GnB3MT3H1xs2id9+TCl4fDBA= -cloud.google.com/go/storage v1.5.0 h1:RPUcBvDeYgQFMfQu1eBMq6piD1SXmLH+vK3qjewZPus= cloud.google.com/go/storage v1.5.0/go.mod h1:tpKbwo567HUNpVclU5sGELwQWBDZ8gh0ZeosJ0Rtdos= cloud.google.com/go/storage v1.6.0 h1:UDpwYIwla4jHGzZJaEJYx1tOejbgSoNqsAfHAUYe2r8= cloud.google.com/go/storage v1.6.0/go.mod h1:N7U0C8pVQ/+NIKOBQyamJIeKQKkZ+mxpohlUTyfDhBk= @@ -45,7 +40,6 @@ github.com/BurntSushi/toml v0.3.1/go.mod h1:xHWCNGjB5oqiDr8zfno3MHue2Ht5sIBksp03 github.com/BurntSushi/xgb v0.0.0-20160522181843-27f122750802/go.mod h1:IVnqGOEym/WlBOVXweHU+Q+/VP0lqqI8lqeDx9IjBqo= github.com/ClickHouse/clickhouse-go v1.3.13/go.mod h1:EaI/sW7Azgz9UATzd5ZdZHRUhHgv5+JMS9NSr2smCJI= github.com/DATA-DOG/go-sqlmock v1.3.3/go.mod h1:f/Ixk793poVmq4qj/V1dPUg2JEAKC73Q5eFN3EC/SaM= -github.com/DATA-DOG/go-sqlmock v1.4.1 h1:ThlnYciV1iM/V0OSF/dtkqWb6xo5qITT1TJBG1MRDJM= github.com/DATA-DOG/go-sqlmock v1.4.1/go.mod h1:f/Ixk793poVmq4qj/V1dPUg2JEAKC73Q5eFN3EC/SaM= github.com/DATA-DOG/go-sqlmock v1.5.0 h1:Shsta01QNfFxHCfpW6YH2STWB0MudeXXEWMr20OEh60= github.com/DATA-DOG/go-sqlmock v1.5.0/go.mod h1:f/Ixk793poVmq4qj/V1dPUg2JEAKC73Q5eFN3EC/SaM= @@ -55,7 +49,6 @@ github.com/HdrHistogram/hdrhistogram-go v0.9.0 h1:dpujRju0R4M/QZzcnR1LH1qm+TVG3U github.com/HdrHistogram/hdrhistogram-go v0.9.0/go.mod h1:nxrse8/Tzg2tg3DZcZjm6qEclQKK70g0KxO61gFFZD4= github.com/Jeffail/gabs/v2 v2.5.1 h1:ANfZYjpMlfTTKebycu4X1AgkVWumFVDYQl7JwOr4mDk= github.com/Jeffail/gabs/v2 v2.5.1/go.mod h1:xCn81vdHKxFUuWWAaD5jCTQDNPBMh5pPs9IJ+NcziBI= -github.com/KyleBanks/depth v1.2.1 h1:5h8fQADFrWtarTdtDudMmGsC7GPbOAu6RVB3ffsVFHc= github.com/KyleBanks/depth v1.2.1/go.mod h1:jzSb9d0L43HxTQfT+oSA1EEp2q+ne2uh6XgeJcm8brE= github.com/Masterminds/semver v1.4.2/go.mod h1:MB6lktGJrhw8PrUyiEoblNEGEQ+RzHPF078ddwwvV3Y= github.com/Masterminds/semver v1.5.0/go.mod h1:MB6lktGJrhw8PrUyiEoblNEGEQ+RzHPF078ddwwvV3Y= @@ -63,9 +56,7 @@ github.com/MichaelS11/go-cql-driver v0.0.0-20190914174813-cf3b3196aa43/go.mod h1 github.com/OneOfOne/xxhash v1.2.2 h1:KMrpdQIwFcEqXDklaen+P1axHaj9BSKzvpUUfnHldSE= github.com/OneOfOne/xxhash v1.2.2/go.mod h1:HSdplMjZKSmBqAxg5vPj2TmRDmfkzw+cTzAElWljhcU= github.com/PuerkitoBio/purell v1.1.0/go.mod h1:c11w/QuzBsJSee3cPx9rAFu61PvFxuPbtSwDGJws/X0= -github.com/PuerkitoBio/purell v1.1.1 h1:WEQqlqaGbrPkxLJWfBwQmfEAE1Z7ONdDLqrN38tNFfI= github.com/PuerkitoBio/purell v1.1.1/go.mod h1:c11w/QuzBsJSee3cPx9rAFu61PvFxuPbtSwDGJws/X0= -github.com/PuerkitoBio/urlesc v0.0.0-20170810143723-de5bf2ad4578 h1:d+Bc7a5rLufV/sSk/8dngufqelfh6jnri85riMAaF/M= github.com/PuerkitoBio/urlesc v0.0.0-20170810143723-de5bf2ad4578/go.mod h1:uGdkoq3SwY9Y+13GIhn11/XLaGBb4BfwItxLd5jeuXE= github.com/RoaringBitmap/roaring v0.4.21/go.mod h1:D0gp8kJQgE1A4LQ5wFLggQEyvDi06Mq5mKs52e1TwOo= github.com/SAP/go-hdb v0.14.1/go.mod h1:7fdQLVC2lER3urZLjZCm0AuMQfApof92n3aylBPEkMo= @@ -91,10 +82,8 @@ github.com/alecthomas/kong v0.2.1-0.20190708041108-0548c6b1afae/go.mod h1:+inYUS github.com/alecthomas/kong-hcl v0.1.8-0.20190615233001-b21fea9723c8/go.mod h1:MRgZdU3vrFd05IQ89AxUZ0aYdF39BYoNFa324SodPCA= github.com/alecthomas/repr v0.0.0-20180818092828-117648cd9897/go.mod h1:xTS7Pm1pD1mvyM075QCDSRqH6qRLXylzS24ZTpRiSzQ= github.com/alecthomas/repr v0.0.0-20181024024818-d37bc2a10ba1/go.mod h1:xTS7Pm1pD1mvyM075QCDSRqH6qRLXylzS24ZTpRiSzQ= -github.com/alecthomas/template v0.0.0-20160405071501-a0175ee3bccc h1:cAKDfWh5VpdgMhJosfJnn5/FoN2SRZ4p7fJNX58YPaU= github.com/alecthomas/template v0.0.0-20160405071501-a0175ee3bccc/go.mod h1:LOuyumcjzFXgccqObfd/Ljyb9UuFJ6TxHnclSeseNhc= github.com/alecthomas/template v0.0.0-20190718012654-fb15b899a751/go.mod h1:LOuyumcjzFXgccqObfd/Ljyb9UuFJ6TxHnclSeseNhc= -github.com/alecthomas/units v0.0.0-20151022065526-2efee857e7cf h1:qet1QNfXsQxTZqLG4oE62mJzwPIB8+Tee4RNCL9ulrY= github.com/alecthomas/units v0.0.0-20151022065526-2efee857e7cf/go.mod h1:ybxpYRFXyAe+OPACYpWeL0wqObRcbAqCMya13uyzqw0= github.com/alecthomas/units v0.0.0-20190717042225-c3de453c63f4/go.mod h1:ybxpYRFXyAe+OPACYpWeL0wqObRcbAqCMya13uyzqw0= github.com/alecthomas/units v0.0.0-20190924025748-f65c72e2690d/go.mod h1:rBZYJk541a8SKzHPHnH3zbiI+7dagKZ0cgpgrD7Fyho= @@ -112,16 +101,12 @@ github.com/appleboy/gofight/v2 v2.1.2/go.mod h1:frW+U1QZEdDgixycTj4CygQ48yLTUhpl github.com/armon/consul-api v0.0.0-20180202201655-eb2c6b5be1b6/go.mod h1:grANhF5doyWs3UAsr3K4I6qtAmlQcZDesFNEHPZAzj8= github.com/asaskevich/EventBus v0.0.0-20180315140547-d46933a94f05/go.mod h1:JS7hed4L1fj0hXcyEejnW57/7LCetXggd+vwrRnYeII= github.com/aws/aws-sdk-go v1.26.1/go.mod h1:KmX6BPdI08NWTb3/sm4ZGu5ShLoqVDhKgpiN924inxo= -github.com/aws/aws-sdk-go v1.30.24 h1:y3JPD51VuEmVqN3BEDVm4amGpDma2cKJcDPuAU1OR58= github.com/aws/aws-sdk-go v1.30.24/go.mod h1:5zCpMtNQVjRREroY7sYe8lOMRSxkhG6MZveU8YkpAk0= -github.com/aws/aws-sdk-go v1.32.4 h1:J2OMvipVB5dPIn+VH7L5rOqM4WoTsBxOqv+I06sjYOM= github.com/aws/aws-sdk-go v1.32.4/go.mod h1:5zCpMtNQVjRREroY7sYe8lOMRSxkhG6MZveU8YkpAk0= github.com/aws/aws-sdk-go v1.35.3 h1:r0puXncSaAfRt7Btml2swUo74Kao+vKhO3VLjwDjK54= github.com/aws/aws-sdk-go v1.35.3/go.mod h1:H7NKnBqNVzoTJpGfLrQkkD+ytBA93eiDYi/+8rV9s48= github.com/aybabtme/rgbterm v0.0.0-20170906152045-cc83f3b3ce59/go.mod h1:q/89r3U2H7sSsE2t6Kca0lfwTK8JdoNGS/yzM/4iH5I= -github.com/beorn7/perks v0.0.0-20180321164747-3a771d992973 h1:xJ4a3vCFaGF/jqvzLMYoU8P317H5OQ+Via4RmuPwCS0= github.com/beorn7/perks v0.0.0-20180321164747-3a771d992973/go.mod h1:Dwedo/Wpr24TaqPxmxbtue+5NUziq4I4S80YR8gNf3Q= -github.com/beorn7/perks v1.0.0 h1:HWo1m869IqiPhD389kmkxeTalrjNbbJTC8LXupb+sl0= github.com/beorn7/perks v1.0.0/go.mod h1:KWe93zE9D1o94FZ5RNwFwVgaQK1VOXiVxmqh+CedLV8= github.com/beorn7/perks v1.0.1 h1:VlbKKnNfV8bJzeqoa4cOKqO6bYr3WgKZxO8Z16+hsOM= github.com/beorn7/perks v1.0.1/go.mod h1:G2ZrVWU2WbWT9wwq4/hrbKbnv/1ERSJQ0ibhJ6rlkpw= @@ -173,7 +158,6 @@ github.com/cloudflare/golz4 v0.0.0-20150217214814-ef862a3cdc58/go.mod h1:EOBUe0h github.com/cockroachdb/apd v1.1.0/go.mod h1:8Sl8LxpKi29FqWXR16WEFZRNSz3SoPzUzeMeY4+DwBQ= github.com/cockroachdb/datadriven v0.0.0-20190809214429-80d97fb3cbaa h1:OaNxuTZr7kxeODyLWsRMC+OD03aFUH+mW6r2d+MWa5Y= github.com/cockroachdb/datadriven v0.0.0-20190809214429-80d97fb3cbaa/go.mod h1:zn76sxSg3SzpJ0PPJaLDCu+Bu0Lg3sKTORVIj19EIF8= -github.com/codahale/hdrhistogram v0.0.0-20161010025455-3a0bb77429bd h1:qMd81Ts1T2OTKmB4acZcyKaMtRnY5Y44NuXGX2GFJ1w= github.com/codahale/hdrhistogram v0.0.0-20161010025455-3a0bb77429bd/go.mod h1:sE/e/2PUdi/liOCUjSTXgM1o87ZssimdTWN964YiIeI= github.com/codahale/hdrhistogram v0.9.0 h1:9GjrtRI+mLEFPtTfR/AZhcxp+Ii8NZYWq5104FbZQY0= github.com/codahale/hdrhistogram v0.9.0/go.mod h1:sE/e/2PUdi/liOCUjSTXgM1o87ZssimdTWN964YiIeI= @@ -186,16 +170,13 @@ github.com/coocood/freecache v1.1.1/go.mod h1:OKrEjkGVoxZhyWAJoeFi5BMLUJm2Tit0kp github.com/coocood/rtutil v0.0.0-20190304133409-c84515f646f2 h1:NnLfQ77q0G4k2Of2c1ceQ0ec6MkLQyDp+IGdVM0D8XM= github.com/coocood/rtutil v0.0.0-20190304133409-c84515f646f2/go.mod h1:7qG7YFnOALvsx6tKTNmQot8d7cGFXM9TidzvRFLWYwM= github.com/coreos/bbolt v1.3.2/go.mod h1:iRUV2dpdMOn7Bo10OQBFzIJO9kkE559Wcmn+qkEiiKk= -github.com/coreos/etcd v3.3.10+incompatible h1:jFneRYjIvLMLhDLCzuTuU4rSJUjRplcJQ7pD7MnhC04= github.com/coreos/etcd v3.3.10+incompatible/go.mod h1:uF7uidLiAD3TWHmW31ZFd/JWoc32PjwdhPthX9715RE= github.com/coreos/etcd v3.3.12+incompatible/go.mod h1:uF7uidLiAD3TWHmW31ZFd/JWoc32PjwdhPthX9715RE= github.com/coreos/go-etcd v2.0.0+incompatible/go.mod h1:Jez6KQU2B/sWsbdaef3ED8NzMklzPG4d5KIOhIy30Tk= -github.com/coreos/go-semver v0.2.0 h1:3Jm3tLmsgAYcjC+4Up7hJrFBPr+n7rAqYeSw/SZazuY= github.com/coreos/go-semver v0.2.0/go.mod h1:nnelYz7RCh+5ahJtPPxZlU+153eP4D4r3EedlOD2RNk= github.com/coreos/go-semver v0.3.0 h1:wkHLiw0WNATZnSG7epLsujiMCgPAc9xhjJ4tgnAxmfM= github.com/coreos/go-semver v0.3.0/go.mod h1:nnelYz7RCh+5ahJtPPxZlU+153eP4D4r3EedlOD2RNk= github.com/coreos/go-systemd v0.0.0-20180511133405-39ca1b05acc7/go.mod h1:F5haX7vjVVG0kc13fIWeqUViNPyEJxv/OmvnBo0Yme4= -github.com/coreos/go-systemd v0.0.0-20181031085051-9002847aa142 h1:3jFq2xL4ZajGK4aZY8jz+DAF0FHjI51BXjjSwCzS1Dk= github.com/coreos/go-systemd v0.0.0-20181031085051-9002847aa142/go.mod h1:F5haX7vjVVG0kc13fIWeqUViNPyEJxv/OmvnBo0Yme4= github.com/coreos/go-systemd v0.0.0-20190212144455-93d5ec2c7f76/go.mod h1:F5haX7vjVVG0kc13fIWeqUViNPyEJxv/OmvnBo0Yme4= github.com/coreos/go-systemd v0.0.0-20190321100706-95778dfbb74e/go.mod h1:F5haX7vjVVG0kc13fIWeqUViNPyEJxv/OmvnBo0Yme4= @@ -212,10 +193,8 @@ github.com/couchbase/gomemcached v0.0.0-20191205220529-2b26ed9d054e/go.mod h1:sr github.com/couchbase/goutils v0.0.0-20191018232750-b49639060d85/go.mod h1:BQwMFlJzDjFDG3DJUdU0KORxn88UlsOULuxLExMh3Hs= github.com/couchbase/moss v0.1.0/go.mod h1:9MaHIaRuy9pvLPUJxB8sh8OrLfyDczECVL37grCIubs= github.com/couchbase/vellum v1.0.1/go.mod h1:FcwrEivFpNi24R3jLOs3n+fs5RnuQnQqCLBJ1uAg1W4= -github.com/cpuguy83/go-md2man v1.0.10 h1:BSKMNlYxDvnunlTymqtgONjNnaRV1sTpcovwwjF22jk= github.com/cpuguy83/go-md2man v1.0.10/go.mod h1:SmD6nW6nTyfqj6ABTjUi3V3JVMnlJmwcJI5acqYI6dE= github.com/cpuguy83/go-md2man/v2 v2.0.0-20190314233015-f79a8a8ca69d/go.mod h1:maD7wRr/U5Z6m/iR4s+kqSMx2CaBsrgA7czyZG/E6dU= -github.com/cpuguy83/go-md2man/v2 v2.0.0 h1:EoUDS0afbrsXAZ9YQ9jdu/mZ2sXgT1/2yyNng4PGlyM= github.com/cpuguy83/go-md2man/v2 v2.0.0/go.mod h1:maD7wRr/U5Z6m/iR4s+kqSMx2CaBsrgA7czyZG/E6dU= github.com/creack/pty v1.1.7/go.mod h1:lj5s0c3V2DBrqTV7llrYr5NG6My20zk30Fl46Y7DoTY= github.com/creack/pty v1.1.9/go.mod h1:oKZEueFk5CKHvIhNR5MUki03XCEU+Q6VDXinZuGJ33E= @@ -277,7 +256,6 @@ github.com/facebookgo/subset v0.0.0-20200203212716-c811ad88dec4/go.mod h1:5tD+ne github.com/fatih/color v1.7.0/go.mod h1:Zm6kSWBoL9eyXnKyktHP6abPY2pDugNf5KwzbycvMj4= github.com/fatih/color v1.9.0 h1:8xPHl4/q1VyqGIPif1F+1V3Y3lSmrq01EabUW3CoW5s= github.com/fatih/color v1.9.0/go.mod h1:eQcE1qtQxscV5RaZvpXrrb8Drkc3/DdQ+uUYCNjL+zU= -github.com/fatih/structtag v1.2.0 h1:/OdNE99OxoI/PqaW/SuSK9uxxT3f/tcSZgon/ssNSx4= github.com/fatih/structtag v1.2.0/go.mod h1:mBJUNpUnHmRKrKlQQlmCrh5PuhftFbNv8Ys4/aAZl94= github.com/fogleman/gg v1.2.1-0.20190220221249-0403632d5b90/go.mod h1:R/bRT+9gY/C5z7JzPU0zXsXHKM4/ayA+zqcVNZzPa1k= github.com/fogleman/gg v1.3.0/go.mod h1:R/bRT+9gY/C5z7JzPU0zXsXHKM4/ayA+zqcVNZzPa1k= @@ -288,12 +266,10 @@ github.com/frankban/quicktest v1.11.1/go.mod h1:K+q6oSqb0W0Ininfk863uOk1lMy69l/P github.com/fsnotify/fsnotify v1.4.7 h1:IXs+QLmnXW2CcXuY+8Mzv/fWEsPGWxqefPtCP5CnV9I= github.com/fsnotify/fsnotify v1.4.7/go.mod h1:jwhsz4b93w/PPRr/qN1Yymfu8t87LnFCMoQvtojpjFo= github.com/fsouza/fake-gcs-server v1.15.0/go.mod h1:HNxAJ/+FY/XSsxuwz8iIYdp2GtMmPbJ8WQjjGMxd6Qk= -github.com/fsouza/fake-gcs-server v1.17.0 h1:OeH75kBZcZa3ZE+zz/mFdJ2btt9FgqfjI7gIh9+5fvk= github.com/fsouza/fake-gcs-server v1.17.0/go.mod h1:D1rTE4YCyHFNa99oyJJ5HyclvN/0uQR+pM/VdlL83bw= github.com/fsouza/fake-gcs-server v1.19.0 h1:XyaGOlqo+R5sjT03x2ymk0xepaQlgwhRLTT2IopW0zA= github.com/fsouza/fake-gcs-server v1.19.0/go.mod h1:JtXHY/QzHhtyIxsNfIuQ+XgHtRb5B/w8nqbL5O8zqo0= github.com/ghodss/yaml v1.0.0/go.mod h1:4dBDuWmgqj2HViK6kFavaiC9ZROes6MMH2rRYeMEF04= -github.com/ghodss/yaml v1.0.1-0.20190212211648-25d852aebe32 h1:Mn26/9ZMNWSw9C9ERFA1PUxfmGpolnw2v0bKOREu5ew= github.com/ghodss/yaml v1.0.1-0.20190212211648-25d852aebe32/go.mod h1:GIjDIg/heH5DOkXY3YJ/wNhfHsQHoXGjl8G8amsYQ1I= github.com/gibson042/canonicaljson-go v1.0.3/go.mod h1:DsLpJTThXyGNO+KZlI85C1/KDcImpP67k/RKVjcaEqo= github.com/gin-contrib/gzip v0.0.1/go.mod h1:fGBJBCdt6qCZuCAOwWuFhBB4OOq9EFqlo5dEaFhhu5w= @@ -323,31 +299,25 @@ github.com/go-ole/go-ole v1.2.4 h1:nNBDSCOigTSiarFpYE9J/KtEA1IOW4CNeqT9TQDqCxI= github.com/go-ole/go-ole v1.2.4/go.mod h1:XCwSNxSkXRo4vlyPy93sltvi/qJq0jqQhjqQNIwKuxM= github.com/go-openapi/jsonpointer v0.17.0/go.mod h1:cOnomiV+CVVwFLk0A/MExoFMjwdsUdVpsRhURCKh+3M= github.com/go-openapi/jsonpointer v0.19.2/go.mod h1:3akKfEdA7DF1sugOqz1dVQHBcuDBPKZGEoHC/NkiQRg= -github.com/go-openapi/jsonpointer v0.19.3 h1:gihV7YNZK1iK6Tgwwsxo2rJbD1GTbdm72325Bq8FI3w= github.com/go-openapi/jsonpointer v0.19.3/go.mod h1:Pl9vOtqEWErmShwVjC8pYs9cog34VGT37dQOVbmoatg= github.com/go-openapi/jsonreference v0.17.0/go.mod h1:g4xxGn04lDIRh0GJb5QlpE3HfopLOL6uZrK/VgnsK9I= github.com/go-openapi/jsonreference v0.19.0/go.mod h1:g4xxGn04lDIRh0GJb5QlpE3HfopLOL6uZrK/VgnsK9I= github.com/go-openapi/jsonreference v0.19.2/go.mod h1:jMjeRr2HHw6nAVajTXJ4eiUwohSTlpa0o73RUL1owJc= -github.com/go-openapi/jsonreference v0.19.3 h1:5cxNfTy0UVC3X8JL5ymxzyoUZmo8iZb+jeTWn7tUa8o= github.com/go-openapi/jsonreference v0.19.3/go.mod h1:rjx6GuL8TTa9VaixXglHmQmIL98+wF9xc8zWvFonSJ8= github.com/go-openapi/spec v0.19.0/go.mod h1:XkF/MOi14NmjsfZ8VtAKf8pIlbZzyoTvZsdfssdxcBI= github.com/go-openapi/spec v0.19.4/go.mod h1:FpwSN1ksY1eteniUU7X0N/BgJ7a4WvBFVA8Lj9mJglo= -github.com/go-openapi/spec v0.19.7 h1:0xWSeMd35y5avQAThZR2PkEuqSosoS5t6gDH4L8n11M= github.com/go-openapi/spec v0.19.7/go.mod h1:Hm2Jr4jv8G1ciIAo+frC/Ft+rR2kQDh8JHKHb3gWUSk= github.com/go-openapi/swag v0.17.0/go.mod h1:AByQ+nYG6gQg71GINrmuDXCPWdL640yX49/kXLo40Tg= github.com/go-openapi/swag v0.19.2/go.mod h1:POnQmlKehdgb5mhVOsnJFsivZCEZ/vjK9gh66Z9tfKk= github.com/go-openapi/swag v0.19.5/go.mod h1:POnQmlKehdgb5mhVOsnJFsivZCEZ/vjK9gh66Z9tfKk= -github.com/go-openapi/swag v0.19.8 h1:vfK6jLhs7OI4tAXkvkooviaE1JEPcw3mutyegLHHjmk= github.com/go-openapi/swag v0.19.8/go.mod h1:ao+8BpOPyKdpQz3AOJfbeEVpLmWAvlT1IfTe5McPyhY= github.com/go-playground/locales v0.12.1/go.mod h1:IUMDtCfWo/w/mtMfIE/IG2K+Ey3ygWanZIBtBW0W2TM= github.com/go-playground/locales v0.13.0/go.mod h1:taPMhCMXrRLJO55olJkUXHZBHCxTMfnGwq/HNwmWNS8= -github.com/go-playground/overalls v0.0.0-20180201144345-22ec1a223b7c h1:3bjbKXoj7jBYdHpQFbKL2546c4dtltTHzjo+5i4CHBU= github.com/go-playground/overalls v0.0.0-20180201144345-22ec1a223b7c/go.mod h1:UqxAgEOt89sCiXlrc/ycnx00LVvUO/eS8tMUkWX4R7w= github.com/go-playground/universal-translator v0.16.0/go.mod h1:1AnU7NaIRDWWzGEKwgtJRd2xk99HeFyHw3yid4rvQIY= github.com/go-playground/universal-translator v0.17.0/go.mod h1:UkSxE5sNxxRwHyU+Scu5vgOQjsIJAF8j9muTVoKLVtA= github.com/go-redis/redis v6.15.1+incompatible/go.mod h1:NAIEuMOZ/fxfXJIrKDQDz8wamY7mA7PouImQ2Jvg6kA= github.com/go-sql-driver/mysql v1.4.0/go.mod h1:zAC/RDZ24gD3HViQzih4MyKcchzm+sOG5ZlKdlhCg5w= -github.com/go-sql-driver/mysql v1.4.1 h1:g24URVg0OFbNUTx9qqY1IRZ9D9z3iPyi5zKhQZpNwpA= github.com/go-sql-driver/mysql v1.4.1/go.mod h1:zAC/RDZ24gD3HViQzih4MyKcchzm+sOG5ZlKdlhCg5w= github.com/go-sql-driver/mysql v1.5.0 h1:ozyZYNQW3x3HtqT1jira07DN2PArx2v7/mN66gGcHOs= github.com/go-sql-driver/mysql v1.5.0/go.mod h1:DCzpHaOWr8IXmIStZouvnhqoel9Qv2LBy8hT2VhHyBg= @@ -366,7 +336,6 @@ github.com/gogo/protobuf v0.0.0-20171007142547-342cbe0a0415/go.mod h1:r8qH/GZQm5 github.com/gogo/protobuf v0.0.0-20180717141946-636bf0302bc9/go.mod h1:r8qH/GZQm5c6nD/R0oafs1akxWv10x8SbQlK7atdtwQ= github.com/gogo/protobuf v1.1.1/go.mod h1:r8qH/GZQm5c6nD/R0oafs1akxWv10x8SbQlK7atdtwQ= github.com/gogo/protobuf v1.2.0/go.mod h1:r8qH/GZQm5c6nD/R0oafs1akxWv10x8SbQlK7atdtwQ= -github.com/gogo/protobuf v1.2.1 h1:/s5zKNz0uPFCZ5hddgPdo2TK2TVrUNMn0OOX8/aZMTE= github.com/gogo/protobuf v1.2.1/go.mod h1:hp+jE20tsWTFYpLwKvXlhS1hjn+gTNwPg2I6zVXpSg4= github.com/gogo/protobuf v1.3.1 h1:DqDEcV5aeaTmdFBePNpYsp3FlcVH/2ISVVM9Qf8PSls= github.com/gogo/protobuf v1.3.1/go.mod h1:SlYgWuQ5SjCEi6WLHjHCa1yvBfUnHcTbrrZtXPKa29o= @@ -377,38 +346,28 @@ github.com/golang/geo v0.0.0-20190916061304-5b978397cfec/go.mod h1:QZ0nwyI2jOfgR github.com/golang/glog v0.0.0-20160126235308-23def4e6c14b h1:VKtxabqXZkF25pY9ekfRL6a582T4P37/31XEstQ5p58= github.com/golang/glog v0.0.0-20160126235308-23def4e6c14b/go.mod h1:SBH7ygxi8pfUlaOkMMuAQtPIUF8ecWP5IEl/CR7VP2Q= github.com/golang/groupcache v0.0.0-20160516000752-02826c3e7903/go.mod h1:cIg4eruTrX1D+g88fzRXU5OdNfaM+9IcxsU14FzY7Hc= -github.com/golang/groupcache v0.0.0-20181024230925-c65c006176ff h1:kOkM9whyQYodu09SJ6W3NCsHG7crFaJILQ22Gozp3lg= github.com/golang/groupcache v0.0.0-20181024230925-c65c006176ff/go.mod h1:cIg4eruTrX1D+g88fzRXU5OdNfaM+9IcxsU14FzY7Hc= github.com/golang/groupcache v0.0.0-20190129154638-5b532d6fd5ef/go.mod h1:cIg4eruTrX1D+g88fzRXU5OdNfaM+9IcxsU14FzY7Hc= -github.com/golang/groupcache v0.0.0-20190702054246-869f871628b6 h1:ZgQEtGgCBiWRM39fZuwSd1LwSqqSW0hOdXCYYDX0R3I= github.com/golang/groupcache v0.0.0-20190702054246-869f871628b6/go.mod h1:cIg4eruTrX1D+g88fzRXU5OdNfaM+9IcxsU14FzY7Hc= github.com/golang/groupcache v0.0.0-20191027212112-611e8accdfc9/go.mod h1:cIg4eruTrX1D+g88fzRXU5OdNfaM+9IcxsU14FzY7Hc= -github.com/golang/groupcache v0.0.0-20191227052852-215e87163ea7 h1:5ZkaAPbicIKTF2I64qf5Fh8Aa83Q/dnOafMYV0OMwjA= github.com/golang/groupcache v0.0.0-20191227052852-215e87163ea7/go.mod h1:cIg4eruTrX1D+g88fzRXU5OdNfaM+9IcxsU14FzY7Hc= github.com/golang/groupcache v0.0.0-20200121045136-8c9f03a8e57e h1:1r7pUrabqp18hOBcwBwiTsbnFeTZHV9eER/QT5JVZxY= github.com/golang/groupcache v0.0.0-20200121045136-8c9f03a8e57e/go.mod h1:cIg4eruTrX1D+g88fzRXU5OdNfaM+9IcxsU14FzY7Hc= -github.com/golang/mock v1.1.1 h1:G5FRp8JnTd7RQH5kemVNlMeyXQAztQ3mOWV95KxsXH8= github.com/golang/mock v1.1.1/go.mod h1:oTYuIxOrZwtPieC+H1uAHpcLFnEyAGVDL/k47Jfbm0A= github.com/golang/mock v1.2.0/go.mod h1:oTYuIxOrZwtPieC+H1uAHpcLFnEyAGVDL/k47Jfbm0A= -github.com/golang/mock v1.3.1 h1:qGJ6qTW+x6xX/my+8YUVl4WNpX9B7+/l2tRsHGZ7f2s= github.com/golang/mock v1.3.1/go.mod h1:sBzyDLLjw3U8JLTeZvSv8jJB+tU5PVekmnlKIyFUx0Y= github.com/golang/mock v1.4.0/go.mod h1:UOMv5ysSaYNkG+OFQykRIcU/QvvxJf3p21QfJ2Bt3cw= github.com/golang/mock v1.4.4 h1:l75CXGRSwbaYNpl/Z2X1XIIAMSCquvXgpVZDhwEIJsc= github.com/golang/mock v1.4.4/go.mod h1:l3mdAwkq5BuhzHwde/uurv3sEJeZMXNpwsxVWU71h+4= github.com/golang/protobuf v0.0.0-20180814211427-aa810b61a9c7/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= -github.com/golang/protobuf v1.2.0 h1:P3YflyNX/ehuJFLhxviNdFxQPkGK5cDcApsge1SqnvM= github.com/golang/protobuf v1.2.0/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= -github.com/golang/protobuf v1.3.1 h1:YF8+flBXS5eO826T4nzqPrxfhQThhXl0YzfuUPu4SBg= github.com/golang/protobuf v1.3.1/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= -github.com/golang/protobuf v1.3.2 h1:6nsPYzhq5kReh6QImI3k5qWzO4PEbvbIW2cwSfR/6xs= github.com/golang/protobuf v1.3.2/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= github.com/golang/protobuf v1.3.3/go.mod h1:vzj43D7+SQXF/4pzW/hwtAqwc6iTitCiVSaWz5lYuqw= github.com/golang/protobuf v1.3.4 h1:87PNWwrRvUSnqS4dlcBU/ftvOIBep4sYuBLlh6rX2wk= github.com/golang/protobuf v1.3.4/go.mod h1:vzj43D7+SQXF/4pzW/hwtAqwc6iTitCiVSaWz5lYuqw= github.com/golang/snappy v0.0.0-20170215233205-553a64147049/go.mod h1:/XxbfmMg8lxefKM7IXC3fBNl/7bRcc72aCRzEWrmP2Q= -github.com/golang/snappy v0.0.0-20180518054509-2e65f85255db h1:woRePGFeVFfLKN/pOkfl+p/TAqKOfFu+7KPlMVpok/w= github.com/golang/snappy v0.0.0-20180518054509-2e65f85255db/go.mod h1:/XxbfmMg8lxefKM7IXC3fBNl/7bRcc72aCRzEWrmP2Q= -github.com/golang/snappy v0.0.1 h1:Qgr9rKW7uDUkrbSmQeiDsGa8SjGyCOGtuasMWwvp2P4= github.com/golang/snappy v0.0.1/go.mod h1:/XxbfmMg8lxefKM7IXC3fBNl/7bRcc72aCRzEWrmP2Q= github.com/golang/snappy v0.0.2-0.20190904063534-ff6b7dc882cf h1:gFVkHXmVAhEbxZVDln5V9GKrLaluNoFHDbrZwAWZgws= github.com/golang/snappy v0.0.2-0.20190904063534-ff6b7dc882cf/go.mod h1:/XxbfmMg8lxefKM7IXC3fBNl/7bRcc72aCRzEWrmP2Q= @@ -418,9 +377,7 @@ github.com/google/btree v1.0.0/go.mod h1:lNA+9X1NB3Zf8V7Ke586lFgjr2dZNuvo3lPJSGZ github.com/google/flatbuffers v1.11.0/go.mod h1:1AeVuKshWv4vARoZatz6mlQ0JxURH0Kv5+zNeJKJCa8= github.com/google/go-cmp v0.2.0/go.mod h1:oXzfMopK8JAjlY9xF4vHSVASa0yLyX7SntLO5aqRK0M= github.com/google/go-cmp v0.3.0/go.mod h1:8QqcDgzrUqlUb/G2PQTWiueGozuR1884gddMywk6iLU= -github.com/google/go-cmp v0.3.1 h1:Xye71clBPdm5HgqGwUkwhbynsUJZhDbS20FvLhQ2izg= github.com/google/go-cmp v0.3.1/go.mod h1:8QqcDgzrUqlUb/G2PQTWiueGozuR1884gddMywk6iLU= -github.com/google/go-cmp v0.4.0 h1:xsAVV57WRhGj6kEIi8ReJzQlHHqcBYCElAvkovg3B/4= github.com/google/go-cmp v0.4.0/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= github.com/google/go-cmp v0.5.2 h1:X2ev0eStA3AbceY54o37/0PQ/UWqKEiiO2dKL5OPaFM= github.com/google/go-cmp v0.5.2/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= @@ -429,7 +386,6 @@ github.com/google/martian v2.1.0+incompatible h1:/CP5g8u/VJHijgedC/Legn3BAbAaWPg github.com/google/martian v2.1.0+incompatible/go.mod h1:9I4somxYTbIHy5NJKHRl3wXiIaQGbYVAs8BPL6v8lEs= github.com/google/pprof v0.0.0-20181206194817-3ea8567a2e57/go.mod h1:zfwlbNMJ+OItoe0UupaVj+oy1omPYYDuagoSzA8v9mc= github.com/google/pprof v0.0.0-20190515194954-54271f7e092f/go.mod h1:zfwlbNMJ+OItoe0UupaVj+oy1omPYYDuagoSzA8v9mc= -github.com/google/pprof v0.0.0-20190930153522-6ce02741cba3 h1:3CYI9xg87xNAD+es02gZxbX/ky4KQeoFBsNOzuoAQZg= github.com/google/pprof v0.0.0-20190930153522-6ce02741cba3/go.mod h1:zfwlbNMJ+OItoe0UupaVj+oy1omPYYDuagoSzA8v9mc= github.com/google/pprof v0.0.0-20191218002539-d4f498aebedc/go.mod h1:ZgVRPoUq/hfqzAqh7sHMqb3I9Rq5C59dIz2SbBwJ4eM= github.com/google/pprof v0.0.0-20200212024743-f11f1df84d12/go.mod h1:ZgVRPoUq/hfqzAqh7sHMqb3I9Rq5C59dIz2SbBwJ4eM= @@ -452,19 +408,16 @@ github.com/gorilla/handlers v1.4.1/go.mod h1:Qkdc/uu4tH4g6mTK6auzZ766c4CA0Ng8+o/ github.com/gorilla/handlers v1.4.2 h1:0QniY0USkHQ1RGCLfKxeNHK9bkDHGRYGNDFBCS+YARg= github.com/gorilla/handlers v1.4.2/go.mod h1:Qkdc/uu4tH4g6mTK6auzZ766c4CA0Ng8+o/OAirnOIQ= github.com/gorilla/mux v1.7.0/go.mod h1:1lud6UwP+6orDFRuTfBEV8e9/aOM/c4fVVCaMa2zaAs= -github.com/gorilla/mux v1.7.3 h1:gnP5JzjVOuiZD07fKKToCAOjS0yOpj/qPETTXCCS6hw= github.com/gorilla/mux v1.7.3/go.mod h1:1lud6UwP+6orDFRuTfBEV8e9/aOM/c4fVVCaMa2zaAs= github.com/gorilla/mux v1.7.4 h1:VuZ8uybHlWmqV03+zRzdwKL4tUnIp1MAQtp1mIFE1bc= github.com/gorilla/mux v1.7.4/go.mod h1:DVbg23sWSpFRCP0SfiEN6jmj59UnW/n46BH5rLB71So= github.com/gorilla/securecookie v1.1.1/go.mod h1:ra0sb63/xPlUeL+yeDciTfxMRAA+MP+HVt/4epWDjd4= github.com/gorilla/websocket v0.0.0-20170926233335-4201258b820c/go.mod h1:E7qHFY5m1UJ88s3WnNqhKjPHQ0heANvMoAMk2YaljkQ= github.com/gorilla/websocket v1.2.0/go.mod h1:E7qHFY5m1UJ88s3WnNqhKjPHQ0heANvMoAMk2YaljkQ= -github.com/gorilla/websocket v1.4.0 h1:WDFjx/TMzVgy9VdMMQi2K2Emtwi2QcUQsztZ/zLaH/Q= github.com/gorilla/websocket v1.4.0/go.mod h1:E7qHFY5m1UJ88s3WnNqhKjPHQ0heANvMoAMk2YaljkQ= github.com/gorilla/websocket v1.4.1 h1:q7AeDBpnBk8AogcD4DSag/Ukw/KV+YhzLj2bP5HvKCM= github.com/gorilla/websocket v1.4.1/go.mod h1:YR8l580nyteQvAITg2hZ9XVh4b55+EU/adAjf1fMHhE= github.com/grpc-ecosystem/go-grpc-middleware v1.0.0/go.mod h1:FiyG127CGDf3tlThmgyCl78X/SZQqEOJBCDaAfeWzPs= -github.com/grpc-ecosystem/go-grpc-middleware v1.0.1-0.20190118093823-f849b5445de4 h1:z53tR0945TRRQO/fLEVPI6SMv7ZflF0TEaTAoU7tOzg= github.com/grpc-ecosystem/go-grpc-middleware v1.0.1-0.20190118093823-f849b5445de4/go.mod h1:FiyG127CGDf3tlThmgyCl78X/SZQqEOJBCDaAfeWzPs= github.com/grpc-ecosystem/go-grpc-middleware v1.1.0 h1:THDBEeQ9xZ8JEaCLyLQqXMMdRqNr0QAUJTIkQAUtFjg= github.com/grpc-ecosystem/go-grpc-middleware v1.1.0/go.mod h1:f5nM7jw/oeRSadq3xCzHAvxcr8HZnzsqU6ILg/0NiiE= @@ -474,7 +427,6 @@ github.com/grpc-ecosystem/grpc-gateway v1.8.1/go.mod h1:vNeuVxBJEsws4ogUvrchl83t github.com/grpc-ecosystem/grpc-gateway v1.8.5/go.mod h1:vNeuVxBJEsws4ogUvrchl83t/GYV9WGTSLVdBhOQFDY= github.com/grpc-ecosystem/grpc-gateway v1.9.0/go.mod h1:vNeuVxBJEsws4ogUvrchl83t/GYV9WGTSLVdBhOQFDY= github.com/grpc-ecosystem/grpc-gateway v1.9.5/go.mod h1:vNeuVxBJEsws4ogUvrchl83t/GYV9WGTSLVdBhOQFDY= -github.com/grpc-ecosystem/grpc-gateway v1.12.1 h1:zCy2xE9ablevUOrUZc3Dl72Dt+ya2FNAvC2yLYMHzi4= github.com/grpc-ecosystem/grpc-gateway v1.12.1/go.mod h1:8XEsbTttt/W+VvjtQhLACqCisSPWTxCZ7sBRjU6iH9c= github.com/grpc-ecosystem/grpc-gateway v1.14.3 h1:OCJlWkOUoTnl0neNGlf4fUm3TmbEtguw7vR+nGtnDjY= github.com/grpc-ecosystem/grpc-gateway v1.14.3/go.mod h1:6CwZWGDSPRJidgKAtJVvND6soZe6fT7iteq8wDPdhb0= @@ -511,7 +463,6 @@ github.com/jinzhu/gorm v1.9.12/go.mod h1:vhTjlKSJUTWNtcbQtrMBFCxy7eXTzeCAzfL5fBZ github.com/jinzhu/inflection v1.0.0/go.mod h1:h+uFLlag+Qp1Va5pdKtLDYj+kHp5pxUVkryuEj+Srlc= github.com/jinzhu/now v1.0.1/go.mod h1:d3SSVoowX0Lcu0IBviAWJpolVfI5UJVZZ7cO71lE/z8= github.com/jmespath/go-jmespath v0.0.0-20180206201540-c2b33e8439af/go.mod h1:Nht3zPeWKUH0NzdCt2Blrr5ys8VGpn0CEB0cQHVjt7k= -github.com/jmespath/go-jmespath v0.3.0 h1:OS12ieG61fsCg5+qLJ+SsW9NicxNkg3b25OyT2yCeUc= github.com/jmespath/go-jmespath v0.3.0/go.mod h1:9QtRXoHjLGCJ5IBSaohpXITPlowMeeYCZ7fLUTSywik= github.com/jmespath/go-jmespath v0.4.0 h1:BEgLn5cpjn8UN1mAw4NjwDrS35OdebyEtFe+9YPoQUg= github.com/jmespath/go-jmespath v0.4.0/go.mod h1:T8mJZnbsbmF+m6zOOFylbeCJqk5+pHWvzYPziyZiYoo= @@ -524,9 +475,7 @@ github.com/jonboulle/clockwork v0.1.0 h1:VKV+ZcuP6l3yW9doeqz6ziZGgcynBVQO+obU0+0 github.com/jonboulle/clockwork v0.1.0/go.mod h1:Ii8DK3G1RaLaWxj9trq07+26W01tbo22gdxWY5EU2bo= github.com/joomcode/errorx v1.0.1/go.mod h1:kgco15ekB6cs+4Xjzo7SPeXzx38PbJzBwbnu9qfVNHQ= github.com/json-iterator/go v1.1.5/go.mod h1:+SdeFBvtyEkXs7REEP0seUULqWtbJapLOCVDaaPEHmU= -github.com/json-iterator/go v1.1.6 h1:MrUvLMLTMxbqFJ9kzlvat/rYZqZnW3u4wkLzWTaFwKs= github.com/json-iterator/go v1.1.6/go.mod h1:+SdeFBvtyEkXs7REEP0seUULqWtbJapLOCVDaaPEHmU= -github.com/json-iterator/go v1.1.7 h1:KfgG9LzI+pYjr4xvmz/5H4FXjokeP+rlHLhv3iH62Fo= github.com/json-iterator/go v1.1.7/go.mod h1:KdQUCv79m/52Kvf8AW2vK1V8akMuk1QjK/uOdHXbAo4= github.com/json-iterator/go v1.1.9 h1:9yzud/Ht36ygwatGx56VwCZtlI/2AD15T1X2sjSuGns= github.com/json-iterator/go v1.1.9/go.mod h1:KdQUCv79m/52Kvf8AW2vK1V8akMuk1QjK/uOdHXbAo4= @@ -538,11 +487,9 @@ github.com/jtolds/gls v4.20.0+incompatible/go.mod h1:QJZ7F/aHp+rZTRtaJ1ow/lLfFfV github.com/juju/ansiterm v0.0.0-20180109212912-720a0952cc2a/go.mod h1:UJSiEoRfvx3hP73CvoARgeLjaIOjybY9vj8PUPPFGeU= github.com/juju/clock v0.0.0-20180524022203-d293bb356ca4/go.mod h1:nD0vlnrUjcjJhqN5WuCWZyzfd5AHZAC9/ajvbSx69xA= github.com/juju/errors v0.0.0-20150916125642-1b5e39b83d18/go.mod h1:W54LbzXuIE0boCoNJfwqpmkKJ1O4TCTZMetAt6jGk7Q= -github.com/juju/errors v0.0.0-20181118221551-089d3ea4e4d5 h1:rhqTjzJlm7EbkELJDKMTU7udov+Se0xZkWmugr6zGok= github.com/juju/errors v0.0.0-20181118221551-089d3ea4e4d5/go.mod h1:W54LbzXuIE0boCoNJfwqpmkKJ1O4TCTZMetAt6jGk7Q= github.com/juju/errors v0.0.0-20190930114154-d42613fe1ab9 h1:hJix6idebFclqlfZCHE7EUX7uqLCyb70nHNHH1XKGBg= github.com/juju/errors v0.0.0-20190930114154-d42613fe1ab9/go.mod h1:W54LbzXuIE0boCoNJfwqpmkKJ1O4TCTZMetAt6jGk7Q= -github.com/juju/loggo v0.0.0-20170605014607-8232ab8918d9 h1:Y+lzErDTURqeXqlqYi4YBYbDd7ycU74gW1ADt57/bgY= github.com/juju/loggo v0.0.0-20170605014607-8232ab8918d9/go.mod h1:vgyd7OREkbtVEN/8IXZe5Ooef3LQePvuBm9UWj6ZL8U= github.com/juju/loggo v0.0.0-20190526231331-6e530bcce5d8 h1:UUHMLvzt/31azWTN/ifGWef4WUqvXk0iRqdhdy/2uzI= github.com/juju/loggo v0.0.0-20190526231331-6e530bcce5d8/go.mod h1:vgyd7OREkbtVEN/8IXZe5Ooef3LQePvuBm9UWj6ZL8U= @@ -551,7 +498,6 @@ github.com/juju/ratelimit v1.0.1/go.mod h1:qapgC/Gy+xNh9UxzV13HGGl/6UXNN+ct+vwSg github.com/juju/retry v0.0.0-20160928201858-1998d01ba1c3/go.mod h1:OohPQGsr4pnxwD5YljhQ+TZnuVRYpa5irjugL1Yuif4= github.com/juju/testing v0.0.0-20191001232224-ce9dec17d28b/go.mod h1:63prj8cnj0tU0S9OHjGJn+b1h0ZghCndfnbQolrYTwA= github.com/juju/testing v0.0.0-20200510222523-6c8c298c77a0/go.mod h1:hpGvhGHPVbNBraRLZEhoQwFLMrjK8PSlO4D3nDjKYXo= -github.com/juju/testing v0.0.0-20200608005635-e4eedbc6f7aa h1:v1ZEHRVaUgTIkxzYaT78fJ+3bV3vjxj9jfNJcYzi9pY= github.com/juju/testing v0.0.0-20200608005635-e4eedbc6f7aa/go.mod h1:hpGvhGHPVbNBraRLZEhoQwFLMrjK8PSlO4D3nDjKYXo= github.com/juju/testing v0.0.0-20200706033705-4c23f9c453cd h1:4MRI5TGW0cRgovUipCGLF4uF+31Fo8VzkV2753OAfEE= github.com/juju/testing v0.0.0-20200706033705-4c23f9c453cd/go.mod h1:hpGvhGHPVbNBraRLZEhoQwFLMrjK8PSlO4D3nDjKYXo= @@ -563,18 +509,14 @@ github.com/jwilder/encoding v0.0.0-20170811194829-b4e1701a28ef/go.mod h1:Ct9fl0F github.com/kami-zh/go-capturer v0.0.0-20171211120116-e492ea43421d h1:cVtBfNW5XTHiKQe7jDaDBSh/EVM4XLPutLAGboIXuM0= github.com/kami-zh/go-capturer v0.0.0-20171211120116-e492ea43421d/go.mod h1:P2viExyCEfeWGU259JnaQ34Inuec4R38JCyBx2edgD0= github.com/kardianos/osext v0.0.0-20190222173326-2bc1f35cddc0/go.mod h1:1NbS8ALrpOvjt0rHPNLyCIeMtbizbir8U//inJ+zuB8= -github.com/kevinburke/go-bindata v3.18.0+incompatible h1:NfOP49jFW7KyBl7UwTg0xkhSfHjESEwe2VMrcnSHG20= github.com/kevinburke/go-bindata v3.18.0+incompatible/go.mod h1:/pEEZ72flUW2p0yi30bslSp9YqD9pysLxunQDdb2CPM= github.com/kisielk/errcheck v1.1.0/go.mod h1:EZBBE59ingxPouuu3KfxchcWSUPOHkagtvWXihfKN4Q= -github.com/kisielk/errcheck v1.2.0 h1:reN85Pxc5larApoH1keMBiu2GWtPqXQ1nc9gx+jOU+E= github.com/kisielk/errcheck v1.2.0/go.mod h1:/BMXB+zMLi60iA8Vv6Ksmxu/1UDYcXs4uQLJ+jE2L00= github.com/kisielk/gotool v1.0.0/go.mod h1:XhKaO+MFFWcvkIS/tQcRk01m1F5IRFswLeQ+oQHNcck= github.com/klauspost/compress v1.4.0/go.mod h1:RyIbtBH6LamlWaDj8nUwkbUhJ87Yi3uG0guNDohfE1A= -github.com/klauspost/compress v1.9.5 h1:U+CaK85mrNNb4k8BNOfgJtJ/gr6kswUCFj6miSzVC6M= github.com/klauspost/compress v1.9.5/go.mod h1:RyIbtBH6LamlWaDj8nUwkbUhJ87Yi3uG0guNDohfE1A= github.com/klauspost/compress v1.10.5 h1:7q6vHIqubShURwQz8cQK6yIe/xC3IF0Vm7TGfqjewrc= github.com/klauspost/compress v1.10.5/go.mod h1:aoV0uJVorq1K+umq18yTdKaF57EivdYsUV+/s2qKfXs= -github.com/klauspost/cpuid v0.0.0-20170728055534-ae7887de9fa5 h1:2U0HzY8BJ8hVwDKIzp7y4voR9CX/nvcfymLmg2UiOio= github.com/klauspost/cpuid v0.0.0-20170728055534-ae7887de9fa5/go.mod h1:Pj4uuM528wm8OyEC2QMXAi2YiTZ96dNQPGgoMS4s3ek= github.com/klauspost/cpuid v1.2.0/go.mod h1:Pj4uuM528wm8OyEC2QMXAi2YiTZ96dNQPGgoMS4s3ek= github.com/klauspost/cpuid v1.2.1 h1:vJi+O/nMdFt0vqm8NZBI6wzALWdA2X+egi0ogNyrC/w= @@ -582,20 +524,16 @@ github.com/klauspost/cpuid v1.2.1/go.mod h1:Pj4uuM528wm8OyEC2QMXAi2YiTZ96dNQPGgo github.com/klauspost/crc32 v0.0.0-20161016154125-cb6bfca970f6/go.mod h1:+ZoRqAPRLkC4NPOvfYeR5KNOrY6TD+/sAC3HXPZgDYg= github.com/klauspost/pgzip v1.0.2-0.20170402124221-0bf5dcad4ada/go.mod h1:Ch1tH69qFZu15pkjo5kYi6mth2Zzwzt50oCQKQE9RUs= github.com/kljensen/snowball v0.6.0/go.mod h1:27N7E8fVU5H68RlUmnWwZCfxgt4POBJfENGMvNRhldw= -github.com/konsorten/go-windows-terminal-sequences v1.0.1 h1:mweAR1A6xJ3oS2pRaGiHgQ4OO8tzTaLawm8vnODuwDk= github.com/konsorten/go-windows-terminal-sequences v1.0.1/go.mod h1:T0+1ngSBFLxvqU3pZ+m/2kptfBszLMUkC4ZK/EgS/cQ= -github.com/konsorten/go-windows-terminal-sequences v1.0.2 h1:DB17ag19krx9CFsz4o3enTrPXyIXCl+2iCXH/aMAp9s= github.com/konsorten/go-windows-terminal-sequences v1.0.2/go.mod h1:T0+1ngSBFLxvqU3pZ+m/2kptfBszLMUkC4ZK/EgS/cQ= github.com/konsorten/go-windows-terminal-sequences v1.0.3 h1:CE8S1cTafDpPvMhIxNJKvHsGVBgn1xWYf1NbHQhywc8= github.com/konsorten/go-windows-terminal-sequences v1.0.3/go.mod h1:T0+1ngSBFLxvqU3pZ+m/2kptfBszLMUkC4ZK/EgS/cQ= github.com/kr/logfmt v0.0.0-20140226030751-b84e30acd515/go.mod h1:+0opPa2QZZtGFBFZlji/RkVcI2GknAs/DXo4wKdlNEc= -github.com/kr/pretty v0.1.0 h1:L/CwN0zerZDmRFUapSPitk6f+Q3+0za1rQkzVuMiMFI= github.com/kr/pretty v0.1.0/go.mod h1:dAy3ld7l9f0ibDNOQOHHMYYIIbhfbHSm3C4ZsoJORNo= github.com/kr/pretty v0.2.1 h1:Fmg33tUaq4/8ym9TJN1x7sLJnHVwhP33CNkpYV/7rwI= github.com/kr/pretty v0.2.1/go.mod h1:ipq/a2n7PKx3OHsz4KJII5eveXtPO4qwEXGdVfWzfnI= github.com/kr/pty v1.1.1/go.mod h1:pFQYn66WHrOpPYNljwOMqo10TkYh1fy3cYio2l3bCsQ= github.com/kr/pty v1.1.5/go.mod h1:9r2w37qlBe7rQ6e1fg1S/9xpWHSnaqNdHD3WcMdbPDA= -github.com/kr/text v0.1.0 h1:45sCR5RtlFHMR4UwH9sdQ5TC8v0qDQCHnXt+kaKSTVE= github.com/kr/text v0.1.0/go.mod h1:4Jbv+DJW3UT/LiOwJeYQe1efqtUx/iVham/4vfdArNI= github.com/kr/text v0.2.0 h1:5Nx0Ya0ZqY2ygV366QzturHI13Jq95ApcVaJBhpS+AY= github.com/kr/text v0.2.0/go.mod h1:eLer722TekiGuMkidMxC/pM04lWEeraHUUmBw8l2grE= @@ -612,13 +550,11 @@ github.com/magiconair/properties v1.8.0/go.mod h1:PppfXfuXeibc/6YijjN8zIbojt8czP github.com/mailru/easyjson v0.0.0-20180823135443-60711f1a8329/go.mod h1:C1wdFJiN94OJF2b5HbByQZoLdCWB1Yqtg26g4irojpc= github.com/mailru/easyjson v0.0.0-20190614124828-94de47d64c63/go.mod h1:C1wdFJiN94OJF2b5HbByQZoLdCWB1Yqtg26g4irojpc= github.com/mailru/easyjson v0.0.0-20190626092158-b2ccc519800e/go.mod h1:C1wdFJiN94OJF2b5HbByQZoLdCWB1Yqtg26g4irojpc= -github.com/mailru/easyjson v0.7.1 h1:mdxE1MF9o53iCb2Ghj1VfWvh7ZOwHpnVG/xwXrV90U8= github.com/mailru/easyjson v0.7.1/go.mod h1:KAzv3t3aY1NaHWoQz1+4F1ccyAH66Jk7yos7ldAVICs= github.com/markbates/pkger v0.16.0/go.mod h1:0JoVlrol20BSywW79rN3kdFFsE5xYM+rSCQDXbLhiuI= github.com/mattn/go-adodb v0.0.1/go.mod h1:jaSTRde4bohMuQgYQPxW3xRTPtX/cZKyxPrFVseJULo= github.com/mattn/go-colorable v0.0.9/go.mod h1:9vuHe8Xs5qXnSaW/c/ABM9alt+Vo+STaOChaDxuIBZU= github.com/mattn/go-colorable v0.1.2/go.mod h1:U0ppj6V5qS13XJ6of8GYAs25YV2eR4EVcfRqFIhoBtE= -github.com/mattn/go-colorable v0.1.4 h1:snbPLB8fVfU9iwbbo30TPtbLRzwWu6aJS6Xh4eaaviA= github.com/mattn/go-colorable v0.1.4/go.mod h1:U0ppj6V5qS13XJ6of8GYAs25YV2eR4EVcfRqFIhoBtE= github.com/mattn/go-colorable v0.1.6/go.mod h1:u6P/XSegPjTcexA+o6vUJrdnUu04hMope9wVRipJSqc= github.com/mattn/go-colorable v0.1.7 h1:bQGKb3vps/j0E9GfJQ03JyhRuxsvdAanXlT9BTw3mdw= @@ -628,14 +564,12 @@ github.com/mattn/go-isatty v0.0.7/go.mod h1:Iq45c/XA43vh69/j3iqttzPXn0bhXyGjM0Hd github.com/mattn/go-isatty v0.0.8/go.mod h1:Iq45c/XA43vh69/j3iqttzPXn0bhXyGjM0Hdxcsrc5s= github.com/mattn/go-isatty v0.0.9/go.mod h1:YNRxwqDuOph6SZLI9vUUz6OYw3QyUt7WiY2yME+cCiQ= github.com/mattn/go-isatty v0.0.10/go.mod h1:qgIWMr58cqv1PHHyhnkY9lrL7etaEgOFcMEpPG5Rm84= -github.com/mattn/go-isatty v0.0.11 h1:FxPOTFNqGkuDUGi3H/qkUbQO4ZiBa2brKq5r0l8TGeM= github.com/mattn/go-isatty v0.0.11/go.mod h1:PhnuNfih5lzO57/f3n+odYbM4JtupLOxQOAqxQCu2WE= github.com/mattn/go-isatty v0.0.12 h1:wuysRhFDzyxgEmMf5xjvJ2M9dZoWAXNNr5LSBS7uHXY= github.com/mattn/go-isatty v0.0.12/go.mod h1:cbi8OIDigv2wuxKPP5vlRcQ1OAZbq2CE4Kysco4FUpU= github.com/mattn/go-runewidth v0.0.2/go.mod h1:LwmH8dsx7+W8Uxz3IHJYH5QSwggIsqBzpuz5H//U1FU= github.com/mattn/go-runewidth v0.0.3/go.mod h1:LwmH8dsx7+W8Uxz3IHJYH5QSwggIsqBzpuz5H//U1FU= github.com/mattn/go-runewidth v0.0.4/go.mod h1:LwmH8dsx7+W8Uxz3IHJYH5QSwggIsqBzpuz5H//U1FU= -github.com/mattn/go-runewidth v0.0.7 h1:Ei8KR0497xHyKJPAv59M1dkC+rOZCMBJ+t3fZ+twI54= github.com/mattn/go-runewidth v0.0.7/go.mod h1:H031xJmbD/WCDINGzjvQ9THkh0rPKHF+m2gUSrubnMI= github.com/mattn/go-runewidth v0.0.9 h1:Lm995f3rfxdpd6TSmuVCHVb/QhupuXlYr8sCI/QdE+0= github.com/mattn/go-runewidth v0.0.9/go.mod h1:H031xJmbD/WCDINGzjvQ9THkh0rPKHF+m2gUSrubnMI= @@ -648,12 +582,9 @@ github.com/mattn/go-sqlite3 v2.0.2+incompatible/go.mod h1:FPy6KqzDD04eiIsT53CuJW github.com/mattn/go-tty v0.0.0-20180907095812-13ff1204f104/go.mod h1:XPvLUNfbS4fJH25nqRHfWLMa1ONC8Amw+mIA639KxkE= github.com/matttproud/golang_protobuf_extensions v1.0.1 h1:4hp9jkHxhMHkqkrB3Ix0jegS5sx/RkqARlsWZ6pIwiU= github.com/matttproud/golang_protobuf_extensions v1.0.1/go.mod h1:D8He9yQNgCq6Z5Ld7szi9bcBfOoFv/3dc6xSMkL2PC0= -github.com/mgechev/dots v0.0.0-20190921121421-c36f7dcfbb81 h1:QASJXOGm2RZ5Ardbc86qNFvby9AqkLDibfChMtAg5QM= github.com/mgechev/dots v0.0.0-20190921121421-c36f7dcfbb81/go.mod h1:KQ7+USdGKfpPjXk4Ga+5XxQM4Lm4e3gAogrreFAYpOg= -github.com/mgechev/revive v1.0.2 h1:v0NxxQ7fSFz/u1NQydPo6EGdq7va0J1BtsZmae6kzUg= github.com/mgechev/revive v1.0.2/go.mod h1:rb0dQy1LVAxW9SWy5R3LPUjevzUbUS316U5MFySA2lo= github.com/minio/minio-go v6.0.14+incompatible/go.mod h1:7guKYtitv8dktvNUGrhzmNlA5wrAABTQXCoesZdFQO8= -github.com/mitchellh/go-homedir v1.1.0 h1:lukF9ziXFxDFPkA1vsr5zpc1XuPDn/wFntq5mG+4E0Y= github.com/mitchellh/go-homedir v1.1.0/go.mod h1:SfyaCUpYCn1Vlf4IUYiD9fPX4A5wJrkLzIz1N1q0pr0= github.com/mitchellh/go-wordwrap v0.0.0-20150314170334-ad45545899c7/go.mod h1:ZXFpozHsX6DPmq2I0TCekCxypsnAUbP2oI0UX1GXzOo= github.com/mitchellh/go-wordwrap v1.0.0/go.mod h1:ZXFpozHsX6DPmq2I0TCekCxypsnAUbP2oI0UX1GXzOo= @@ -665,7 +596,6 @@ github.com/modern-go/reflect2 v0.0.0-20180701023420-4b7aa43c6742/go.mod h1:bx2lN github.com/modern-go/reflect2 v1.0.1 h1:9f412s+6RmYXLWZSEzVVgPGK7C2PphHj5RJrvfx9AWI= github.com/modern-go/reflect2 v1.0.1/go.mod h1:bx2lNnkwVCuqBIxFjflWJWanXIb3RllmbCylyMrvgv0= github.com/montanaflynn/stats v0.0.0-20151014174947-eeaced052adb/go.mod h1:wL8QJuTMNUDYhXwkmfOly8iTdp5TEcJFWZD2D7SIkUc= -github.com/montanaflynn/stats v0.0.0-20180911141734-db72e6cae808 h1:pmpDGKLw4n82EtrNiLqB+xSz/JQwFOaZuMALYUHwX5s= github.com/montanaflynn/stats v0.0.0-20180911141734-db72e6cae808/go.mod h1:wL8QJuTMNUDYhXwkmfOly8iTdp5TEcJFWZD2D7SIkUc= github.com/montanaflynn/stats v0.5.0 h1:2EkzeTSqBB4V4bJwWrt5gIIrZmpJBcoIRGS2kWLgzmk= github.com/montanaflynn/stats v0.5.0/go.mod h1:wL8QJuTMNUDYhXwkmfOly8iTdp5TEcJFWZD2D7SIkUc= @@ -686,18 +616,12 @@ github.com/ngaut/unistore v0.0.0-20200521040325-2af94f1b0c83/go.mod h1:odn0MiR+D github.com/ngaut/unistore v0.0.0-20200603091253-e0b717679796/go.mod h1:9mpqZeS1CkNlgZwJ0LZXb+Qd7xVO5o55ngys7T1/oH8= github.com/ngaut/unistore v0.0.0-20200604043635-5004cdad650f/go.mod h1:5Vec+R2BwOyugVQ8Id8uDmlIYbqodCvykM50IpaAjk4= github.com/ngaut/unistore v0.0.0-20200604061006-d8e9dc0ad154/go.mod h1:YGQzxn9cVy0q2puXVt1X8l5OohRHv2djR/ziz1k14XQ= -github.com/ngaut/unistore v0.0.0-20200630072006-0c4035925f69 h1:tn65bmj+EBa8gXYKQHPRlGcDdo+tDEjUK88gO4QK11s= github.com/ngaut/unistore v0.0.0-20200630072006-0c4035925f69/go.mod h1:Hxlp5VAoPOHwcXLUw/E+P3XjJX1EP38NWjXPpc4nuOE= github.com/ngaut/unistore v0.0.0-20200803051709-607d96233b1d/go.mod h1:2QAH8tXCjeHuCSLEWKLYAzHPz2dB59VnhpPA2IDVeW4= -github.com/ngaut/unistore v0.0.0-20200806113332-5b9f73333a19 h1:N8pW0PTJEGTyHzZuN7sofxVsFmuvR+vFD0BNJ243k2o= github.com/ngaut/unistore v0.0.0-20200806113332-5b9f73333a19/go.mod h1:RtZJKyiaHRiII+b9/g/4339rSikSvfrUJmIbrUkYVi4= -github.com/ngaut/unistore v0.0.0-20200820080223-c734bcc4ea53 h1:mamcR5THP4UTx6dMZp00LUq5kMILU6o2FnHEWzYZQhE= github.com/ngaut/unistore v0.0.0-20200820080223-c734bcc4ea53/go.mod h1:85S5ZgzoHtTMyaEYhaWnxv9OWMBfyhTNuWypXCfVn/0= -github.com/ngaut/unistore v0.0.0-20200828072424-1c0ede06a3fc h1:aWjX4/AooiJvLllPt+d7+4umIgFDKooKfLH+IRaQiGU= github.com/ngaut/unistore v0.0.0-20200828072424-1c0ede06a3fc/go.mod h1:iSlx5Ub/926GvQn6+d2B2C16wJJwgQIsi6k/bEU0vl4= -github.com/ngaut/unistore v0.0.0-20200918091209-68a2db6bb775 h1:9z7ZQ//+b6DjI79ak0sNf7RMgmjYh/BTB+NkipfdLiM= github.com/ngaut/unistore v0.0.0-20200918091209-68a2db6bb775/go.mod h1:ZR3NH+HzqfiYetwdoAivApnIy8iefPZHTMLfrFNm8g4= -github.com/ngaut/unistore v0.0.0-20201217024625-f53801ce8d4f h1:U3HZchmjtkfNbPgphbgB7wEiYRzllX85J70J2MoMyFo= github.com/ngaut/unistore v0.0.0-20201217024625-f53801ce8d4f/go.mod h1:ZR3NH+HzqfiYetwdoAivApnIy8iefPZHTMLfrFNm8g4= github.com/ngaut/unistore v0.0.0-20210201072520-a1a5525d7218 h1:q7sDtYh4i9kKAR7sOlaksKfKFd7NUxtrIX51U01YviM= github.com/ngaut/unistore v0.0.0-20210201072520-a1a5525d7218/go.mod h1:ZR3NH+HzqfiYetwdoAivApnIy8iefPZHTMLfrFNm8g4= @@ -709,23 +633,19 @@ github.com/nsf/termbox-go v0.0.0-20190121233118-02980233997d/go.mod h1:IuKpRQcYE github.com/oklog/ulid v1.3.1/go.mod h1:CirwcVhetQ6Lv90oh/F+FBtV6XMibvdAFo93nm5qn4U= github.com/oleiade/reflections v1.0.0/go.mod h1:RbATFBbKYkVdqmSFtx13Bb/tVhR0lgOBXunWTZKeL4w= github.com/olekukonko/tablewriter v0.0.0-20170122224234-a0225b3f23b5/go.mod h1:vsDQFd/mU46D+Z4whnwzcISnGGzXWMclvtLoiIKAKIo= -github.com/olekukonko/tablewriter v0.0.4 h1:vHD/YYe1Wolo78koG299f7V/VAS08c6IpCLn+Ejf/w8= github.com/olekukonko/tablewriter v0.0.4/go.mod h1:zq6QwlOf5SlnkVbMSr5EoBv3636FWnp+qbPhuoO21uA= github.com/onsi/ginkgo v1.6.0/go.mod h1:lLunBs/Ym6LB5Z9jYTR76FiuTmxDTDusOGeTQH+WWjE= -github.com/onsi/ginkgo v1.7.0 h1:WSHQ+IS43OoUrWtD1/bbclrwK8TTH5hzp+umCiuxHgs= github.com/onsi/ginkgo v1.7.0/go.mod h1:lLunBs/Ym6LB5Z9jYTR76FiuTmxDTDusOGeTQH+WWjE= github.com/onsi/ginkgo v1.9.0/go.mod h1:lLunBs/Ym6LB5Z9jYTR76FiuTmxDTDusOGeTQH+WWjE= github.com/onsi/ginkgo v1.11.0 h1:JAKSXpt1YjtLA7YpPiqO9ss6sNXEsPfSGdwN0UHqzrw= github.com/onsi/ginkgo v1.11.0/go.mod h1:lLunBs/Ym6LB5Z9jYTR76FiuTmxDTDusOGeTQH+WWjE= github.com/onsi/gomega v1.4.2/go.mod h1:ex+gbHU/CVuBBDIJjb2X0qEXbFg53c61hWP/1CpauHY= -github.com/onsi/gomega v1.4.3 h1:RE1xgDvH7imwFD45h+u2SgIfERHlS2yNG4DObb5BSKU= github.com/onsi/gomega v1.4.3/go.mod h1:ex+gbHU/CVuBBDIJjb2X0qEXbFg53c61hWP/1CpauHY= github.com/onsi/gomega v1.6.0/go.mod h1:ex+gbHU/CVuBBDIJjb2X0qEXbFg53c61hWP/1CpauHY= github.com/onsi/gomega v1.8.1 h1:C5Dqfs/LeauYDX0jJXIe2SWmwCbGzx9yF8C8xy3Lh34= github.com/onsi/gomega v1.8.1/go.mod h1:Ho0h+IUsWyvy1OpqCwxlQ/21gkhVunqlU8fDGcoTdcA= github.com/opentracing/basictracer-go v1.0.0 h1:YyUAhaEfjoWXclZVJ9sGoNct7j4TVk7lZWlQw5UXuoo= github.com/opentracing/basictracer-go v1.0.0/go.mod h1:QfBfYuafItcjQuMwinw9GhYKwFXS9KnPs5lxoYwgW74= -github.com/opentracing/opentracing-go v1.0.2 h1:3jA2P6O1F9UOrWVpwrIo17pu01KWvNWg4X946/Y5Zwg= github.com/opentracing/opentracing-go v1.0.2/go.mod h1:UkNAQd3GIcIGf0SeVgPpRdFStlNbqXla1AfSYxPUl2o= github.com/opentracing/opentracing-go v1.0.3-0.20180606204148-bd9c31933947/go.mod h1:UkNAQd3GIcIGf0SeVgPpRdFStlNbqXla1AfSYxPUl2o= github.com/opentracing/opentracing-go v1.1.0 h1:pWlfV3Bxv7k65HYwkikxat0+s3pV4bsqf19k25Ur8rU= @@ -739,14 +659,11 @@ github.com/paulbellamy/ratecounter v0.2.0/go.mod h1:Hfx1hDpSGoqxkVVpBi/IlYD7kChl github.com/pelletier/go-toml v1.2.0/go.mod h1:5z9KED0ma1S8pY6P1sdut58dfprrGBbd/94hg7ilaic= github.com/pelletier/go-toml v1.3.0/go.mod h1:PN7xzY2wHTK0K9p34ErDQMlFxa51Fk0OUruD3k1mMwo= github.com/peterh/liner v1.0.1-0.20180619022028-8c1271fcf47f/go.mod h1:xIteQHvHuaLYG9IFj6mSxM0fCKrs34IrEQUhOYuGPHc= -github.com/petermattis/goid v0.0.0-20180202154549-b0b1615b78e5 h1:q2e307iGHPdTGp0hoxKjt1H5pDo6utceo3dQVK3I5XQ= github.com/petermattis/goid v0.0.0-20180202154549-b0b1615b78e5/go.mod h1:jvVRKCrJTQWu0XVbaOlby/2lO20uSCHEMzzplHXte1o= -github.com/phayes/freeport v0.0.0-20180830031419-95f893ade6f2 h1:JhzVVoYvbOACxoUmOs6V/G4D5nPVUW73rKvXxP4XUJc= github.com/phayes/freeport v0.0.0-20180830031419-95f893ade6f2/go.mod h1:iIss55rKnNBTvrwdmkUpLnDpZoAHvWaiq5+iMmen4AE= github.com/phf/go-queue v0.0.0-20170504031614-9abe38d0371d h1:U+PMnTlV2tu7RuMK5etusZG3Cf+rpow5hqQByeCzJ2g= github.com/phf/go-queue v0.0.0-20170504031614-9abe38d0371d/go.mod h1:lXfE4PvvTW5xOjO6Mba8zDPyw8M93B6AQ7frTGnMlA8= github.com/philhofer/fwd v1.0.0/go.mod h1:gk3iGcWd9+svBvR0sR+KPcfE+RNWozjowpeBVG3ZVNU= -github.com/pierrec/lz4 v2.0.5+incompatible h1:2xWsjqPFWcplujydGg4WmhC/6fZqK42wMM8aXeqhl0I= github.com/pierrec/lz4 v2.0.5+incompatible/go.mod h1:pdkljMzZIN41W+lC3N2tnIh5sFi+IEE17M5jbnwPHcY= github.com/pierrec/lz4 v2.2.6+incompatible/go.mod h1:pdkljMzZIN41W+lC3N2tnIh5sFi+IEE17M5jbnwPHcY= github.com/pierrec/lz4 v2.5.2+incompatible h1:WCjObylUIOlKy/+7Abdn34TLIkXiA4UWUMhxq9m9ZXI= @@ -758,11 +675,8 @@ github.com/pingcap-incubator/tidb-dashboard v0.0.0-20200715070228-47f5de8a6992/g github.com/pingcap-incubator/tidb-dashboard v0.0.0-20200807020752-01f0abe88e93/go.mod h1:9yaAM77sPfa5/f6sdxr3jSkKfIz463KRHyiFHiGjdes= github.com/pingcap-incubator/tidb-dashboard v0.0.0-20201022065613-94d8dc38a204/go.mod h1:X3r7/4Wr9fSC5KlsfezBh/5noeWGEJNQuSvjgS2rvdI= github.com/pingcap-incubator/tidb-dashboard v0.0.0-20210104140916-41a0a3a87e75/go.mod h1:EONGys2gM5n14pII2vjmU/5VG3Dtj6kpqUT1GUZ4ysw= -github.com/pingcap/badger v1.5.1-0.20200604041313-19c397305fcc h1:aL83XYtYRGJHGruiw5Kk/vJiJl2xFiWfqCgnvHin7ek= github.com/pingcap/badger v1.5.1-0.20200604041313-19c397305fcc/go.mod h1:LyrqUOHZrUDf9oGi1yoz1+qw9ckSIhQb5eMa1acOLNQ= -github.com/pingcap/badger v1.5.1-0.20200714132513-80ba2000f159 h1:cmZSuRbdfOJd3kJjRIClrLbt3nD0xi4oqYR1c/ZrPKg= github.com/pingcap/badger v1.5.1-0.20200714132513-80ba2000f159/go.mod h1:LyrqUOHZrUDf9oGi1yoz1+qw9ckSIhQb5eMa1acOLNQ= -github.com/pingcap/badger v1.5.1-0.20200810065601-8c92a97807f9 h1:tjVCYovZalb5nxH6Imn70Ifcu/FlBxMf0xFqUNW9q7Y= github.com/pingcap/badger v1.5.1-0.20200810065601-8c92a97807f9/go.mod h1:LyrqUOHZrUDf9oGi1yoz1+qw9ckSIhQb5eMa1acOLNQ= github.com/pingcap/badger v1.5.1-0.20200908111422-2e78ee155d19 h1:IXpGy7y9HyoShAFmzW2OPF0xCA5EOoSTyZHwsgYk9Ro= github.com/pingcap/badger v1.5.1-0.20200908111422-2e78ee155d19/go.mod h1:LyrqUOHZrUDf9oGi1yoz1+qw9ckSIhQb5eMa1acOLNQ= @@ -772,21 +686,15 @@ github.com/pingcap/br v0.0.0-20200610051721-b057d65ff579/go.mod h1:Gq6o66nDReG0f github.com/pingcap/br v0.0.0-20200617120402-56e151ad8b67/go.mod h1:/3QzpDG7YTPrDsrg8i1lwdYUrplJ0jVD+9pxhh19+k4= github.com/pingcap/br v0.0.0-20200727092753-a475692725db/go.mod h1:4iTqZAMbEPmjBggYixqIg2FwIHBQtyImTM/QYlpTBGk= github.com/pingcap/br v0.0.0-20200803052654-e6f63fc1807a/go.mod h1:8j7vGUfHCETYbeBfASLTDywC3NFSx90z9nuk0PV9rpo= -github.com/pingcap/br v0.0.0-20200805121136-181c081ba6ac h1:UTDTEFuFdS/cyCqYAcp8rWBfG8qJnzHIckC5FPUKfqw= github.com/pingcap/br v0.0.0-20200805121136-181c081ba6ac/go.mod h1:9P24mNzNmXjggYBm4pnb08slSbua8FA6QIyg68GpuhQ= -github.com/pingcap/br v0.0.0-20200820083933-d9d6207c0aa7 h1:7YWkuK/QY7/nz819lnxb0qDXqLrApDjZHjYPo+tduGA= github.com/pingcap/br v0.0.0-20200820083933-d9d6207c0aa7/go.mod h1:5ri8663t7CtJuG0kiOKKoBmwk9HOCX5MoKpmh1fW4CE= -github.com/pingcap/br v0.0.0-20200923023944-7456456854e4 h1:f1e1xbBAMc6mOrnBtrPRke52Zxv8zVlyr5g0Tz/pySQ= github.com/pingcap/br v0.0.0-20200923023944-7456456854e4/go.mod h1:DGsMcZVYt2haeDF/xGerf77c2RpTymgYY5+bMg8uArA= github.com/pingcap/br v4.0.0-beta.2.0.20210115100158-7a7b4a421c0a+incompatible/go.mod h1:ymVmo50lQydxib0tmK5hHk4oteB7hZ0IMCArunwy3UQ= github.com/pingcap/br v4.0.0-beta.2.0.20210203034957-7bc483ab69d5+incompatible/go.mod h1:ymVmo50lQydxib0tmK5hHk4oteB7hZ0IMCArunwy3UQ= github.com/pingcap/br v4.0.0-beta.2.0.20210220133344-578be7fb5165+incompatible h1:Zd4LjoIYVmGF9KW484B0F+XvFHlcp9hraI5FAB9h1/I= github.com/pingcap/br v4.0.0-beta.2.0.20210220133344-578be7fb5165+incompatible/go.mod h1:ymVmo50lQydxib0tmK5hHk4oteB7hZ0IMCArunwy3UQ= -github.com/pingcap/check v0.0.0-20190102082844-67f458068fc8 h1:USx2/E1bX46VG32FIw034Au6seQ2fY9NEILmNh/UlQg= github.com/pingcap/check v0.0.0-20190102082844-67f458068fc8/go.mod h1:B1+S9LNcuMyLH/4HMTViQOJevkGiik3wW2AN9zb2fNQ= -github.com/pingcap/check v0.0.0-20191107115940-caf2b9e6ccf4 h1:iRtOAQ6FXkY/BGvst3CDfTva4nTqh6CL8WXvanLdbu0= github.com/pingcap/check v0.0.0-20191107115940-caf2b9e6ccf4/go.mod h1:PYMCGwN0JHjoqGr3HrZoD+b8Tgx8bKnArhSq8YVzUMc= -github.com/pingcap/check v0.0.0-20191216031241-8a5a85928f12 h1:rfD9v3+ppLPzoQBgZev0qYCpegrwyFx/BUpkApEiKdY= github.com/pingcap/check v0.0.0-20191216031241-8a5a85928f12/go.mod h1:PYMCGwN0JHjoqGr3HrZoD+b8Tgx8bKnArhSq8YVzUMc= github.com/pingcap/check v0.0.0-20200212061837-5e12011dc712 h1:R8gStypOBmpnHEx1qi//SaqxJVI4inOqljg/Aj5/390= github.com/pingcap/check v0.0.0-20200212061837-5e12011dc712/go.mod h1:PYMCGwN0JHjoqGr3HrZoD+b8Tgx8bKnArhSq8YVzUMc= @@ -796,35 +704,23 @@ github.com/pingcap/dumpling v0.0.0-20210226040140-2e8afecad630 h1:d53sL21i/6qnVR github.com/pingcap/dumpling v0.0.0-20210226040140-2e8afecad630/go.mod h1:p4siH/MkS7xQRvtk/2+WwLGFMyFWTWAihheeZuWqTzY= github.com/pingcap/errcode v0.0.0-20180921232412-a1a7271709d9/go.mod h1:4b2X8xSqxIroj/IZ9MX/VGZhAwc11wB9wRIzHvz6SeM= github.com/pingcap/errcode v0.3.0/go.mod h1:4b2X8xSqxIroj/IZ9MX/VGZhAwc11wB9wRIzHvz6SeM= -github.com/pingcap/errors v0.11.0 h1:DCJQB8jrHbQ1VVlMFIrbj2ApScNNotVmkSNplu2yUt4= github.com/pingcap/errors v0.11.0/go.mod h1:Oi8TUi2kEtXXLMJk9l1cGmz20kV3TaQ0usTwv5KuLY8= github.com/pingcap/errors v0.11.1/go.mod h1:Oi8TUi2kEtXXLMJk9l1cGmz20kV3TaQ0usTwv5KuLY8= -github.com/pingcap/errors v0.11.4 h1:lFuQV/oaUMGcD2tqt+01ROSmJs75VG1ToEOkZIZ4nE4= github.com/pingcap/errors v0.11.4/go.mod h1:Oi8TUi2kEtXXLMJk9l1cGmz20kV3TaQ0usTwv5KuLY8= -github.com/pingcap/errors v0.11.5-0.20190809092503-95897b64e011 h1:58naV4XMEqm0hl9LcYo6cZoGBGiLtefMQMF/vo3XLgQ= github.com/pingcap/errors v0.11.5-0.20190809092503-95897b64e011/go.mod h1:Oi8TUi2kEtXXLMJk9l1cGmz20kV3TaQ0usTwv5KuLY8= -github.com/pingcap/errors v0.11.5-0.20200729012136-4e113ddee29e h1:/EGWHNOyEgizEBuAujWsb9vXrPZtt1b7ooDPyjEkjDw= github.com/pingcap/errors v0.11.5-0.20200729012136-4e113ddee29e/go.mod h1:g4vx//d6VakjJ0mk7iLBlKA8LFavV/sAVINT/1PFxeQ= -github.com/pingcap/errors v0.11.5-0.20200820035142-66eb5bf1d1cd h1:ay+wAVWHI/Z6vIik13hsK+FT9ZCNSPBElGr0qgiZpjg= github.com/pingcap/errors v0.11.5-0.20200820035142-66eb5bf1d1cd/go.mod h1:g4vx//d6VakjJ0mk7iLBlKA8LFavV/sAVINT/1PFxeQ= -github.com/pingcap/errors v0.11.5-0.20200902104258-eba4f1d8f6de h1:mW8hC2yXTpflfyTeJgcN4aJQfwcYODde8YgjBgAy6do= github.com/pingcap/errors v0.11.5-0.20200902104258-eba4f1d8f6de/go.mod h1:g4vx//d6VakjJ0mk7iLBlKA8LFavV/sAVINT/1PFxeQ= -github.com/pingcap/errors v0.11.5-0.20200917111840-a15ef68f753d h1:TH18wFO5Nq/zUQuWu9ms2urgZnLP69XJYiI2JZAkUGc= github.com/pingcap/errors v0.11.5-0.20200917111840-a15ef68f753d/go.mod h1:g4vx//d6VakjJ0mk7iLBlKA8LFavV/sAVINT/1PFxeQ= -github.com/pingcap/errors v0.11.5-0.20201029093017-5a7df2af2ac7 h1:wQKuKP2HUtej2gSvx1cZmY4DENUH6tlOxRkfvPT8EBU= github.com/pingcap/errors v0.11.5-0.20201029093017-5a7df2af2ac7/go.mod h1:G7x87le1poQzLB/TqvTJI2ILrSgobnq4Ut7luOwvfvI= github.com/pingcap/errors v0.11.5-0.20201126102027-b0a155152ca3 h1:LllgC9eGfqzkfubMgjKIDyZYaa609nNWAyNZtpy2B3M= github.com/pingcap/errors v0.11.5-0.20201126102027-b0a155152ca3/go.mod h1:G7x87le1poQzLB/TqvTJI2ILrSgobnq4Ut7luOwvfvI= -github.com/pingcap/failpoint v0.0.0-20191029060244-12f4ac2fd11d h1:F8vp38kTAckN+v8Jlc98uMBvKIzr1a+UhnLyVYn8Q5Q= github.com/pingcap/failpoint v0.0.0-20191029060244-12f4ac2fd11d/go.mod h1:DNS3Qg7bEDhU6EXNHF+XSv/PGznQaMJ5FWvctpm6pQI= -github.com/pingcap/failpoint v0.0.0-20200210140405-f8f9fb234798 h1:6DMbRqPI1qzQ8N1xc3+nKY8IxSACd9VqQKkRVvbyoIg= github.com/pingcap/failpoint v0.0.0-20200210140405-f8f9fb234798/go.mod h1:DNS3Qg7bEDhU6EXNHF+XSv/PGznQaMJ5FWvctpm6pQI= github.com/pingcap/failpoint v0.0.0-20200506114213-c17f16071c53/go.mod h1:w4PEZ5y16LeofeeGwdgZB4ddv9bLyDuIX+ljstgKZyk= -github.com/pingcap/failpoint v0.0.0-20200603062251-b230c36c413c h1:cm0zAj+Tab94mp4OH+VoLJiSNQvZO4pWDGJ8KEk2a0c= github.com/pingcap/failpoint v0.0.0-20200603062251-b230c36c413c/go.mod h1:w4PEZ5y16LeofeeGwdgZB4ddv9bLyDuIX+ljstgKZyk= github.com/pingcap/failpoint v0.0.0-20200702092429-9f69995143ce h1:Y1kCxlCtlPTMtVcOkjUcuQKh+YrluSo7+7YMCQSzy30= github.com/pingcap/failpoint v0.0.0-20200702092429-9f69995143ce/go.mod h1:w4PEZ5y16LeofeeGwdgZB4ddv9bLyDuIX+ljstgKZyk= -github.com/pingcap/fn v0.0.0-20191016082858-07623b84a47d h1:rCmRK0lCRrHMUbS99BKFYhK9YxJDNw0xB033cQbYo0s= github.com/pingcap/fn v0.0.0-20191016082858-07623b84a47d/go.mod h1:fMRU1BA1y+r89AxUoaAar4JjrhUkVDt0o0Np6V8XbDQ= github.com/pingcap/fn v0.0.0-20200306044125-d5540d389059 h1:Pe2LbxRmbTfAoKJ65bZLmhahmvHm7n9DUxGRQT00208= github.com/pingcap/fn v0.0.0-20200306044125-d5540d389059/go.mod h1:fMRU1BA1y+r89AxUoaAar4JjrhUkVDt0o0Np6V8XbDQ= @@ -841,7 +737,6 @@ github.com/pingcap/kvproto v0.0.0-20191211054548-3c6b38ea5107/go.mod h1:WWLmULLO github.com/pingcap/kvproto v0.0.0-20200214064158-62d31900d88e/go.mod h1:IOdRDPLyda8GX2hE/jO7gqaCV/PNFh8BZQCQZXfIOqI= github.com/pingcap/kvproto v0.0.0-20200411081810-b85805c9476c/go.mod h1:IOdRDPLyda8GX2hE/jO7gqaCV/PNFh8BZQCQZXfIOqI= github.com/pingcap/kvproto v0.0.0-20200417092353-efbe03bcffbd/go.mod h1:IOdRDPLyda8GX2hE/jO7gqaCV/PNFh8BZQCQZXfIOqI= -github.com/pingcap/kvproto v0.0.0-20200420075417-e0c6e8842f22 h1:D5EBGKd6o4A0PV0sUaUduPSCShiNi0OwFJmf+xRzpuI= github.com/pingcap/kvproto v0.0.0-20200420075417-e0c6e8842f22/go.mod h1:IOdRDPLyda8GX2hE/jO7gqaCV/PNFh8BZQCQZXfIOqI= github.com/pingcap/kvproto v0.0.0-20200423020121-038e31959c2a/go.mod h1:IOdRDPLyda8GX2hE/jO7gqaCV/PNFh8BZQCQZXfIOqI= github.com/pingcap/kvproto v0.0.0-20200424032552-6650270c39c3/go.mod h1:IOdRDPLyda8GX2hE/jO7gqaCV/PNFh8BZQCQZXfIOqI= @@ -849,29 +744,19 @@ github.com/pingcap/kvproto v0.0.0-20200428135407-0f5ffe459677/go.mod h1:IOdRDPLy github.com/pingcap/kvproto v0.0.0-20200518112156-d4aeb467de29/go.mod h1:IOdRDPLyda8GX2hE/jO7gqaCV/PNFh8BZQCQZXfIOqI= github.com/pingcap/kvproto v0.0.0-20200608081027-d02a6f65e956/go.mod h1:IOdRDPLyda8GX2hE/jO7gqaCV/PNFh8BZQCQZXfIOqI= github.com/pingcap/kvproto v0.0.0-20200701055533-4ef28cac01f8/go.mod h1:IOdRDPLyda8GX2hE/jO7gqaCV/PNFh8BZQCQZXfIOqI= -github.com/pingcap/kvproto v0.0.0-20200706115936-1e0910aabe6c h1:VnLpCAxMAeDxc7HXTetwDQB+/MtDQjHAOBsd4QnGVwA= github.com/pingcap/kvproto v0.0.0-20200706115936-1e0910aabe6c/go.mod h1:IOdRDPLyda8GX2hE/jO7gqaCV/PNFh8BZQCQZXfIOqI= github.com/pingcap/kvproto v0.0.0-20200715040832-c3e2e0b163ee/go.mod h1:IOdRDPLyda8GX2hE/jO7gqaCV/PNFh8BZQCQZXfIOqI= -github.com/pingcap/kvproto v0.0.0-20200803054707-ebd5de15093f h1:y247vlXfpe8MYGt6pkVKqIoYmx2KoeF8jmBLbAlse6Y= github.com/pingcap/kvproto v0.0.0-20200803054707-ebd5de15093f/go.mod h1:IOdRDPLyda8GX2hE/jO7gqaCV/PNFh8BZQCQZXfIOqI= -github.com/pingcap/kvproto v0.0.0-20200810113304-6157337686b1 h1:hv22UEhdqeIqa5Jx0oeqDQNGHUBSW3LVOx02gqbd5Gg= github.com/pingcap/kvproto v0.0.0-20200810113304-6157337686b1/go.mod h1:IOdRDPLyda8GX2hE/jO7gqaCV/PNFh8BZQCQZXfIOqI= github.com/pingcap/kvproto v0.0.0-20200827082727-23dedec2339b/go.mod h1:IOdRDPLyda8GX2hE/jO7gqaCV/PNFh8BZQCQZXfIOqI= -github.com/pingcap/kvproto v0.0.0-20200828054126-d677e6fd224a h1:/FW9upn9/MJ8e6SdiGZcJ4MeZLXdYUG+L5RHp1OhPR4= github.com/pingcap/kvproto v0.0.0-20200828054126-d677e6fd224a/go.mod h1:IOdRDPLyda8GX2hE/jO7gqaCV/PNFh8BZQCQZXfIOqI= -github.com/pingcap/kvproto v0.0.0-20201113092725-08f2872278eb h1:K3r4KjVQeD4nLnfj44ibdLIXnUh58aQpkgVNWuBO9z0= github.com/pingcap/kvproto v0.0.0-20201113092725-08f2872278eb/go.mod h1:IOdRDPLyda8GX2hE/jO7gqaCV/PNFh8BZQCQZXfIOqI= -github.com/pingcap/kvproto v0.0.0-20210125053538-4fb47d0584f2 h1:VloKJFPtnH9e75gZfhzs5RUMxeeRYCMmiYp+01b2WHI= github.com/pingcap/kvproto v0.0.0-20210125053538-4fb47d0584f2/go.mod h1:IOdRDPLyda8GX2hE/jO7gqaCV/PNFh8BZQCQZXfIOqI= github.com/pingcap/kvproto v0.0.0-20210204054616-1c1ed89bb167 h1:a9nvMHVtsKMXMHxJzCt4vwxf3wD6FD7VSTQQjfhQ11E= github.com/pingcap/kvproto v0.0.0-20210204054616-1c1ed89bb167/go.mod h1:IOdRDPLyda8GX2hE/jO7gqaCV/PNFh8BZQCQZXfIOqI= -github.com/pingcap/log v0.0.0-20191012051959-b742a5d432e9 h1:AJD9pZYm72vMgPcQDww9rkZ1DnWfl0pXV3BOWlkYIjA= github.com/pingcap/log v0.0.0-20191012051959-b742a5d432e9/go.mod h1:4rbK1p9ILyIfb6hU7OG2CiWSqMXnp3JMbiaVJ6mvoY8= -github.com/pingcap/log v0.0.0-20200117041106-d28c14d3b1cd h1:CV3VsP3Z02MVtdpTMfEgRJ4T9NGgGTxdHpJerent7rM= github.com/pingcap/log v0.0.0-20200117041106-d28c14d3b1cd/go.mod h1:4rbK1p9ILyIfb6hU7OG2CiWSqMXnp3JMbiaVJ6mvoY8= -github.com/pingcap/log v0.0.0-20200511115504-543df19646ad h1:SveG82rmu/GFxYanffxsSF503SiQV+2JLnWEiGiF+Tc= github.com/pingcap/log v0.0.0-20200511115504-543df19646ad/go.mod h1:4rbK1p9ILyIfb6hU7OG2CiWSqMXnp3JMbiaVJ6mvoY8= -github.com/pingcap/log v0.0.0-20200828042413-fce0951f1463 h1:Jboj+s4jSCp5E1WDgmRUv5rIFKFHaaSWuSZ4wMwXIcc= github.com/pingcap/log v0.0.0-20200828042413-fce0951f1463/go.mod h1:4rbK1p9ILyIfb6hU7OG2CiWSqMXnp3JMbiaVJ6mvoY8= github.com/pingcap/log v0.0.0-20201112100606-8f1e84a3abc8 h1:M+DNpOu/I3uDmwee6vcnoPd6GgSMqND4gxvDQ/W584U= github.com/pingcap/log v0.0.0-20201112100606-8f1e84a3abc8/go.mod h1:4rbK1p9ILyIfb6hU7OG2CiWSqMXnp3JMbiaVJ6mvoY8= @@ -887,27 +772,20 @@ github.com/pingcap/parser v0.0.0-20200730092557-34a468e9b774/go.mod h1:vQdbJqobJ github.com/pingcap/parser v0.0.0-20200731033026-84f62115187c/go.mod h1:vQdbJqobJAgFyiRNNtXahpMoGWwPEuWciVEK5A20NS0= github.com/pingcap/parser v0.0.0-20200813083329-a4bff035d3e2/go.mod h1:vQdbJqobJAgFyiRNNtXahpMoGWwPEuWciVEK5A20NS0= github.com/pingcap/parser v0.0.0-20200821073936-cf85e80665c4/go.mod h1:vQdbJqobJAgFyiRNNtXahpMoGWwPEuWciVEK5A20NS0= -github.com/pingcap/parser v0.0.0-20200924053142-5d7e8ebf605e h1:IFD2pEbIcN+EzG/RGMLrv/Tt6U9KzJGT6hSbGkQ1v7c= github.com/pingcap/parser v0.0.0-20200924053142-5d7e8ebf605e/go.mod h1:RlLfMRJwFBSiXd2lUaWdV5pSXtrpyvZM8k5bbZWsheU= github.com/pingcap/parser v0.0.0-20210125075924-ffe0fda947cb/go.mod h1:GbEr2PgY72/4XqPZzmzstlOU/+il/wrjeTNFs6ihsSE= github.com/pingcap/parser v0.0.0-20210203141130-32ef3e0e2e13/go.mod h1:GbEr2PgY72/4XqPZzmzstlOU/+il/wrjeTNFs6ihsSE= github.com/pingcap/parser v0.0.0-20210224050355-ce3c7711a45f h1:pifEcAWoLMCwNcaUuXWyDOjmh1MZKnkffC+WTTmAr6A= github.com/pingcap/parser v0.0.0-20210224050355-ce3c7711a45f/go.mod h1:GbEr2PgY72/4XqPZzmzstlOU/+il/wrjeTNFs6ihsSE= github.com/pingcap/pd v2.1.5+incompatible/go.mod h1:nD3+EoYes4+aNNODO99ES59V83MZSI+dFbhyr667a0E= -github.com/pingcap/pd/v4 v4.0.0-rc.1.0.20200422143320-428acd53eba2 h1:JTzYYukREvxVSKW/ncrzNjFitd8snoQ/Xz32pw8i+s8= github.com/pingcap/pd/v4 v4.0.0-rc.1.0.20200422143320-428acd53eba2/go.mod h1:s+utZtXDznOiL24VK0qGmtoHjjXNsscJx3m1n8cC56s= github.com/pingcap/pd/v4 v4.0.0-rc.2.0.20200520083007-2c251bd8f181/go.mod h1:q4HTx/bA8aKBa4S7L+SQKHvjRPXCRV0tA0yRw0qkZSA= -github.com/pingcap/pd/v4 v4.0.0-rc.2.0.20200714122454-1a64f969cb3c h1:aOiyGetA256/LUkdmhny0Q/PWTBQiF/TPNhJuJMGRSY= github.com/pingcap/pd/v4 v4.0.0-rc.2.0.20200714122454-1a64f969cb3c/go.mod h1:v/dY4mVkt3dh/Liphhk0E4ScOkZpIk0m0GvWJ9FapDs= github.com/pingcap/pd/v4 v4.0.0-rc.2.0.20200715061836-4971825321cf/go.mod h1:Ikvk5tw09nV6ZR4vHgrmAIAbZJttZ2cKQvOjN5lekBk= -github.com/pingcap/pd/v4 v4.0.0-rc.2.0.20200730093003-dc8c75cf7ca0 h1:cSHKKU5Tt4oI+Bky4KCnLr8prM/4HS0S0LWtVOm9bh4= github.com/pingcap/pd/v4 v4.0.0-rc.2.0.20200730093003-dc8c75cf7ca0/go.mod h1:szYFB2rf8yrSGJuI8hm9RLWvsK+xt1exLTj511WPCnE= github.com/pingcap/sysutil v0.0.0-20200206130906-2bfa6dc40bcd/go.mod h1:EB/852NMQ+aRKioCpToQ94Wl7fktV+FNnxf3CX/TTXI= -github.com/pingcap/sysutil v0.0.0-20200408114249-ed3bd6f7fdb1 h1:PI8YpTl45F8ilNkrPtT4IdbcZB1SCEa+gK/U5GJYl3E= github.com/pingcap/sysutil v0.0.0-20200408114249-ed3bd6f7fdb1/go.mod h1:EB/852NMQ+aRKioCpToQ94Wl7fktV+FNnxf3CX/TTXI= -github.com/pingcap/sysutil v0.0.0-20200715082929-4c47bcac246a h1:i2RElJ2aykSqZKeY+3SK18NHhajil8cQdG77wHe+P1Y= github.com/pingcap/sysutil v0.0.0-20200715082929-4c47bcac246a/go.mod h1:EB/852NMQ+aRKioCpToQ94Wl7fktV+FNnxf3CX/TTXI= -github.com/pingcap/sysutil v0.0.0-20201021075216-f93ced2829e2 h1:b2G/eqDeywtdJF3w9nIUdqMmXChsmpLvf4FzUxJ9Vmk= github.com/pingcap/sysutil v0.0.0-20201021075216-f93ced2829e2/go.mod h1:EB/852NMQ+aRKioCpToQ94Wl7fktV+FNnxf3CX/TTXI= github.com/pingcap/sysutil v0.0.0-20201130064824-f0c8aa6a6966 h1:JI0wOAb8aQML0vAVLHcxTEEC0VIwrk6gtw3WjbHvJLA= github.com/pingcap/sysutil v0.0.0-20201130064824-f0c8aa6a6966/go.mod h1:EB/852NMQ+aRKioCpToQ94Wl7fktV+FNnxf3CX/TTXI= @@ -927,9 +805,7 @@ github.com/pingcap/tidb v1.1.0-beta.0.20200806060043-574540aa06ba/go.mod h1:NHcZ github.com/pingcap/tidb v1.1.0-beta.0.20200810064414-d81150394f9d/go.mod h1:vLYo4E7Q6kzKYTskhP2MHBsodmZIRRUU63qdiFjlULA= github.com/pingcap/tidb v1.1.0-beta.0.20200820085534-0d997f2b8b3c/go.mod h1:z7Hn1KY8Crt9cHhWtbGPKMBcjvmSJXIoOjO4rMk165w= github.com/pingcap/tidb v1.1.0-beta.0.20200831085451-438945d2948e/go.mod h1:VXxiC2f+HY3/5phR1841YJrX4on56kTEXrtEzRezcj4= -github.com/pingcap/tidb v1.1.0-beta.0.20200927065602-486e473a86e9 h1:m+xtcIZspdIwEV1sD2GKb9X7VXjH0rcqe6xVSM7HmIo= github.com/pingcap/tidb v1.1.0-beta.0.20200927065602-486e473a86e9/go.mod h1:6eExMFLTdLTVWGZTikUCwalifvY+78PtqGwQB3YkmSo= -github.com/pingcap/tidb v1.1.0-beta.0.20210129045644-ea6ccf82e934 h1:Ib2e/WNwHJDTC20tcakjHwOWz0uFmXqyqCu54/537ps= github.com/pingcap/tidb v1.1.0-beta.0.20210129045644-ea6ccf82e934/go.mod h1:+UTK0n9hqXRpS0QEX7RRsahxZhQ6qJCIA9Efiz3BC8E= github.com/pingcap/tidb v1.1.0-beta.0.20210224053248-b226130926a2 h1:cPFc4GaanyZjsOrgfUkQCj6HIGdB4Yrb+mxSgzbtXxs= github.com/pingcap/tidb v1.1.0-beta.0.20210224053248-b226130926a2/go.mod h1:veuTMrHXY5NtJsGtmfxymjo5obL244ifJBfywFPBaEo= @@ -942,26 +818,20 @@ github.com/pingcap/tidb-tools v4.0.0+incompatible/go.mod h1:XGdcy9+yqlDSEMTpOXnw github.com/pingcap/tidb-tools v4.0.1+incompatible/go.mod h1:XGdcy9+yqlDSEMTpOXnwf3hiTeqrV6MN/u1se9N8yIM= github.com/pingcap/tidb-tools v4.0.5-0.20200820082341-afeaaaaaa153+incompatible/go.mod h1:XGdcy9+yqlDSEMTpOXnwf3hiTeqrV6MN/u1se9N8yIM= github.com/pingcap/tidb-tools v4.0.5-0.20200820092506-34ea90c93237+incompatible/go.mod h1:XGdcy9+yqlDSEMTpOXnwf3hiTeqrV6MN/u1se9N8yIM= -github.com/pingcap/tidb-tools v4.0.9-0.20201127090955-2707c97b3853+incompatible h1:ceznmu/lLseGHP/jKyOa/3u/5H3wtLLLqkH2V3ssSjg= github.com/pingcap/tidb-tools v4.0.9-0.20201127090955-2707c97b3853+incompatible/go.mod h1:XGdcy9+yqlDSEMTpOXnwf3hiTeqrV6MN/u1se9N8yIM= github.com/pingcap/tidb-tools v5.0.0-rc.0.20210310030049-c82efd92f571+incompatible h1:aUnAWPMM5gs7pQ/ef2UqrgATMoLZkCrwFq6rfU8EvuA= github.com/pingcap/tidb-tools v5.0.0-rc.0.20210310030049-c82efd92f571+incompatible/go.mod h1:XGdcy9+yqlDSEMTpOXnwf3hiTeqrV6MN/u1se9N8yIM= -github.com/pingcap/tipb v0.0.0-20190428032612-535e1abaa330 h1:rRMLMjIMFulCX9sGKZ1hoov/iROMsKyC8Snc02nSukw= github.com/pingcap/tipb v0.0.0-20190428032612-535e1abaa330/go.mod h1:RtkHW8WbcNxj8lsbzjaILci01CtYnYbIkQhjyZWrWVI= -github.com/pingcap/tipb v0.0.0-20200417094153-7316d94df1ee h1:XJQ6/LGzOSc/jo33AD8t7jtc4GohxcyODsYnb+kZXJM= github.com/pingcap/tipb v0.0.0-20200417094153-7316d94df1ee/go.mod h1:RtkHW8WbcNxj8lsbzjaILci01CtYnYbIkQhjyZWrWVI= github.com/pingcap/tipb v0.0.0-20200604070248-508f03b0b342/go.mod h1:RtkHW8WbcNxj8lsbzjaILci01CtYnYbIkQhjyZWrWVI= github.com/pingcap/tipb v0.0.0-20200615034523-dcfcea0b5965/go.mod h1:RtkHW8WbcNxj8lsbzjaILci01CtYnYbIkQhjyZWrWVI= -github.com/pingcap/tipb v0.0.0-20200618092958-4fad48b4c8c3 h1:ESL3eIt1kUt8IMvR1011ejZlAyDcOzw89ARvVHvpD5k= github.com/pingcap/tipb v0.0.0-20200618092958-4fad48b4c8c3/go.mod h1:RtkHW8WbcNxj8lsbzjaILci01CtYnYbIkQhjyZWrWVI= -github.com/pingcap/tipb v0.0.0-20201229060814-148bc717ce4c h1:kvrdp2hY+asgSvVXCj4eebA9DH4SSouRVQUZpa1Se/Y= github.com/pingcap/tipb v0.0.0-20201229060814-148bc717ce4c/go.mod h1:RtkHW8WbcNxj8lsbzjaILci01CtYnYbIkQhjyZWrWVI= github.com/pingcap/tipb v0.0.0-20210204051656-2870a0852037 h1:FVIyv52hHnkhWX7FIUCrfXC5BBDo+yaGX2+w5lV65Xs= github.com/pingcap/tipb v0.0.0-20210204051656-2870a0852037/go.mod h1:nsEhnMokcn7MRqd2J60yxpn/ac3ZH8A6GOJ9NslabUo= github.com/pingcap/tiup v1.2.3/go.mod h1:q8WzflNHjE1U49k2qstTL0clx2pKh8pkOzUFV4RTvQo= github.com/pkg/browser v0.0.0-20180916011732-0a3d74bf9ce4/go.mod h1:4OwLy04Bl9Ef3GJJCoec+30X3LQs/0/m4HFRt/2LUSA= github.com/pkg/errors v0.8.0/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0= -github.com/pkg/errors v0.8.1 h1:iURUrRGxPUNPdy5/HRSm+Yj6okJ6UtLINN0Q9M4+h3I= github.com/pkg/errors v0.8.1/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0= github.com/pkg/errors v0.9.1 h1:FEBLx1zS214owpjy7qsBeixbURkuhQAwrK5UwLGTwt4= github.com/pkg/errors v0.9.1/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0= @@ -970,7 +840,6 @@ github.com/pmezard/go-difflib v1.0.0 h1:4DBwDE0NGyQoBHbLQYPwSUPoCMWR5BEzIk/f1lZb github.com/pmezard/go-difflib v1.0.0/go.mod h1:iKH77koFhYxTK1pcRnkKkqfTogsbg7gZNVY4sRDYZ/4= github.com/prestodb/presto-go-client v0.0.0-20191203220353-4980913e2459/go.mod h1:psTrT5qzpQmYNpaWiyurHCiVTLMYYpAd5o0YQAJWpQg= github.com/prometheus/client_golang v0.9.0/go.mod h1:7SWBe2y4D6OKWSNQJUaRYU/AaXPKyh/dDVn+NZz0KFw= -github.com/prometheus/client_golang v0.9.1 h1:K47Rk0v/fkEfwfQet2KWhscE0cJzjgCCDBG2KHZoVno= github.com/prometheus/client_golang v0.9.1/go.mod h1:7SWBe2y4D6OKWSNQJUaRYU/AaXPKyh/dDVn+NZz0KFw= github.com/prometheus/client_golang v0.9.2/go.mod h1:OsXs2jCmiKlQ1lTBmv21f2mNfw4xf/QclQDMrYNZzcM= github.com/prometheus/client_golang v0.9.3/go.mod h1:/TN21ttK/J9q6uSwhBd54HahCDft0ttaMvbicHlPoso= @@ -978,11 +847,8 @@ github.com/prometheus/client_golang v1.0.0/go.mod h1:db9x61etRT2tGnBNRi70OPL5Fsn github.com/prometheus/client_golang v1.2.1/go.mod h1:XMU6Z2MjaRKVu/dC1qupJI9SiNkDYzz3xecMgSW/F+U= github.com/prometheus/client_golang v1.5.1 h1:bdHYieyGlH+6OLEk2YQha8THib30KP0/yD0YH9m6xcA= github.com/prometheus/client_golang v1.5.1/go.mod h1:e9GMxYsXl05ICDXkRhurwBS4Q3OK1iX/F2sw+iXX5zU= -github.com/prometheus/client_model v0.0.0-20180712105110-5c3871d89910 h1:idejC8f05m9MGOsuEi1ATq9shN03HrxNkD/luQvxCv8= github.com/prometheus/client_model v0.0.0-20180712105110-5c3871d89910/go.mod h1:MbSGuTsp3dbXC40dX6PRTWyKYBIrTGTE9sqQNg2J8bo= -github.com/prometheus/client_model v0.0.0-20190129233127-fd36f4220a90 h1:S/YWwWx/RA8rT8tKFRuGUZhuA90OyIBpPCXkcbwU8DE= github.com/prometheus/client_model v0.0.0-20190129233127-fd36f4220a90/go.mod h1:xMI15A0UPsDsEKsMN9yxemIoYk6Tm2C1GtYGdfGttqA= -github.com/prometheus/client_model v0.0.0-20190812154241-14fe0d1b01d4 h1:gQz4mCbXsO+nc9n1hCxHcGA3Zx3Eo+UHZoInFGUIXNM= github.com/prometheus/client_model v0.0.0-20190812154241-14fe0d1b01d4/go.mod h1:xMI15A0UPsDsEKsMN9yxemIoYk6Tm2C1GtYGdfGttqA= github.com/prometheus/client_model v0.1.0/go.mod h1:xMI15A0UPsDsEKsMN9yxemIoYk6Tm2C1GtYGdfGttqA= github.com/prometheus/client_model v0.2.0 h1:uq5h0d+GuxiXLJLNABMgp2qUWDPiLvgCzz2dUR+/W/M= @@ -991,7 +857,6 @@ github.com/prometheus/common v0.0.0-20181020173914-7e9e6cabbd39/go.mod h1:daVV7q github.com/prometheus/common v0.0.0-20181113130724-41aa239b4cce/go.mod h1:daVV7qP5qjZbuso7PdcryaAu0sAZbrN9i7WWcTMWvro= github.com/prometheus/common v0.0.0-20181126121408-4724e9255275/go.mod h1:daVV7qP5qjZbuso7PdcryaAu0sAZbrN9i7WWcTMWvro= github.com/prometheus/common v0.4.0/go.mod h1:TNfzLD0ON7rHzMJeJkieUDPYmFC7Snx/y86RQel1bk4= -github.com/prometheus/common v0.4.1 h1:K0MGApIoQvMw27RTdJkPbr3JZ7DNbtxQNyi5STVM6Kw= github.com/prometheus/common v0.4.1/go.mod h1:TNfzLD0ON7rHzMJeJkieUDPYmFC7Snx/y86RQel1bk4= github.com/prometheus/common v0.6.0/go.mod h1:eBmuwkDJBwy6iBfxCBob6t6dR6ENT/y+J+Zk0j9GMYc= github.com/prometheus/common v0.7.0/go.mod h1:DjGbpBbp5NYNiECxcL/VnbXCCaQpKd3tt26CguLLsqA= @@ -1000,7 +865,6 @@ github.com/prometheus/common v0.9.1/go.mod h1:yhUN8i9wzaXS3w1O07YhxHEBxD+W35wd8b github.com/prometheus/procfs v0.0.0-20181005140218-185b4288413d/go.mod h1:c3At6R/oaqEKCNdg8wHV1ftS6bRYblBhIjjI8uT2IGk= github.com/prometheus/procfs v0.0.0-20181204211112-1dc9a6cbc91a/go.mod h1:c3At6R/oaqEKCNdg8wHV1ftS6bRYblBhIjjI8uT2IGk= github.com/prometheus/procfs v0.0.0-20190507164030-5867b95ac084/go.mod h1:TjEm7ze935MbeOT/UhFTIMYKhuLP4wbCsTZCD3I8kEA= -github.com/prometheus/procfs v0.0.2 h1:6LJUbpNm42llc4HRCuvApCSWB/WfhuNo9K98Q9sNGfs= github.com/prometheus/procfs v0.0.2/go.mod h1:TjEm7ze935MbeOT/UhFTIMYKhuLP4wbCsTZCD3I8kEA= github.com/prometheus/procfs v0.0.5/go.mod h1:4A/X28fw3Fc593LaREMrKMqOKvUAntwMDaekg4FpcdQ= github.com/prometheus/procfs v0.0.6/go.mod h1:7Qr8sr6344vo1JqZ6HhLceV9o3AJ1Ff+GxbHq6oeK9A= @@ -1015,7 +879,6 @@ github.com/rcrowley/go-metrics v0.0.0-20190826022208-cac0b30c2563/go.mod h1:bCqn github.com/relex/aini v1.2.0/go.mod h1:oFQyhvkzwi8GChiLukpBHkV2v142ls2L1MTeOSD2vic= github.com/remyoudompheng/bigfft v0.0.0-20170806203942-52369c62f446/go.mod h1:uYEyJGbgTkfkS4+E/PavXkNJcbFIpEtjt2B0KDQ5+9M= github.com/remyoudompheng/bigfft v0.0.0-20190512091148-babf20351dd7/go.mod h1:qqbHyh8v60DhA7CoWK5oRCqLrMHRGoxYCSS9EjAz6Eo= -github.com/remyoudompheng/bigfft v0.0.0-20190728182440-6a916e37a237 h1:HQagqIiBmr8YXawX/le3+O26N+vPPC1PtjaF3mwnook= github.com/remyoudompheng/bigfft v0.0.0-20190728182440-6a916e37a237/go.mod h1:qqbHyh8v60DhA7CoWK5oRCqLrMHRGoxYCSS9EjAz6Eo= github.com/remyoudompheng/bigfft v0.0.0-20200410134404-eec4a21b6bb0 h1:OdAsTTz6OkFY5QxjkYwrChwuRruF69c169dPK26NUlk= github.com/remyoudompheng/bigfft v0.0.0-20200410134404-eec4a21b6bb0/go.mod h1:qqbHyh8v60DhA7CoWK5oRCqLrMHRGoxYCSS9EjAz6Eo= @@ -1024,59 +887,42 @@ github.com/rogpeppe/fastuuid v0.0.0-20150106093220-6724a57986af/go.mod h1:XWv6So github.com/rogpeppe/fastuuid v1.2.0/go.mod h1:jVj6XXZzXRy/MSR5jhDC/2q6DgLz+nrA6LYCDYWNEvQ= github.com/rogpeppe/go-internal v1.3.0/go.mod h1:M8bDsm7K2OlrFYOpmOWEs/qY81heoFRclV5y23lUDJ4= github.com/rs/cors v1.7.0/go.mod h1:gFx+x8UowdsKA9AchylcLynDq+nNFfI8FkUZdN/jGCU= -github.com/russross/blackfriday v1.5.2 h1:HyvC0ARfnZBqnXwABFeSZHpKvJHJJfPz81GNueLj0oo= github.com/russross/blackfriday v1.5.2/go.mod h1:JO/DiYxRf+HjHt06OyowR9PTA263kcR/rfWxYHBV53g= -github.com/russross/blackfriday/v2 v2.0.1 h1:lPqVAte+HuHNfhJ/0LC98ESWRz8afy9tM/0RK8m9o+Q= github.com/russross/blackfriday/v2 v2.0.1/go.mod h1:+Rmxgy9KzJVeS9/2gXHxylqXiyQDYRxCVz55jmeOWTM= -github.com/sasha-s/go-deadlock v0.2.0 h1:lMqc+fUb7RrFS3gQLtoQsJ7/6TV/pAIFvBsqX73DK8Y= github.com/sasha-s/go-deadlock v0.2.0/go.mod h1:StQn567HiB1fF2yJ44N9au7wOhrPS3iZqiDbRupzT10= -github.com/satori/go.uuid v1.2.0 h1:0uYX9dsZ2yD7q2RtLRtPSdGDWzjeM3TbMJP9utgA0ww= github.com/satori/go.uuid v1.2.0/go.mod h1:dA0hQrYB0VpLJoorglMZABFdXlWrHn1NEOzdhQKdks0= github.com/satori/go.uuid v1.2.1-0.20181028125025-b2ce2384e17b h1:gQZ0qzfKHQIybLANtM3mBXNUtOfsCFXeTsnBqCsx1KM= github.com/satori/go.uuid v1.2.1-0.20181028125025-b2ce2384e17b/go.mod h1:dA0hQrYB0VpLJoorglMZABFdXlWrHn1NEOzdhQKdks0= github.com/segmentio/kafka-go v0.1.0/go.mod h1:X6itGqS9L4jDletMsxZ7Dz+JFWxM6JHfPOCvTvk+EJo= github.com/segmentio/kafka-go v0.2.0/go.mod h1:X6itGqS9L4jDletMsxZ7Dz+JFWxM6JHfPOCvTvk+EJo= github.com/sergi/go-diff v1.0.0/go.mod h1:0CfEIISq7TuYL3j771MWULgwwjU+GofnZX9QAmXWZgo= -github.com/sergi/go-diff v1.0.1-0.20180205163309-da645544ed44 h1:tB9NOR21++IjLyVx3/PCPhWMwqGNCMQEH96A6dMZ/gc= github.com/sergi/go-diff v1.0.1-0.20180205163309-da645544ed44/go.mod h1:0CfEIISq7TuYL3j771MWULgwwjU+GofnZX9QAmXWZgo= -github.com/shirou/gopsutil v2.19.10+incompatible h1:lA4Pi29JEVIQIgATSeftHSY0rMGI9CLrl2ZvDLiahto= github.com/shirou/gopsutil v2.19.10+incompatible/go.mod h1:5b4v6he4MtMOwMlS0TUMTu2PcXUg8+E1lC7eC3UO/RA= -github.com/shirou/gopsutil v2.20.3+incompatible h1:0JVooMPsT7A7HqEYdydp/OfjSOYSjhXV7w1hkKj/NPQ= github.com/shirou/gopsutil v2.20.3+incompatible/go.mod h1:5b4v6he4MtMOwMlS0TUMTu2PcXUg8+E1lC7eC3UO/RA= -github.com/shirou/gopsutil v2.20.6+incompatible h1:P37G9YH8M4vqkKcwBosp+URN5O8Tay67D2MbR361ioY= github.com/shirou/gopsutil v2.20.6+incompatible/go.mod h1:5b4v6he4MtMOwMlS0TUMTu2PcXUg8+E1lC7eC3UO/RA= github.com/shirou/gopsutil v3.20.12+incompatible h1:6VEGkOXP/eP4o2Ilk8cSsX0PhOEfX6leqAnD+urrp9M= github.com/shirou/gopsutil v3.20.12+incompatible/go.mod h1:5b4v6he4MtMOwMlS0TUMTu2PcXUg8+E1lC7eC3UO/RA= -github.com/shirou/w32 v0.0.0-20160930032740-bb4de0191aa4 h1:udFKJ0aHUL60LboW/A+DfgoHVedieIzIXE8uylPue0U= github.com/shirou/w32 v0.0.0-20160930032740-bb4de0191aa4/go.mod h1:qsXQc7+bwAM3Q1u/4XEfrquwF8Lw7D7y5cD8CuHnfIc= -github.com/shopspring/decimal v0.0.0-20180709203117-cd690d0c9e24 h1:pntxY8Ary0t43dCZ5dqY4YTJCObLY1kIXl0uzMv+7DE= github.com/shopspring/decimal v0.0.0-20180709203117-cd690d0c9e24/go.mod h1:M+9NzErvs504Cn4c5DxATwIqPbtswREoFCre64PpcG4= -github.com/shopspring/decimal v0.0.0-20191125035519-b054a8dfd10d h1:976xhcFOjbSk2cmjzMkzePBLTAPkErAI5x/J6hsAEmw= github.com/shopspring/decimal v0.0.0-20191125035519-b054a8dfd10d/go.mod h1:M+9NzErvs504Cn4c5DxATwIqPbtswREoFCre64PpcG4= github.com/shopspring/decimal v0.0.0-20200105231215-408a2507e114 h1:Pm6R878vxWWWR+Sa3ppsLce/Zq+JNTs6aVvRu13jv9A= github.com/shopspring/decimal v0.0.0-20200105231215-408a2507e114/go.mod h1:DKyhrW/HYNuLGql+MJL6WCR6knT2jwCFRcu2hWCYk4o= github.com/shurcooL/httpfs v0.0.0-20190707220628-8d4bc4ba7749 h1:bUGsEnyNbVPw06Bs80sCeARAlK8lhwqGyi6UT8ymuGk= github.com/shurcooL/httpfs v0.0.0-20190707220628-8d4bc4ba7749/go.mod h1:ZY1cvUeJuFPAdZ/B6v7RHavJWZn2YPVFQ1OSXhCGOkg= -github.com/shurcooL/httpgzip v0.0.0-20190720172056-320755c1c1b0 h1:mj/nMDAwTBiaCqMEs4cYCqF7pO6Np7vhy1D1wcQGz+E= github.com/shurcooL/httpgzip v0.0.0-20190720172056-320755c1c1b0/go.mod h1:919LwcH0M7/W4fcZ0/jy0qGght1GIhqyS/EgWGH2j5Q= -github.com/shurcooL/sanitized_anchor_name v1.0.0 h1:PdmoCO6wvbs+7yrJyMORt4/BmY5IYyJwS/kOiWx8mHo= github.com/shurcooL/sanitized_anchor_name v1.0.0/go.mod h1:1NzhyTcUVG4SuEtjjoZeVRXNmyL/1OwPU0+IJeTBvfc= -github.com/shurcooL/vfsgen v0.0.0-20181020040650-a97a25d856ca h1:3fECS8atRjByijiI8yYiuwLwQ2ZxXobW7ua/8GRB3pI= github.com/shurcooL/vfsgen v0.0.0-20181020040650-a97a25d856ca/go.mod h1:TrYk7fJVaAttu97ZZKrO9UbRa8izdowaMIZcxYMbVaw= github.com/shurcooL/vfsgen v0.0.0-20181202132449-6a9ea43bcacd h1:ug7PpSOB5RBPK1Kg6qskGBoP3Vnj/aNYFTznWvlkGo0= github.com/shurcooL/vfsgen v0.0.0-20181202132449-6a9ea43bcacd/go.mod h1:TrYk7fJVaAttu97ZZKrO9UbRa8izdowaMIZcxYMbVaw= github.com/siddontang/go v0.0.0-20180604090527-bdc77568d726 h1:xT+JlYxNGqyT+XcU8iUrN18JYed2TvG9yN5ULG2jATM= github.com/siddontang/go v0.0.0-20180604090527-bdc77568d726/go.mod h1:3yhqj7WBBfRhbBlzyOC3gUxftwsU0u8gqevxwIHQpMw= -github.com/siddontang/go-log v0.0.0-20180807004314-8d05993dda07 h1:oI+RNwuC9jF2g2lP0u0cVEEZrc/AYBCuFdvwrLWM/6Q= github.com/siddontang/go-log v0.0.0-20180807004314-8d05993dda07/go.mod h1:yFdBgwXP24JziuRl2NMUahT7nGLNOKi1SIiFxMttVD4= github.com/siddontang/go-log v0.0.0-20190221022429-1e957dd83bed h1:KMgQoLJGCq1IoZpLZE3AIffh9veYWoVlsvA4ib55TMM= github.com/siddontang/go-log v0.0.0-20190221022429-1e957dd83bed/go.mod h1:yFdBgwXP24JziuRl2NMUahT7nGLNOKi1SIiFxMttVD4= -github.com/siddontang/go-mysql v0.0.0-20200222075837-12e89848f047 h1:boyJ8EgQN/aC3grvx8QUoJrptt7RvneezSJSCbW25a4= github.com/siddontang/go-mysql v0.0.0-20200222075837-12e89848f047/go.mod h1:+W4RCzesQDI11HvIkaDjS8yM36SpAnGNQ7jmTLn5BnU= github.com/sirupsen/logrus v1.2.0/go.mod h1:LxeOpSwHxABJmUn/MG1IvRgCAasNZTLOkJPxbbu5VWo= github.com/sirupsen/logrus v1.3.0/go.mod h1:LxeOpSwHxABJmUn/MG1IvRgCAasNZTLOkJPxbbu5VWo= github.com/sirupsen/logrus v1.4.1/go.mod h1:ni0Sbl8bgC9z8RoU9G6nDWqqs/fq4eDPysMBDgk/93Q= -github.com/sirupsen/logrus v1.4.2 h1:SPIRibHv4MatM3XXNO2BJeFLZwZ2LvZgfQ5+UNI2im4= github.com/sirupsen/logrus v1.4.2/go.mod h1:tLMulIdttU9McNUspp0xgXVQah82FyeX6MwdIuYE2rE= github.com/sirupsen/logrus v1.6.0 h1:UBcNElsrwanuuMsnGSlYmtmgbb23qDR5dG+6X6Oo89I= github.com/sirupsen/logrus v1.6.0/go.mod h1:7uNnSEd1DgxDLC74fIahvMZmmYsHGZGEOFrfsX/uA88= @@ -1087,7 +933,6 @@ github.com/smartystreets/goconvey v1.6.4/go.mod h1:syvi0/a8iFYH4r/RixwvyeAJjdLS9 github.com/snowflakedb/gosnowflake v1.3.4/go.mod h1:NsRq2QeiMUuoNUJhp5Q6xGC4uBrsS9g6LwZVEkTWgsE= github.com/soheilhy/cmux v0.1.4 h1:0HKaf1o97UwFjHH9o5XsHUOF+tqmdA7KEzXLpiyaw0E= github.com/soheilhy/cmux v0.1.4/go.mod h1:IM3LyeVVIOuxMH7sFAkER9+bJ4dT7Ms6E4xg4kGIyLM= -github.com/spaolacci/murmur3 v0.0.0-20180118202830-f09979ecbc72 h1:qLC7fQah7D6K1B0ujays3HV9gkFtllcxhzImRR7ArPQ= github.com/spaolacci/murmur3 v0.0.0-20180118202830-f09979ecbc72/go.mod h1:JwIasOWyU6f++ZhiEuf87xNszmSA2myDM2Kzu9HwQUA= github.com/spaolacci/murmur3 v1.1.0 h1:7c1g84S4BPRrfL5Xrdp6fOJ206sU9y293DDHaoy0bLI= github.com/spaolacci/murmur3 v1.1.0/go.mod h1:JwIasOWyU6f++ZhiEuf87xNszmSA2myDM2Kzu9HwQUA= @@ -1100,7 +945,6 @@ github.com/spf13/cobra v1.0.0 h1:6m/oheQuQ13N9ks4hubMG6BnvwOeaJrqSPLahSnczz8= github.com/spf13/cobra v1.0.0/go.mod h1:/6GTrnGXV9HjY+aR4k0oJ5tcvakLuG6EuKReYlHNrgE= github.com/spf13/jwalterweatherman v1.0.0/go.mod h1:cQK4TGJAtQXfYWX+Ddv3mKDzgVb68N+wFjFa4jdeBTo= github.com/spf13/pflag v1.0.1/go.mod h1:DYY7MBk1bdzusC3SYhjObp+wFpr4gzcvqqNjLnInEg4= -github.com/spf13/pflag v1.0.3 h1:zPAT6CGy6wXeQ7NtTnaTerfKOsV6V6F8agHXFiazDkg= github.com/spf13/pflag v1.0.3/go.mod h1:DYY7MBk1bdzusC3SYhjObp+wFpr4gzcvqqNjLnInEg4= github.com/spf13/pflag v1.0.5 h1:iy+VFUOCP1a+8yFto/drg2CJ5u0yRoB7fZw3DKv/JXA= github.com/spf13/pflag v1.0.5/go.mod h1:McXfInJRrz4CZXVZOBLb0bTZqETkiAhM9Iw0y3An2Bg= @@ -1112,11 +956,8 @@ github.com/stretchr/objx v0.1.1/go.mod h1:HFkY916IF+rwdDfMAkV7OtwuqBVzrE8GR6GFx+ github.com/stretchr/objx v0.2.0/go.mod h1:qt09Ya8vawLte6SNmTgCsAVtYtaKzEcn8ATUoHMkEqE= github.com/stretchr/testify v1.2.0/go.mod h1:a8OnRcib4nhh0OaRAV+Yts87kKdq0PP7pXfy6kDkUVs= github.com/stretchr/testify v1.2.2/go.mod h1:a8OnRcib4nhh0OaRAV+Yts87kKdq0PP7pXfy6kDkUVs= -github.com/stretchr/testify v1.3.0 h1:TivCn/peBQ7UY8ooIcPgZFpTNSz0Q2U6UrFlUfqbe0Q= github.com/stretchr/testify v1.3.0/go.mod h1:M5WIy9Dh21IEIfnGCwXGc5bZfKNJtfHm1UVUgZn+9EI= -github.com/stretchr/testify v1.4.0 h1:2E4SXV/wtOkTonXsotYi4li6zVWxYlZuYNCXe9XRJyk= github.com/stretchr/testify v1.4.0/go.mod h1:j7eGeouHqKxXV5pUuKE4zz7dFj8WfuZ+81PSLYec5m4= -github.com/stretchr/testify v1.5.1 h1:nOGnQDM7FYENwehXlg/kFVnos3rEvtKTjRvOWSzb6H4= github.com/stretchr/testify v1.5.1/go.mod h1:5W2xD1RspED5o8YsWQXVCued0rvSQ+mT+I5cxcmMvtA= github.com/stretchr/testify v1.6.0/go.mod h1:6Fq8oRcR53rry900zMqJjRRixrwX3KX962/h/Wwjteg= github.com/stretchr/testify v1.6.1 h1:hDPOHmpOpP40lSULcqw7IrRb/u7w6RpDC9399XyoNd0= @@ -1128,9 +969,7 @@ github.com/swaggo/http-swagger v0.0.0-20200308142732-58ac5e232fba/go.mod h1:O1lA github.com/swaggo/swag v1.5.1/go.mod h1:1Bl9F/ZBpVWh22nY0zmYyASPO1lI/zIwRDrpZU+tv8Y= github.com/swaggo/swag v1.6.3/go.mod h1:wcc83tB4Mb2aNiL/HP4MFeQdpHUrca+Rp/DRNgWAUio= github.com/swaggo/swag v1.6.5/go.mod h1:Y7ZLSS0d0DdxhWGVhQdu+Bu1QhaF5k0RD7FKdiAykeY= -github.com/swaggo/swag v1.6.6-0.20200323071853-8e21f4cefeea h1:e4navjNdMYbOJeqjea2DcUC00Djp1ewI7sJqmp1xd+I= github.com/swaggo/swag v1.6.6-0.20200323071853-8e21f4cefeea/go.mod h1:xDhTyuFIujYiN3DKWC/H/83xcfHp+UE/IzWWampG7Zc= -github.com/swaggo/swag v1.6.6-0.20200529100950-7c765ddd0476 h1:UjnSXdNPIG+5FJ6xLQODEdk7gSnJlMldu3sPAxxCO+4= github.com/swaggo/swag v1.6.6-0.20200529100950-7c765ddd0476/go.mod h1:xDhTyuFIujYiN3DKWC/H/83xcfHp+UE/IzWWampG7Zc= github.com/syndtr/goleveldb v0.0.0-20180815032940-ae2bd5eed72d/go.mod h1:Z4AUp2Km+PwemOoO/VB5AOx9XSsIItzFjoJlOSiYmn0= github.com/syndtr/goleveldb v1.0.0/go.mod h1:ZVVdQEZoIme9iO1Ch2Jdy24qqXrMMOU6lpPAyBWyWuQ= @@ -1142,13 +981,11 @@ github.com/thda/tds v0.1.7/go.mod h1:isLIF1oZdXfkqVMJM8RyNrsjlHPlTKnPlnsBs7ngZcM github.com/thoas/go-funk v0.7.0/go.mod h1:+IWnUfUmFO1+WVYQWQtIJHeRRdaIyyYglZN7xzUPe4Q= github.com/tiancaiamao/appdash v0.0.0-20181126055449-889f96f722a2 h1:mbAskLJ0oJfDRtkanvQPiooDH8HvJ2FBh+iKT/OmiQQ= github.com/tiancaiamao/appdash v0.0.0-20181126055449-889f96f722a2/go.mod h1:2PfKggNGDuadAa0LElHrByyrz4JPZ9fFx6Gs7nx7ZZU= -github.com/tidwall/gjson v1.3.5 h1:2oW9FBNu8qt9jy5URgrzsVx/T/KSn3qn/smJQ0crlDQ= github.com/tidwall/gjson v1.3.5/go.mod h1:P256ACg0Mn+j1RXIDXoss50DeIABTYK1PULOJHhxOls= github.com/tidwall/gjson v1.6.1 h1:LRbvNuNuvAiISWg6gxLEFuCe72UKy5hDqhxW/8183ws= github.com/tidwall/gjson v1.6.1/go.mod h1:BaHyNc5bjzYkPqgLq7mdVzeiRtULKULXLgZFKsxEHI0= github.com/tidwall/match v1.0.1 h1:PnKP62LPNxHKTwvHHZZzdOAOCtsJTjo6dZLCwpKm5xc= github.com/tidwall/match v1.0.1/go.mod h1:LujAq0jyVjBy028G1WhWfIzbpQfMO8bBZ6Tyb0+pL9E= -github.com/tidwall/pretty v1.0.0 h1:HsD+QiTn7sK6flMKIvNmpqz1qrpP3Ps6jOKIKMooyg4= github.com/tidwall/pretty v1.0.0/go.mod h1:XNkn88O1ChpSDQmQeStsy+sBenx6DDtFZJxhVysOjyk= github.com/tidwall/pretty v1.0.2 h1:Z7S3cePv9Jwm1KwS0513MRaoUe3S01WPbLNV40pwWZU= github.com/tidwall/pretty v1.0.2/go.mod h1:XNkn88O1ChpSDQmQeStsy+sBenx6DDtFZJxhVysOjyk= @@ -1157,9 +994,7 @@ github.com/tidwall/sjson v1.1.2/go.mod h1:SEzaDwxiPzKzNfUEO4HbYF/m4UCSJDsGgNqsS1 github.com/tikv/client-go v0.0.0-20190421092910-44b82dcc9f4a/go.mod h1:K0NcdVNrXDq92YPLytsrAwRMyuXi7GZCO6dXNH7OzQc= github.com/tikv/pd v1.1.0-beta.0.20200818122340-ef1a4e920b2f/go.mod h1:mwZ3Lip1YXgtgBx6blADUPMxrqPGCfwABlreDzuJul8= github.com/tikv/pd v1.1.0-beta.0.20200824114021-f8c45ae287fd/go.mod h1:quwjWtCmawAvS+YdxtSKG08sEexLzkhQgAno59wW+lI= -github.com/tikv/pd v1.1.0-beta.0.20200907085700-5b04bec39b99 h1:p2EYnx1jl7VJ5AuYoit0bDxGndYmsiUquDSAph5Ao1Q= github.com/tikv/pd v1.1.0-beta.0.20200907085700-5b04bec39b99/go.mod h1:h0GTvNPZrjA06ToexaL13DxlzAvm/6kkJWz12baD68M= -github.com/tikv/pd v1.1.0-beta.0.20201125070607-d4b90eee0c70 h1:KGyN3zjxk8kXvAYVDJvcZ9QNwgVf7bXnf3P49+JV4pw= github.com/tikv/pd v1.1.0-beta.0.20201125070607-d4b90eee0c70/go.mod h1:qqRJlVDCqe6r+Fs4yWPORwxVOa3VDinww8fg7zGiKfk= github.com/tikv/pd v1.1.0-beta.0.20210204070145-eb7fc53a8d98 h1:otWaKcPoX0tdYAWa9gUs/sY327OVJtO4Do03EoigDxo= github.com/tikv/pd v1.1.0-beta.0.20210204070145-eb7fc53a8d98/go.mod h1:h0BW5SX8pGzKfD/uRC+TpgYNhg/1jNkQrczv/jVERBo= @@ -1169,26 +1004,21 @@ github.com/tj/assert v0.0.0-20190920132354-ee03d75cd160/go.mod h1:mZ9/Rh9oLWpLLD github.com/tj/go-css v0.0.0-20191108133013-220a796d1705/go.mod h1:e+JPLQ9wyQCgRnPenX2bo7MJoLphBHz5c1WUqaANSeA= github.com/tj/go-termd v0.0.2-0.20200115111609-7f6aeb166380/go.mod h1:7JlPhw1+Bkn5PLz+kqAfzL8ij69OlQ1a4O5bbY98axo= github.com/tmc/grpc-websocket-proxy v0.0.0-20170815181823-89b8d40f7ca8/go.mod h1:ncp9v5uamzpCO7NfCPTXjqaC+bZgJeR0sMTm6dMHP7U= -github.com/tmc/grpc-websocket-proxy v0.0.0-20171017195756-830351dc03c6 h1:lYIiVDtZnyTWlNwiAxLj0bbpTcx1BWCFhXjfsvmPdNc= github.com/tmc/grpc-websocket-proxy v0.0.0-20171017195756-830351dc03c6/go.mod h1:ncp9v5uamzpCO7NfCPTXjqaC+bZgJeR0sMTm6dMHP7U= github.com/tmc/grpc-websocket-proxy v0.0.0-20190109142713-0ad062ec5ee5 h1:LnC5Kc/wtumK+WB441p7ynQJzVuNRJiqddSIE3IlSEQ= github.com/tmc/grpc-websocket-proxy v0.0.0-20190109142713-0ad062ec5ee5/go.mod h1:ncp9v5uamzpCO7NfCPTXjqaC+bZgJeR0sMTm6dMHP7U= github.com/twitchtv/retool v1.3.8-0.20180918173430-41330f8b4e07/go.mod h1:88ZJJaNbdr4rvnbFzU5l5d17iJ69JoQX2+JJ5B0Wteo= github.com/twmb/murmur3 v1.1.3 h1:D83U0XYKcHRYwYIpBKf3Pks91Z0Byda/9SJ8B6EMRcA= github.com/twmb/murmur3 v1.1.3/go.mod h1:Qq/R7NUyOfr65zD+6Q5IHKsJLwP7exErjN6lyyq3OSQ= -github.com/uber-go/atomic v1.3.2 h1:Azu9lPBWRNKzYXSIwRfgRuDuS0YKsK4NFhiQv98gkxo= github.com/uber-go/atomic v1.3.2/go.mod h1:/Ct5t2lcmbJ4OSe/waGBoaVvVqtO0bmtfVNex1PFV8g= github.com/uber-go/atomic v1.4.0 h1:yOuPqEq4ovnhEjpHmfFwsqBXDYbQeT6Nb0bwD6XnD5o= github.com/uber-go/atomic v1.4.0/go.mod h1:/Ct5t2lcmbJ4OSe/waGBoaVvVqtO0bmtfVNex1PFV8g= -github.com/uber/jaeger-client-go v2.15.0+incompatible h1:NP3qsSqNxh8VYr956ur1N/1C1PjvOJnJykCzcD5QHbk= github.com/uber/jaeger-client-go v2.15.0+incompatible/go.mod h1:WVhlPFC8FDjOFMMWRy2pZqQJSXxYSwNYOkTr/Z6d3Kk= github.com/uber/jaeger-client-go v2.16.0+incompatible/go.mod h1:WVhlPFC8FDjOFMMWRy2pZqQJSXxYSwNYOkTr/Z6d3Kk= github.com/uber/jaeger-client-go v2.22.1+incompatible h1:NHcubEkVbahf9t3p75TOCR83gdUHXjRJvjoBh1yACsM= github.com/uber/jaeger-client-go v2.22.1+incompatible/go.mod h1:WVhlPFC8FDjOFMMWRy2pZqQJSXxYSwNYOkTr/Z6d3Kk= -github.com/uber/jaeger-lib v1.5.0 h1:OHbgr8l656Ub3Fw5k9SWnBfIEwvoHQ+W2y+Aa9D1Uyo= github.com/uber/jaeger-lib v1.5.0/go.mod h1:ComeNDZlWwrWnDv8aPp0Ba6+uUTzImX/AauajbLI56U= github.com/uber/jaeger-lib v2.0.0+incompatible/go.mod h1:ComeNDZlWwrWnDv8aPp0Ba6+uUTzImX/AauajbLI56U= -github.com/uber/jaeger-lib v2.2.0+incompatible h1:MxZXOiR2JuoANZ3J6DE/U0kSFv/eJ/GfSYVCjK7dyaw= github.com/uber/jaeger-lib v2.2.0+incompatible/go.mod h1:ComeNDZlWwrWnDv8aPp0Ba6+uUTzImX/AauajbLI56U= github.com/uber/jaeger-lib v2.4.0+incompatible h1:fY7QsGQWiCt8pajv4r7JEvmATdCVaWxXbjwyYwsNaLQ= github.com/uber/jaeger-lib v2.4.0+incompatible/go.mod h1:ComeNDZlWwrWnDv8aPp0Ba6+uUTzImX/AauajbLI56U= @@ -1206,11 +1036,8 @@ github.com/unrolled/render v1.0.0/go.mod h1:tu82oB5W2ykJRVioYsB+IQKcft7ryBr7w12q github.com/unrolled/render v1.0.1 h1:VDDnQQVfBMsOsp3VaCJszSO0nkBIVEYoPWeRThk9spY= github.com/unrolled/render v1.0.1/go.mod h1:gN9T0NhL4Bfbwu8ann7Ry/TGHYfosul+J0obPf6NBdM= github.com/urfave/cli v1.20.0/go.mod h1:70zkFmudgCuE/ngEzBv17Jvp/497gISqfk5gWijbERA= -github.com/urfave/cli v1.22.2 h1:gsqYFH8bb9ekPA12kRo0hfjngWQjkJPlN9R0N78BoUo= github.com/urfave/cli v1.22.2/go.mod h1:Gos4lmkARVdJ6EkW0WaNv/tZAAMe9V7XWyB60NtXRu0= -github.com/urfave/cli/v2 v2.1.1 h1:Qt8FeAtxE/vfdrLmR3rxR6JRE0RoVmbXu8+6kZtYU4k= github.com/urfave/cli/v2 v2.1.1/go.mod h1:SE9GqnLQmjVa0iPEY0f1w3ygNIYcIJ0OKPMoW2caLfQ= -github.com/urfave/negroni v0.3.0 h1:PaXOb61mWeZJxc1Ji2xJjpVg9QfPo0rrB+lHyBxGNSU= github.com/urfave/negroni v0.3.0/go.mod h1:Meg73S6kFm/4PpbYdq35yYWoCZ9mS/YSx+lKnmiohz4= github.com/valyala/bytebufferpool v1.0.0/go.mod h1:6bBcMArwyJ5K/AmCkWv1jt77kVWyCJ6HpOuEn7z0Csc= github.com/valyala/fasttemplate v1.0.1/go.mod h1:UQGH1tvbgY+Nz5t2n7tXsz52dQxojPUpymEIMZ47gx8= @@ -1233,7 +1060,6 @@ github.com/xo/terminfo v0.0.0-20190125114736-1a4775eeeb62/go.mod h1:6Yhx5ZJl5942 github.com/xo/usql v0.7.8/go.mod h1:lmjb2xMb8yY1le5mlSvOSat4EoMM2HseDhVIPKAhAps= github.com/xo/xoutil v0.0.0-20171112033149-46189f4026a5/go.mod h1:GngMELAA694UVFs172352HAA2KQEf4XuETgWmL4XSoY= github.com/xordataexchange/crypt v0.0.3-0.20170626215501-b2862e3d0a77/go.mod h1:aYKd//L2LvnjZzWKhF00oedf4jCCReLcmhLdhm1A27Q= -github.com/yookoala/realpath v1.0.0 h1:7OA9pj4FZd+oZDsyvXWQvjn5oBdcHRTV44PpdMSuImQ= github.com/yookoala/realpath v1.0.0/go.mod h1:gJJMA9wuX7AcqLy1+ffPatSCySA1FQ2S8Ya9AIoYBpE= github.com/yuin/goldmark v1.1.25/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= github.com/yuin/goldmark v1.1.27/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= @@ -1241,37 +1067,29 @@ github.com/yuin/goldmark v1.1.32/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9de github.com/yuin/goldmark v1.2.1/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= github.com/yuin/gopher-lua v0.0.0-20181031023651-12c4817b42c5/go.mod h1:aEV29XrmTYFr3CiRxZeGHpkvbwq+prZduBqMaascyCU= github.com/zaf/temp v0.0.0-20170209143821-94e385923345/go.mod h1:sXsZgXwh6DB0qlskmZVB4HE93e5YrktMrgUDPy9iYmY= -github.com/zhangjinpeng1987/raft v0.0.0-20190624145930-deeb32d6553d h1:rGkexfPDxNuTCObUwTbsRUlti+evR/Ksb4dKy6esXW0= github.com/zhangjinpeng1987/raft v0.0.0-20190624145930-deeb32d6553d/go.mod h1:1KDQ09J8MRHEtHze4at7BJZDW/doUAgkJ8w9KjEUhSo= github.com/zhangjinpeng1987/raft v0.0.0-20200819064223-df31bb68a018 h1:T3OrqVdcH6z6SakR7WkECvGpdkfB0MAur/6zf66GPxQ= github.com/zhangjinpeng1987/raft v0.0.0-20200819064223-df31bb68a018/go.mod h1:rTSjwgeYU2on64W50csWDlhyy0x9UYVYJUovHlYdt5s= github.com/ziutek/mymysql v1.5.4/go.mod h1:LMSpPZ6DbqWFxNCHW77HeMg9I646SAhApZ/wKdgO/C0= gitlab.com/nyarla/go-crypt v0.0.0-20160106005555-d9a5dc2b789b/go.mod h1:T3BPAOm2cqquPa0MKWeNkmOM5RQsRhkrwMWonFMN7fE= -go.etcd.io/bbolt v1.3.2 h1:Z/90sZLPOeCy2PwprqkFa25PdkusRzaj9P8zm/KNyvk= go.etcd.io/bbolt v1.3.2/go.mod h1:IbVyRI1SCnLcuJnV2u8VeU0CEYM7e686BmAb1XKL+uU= -go.etcd.io/bbolt v1.3.3 h1:MUGmc65QhB3pIlaQ5bB4LwqSj6GIonVJXpZiaKNyaKk= go.etcd.io/bbolt v1.3.3/go.mod h1:IbVyRI1SCnLcuJnV2u8VeU0CEYM7e686BmAb1XKL+uU= go.etcd.io/bbolt v1.3.4/go.mod h1:G5EMThwa9y8QZGBClrRx5EY+Yw9kAhnjy3bSjsnlVTQ= go.etcd.io/bbolt v1.3.5 h1:XAzx9gjCb0Rxj7EoqcClPD1d5ZBxZJk0jbuoPHenBt0= go.etcd.io/bbolt v1.3.5/go.mod h1:G5EMThwa9y8QZGBClrRx5EY+Yw9kAhnjy3bSjsnlVTQ= go.etcd.io/etcd v0.0.0-20191023171146-3cf2f69b5738/go.mod h1:dnLIgRNXwCJa5e+c6mIZCrds/GIG4ncV9HhK5PX7jPg= -go.etcd.io/etcd v0.5.0-alpha.5.0.20191023171146-3cf2f69b5738 h1:lWF4f9Nypl1ZqSb4gLeh/DGvBYVaUYHuiB93teOmwgc= go.etcd.io/etcd v0.5.0-alpha.5.0.20191023171146-3cf2f69b5738/go.mod h1:dnLIgRNXwCJa5e+c6mIZCrds/GIG4ncV9HhK5PX7jPg= go.etcd.io/etcd v0.5.0-alpha.5.0.20200824191128-ae9734ed278b h1:3kC4J3eQF6p1UEfQTkC67eEeb3rTk+shQqdX6tFyq9Q= go.etcd.io/etcd v0.5.0-alpha.5.0.20200824191128-ae9734ed278b/go.mod h1:yVHk9ub3CSBatqGNg7GRmsnfLWtoW60w4eDYfh7vHDg= go.mongodb.org/mongo-driver v1.0.2/go.mod h1:u7ryQJ+DOzQmeO7zB6MHyr8jkEQvC8vH7qLUO4lqsUM= go.opencensus.io v0.21.0/go.mod h1:mSImk1erAIZhrmZN+AvHh14ztQfjbGwt4TtuofqLduU= go.opencensus.io v0.22.0/go.mod h1:+kGneAE2xo2IficOXnaByMWTGM9T73dGwxeWcUqIpI8= -go.opencensus.io v0.22.2 h1:75k/FF0Q2YM8QYo07VPddOLBslDt1MZOdEslOHvmzAs= go.opencensus.io v0.22.2/go.mod h1:yxeiOL68Rb0Xd1ddK5vPZ/oVn4vY4Ynel7k9FzqtOIw= go.opencensus.io v0.22.3 h1:8sGtKOrtQqkN1bp2AtX+misvLIlOmsEsNd+9NIcPEm8= go.opencensus.io v0.22.3/go.mod h1:yxeiOL68Rb0Xd1ddK5vPZ/oVn4vY4Ynel7k9FzqtOIw= -go.uber.org/atomic v1.3.2 h1:2Oa65PReHzfn29GpvgsYwloV9AVFHPDk8tYxt2c2tr4= go.uber.org/atomic v1.3.2/go.mod h1:gD2HeocX3+yG+ygLZcrzQJaqmWj9AIm7n08wl/qW/PE= go.uber.org/atomic v1.4.0/go.mod h1:gD2HeocX3+yG+ygLZcrzQJaqmWj9AIm7n08wl/qW/PE= -go.uber.org/atomic v1.5.0 h1:OI5t8sDa1Or+q8AeE+yKeB/SDYioSHAgcVljj9JIETY= go.uber.org/atomic v1.5.0/go.mod h1:sABNBOSYdrvTF6hTgEIbc7YasKWGhgEQZyfxyTvoXHQ= -go.uber.org/atomic v1.6.0 h1:Ezj3JGmsOnG1MoRWQkPBsKLe9DwWD9QeXzTRzzldNVk= go.uber.org/atomic v1.6.0/go.mod h1:sABNBOSYdrvTF6hTgEIbc7YasKWGhgEQZyfxyTvoXHQ= go.uber.org/atomic v1.7.0 h1:ADUqmZGgLDDfbSL9ZmPxKTybcoEYHgpYfELNoN+7hsw= go.uber.org/atomic v1.7.0/go.mod h1:fEN4uk6kAWBTFdckzkM89CLk9XfWZrxpCo0nPH17wJc= @@ -1280,37 +1098,26 @@ go.uber.org/dig v1.8.0/go.mod h1:X34SnWGr8Fyla9zQNO2GSO2D+TIuqB14OS8JhYocIyw= go.uber.org/fx v1.10.0/go.mod h1:vLRicqpG/qQEzno4SYU86iCwfT95EZza+Eba0ItuxqY= go.uber.org/goleak v0.10.0 h1:G3eWbSNIskeRqtsN/1uI5B+eP73y3JUuBsv9AZjehb4= go.uber.org/goleak v0.10.0/go.mod h1:VCZuO8V8mFPlL0F5J5GK1rtHV3DrFcQ1R8ryq7FK0aI= -go.uber.org/multierr v1.1.0 h1:HoEmRHQPVSqub6w2z2d2EOVs2fjyFRGyofhKuyDq0QI= go.uber.org/multierr v1.1.0/go.mod h1:wR5kodmAFQ0UK8QlbwjlSNy0Z68gJhDJUG5sjR94q/0= go.uber.org/multierr v1.3.0/go.mod h1:VgVr7evmIr6uPjLBxg28wmKNXyqE9akIJ5XnfpiKl+4= -go.uber.org/multierr v1.4.0 h1:f3WCSC2KzAcBXGATIxAB1E2XuCpNU255wNKZ505qi3E= go.uber.org/multierr v1.4.0/go.mod h1:VgVr7evmIr6uPjLBxg28wmKNXyqE9akIJ5XnfpiKl+4= -go.uber.org/multierr v1.5.0 h1:KCa4XfM8CWFCpxXRGok+Q0SS/0XBhMDbHHGABQLvD2A= go.uber.org/multierr v1.5.0/go.mod h1:FeouvMocqHpRaaGuG9EjoKcStLC43Zu/fmqdUMPcKYU= go.uber.org/multierr v1.6.0 h1:y6IPFStTAIT5Ytl7/XYmHvzXQ7S3g/IeZW9hyZ5thw4= go.uber.org/multierr v1.6.0/go.mod h1:cdWPpRnG4AhwMwsgIHip0KRBQjJy5kYEpYjJxpXp9iU= -go.uber.org/tools v0.0.0-20190618225709-2cfd321de3ee h1:0mgffUl7nfd+FpvXMVz4IDEaUSmT1ysygQC7qYo7sG4= go.uber.org/tools v0.0.0-20190618225709-2cfd321de3ee/go.mod h1:vJERXedbb3MVM5f9Ejo0C68/HhF8uaILCdgjnY+goOA= go.uber.org/zap v1.8.0/go.mod h1:vwi/ZaCAaUcBkycHslxD9B2zi4UTXhF60s6SWpuDF0Q= -go.uber.org/zap v1.9.1 h1:XCJQEf3W6eZaVwhRBof6ImoYGJSITeKWsyeh3HFu/5o= go.uber.org/zap v1.9.1/go.mod h1:vwi/ZaCAaUcBkycHslxD9B2zi4UTXhF60s6SWpuDF0Q= -go.uber.org/zap v1.10.0 h1:ORx85nbTijNz8ljznvCMR1ZBIPKFn3jQrag10X2AsuM= go.uber.org/zap v1.10.0/go.mod h1:vwi/ZaCAaUcBkycHslxD9B2zi4UTXhF60s6SWpuDF0Q= -go.uber.org/zap v1.12.0 h1:dySoUQPFBGj6xwjmBzageVL8jGi8uxc6bEmJQjA06bw= go.uber.org/zap v1.12.0/go.mod h1:zwrFLgMcdUuIBviXEYEH1YKNaOBnKXsx2IPda5bBwHM= -go.uber.org/zap v1.13.0 h1:nR6NoDBgAf67s68NhaXbsojM+2gxp3S1hWkHDl27pVU= go.uber.org/zap v1.13.0/go.mod h1:zwrFLgMcdUuIBviXEYEH1YKNaOBnKXsx2IPda5bBwHM= go.uber.org/zap v1.14.0/go.mod h1:zwrFLgMcdUuIBviXEYEH1YKNaOBnKXsx2IPda5bBwHM= -go.uber.org/zap v1.14.1 h1:nYDKopTbvAPq/NrUVZwT15y2lpROBiLLyoRTbXOYWOo= go.uber.org/zap v1.14.1/go.mod h1:Mb2vm2krFEG5DV0W9qcHBYFtp/Wku1cvYaqPsS/WYfc= -go.uber.org/zap v1.15.0 h1:ZZCA22JRF2gQE5FoNmhmrf7jeJJ2uhqDUNRYKm8dvmM= go.uber.org/zap v1.15.0/go.mod h1:Mb2vm2krFEG5DV0W9qcHBYFtp/Wku1cvYaqPsS/WYfc= go.uber.org/zap v1.16.0 h1:uFRZXykJGK9lLY4HtgSw44DnIcAM+kRBP7x5m+NpAOM= go.uber.org/zap v1.16.0/go.mod h1:MA8QOfq0BHJwdXa996Y4dYkAqRKB8/1K1QMMZVaNZjQ= golang.org/x/crypto v0.0.0-20180214000028-650f4a345ab4/go.mod h1:6SG95UA2DQfeDnfUPMdvaQW0Q7yPrPDi9nlGo2tz2b4= golang.org/x/crypto v0.0.0-20180904163835-0709b304e793/go.mod h1:6SG95UA2DQfeDnfUPMdvaQW0Q7yPrPDi9nlGo2tz2b4= golang.org/x/crypto v0.0.0-20181203042331-505ab145d0a9/go.mod h1:6SG95UA2DQfeDnfUPMdvaQW0Q7yPrPDi9nlGo2tz2b4= -golang.org/x/crypto v0.0.0-20190308221718-c2843e01d9a2 h1:VklqNMn3ovrHsnt90PveolxSbWFaJdECFbxSq0Mqo2M= golang.org/x/crypto v0.0.0-20190308221718-c2843e01d9a2/go.mod h1:djNgcEr1/C05ACkg1iLfiJU5Ep61QUkGW8qpdssI0+w= golang.org/x/crypto v0.0.0-20190325154230-a5d413f7728c/go.mod h1:djNgcEr1/C05ACkg1iLfiJU5Ep61QUkGW8qpdssI0+w= golang.org/x/crypto v0.0.0-20190424203555-c05e17bb3b2d/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= @@ -1320,15 +1127,11 @@ golang.org/x/crypto v0.0.0-20190611184440-5c40567a22f8/go.mod h1:yigFU9vqHzYiE8U golang.org/x/crypto v0.0.0-20190701094942-4def268fd1a4/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= golang.org/x/crypto v0.0.0-20190911031432-227b76d455e7/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= golang.org/x/crypto v0.0.0-20191011191535-87dc89f01550/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= -golang.org/x/crypto v0.0.0-20191205180655-e7c4368fe9dd h1:GGJVjV8waZKRHrgwvtH66z9ZGVurTD1MT0n1Bb+q4aM= golang.org/x/crypto v0.0.0-20191205180655-e7c4368fe9dd/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto= -golang.org/x/crypto v0.0.0-20191206172530-e9b2fee46413 h1:ULYEB3JvPRE/IfO+9uO7vKV/xzVTO7XPAwm8xbf4w2g= golang.org/x/crypto v0.0.0-20191206172530-e9b2fee46413/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto= golang.org/x/crypto v0.0.0-20191227163750-53104e6ec876/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto= golang.org/x/crypto v0.0.0-20200109152110-61a87790db17/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto= -golang.org/x/crypto v0.0.0-20200204104054-c9f3fb736b72 h1:+ELyKg6m8UBf0nPFSqD0mi7zUfwPyXo23HNjMnXPz7w= golang.org/x/crypto v0.0.0-20200204104054-c9f3fb736b72/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto= -golang.org/x/crypto v0.0.0-20200622213623-75b288015ac9 h1:psW17arqaxU48Z5kZ0CQnkZWQJsqcURM6tKiBApRjXI= golang.org/x/crypto v0.0.0-20200622213623-75b288015ac9/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto= golang.org/x/crypto v0.0.0-20200820211705-5c72a883971a h1:vclmkQCjlDX5OydZ9wv8rBCcS0QyQY66Mpf/7BZbInM= golang.org/x/crypto v0.0.0-20200820211705-5c72a883971a/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto= @@ -1341,7 +1144,6 @@ golang.org/x/exp v0.0.0-20190510132918-efd6b22b2522/go.mod h1:ZjyILWgesfNpC6sMxT golang.org/x/exp v0.0.0-20190829153037-c13cbed26979/go.mod h1:86+5VVa7VpoJ4kLfm080zCjGlMRFzhUhsZKEZO7MGek= golang.org/x/exp v0.0.0-20191030013958-a1ab85dbe136/go.mod h1:JXzH8nQsPlswgeRAPE3MuO9GYsAcnJvJ4vnMwN/5qkY= golang.org/x/exp v0.0.0-20191129062945-2f5052295587/go.mod h1:2RIsYlXP63K8oxa1u096TMicItID8zy7Y6sNkU49FU4= -golang.org/x/exp v0.0.0-20191227195350-da58074b4299 h1:zQpM52jfKHG6II1ISZY1ZcpygvuSFZpLwfluuF89XOg= golang.org/x/exp v0.0.0-20191227195350-da58074b4299/go.mod h1:2RIsYlXP63K8oxa1u096TMicItID8zy7Y6sNkU49FU4= golang.org/x/exp v0.0.0-20200119233911-0405dc783f0a/go.mod h1:2RIsYlXP63K8oxa1u096TMicItID8zy7Y6sNkU49FU4= golang.org/x/exp v0.0.0-20200207192155-f17229e696bd/go.mod h1:J/WKrq2StrnmMY6+EHIKF9dgMWnmCNThgcyBT1FY9mM= @@ -1358,7 +1160,6 @@ golang.org/x/lint v0.0.0-20190301231843-5614ed5bae6f/go.mod h1:UVdnD1Gm6xHRNCYTk golang.org/x/lint v0.0.0-20190313153728-d0100b6bd8b3/go.mod h1:6SW0HCj/g11FgYtHlgUYUwCkIfeOF89ocIRzGO/8vkc= golang.org/x/lint v0.0.0-20190409202823-959b441ac422/go.mod h1:6SW0HCj/g11FgYtHlgUYUwCkIfeOF89ocIRzGO/8vkc= golang.org/x/lint v0.0.0-20190909230951-414d861bb4ac/go.mod h1:6SW0HCj/g11FgYtHlgUYUwCkIfeOF89ocIRzGO/8vkc= -golang.org/x/lint v0.0.0-20190930215403-16217165b5de h1:5hukYrvBGR8/eNkX5mdUezrA6JiaEZDtJb9Ei+1LlBs= golang.org/x/lint v0.0.0-20190930215403-16217165b5de/go.mod h1:6SW0HCj/g11FgYtHlgUYUwCkIfeOF89ocIRzGO/8vkc= golang.org/x/lint v0.0.0-20191125180803-fdd1cda4f05f/go.mod h1:5qLYkcX4OjUUV8bRuDixDT3tpyyb+LUpUlRWLxfhWrs= golang.org/x/lint v0.0.0-20200130185559-910be7a94367/go.mod h1:3xt1FjdF8hUf6vQPIChWIBhFzV8gjjsPE/fR3IyQdNY= @@ -1368,10 +1169,8 @@ golang.org/x/mobile v0.0.0-20190312151609-d3739f865fa6/go.mod h1:z+o9i4GpDbdi3rU golang.org/x/mobile v0.0.0-20190719004257-d2bd2a29d028/go.mod h1:E/iHnbuqvinMTCcRqshq8CkpyQDoeVncDDYHnLhea+o= golang.org/x/mod v0.0.0-20190513183733-4bf6d317e70e/go.mod h1:mXi4GBBbnImb6dmsKGUJ2LatrhH/nqhxcFungHvyanc= golang.org/x/mod v0.1.0/go.mod h1:0QHyrYULN0/3qlju5TqG8bIK38QM8yzMo5ekMj3DlcY= -golang.org/x/mod v0.1.1-0.20191105210325-c90efee705ee h1:WG0RUwxtNT4qqaXX3DPA8zHFNm/D9xaBpxzHt1WcA/E= golang.org/x/mod v0.1.1-0.20191105210325-c90efee705ee/go.mod h1:QqPTAvyqsEbceGzBzNggFXnrqF1CaUcvgkdR5Ot7KZg= golang.org/x/mod v0.1.1-0.20191107180719-034126e5016b/go.mod h1:QqPTAvyqsEbceGzBzNggFXnrqF1CaUcvgkdR5Ot7KZg= -golang.org/x/mod v0.2.0 h1:KU7oHjnv3XNWfa5COkzUifxZmxp1TyI7ImMXqFxLwvQ= golang.org/x/mod v0.2.0/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA= golang.org/x/mod v0.3.0 h1:RM4zey1++hCTbCVQfnWeKs9/IEsaBLA8vTkd0WVtmH4= golang.org/x/mod v0.3.0/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA= @@ -1385,7 +1184,6 @@ golang.org/x/net v0.0.0-20181201002055-351d144fa1fc/go.mod h1:mL1N/T3taQHkDXs73r golang.org/x/net v0.0.0-20181220203305-927f97764cc3/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= golang.org/x/net v0.0.0-20190108225652-1e06a53dbb7e/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= golang.org/x/net v0.0.0-20190213061140-3a22650c66bd/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= -golang.org/x/net v0.0.0-20190311183353-d8887717615a h1:oWX7TPOiFAMXLq8o0ikBYfCJVlRHBcsciT5bXOrH628= golang.org/x/net v0.0.0-20190311183353-d8887717615a/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg= golang.org/x/net v0.0.0-20190404232315-eb5bcb51f2a3/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg= golang.org/x/net v0.0.0-20190501004415-9ce7a6920f09/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg= @@ -1394,7 +1192,6 @@ golang.org/x/net v0.0.0-20190522155817-f3200d17e092/go.mod h1:HSz+uSET+XFnRR8LxR golang.org/x/net v0.0.0-20190603091049-60506f45cf65/go.mod h1:HSz+uSET+XFnRR8LxR5pz3Of3rY3CfYBVs4xY44aLks= golang.org/x/net v0.0.0-20190611141213-3f473d35a33a/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= golang.org/x/net v0.0.0-20190613194153-d28f0bde5980/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= -golang.org/x/net v0.0.0-20190620200207-3b0461eec859 h1:R/3boaszxrf1GEUWTVDzSKVwLmSJpwZ1yqXm8j0v2QI= golang.org/x/net v0.0.0-20190620200207-3b0461eec859/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= golang.org/x/net v0.0.0-20190724013045-ca1201d0de80/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= golang.org/x/net v0.0.0-20190813141303-74dc4d7220e7/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= @@ -1406,14 +1203,11 @@ golang.org/x/net v0.0.0-20200202094626-16171245cfb2/go.mod h1:z5CRVTTTmAJ677TzLL golang.org/x/net v0.0.0-20200222125558-5a598a2470a0/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= golang.org/x/net v0.0.0-20200226121028-0de0cce0169b/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= golang.org/x/net v0.0.0-20200301022130-244492dfa37a/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= -golang.org/x/net v0.0.0-20200324143707-d3edc9973b7e h1:3G+cUijn7XD+S4eJFddp53Pv7+slrESplyjG25HgL+k= golang.org/x/net v0.0.0-20200324143707-d3edc9973b7e/go.mod h1:qpuaurCH72eLCgpAm/N6yyVIVM9cpaDIP3A8BGJEC5A= golang.org/x/net v0.0.0-20200421231249-e086a090c8fd/go.mod h1:qpuaurCH72eLCgpAm/N6yyVIVM9cpaDIP3A8BGJEC5A= -golang.org/x/net v0.0.0-20200520182314-0ba52f642ac2 h1:eDrdRpKgkcCqKZQwyZRyeFZgfqt37SL7Kv3tok06cKE= golang.org/x/net v0.0.0-20200520182314-0ba52f642ac2/go.mod h1:qpuaurCH72eLCgpAm/N6yyVIVM9cpaDIP3A8BGJEC5A= golang.org/x/net v0.0.0-20200625001655-4c5254603344/go.mod h1:/O7V0waA8r7cgGh81Ro3o1hOxt32SMVPicZroKQ2sZA= golang.org/x/net v0.0.0-20200813134508-3edf25e44fcc/go.mod h1:/O7V0waA8r7cgGh81Ro3o1hOxt32SMVPicZroKQ2sZA= -golang.org/x/net v0.0.0-20200904194848-62affa334b73 h1:MXfv8rhZWmFeqX3GNZRsd6vOLoaCHjYEX3qkRo3YBUA= golang.org/x/net v0.0.0-20200904194848-62affa334b73/go.mod h1:/O7V0waA8r7cgGh81Ro3o1hOxt32SMVPicZroKQ2sZA= golang.org/x/net v0.0.0-20201021035429-f5854403a974/go.mod h1:sp8m0HH+o8qH0wwXwYZr8TS3Oi6o0r6Gce1SSxlDquU= golang.org/x/net v0.0.0-20201110031124-69a78807bb2b h1:uwuIcX0g4Yl1NC5XAz37xsr2lTtcqevgzYNVt49waME= @@ -1421,19 +1215,15 @@ golang.org/x/net v0.0.0-20201110031124-69a78807bb2b/go.mod h1:sp8m0HH+o8qH0wwXwY golang.org/x/oauth2 v0.0.0-20180821212333-d2e6202438be/go.mod h1:N/0e6XlmueqKjAGxoOufVs8QHGRruUQn6yWY3a++T0U= golang.org/x/oauth2 v0.0.0-20190226205417-e64efc72b421/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw= golang.org/x/oauth2 v0.0.0-20190604053449-0f29369cfe45/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw= -golang.org/x/oauth2 v0.0.0-20191202225959-858c2ad4c8b6 h1:pE8b58s1HRDMi8RDc79m0HISf9D4TzseP40cEA6IGfs= golang.org/x/oauth2 v0.0.0-20191202225959-858c2ad4c8b6/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw= golang.org/x/oauth2 v0.0.0-20200107190931-bf48bf16ab8d h1:TzXSXBo42m9gQenoE3b9BGiEpg5IG2JkU5FkPIawgtw= golang.org/x/oauth2 v0.0.0-20200107190931-bf48bf16ab8d/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw= golang.org/x/sync v0.0.0-20180314180146-1d60e4601c6f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20181108010431-42b317875d0f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= -golang.org/x/sync v0.0.0-20181221193216-37e7f081c4d4 h1:YUO/7uOKsKeq9UokNS62b8FYywz3ker1l1vDZRCRefw= golang.org/x/sync v0.0.0-20181221193216-37e7f081c4d4/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20190227155943-e225da77a7e6/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20190423024810-112230192c58/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= -golang.org/x/sync v0.0.0-20190911185100-cd5d95a43a6e h1:vcxGaoTs7kV8m5Np9uUNQin4BrLOthgV7252N8V+FwY= golang.org/x/sync v0.0.0-20190911185100-cd5d95a43a6e/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= -golang.org/x/sync v0.0.0-20200625203802-6e8e738ad208 h1:qwRHBd0NqMbJxfbotnDhm2ByMI1Shq4Y6oRJo21SGJA= golang.org/x/sync v0.0.0-20200625203802-6e8e738ad208/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20201020160332-67f06af15bc9 h1:SQFwaSi55rU7vdNs9Yr0Z324VNlrF+0wMqRXT4St8ck= golang.org/x/sync v0.0.0-20201020160332-67f06af15bc9/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= @@ -1479,31 +1269,24 @@ golang.org/x/sys v0.0.0-20200122134326-e047566fdf82/go.mod h1:h1NjWce9XRLGQEsW7w golang.org/x/sys v0.0.0-20200202164722-d101bd2416d5/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20200212091648-12a6c2dcc1e4/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20200223170610-d5e6a3e2c0ae/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20200323222414-85ca7c5b95cd h1:xhmwyvizuTgC2qz7ZlMluP20uW+C3Rm0FD/WLDX8884= golang.org/x/sys v0.0.0-20200323222414-85ca7c5b95cd/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20200413165638-669c56c373c4/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20200420163511-1957bb5e6d1f/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20200819171115-d785dc25833f/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20200824131525-c12d262b63d8 h1:AvbQYmiaaaza3cW3QXRyPo5kYgpFIzOAfeAAN7m3qQ4= golang.org/x/sys v0.0.0-20200824131525-c12d262b63d8/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20200930185726-fdedc70b468f/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20210119212857-b64e53b001e4 h1:myAQVi0cGEoqQVR5POX+8RR2mrocKqNN1hmeMqhX27k= golang.org/x/sys v0.0.0-20210119212857-b64e53b001e4/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/text v0.0.0-20170915032832-14c0d48ead0c/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= -golang.org/x/text v0.3.0 h1:g61tztE5qeGQ89tm6NTjjM9VPIm088od1l6aSorWRWg= golang.org/x/text v0.3.0/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= golang.org/x/text v0.3.1-0.20180807135948-17ff2d5776d2/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= -golang.org/x/text v0.3.2 h1:tW2bmiBqwgJj/UpqtC8EpXEZVYOwU0yG4iWbprSVAcs= golang.org/x/text v0.3.2/go.mod h1:bEr9sfX3Q8Zfm5fL9x+3itogRgK3+ptLWKqgva+5dAk= -golang.org/x/text v0.3.3 h1:cokOdA+Jmi5PJGXLlLllQSgYigAEfHXJAERHVMaCc2k= golang.org/x/text v0.3.3/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ= -golang.org/x/text v0.3.4 h1:0YWbFKbhXG/wIiuHDSKpS0Iy7FSA+u45VtBMfQcFTTc= golang.org/x/text v0.3.4/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ= golang.org/x/text v0.3.5 h1:i6eZZ+zk0SOf0xgBpEpPD18qWcJda6q1sxt3S0kzyUQ= golang.org/x/text v0.3.5/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ= golang.org/x/time v0.0.0-20180412165947-fbb02b2291d2/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= golang.org/x/time v0.0.0-20181108054448-85acf8d2951c/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= -golang.org/x/time v0.0.0-20190308202827-9d24e82272b4 h1:SvFZT6jyqRaOeXpc5h/JSfZenJ2O330aBsf7JfSUXmQ= golang.org/x/time v0.0.0-20190308202827-9d24e82272b4/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= golang.org/x/time v0.0.0-20191024005414-555d28b269f0 h1:/5xXl8Y5W96D+TtHSlonuFqGHIWVuyCkGJLwGh9JJFs= golang.org/x/time v0.0.0-20191024005414-555d28b269f0/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= @@ -1518,7 +1301,6 @@ golang.org/x/tools v0.0.0-20190311212946-11955173bddd/go.mod h1:LCzVGOaR6xXOjkQ3 golang.org/x/tools v0.0.0-20190312151545-0bb0c0a6e846/go.mod h1:LCzVGOaR6xXOjkQ3onu1FJEFr0SW1gC7cKk1uF8kGRs= golang.org/x/tools v0.0.0-20190312170243-e65039ee4138/go.mod h1:LCzVGOaR6xXOjkQ3onu1FJEFr0SW1gC7cKk1uF8kGRs= golang.org/x/tools v0.0.0-20190328211700-ab21143f2384/go.mod h1:LCzVGOaR6xXOjkQ3onu1FJEFr0SW1gC7cKk1uF8kGRs= -golang.org/x/tools v0.0.0-20190425150028-36563e24a262 h1:qsl9y/CJx34tuA7QCPNp86JNJe4spst6Ff8MjvPUdPg= golang.org/x/tools v0.0.0-20190425150028-36563e24a262/go.mod h1:RgjU9mgBXZiqYHBnxXauZ1Gv1EHHAz9KjViQ78xBX0Q= golang.org/x/tools v0.0.0-20190506145303-2d16b83fe98c/go.mod h1:RgjU9mgBXZiqYHBnxXauZ1Gv1EHHAz9KjViQ78xBX0Q= golang.org/x/tools v0.0.0-20190524140312-2c0ae7006135/go.mod h1:RgjU9mgBXZiqYHBnxXauZ1Gv1EHHAz9KjViQ78xBX0Q= @@ -1536,7 +1318,6 @@ golang.org/x/tools v0.0.0-20191029041327-9cc4af7d6b2c/go.mod h1:b+2E5dAYhXwXZwtn golang.org/x/tools v0.0.0-20191029190741-b9c20aec41a5/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= golang.org/x/tools v0.0.0-20191030062658-86caa796c7ab/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= golang.org/x/tools v0.0.0-20191105231337-689d0f08e67a/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= -golang.org/x/tools v0.0.0-20191107010934-f79515f33823 h1:akkRBeitX2EZP59KdtKw310CI4WGPCNPyrLbE7WZA8Y= golang.org/x/tools v0.0.0-20191107010934-f79515f33823/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= golang.org/x/tools v0.0.0-20191113191852-77e3bb0ad9e7/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= golang.org/x/tools v0.0.0-20191114200427-caa0b0f7d508/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= @@ -1556,24 +1337,19 @@ golang.org/x/tools v0.0.0-20200207183749-b753a1ba74fa/go.mod h1:TB2adYChydJhpapK golang.org/x/tools v0.0.0-20200212150539-ea181f53ac56/go.mod h1:TB2adYChydJhpapKDTa4BR/hXlZSLoq2Wpct/0txZ28= golang.org/x/tools v0.0.0-20200224181240-023911ca70b2/go.mod h1:TB2adYChydJhpapKDTa4BR/hXlZSLoq2Wpct/0txZ28= golang.org/x/tools v0.0.0-20200225230052-807dcd883420/go.mod h1:TB2adYChydJhpapKDTa4BR/hXlZSLoq2Wpct/0txZ28= -golang.org/x/tools v0.0.0-20200325010219-a49f79bcc224 h1:azwY/v0y0K4mFHVsg5+UrTgchqALYWpqVo6vL5OmkmI= golang.org/x/tools v0.0.0-20200325010219-a49f79bcc224/go.mod h1:Sl4aGygMT6LrqrWclx+PTx3U+LnKx/seiNR+3G19Ar8= golang.org/x/tools v0.0.0-20200325203130-f53864d0dba1/go.mod h1:Sl4aGygMT6LrqrWclx+PTx3U+LnKx/seiNR+3G19Ar8= golang.org/x/tools v0.0.0-20200422205258-72e4a01eba43/go.mod h1:EkVYQZoAsY45+roYkvgYkIh4xh/qjgUK9TdY2XT94GE= golang.org/x/tools v0.0.0-20200521211927-2b542361a4fc/go.mod h1:EkVYQZoAsY45+roYkvgYkIh4xh/qjgUK9TdY2XT94GE= -golang.org/x/tools v0.0.0-20200527183253-8e7acdbce89d h1:SR+e35rACZFBohNb4Om1ibX6N3iO0FtdbwqGSuD9dBU= golang.org/x/tools v0.0.0-20200527183253-8e7acdbce89d/go.mod h1:EkVYQZoAsY45+roYkvgYkIh4xh/qjgUK9TdY2XT94GE= golang.org/x/tools v0.0.0-20200609164405-eb789aa7ce50/go.mod h1:EkVYQZoAsY45+roYkvgYkIh4xh/qjgUK9TdY2XT94GE= golang.org/x/tools v0.0.0-20200820010801-b793a1359eac/go.mod h1:njjCfa9FT2d7l9Bc6FUM5FLjQPp3cFF28FI3qnDFljA= -golang.org/x/tools v0.0.0-20200823205832-c024452afbcd h1:KNSumuk5eGuQV7zbOrDDZ3MIkwsQr0n5oKiH4oE0/hU= golang.org/x/tools v0.0.0-20200823205832-c024452afbcd/go.mod h1:njjCfa9FT2d7l9Bc6FUM5FLjQPp3cFF28FI3qnDFljA= golang.org/x/tools v0.0.0-20201125231158-b5590deeca9b/go.mod h1:emZCQorbCU4vsT4fOWvOPXz4eW1wZW4PmDk9uLelYpA= golang.org/x/tools v0.1.0 h1:po9/4sTYwZU9lPhi1tOrb4hCv3qrhiQ77LZfGa2OjwY= golang.org/x/tools v0.1.0/go.mod h1:xkSsbof2nBLbhDlRMhhhyNLN/zl3eTqcnHD5viDpcZ0= golang.org/x/xerrors v0.0.0-20190717185122-a985d3407aa7/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= -golang.org/x/xerrors v0.0.0-20191011141410-1b5146add898 h1:/atklqdjdhuosWIl6AIbOeHJjicWYPqR9bpxqxYG2pA= golang.org/x/xerrors v0.0.0-20191011141410-1b5146add898/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= -golang.org/x/xerrors v0.0.0-20191204190536-9bdfabe68543 h1:E7g+9GITq07hpfrRu66IVDexMakfv52eLZ2CXBWiKr4= golang.org/x/xerrors v0.0.0-20191204190536-9bdfabe68543/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= golang.org/x/xerrors v0.0.0-20200804184101-5ec99f83aff1 h1:go1bK/D/BFZV2I8cIQd1NKEZ+0owSTG1fDTci4IqFcE= golang.org/x/xerrors v0.0.0-20200804184101-5ec99f83aff1/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= @@ -1590,14 +1366,11 @@ google.golang.org/api v0.9.0/go.mod h1:o4eAsZoiT+ibD93RtjEohWalFOjRDx6CVaqeizhEn google.golang.org/api v0.13.0/go.mod h1:iLdEw5Ide6rF15KTC1Kkl0iskquN2gFfn9o9XIsbkAI= google.golang.org/api v0.14.0/go.mod h1:iLdEw5Ide6rF15KTC1Kkl0iskquN2gFfn9o9XIsbkAI= google.golang.org/api v0.15.0/go.mod h1:iLdEw5Ide6rF15KTC1Kkl0iskquN2gFfn9o9XIsbkAI= -google.golang.org/api v0.15.1 h1:5mMS6mYvK5LVB8+ujVBC33Y8gltBo/kT6HBm6kU80G4= google.golang.org/api v0.15.1/go.mod h1:iLdEw5Ide6rF15KTC1Kkl0iskquN2gFfn9o9XIsbkAI= google.golang.org/api v0.17.0/go.mod h1:BwFmGc8tA3vsd7r/7kR8DY7iEEGSU04BFxCo5jP/sfE= -google.golang.org/api v0.18.0 h1:TgDr+1inK2XVUKZx3BYAqQg/GwucGdBkzZjWaTg/I+A= google.golang.org/api v0.18.0/go.mod h1:BwFmGc8tA3vsd7r/7kR8DY7iEEGSU04BFxCo5jP/sfE= google.golang.org/api v0.22.0 h1:J1Pl9P2lnmYFSJvgs70DKELqHNh8CNWXPbud4njEE2s= google.golang.org/api v0.22.0/go.mod h1:BwFmGc8tA3vsd7r/7kR8DY7iEEGSU04BFxCo5jP/sfE= -google.golang.org/appengine v1.1.0 h1:igQkv0AAhEIvTEpD5LIpAfav2eeVO9HBTjvKHVJPRSs= google.golang.org/appengine v1.1.0/go.mod h1:EbEs0AVv82hx2wNQdGPgUI5lhzA/G0D9YwlJXL52JkM= google.golang.org/appengine v1.4.0/go.mod h1:xpcJRLb0r/rnEns0DIKYYv+WjYCduHsrkT7/EB5XEv4= google.golang.org/appengine v1.5.0/go.mod h1:xpcJRLb0r/rnEns0DIKYYv+WjYCduHsrkT7/EB5XEv4= @@ -1622,7 +1395,6 @@ google.golang.org/genproto v0.0.0-20191114150713-6bbd007550de/go.mod h1:n3cpQtvx google.golang.org/genproto v0.0.0-20191115194625-c23dd37a84c9/go.mod h1:n3cpQtvxv34hfy77yVDNjmbRyujviMdxYliBSkLhpCc= google.golang.org/genproto v0.0.0-20191206224255-0243a4be9c8f/go.mod h1:n3cpQtvxv34hfy77yVDNjmbRyujviMdxYliBSkLhpCc= google.golang.org/genproto v0.0.0-20191216164720-4f79533eabd1/go.mod h1:n3cpQtvxv34hfy77yVDNjmbRyujviMdxYliBSkLhpCc= -google.golang.org/genproto v0.0.0-20191230161307-f3c370f40bfb h1:ADPHZzpzM4tk4V4S5cnCrr5SwzvlrPRmqqCuJDB8UTs= google.golang.org/genproto v0.0.0-20191230161307-f3c370f40bfb/go.mod h1:n3cpQtvxv34hfy77yVDNjmbRyujviMdxYliBSkLhpCc= google.golang.org/genproto v0.0.0-20200108215221-bd8f9a0ef82f/go.mod h1:n3cpQtvxv34hfy77yVDNjmbRyujviMdxYliBSkLhpCc= google.golang.org/genproto v0.0.0-20200115191322-ca5a22157cba/go.mod h1:n3cpQtvxv34hfy77yVDNjmbRyujviMdxYliBSkLhpCc= @@ -1638,27 +1410,20 @@ google.golang.org/grpc v1.20.1/go.mod h1:10oTOabMzJvdu6/UiuZezV6QK5dSlG84ov/aaiq google.golang.org/grpc v1.21.0/go.mod h1:oYelfM1adQP15Ek0mdvEgi9Df8B9CZIaU1084ijfRaM= google.golang.org/grpc v1.21.1/go.mod h1:oYelfM1adQP15Ek0mdvEgi9Df8B9CZIaU1084ijfRaM= google.golang.org/grpc v1.23.0/go.mod h1:Y5yQAOtifL1yxbo5wqy6BxZv8vAUGQwXBOALyacEbxg= -google.golang.org/grpc v1.23.1 h1:q4XQuHFC6I28BKZpo6IYyb3mNO+l7lSOxRuYTCiDfXk= google.golang.org/grpc v1.23.1/go.mod h1:Y5yQAOtifL1yxbo5wqy6BxZv8vAUGQwXBOALyacEbxg= google.golang.org/grpc v1.24.0/go.mod h1:XDChyiUovWa60DnaeDeZmSW86xtLtjtZbwvSiRnRtcA= -google.golang.org/grpc v1.25.1 h1:wdKvqQk7IttEw92GoRyKG2IDrUIpgpj6H6m81yfeMW0= google.golang.org/grpc v1.25.1/go.mod h1:c3i+UQWmh7LiEpx4sFZnkU36qjEYZ0imhYfXVyQciAY= -google.golang.org/grpc v1.26.0 h1:2dTRdpdFEEhJYQD8EMLB61nnrzSCTbG38PhqdhvOltg= google.golang.org/grpc v1.26.0/go.mod h1:qbnxyOmOxrQa7FizSgH+ReBfzJrCY1pSN7KXBS8abTk= google.golang.org/grpc v1.27.0/go.mod h1:qbnxyOmOxrQa7FizSgH+ReBfzJrCY1pSN7KXBS8abTk= google.golang.org/grpc v1.27.1 h1:zvIju4sqAGvwKspUQOhwnpcqSbzi7/H6QomNNjTL4sk= google.golang.org/grpc v1.27.1/go.mod h1:qbnxyOmOxrQa7FizSgH+ReBfzJrCY1pSN7KXBS8abTk= gopkg.in/VividCortex/ewma.v1 v1.1.1/go.mod h1:TekXuFipeiHWiAlO1+wSS23vTcyFau5u3rxXUSXj710= gopkg.in/alecthomas/gometalinter.v2 v2.0.12/go.mod h1:NDRytsqEZyolNuAgTzJkZMkSQM7FIKyzVzGhjB/qfYo= -gopkg.in/alecthomas/kingpin.v2 v2.2.6 h1:jMFz6MfLP0/4fUyZle81rXUoxOBFi19VUFKVDOQfozc= gopkg.in/alecthomas/kingpin.v2 v2.2.6/go.mod h1:FMv+mEhP44yOT+4EoQTLFTRgOQ1FBLkstjWtayDeSgw= gopkg.in/alecthomas/kingpin.v3-unstable v3.0.0-20180810215634-df19058c872c/go.mod h1:3HH7i1SgMqlzxCcBmUHW657sD4Kvv9sC3HpL3YukzwA= -gopkg.in/check.v1 v0.0.0-20161208181325-20d25e280405 h1:yhCVgyC4o1eVCa2tZl7eS0r+SDo693bJlVdllGtEeKM= gopkg.in/check.v1 v0.0.0-20161208181325-20d25e280405/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= gopkg.in/check.v1 v1.0.0-20160105164936-4f90aeace3a2/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= -gopkg.in/check.v1 v1.0.0-20180628173108-788fd7840127 h1:qIbj1fsPNlZgppZ+VLlY7N33q108Sa+fhmuc+sWQYwY= gopkg.in/check.v1 v1.0.0-20180628173108-788fd7840127/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= -gopkg.in/check.v1 v1.0.0-20190902080502-41f04d3bba15 h1:YR8cESwS4TdDjEe65xsg0ogRM/Nc3DYOhEAlW+xobZo= gopkg.in/check.v1 v1.0.0-20190902080502-41f04d3bba15/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= gopkg.in/check.v1 v1.0.0-20200902074654-038fdea0a05b h1:QRR6H1YWRnHb4Y/HeNFCTJLFVxaq6wH4YuVdsUOr75U= gopkg.in/check.v1 v1.0.0-20200902074654-038fdea0a05b/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= @@ -1685,7 +1450,6 @@ gopkg.in/jcmturner/rpc.v1 v1.1.0/go.mod h1:YIdkC4XfD6GXbzje11McwsDuOlZQSb9W4vfLv gopkg.in/mattn/go-colorable.v0 v0.1.0/go.mod h1:BVJlBXzARQxdi3nZo6f6bnl5yR20/tOL6p+V0KejgSY= gopkg.in/mattn/go-isatty.v0 v0.0.4/go.mod h1:wt691ab7g0X4ilKZNmMII3egK0bTxl37fEn/Fwbd8gc= gopkg.in/mattn/go-runewidth.v0 v0.0.4/go.mod h1:BmXejnxvhwdaATwiJbB1vZ2dtXkQKZGu9yLFCZb4msQ= -gopkg.in/mgo.v2 v2.0.0-20160818015218-f2b6f6c918c4 h1:hILp2hNrRnYjZpmIbx70psAHbBSEcQ1NIzDcUbJ1b6g= gopkg.in/mgo.v2 v2.0.0-20160818015218-f2b6f6c918c4/go.mod h1:yeKp02qBN3iKW1OzL3MGk2IdtZzaj7SFntXj72NppTA= gopkg.in/mgo.v2 v2.0.0-20190816093944-a6b53ec6cb22 h1:VpOs+IwYnYBaFnrNAeB8UUWtL3vEUnzSCL1nVjPhqrw= gopkg.in/mgo.v2 v2.0.0-20190816093944-a6b53ec6cb22/go.mod h1:yeKp02qBN3iKW1OzL3MGk2IdtZzaj7SFntXj72NppTA= @@ -1699,13 +1463,11 @@ gopkg.in/tomb.v2 v2.0.0-20161208151619-d5d1b5820637/go.mod h1:BHsqpu/nsuzkT5BpiH gopkg.in/yaml.v2 v2.0.0-20170712054546-1be3d31502d6/go.mod h1:JAlM8MvJe8wmxCU4Bli9HhUf9+ttbYbLASfIpnQbh74= gopkg.in/yaml.v2 v2.0.0-20170812160011-eb3733d160e7/go.mod h1:JAlM8MvJe8wmxCU4Bli9HhUf9+ttbYbLASfIpnQbh74= gopkg.in/yaml.v2 v2.2.1/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= -gopkg.in/yaml.v2 v2.2.2 h1:ZCJp+EgiOT7lHqUV2J862kp8Qj64Jo6az82+3Td9dZw= gopkg.in/yaml.v2 v2.2.2/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= gopkg.in/yaml.v2 v2.2.3/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= gopkg.in/yaml.v2 v2.2.4/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= gopkg.in/yaml.v2 v2.2.5/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= gopkg.in/yaml.v2 v2.2.7/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= -gopkg.in/yaml.v2 v2.2.8 h1:obN1ZagJSUGI0Ek/LBmuj4SNLPfIny3KsKFopxRdj10= gopkg.in/yaml.v2 v2.2.8/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= gopkg.in/yaml.v2 v2.3.0 h1:clyUAQHOM3G0M3f5vQj7LuJrETvjVot3Z5el9nffUtU= gopkg.in/yaml.v2 v2.3.0/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= @@ -1715,15 +1477,10 @@ honnef.co/go/tools v0.0.0-20190102054323-c2f93a96b099/go.mod h1:rf3lG4BRIbNafJWh honnef.co/go/tools v0.0.0-20190106161140-3f1c8253044a/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4= honnef.co/go/tools v0.0.0-20190418001031-e561f6794a2a/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4= honnef.co/go/tools v0.0.0-20190523083050-ea95bdfd59fc/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4= -honnef.co/go/tools v0.0.1-2019.2.3 h1:3JgtbtFHMiCmsznwGVTUWbgGov+pVqnlf1dEJTNAXeM= honnef.co/go/tools v0.0.1-2019.2.3/go.mod h1:a3bituU0lyd329TUQxRnasdCoJDkEUEAqEt0JzvZhAg= -honnef.co/go/tools v0.0.1-2020.1.3 h1:sXmLre5bzIR6ypkjXCDI3jHPssRhc8KD/Ome589sc3U= honnef.co/go/tools v0.0.1-2020.1.3/go.mod h1:X/FiERA/W4tHapMX5mGpAtMSVEeEUOyHaw9vFzvIQ3k= -honnef.co/go/tools v0.0.1-2020.1.4 h1:UoveltGrhghAA7ePc+e+QYDHXrBps2PqFZiHkGR/xK8= honnef.co/go/tools v0.0.1-2020.1.4/go.mod h1:X/FiERA/W4tHapMX5mGpAtMSVEeEUOyHaw9vFzvIQ3k= -honnef.co/go/tools v0.0.1-2020.1.5 h1:nI5egYTGJakVyOryqLs1cQO5dO0ksin5XXs2pspk75k= honnef.co/go/tools v0.0.1-2020.1.5/go.mod h1:X/FiERA/W4tHapMX5mGpAtMSVEeEUOyHaw9vFzvIQ3k= -honnef.co/go/tools v0.1.0 h1:AWNL1W1i7f0wNZ8VwOKNJ0sliKvOF/adn0EHenfUh+c= honnef.co/go/tools v0.1.0/go.mod h1:XtegFAyX/PfluP4921rXU5IkjkqBCDnUq4W8VCIoKvM= honnef.co/go/tools v0.1.1 h1:EVDuO03OCZwpV2t/tLLxPmPiomagMoBOgfPt0FM+4IY= honnef.co/go/tools v0.1.1/go.mod h1:NgwopIslSNH47DimFoV78dnkksY2EFtX0ajyb3K/las= @@ -1744,12 +1501,10 @@ rsc.io/binaryregexp v0.2.0/go.mod h1:qTv7/COck+e2FymRvadv62gMdZztPaShugOCi3I+8D8 rsc.io/pdf v0.1.1/go.mod h1:n8OzWcQ6Sp37PL01nO98y4iUCRdTGarVfzxY20ICaU4= rsc.io/quote/v3 v3.1.0/go.mod h1:yEA65RcK8LyAZtP9Kv3t0HmxON59tX3rD+tICJqUlj0= rsc.io/sampler v1.3.0/go.mod h1:T1hPZKmBbMNahiBKFy5HrXp6adAjACjK9JXDnKaTXpA= -sigs.k8s.io/yaml v1.1.0 h1:4A07+ZFc2wgJwo8YNlQpr1rVlgUDlxXHhPJciaPY5gs= sigs.k8s.io/yaml v1.1.0/go.mod h1:UJmg0vDUVViEyp3mgSv9WPwZCDxu4rQW1olrI1uml+o= sigs.k8s.io/yaml v1.2.0 h1:kr/MCeFWJWTwyaHoR9c8EjH9OumOmoF9YGiZd7lFm/Q= sigs.k8s.io/yaml v1.2.0/go.mod h1:yfXDCHCao9+ENCvLSE62v9VSji2MKu5jeNfTrofGhJc= software.sslmate.com/src/go-pkcs12 v0.0.0-20200619203921-c9ed90bd32dc/go.mod h1:/xvNRWUqm0+/ZMiF4EX00vrSCMsE4/NHb+Pt3freEeQ= -sourcegraph.com/sourcegraph/appdash v0.0.0-20180531100431-4c381bd170b4 h1:VO9oZbbkvTwqLimlQt15QNdOOBArT2dw/bvzsMZBiqQ= sourcegraph.com/sourcegraph/appdash v0.0.0-20180531100431-4c381bd170b4/go.mod h1:hI742Nqp5OhwiqlzhgfbWU4mW4yO10fP+LoT9WOswdU= sourcegraph.com/sourcegraph/appdash v0.0.0-20190731080439-ebfcffb1b5c0 h1:ucqkfpjg9WzSUubAO62csmucvxl4/JeW3F4I4909XkM= sourcegraph.com/sourcegraph/appdash v0.0.0-20190731080439-ebfcffb1b5c0/go.mod h1:hI742Nqp5OhwiqlzhgfbWU4mW4yO10fP+LoT9WOswdU= diff --git a/pkg/shardddl/optimism/lock.go b/pkg/shardddl/optimism/lock.go index e677534adb..51f6946b3e 100644 --- a/pkg/shardddl/optimism/lock.go +++ b/pkg/shardddl/optimism/lock.go @@ -170,7 +170,7 @@ func (l *Lock) TrySync(info Info, tts []TargetTable) (newDDLs []string, err erro // only update table info if no error or ignore conflict if ignoreConflict || err == nil { log.L().Info("update table info", zap.String("lock", l.ID), zap.String("source", callerSource), zap.String("schema", callerSchema), zap.String("table", callerTable), - zap.Stringer("from", prevTable), zap.Stringer("to", lastTableInfo), zap.Strings("ddls", ddls)) + zap.Stringer("from", l.tables[callerSource][callerSchema][callerTable]), zap.Stringer("to", lastTableInfo), zap.Strings("ddls", ddls)) l.tables[callerSource][callerSchema][callerTable] = lastTableInfo } }() @@ -198,9 +198,8 @@ func (l *Lock) TrySync(info Info, tts []TargetTable) (newDDLs []string, err erro nextTable = schemacmp.Encode(newTI) // special case: check whether DDLs making the schema become part of larger and another part of smaller. if _, err = prevTable.Compare(nextTable); err != nil { - err = terror.ErrShardDDLOptimismTrySyncFail.Delegate( + return emptyDDLs, terror.ErrShardDDLOptimismTrySyncFail.Delegate( err, l.ID, fmt.Sprintf("there will be conflicts if DDLs %s are applied to the downstream. old table info: %s, new table info: %s", ddls, prevTable, nextTable)) - return emptyDDLs, err } // special case: if the DDL does not affect the schema at all, assume it is @@ -225,7 +224,6 @@ func (l *Lock) TrySync(info Info, tts []TargetTable) (newDDLs []string, err erro // resolving conflict in non-intrusive mode. log.L().Warn("resolving conflict", zap.String("lock", l.ID), zap.String("source", callerSource), zap.String("schema", callerSchema), zap.String("table", callerTable), zap.Stringer("joined-from", oldJoined), zap.Stringer("joined-to", newJoined), zap.Strings("ddls", ddls)) - err = nil return ddls, nil } if cmp != 0 { @@ -282,9 +280,8 @@ func (l *Lock) TrySync(info Info, tts []TargetTable) (newDDLs []string, err erro // compare the current table's info with joined info. cmp, err = nextTable.Compare(newJoined) if err != nil { - err = terror.ErrShardDDLOptimismTrySyncFail.Delegate( + return emptyDDLs, terror.ErrShardDDLOptimismTrySyncFail.Delegate( err, l.ID, "can't compare table info (new table info) %s with (new joined table info) %s", nextTable, newJoined) // NOTE: this should not happen. - return emptyDDLs, err } if cmp < 0 { // no need to replicate DDLs, because has a larger joined schema (in the downstream). diff --git a/tests/shardddl1/run.sh b/tests/shardddl1/run.sh index 0f4497ee39..faf12d7824 100644 --- a/tests/shardddl1/run.sh +++ b/tests/shardddl1/run.sh @@ -591,16 +591,16 @@ function DM_RENAME_COLUMN_OPTIMISTIC() { function run() { init_cluster init_database -# start=1 -# end=35 -# except=(024 025 029) -# for i in $(seq -f "%03g" ${start} ${end}); do -# if [[ ${except[@]} =~ $i ]]; then -# continue -# fi -# DM_${i} -# sleep 1 -# done + start=1 + end=35 + except=(024 025 029) + for i in $(seq -f "%03g" ${start} ${end}); do + if [[ ${except[@]} =~ $i ]]; then + continue + fi + DM_${i} + sleep 1 + done DM_RENAME_COLUMN_OPTIMISTIC } From be6fd8785eead92ca1e6222e5636b67e079cd7c0 Mon Sep 17 00:00:00 2001 From: GMHDBJD <35025882+GMHDBJD@users.noreply.github.com> Date: Fri, 12 Mar 2021 16:26:55 +0800 Subject: [PATCH 07/17] Update dm/ctl/master/operate_schema.go Co-authored-by: lance6716 --- dm/ctl/master/operate_schema.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dm/ctl/master/operate_schema.go b/dm/ctl/master/operate_schema.go index 698c2ca290..0df828cdec 100644 --- a/dm/ctl/master/operate_schema.go +++ b/dm/ctl/master/operate_schema.go @@ -33,7 +33,7 @@ func NewOperateSchemaCmd() *cobra.Command { } cmd.Flags().StringP("database", "d", "", "database name of the table") cmd.Flags().StringP("table", "t", "", "table name") - cmd.Flags().Bool("flush", false, "flush the table info and checkpoint imediately") + cmd.Flags().Bool("flush", false, "flush the table info and checkpoint immediately") cmd.Flags().Bool("sync", false, "sync the table info to master to resolve shard ddl lock, only for optimistic mode now") return cmd } From 9faf54056d8c8c0944fe0c42678a309798461755 Mon Sep 17 00:00:00 2001 From: gmhdbjd Date: Fri, 12 Mar 2021 16:28:31 +0800 Subject: [PATCH 08/17] address comment --- syncer/checkpoint.go | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/syncer/checkpoint.go b/syncer/checkpoint.go index dac3179171..a30840f786 100644 --- a/syncer/checkpoint.go +++ b/syncer/checkpoint.go @@ -558,8 +558,8 @@ func (cp *RemoteCheckPoint) FlushPointsExcept(tctx *tcontext.Context, exceptTabl // FlushPointWithTableInfo implements CheckPoint.FlushPointWithTableInfo func (cp *RemoteCheckPoint) FlushPointWithTableInfo(tctx *tcontext.Context, schema string, table string, ti *model.TableInfo) error { - cp.RLock() - defer cp.RUnlock() + cp.Lock() + defer cp.Unlock() sqls := make([]string, 0, 10) args := make([][]interface{}, 0, 10) From 631188364448dc11515dd5122e1a85e59e0f19d6 Mon Sep 17 00:00:00 2001 From: gmhdbjd Date: Fri, 12 Mar 2021 17:09:32 +0800 Subject: [PATCH 09/17] use default db timeout --- syncer/checkpoint.go | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/syncer/checkpoint.go b/syncer/checkpoint.go index a30840f786..c105067ec4 100644 --- a/syncer/checkpoint.go +++ b/syncer/checkpoint.go @@ -34,6 +34,7 @@ import ( "github.com/pingcap/dm/pkg/log" "github.com/pingcap/dm/pkg/schema" "github.com/pingcap/dm/pkg/terror" + "github.com/pingcap/dm/pkg/utils" "github.com/pingcap/failpoint" "github.com/pingcap/parser/model" @@ -577,7 +578,7 @@ func (cp *RemoteCheckPoint) FlushPointWithTableInfo(tctx *tcontext.Context, sche args = append(args, arg) // use a new context apart from syncer, to make sure when syncer call `cancel` checkpoint could update - tctx2, cancel := tctx.WithContext(context.Background()).WithTimeout(maxDMLConnectionDuration) + tctx2, cancel := tctx.WithContext(context.Background()).WithTimeout(utils.DefaultDBTimeout) defer cancel() _, err = cp.dbConn.executeSQL(tctx2, sqls, args...) if err != nil { From 4a2f02058223cbd6125096b25b76ef2c09005192 Mon Sep 17 00:00:00 2001 From: GMHDBJD <35025882+GMHDBJD@users.noreply.github.com> Date: Mon, 15 Mar 2021 13:33:27 +0800 Subject: [PATCH 10/17] Update syncer/schema.go Co-authored-by: lance6716 --- syncer/schema.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/syncer/schema.go b/syncer/schema.go index 1ff981fa46..6d0d3526fd 100644 --- a/syncer/schema.go +++ b/syncer/schema.go @@ -104,7 +104,7 @@ func (s *Syncer) OperateSchema(ctx context.Context, req *pb.OperateWorkerSchemaR downSchema, downTable := s.renameShardingSchema(req.Database, req.Table) info := s.optimist.ConstructInfo(req.Database, req.Table, downSchema, downTable, []string{""}, nil, []*model.TableInfo{ti}) info.IgnoreConflict = true - log.L().Info("sync info with operateschema", zap.Stringer("info", info)) + log.L().Info("sync info with operate-schema", zap.Stringer("info", info)) _, err = s.optimist.PutInfo(info) if err != nil { return "", err From 4e178f29e301cd11a421b4467037713fbb1d422f Mon Sep 17 00:00:00 2001 From: gmhdbjd Date: Mon, 15 Mar 2021 14:04:05 +0800 Subject: [PATCH 11/17] make sure order --- tests/handle_error/run.sh | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/tests/handle_error/run.sh b/tests/handle_error/run.sh index 88e83d357e..bbb2bfb7c4 100644 --- a/tests/handle_error/run.sh +++ b/tests/handle_error/run.sh @@ -1420,6 +1420,10 @@ function DM_REPLACE_DEFAULT_VALUE_CASE() { run_sql_source1 "insert into ${db}.${tb1} values(4,4);" run_sql_source2 "insert into ${db}.${tb1} values(5);" run_sql_source2 "insert into ${db}.${tb2} values(6);" + + # make sure order is source1.table1, source2.table1, source2.table2 + run_sql_tidb_with_retry "select count(1) from ${db}.${tb}" "count(1): 6" + run_sql_source2 "alter table ${db}.${tb1} add new_col1 int default 2;" run_sql_source1 "insert into ${db}.${tb1} values(7,7);" run_sql_source2 "insert into ${db}.${tb1} values(8,8);" From 8bd6a01f0cd1858607faf37b810f1c9642dcde54 Mon Sep 17 00:00:00 2001 From: GMHDBJD <35025882+GMHDBJD@users.noreply.github.com> Date: Wed, 17 Mar 2021 17:01:17 +0800 Subject: [PATCH 12/17] Update syncer/checkpoint.go Co-authored-by: Chunzhu Li --- syncer/checkpoint.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/syncer/checkpoint.go b/syncer/checkpoint.go index c105067ec4..8c55ccadda 100644 --- a/syncer/checkpoint.go +++ b/syncer/checkpoint.go @@ -213,7 +213,7 @@ type CheckPoint interface { FlushPointsExcept(tctx *tcontext.Context, exceptTables [][]string, extraSQLs []string, extraArgs [][]interface{}) error // FlushPointWithTableInfo flushed the table point with given table info - FlushPointWithTableInfo(tctx *tcontext.Context, schema string, table string, ti *model.TableInfo) error + FlushPointWithTableInfo(tctx *tcontext.Context, schema, table string, ti *model.TableInfo) error // GlobalPoint returns the global binlog stream's checkpoint // corresponding to Meta.Pos and Meta.GTID From 0f0ec9a3a4da7c00d38805debf5e44b12080557f Mon Sep 17 00:00:00 2001 From: GMHDBJD <35025882+GMHDBJD@users.noreply.github.com> Date: Wed, 17 Mar 2021 17:01:34 +0800 Subject: [PATCH 13/17] Update syncer/checkpoint.go Co-authored-by: Chunzhu Li --- syncer/checkpoint.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/syncer/checkpoint.go b/syncer/checkpoint.go index 8c55ccadda..aadd9d4822 100644 --- a/syncer/checkpoint.go +++ b/syncer/checkpoint.go @@ -562,7 +562,7 @@ func (cp *RemoteCheckPoint) FlushPointWithTableInfo(tctx *tcontext.Context, sche cp.Lock() defer cp.Unlock() - sqls := make([]string, 0, 10) + sqls := make([]string, 0, 1) args := make([][]interface{}, 0, 10) point := cp.points[schema][table] From 17bab660a27fdd528dae58738d742cdf1df21843 Mon Sep 17 00:00:00 2001 From: gmhdbjd Date: Wed, 17 Mar 2021 17:04:13 +0800 Subject: [PATCH 14/17] rename args --- dm/master/shardddl/optimist.go | 4 ++++ syncer/checkpoint.go | 10 +++++----- 2 files changed, 9 insertions(+), 5 deletions(-) diff --git a/dm/master/shardddl/optimist.go b/dm/master/shardddl/optimist.go index 7cd7eca1c5..6e32d0a3b6 100644 --- a/dm/master/shardddl/optimist.go +++ b/dm/master/shardddl/optimist.go @@ -500,6 +500,10 @@ func (o *Optimist) handleLock(info optimism.Info, tts []optimism.TargetTable, sk lockID, newDDLs, err := o.lk.TrySync(info, tts) var cfStage = optimism.ConflictNone if info.IgnoreConflict { + if err != nil { + o.logger.Warn("error occur when trying to sync for shard DDL info, this often means shard DDL conflict detected", + zap.String("lock", lockID), zap.Stringer("info", info), zap.Bool("is deleted", info.IsDeleted), log.ShortError(err)) + } o.logger.Warn("handle lock in ignore conflict mode", zap.String("lock", lockID), zap.Stringer("info", info)) } else if err != nil { cfStage = optimism.ConflictDetected // we treat any errors returned from `TrySync` as conflict detected now. diff --git a/syncer/checkpoint.go b/syncer/checkpoint.go index aadd9d4822..89c7ff4c84 100644 --- a/syncer/checkpoint.go +++ b/syncer/checkpoint.go @@ -213,7 +213,7 @@ type CheckPoint interface { FlushPointsExcept(tctx *tcontext.Context, exceptTables [][]string, extraSQLs []string, extraArgs [][]interface{}) error // FlushPointWithTableInfo flushed the table point with given table info - FlushPointWithTableInfo(tctx *tcontext.Context, schema, table string, ti *model.TableInfo) error + FlushPointWithTableInfo(tctx *tcontext.Context, sourceSchema, sourceTable string, ti *model.TableInfo) error // GlobalPoint returns the global binlog stream's checkpoint // corresponding to Meta.Pos and Meta.GTID @@ -558,22 +558,22 @@ func (cp *RemoteCheckPoint) FlushPointsExcept(tctx *tcontext.Context, exceptTabl } // FlushPointWithTableInfo implements CheckPoint.FlushPointWithTableInfo -func (cp *RemoteCheckPoint) FlushPointWithTableInfo(tctx *tcontext.Context, schema string, table string, ti *model.TableInfo) error { +func (cp *RemoteCheckPoint) FlushPointWithTableInfo(tctx *tcontext.Context, sourceSchema string, sourceTable string, ti *model.TableInfo) error { cp.Lock() defer cp.Unlock() sqls := make([]string, 0, 1) args := make([][]interface{}, 0, 10) - point := cp.points[schema][table] + point := cp.points[sourceSchema][sourceTable] tiBytes, err := json.Marshal(ti) if err != nil { - return terror.ErrSchemaTrackerCannotSerialize.Delegate(err, schema, table) + return terror.ErrSchemaTrackerCannotSerialize.Delegate(err, sourceSchema, sourceTable) } location := point.MySQLLocation() - sql2, arg := cp.genUpdateSQL(schema, table, location, nil, tiBytes, false) + sql2, arg := cp.genUpdateSQL(sourceSchema, sourceTable, location, nil, tiBytes, false) sqls = append(sqls, sql2) args = append(args, arg) From f44e9f1e7d06fc0c2053e29b914f81204a122d43 Mon Sep 17 00:00:00 2001 From: gmhdbjd Date: Wed, 17 Mar 2021 17:45:47 +0800 Subject: [PATCH 15/17] address comment --- pkg/shardddl/optimism/lock_test.go | 282 ++++++++--------------------- 1 file changed, 78 insertions(+), 204 deletions(-) diff --git a/pkg/shardddl/optimism/lock_test.go b/pkg/shardddl/optimism/lock_test.go index 86760731fd..44b39f315b 100644 --- a/pkg/shardddl/optimism/lock_test.go +++ b/pkg/shardddl/optimism/lock_test.go @@ -65,8 +65,7 @@ func (t *testLock) TestLockTrySyncNormal(c *C) { newTargetTable(task, sources[1], downSchema, downTable, tables), } - l = NewLock(ID, task, downSchema, downTable, ti0, tts) - info = NewInfo("", "", "", "", "", "", []string{}, nil, nil) + l = NewLock(ID, task, downSchema, downTable, ti0, tts) vers = map[string]map[string]map[string]int64{ sources[0]: { @@ -101,9 +100,7 @@ func (t *testLock) TestLockTrySyncNormal(c *C) { for _, db := range dbs { for _, tbl := range tbls { - vers[source][db][tbl]++ - info = NewInfo(task, source, db, tbl, downSchema, downTable, DDLs1, nil, []*model.TableInfo{ti1}) - info.Version = vers[source][db][tbl] + info := newInfoWithVersion(task, source, db, tbl, downSchema, downTable, DDLs1, nil, []*model.TableInfo{ti1}, vers) DDLs, err := l.TrySync(info, tts) c.Assert(err, IsNil) c.Assert(DDLs, DeepEquals, DDLs1) @@ -122,9 +119,7 @@ func (t *testLock) TestLockTrySyncNormal(c *C) { t.checkLockNoDone(c, l) // CASE: TrySync again after synced is idempotent. - vers[sources[0]][dbs[0]][tbls[0]]++ - info = NewInfo(task, sources[0], dbs[0], tbls[0], downSchema, downTable, DDLs1, nil, []*model.TableInfo{ti1}) - info.Version = vers[sources[0]][dbs[0]][tbls[0]] + info := newInfoWithVersion(task, sources[0], dbs[0], tbls[0], downSchema, downTable, DDLs1, nil, []*model.TableInfo{ti1}, vers) DDLs, err := l.TrySync(info, tts) c.Assert(err, IsNil) c.Assert(DDLs, DeepEquals, DDLs1) @@ -134,9 +129,7 @@ func (t *testLock) TestLockTrySyncNormal(c *C) { // CASE: need to add more than one DDL to reach the desired schema (schema become larger). // add two columns for one table. - vers[sources[0]][dbs[0]][tbls[0]]++ - info = NewInfo(task, sources[0], dbs[0], tbls[0], downSchema, downTable, DDLs2, nil, []*model.TableInfo{ti2_1, ti2}) - info.Version = vers[sources[0]][dbs[0]][tbls[0]] + info = newInfoWithVersion(task, sources[0], dbs[0], tbls[0], downSchema, downTable, DDLs2, nil, []*model.TableInfo{ti2_1, ti2}, vers) DDLs, err = l.TrySync(info, tts) c.Assert(err, IsNil) c.Assert(DDLs, DeepEquals, DDLs2) @@ -146,9 +139,7 @@ func (t *testLock) TestLockTrySyncNormal(c *C) { c.Assert(ready[sources[0]][dbs[0]][tbls[1]], IsFalse) // TrySync again is idempotent (more than one DDL). - vers[sources[0]][dbs[0]][tbls[0]]++ - info = NewInfo(task, sources[0], dbs[0], tbls[0], downSchema, downTable, DDLs2, nil, []*model.TableInfo{ti2_1, ti2}) - info.Version = vers[sources[0]][dbs[0]][tbls[0]] + info = newInfoWithVersion(task, sources[0], dbs[0], tbls[0], downSchema, downTable, DDLs2, nil, []*model.TableInfo{ti2_1, ti2}, vers) DDLs, err = l.TrySync(info, tts) c.Assert(err, IsNil) c.Assert(DDLs, DeepEquals, DDLs2) @@ -158,9 +149,7 @@ func (t *testLock) TestLockTrySyncNormal(c *C) { c.Assert(ready[sources[0]][dbs[0]][tbls[1]], IsFalse) // add only the first column for another table. - vers[sources[0]][dbs[0]][tbls[1]]++ - info = NewInfo(task, sources[0], dbs[0], tbls[1], downSchema, downTable, DDLs2[0:1], nil, []*model.TableInfo{ti2_1}) - info.Version = vers[sources[0]][dbs[0]][tbls[1]] + info = newInfoWithVersion(task, sources[0], dbs[0], tbls[1], downSchema, downTable, DDLs2[0:1], nil, []*model.TableInfo{ti2_1}, vers) DDLs, err = l.TrySync(info, tts) c.Assert(err, IsNil) c.Assert(DDLs, DeepEquals, DDLs2[0:1]) @@ -177,9 +166,7 @@ func (t *testLock) TestLockTrySyncNormal(c *C) { c.Assert(cmp, Equals, 1) // TrySync again (only the first DDL). - vers[sources[0]][dbs[0]][tbls[1]]++ - info = NewInfo(task, sources[0], dbs[0], tbls[1], downSchema, downTable, DDLs2[0:1], nil, []*model.TableInfo{ti2_1}) - info.Version = vers[sources[0]][dbs[0]][tbls[1]] + info = newInfoWithVersion(task, sources[0], dbs[0], tbls[1], downSchema, downTable, DDLs2[0:1], nil, []*model.TableInfo{ti2_1}, vers) DDLs, err = l.TrySync(info, tts) c.Assert(err, IsNil) c.Assert(DDLs, DeepEquals, []string{}) // NOTE: special case, joined has larger schema. @@ -188,9 +175,7 @@ func (t *testLock) TestLockTrySyncNormal(c *C) { c.Assert(ready[sources[0]][dbs[0]][tbls[1]], IsFalse) // add the second column for another table. - vers[sources[0]][dbs[0]][tbls[1]]++ - info = NewInfo(task, sources[0], dbs[0], tbls[1], downSchema, downTable, DDLs2[1:2], nil, []*model.TableInfo{ti2}) - info.Version = vers[sources[0]][dbs[0]][tbls[1]] + info = newInfoWithVersion(task, sources[0], dbs[0], tbls[1], downSchema, downTable, DDLs2[1:2], nil, []*model.TableInfo{ti2}, vers) DDLs, err = l.TrySync(info, tts) c.Assert(err, IsNil) c.Assert(DDLs, DeepEquals, DDLs2[1:2]) @@ -206,9 +191,7 @@ func (t *testLock) TestLockTrySyncNormal(c *C) { c.Assert(cmp, Equals, 0) // Try again (for the second DDL). - vers[sources[0]][dbs[0]][tbls[1]]++ - info = NewInfo(task, sources[0], dbs[0], tbls[1], downSchema, downTable, DDLs2[1:2], nil, []*model.TableInfo{ti2}) - info.Version = vers[sources[0]][dbs[0]][tbls[1]] + info = newInfoWithVersion(task, sources[0], dbs[0], tbls[1], downSchema, downTable, DDLs2[1:2], nil, []*model.TableInfo{ti2}, vers) DDLs, err = l.TrySync(info, tts) c.Assert(err, IsNil) c.Assert(DDLs, DeepEquals, DDLs2[1:2]) @@ -247,9 +230,7 @@ func (t *testLock) TestLockTrySyncNormal(c *C) { for _, db := range dbs { for _, tbl := range tbls { syncedCount++ - vers[source][db][tbl]++ - info = NewInfo(task, source, db, tbl, downSchema, downTable, DDLs3, nil, []*model.TableInfo{ti3}) - info.Version = vers[source][db][tbl] + info = newInfoWithVersion(task, source, db, tbl, downSchema, downTable, DDLs3, nil, []*model.TableInfo{ti3}, vers) DDLs, err = l.TrySync(info, tts) c.Assert(err, IsNil) c.Assert(l.versions, DeepEquals, vers) @@ -272,9 +253,7 @@ func (t *testLock) TestLockTrySyncNormal(c *C) { // CASE: need to drop more than one DDL to reach the desired schema (schema become smaller). // drop two columns for one table. - vers[sources[0]][dbs[0]][tbls[0]]++ - info = NewInfo(task, sources[0], dbs[0], tbls[0], downSchema, downTable, DDLs4, nil, []*model.TableInfo{ti4_1, ti4}) - info.Version = vers[sources[0]][dbs[0]][tbls[0]] + info = newInfoWithVersion(task, sources[0], dbs[0], tbls[0], downSchema, downTable, DDLs4, nil, []*model.TableInfo{ti4_1, ti4}, vers) DDLs, err = l.TrySync(info, tts) c.Assert(err, IsNil) c.Assert(DDLs, DeepEquals, []string{}) @@ -284,9 +263,7 @@ func (t *testLock) TestLockTrySyncNormal(c *C) { c.Assert(ready[sources[0]][dbs[0]][tbls[1]], IsTrue) // TrySync again is idempotent. - vers[sources[0]][dbs[0]][tbls[0]]++ - info = NewInfo(task, sources[0], dbs[0], tbls[0], downSchema, downTable, DDLs4, nil, []*model.TableInfo{ti4_1, ti4}) - info.Version = vers[sources[0]][dbs[0]][tbls[0]] + info = newInfoWithVersion(task, sources[0], dbs[0], tbls[0], downSchema, downTable, DDLs4, nil, []*model.TableInfo{ti4_1, ti4}, vers) DDLs, err = l.TrySync(info, tts) c.Assert(err, IsNil) c.Assert(DDLs, DeepEquals, DDLs4[:1]) @@ -296,9 +273,7 @@ func (t *testLock) TestLockTrySyncNormal(c *C) { c.Assert(ready[sources[0]][dbs[0]][tbls[1]], IsTrue) // drop only the first column for another table. - vers[sources[0]][dbs[0]][tbls[1]]++ - info = NewInfo(task, sources[0], dbs[0], tbls[1], downSchema, downTable, DDLs4[0:1], nil, []*model.TableInfo{ti4_1}) - info.Version = vers[sources[0]][dbs[0]][tbls[1]] + info = newInfoWithVersion(task, sources[0], dbs[0], tbls[1], downSchema, downTable, DDLs4[0:1], nil, []*model.TableInfo{ti4_1}, vers) DDLs, err = l.TrySync(info, tts) c.Assert(err, IsNil) c.Assert(DDLs, DeepEquals, []string{}) @@ -311,18 +286,14 @@ func (t *testLock) TestLockTrySyncNormal(c *C) { c.Assert(cmp, Equals, -1) // TrySync again (only the first DDL). - vers[sources[0]][dbs[0]][tbls[1]]++ - info = NewInfo(task, sources[0], dbs[0], tbls[1], downSchema, downTable, DDLs4[0:1], nil, []*model.TableInfo{ti4_1}) - info.Version = vers[sources[0]][dbs[0]][tbls[1]] + info = newInfoWithVersion(task, sources[0], dbs[0], tbls[1], downSchema, downTable, DDLs4[0:1], nil, []*model.TableInfo{ti4_1}, vers) DDLs, err = l.TrySync(info, tts) c.Assert(err, IsNil) c.Assert(DDLs, DeepEquals, []string{}) c.Assert(l.versions, DeepEquals, vers) // drop the second column for another table. - vers[sources[0]][dbs[0]][tbls[1]]++ - info = NewInfo(task, sources[0], dbs[0], tbls[1], downSchema, downTable, DDLs4[1:2], nil, []*model.TableInfo{ti4}) - info.Version = vers[sources[0]][dbs[0]][tbls[1]] + info = newInfoWithVersion(task, sources[0], dbs[0], tbls[1], downSchema, downTable, DDLs4[1:2], nil, []*model.TableInfo{ti4}, vers) DDLs, err = l.TrySync(info, tts) c.Assert(err, IsNil) c.Assert(DDLs, DeepEquals, []string{}) @@ -335,9 +306,7 @@ func (t *testLock) TestLockTrySyncNormal(c *C) { c.Assert(cmp, Equals, 0) // TrySync again (for the second DDL). - vers[sources[0]][dbs[0]][tbls[1]]++ - info = NewInfo(task, sources[0], dbs[0], tbls[1], downSchema, downTable, DDLs4[1:2], nil, []*model.TableInfo{ti4}) - info.Version = vers[sources[0]][dbs[0]][tbls[1]] + info = newInfoWithVersion(task, sources[0], dbs[0], tbls[1], downSchema, downTable, DDLs4[1:2], nil, []*model.TableInfo{ti4}, vers) DDLs, err = l.TrySync(info, tts) c.Assert(err, IsNil) c.Assert(DDLs, DeepEquals, []string{}) @@ -349,8 +318,7 @@ func (t *testLock) TestLockTrySyncNormal(c *C) { for schema, tables := range schemaTables { for table, synced2 := range tables { if synced2 { // do not `TrySync` again for previous two (un-synced now). - info = NewInfo(task, source, schema, table, downSchema, downTable, DDLs4, nil, []*model.TableInfo{ti4_1, ti4}) - info.Version = vers[source][schema][table] + info = newInfoWithVersion(task, source, schema, table, downSchema, downTable, DDLs4, nil, []*model.TableInfo{ti4_1, ti4}, vers) DDLs, err = l.TrySync(info, tts) c.Assert(err, IsNil) c.Assert(l.versions, DeepEquals, vers) @@ -407,9 +375,7 @@ func (t *testLock) TestLockTrySyncIndex(c *C) { // try sync for one table, `DROP INDEX` returned directly (to make schema become more compatible). // `DROP INDEX` is handled like `ADD COLUMN`. - vers[source][db][tbls[0]]++ - info := NewInfo(task, source, db, tbls[0], downSchema, downTable, DDLs1, nil, []*model.TableInfo{ti1}) - info.Version = vers[source][db][tbls[0]] + info := newInfoWithVersion(task, source, db, tbls[0], downSchema, downTable, DDLs1, nil, []*model.TableInfo{ti1}, vers) DDLs, err := l.TrySync(info, tts) c.Assert(err, IsNil) c.Assert(DDLs, DeepEquals, DDLs1) @@ -420,9 +386,7 @@ func (t *testLock) TestLockTrySyncIndex(c *C) { c.Assert(remain, Equals, 1) // try sync for another table, also got `DROP INDEX` now. - vers[source][db][tbls[1]]++ - info = NewInfo(task, source, db, tbls[1], downSchema, downTable, DDLs1, nil, []*model.TableInfo{ti1}) - info.Version = vers[source][db][tbls[1]] + info = newInfoWithVersion(task, source, db, tbls[1], downSchema, downTable, DDLs1, nil, []*model.TableInfo{ti1}, vers) DDLs, err = l.TrySync(info, tts) c.Assert(err, IsNil) c.Assert(DDLs, DeepEquals, DDLs1) @@ -431,9 +395,7 @@ func (t *testLock) TestLockTrySyncIndex(c *C) { // try sync for one table, `ADD INDEX` not returned directly (to keep the schema more compatible). // `ADD INDEX` is handled like `DROP COLUMN`. - vers[source][db][tbls[0]]++ - info = NewInfo(task, source, db, tbls[0], downSchema, downTable, DDLs2, nil, []*model.TableInfo{ti2}) - info.Version = vers[source][db][tbls[0]] + info = newInfoWithVersion(task, source, db, tbls[0], downSchema, downTable, DDLs2, nil, []*model.TableInfo{ti2}, vers) DDLs, err = l.TrySync(info, tts) c.Assert(err, IsNil) c.Assert(DDLs, DeepEquals, []string{}) // no DDLs returned @@ -444,9 +406,7 @@ func (t *testLock) TestLockTrySyncIndex(c *C) { c.Assert(remain, Equals, 1) // try sync for another table, got `ADD INDEX` now. - vers[source][db][tbls[1]]++ - info = NewInfo(task, source, db, tbls[1], downSchema, downTable, DDLs2, nil, []*model.TableInfo{ti2}) - info.Version = vers[source][db][tbls[1]] + info = newInfoWithVersion(task, source, db, tbls[1], downSchema, downTable, DDLs2, nil, []*model.TableInfo{ti2}, vers) DDLs, err = l.TrySync(info, tts) c.Assert(err, IsNil) c.Assert(DDLs, DeepEquals, DDLs2) @@ -494,36 +454,28 @@ func (t *testLock) TestLockTrySyncNullNotNull(c *C) { for i := 0; i < 2; i++ { // two round // try sync for one table, from `NULL` to `NOT NULL`, no DDLs returned. - vers[source][db][tbls[0]]++ - info := NewInfo(task, source, db, tbls[0], downSchema, downTable, DDLs1, nil, []*model.TableInfo{ti1}) - info.Version = vers[source][db][tbls[0]] + info := newInfoWithVersion(task, source, db, tbls[0], downSchema, downTable, DDLs1, nil, []*model.TableInfo{ti1}, vers) DDLs, err := l.TrySync(info, tts) c.Assert(err, IsNil) c.Assert(DDLs, DeepEquals, []string{}) c.Assert(l.versions, DeepEquals, vers) // try sync for another table, DDLs returned. - vers[source][db][tbls[1]]++ - info = NewInfo(task, source, db, tbls[1], downSchema, downTable, DDLs1, nil, []*model.TableInfo{ti1}) - info.Version = vers[source][db][tbls[1]] + info = newInfoWithVersion(task, source, db, tbls[1], downSchema, downTable, DDLs1, nil, []*model.TableInfo{ti1}, vers) DDLs, err = l.TrySync(info, tts) c.Assert(err, IsNil) c.Assert(DDLs, DeepEquals, DDLs1) c.Assert(l.versions, DeepEquals, vers) // try sync for one table, from `NOT NULL` to `NULL`, DDLs returned. - vers[source][db][tbls[0]]++ - info = NewInfo(task, source, db, tbls[0], downSchema, downTable, DDLs2, nil, []*model.TableInfo{ti2}) - info.Version = vers[source][db][tbls[0]] + info = newInfoWithVersion(task, source, db, tbls[0], downSchema, downTable, DDLs2, nil, []*model.TableInfo{ti2}, vers) DDLs, err = l.TrySync(info, tts) c.Assert(err, IsNil) c.Assert(DDLs, DeepEquals, DDLs2) c.Assert(l.versions, DeepEquals, vers) // try sync for another table, from `NOT NULL` to `NULL`, DDLs, returned. - vers[source][db][tbls[1]]++ - info = NewInfo(task, source, db, tbls[1], downSchema, downTable, DDLs2, nil, []*model.TableInfo{ti2}) - info.Version = vers[source][db][tbls[1]] + info = newInfoWithVersion(task, source, db, tbls[1], downSchema, downTable, DDLs2, nil, []*model.TableInfo{ti2}, vers) DDLs, err = l.TrySync(info, tts) c.Assert(err, IsNil) c.Assert(DDLs, DeepEquals, DDLs2) @@ -568,18 +520,14 @@ func (t *testLock) TestLockTrySyncIntBigint(c *C) { t.checkLockNoDone(c, l) // try sync for one table, from `INT` to `BIGINT`, DDLs returned. - vers[source][db][tbls[0]]++ - info := NewInfo(task, source, db, tbls[0], downSchema, downTable, DDLs1, nil, []*model.TableInfo{ti1}) - info.Version = vers[source][db][tbls[0]] + info := newInfoWithVersion(task, source, db, tbls[0], downSchema, downTable, DDLs1, nil, []*model.TableInfo{ti1}, vers) DDLs, err := l.TrySync(info, tts) c.Assert(err, IsNil) c.Assert(DDLs, DeepEquals, DDLs1) c.Assert(l.versions, DeepEquals, vers) // try sync for another table, DDLs returned. - vers[source][db][tbls[1]]++ - info = NewInfo(task, source, db, tbls[1], downSchema, downTable, DDLs1, nil, []*model.TableInfo{ti1}) - info.Version = vers[source][db][tbls[1]] + info = newInfoWithVersion(task, source, db, tbls[1], downSchema, downTable, DDLs1, nil, []*model.TableInfo{ti1}, vers) DDLs, err = l.TrySync(info, tts) c.Assert(err, IsNil) c.Assert(DDLs, DeepEquals, DDLs1) @@ -623,9 +571,7 @@ func (t *testLock) TestLockTrySyncNoDiff(c *C) { t.checkLockNoDone(c, l) // try sync for one table. - vers[source][db][tbls[0]]++ - info := NewInfo(task, source, db, tbls[0], downSchema, downTable, DDLs1, nil, []*model.TableInfo{ti1}) - info.Version = vers[source][db][tbls[0]] + info := newInfoWithVersion(task, source, db, tbls[0], downSchema, downTable, DDLs1, nil, []*model.TableInfo{ti1}, vers) DDLs, err := l.TrySync(info, tts) c.Assert(terror.ErrShardDDLOptimismTrySyncFail.Equal(err), IsTrue) c.Assert(DDLs, DeepEquals, []string{}) @@ -669,9 +615,7 @@ func (t *testLock) TestLockTrySyncNewTable(c *C) { t.checkLockNoDone(c, l) // TrySync for a new table as the caller. - vers[source2][db2][tbl2]++ - info := NewInfo(task, source2, db2, tbl2, downSchema, downTable, DDLs1, nil, []*model.TableInfo{ti1}) - info.Version = vers[source2][db2][tbl2] + info := newInfoWithVersion(task, source2, db2, tbl2, downSchema, downTable, DDLs1, nil, []*model.TableInfo{ti1}, vers) DDLs, err := l.TrySync(info, tts) c.Assert(err, IsNil) c.Assert(DDLs, DeepEquals, DDLs1) @@ -695,9 +639,7 @@ func (t *testLock) TestLockTrySyncNewTable(c *C) { vers[source1][db1][tbl2] = 0 vers[source2][db2][tbl1] = 0 - vers[source1][db1][tbl1]++ - info = NewInfo(task, source1, db1, tbl1, downSchema, downTable, DDLs1, nil, []*model.TableInfo{ti1}) - info.Version = vers[source1][db1][tbl1] + info = newInfoWithVersion(task, source1, db1, tbl1, downSchema, downTable, DDLs1, nil, []*model.TableInfo{ti1}, vers) DDLs, err = l.TrySync(info, tts) c.Assert(err, IsNil) c.Assert(DDLs, DeepEquals, DDLs1) @@ -765,9 +707,7 @@ func (t *testLock) TestLockTrySyncRevert(c *C) { // CASE: revert for single DDL. // TrySync for one table. - vers[source][db][tbls[0]]++ - info := NewInfo(task, source, db, tbls[0], downSchema, downTable, DDLs1, nil, []*model.TableInfo{ti1}) - info.Version = vers[source][db][tbls[0]] + info := newInfoWithVersion(task, source, db, tbls[0], downSchema, downTable, DDLs1, nil, []*model.TableInfo{ti1}, vers) DDLs, err := l.TrySync(info, tts) c.Assert(err, IsNil) c.Assert(DDLs, DeepEquals, DDLs1) @@ -783,9 +723,7 @@ func (t *testLock) TestLockTrySyncRevert(c *C) { c.Assert(cmp, Equals, -1) // revert for the table, become synced again. - vers[source][db][tbls[0]]++ - info = NewInfo(task, source, db, tbls[0], downSchema, downTable, DDLs2, nil, []*model.TableInfo{ti2}) - info.Version = vers[source][db][tbls[0]] + info = newInfoWithVersion(task, source, db, tbls[0], downSchema, downTable, DDLs2, nil, []*model.TableInfo{ti2}, vers) DDLs, err = l.TrySync(info, tts) c.Assert(err, IsNil) c.Assert(DDLs, DeepEquals, DDLs2) @@ -795,9 +733,7 @@ func (t *testLock) TestLockTrySyncRevert(c *C) { // CASE: revert for multiple DDLs. // TrySync for one table. - vers[source][db][tbls[0]]++ - info = NewInfo(task, source, db, tbls[0], downSchema, downTable, DDLs3, nil, []*model.TableInfo{ti4, ti3}) - info.Version = vers[source][db][tbls[0]] + info = newInfoWithVersion(task, source, db, tbls[0], downSchema, downTable, DDLs3, nil, []*model.TableInfo{ti4, ti3}, vers) DDLs, err = l.TrySync(info, tts) c.Assert(err, IsNil) c.Assert(DDLs, DeepEquals, DDLs3) @@ -813,9 +749,7 @@ func (t *testLock) TestLockTrySyncRevert(c *C) { c.Assert(cmp, Equals, -1) // revert part of the DDLs. - vers[source][db][tbls[0]]++ - info = NewInfo(task, source, db, tbls[0], downSchema, downTable, DDLs4, nil, []*model.TableInfo{ti4}) - info.Version = vers[source][db][tbls[0]] + info = newInfoWithVersion(task, source, db, tbls[0], downSchema, downTable, DDLs4, nil, []*model.TableInfo{ti4}, vers) DDLs, err = l.TrySync(info, tts) c.Assert(err, IsNil) c.Assert(DDLs, DeepEquals, DDLs4) @@ -830,9 +764,7 @@ func (t *testLock) TestLockTrySyncRevert(c *C) { c.Assert(cmp, Equals, -1) // revert the reset part of the DDLs. - vers[source][db][tbls[0]]++ - info = NewInfo(task, source, db, tbls[0], downSchema, downTable, DDLs5, nil, []*model.TableInfo{ti5}) - info.Version = vers[source][db][tbls[0]] + info = newInfoWithVersion(task, source, db, tbls[0], downSchema, downTable, DDLs5, nil, []*model.TableInfo{ti5}, vers) DDLs, err = l.TrySync(info, tts) c.Assert(err, IsNil) c.Assert(DDLs, DeepEquals, DDLs5) @@ -842,9 +774,7 @@ func (t *testLock) TestLockTrySyncRevert(c *C) { // CASE: revert part of multiple DDLs. // TrySync for one table. - vers[source][db][tbls[0]]++ - info = NewInfo(task, source, db, tbls[0], downSchema, downTable, DDLs6, nil, []*model.TableInfo{ti7, ti6}) - info.Version = vers[source][db][tbls[0]] + info = newInfoWithVersion(task, source, db, tbls[0], downSchema, downTable, DDLs6, nil, []*model.TableInfo{ti7, ti6}, vers) DDLs, err = l.TrySync(info, tts) c.Assert(err, IsNil) c.Assert(DDLs, DeepEquals, DDLs6) @@ -859,9 +789,7 @@ func (t *testLock) TestLockTrySyncRevert(c *C) { c.Assert(cmp, Equals, -1) // revert part of the DDLs. - vers[source][db][tbls[0]]++ - info = NewInfo(task, source, db, tbls[0], downSchema, downTable, DDLs7, nil, []*model.TableInfo{ti7}) - info.Version = vers[source][db][tbls[0]] + info = newInfoWithVersion(task, source, db, tbls[0], downSchema, downTable, DDLs7, nil, []*model.TableInfo{ti7}, vers) DDLs, err = l.TrySync(info, tts) c.Assert(err, IsNil) c.Assert(DDLs, DeepEquals, DDLs7) @@ -876,9 +804,7 @@ func (t *testLock) TestLockTrySyncRevert(c *C) { c.Assert(cmp, Equals, -1) // TrySync for another table. - vers[source][db][tbls[1]]++ - info = NewInfo(task, source, db, tbls[1], downSchema, downTable, DDLs8, nil, []*model.TableInfo{ti8}) - info.Version = vers[source][db][tbls[1]] + info = newInfoWithVersion(task, source, db, tbls[1], downSchema, downTable, DDLs8, nil, []*model.TableInfo{ti8}, vers) DDLs, err = l.TrySync(info, tts) c.Assert(err, IsNil) c.Assert(DDLs, DeepEquals, DDLs8) @@ -927,9 +853,7 @@ func (t *testLock) TestLockTrySyncConflictNonIntrusive(c *C) { t.checkLockNoDone(c, l) // TrySync for the first table, construct the joined schema. - vers[source][db][tbls[0]]++ - info := NewInfo(task, source, db, tbls[0], downSchema, downTable, DDLs1, nil, []*model.TableInfo{ti1}) - info.Version = vers[source][db][tbls[0]] + info := newInfoWithVersion(task, source, db, tbls[0], downSchema, downTable, DDLs1, nil, []*model.TableInfo{ti1}, vers) DDLs, err := l.TrySync(info, tts) c.Assert(err, IsNil) c.Assert(DDLs, DeepEquals, DDLs1) @@ -945,9 +869,7 @@ func (t *testLock) TestLockTrySyncConflictNonIntrusive(c *C) { c.Assert(cmp, Equals, -1) // TrySync for the second table with another schema (add two columns, one of them will cause conflict). - vers[source][db][tbls[1]]++ - info = NewInfo(task, source, db, tbls[1], downSchema, downTable, DDLs2, nil, []*model.TableInfo{ti2_1, ti2}) - info.Version = vers[source][db][tbls[1]] + info = newInfoWithVersion(task, source, db, tbls[1], downSchema, downTable, DDLs2, nil, []*model.TableInfo{ti2_1, ti2}, vers) DDLs, err = l.TrySync(info, tts) c.Assert(terror.ErrShardDDLOptimismTrySyncFail.Equal(err), IsTrue) c.Assert(DDLs, DeepEquals, []string{}) @@ -960,9 +882,7 @@ func (t *testLock) TestLockTrySyncConflictNonIntrusive(c *C) { c.Assert(ready[source][db][tbls[1]], IsFalse) // TrySync for the first table to resolve the conflict. - vers[source][db][tbls[0]]++ - info = NewInfo(task, source, db, tbls[0], downSchema, downTable, DDLs3, nil, []*model.TableInfo{ti3}) - info.Version = vers[source][db][tbls[0]] + info = newInfoWithVersion(task, source, db, tbls[0], downSchema, downTable, DDLs3, nil, []*model.TableInfo{ti3}, vers) DDLs, err = l.TrySync(info, tts) c.Assert(err, IsNil) c.Assert(DDLs, DeepEquals, DDLs3) @@ -978,9 +898,7 @@ func (t *testLock) TestLockTrySyncConflictNonIntrusive(c *C) { c.Assert(cmp, Equals, 0) // TrySync for the second table, succeed now - vers[source][db][tbls[1]]++ - info = NewInfo(task, source, db, tbls[1], downSchema, downTable, DDLs2, nil, []*model.TableInfo{ti2_1, ti2}) - info.Version = vers[source][db][tbls[1]] + info = newInfoWithVersion(task, source, db, tbls[1], downSchema, downTable, DDLs2, nil, []*model.TableInfo{ti2_1, ti2}, vers) DDLs, err = l.TrySync(info, tts) c.Assert(err, IsNil) c.Assert(DDLs, DeepEquals, DDLs2) @@ -992,9 +910,7 @@ func (t *testLock) TestLockTrySyncConflictNonIntrusive(c *C) { c.Assert(ready[source][db][tbls[1]], IsTrue) // TrySync for the first table. - vers[source][db][tbls[0]]++ - info = NewInfo(task, source, db, tbls[0], downSchema, downTable, DDLs4, nil, []*model.TableInfo{ti4_1, ti4}) - info.Version = vers[source][db][tbls[0]] + info = newInfoWithVersion(task, source, db, tbls[0], downSchema, downTable, DDLs4, nil, []*model.TableInfo{ti4_1, ti4}, vers) DDLs, err = l.TrySync(info, tts) c.Assert(err, IsNil) c.Assert(DDLs, DeepEquals, DDLs4) @@ -1053,9 +969,7 @@ func (t *testLock) TestLockTrySyncConflictIntrusive(c *C) { // CASE: conflict happen, revert all changes to resolve the conflict. // TrySync for the first table, construct the joined schema. - vers[source][db][tbls[0]]++ - info := NewInfo(task, source, db, tbls[0], downSchema, downTable, DDLs1, nil, []*model.TableInfo{ti1}) - info.Version = vers[source][db][tbls[0]] + info := newInfoWithVersion(task, source, db, tbls[0], downSchema, downTable, DDLs1, nil, []*model.TableInfo{ti1}, vers) DDLs, err := l.TrySync(info, tts) c.Assert(err, IsNil) c.Assert(DDLs, DeepEquals, DDLs1) @@ -1071,9 +985,7 @@ func (t *testLock) TestLockTrySyncConflictIntrusive(c *C) { c.Assert(cmp, Equals, -1) // TrySync for the second table with another schema (add two columns, one of them will cause conflict). - vers[source][db][tbls[1]]++ - info = NewInfo(task, source, db, tbls[1], downSchema, downTable, DDLs2, nil, []*model.TableInfo{ti3, ti2}) - info.Version = vers[source][db][tbls[1]] + info = newInfoWithVersion(task, source, db, tbls[1], downSchema, downTable, DDLs2, nil, []*model.TableInfo{ti3, ti2}, vers) DDLs, err = l.TrySync(info, tts) c.Assert(terror.ErrShardDDLOptimismTrySyncFail.Equal(err), IsTrue) c.Assert(DDLs, DeepEquals, []string{}) @@ -1086,9 +998,7 @@ func (t *testLock) TestLockTrySyncConflictIntrusive(c *C) { c.Assert(ready[source][db][tbls[1]], IsFalse) // TrySync again. - vers[source][db][tbls[1]]++ - info = NewInfo(task, source, db, tbls[1], downSchema, downTable, DDLs2, nil, []*model.TableInfo{ti3, ti2}) - info.Version = vers[source][db][tbls[1]] + info = newInfoWithVersion(task, source, db, tbls[1], downSchema, downTable, DDLs2, nil, []*model.TableInfo{ti3, ti2}, vers) DDLs, err = l.TrySync(info, tts) c.Assert(terror.ErrShardDDLOptimismTrySyncFail.Equal(err), IsTrue) c.Assert(DDLs, DeepEquals, []string{}) @@ -1098,9 +1008,7 @@ func (t *testLock) TestLockTrySyncConflictIntrusive(c *C) { c.Assert(cmp, Equals, -1) // TrySync for the second table to drop the non-conflict column, the conflict should still exist. - vers[source][db][tbls[1]]++ - info = NewInfo(task, source, db, tbls[1], downSchema, downTable, DDLs3, nil, []*model.TableInfo{ti3}) - info.Version = vers[source][db][tbls[1]] + info = newInfoWithVersion(task, source, db, tbls[1], downSchema, downTable, DDLs3, nil, []*model.TableInfo{ti3}, vers) DDLs, err = l.TrySync(info, tts) c.Assert(terror.ErrShardDDLOptimismTrySyncFail.Equal(err), IsTrue) c.Assert(DDLs, DeepEquals, []string{}) @@ -1112,9 +1020,7 @@ func (t *testLock) TestLockTrySyncConflictIntrusive(c *C) { c.Assert(ready[source][db][tbls[1]], IsFalse) // TrySync for the second table to drop the conflict column, the conflict should be resolved. - vers[source][db][tbls[1]]++ - info = NewInfo(task, source, db, tbls[1], downSchema, downTable, DDLs4, nil, []*model.TableInfo{ti4}) - info.Version = vers[source][db][tbls[1]] + info = newInfoWithVersion(task, source, db, tbls[1], downSchema, downTable, DDLs4, nil, []*model.TableInfo{ti4}, vers) DDLs, err = l.TrySync(info, tts) c.Assert(err, IsNil) c.Assert(DDLs, DeepEquals, []string{}) @@ -1126,9 +1032,7 @@ func (t *testLock) TestLockTrySyncConflictIntrusive(c *C) { c.Assert(ready[source][db][tbls[1]], IsFalse) // TrySync for the second table as we did for the first table, the lock should be synced. - vers[source][db][tbls[1]]++ - info = NewInfo(task, source, db, tbls[1], downSchema, downTable, DDLs1, nil, []*model.TableInfo{ti1}) - info.Version = vers[source][db][tbls[1]] + info = newInfoWithVersion(task, source, db, tbls[1], downSchema, downTable, DDLs1, nil, []*model.TableInfo{ti1}, vers) DDLs, err = l.TrySync(info, tts) c.Assert(err, IsNil) c.Assert(DDLs, DeepEquals, DDLs) @@ -1141,9 +1045,7 @@ func (t *testLock) TestLockTrySyncConflictIntrusive(c *C) { // CASE: conflict happen, revert part of changes to resolve the conflict. // TrySync for the first table, construct the joined schema. - vers[source][db][tbls[0]]++ - info = NewInfo(task, source, db, tbls[0], downSchema, downTable, DDLs5, nil, []*model.TableInfo{ti5}) - info.Version = vers[source][db][tbls[0]] + info = newInfoWithVersion(task, source, db, tbls[0], downSchema, downTable, DDLs5, nil, []*model.TableInfo{ti5}, vers) DDLs, err = l.TrySync(info, tts) c.Assert(err, IsNil) c.Assert(DDLs, DeepEquals, DDLs5) @@ -1159,9 +1061,7 @@ func (t *testLock) TestLockTrySyncConflictIntrusive(c *C) { c.Assert(cmp, Equals, -1) // TrySync for the second table with another schema (add two columns, one of them will cause conflict). - vers[source][db][tbls[1]]++ - info = NewInfo(task, source, db, tbls[1], downSchema, downTable, DDLs6, nil, []*model.TableInfo{ti6_1, ti6}) - info.Version = vers[source][db][tbls[1]] + info = newInfoWithVersion(task, source, db, tbls[1], downSchema, downTable, DDLs6, nil, []*model.TableInfo{ti6_1, ti6}, vers) DDLs, err = l.TrySync(info, tts) c.Assert(terror.ErrShardDDLOptimismTrySyncFail.Equal(err), IsTrue) c.Assert(DDLs, DeepEquals, []string{}) @@ -1174,9 +1074,7 @@ func (t *testLock) TestLockTrySyncConflictIntrusive(c *C) { // TrySync for the second table to drop the conflict column, the conflict should be resolved. // but both of tables are not synced now. - vers[source][db][tbls[1]]++ - info = NewInfo(task, source, db, tbls[1], downSchema, downTable, DDLs7, nil, []*model.TableInfo{ti7}) - info.Version = vers[source][db][tbls[1]] + info = newInfoWithVersion(task, source, db, tbls[1], downSchema, downTable, DDLs7, nil, []*model.TableInfo{ti7}, vers) DDLs, err = l.TrySync(info, tts) c.Assert(err, IsNil) c.Assert(DDLs, DeepEquals, DDLs7) // special case: these DDLs should not be replicated to the downstream. @@ -1192,9 +1090,7 @@ func (t *testLock) TestLockTrySyncConflictIntrusive(c *C) { c.Assert(cmp, Equals, -1) // TrySync for the first table to become synced. - vers[source][db][tbls[0]]++ - info = NewInfo(task, source, db, tbls[0], downSchema, downTable, DDLs8_1, nil, []*model.TableInfo{ti8}) - info.Version = vers[source][db][tbls[0]] + info = newInfoWithVersion(task, source, db, tbls[0], downSchema, downTable, DDLs8_1, nil, []*model.TableInfo{ti8}, vers) DDLs, err = l.TrySync(info, tts) c.Assert(err, IsNil) c.Assert(DDLs, DeepEquals, DDLs8_1) @@ -1202,9 +1098,7 @@ func (t *testLock) TestLockTrySyncConflictIntrusive(c *C) { c.Assert(ready[source][db][tbls[0]], IsTrue) // TrySync for the second table to become synced. - vers[source][db][tbls[1]]++ - info = NewInfo(task, source, db, tbls[1], downSchema, downTable, DDLs8_2, nil, []*model.TableInfo{ti8}) - info.Version = vers[source][db][tbls[1]] + info = newInfoWithVersion(task, source, db, tbls[1], downSchema, downTable, DDLs8_2, nil, []*model.TableInfo{ti8}, vers) DDLs, err = l.TrySync(info, tts) c.Assert(err, IsNil) c.Assert(DDLs, DeepEquals, DDLs8_2) @@ -1269,16 +1163,12 @@ func (t *testLock) TestLockTrySyncMultipleChangeDDL(c *C) { t.checkLockNoDone(c, l) // inconsistent ddls and table infos - vers[sources[0]][dbs[0]][tbls[0]]++ - info := NewInfo(task, sources[0], dbs[0], tbls[0], downSchema, downTable, DDLs1[:1], nil, []*model.TableInfo{ti1_1, ti1}) - info.Version = vers[sources[0]][dbs[0]][tbls[0]] + info := newInfoWithVersion(task, sources[0], dbs[0], tbls[0], downSchema, downTable, DDLs1[:1], nil, []*model.TableInfo{ti1_1, ti1}, vers) DDLs, err := l.TrySync(info, tts) c.Assert(DDLs, DeepEquals, DDLs1[:1]) c.Assert(terror.ErrMasterInconsistentOptimisticDDLsAndInfo.Equal(err), IsTrue) - vers[sources[0]][dbs[0]][tbls[0]]++ - info = NewInfo(task, sources[0], dbs[0], tbls[0], downSchema, downTable, DDLs1, nil, []*model.TableInfo{ti1}) - info.Version = vers[sources[0]][dbs[0]][tbls[0]] + info = newInfoWithVersion(task, sources[0], dbs[0], tbls[0], downSchema, downTable, DDLs1, nil, []*model.TableInfo{ti1}, vers) DDLs, err = l.TrySync(info, tts) c.Assert(DDLs, DeepEquals, DDLs1) c.Assert(terror.ErrMasterInconsistentOptimisticDDLsAndInfo.Equal(err), IsTrue) @@ -1301,9 +1191,7 @@ func (t *testLock) TestLockTrySyncMultipleChangeDDL(c *C) { for _, source := range sources { for _, db := range dbs { for _, tbl := range tbls { - vers[source][db][tbl]++ - info = NewInfo(task, source, db, tbl, downSchema, downTable, DDLs1, nil, []*model.TableInfo{ti1_1, ti1}) - info.Version = vers[source][db][tbl] + info = newInfoWithVersion(task, source, db, tbl, downSchema, downTable, DDLs1, nil, []*model.TableInfo{ti1_1, ti1}, vers) DDLs, err = l.TrySync(info, tts) c.Assert(err, IsNil) c.Assert(DDLs, DeepEquals, resultDDLs1[source][db][tbl]) @@ -1322,9 +1210,7 @@ func (t *testLock) TestLockTrySyncMultipleChangeDDL(c *C) { // CASE: TrySync again after synced is idempotent. // both ddl will sync again - vers[sources[0]][dbs[0]][tbls[0]]++ - info = NewInfo(task, sources[0], dbs[0], tbls[0], downSchema, downTable, DDLs1, nil, []*model.TableInfo{ti1_1, ti1}) - info.Version = vers[sources[0]][dbs[0]][tbls[0]] + info = newInfoWithVersion(task, sources[0], dbs[0], tbls[0], downSchema, downTable, DDLs1, nil, []*model.TableInfo{ti1_1, ti1}, vers) DDLs, err = l.TrySync(info, tts) c.Assert(err, IsNil) c.Assert(DDLs, DeepEquals, DDLs1) @@ -1347,9 +1233,7 @@ func (t *testLock) TestLockTrySyncMultipleChangeDDL(c *C) { for _, source := range sources { for _, db := range dbs { for _, tbl := range tbls { - vers[source][db][tbl]++ - info = NewInfo(task, source, db, tbl, downSchema, downTable, DDLs2, nil, []*model.TableInfo{ti2_1, ti2}) - info.Version = vers[source][db][tbl] + info = newInfoWithVersion(task, source, db, tbl, downSchema, downTable, DDLs2, nil, []*model.TableInfo{ti2_1, ti2}, vers) DDLs, err = l.TrySync(info, tts) c.Assert(err, IsNil) c.Assert(DDLs, DeepEquals, resultDDLs2[source][db][tbl]) @@ -1368,9 +1252,7 @@ func (t *testLock) TestLockTrySyncMultipleChangeDDL(c *C) { // CASE: TrySync again after synced is idempotent. // only the second ddl(ADD COLUMN) will sync, the first one(DROP COLUMN) will not sync since oldJoined==newJoined - vers[sources[0]][dbs[0]][tbls[0]]++ - info = NewInfo(task, sources[0], dbs[0], tbls[0], downSchema, downTable, DDLs2, nil, []*model.TableInfo{ti2_1, ti2}) - info.Version = vers[sources[0]][dbs[0]][tbls[0]] + info = newInfoWithVersion(task, sources[0], dbs[0], tbls[0], downSchema, downTable, DDLs2, nil, []*model.TableInfo{ti2_1, ti2}, vers) DDLs, err = l.TrySync(info, tts) c.Assert(err, IsNil) c.Assert(DDLs, DeepEquals, DDLs2[1:]) @@ -1415,9 +1297,7 @@ func (t *testLock) TestTryRemoveTable(c *C) { // CASE: remove a table as normal. // TrySync for the first table. - vers[source][db][tbl1]++ - info := NewInfo(task, source, db, tbl1, downSchema, downTable, DDLs1, nil, []*model.TableInfo{ti1}) - info.Version = vers[source][db][tbl1] + info := newInfoWithVersion(task, source, db, tbl1, downSchema, downTable, DDLs1, nil, []*model.TableInfo{ti1}, vers) DDLs, err := l.TrySync(info, tts) c.Assert(err, IsNil) c.Assert(DDLs, DeepEquals, DDLs1) @@ -1441,9 +1321,8 @@ func (t *testLock) TestTryRemoveTable(c *C) { // CASE: remove a table will not rebuild joined schema now. // TrySync to add the second back. - vers[source][db][tbl2] = 1 - info = NewInfo(task, source, db, tbl2, downSchema, downTable, DDLs2, nil, []*model.TableInfo{ti2}) - info.Version = vers[source][db][tbl2] + vers[source][db][tbl2] = 0 + info = newInfoWithVersion(task, source, db, tbl2, downSchema, downTable, DDLs2, nil, []*model.TableInfo{ti2}, vers) DDLs, err = l.TrySync(info, tts) c.Assert(err, IsNil) c.Assert(DDLs, DeepEquals, DDLs2) @@ -1508,9 +1387,7 @@ func (t *testLock) TestLockTryMarkDone(c *C) { c.Assert(l.IsResolved(), IsFalse) // TrySync for the first table, no table has done the DDLs operation. - vers[source][db][tbls[0]]++ - info := NewInfo(task, source, db, tbls[0], downSchema, downTable, DDLs1, nil, []*model.TableInfo{ti1}) - info.Version = vers[source][db][tbls[0]] + info := newInfoWithVersion(task, source, db, tbls[0], downSchema, downTable, DDLs1, nil, []*model.TableInfo{ti1}, vers) DDLs, err := l.TrySync(info, tts) c.Assert(err, IsNil) c.Assert(DDLs, DeepEquals, DDLs1) @@ -1525,9 +1402,7 @@ func (t *testLock) TestLockTryMarkDone(c *C) { c.Assert(l.IsResolved(), IsFalse) // TrySync for the second table, the joined schema become larger. - vers[source][db][tbls[1]]++ - info = NewInfo(task, source, db, tbls[1], downSchema, downTable, DDLs2, nil, []*model.TableInfo{ti1, ti2}) - info.Version = vers[source][db][tbls[1]] + info = newInfoWithVersion(task, source, db, tbls[1], downSchema, downTable, DDLs2, nil, []*model.TableInfo{ti1, ti2}, vers) DDLs, err = l.TrySync(info, tts) c.Assert(err, IsNil) c.Assert(DDLs, DeepEquals, DDLs2) @@ -1546,9 +1421,7 @@ func (t *testLock) TestLockTryMarkDone(c *C) { c.Assert(l.IsResolved(), IsFalse) // TrySync for the first table, all tables become synced. - vers[source][db][tbls[0]]++ - info = NewInfo(task, source, db, tbls[0], downSchema, downTable, DDLs3, nil, []*model.TableInfo{ti3}) - info.Version = vers[source][db][tbls[0]] + info = newInfoWithVersion(task, source, db, tbls[0], downSchema, downTable, DDLs3, nil, []*model.TableInfo{ti3}, vers) DDLs, err = l.TrySync(info, tts) c.Assert(err, IsNil) c.Assert(DDLs, DeepEquals, DDLs3) @@ -1622,9 +1495,7 @@ func (t *testLock) TestAddDifferentFieldLenColumns(c *C) { c.Assert(l.IsResolved(), IsFalse) // TrySync for the first table, no table has done the DDLs operation. - vers[source][db][tbls[0]]++ - info := NewInfo(task, source, db, tbls[0], downSchema, downTable, DDLs1, nil, []*model.TableInfo{ti1}) - info.Version = vers[source][db][tbls[0]] + info := newInfoWithVersion(task, source, db, tbls[0], downSchema, downTable, DDLs1, nil, []*model.TableInfo{ti1}, vers) DDLs, err := l.TrySync(info, tts) c.Assert(err, IsNil) c.Assert(DDLs, DeepEquals, DDLs1) @@ -1633,9 +1504,7 @@ func (t *testLock) TestAddDifferentFieldLenColumns(c *C) { c.Assert(l.IsResolved(), IsFalse) // TrySync for the second table, add a table with a larger field length - vers[source][db][tbls[1]]++ - info = NewInfo(task, source, db, tbls[1], downSchema, downTable, DDLs2, nil, []*model.TableInfo{ti2}) - info.Version = vers[source][db][tbls[0]] + info = newInfoWithVersion(task, source, db, tbls[1], downSchema, downTable, DDLs2, nil, []*model.TableInfo{ti2}, vers) DDLs, err = l.TrySync(info, tts) c.Assert(err, ErrorMatches, ".*add columns with different field lengths.*") c.Assert(DDLs, DeepEquals, DDLs2) @@ -1656,9 +1525,7 @@ func (t *testLock) TestAddDifferentFieldLenColumns(c *C) { c.Assert(l.IsResolved(), IsFalse) // TrySync for the second table, add a table with a smaller field length - vers[source][db][tbls[0]]++ - info = NewInfo(task, source, db, tbls[0], downSchema, downTable, DDLs1, nil, []*model.TableInfo{ti1}) - info.Version = vers[source][db][tbls[0]] + info = newInfoWithVersion(task, source, db, tbls[0], downSchema, downTable, DDLs1, nil, []*model.TableInfo{ti1}, vers) DDLs, err = l.TrySync(info, tts) c.Assert(err, ErrorMatches, ".*add columns with different field lengths.*") c.Assert(DDLs, DeepEquals, DDLs1) @@ -1670,8 +1537,7 @@ func (t *testLock) trySyncForAllTablesLarger(c *C, l *Lock, for source, schemaTables := range l.Ready() { for schema, tables := range schemaTables { for table := range tables { - info := NewInfo(l.Task, source, schema, table, l.DownSchema, l.DownTable, DDLs, nil, tis) - info.Version = vers[source][schema][table] + info := newInfoWithVersion(l.Task, source, schema, table, l.DownSchema, l.DownTable, DDLs, nil, tis, vers) DDLs2, err := l.TrySync(info, tts) c.Assert(err, IsNil) c.Assert(DDLs2, DeepEquals, resultDDLs[source][schema][table]) @@ -1706,3 +1572,11 @@ func (t *testLock) checkLockNoDone(c *C, l *Lock) { } } } + +func newInfoWithVersion(task, source, upSchema, upTable, downSchema, downTable string, DDLs []string, tableInfoBefore *model.TableInfo, + tableInfosAfter []*model.TableInfo, vers map[string]map[string]map[string]int64) Info { + info := NewInfo(task, source, upSchema, upTable, downSchema, downTable, DDLs, tableInfoBefore, tableInfosAfter) + vers[source][upSchema][upTable]++ + info.Version = vers[source][upSchema][upTable] + return info +} From 412f335406c1c75dd948a3e5b34192f537f7911e Mon Sep 17 00:00:00 2001 From: GMHDBJD <35025882+GMHDBJD@users.noreply.github.com> Date: Wed, 17 Mar 2021 19:20:05 +0800 Subject: [PATCH 16/17] Update dm/master/shardddl/optimist.go Co-authored-by: Chunzhu Li --- dm/master/shardddl/optimist.go | 5 +---- 1 file changed, 1 insertion(+), 4 deletions(-) diff --git a/dm/master/shardddl/optimist.go b/dm/master/shardddl/optimist.go index 6e32d0a3b6..ca79619f25 100644 --- a/dm/master/shardddl/optimist.go +++ b/dm/master/shardddl/optimist.go @@ -500,11 +500,8 @@ func (o *Optimist) handleLock(info optimism.Info, tts []optimism.TargetTable, sk lockID, newDDLs, err := o.lk.TrySync(info, tts) var cfStage = optimism.ConflictNone if info.IgnoreConflict { - if err != nil { - o.logger.Warn("error occur when trying to sync for shard DDL info, this often means shard DDL conflict detected", + o.logger.Warn("error occur when trying to sync for shard DDL info, this often means shard DDL conflict detected", zap.String("lock", lockID), zap.Stringer("info", info), zap.Bool("is deleted", info.IsDeleted), log.ShortError(err)) - } - o.logger.Warn("handle lock in ignore conflict mode", zap.String("lock", lockID), zap.Stringer("info", info)) } else if err != nil { cfStage = optimism.ConflictDetected // we treat any errors returned from `TrySync` as conflict detected now. o.logger.Warn("error occur when trying to sync for shard DDL info, this often means shard DDL conflict detected", From c5a2347d37ceee381fedcfc32f6a2dc73308327c Mon Sep 17 00:00:00 2001 From: gmhdbjd Date: Wed, 17 Mar 2021 19:28:52 +0800 Subject: [PATCH 17/17] fix fmt --- dm/master/shardddl/optimist.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dm/master/shardddl/optimist.go b/dm/master/shardddl/optimist.go index ca79619f25..70067ed91e 100644 --- a/dm/master/shardddl/optimist.go +++ b/dm/master/shardddl/optimist.go @@ -501,7 +501,7 @@ func (o *Optimist) handleLock(info optimism.Info, tts []optimism.TargetTable, sk var cfStage = optimism.ConflictNone if info.IgnoreConflict { o.logger.Warn("error occur when trying to sync for shard DDL info, this often means shard DDL conflict detected", - zap.String("lock", lockID), zap.Stringer("info", info), zap.Bool("is deleted", info.IsDeleted), log.ShortError(err)) + zap.String("lock", lockID), zap.Stringer("info", info), zap.Bool("is deleted", info.IsDeleted), log.ShortError(err)) } else if err != nil { cfStage = optimism.ConflictDetected // we treat any errors returned from `TrySync` as conflict detected now. o.logger.Warn("error occur when trying to sync for shard DDL info, this often means shard DDL conflict detected",