From 05c18b1b4761f02b7401896920b2d12249fc7403 Mon Sep 17 00:00:00 2001 From: Chunzhu Li Date: Thu, 11 Mar 2021 20:23:45 +0800 Subject: [PATCH 01/23] add unit tests and partially dropped columns without etcd --- dm/master/shardddl/optimist.go | 4 + pkg/shardddl/optimism/lock.go | 93 +++++++++++++++++----- pkg/shardddl/optimism/lock_test.go | 123 +++++++++++++++++++++++++++-- 3 files changed, 195 insertions(+), 25 deletions(-) diff --git a/dm/master/shardddl/optimist.go b/dm/master/shardddl/optimist.go index b63da694ea..ea67dbd2e7 100644 --- a/dm/master/shardddl/optimist.go +++ b/dm/master/shardddl/optimist.go @@ -469,6 +469,10 @@ func (o *Optimist) handleOperationPut(ctx context.Context, opCh <-chan optimism. continue } + err := lock.UpdateColumns(op.DDLs) + if err != nil { + o.logger.Error("fail to update lock columns", zap.Error(err)) + } // in optimistic mode, we always try to mark a table as done after received the `done` status of the DDLs operation. // NOTE: even all tables have done their previous DDLs operations, the lock may still not resolved, // because these tables may have different schemas. diff --git a/pkg/shardddl/optimism/lock.go b/pkg/shardddl/optimism/lock.go index e537591f04..66c42681f4 100644 --- a/pkg/shardddl/optimism/lock.go +++ b/pkg/shardddl/optimism/lock.go @@ -56,6 +56,10 @@ type Lock struct { // upstream source ID -> upstream schema name -> upstream table name -> info version. versions map[string]map[string]map[string]int64 + + // record the partially dropped columns + // column name -> interface{} + columns map[string]interface{} } // NewLock creates a new Lock instance. @@ -71,6 +75,7 @@ func NewLock(ID, task, downSchema, downTable string, ti *model.TableInfo, tts [] done: make(map[string]map[string]map[string]bool), synced: true, versions: make(map[string]map[string]map[string]int64), + columns: make(map[string]interface{}), } l.addTables(tts) metrics.ReportDDLPending(task, metrics.DDLPendingNone, metrics.DDLPendingSynced) @@ -194,6 +199,12 @@ func (l *Lock) TrySync(callerSource, callerSchema, callerTable string, // if any real conflicts after joined exist, they will be detected by the following steps. var cmp int if cmp, err = nextTable.Compare(oldJoined); err == nil && cmp == 0 { + if col, err := GetColumnName(l.ID, ddls[idx], ast.AlterTableAddColumns); err != nil { + return newDDLs, err + } else if _, ok := l.columns[col]; len(col) > 0 && ok { + return newDDLs, terror.ErrShardDDLOptimismTrySyncFail.Generate( + l.ID, fmt.Sprintf("add column %s that wasn't fully dropped in downstream. ddl: %s", col, ddls[idx])) + } newDDLs = append(newDDLs, ddls[idx]) continue } @@ -221,9 +232,9 @@ func (l *Lock) TrySync(callerSource, callerSchema, callerTable string, // for these two cases, we should execute the DDLs to the downstream to update the schema. log.L().Info("joined table info changed", zap.String("lock", l.ID), zap.Int("cmp", cmp), zap.Stringer("from", oldJoined), zap.Stringer("to", newJoined), zap.String("source", callerSource), zap.String("schema", callerSchema), zap.String("table", callerTable), zap.Strings("ddls", ddls)) - // check for add column with different field lengths + // check for add column with a larger field len if cmp < 0 { - err = AddDifferentFieldLenColumns(l.ID, ddls[idx], oldJoined, newJoined) + _, err = AddDifferentFieldLenColumns(l.ID, ddls[idx], oldJoined, newJoined) if err != nil { return ddls, err } @@ -251,15 +262,22 @@ func (l *Lock) TrySync(callerSource, callerSchema, callerTable string, cmp, _ = prevTable.Compare(nextTable) // we have checked `err` returned above. if cmp < 0 { - // check for add column with different field lengths - err = AddDifferentFieldLenColumns(l.ID, ddls[idx], nextTable, newJoined) - if err != nil { + // check for add column with a smaller field len + if col, err := AddDifferentFieldLenColumns(l.ID, ddls[idx], nextTable, newJoined); err != nil { return ddls, err + } else if _, ok := l.columns[col]; len(col) > 0 && ok { + return ddls, terror.ErrShardDDLOptimismTrySyncFail.Generate( + l.ID, fmt.Sprintf("add column %s that wasn't fully dropped in downstream. ddl: %s", col, ddls[idx])) } // let every table to replicate the DDL. newDDLs = append(newDDLs, ddls[idx]) continue } else if cmp > 0 { + if col, err := GetColumnName(l.ID, ddls[idx], ast.AlterTableDropColumn); err != nil { + return ddls, err + } else if len(col) > 0 { + l.columns[col] = struct{}{} + } // last shard table won't go here continue } @@ -488,27 +506,64 @@ func (l *Lock) GetVersion(source string, schema string, table string) int64 { return l.versions[source][schema][table] } +// GetVersion return version of info in lock. +func (l *Lock) UpdateColumns(ddls []string) error { + l.mu.Lock() + defer l.mu.Unlock() + + for _, ddl := range ddls { + col, err := GetColumnName(l.ID, ddl, ast.AlterTableDropColumn) + if err != nil { + return err + } + if len(col) > 0 { + delete(l.columns, col) + } + } + return nil +} + // AddDifferentFieldLenColumns checks whether dm adds columns with different field lengths -func AddDifferentFieldLenColumns(lockID, ddl string, oldJoined, newJoined schemacmp.Table) error { +func AddDifferentFieldLenColumns(lockID, ddl string, oldJoined, newJoined schemacmp.Table) (string, error) { + col, err := GetColumnName(lockID, ddl, ast.AlterTableAddColumns) + if err != nil { + return col, err + } + if len(col) > 0 { + oldJoinedCols := schemacmp.DecodeColumnFieldTypes(oldJoined) + newJoinedCols := schemacmp.DecodeColumnFieldTypes(newJoined) + oldCol, ok1 := oldJoinedCols[col] + newCol, ok2 := newJoinedCols[col] + if ok1 && ok2 { + if newCol.Flen != oldCol.Flen { + return col, terror.ErrShardDDLOptimismTrySyncFail.Generate( + lockID, fmt.Sprintf("add columns with different field lengths."+ + "ddl: %s, origLen: %d, newLen: %d", ddl, oldCol.Flen, newCol.Flen)) + } + } + } + return col, nil +} + +// GetColumnName checks whether dm adds/drops a column, and return this column's name +func GetColumnName(lockID, ddl string, tp ast.AlterTableType) (string, error) { if stmt, err := parser.New().ParseOneStmt(ddl, "", ""); err != nil { - return terror.ErrShardDDLOptimismTrySyncFail.Delegate( + return "", terror.ErrShardDDLOptimismTrySyncFail.Delegate( err, lockID, fmt.Sprintf("fail to parse ddl %s", ddl)) } else if v, ok := stmt.(*ast.AlterTableStmt); ok && len(v.Specs) > 0 { spec := v.Specs[0] - if spec.Tp == ast.AlterTableAddColumns && len(spec.NewColumns) > 0 { - col := spec.NewColumns[0].Name.Name.O - oldJoinedCols := schemacmp.DecodeColumnFieldTypes(oldJoined) - newJoinedCols := schemacmp.DecodeColumnFieldTypes(newJoined) - oldCol, ok1 := oldJoinedCols[col] - newCol, ok2 := newJoinedCols[col] - if ok1 && ok2 { - if newCol.Flen != oldCol.Flen { - return terror.ErrShardDDLOptimismTrySyncFail.Generate( - lockID, fmt.Sprintf("add columns with different field lengths."+ - "ddl: %s, origLen: %d, newLen: %d", ddl, oldCol.Flen, newCol.Flen)) + if spec.Tp == tp { + switch spec.Tp { + case ast.AlterTableAddColumns: + if len(spec.NewColumns) > 0 { + return spec.NewColumns[0].Name.Name.O, nil + } + case ast.AlterTableDropColumn: + if spec.OldColumnName != nil { + return spec.OldColumnName.Name.O, nil } } } } - return nil + return "", nil } diff --git a/pkg/shardddl/optimism/lock_test.go b/pkg/shardddl/optimism/lock_test.go index 624c9cc9ad..0a9d1a696f 100644 --- a/pkg/shardddl/optimism/lock_test.go +++ b/pkg/shardddl/optimism/lock_test.go @@ -16,6 +16,7 @@ package optimism import ( . "github.com/pingcap/check" "github.com/pingcap/parser" + "github.com/pingcap/parser/ast" "github.com/pingcap/parser/model" "github.com/pingcap/tidb-tools/pkg/schemacmp" "github.com/pingcap/tidb/util/mock" @@ -1122,7 +1123,7 @@ func (t *testLock) TestLockTrySyncMultipleChangeDDL(c *C) { p = parser.New() se = mock.NewContext() tblID int64 = 111 - DDLs1 = []string{"ALTER TABLE bar ADD COLUMN c2 INT", "ALTER TABLE DROP COLUMN c1"} + DDLs1 = []string{"ALTER TABLE bar ADD COLUMN c2 INT", "ALTER TABLE bar DROP COLUMN c1"} DDLs2 = []string{"ALTER TABLE bar DROP COLUMN c2", "ALTER TABLE bar ADD COLUMN c3 TEXT"} // DDLs3 = []string{"ALTER TABLE bar DROP COLUMN c3"} // DDLs4 = []string{"ALTER TABLE bar DROP COLUMN c2", "ALTER TABLE bar DROP COLUMN c1"} @@ -1451,8 +1452,8 @@ func (t *testLock) TestLockTryMarkDone(c *C) { func (t *testLock) TestAddDifferentFieldLenColumns(c *C) { var ( - ID = "test_lock_try_mark_done-`foo`.`bar`" - task = "test_lock_try_mark_done" + ID = "test_lock_add_diff_flen_cols-`foo`.`bar`" + task = "test_lock_add_diff_flen_cols" source = "mysql-replica-1" downSchema = "foo" downTable = "bar" @@ -1483,9 +1484,15 @@ func (t *testLock) TestAddDifferentFieldLenColumns(c *C) { }, } ) - c.Assert(AddDifferentFieldLenColumns(ID, DDLs1[0], table1, table2), IsNil) - c.Assert(AddDifferentFieldLenColumns(ID, DDLs2[0], table2, table3), ErrorMatches, ".*add columns with different field lengths.*") - c.Assert(AddDifferentFieldLenColumns(ID, DDLs1[0], table3, table2), ErrorMatches, ".*add columns with different field lengths.*") + col, err := AddDifferentFieldLenColumns(ID, DDLs1[0], table1, table2) + c.Assert(col, Equals, "c1") + c.Assert(err, IsNil) + col, err = AddDifferentFieldLenColumns(ID, DDLs2[0], table2, table3) + c.Assert(col, Equals, "c1") + c.Assert(err, ErrorMatches, ".*add columns with different field lengths.*") + col, err = AddDifferentFieldLenColumns(ID, DDLs1[0], table3, table2) + c.Assert(col, Equals, "c1") + c.Assert(err, ErrorMatches, ".*add columns with different field lengths.*") // the initial status is synced but not resolved. t.checkLockSynced(c, l) @@ -1528,6 +1535,110 @@ func (t *testLock) TestAddDifferentFieldLenColumns(c *C) { c.Assert(l.versions, DeepEquals, vers) } +func (t *testLock) TestAddNotFullyDroppedColumns(c *C) { + var ( + ID = "test_lock_add_not_fully_dropped_cols-`foo`.`bar`" + task = "test_lock_add_not_fully_dropped_cols" + source = "mysql-replica-1" + downSchema = "foo" + downTable = "bar" + db = "foo" + tbls = []string{"bar1", "bar2"} + p = parser.New() + se = mock.NewContext() + + tblID int64 = 111 + ti0 = createTableInfo(c, p, se, tblID, `CREATE TABLE bar (id INT PRIMARY KEY, b int, c int)`) + ti1 = createTableInfo(c, p, se, tblID, `CREATE TABLE bar (id INT PRIMARY KEY, b int)`) + ti2 = createTableInfo(c, p, se, tblID, `CREATE TABLE bar (id INT PRIMARY KEY)`) + ti3 = createTableInfo(c, p, se, tblID, `CREATE TABLE bar (id INT PRIMARY KEY, c int)`) + + DDLs1 = []string{"ALTER TABLE bar DROP COLUMN c"} + DDLs2 = []string{"ALTER TABLE bar DROP COLUMN b"} + DDLs3 = []string{"ALTER TABLE bar ADD COLUMN b INT"} + DDLs4 = []string{"ALTER TABLE bar ADD COLUMN c INT"} + + tables = map[string]map[string]struct{}{db: {tbls[0]: struct{}{}, tbls[1]: struct{}{}}} + tts = []TargetTable{newTargetTable(task, source, downSchema, downTable, tables)} + l = NewLock(ID, task, downSchema, downTable, ti0, tts) + + vers = map[string]map[string]map[string]int64{ + source: { + db: {tbls[0]: 0, tbls[1]: 0}, + }, + } + ) + col, err := GetColumnName(ID, DDLs1[0], ast.AlterTableDropColumn) + c.Assert(col, Equals, "c") + c.Assert(err, IsNil) + col, err = GetColumnName(ID, DDLs2[0], ast.AlterTableDropColumn) + c.Assert(col, Equals, "b") + c.Assert(err, IsNil) + + // the initial status is synced but not resolved. + t.checkLockSynced(c, l) + t.checkLockNoDone(c, l) + c.Assert(l.IsResolved(), IsFalse) + + // TrySync for the first table, drop the first column + vers[source][db][tbls[0]]++ + DDLs, err := l.TrySync(source, db, tbls[0], DDLs1, []*model.TableInfo{ti1}, tts, vers[source][db][tbls[0]]) + c.Assert(err, IsNil) + c.Assert(DDLs, DeepEquals, []string{}) + c.Assert(l.versions, DeepEquals, vers) + c.Assert(l.IsResolved(), IsFalse) + + // TrySync for the first table, drop column b + vers[source][db][tbls[0]]++ + DDLs, err = l.TrySync(source, db, tbls[0], DDLs2, []*model.TableInfo{ti2}, tts, vers[source][db][tbls[0]]) + c.Assert(err, IsNil) + c.Assert(DDLs, DeepEquals, []string{}) + c.Assert(l.versions, DeepEquals, vers) + c.Assert(l.IsResolved(), IsFalse) + + // TrySync for the second table, drop column b, this column should be dropped + vers[source][db][tbls[1]]++ + DDLs, err = l.TrySync(source, db, tbls[1], DDLs2, []*model.TableInfo{ti3}, tts, vers[source][db][tbls[1]]) + c.Assert(err, IsNil) + c.Assert(DDLs, DeepEquals, DDLs2) + c.Assert(l.versions, DeepEquals, vers) + c.Assert(l.IsResolved(), IsFalse) + // Simulate watch done operation from dm-worker + c.Assert(l.UpdateColumns(DDLs), IsNil) + + // TrySync for the first table, add column b, should succeed, because this column is fully dropped in the downstream + vers[source][db][tbls[0]]++ + DDLs, err = l.TrySync(source, db, tbls[0], DDLs3, []*model.TableInfo{ti1}, tts, vers[source][db][tbls[0]]) + c.Assert(err, IsNil) + c.Assert(DDLs, DeepEquals, DDLs3) + c.Assert(l.versions, DeepEquals, vers) + c.Assert(l.IsResolved(), IsFalse) + + // TrySync for the first table, add column b, should fail, because this column isn't fully dropped in the downstream + vers[source][db][tbls[0]]++ + DDLs, err = l.TrySync(source, db, tbls[0], DDLs4, []*model.TableInfo{ti0}, tts, vers[source][db][tbls[0]]) + c.Assert(err, ErrorMatches, ".*add column c that wasn't fully dropped in downstream.*") + c.Assert(l.IsResolved(), IsFalse) + + // TrySync for the second table, drop column b, this column should be dropped + vers[source][db][tbls[1]]++ + DDLs, err = l.TrySync(source, db, tbls[1], DDLs1, []*model.TableInfo{ti2}, tts, vers[source][db][tbls[1]]) + c.Assert(err, IsNil) + c.Assert(DDLs, DeepEquals, DDLs1) + c.Assert(l.versions, DeepEquals, vers) + c.Assert(l.IsResolved(), IsFalse) + // Simulate watch done operation from dm-worker + c.Assert(l.UpdateColumns(DDLs), IsNil) + + // TrySync for the first table, add column b, should fail, because this column isn't fully dropped in the downstream + vers[source][db][tbls[0]]++ + DDLs, err = l.TrySync(source, db, tbls[0], DDLs4, []*model.TableInfo{ti0}, tts, vers[source][db][tbls[0]]) + c.Assert(err, IsNil) + c.Assert(DDLs, DeepEquals, DDLs4) + c.Assert(l.versions, DeepEquals, vers) + c.Assert(l.IsResolved(), IsFalse) +} + func (t *testLock) trySyncForAllTablesLarger(c *C, l *Lock, DDLs []string, tis []*model.TableInfo, tts []TargetTable, vers map[string]map[string]map[string]int64, resultDDLs map[string]map[string]map[string][]string) { for source, schemaTables := range l.Ready() { From d774402c94bbd539c3b88cfa568da83ca31f79da Mon Sep 17 00:00:00 2001 From: Chunzhu Li Date: Mon, 15 Mar 2021 15:48:56 +0800 Subject: [PATCH 02/23] add etcd info and integration test --- dm/common/common.go | 5 +- dm/master/shardddl/optimist.go | 27 ++++++-- pkg/etcdutil/etcdutil.go | 2 +- pkg/shardddl/optimism/column.go | 79 +++++++++++++++++++++ pkg/shardddl/optimism/column_test.go | 54 +++++++++++++++ pkg/shardddl/optimism/info.go | 3 +- pkg/shardddl/optimism/keeper.go | 21 +++++- pkg/shardddl/optimism/keeper_test.go | 14 ++-- pkg/shardddl/optimism/lock.go | 39 +++++++++-- pkg/shardddl/optimism/lock_test.go | 55 ++++++++++----- tests/_utils/test_prepare | 7 +- tests/shardddl3/run.sh | 100 +++++++++++++++++++++++++-- 12 files changed, 360 insertions(+), 46 deletions(-) create mode 100644 pkg/shardddl/optimism/column.go create mode 100644 pkg/shardddl/optimism/column_test.go diff --git a/dm/common/common.go b/dm/common/common.go index 53f47d8040..2070a39047 100644 --- a/dm/common/common.go +++ b/dm/common/common.go @@ -76,6 +76,9 @@ var ( // ShardDDLOptimismInitSchemaKeyAdapter used to store the initial schema (before constructed the lock) of merged tables. // k/v: Encode(task-name, downstream-schema-name, downstream-table-name) -> table schema. ShardDDLOptimismInitSchemaKeyAdapter KeyAdapter = keyHexEncoderDecoder("/dm-master/shardddl-optimism/init-schema/") + // ShardDDLOptimismDroppedColumnsKeyAdapter is used to store the columns that are not fully dropped + // k/v: Encode(lock-id, column-name) -> empty + ShardDDLOptimismDroppedColumnsKeyAdapter KeyAdapter = keyHexEncoderDecoder("/dm-master/shardddl-optimism/undropped-columns/") ) func keyAdapterKeysLen(s KeyAdapter) int { @@ -86,7 +89,7 @@ func keyAdapterKeysLen(s KeyAdapter) int { return 1 case UpstreamSubTaskKeyAdapter, StageSubTaskKeyAdapter, ShardDDLPessimismInfoKeyAdapter, ShardDDLPessimismOperationKeyAdapter, - ShardDDLOptimismSourceTablesKeyAdapter: + ShardDDLOptimismSourceTablesKeyAdapter, ShardDDLOptimismDroppedColumnsKeyAdapter: return 2 case ShardDDLOptimismInitSchemaKeyAdapter: return 3 diff --git a/dm/master/shardddl/optimist.go b/dm/master/shardddl/optimist.go index ea67dbd2e7..9efdc8ad73 100644 --- a/dm/master/shardddl/optimist.go +++ b/dm/master/shardddl/optimist.go @@ -245,27 +245,39 @@ func (o *Optimist) rebuildLocks() (revSource, revInfo, revOperation int64, err e } o.logger.Info("get history shard DDL lock operation", zap.Int64("revision", revOperation)) + colm, _, err := optimism.GetAllDroppedColumns(o.cli) + if err != nil { + // only log the error, and don't return it to forbid the startup of the DM-master leader. + // then these unexpected columns can be handled by the user. + o.logger.Error("fail to recover colms", log.ShortError(err)) + } + // recover the shard DDL lock based on history shard DDL info & lock operation. - err = o.recoverLocks(ifm, opm) + err = o.recoverLocks(ifm, opm, colm) if err != nil { // only log the error, and don't return it to forbid the startup of the DM-master leader. // then these unexpected locks can be handled by the user. o.logger.Error("fail to recover locks", log.ShortError(err)) } + return revSource, revInfo, revOperation, nil } // recoverLocks recovers shard DDL locks based on shard DDL info and shard DDL lock operation. func (o *Optimist) recoverLocks( ifm map[string]map[string]map[string]map[string]optimism.Info, - opm map[string]map[string]map[string]map[string]optimism.Operation) error { + opm map[string]map[string]map[string]map[string]optimism.Operation, + colm map[string]map[string]interface{}) error { // construct locks based on the shard DDL info. + o.lk.SetColumnMap(colm) + defer o.lk.SetColumnMap(nil) + for task, ifTask := range ifm { for _, ifSource := range ifTask { for _, ifSchema := range ifSource { for _, info := range ifSchema { tts := o.tk.FindTables(task, info.DownSchema, info.DownTable) - _, _, err := o.lk.TrySync(info, tts) + _, _, err := o.lk.TrySync(o.cli, info, tts) if err != nil { return err } @@ -291,6 +303,11 @@ func (o *Optimist) recoverLocks( } if op.Done { lock.TryMarkDone(op.Source, op.UpSchema, op.UpTable) + err := lock.DeleteColumnsByDDLs(op.DDLs) + if err != nil { + o.logger.Error("fail to update lock columns", zap.Error(err)) + continue + } } } } @@ -469,7 +486,7 @@ func (o *Optimist) handleOperationPut(ctx context.Context, opCh <-chan optimism. continue } - err := lock.UpdateColumns(op.DDLs) + err := lock.DeleteColumnsByDDLs(op.DDLs) if err != nil { o.logger.Error("fail to update lock columns", zap.Error(err)) } @@ -501,7 +518,7 @@ func (o *Optimist) handleOperationPut(ctx context.Context, opCh <-chan optimism. // handleLock handles a single shard DDL lock. func (o *Optimist) handleLock(info optimism.Info, tts []optimism.TargetTable, skipDone bool) error { - lockID, newDDLs, err := o.lk.TrySync(info, tts) + lockID, newDDLs, err := o.lk.TrySync(o.cli, info, tts) var cfStage = optimism.ConflictNone if err != nil { cfStage = optimism.ConflictDetected // we treat any errors returned from `TrySync` as conflict detected now. diff --git a/pkg/etcdutil/etcdutil.go b/pkg/etcdutil/etcdutil.go index 48bc1eff7d..6d8c9b9b51 100644 --- a/pkg/etcdutil/etcdutil.go +++ b/pkg/etcdutil/etcdutil.go @@ -108,7 +108,7 @@ func DoOpsInOneTxnWithRetry(cli *clientv3.Client, ops ...clientv3.Op) (*clientv3 ret, _, err := etcdDefaultTxnStrategy.Apply(tctx, etcdDefaultTxnRetryParam, func(t *tcontext.Context) (ret interface{}, err error) { resp, err := cli.Txn(ctx).Then(ops...).Commit() if err != nil { - return nil, err + return nil, errors.Trace(err) } return resp, nil }) diff --git a/pkg/shardddl/optimism/column.go b/pkg/shardddl/optimism/column.go new file mode 100644 index 0000000000..c765e01c71 --- /dev/null +++ b/pkg/shardddl/optimism/column.go @@ -0,0 +1,79 @@ +// Copyright 2021 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package optimism + +import ( + "go.etcd.io/etcd/clientv3" + + "github.com/pingcap/dm/dm/common" + "github.com/pingcap/dm/pkg/etcdutil" +) + +// GetAllDroppedColumns gets the all dropped columns. +func GetAllDroppedColumns(cli *clientv3.Client) (map[string]map[string]interface{}, int64, error) { + colm := make(map[string]map[string]interface{}) + op := clientv3.OpGet(common.ShardDDLOptimismDroppedColumnsKeyAdapter.Path(), clientv3.WithPrefix()) + respTxn, rev, err := etcdutil.DoOpsInOneTxnWithRetry(cli, op) + if err != nil { + return colm, 0, err + } + resp := respTxn.Responses[0].GetResponseRange() + + if resp.Count > 0 { + for _, kv := range resp.Kvs { + keys, err := common.ShardDDLOptimismDroppedColumnsKeyAdapter.Decode(string(kv.Key)) + if err != nil { + return colm, 0, err + } + lockID := keys[0] + column := keys[1] + if _, ok := colm[lockID]; !ok { + colm[lockID] = make(map[string]interface{}) + } + colm[lockID][column] = struct{}{} + } + } + return colm, rev, nil +} + +// PutDroppedColumn puts the undropped column name into ectd. +func PutDroppedColumn(cli *clientv3.Client, lockID, column string) (rev int64, putted bool, err error) { + key := common.ShardDDLOptimismDroppedColumnsKeyAdapter.Encode(lockID, column) + + op := clientv3.OpPut(key, "") + + resp, rev, err := etcdutil.DoOpsInOneTxnWithRetry(cli, op) + if err != nil { + return 0, false, err + } + return rev, resp.Succeeded, nil +} + +// DeleteDroppedColumns tries to delete the dropped columns for the specified lock ID. +func DeleteDroppedColumns(cli *clientv3.Client, lockID string, columns ...string) (rev int64, deleted bool, err error) { + ops := make([]clientv3.Op, 0, len(columns)) + for _, col := range columns { + ops = append(ops, deleteDroppedColumnOp(lockID, col)) + } + resp, rev, err := etcdutil.DoOpsInOneTxnWithRetry(cli, ops...) + if err != nil { + return 0, false, err + } + return rev, resp.Succeeded, nil +} + +// deleteDroppedColumnOp returns a DELETE etcd operation for init schema. +func deleteDroppedColumnOp(lockID, column string) clientv3.Op { + return clientv3.OpDelete(common.ShardDDLOptimismDroppedColumnsKeyAdapter.Encode(lockID, column)) +} diff --git a/pkg/shardddl/optimism/column_test.go b/pkg/shardddl/optimism/column_test.go new file mode 100644 index 0000000000..656ee9922a --- /dev/null +++ b/pkg/shardddl/optimism/column_test.go @@ -0,0 +1,54 @@ +// Copyright 2021 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package optimism + +import . "github.com/pingcap/check" + +type testColumn struct{} + +var _ = Suite(&testColumn{}) + +func (t *testColumn) TestColumnETCD(c *C) { + defer clearTestInfoOperation(c) + + lockID := "test-`shardddl`.`tb`" + rev1, putted, err := PutDroppedColumn(etcdTestCli, lockID, "a") + c.Assert(err, IsNil) + c.Assert(putted, IsTrue) + rev2, putted, err := PutDroppedColumn(etcdTestCli, lockID, "b") + c.Assert(err, IsNil) + c.Assert(putted, IsTrue) + c.Assert(rev2, Greater, rev1) + + expectedColm := map[string]map[string]interface{}{ + lockID: { + "a": struct{}{}, + "b": struct{}{}}, + } + colm, rev3, err := GetAllDroppedColumns(etcdTestCli) + c.Assert(err, IsNil) + c.Assert(colm, DeepEquals, expectedColm) + c.Assert(rev3, Equals, rev2) + + rev4, deleted, err := DeleteDroppedColumns(etcdTestCli, lockID, "b") + c.Assert(err, IsNil) + c.Assert(deleted, IsTrue) + c.Assert(rev4, Greater, rev3) + + delete(expectedColm[lockID], "b") + colm, rev5, err := GetAllDroppedColumns(etcdTestCli) + c.Assert(err, IsNil) + c.Assert(colm, DeepEquals, expectedColm) + c.Assert(rev5, Equals, rev4) +} diff --git a/pkg/shardddl/optimism/info.go b/pkg/shardddl/optimism/info.go index 31254b02bb..d0c715bc7a 100644 --- a/pkg/shardddl/optimism/info.go +++ b/pkg/shardddl/optimism/info.go @@ -229,6 +229,7 @@ func ClearTestInfoOperationSchema(cli *clientv3.Client) error { clearInfo := clientv3.OpDelete(common.ShardDDLOptimismInfoKeyAdapter.Path(), clientv3.WithPrefix()) clearOp := clientv3.OpDelete(common.ShardDDLOptimismOperationKeyAdapter.Path(), clientv3.WithPrefix()) clearISOp := clientv3.OpDelete(common.ShardDDLOptimismInitSchemaKeyAdapter.Path(), clientv3.WithPrefix()) - _, err := cli.Txn(context.Background()).Then(clearSource, clearInfo, clearOp, clearISOp).Commit() + clearColumns := clientv3.OpDelete(common.ShardDDLOptimismDroppedColumnsKeyAdapter.Path(), clientv3.WithPrefix()) + _, err := cli.Txn(context.Background()).Then(clearSource, clearInfo, clearOp, clearISOp, clearColumns).Commit() return err } diff --git a/pkg/shardddl/optimism/keeper.go b/pkg/shardddl/optimism/keeper.go index 0a30aa70a6..314c0da89e 100644 --- a/pkg/shardddl/optimism/keeper.go +++ b/pkg/shardddl/optimism/keeper.go @@ -17,6 +17,8 @@ import ( "sort" "sync" + "go.etcd.io/etcd/clientv3" + "github.com/pingcap/dm/pkg/utils" ) @@ -24,7 +26,8 @@ import ( // The lock information do not need to be persistent, and can be re-constructed from the shard DDL info. type LockKeeper struct { mu sync.RWMutex - locks map[string]*Lock // lockID -> Lock + locks map[string]*Lock // lockID -> Lock + colm map[string]map[string]interface{} // lockID -> not fully dropped column name -> interface{} } // NewLockKeeper creates a new LockKeeper instance. @@ -35,7 +38,7 @@ func NewLockKeeper() *LockKeeper { } // TrySync tries to sync the lock. -func (lk *LockKeeper) TrySync(info Info, tts []TargetTable) (string, []string, error) { +func (lk *LockKeeper) TrySync(cli *clientv3.Client, info Info, tts []TargetTable) (string, []string, error) { var ( lockID = genDDLLockID(info) l *Lock @@ -46,8 +49,13 @@ func (lk *LockKeeper) TrySync(info Info, tts []TargetTable) (string, []string, e defer lk.mu.Unlock() if l, ok = lk.locks[lockID]; !ok { - lk.locks[lockID] = NewLock(lockID, info.Task, info.DownSchema, info.DownTable, info.TableInfoBefore, tts) + lk.locks[lockID] = NewLock(cli, lockID, info.Task, info.DownSchema, info.DownTable, info.TableInfoBefore, tts) l = lk.locks[lockID] + if lk.colm != nil { + if columns, ok := lk.colm[lockID]; ok { + l.columns = columns + } + } } newDDLs, err := l.TrySync(info.Source, info.UpSchema, info.UpTable, info.DDLs, info.TableInfosAfter, tts, info.Version) @@ -103,6 +111,13 @@ func (lk *LockKeeper) Clear() { lk.locks = make(map[string]*Lock) } +func (lk *LockKeeper) SetColumnMap(colm map[string]map[string]interface{}) { + lk.mu.Lock() + defer lk.mu.Unlock() + + lk.colm = colm +} + // genDDLLockID generates DDL lock ID from its info. func genDDLLockID(info Info) string { return utils.GenDDLLockID(info.Task, info.DownSchema, info.DownTable) diff --git a/pkg/shardddl/optimism/keeper_test.go b/pkg/shardddl/optimism/keeper_test.go index e1d62423c3..6074a73f2f 100644 --- a/pkg/shardddl/optimism/keeper_test.go +++ b/pkg/shardddl/optimism/keeper_test.go @@ -57,7 +57,7 @@ func (t *testKeeper) TestLockKeeper(c *C) { ) // lock with 2 sources. - lockID1, newDDLs, err := lk.TrySync(i11, tts1) + lockID1, newDDLs, err := lk.TrySync(etcdTestCli, i11, tts1) c.Assert(err, IsNil) c.Assert(lockID1, Equals, "task1-`foo`.`bar`") c.Assert(newDDLs, DeepEquals, DDLs) @@ -69,7 +69,7 @@ func (t *testKeeper) TestLockKeeper(c *C) { c.Assert(synced, IsFalse) c.Assert(remain, Equals, 1) - lockID1, newDDLs, err = lk.TrySync(i12, tts1) + lockID1, newDDLs, err = lk.TrySync(etcdTestCli, i12, tts1) c.Assert(err, IsNil) c.Assert(lockID1, Equals, "task1-`foo`.`bar`") c.Assert(newDDLs, DeepEquals, DDLs) @@ -81,7 +81,7 @@ func (t *testKeeper) TestLockKeeper(c *C) { c.Assert(remain, Equals, 0) // lock with only 1 source. - lockID2, newDDLs, err := lk.TrySync(i21, tts2) + lockID2, newDDLs, err := lk.TrySync(etcdTestCli, i21, tts2) c.Assert(err, IsNil) c.Assert(lockID2, Equals, "task2-`foo`.`bar`") c.Assert(newDDLs, DeepEquals, DDLs) @@ -150,13 +150,13 @@ func (t *testKeeper) TestLockKeeperMultipleTarget(c *C) { ) // lock for target1. - lockID1, newDDLs, err := lk.TrySync(i11, tts1) + lockID1, newDDLs, err := lk.TrySync(etcdTestCli, i11, tts1) c.Assert(err, IsNil) c.Assert(lockID1, DeepEquals, "test-lock-keeper-multiple-target-`foo`.`bar`") c.Assert(newDDLs, DeepEquals, DDLs) // lock for target2. - lockID2, newDDLs, err := lk.TrySync(i21, tts2) + lockID2, newDDLs, err := lk.TrySync(etcdTestCli, i21, tts2) c.Assert(err, IsNil) c.Assert(lockID2, DeepEquals, "test-lock-keeper-multiple-target-`foo`.`rab`") c.Assert(newDDLs, DeepEquals, DDLs) @@ -178,11 +178,11 @@ func (t *testKeeper) TestLockKeeperMultipleTarget(c *C) { c.Assert(remain, Equals, 1) // sync for two locks. - lockID1, newDDLs, err = lk.TrySync(i12, tts1) + lockID1, newDDLs, err = lk.TrySync(etcdTestCli, i12, tts1) c.Assert(err, IsNil) c.Assert(lockID1, DeepEquals, "test-lock-keeper-multiple-target-`foo`.`bar`") c.Assert(newDDLs, DeepEquals, DDLs) - lockID2, newDDLs, err = lk.TrySync(i22, tts2) + lockID2, newDDLs, err = lk.TrySync(etcdTestCli, i22, tts2) c.Assert(err, IsNil) c.Assert(lockID2, DeepEquals, "test-lock-keeper-multiple-target-`foo`.`rab`") c.Assert(newDDLs, DeepEquals, DDLs) diff --git a/pkg/shardddl/optimism/lock.go b/pkg/shardddl/optimism/lock.go index 66c42681f4..0c10af6ed3 100644 --- a/pkg/shardddl/optimism/lock.go +++ b/pkg/shardddl/optimism/lock.go @@ -21,6 +21,7 @@ import ( "github.com/pingcap/parser/ast" "github.com/pingcap/parser/model" "github.com/pingcap/tidb-tools/pkg/schemacmp" + "go.etcd.io/etcd/clientv3" "go.uber.org/zap" "github.com/pingcap/dm/dm/master/metrics" @@ -33,6 +34,8 @@ import ( type Lock struct { mu sync.RWMutex + cli *clientv3.Client + ID string // lock's ID Task string // lock's corresponding task name @@ -64,8 +67,9 @@ type Lock struct { // NewLock creates a new Lock instance. // NOTE: we MUST give the initial table info when creating the lock now. -func NewLock(ID, task, downSchema, downTable string, ti *model.TableInfo, tts []TargetTable) *Lock { +func NewLock(cli *clientv3.Client, ID, task, downSchema, downTable string, ti *model.TableInfo, tts []TargetTable) *Lock { l := &Lock{ + cli: cli, ID: ID, Task: task, DownSchema: downSchema, @@ -276,7 +280,10 @@ func (l *Lock) TrySync(callerSource, callerSchema, callerTable string, if col, err := GetColumnName(l.ID, ddls[idx], ast.AlterTableDropColumn); err != nil { return ddls, err } else if len(col) > 0 { - l.columns[col] = struct{}{} + err = l.AddDroppedColumn(col) + if err != nil { + log.L().Error("fail to add dropped column info in etcd", zap.Error(err)) + } } // last shard table won't go here continue @@ -506,20 +513,42 @@ func (l *Lock) GetVersion(source string, schema string, table string) int64 { return l.versions[source][schema][table] } -// GetVersion return version of info in lock. -func (l *Lock) UpdateColumns(ddls []string) error { +func (l *Lock) AddDroppedColumn(col string) error { + if _, ok := l.columns[col]; ok { + return nil + } + + _, _, err := PutDroppedColumn(l.cli, l.ID, col) + if err != nil { + return err + } + l.columns[col] = struct{}{} + return nil +} + +// DeleteColumnsByDDLs deletes the dropped columns by DDLs. +func (l *Lock) DeleteColumnsByDDLs(ddls []string) error { l.mu.Lock() defer l.mu.Unlock() + colsToDelete := make([]string, 0, len(ddls)) for _, ddl := range ddls { col, err := GetColumnName(l.ID, ddl, ast.AlterTableDropColumn) if err != nil { return err } if len(col) > 0 { - delete(l.columns, col) + colsToDelete = append(colsToDelete, col) } } + _, _, err := DeleteDroppedColumns(l.cli, l.ID, colsToDelete...) + if err != nil { + return err + } + for _, col := range colsToDelete { + delete(l.columns, col) + } + return nil } diff --git a/pkg/shardddl/optimism/lock_test.go b/pkg/shardddl/optimism/lock_test.go index 0a9d1a696f..dd4780297e 100644 --- a/pkg/shardddl/optimism/lock_test.go +++ b/pkg/shardddl/optimism/lock_test.go @@ -31,6 +31,7 @@ var _ = Suite(&testLock{}) func (t *testLock) SetUpSuite(c *C) { c.Assert(log.InitLogger(&log.Config{}), IsNil) + clearTestInfoOperation(c) } func (t *testLock) TestLockTrySyncNormal(c *C) { @@ -66,7 +67,7 @@ func (t *testLock) TestLockTrySyncNormal(c *C) { newTargetTable(task, sources[1], downSchema, downTable, tables), } - l = NewLock(ID, task, downSchema, downTable, ti0, tts) + l = NewLock(etcdTestCli, ID, task, downSchema, downTable, ti0, tts) vers = map[string]map[string]map[string]int64{ sources[0]: { @@ -360,7 +361,7 @@ func (t *testLock) TestLockTrySyncIndex(c *C) { newTargetTable(task, source, downSchema, downTable, tables), } - l = NewLock(ID, task, downSchema, downTable, ti0, tts) + l = NewLock(etcdTestCli, ID, task, downSchema, downTable, ti0, tts) vers = map[string]map[string]map[string]int64{ source: { @@ -439,7 +440,7 @@ func (t *testLock) TestLockTrySyncNullNotNull(c *C) { newTargetTable(task, source, downSchema, downTable, tables), } - l = NewLock(ID, task, downSchema, downTable, ti0, tts) + l = NewLock(etcdTestCli, ID, task, downSchema, downTable, ti0, tts) vers = map[string]map[string]map[string]int64{ source: { @@ -506,7 +507,7 @@ func (t *testLock) TestLockTrySyncIntBigint(c *C) { newTargetTable(task, source, downSchema, downTable, tables), } - l = NewLock(ID, task, downSchema, downTable, ti0, tts) + l = NewLock(etcdTestCli, ID, task, downSchema, downTable, ti0, tts) vers = map[string]map[string]map[string]int64{ source: { @@ -557,7 +558,7 @@ func (t *testLock) TestLockTrySyncNoDiff(c *C) { newTargetTable(task, source, downSchema, downTable, tables), } - l = NewLock(ID, task, downSchema, downTable, ti0, tts) + l = NewLock(etcdTestCli, ID, task, downSchema, downTable, ti0, tts) vers = map[string]map[string]map[string]int64{ source: { @@ -599,7 +600,7 @@ func (t *testLock) TestLockTrySyncNewTable(c *C) { tables = map[string]map[string]struct{}{db1: {tbl1: struct{}{}}} tts = []TargetTable{newTargetTable(task, source1, downSchema, downTable, tables)} - l = NewLock(ID, task, downSchema, downTable, ti0, tts) + l = NewLock(etcdTestCli, ID, task, downSchema, downTable, ti0, tts) vers = map[string]map[string]map[string]int64{ source1: { db1: {tbl1: 0}, @@ -692,7 +693,7 @@ func (t *testLock) TestLockTrySyncRevert(c *C) { tables = map[string]map[string]struct{}{db: {tbls[0]: struct{}{}, tbls[1]: struct{}{}}} tts = []TargetTable{newTargetTable(task, source, downSchema, downTable, tables)} - l = NewLock(ID, task, downSchema, downTable, ti0, tts) + l = NewLock(etcdTestCli, ID, task, downSchema, downTable, ti0, tts) vers = map[string]map[string]map[string]int64{ source: { @@ -839,7 +840,7 @@ func (t *testLock) TestLockTrySyncConflictNonIntrusive(c *C) { tables = map[string]map[string]struct{}{db: {tbls[0]: struct{}{}, tbls[1]: struct{}{}}} tts = []TargetTable{newTargetTable(task, source, downSchema, downTable, tables)} - l = NewLock(ID, task, downSchema, downTable, ti0, tts) + l = NewLock(etcdTestCli, ID, task, downSchema, downTable, ti0, tts) vers = map[string]map[string]map[string]int64{ source: { @@ -954,7 +955,7 @@ func (t *testLock) TestLockTrySyncConflictIntrusive(c *C) { tables = map[string]map[string]struct{}{db: {tbls[0]: struct{}{}, tbls[1]: struct{}{}}} tts = []TargetTable{newTargetTable(task, source, downSchema, downTable, tables)} - l = NewLock(ID, task, downSchema, downTable, ti0, tts) + l = NewLock(etcdTestCli, ID, task, downSchema, downTable, ti0, tts) vers = map[string]map[string]map[string]int64{ source: { @@ -1144,7 +1145,7 @@ func (t *testLock) TestLockTrySyncMultipleChangeDDL(c *C) { newTargetTable(task, sources[1], downSchema, downTable, tables), } - l = NewLock(ID, task, downSchema, downTable, ti0, tts) + l = NewLock(etcdTestCli, ID, task, downSchema, downTable, ti0, tts) vers = map[string]map[string]map[string]int64{ sources[0]: { @@ -1282,7 +1283,7 @@ func (t *testLock) TestTryRemoveTable(c *C) { tables = map[string]map[string]struct{}{db: {tbl1: struct{}{}, tbl2: struct{}{}}} tts = []TargetTable{newTargetTable(task, source, downSchema, downTable, tables)} - l = NewLock(ID, task, downSchema, downTable, ti0, tts) + l = NewLock(etcdTestCli, ID, task, downSchema, downTable, ti0, tts) vers = map[string]map[string]map[string]int64{ source: { @@ -1371,7 +1372,7 @@ func (t *testLock) TestLockTryMarkDone(c *C) { tables = map[string]map[string]struct{}{db: {tbls[0]: struct{}{}, tbls[1]: struct{}{}}} tts = []TargetTable{newTargetTable(task, source, downSchema, downTable, tables)} - l = NewLock(ID, task, downSchema, downTable, ti0, tts) + l = NewLock(etcdTestCli, ID, task, downSchema, downTable, ti0, tts) vers = map[string]map[string]map[string]int64{ source: { @@ -1476,7 +1477,7 @@ func (t *testLock) TestAddDifferentFieldLenColumns(c *C) { tables = map[string]map[string]struct{}{db: {tbls[0]: struct{}{}, tbls[1]: struct{}{}}} tts = []TargetTable{newTargetTable(task, source, downSchema, downTable, tables)} - l = NewLock(ID, task, downSchema, downTable, ti0, tts) + l = NewLock(etcdTestCli, ID, task, downSchema, downTable, ti0, tts) vers = map[string]map[string]map[string]int64{ source: { @@ -1516,7 +1517,7 @@ func (t *testLock) TestAddDifferentFieldLenColumns(c *C) { c.Assert(l.versions, DeepEquals, vers) // case 2: add a column with a smaller field length - l = NewLock(ID, task, downSchema, downTable, ti0, tts) + l = NewLock(etcdTestCli, ID, task, downSchema, downTable, ti0, tts) // TrySync for the first table, no table has done the DDLs operation. vers[source][db][tbls[0]]-- @@ -1560,13 +1561,25 @@ func (t *testLock) TestAddNotFullyDroppedColumns(c *C) { tables = map[string]map[string]struct{}{db: {tbls[0]: struct{}{}, tbls[1]: struct{}{}}} tts = []TargetTable{newTargetTable(task, source, downSchema, downTable, tables)} - l = NewLock(ID, task, downSchema, downTable, ti0, tts) + l = NewLock(etcdTestCli, ID, task, downSchema, downTable, ti0, tts) vers = map[string]map[string]map[string]int64{ source: { db: {tbls[0]: 0, tbls[1]: 0}, }, } + + colm1 = map[string]map[string]interface{}{ + ID: { + "b": struct{}{}, + "c": struct{}{}, + }, + } + colm2 = map[string]map[string]interface{}{ + ID: { + "c": struct{}{}, + }, + } ) col, err := GetColumnName(ID, DDLs1[0], ast.AlterTableDropColumn) c.Assert(col, Equals, "c") @@ -1596,6 +1609,10 @@ func (t *testLock) TestAddNotFullyDroppedColumns(c *C) { c.Assert(l.versions, DeepEquals, vers) c.Assert(l.IsResolved(), IsFalse) + colm, _, err := GetAllDroppedColumns(etcdTestCli) + c.Assert(err, IsNil) + c.Assert(colm, DeepEquals, colm1) + // TrySync for the second table, drop column b, this column should be dropped vers[source][db][tbls[1]]++ DDLs, err = l.TrySync(source, db, tbls[1], DDLs2, []*model.TableInfo{ti3}, tts, vers[source][db][tbls[1]]) @@ -1604,7 +1621,11 @@ func (t *testLock) TestAddNotFullyDroppedColumns(c *C) { c.Assert(l.versions, DeepEquals, vers) c.Assert(l.IsResolved(), IsFalse) // Simulate watch done operation from dm-worker - c.Assert(l.UpdateColumns(DDLs), IsNil) + c.Assert(l.DeleteColumnsByDDLs(DDLs), IsNil) + + colm, _, err = GetAllDroppedColumns(etcdTestCli) + c.Assert(err, IsNil) + c.Assert(colm, DeepEquals, colm2) // TrySync for the first table, add column b, should succeed, because this column is fully dropped in the downstream vers[source][db][tbls[0]]++ @@ -1628,7 +1649,7 @@ func (t *testLock) TestAddNotFullyDroppedColumns(c *C) { c.Assert(l.versions, DeepEquals, vers) c.Assert(l.IsResolved(), IsFalse) // Simulate watch done operation from dm-worker - c.Assert(l.UpdateColumns(DDLs), IsNil) + c.Assert(l.DeleteColumnsByDDLs(DDLs), IsNil) // TrySync for the first table, add column b, should fail, because this column isn't fully dropped in the downstream vers[source][db][tbls[0]]++ diff --git a/tests/_utils/test_prepare b/tests/_utils/test_prepare index 579a04d9ec..34a9163e94 100644 --- a/tests/_utils/test_prepare +++ b/tests/_utils/test_prepare @@ -136,7 +136,12 @@ function run_case() { "\"unit\": \"Sync\"" 2 fi - DM_${case}_CASE $5 + args="" + for((i=5;i<=$#;i++)); do + j=${!i} + args="${args} $j " + done + DM_${case}_CASE $args run_dm_ctl $WORK_DIR "127.0.0.1:$MASTER_PORT" \ "stop-task test" diff --git a/tests/shardddl3/run.sh b/tests/shardddl3/run.sh index d251531b00..7bc079f43c 100644 --- a/tests/shardddl3/run.sh +++ b/tests/shardddl3/run.sh @@ -928,6 +928,16 @@ function DM_RemoveLock() { "bound" 2 } +function restart_master() { + echo "restart dm-master" + ps aux | grep dm-master |awk '{print $2}'|xargs kill || true + check_port_offline $MASTER_PORT 20 + sleep 2 + + run_dm_master $WORK_DIR/master $MASTER_PORT $cur/conf/dm-master.toml + check_rpc_alive $cur/../bin/check_master_online 127.0.0.1:$MASTER_PORT +} + function DM_RestartMaster_CASE() { run_sql_source1 "insert into ${shardddl1}.${tb1} values(1,'aaa');" run_sql_source2 "insert into ${shardddl1}.${tb1} values(2,'bbb');" @@ -956,11 +966,7 @@ function DM_RestartMaster_CASE() { 'mysql-replica-02-`shardddl1`.`tb1`' 1 fi - echo "restart dm-master" - ps aux | grep dm-master |awk '{print $2}'|xargs kill || true - check_port_offline $MASTER_PORT 20 - run_dm_master $WORK_DIR/master $MASTER_PORT $cur/conf/dm-master.toml - check_rpc_alive $cur/../bin/check_master_online 127.0.0.1:$MASTER_PORT + restart_master if [[ "$1" = "pessimistic" ]]; then run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" \ @@ -993,6 +999,88 @@ function DM_RestartMaster() { "clean_table" "optimistic" } +function restart_master_on_pos() { + if [ "$1" = "$2" ]; then + restart_master + fi +} + +function DM_DropAddColumn_CASE() { + reset=$2 + + run_sql_source1 "insert into ${shardddl1}.${tb1} values(1,1,1);" + run_sql_source2 "insert into ${shardddl1}.${tb1} values(2,2,2);" + + check_sync_diff $WORK_DIR $cur/conf/diff_config.toml + + run_sql_source1 "alter table ${shardddl1}.${tb1} drop column c;" + run_sql_source1 "insert into ${shardddl1}.${tb1} values(3,3);" + + restart_master_on_pos $reset "1" + + run_sql_source1 "alter table ${shardddl1}.${tb1} drop column b;" + run_sql_source1 "insert into ${shardddl1}.${tb1} values(4);" + + restart_master_on_pos $reset "2" + + run_sql_source2 "alter table ${shardddl1}.${tb1} drop column c;" + run_sql_source2 "insert into ${shardddl1}.${tb1} values(5,5);" + + restart_master_on_pos $reset "3" + + run_sql_source1 "alter table ${shardddl1}.${tb1} add column c int;" + run_sql_source1 "insert into ${shardddl1}.${tb1} values(6,6);" + + restart_master_on_pos $reset "4" + + # make sure task to step in "Sync" stage + run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "query-status test" \ + "\"stage\": \"Running\"" 3 \ + "\"unit\": \"Sync\"" 2 + + run_sql_source1 "alter table ${shardddl1}.${tb1} add column b int after a;" + + restart_master_on_pos $reset "5" + + run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "query-status test" \ + "because schema conflict detected" 1 + + check_sync_diff $WORK_DIR $cur/conf/diff_config.toml 3 'fail' + + # try to fix data + echo 'CREATE TABLE `tb1` ( `a` int(11) NOT NULL, `b` int(11) DEFAULT NULL, `c` int(11) DEFAULT NULL, PRIMARY KEY (`a`) /*T![clustered_index] NONCLUSTERED */) ENGINE=InnoDB DEFAULT CHARSET=latin1 COLLATE=latin1_bin' > ${WORK_DIR}/schema.sql + run_dm_ctl $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "operate-schema set test ${WORK_DIR}/schema.sql -s mysql-replica-01 -d ${shardddl1} -t ${tb1}" \ + "\"result\": true" 2 + + # skip this error + run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "handle-error test skip" \ + "\"result\": true" 2 \ + "\"source 'mysql-replica-02' has no error\"" 1 + + run_sql_source1 "update ${shardddl1}.${tb1} set b=1 where a=1;" + run_sql_source1 "update ${shardddl1}.${tb1} set b=3 where a=3;" + run_sql_source1 "update ${shardddl1}.${tb1} set b=4 where a=4;" + run_sql_source1 "update ${shardddl1}.${tb1} set b=6 where a=6;" + run_sql_source2 "alter table ${shardddl1}.${tb1} add column c int" + run_sql_source2 "insert into ${shardddl1}.${tb1} values(7,7,7);" + + check_sync_diff $WORK_DIR $cur/conf/diff_config.toml +} + +function DM_DropAddColumn() { + for i in `seq 0 5` + do + run_case DropAddColumn "double-source-optimistic" \ + "run_sql_source1 \"create table ${shardddl1}.${tb1} (a int primary key, b int, c int);\"; \ + run_sql_source2 \"create table ${shardddl1}.${tb1} (a int primary key, b int, c int);\"" \ + "clean_table" "optimistic" "$i" + done +} + function run() { init_cluster init_database @@ -1010,6 +1098,8 @@ function run() { DM_RemoveLock DM_RestartMaster + + DM_DropAddColumn } cleanup_data $shardddl From 01f629224d15bffe95669087e89cec686439f436 Mon Sep 17 00:00:00 2001 From: Chunzhu Li Date: Mon, 15 Mar 2021 15:59:14 +0800 Subject: [PATCH 03/23] fix hound --- pkg/shardddl/optimism/keeper.go | 1 + pkg/shardddl/optimism/lock.go | 1 + 2 files changed, 2 insertions(+) diff --git a/pkg/shardddl/optimism/keeper.go b/pkg/shardddl/optimism/keeper.go index 314c0da89e..067ac7b81d 100644 --- a/pkg/shardddl/optimism/keeper.go +++ b/pkg/shardddl/optimism/keeper.go @@ -111,6 +111,7 @@ func (lk *LockKeeper) Clear() { lk.locks = make(map[string]*Lock) } +// SetColumnMap sets the column map received from etcd func (lk *LockKeeper) SetColumnMap(colm map[string]map[string]interface{}) { lk.mu.Lock() defer lk.mu.Unlock() diff --git a/pkg/shardddl/optimism/lock.go b/pkg/shardddl/optimism/lock.go index 0c10af6ed3..f83fabbc4a 100644 --- a/pkg/shardddl/optimism/lock.go +++ b/pkg/shardddl/optimism/lock.go @@ -513,6 +513,7 @@ func (l *Lock) GetVersion(source string, schema string, table string) int64 { return l.versions[source][schema][table] } +// AddDroppedColumn adds a dropped column name in both etcd and lock's column map func (l *Lock) AddDroppedColumn(col string) error { if _, ok := l.columns[col]; ok { return nil From 1677848170d3f893e445071031b718dbb586ea75 Mon Sep 17 00:00:00 2001 From: Chunzhu Li Date: Mon, 15 Mar 2021 16:55:42 +0800 Subject: [PATCH 04/23] fix lint --- dm/master/server_test.go | 2 +- loader/util.go | 2 +- loader/util_test.go | 2 +- monitoring/dashboards/dashboard.go | 2 +- pkg/shardddl/optimism/lock_test.go | 2 +- pkg/utils/db_test.go | 2 +- syncer/err-operator/operator.go | 2 +- 7 files changed, 7 insertions(+), 7 deletions(-) diff --git a/dm/master/server_test.go b/dm/master/server_test.go index b56f1d9bf0..2262d99040 100644 --- a/dm/master/server_test.go +++ b/dm/master/server_test.go @@ -1470,7 +1470,7 @@ func (t *testMaster) TestOfflineMember(c *check.C) { c.Assert(err, check.IsNil) c.Assert(listResp.Members, check.HasLen, 3) - // make sure s3 is not the leader, otherwise it will take some time to campain a new leader after close s3, and it may cause timeout + // make sure s3 is not the leader, otherwise it will take some time to campaign a new leader after close s3, and it may cause timeout c.Assert(utils.WaitSomething(20, 500*time.Millisecond, func() bool { _, leaderID, _, err = s1.election.LeaderInfo(ctx) if err != nil { diff --git a/loader/util.go b/loader/util.go index 5069778618..f1a8855fea 100644 --- a/loader/util.go +++ b/loader/util.go @@ -123,7 +123,7 @@ func getDBAndTableFromFilename(filename string) (string, string, error) { } fields := strings.Split(filename[:idx], ".") if len(fields) != 2 && len(fields) != 3 { - return "", "", fmt.Errorf("%s doesn't have correct `.` seperator", filename) + return "", "", fmt.Errorf("%s doesn't have correct `.` separator", filename) } return fields[0], fields[1], nil } diff --git a/loader/util_test.go b/loader/util_test.go index 1d399ae65c..cc5bc4d0b5 100644 --- a/loader/util_test.go +++ b/loader/util_test.go @@ -110,7 +110,7 @@ func (t *testUtilSuite) TestGetDBAndTableFromFilename(c *C) { {"sqldb.tbl.0.sql", "sqldb", "tbl", ""}, {"db.tbl.sql0.sql", "db", "tbl", ""}, {"db.tbl.0", "", "", ".*doesn't have a `.sql` suffix.*"}, - {"db.sql", "", "", ".*doesn't have correct `.` seperator.*"}, + {"db.sql", "", "", ".*doesn't have correct `.` separator.*"}, {"db.0.sql", "db", "0", ""}, // treat `0` as the table name. } diff --git a/monitoring/dashboards/dashboard.go b/monitoring/dashboards/dashboard.go index a71db5c8a3..ac6f0421b4 100644 --- a/monitoring/dashboards/dashboard.go +++ b/monitoring/dashboards/dashboard.go @@ -84,7 +84,7 @@ func filterDashboard(str string, dashboard string, title string) string { str = strings.ReplaceAll(str, fmt.Sprintf("${%s}", datasource), datasourceName) } - // delete input defination + // delete input definition if gjson.Get(str, "__inputs").Exists() { str, err = sjson.Delete(str, "__inputs") checkErr(err, "delete path failed") diff --git a/pkg/shardddl/optimism/lock_test.go b/pkg/shardddl/optimism/lock_test.go index dd4780297e..4d003a807d 100644 --- a/pkg/shardddl/optimism/lock_test.go +++ b/pkg/shardddl/optimism/lock_test.go @@ -1637,7 +1637,7 @@ func (t *testLock) TestAddNotFullyDroppedColumns(c *C) { // TrySync for the first table, add column b, should fail, because this column isn't fully dropped in the downstream vers[source][db][tbls[0]]++ - DDLs, err = l.TrySync(source, db, tbls[0], DDLs4, []*model.TableInfo{ti0}, tts, vers[source][db][tbls[0]]) + _, err = l.TrySync(source, db, tbls[0], DDLs4, []*model.TableInfo{ti0}, tts, vers[source][db][tbls[0]]) c.Assert(err, ErrorMatches, ".*add column c that wasn't fully dropped in downstream.*") c.Assert(l.IsResolved(), IsFalse) diff --git a/pkg/utils/db_test.go b/pkg/utils/db_test.go index b68c0d9771..8c9c02370f 100644 --- a/pkg/utils/db_test.go +++ b/pkg/utils/db_test.go @@ -50,7 +50,7 @@ func (t *testDBSuite) TestGetFlavor(c *C) { c.Assert(mock.ExpectationsWereMet(), IsNil) // others - mock.ExpectQuery(`SHOW GLOBAL VARIABLES LIKE 'version';`).WillReturnRows(sqlmock.NewRows([]string{"Variable_name", "Value"}).AddRow("version", "unkown")) + mock.ExpectQuery(`SHOW GLOBAL VARIABLES LIKE 'version';`).WillReturnRows(sqlmock.NewRows([]string{"Variable_name", "Value"}).AddRow("version", "unknown")) flavor, err = GetFlavor(context.Background(), db) c.Assert(err, IsNil) c.Assert(flavor, Equals, "mysql") // as MySQL diff --git a/syncer/err-operator/operator.go b/syncer/err-operator/operator.go index 0e2d89b4ac..33aa1bafc8 100644 --- a/syncer/err-operator/operator.go +++ b/syncer/err-operator/operator.go @@ -179,7 +179,7 @@ func (h *Holder) RemoveOutdated(flushLocation binlog.Location) error { return err } if binlog.ComparePosition(position, flushLocation.Position) == -1 { - h.logger.Info("remove a outdated operator", zap.Stringer("position", position), zap.Stringer("flush postion", flushLocation.Position), zap.Stringer("operator", h.operators[pos])) + h.logger.Info("remove a outdated operator", zap.Stringer("position", position), zap.Stringer("flush position", flushLocation.Position), zap.Stringer("operator", h.operators[pos])) delete(h.operators, pos) } } From 4bcda8aca6f2f78ca8e30134db68e463e1eaf9d9 Mon Sep 17 00:00:00 2001 From: Chunzhu Li Date: Tue, 16 Mar 2021 21:32:18 +0800 Subject: [PATCH 05/23] fix integration tests and unit tests --- dm/common/common.go | 6 +- dm/master/server_test.go | 2 +- dm/master/shardddl/optimist.go | 81 +++++++++++++------------ dm/master/shardddl/optimist_test.go | 24 ++++---- pkg/shardddl/optimism/column.go | 25 +++++--- pkg/shardddl/optimism/column_test.go | 13 ++-- pkg/shardddl/optimism/info.go | 5 ++ pkg/shardddl/optimism/info_test.go | 11 +++- pkg/shardddl/optimism/lock.go | 20 +++--- pkg/shardddl/optimism/operation.go | 18 +++++- pkg/shardddl/optimism/operation_test.go | 23 ++++--- pkg/shardddl/optimism/ops.go | 6 +- pkg/shardddl/optimism/ops_test.go | 10 +-- syncer/shardddl/optimist.go | 2 +- syncer/shardddl/optimist_test.go | 6 +- tests/shardddl3/run.sh | 1 + 16 files changed, 157 insertions(+), 96 deletions(-) diff --git a/dm/common/common.go b/dm/common/common.go index 82f3d39421..5d17145eda 100644 --- a/dm/common/common.go +++ b/dm/common/common.go @@ -80,7 +80,7 @@ var ( // k/v: Encode(task-name, downstream-schema-name, downstream-table-name) -> table schema. ShardDDLOptimismInitSchemaKeyAdapter KeyAdapter = keyHexEncoderDecoder("/dm-master/shardddl-optimism/init-schema/") // ShardDDLOptimismDroppedColumnsKeyAdapter is used to store the columns that are not fully dropped - // k/v: Encode(lock-id, column-name) -> empty + // k/v: Encode(task-name, downstream-schema-name, downstream-table-name, column-name) -> empty ShardDDLOptimismDroppedColumnsKeyAdapter KeyAdapter = keyHexEncoderDecoder("/dm-master/shardddl-optimism/undropped-columns/") ) @@ -92,11 +92,11 @@ func keyAdapterKeysLen(s KeyAdapter) int { return 1 case UpstreamSubTaskKeyAdapter, StageSubTaskKeyAdapter, ShardDDLPessimismInfoKeyAdapter, ShardDDLPessimismOperationKeyAdapter, - ShardDDLOptimismSourceTablesKeyAdapter, ShardDDLOptimismDroppedColumnsKeyAdapter: + ShardDDLOptimismSourceTablesKeyAdapter: return 2 case ShardDDLOptimismInitSchemaKeyAdapter: return 3 - case ShardDDLOptimismInfoKeyAdapter, ShardDDLOptimismOperationKeyAdapter: + case ShardDDLOptimismInfoKeyAdapter, ShardDDLOptimismOperationKeyAdapter, ShardDDLOptimismDroppedColumnsKeyAdapter: return 4 } diff --git a/dm/master/server_test.go b/dm/master/server_test.go index 2262d99040..499c279117 100644 --- a/dm/master/server_test.go +++ b/dm/master/server_test.go @@ -641,7 +641,7 @@ func (t *testMaster) TestStartTaskWithRemoveMeta(c *check.C) { c.Assert(err, check.IsNil) _, err = optimism.PutInfo(etcdTestCli, info1) c.Assert(err, check.IsNil) - _, succ, err = optimism.PutOperation(etcdTestCli, false, op1) + _, succ, err = optimism.PutOperation(etcdTestCli, false, op1, 0) c.Assert(succ, check.IsTrue) c.Assert(err, check.IsNil) diff --git a/dm/master/shardddl/optimist.go b/dm/master/shardddl/optimist.go index 9efdc8ad73..09f8b4bdbc 100644 --- a/dm/master/shardddl/optimist.go +++ b/dm/master/shardddl/optimist.go @@ -271,21 +271,19 @@ func (o *Optimist) recoverLocks( // construct locks based on the shard DDL info. o.lk.SetColumnMap(colm) defer o.lk.SetColumnMap(nil) + var firstErr error + setFirstErr := func(err error) { + if firstErr == nil && err != nil { + firstErr = err + } + } - for task, ifTask := range ifm { + for _, ifTask := range ifm { for _, ifSource := range ifTask { for _, ifSchema := range ifSource { for _, info := range ifSchema { - tts := o.tk.FindTables(task, info.DownSchema, info.DownTable) - _, _, err := o.lk.TrySync(o.cli, info, tts) - if err != nil { - return err - } - // never mark the lock operation from `done` to `not-done` when recovering. - err = o.handleLock(info, tts, true) - if err != nil { - return err - } + err := o.handleInfo(info) + setFirstErr(err) } } } @@ -355,7 +353,7 @@ func (o *Optimist) watchSourceInfoOperation( }() go func() { defer wg.Done() - o.handleInfo(ctx, infoCh) + o.handleInfoPut(ctx, infoCh) }() // watch for the shard DDL lock operation and handle them. @@ -398,8 +396,8 @@ func (o *Optimist) handleSourceTables(ctx context.Context, sourceCh <-chan optim } } -// handleInfo handles PUT and DELETE for the shard DDL info. -func (o *Optimist) handleInfo(ctx context.Context, infoCh <-chan optimism.Info) { +// handleInfoPut handles PUT and DELETE for the shard DDL info. +func (o *Optimist) handleInfoPut(ctx context.Context, infoCh <-chan optimism.Info) { for { select { case <-ctx.Done(): @@ -431,35 +429,38 @@ func (o *Optimist) handleInfo(ctx context.Context, infoCh <-chan optimism.Info) continue } - added := o.tk.AddTable(info.Task, info.Source, info.UpSchema, info.UpTable, info.DownSchema, info.DownTable) - o.logger.Debug("a table added for info", zap.Bool("added", added), zap.Stringer("info", info)) - - tts := o.tk.FindTables(info.Task, info.DownSchema, info.DownTable) - if tts == nil { - // WATCH for SourceTables may fall behind WATCH for Info although PUT earlier, - // so we try to get SourceTables again. - // NOTE: check SourceTables for `info.Source` if needed later. - stm, _, err := optimism.GetAllSourceTables(o.cli) - if err != nil { - o.logger.Error("fail to get source tables", log.ShortError(err)) - } else if tts2 := optimism.TargetTablesForTask(info.Task, info.DownSchema, info.DownTable, stm); tts2 != nil { - tts = tts2 - } - } - // put operation for the table. we don't set `skipDone=true` now, - // because in optimism mode, one table may execute/done multiple DDLs but other tables may do nothing. - err := o.handleLock(info, tts, false) - if err != nil { - o.logger.Error("fail to handle the shard DDL lock", zap.Stringer("info", info), log.ShortError(err)) - metrics.ReportDDLError(info.Task, metrics.InfoErrHandleLock) - o.mu.Unlock() - continue - } + _ = o.handleInfo(info) o.mu.Unlock() } } } +func (o *Optimist) handleInfo(info optimism.Info) error { + added := o.tk.AddTable(info.Task, info.Source, info.UpSchema, info.UpTable, info.DownSchema, info.DownTable) + o.logger.Debug("a table added for info", zap.Bool("added", added), zap.Stringer("info", info)) + + tts := o.tk.FindTables(info.Task, info.DownSchema, info.DownTable) + if tts == nil { + // WATCH for SourceTables may fall behind WATCH for Info although PUT earlier, + // so we try to get SourceTables again. + // NOTE: check SourceTables for `info.Source` if needed later. + stm, _, err := optimism.GetAllSourceTables(o.cli) + if err != nil { + o.logger.Error("fail to get source tables", log.ShortError(err)) + } else if tts2 := optimism.TargetTablesForTask(info.Task, info.DownSchema, info.DownTable, stm); tts2 != nil { + tts = tts2 + } + } + // put operation for the table. we don't set `skipDone=true` now, + // because in optimism mode, one table may execute/done multiple DDLs but other tables may do nothing. + err := o.handleLock(info, tts, false) + if err != nil { + o.logger.Error("fail to handle the shard DDL lock", zap.Stringer("info", info), log.ShortError(err)) + metrics.ReportDDLError(info.Task, metrics.InfoErrHandleLock) + } + return err +} + // handleOperationPut handles PUT for the shard DDL lock operations. func (o *Optimist) handleOperationPut(ctx context.Context, opCh <-chan optimism.Operation) { for { @@ -558,7 +559,7 @@ func (o *Optimist) handleLock(info optimism.Info, tts []optimism.TargetTable, sk } op := optimism.NewOperation(lockID, lock.Task, info.Source, info.UpSchema, info.UpTable, newDDLs, cfStage, false) - rev, succ, err := optimism.PutOperation(o.cli, skipDone, op) + rev, succ, err := optimism.PutOperation(o.cli, skipDone, op, info.ModRevision) if err != nil { return err } @@ -635,7 +636,7 @@ func (o *Optimist) deleteInfosOps(lock *optimism.Lock) (bool, error) { } // NOTE: we rely on only `task`, `downSchema`, and `downTable` used for deletion. initSchema := optimism.NewInitSchema(lock.Task, lock.DownSchema, lock.DownTable, nil) - rev, deleted, err := optimism.DeleteInfosOperationsSchema(o.cli, infos, ops, initSchema) + rev, deleted, err := optimism.DeleteInfosOperationsSchemaColumn(o.cli, infos, ops, initSchema) if err != nil { return deleted, err } diff --git a/dm/master/shardddl/optimist_test.go b/dm/master/shardddl/optimist_test.go index a8c73ed3bb..c8e37068c0 100644 --- a/dm/master/shardddl/optimist_test.go +++ b/dm/master/shardddl/optimist_test.go @@ -272,7 +272,7 @@ func (t *testOptimist) testOptimist(c *C, restart int) { // mark op11 as done. op11c := op11 op11c.Done = true - _, putted, err := optimism.PutOperation(etcdTestCli, false, op11c) + _, putted, err := optimism.PutOperation(etcdTestCli, false, op11c, 0) c.Assert(err, IsNil) c.Assert(putted, IsTrue) c.Assert(utils.WaitSomething(backOff, waitTime, func() bool { @@ -330,7 +330,7 @@ func (t *testOptimist) testOptimist(c *C, restart int) { // mark op12 as done, the lock should be resolved. op12c := op12 op12c.Done = true - _, putted, err = optimism.PutOperation(etcdTestCli, false, op12c) + _, putted, err = optimism.PutOperation(etcdTestCli, false, op12c, 0) c.Assert(err, IsNil) c.Assert(putted, IsTrue) c.Assert(utils.WaitSomething(backOff, waitTime, func() bool { @@ -491,7 +491,7 @@ func (t *testOptimist) testOptimist(c *C, restart int) { // mark op21 as done. op21c := op21 op21c.Done = true - _, putted, err = optimism.PutOperation(etcdTestCli, false, op21c) + _, putted, err = optimism.PutOperation(etcdTestCli, false, op21c, 0) c.Assert(err, IsNil) c.Assert(putted, IsTrue) c.Assert(utils.WaitSomething(backOff, waitTime, func() bool { @@ -511,7 +511,7 @@ func (t *testOptimist) testOptimist(c *C, restart int) { // mark op23 as done. op23c := op23 op23c.Done = true - _, putted, err = optimism.PutOperation(etcdTestCli, false, op23c) + _, putted, err = optimism.PutOperation(etcdTestCli, false, op23c, 0) c.Assert(err, IsNil) c.Assert(putted, IsTrue) c.Assert(utils.WaitSomething(backOff, waitTime, func() bool { @@ -570,7 +570,7 @@ func (t *testOptimist) testOptimist(c *C, restart int) { // mark op31 as done. op31c := op31 op31c.Done = true - _, putted, err = optimism.PutOperation(etcdTestCli, false, op31c) + _, putted, err = optimism.PutOperation(etcdTestCli, false, op31c, 0) c.Assert(err, IsNil) c.Assert(putted, IsTrue) c.Assert(utils.WaitSomething(backOff, waitTime, func() bool { @@ -623,7 +623,7 @@ func (t *testOptimist) testOptimist(c *C, restart int) { // mark op33 as done, the lock should be resolved. op33c := op33 op33c.Done = true - _, putted, err = optimism.PutOperation(etcdTestCli, false, op33c) + _, putted, err = optimism.PutOperation(etcdTestCli, false, op33c, 0) c.Assert(err, IsNil) c.Assert(putted, IsTrue) c.Assert(utils.WaitSomething(backOff, waitTime, func() bool { @@ -866,12 +866,12 @@ func (t *testOptimist) TestOptimistLockMultipleTarget(c *C) { op11c := op12 op11c.Done = true op11c.UpTable = i11.UpTable // overwrite `UpTable`. - _, putted, err := optimism.PutOperation(etcdTestCli, false, op11c) + _, putted, err := optimism.PutOperation(etcdTestCli, false, op11c, 0) c.Assert(err, IsNil) c.Assert(putted, IsTrue) op12c := op12 op12c.Done = true - _, putted, err = optimism.PutOperation(etcdTestCli, false, op12c) + _, putted, err = optimism.PutOperation(etcdTestCli, false, op12c, 0) c.Assert(err, IsNil) c.Assert(putted, IsTrue) c.Assert(utils.WaitSomething(backOff, waitTime, func() bool { @@ -900,12 +900,12 @@ func (t *testOptimist) TestOptimistLockMultipleTarget(c *C) { op21c := op22 op21c.Done = true op21c.UpTable = i21.UpTable // overwrite `UpTable`. - _, putted, err = optimism.PutOperation(etcdTestCli, false, op21c) + _, putted, err = optimism.PutOperation(etcdTestCli, false, op21c, 0) c.Assert(err, IsNil) c.Assert(putted, IsTrue) op22c := op22 op22c.Done = true - _, putted, err = optimism.PutOperation(etcdTestCli, false, op22c) + _, putted, err = optimism.PutOperation(etcdTestCli, false, op22c, 0) c.Assert(err, IsNil) c.Assert(putted, IsTrue) c.Assert(utils.WaitSomething(backOff, waitTime, func() bool { @@ -999,12 +999,12 @@ func (t *testOptimist) TestOptimistInitSchema(c *C) { op11c := op12 op11c.Done = true op11c.UpTable = i11.UpTable // overwrite `UpTable`. - _, putted, err := optimism.PutOperation(etcdTestCli, false, op11c) + _, putted, err := optimism.PutOperation(etcdTestCli, false, op11c, 0) c.Assert(err, IsNil) c.Assert(putted, IsTrue) op12c := op12 op12c.Done = true - _, putted, err = optimism.PutOperation(etcdTestCli, false, op12c) + _, putted, err = optimism.PutOperation(etcdTestCli, false, op12c, 0) c.Assert(err, IsNil) c.Assert(putted, IsTrue) c.Assert(utils.WaitSomething(backOff, waitTime, func() bool { diff --git a/pkg/shardddl/optimism/column.go b/pkg/shardddl/optimism/column.go index c765e01c71..cac067a404 100644 --- a/pkg/shardddl/optimism/column.go +++ b/pkg/shardddl/optimism/column.go @@ -36,8 +36,12 @@ func GetAllDroppedColumns(cli *clientv3.Client) (map[string]map[string]interface if err != nil { return colm, 0, err } - lockID := keys[0] - column := keys[1] + task := keys[0] + downSchema := keys[1] + downTable := keys[2] + column := keys[3] + info := Info{Task: task, DownSchema: downSchema, DownTable: downTable} + lockID := genDDLLockID(info) if _, ok := colm[lockID]; !ok { colm[lockID] = make(map[string]interface{}) } @@ -48,8 +52,8 @@ func GetAllDroppedColumns(cli *clientv3.Client) (map[string]map[string]interface } // PutDroppedColumn puts the undropped column name into ectd. -func PutDroppedColumn(cli *clientv3.Client, lockID, column string) (rev int64, putted bool, err error) { - key := common.ShardDDLOptimismDroppedColumnsKeyAdapter.Encode(lockID, column) +func PutDroppedColumn(cli *clientv3.Client, task, downSchema, downTable, column string) (rev int64, putted bool, err error) { + key := common.ShardDDLOptimismDroppedColumnsKeyAdapter.Encode(task, downSchema, downTable, column) op := clientv3.OpPut(key, "") @@ -61,10 +65,10 @@ func PutDroppedColumn(cli *clientv3.Client, lockID, column string) (rev int64, p } // DeleteDroppedColumns tries to delete the dropped columns for the specified lock ID. -func DeleteDroppedColumns(cli *clientv3.Client, lockID string, columns ...string) (rev int64, deleted bool, err error) { +func DeleteDroppedColumns(cli *clientv3.Client, task, downSchema, downTable string, columns ...string) (rev int64, deleted bool, err error) { ops := make([]clientv3.Op, 0, len(columns)) for _, col := range columns { - ops = append(ops, deleteDroppedColumnOp(lockID, col)) + ops = append(ops, deleteDroppedColumnOp(task, downSchema, downTable, col)) } resp, rev, err := etcdutil.DoOpsInOneTxnWithRetry(cli, ops...) if err != nil { @@ -74,6 +78,11 @@ func DeleteDroppedColumns(cli *clientv3.Client, lockID string, columns ...string } // deleteDroppedColumnOp returns a DELETE etcd operation for init schema. -func deleteDroppedColumnOp(lockID, column string) clientv3.Op { - return clientv3.OpDelete(common.ShardDDLOptimismDroppedColumnsKeyAdapter.Encode(lockID, column)) +func deleteDroppedColumnOp(task, downSchema, downTable, column string) clientv3.Op { + return clientv3.OpDelete(common.ShardDDLOptimismDroppedColumnsKeyAdapter.Encode(task, downSchema, downTable, column)) +} + +// deleteDroppedColumnOp returns a DELETE etcd operation for init schema. +func deleteDroppedColumnsOp(task, downSchema, downTable string) clientv3.Op { + return clientv3.OpDelete(common.ShardDDLOptimismDroppedColumnsKeyAdapter.Encode(task, downSchema, downTable), clientv3.WithPrefix()) } diff --git a/pkg/shardddl/optimism/column_test.go b/pkg/shardddl/optimism/column_test.go index 656ee9922a..3e8910f00d 100644 --- a/pkg/shardddl/optimism/column_test.go +++ b/pkg/shardddl/optimism/column_test.go @@ -22,11 +22,16 @@ var _ = Suite(&testColumn{}) func (t *testColumn) TestColumnETCD(c *C) { defer clearTestInfoOperation(c) - lockID := "test-`shardddl`.`tb`" - rev1, putted, err := PutDroppedColumn(etcdTestCli, lockID, "a") + var ( + task = "test" + downSchema = "shardddl" + downTable = "tb" + lockID = "test-`shardddl`.`tb`" + ) + rev1, putted, err := PutDroppedColumn(etcdTestCli, task, downSchema, downTable, "a") c.Assert(err, IsNil) c.Assert(putted, IsTrue) - rev2, putted, err := PutDroppedColumn(etcdTestCli, lockID, "b") + rev2, putted, err := PutDroppedColumn(etcdTestCli, task, downSchema, downTable, "b") c.Assert(err, IsNil) c.Assert(putted, IsTrue) c.Assert(rev2, Greater, rev1) @@ -41,7 +46,7 @@ func (t *testColumn) TestColumnETCD(c *C) { c.Assert(colm, DeepEquals, expectedColm) c.Assert(rev3, Equals, rev2) - rev4, deleted, err := DeleteDroppedColumns(etcdTestCli, lockID, "b") + rev4, deleted, err := DeleteDroppedColumns(etcdTestCli, task, downSchema, downTable, "b") c.Assert(err, IsNil) c.Assert(deleted, IsTrue) c.Assert(rev4, Greater, rev3) diff --git a/pkg/shardddl/optimism/info.go b/pkg/shardddl/optimism/info.go index d0c715bc7a..23b0319888 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:"-"` + + // only set it when get/watch from etcd + ModRevision int64 `json:"-"` } // NewInfo creates a new Info instance. @@ -129,6 +132,7 @@ func GetAllInfo(cli *clientv3.Client) (map[string]map[string]map[string]map[stri return nil, 0, err2 } info.Version = kv.Version + info.ModRevision = kv.ModRevision if _, ok := ifm[info.Task]; !ok { ifm[info.Task] = make(map[string]map[string]map[string]Info) @@ -179,6 +183,7 @@ func WatchInfo(ctx context.Context, cli *clientv3.Client, revision int64, case mvccpb.PUT: info, err = infoFromJSON(string(ev.Kv.Value)) info.Version = ev.Kv.Version + info.ModRevision = ev.Kv.ModRevision case mvccpb.DELETE: info, err = infoFromJSON(string(ev.PrevKv.Value)) info.IsDeleted = true diff --git a/pkg/shardddl/optimism/info_test.go b/pkg/shardddl/optimism/info_test.go index 37d9dadb06..6831c53da9 100644 --- a/pkg/shardddl/optimism/info_test.go +++ b/pkg/shardddl/optimism/info_test.go @@ -128,6 +128,7 @@ func (t *testForEtcd) TestInfoEtcd(c *C) { c.Assert(ifm[task1][source1][upSchema], HasLen, 1) i11WithVer := i11 i11WithVer.Version = 2 + i11WithVer.ModRevision = rev2 c.Assert(ifm[task1][source1][upSchema][upTable], DeepEquals, i11WithVer) // put another key and get again with 2 info. @@ -141,6 +142,7 @@ func (t *testForEtcd) TestInfoEtcd(c *C) { c.Assert(ifm[task1][source1][upSchema][upTable], DeepEquals, i11WithVer) i12WithVer := i12 i12WithVer.Version = 1 + i12WithVer.ModRevision = rev4 c.Assert(ifm[task1][source2][upSchema][upTable], DeepEquals, i12WithVer) // start the watcher. @@ -157,20 +159,22 @@ func (t *testForEtcd) TestInfoEtcd(c *C) { // put another key for a different task. // version start from 1 - _, err = PutInfo(etcdTestCli, i21) + rev5, err := PutInfo(etcdTestCli, i21) c.Assert(err, IsNil) infoWithVer := <-wch i21WithVer := i21 i21WithVer.Version = 1 + i21WithVer.ModRevision = rev5 c.Assert(infoWithVer, DeepEquals, i21WithVer) c.Assert(len(ech), Equals, 0) // put again // version increase - _, err = PutInfo(etcdTestCli, i21) + rev6, err := PutInfo(etcdTestCli, i21) c.Assert(err, IsNil) infoWithVer = <-wch i21WithVer.Version++ + i21WithVer.ModRevision = rev6 c.Assert(infoWithVer, DeepEquals, i21WithVer) c.Assert(len(ech), Equals, 0) @@ -187,10 +191,11 @@ func (t *testForEtcd) TestInfoEtcd(c *C) { // put again // version reset to 1 - _, err = PutInfo(etcdTestCli, i21) + rev7, err := PutInfo(etcdTestCli, i21) c.Assert(err, IsNil) infoWithVer = <-wch i21WithVer.Version = 1 + i21WithVer.ModRevision = rev7 c.Assert(infoWithVer, DeepEquals, i21WithVer) c.Assert(len(ech), Equals, 0) diff --git a/pkg/shardddl/optimism/lock.go b/pkg/shardddl/optimism/lock.go index f83fabbc4a..a850aca0d4 100644 --- a/pkg/shardddl/optimism/lock.go +++ b/pkg/shardddl/optimism/lock.go @@ -518,8 +518,9 @@ func (l *Lock) AddDroppedColumn(col string) error { if _, ok := l.columns[col]; ok { return nil } + log.L().Debug("add not fully dropped columns", zap.String("lockID", l.ID), zap.String("column", col)) - _, _, err := PutDroppedColumn(l.cli, l.ID, col) + _, _, err := PutDroppedColumn(l.cli, l.Task, l.DownSchema, l.DownTable, col) if err != nil { return err } @@ -542,12 +543,17 @@ func (l *Lock) DeleteColumnsByDDLs(ddls []string) error { colsToDelete = append(colsToDelete, col) } } - _, _, err := DeleteDroppedColumns(l.cli, l.ID, colsToDelete...) - if err != nil { - return err - } - for _, col := range colsToDelete { - delete(l.columns, col) + if len(colsToDelete) > 0 { + log.L().Debug("delete not fully dropped columns", + zap.String("lockID", l.ID), zap.Strings("columns", colsToDelete)) + + _, _, err := DeleteDroppedColumns(l.cli, l.Task, l.DownSchema, l.DownTable, colsToDelete...) + if err != nil { + return err + } + for _, col := range colsToDelete { + delete(l.columns, col) + } } return nil diff --git a/pkg/shardddl/optimism/operation.go b/pkg/shardddl/optimism/operation.go index 645cd2783e..41dfd9de45 100644 --- a/pkg/shardddl/optimism/operation.go +++ b/pkg/shardddl/optimism/operation.go @@ -96,7 +96,7 @@ func operationFromJSON(s string) (o Operation, err error) { } // PutOperation puts the shard DDL operation into etcd. -func PutOperation(cli *clientv3.Client, skipDone bool, op Operation) (rev int64, putted bool, err error) { +func PutOperation(cli *clientv3.Client, skipDone bool, op Operation, infoModRev int64) (rev int64, putted bool, err error) { value, err := op.toJSON() if err != nil { return 0, false, err @@ -106,6 +106,7 @@ func PutOperation(cli *clientv3.Client, skipDone bool, op Operation) (rev int64, cmpsNotExist := make([]clientv3.Cmp, 0, 1) cmpsNotDone := make([]clientv3.Cmp, 0, 1) + cmpsLessRev := make([]clientv3.Cmp, 0, 1) if skipDone { opDone := op opDone.Done = true // set `done` to `true`. @@ -115,6 +116,7 @@ func PutOperation(cli *clientv3.Client, skipDone bool, op Operation) (rev int64, } cmpsNotExist = append(cmpsNotExist, clientv3util.KeyMissing(key)) cmpsNotDone = append(cmpsNotDone, clientv3.Compare(clientv3.Value(key), "!=", valueDone)) + cmpsLessRev = append(cmpsLessRev, clientv3.Compare(clientv3.ModRevision(key), "<", infoModRev)) } ctx, cancel := context.WithTimeout(cli.Ctx(), etcdutil.DefaultRequestTimeout) @@ -130,6 +132,20 @@ func PutOperation(cli *clientv3.Client, skipDone bool, op Operation) (rev int64, // txn 2: try to PUT if the key "the `done`" field is not `true`. resp, err = cli.Txn(ctx).If(cmpsNotDone...).Then(opPut).Commit() + if err != nil { + return 0, false, err + } else if resp.Succeeded { + return resp.Header.Revision, resp.Succeeded, nil + } + + // txn 3: try to PUT if the key has less mod revision than info's mod revision, which means this operation is an old one + // without this, failed case time series: + // 1. dm-master received an old done DDL operation from dm-worker + // 2. dm-worker putted a new DDL info into dm-master + // 3. dm-master quited before dm-master putted the DDL operation to dm-worker + // 4. dm-master restarted and tried to put DDL operation, but found a done one and failed to put + // 5. dm-worker didn't receive a DDL operation, will get blocked forever + resp, err = cli.Txn(ctx).If(cmpsLessRev...).Then(opPut).Commit() if err != nil { return 0, false, err } diff --git a/pkg/shardddl/optimism/operation_test.go b/pkg/shardddl/optimism/operation_test.go index 116b12a4af..c223d484b8 100644 --- a/pkg/shardddl/optimism/operation_test.go +++ b/pkg/shardddl/optimism/operation_test.go @@ -53,10 +53,10 @@ func (t *testForEtcd) TestOperationEtcd(c *C) { ) // put the same keys twice. - rev1, succ, err := PutOperation(etcdTestCli, false, op11) + rev1, succ, err := PutOperation(etcdTestCli, false, op11, 0) c.Assert(err, IsNil) c.Assert(succ, IsTrue) - rev2, succ, err := PutOperation(etcdTestCli, false, op11) + rev2, succ, err := PutOperation(etcdTestCli, false, op11, 0) c.Assert(err, IsNil) c.Assert(succ, IsTrue) c.Assert(rev2, Greater, rev1) @@ -76,7 +76,7 @@ func (t *testForEtcd) TestOperationEtcd(c *C) { c.Assert(<-wch, DeepEquals, op11) // put for another task. - rev3, succ, err := PutOperation(etcdTestCli, false, op21) + rev3, succ, err := PutOperation(etcdTestCli, false, op21, 0) c.Assert(err, IsNil) c.Assert(succ, IsTrue) @@ -109,7 +109,7 @@ func (t *testForEtcd) TestOperationEtcd(c *C) { // put for `skipDone` with `done` in etcd, the operations should not be skipped. // case: kv's "the `done` field is not `true`". - rev5, succ, err := PutOperation(etcdTestCli, true, op11) + rev5, succ, err := PutOperation(etcdTestCli, true, op11, 0) c.Assert(err, IsNil) c.Assert(succ, IsTrue) c.Assert(rev5, Greater, rev4) @@ -126,7 +126,7 @@ func (t *testForEtcd) TestOperationEtcd(c *C) { // put for `skipDone` with `done` in etcd, the operations should not be skipped. // case: kv "not exist". - rev6, succ, err := PutOperation(etcdTestCli, true, op11) + rev6, succ, err := PutOperation(etcdTestCli, true, op11, 0) c.Assert(err, IsNil) c.Assert(succ, IsTrue) @@ -139,15 +139,22 @@ func (t *testForEtcd) TestOperationEtcd(c *C) { // update op11 to `done`. op11c := op11 op11c.Done = true - rev7, succ, err := PutOperation(etcdTestCli, true, op11c) + rev7, succ, err := PutOperation(etcdTestCli, true, op11c, 0) c.Assert(err, IsNil) c.Assert(succ, IsTrue) c.Assert(rev7, Greater, rev6) + // put for `skipDone` with `done` in etcd, the operations should not be skipped. + // case: operation modRevision < info's modRevision + rev8, succ, err := PutOperation(etcdTestCli, true, op11c, rev7+10) + c.Assert(err, IsNil) + c.Assert(succ, IsTrue) + c.Assert(rev8, Greater, rev7) + // put for `skipDone` with `done` in etcd, the operations should be skipped. // case: kv's ("exist" and "the `done` field is `true`"). - rev8, succ, err := PutOperation(etcdTestCli, true, op11) + rev9, succ, err := PutOperation(etcdTestCli, true, op11, rev6) c.Assert(err, IsNil) c.Assert(succ, IsFalse) - c.Assert(rev8, Equals, rev7) + c.Assert(rev9, Equals, rev8) } diff --git a/pkg/shardddl/optimism/ops.go b/pkg/shardddl/optimism/ops.go index b855656b31..56478887f1 100644 --- a/pkg/shardddl/optimism/ops.go +++ b/pkg/shardddl/optimism/ops.go @@ -47,10 +47,10 @@ func PutSourceTablesDeleteInfo(cli *clientv3.Client, st SourceTables, info Info) return rev, err } -// DeleteInfosOperationsSchema deletes the shard DDL infos, operations and init schemas in etcd. +// DeleteInfosOperationsSchemaColumn deletes the shard DDL infos, operations, init schemas and dropped columns in etcd. // This function should often be called by DM-master when removing the lock. // Only delete when all info's version are greater or equal to etcd's version, otherwise it means new info was putted into etcd before. -func DeleteInfosOperationsSchema(cli *clientv3.Client, infos []Info, ops []Operation, schema InitSchema) (int64, bool, error) { +func DeleteInfosOperationsSchemaColumn(cli *clientv3.Client, infos []Info, ops []Operation, schema InitSchema) (int64, bool, error) { opsDel := make([]clientv3.Op, 0, len(infos)+len(ops)) cmps := make([]clientv3.Cmp, 0, len(infos)) for _, info := range infos { @@ -62,6 +62,7 @@ func DeleteInfosOperationsSchema(cli *clientv3.Client, infos []Info, ops []Opera opsDel = append(opsDel, deleteOperationOp(op)) } opsDel = append(opsDel, deleteInitSchemaOp(schema.Task, schema.DownSchema, schema.DownTable)) + opsDel = append(opsDel, deleteDroppedColumnsOp(schema.Task, schema.DownSchema, schema.DownTable)) resp, rev, err := etcdutil.DoOpsInOneCmpsTxnWithRetry(cli, cmps, opsDel, []clientv3.Op{}) if err != nil { return 0, false, err @@ -76,6 +77,7 @@ func DeleteInfosOperationsTablesSchemasByTask(cli *clientv3.Client, task string) opsDel = append(opsDel, clientv3.OpDelete(common.ShardDDLOptimismOperationKeyAdapter.Encode(task), clientv3.WithPrefix())) opsDel = append(opsDel, clientv3.OpDelete(common.ShardDDLOptimismSourceTablesKeyAdapter.Encode(task), clientv3.WithPrefix())) opsDel = append(opsDel, clientv3.OpDelete(common.ShardDDLOptimismInitSchemaKeyAdapter.Encode(task), clientv3.WithPrefix())) + opsDel = append(opsDel, clientv3.OpDelete(common.ShardDDLOptimismDroppedColumnsKeyAdapter.Encode(task), clientv3.WithPrefix())) _, rev, err := etcdutil.DoOpsInOneTxnWithRetry(cli, opsDel...) return rev, err } diff --git a/pkg/shardddl/optimism/ops_test.go b/pkg/shardddl/optimism/ops_test.go index f6f4794e81..804747f18c 100644 --- a/pkg/shardddl/optimism/ops_test.go +++ b/pkg/shardddl/optimism/ops_test.go @@ -34,17 +34,18 @@ func (t *testForEtcd) TestDeleteInfosOperationsSchema(c *C) { ) // put info. - _, err := PutInfo(etcdTestCli, info) + rev, err := PutInfo(etcdTestCli, info) c.Assert(err, IsNil) ifm, _, err := GetAllInfo(etcdTestCli) c.Assert(err, IsNil) c.Assert(ifm, HasLen, 1) infoWithVer := info infoWithVer.Version = 1 + infoWithVer.ModRevision = rev c.Assert(ifm[task][source][upSchema][upTable], DeepEquals, infoWithVer) // put operation. - _, _, err = PutOperation(etcdTestCli, false, op) + _, _, err = PutOperation(etcdTestCli, false, op, 0) c.Assert(err, IsNil) opm, _, err := GetAllOperations(etcdTestCli) c.Assert(err, IsNil) @@ -59,7 +60,7 @@ func (t *testForEtcd) TestDeleteInfosOperationsSchema(c *C) { c.Assert(isc, DeepEquals, is) // DELETE info and operation with version 0 - _, deleted, err := DeleteInfosOperationsSchema(etcdTestCli, []Info{info}, []Operation{op}, is) + _, deleted, err := DeleteInfosOperationsSchemaColumn(etcdTestCli, []Info{info}, []Operation{op}, is) c.Assert(err, IsNil) c.Assert(deleted, IsFalse) @@ -75,7 +76,7 @@ func (t *testForEtcd) TestDeleteInfosOperationsSchema(c *C) { c.Assert(isc.IsEmpty(), IsFalse) // DELETE info and operation with version 1 - _, deleted, err = DeleteInfosOperationsSchema(etcdTestCli, []Info{infoWithVer}, []Operation{op}, is) + _, deleted, err = DeleteInfosOperationsSchemaColumn(etcdTestCli, []Info{infoWithVer}, []Operation{op}, is) c.Assert(err, IsNil) c.Assert(deleted, IsTrue) @@ -133,6 +134,7 @@ func (t *testForEtcd) TestSourceTablesInfo(c *C) { c.Assert(ifm[task][source][upSchema], HasLen, 1) i11WithVer := i11 i11WithVer.Version = 1 + i11WithVer.ModRevision = rev3 c.Assert(ifm[task][source][upSchema][upTable], DeepEquals, i11WithVer) // put/update source tables and delete info. diff --git a/syncer/shardddl/optimist.go b/syncer/shardddl/optimist.go index afb7bd9a56..9a8bba0f39 100644 --- a/syncer/shardddl/optimist.go +++ b/syncer/shardddl/optimist.go @@ -148,7 +148,7 @@ func (o *Optimist) GetOperation(ctx context.Context, info optimism.Info, rev int // DoneOperation marks the shard DDL lock operation as done. func (o *Optimist) DoneOperation(op optimism.Operation) error { op.Done = true - _, _, err := optimism.PutOperation(o.cli, false, op) + _, _, err := optimism.PutOperation(o.cli, false, op, 0) if err != nil { return err } diff --git a/syncer/shardddl/optimist_test.go b/syncer/shardddl/optimist_test.go index 689f2d884f..0700f3d74e 100644 --- a/syncer/shardddl/optimist_test.go +++ b/syncer/shardddl/optimist_test.go @@ -119,7 +119,7 @@ func (t *testOptimist) TestOptimist(c *C) { c.Assert(*info1c, DeepEquals, info1) // put the lock operation. - rev2, putted, err := optimism.PutOperation(etcdTestCli, false, op1) + rev2, putted, err := optimism.PutOperation(etcdTestCli, false, op1, rev1) c.Assert(err, IsNil) c.Assert(rev2, Greater, rev1) c.Assert(putted, IsTrue) @@ -146,6 +146,7 @@ func (t *testOptimist) TestOptimist(c *C) { c.Assert(ifm[task][source][info1.UpSchema], HasLen, 1) info1WithVer := info1 info1WithVer.Version = 1 + info1WithVer.ModRevision = rev1 c.Assert(ifm[task][source][info1.UpSchema][info1.UpTable], DeepEquals, info1WithVer) opc := op1c opc.Done = true @@ -169,6 +170,7 @@ func (t *testOptimist) TestOptimist(c *C) { c.Assert(err, IsNil) infoCreateWithVer := infoCreate infoCreateWithVer.Version = 1 + infoCreateWithVer.ModRevision = rev3 c.Assert(ifm[task][source][infoCreate.UpSchema][infoCreate.UpTable], DeepEquals, infoCreateWithVer) c.Assert(o.tables.Tables[infoCreate.DownSchema][infoCreate.DownTable][infoCreate.UpSchema], HasKey, infoCreate.UpTable) @@ -189,7 +191,7 @@ func (t *testOptimist) TestOptimist(c *C) { c.Assert(o.PendingOperation(), IsNil) // put another lock operation. - rev6, putted, err := optimism.PutOperation(etcdTestCli, false, op2) + rev6, putted, err := optimism.PutOperation(etcdTestCli, false, op2, rev5) c.Assert(err, IsNil) c.Assert(rev6, Greater, rev5) c.Assert(putted, IsTrue) diff --git a/tests/shardddl3/run.sh b/tests/shardddl3/run.sh index 7bc079f43c..1521ef4cc6 100644 --- a/tests/shardddl3/run.sh +++ b/tests/shardddl3/run.sh @@ -1074,6 +1074,7 @@ function DM_DropAddColumn_CASE() { function DM_DropAddColumn() { for i in `seq 0 5` do + echo "run DM_DropAddColumn case #${i}" run_case DropAddColumn "double-source-optimistic" \ "run_sql_source1 \"create table ${shardddl1}.${tb1} (a int primary key, b int, c int);\"; \ run_sql_source2 \"create table ${shardddl1}.${tb1} (a int primary key, b int, c int);\"" \ From cc296521fae2962c24a1bc436c1d5fa38621472b Mon Sep 17 00:00:00 2001 From: Chunzhu Li Date: Thu, 18 Mar 2021 11:04:37 +0800 Subject: [PATCH 06/23] fix unit test and integration test --- dm/common/common.go | 9 ++-- dm/master/shardddl/optimist.go | 2 +- go.sum | 1 + pkg/shardddl/optimism/column.go | 41 ++++++++++----- pkg/shardddl/optimism/column_test.go | 44 +++++++++++----- pkg/shardddl/optimism/keeper.go | 9 ++-- pkg/shardddl/optimism/lock.go | 62 ++++++++++++++++++----- pkg/shardddl/optimism/ops.go | 4 +- tests/sequence_sharding_optimistic/run.sh | 4 +- 9 files changed, 124 insertions(+), 52 deletions(-) diff --git a/dm/common/common.go b/dm/common/common.go index 5d17145eda..2d85ca7931 100644 --- a/dm/common/common.go +++ b/dm/common/common.go @@ -80,8 +80,8 @@ var ( // k/v: Encode(task-name, downstream-schema-name, downstream-table-name) -> table schema. ShardDDLOptimismInitSchemaKeyAdapter KeyAdapter = keyHexEncoderDecoder("/dm-master/shardddl-optimism/init-schema/") // ShardDDLOptimismDroppedColumnsKeyAdapter is used to store the columns that are not fully dropped - // k/v: Encode(task-name, downstream-schema-name, downstream-table-name, column-name) -> empty - ShardDDLOptimismDroppedColumnsKeyAdapter KeyAdapter = keyHexEncoderDecoder("/dm-master/shardddl-optimism/undropped-columns/") + // k/v: Encode(task-name, downstream-schema-name, downstream-table-name, column-name, source-id, upstream-schema-name, upstream-table-name) -> empty + ShardDDLOptimismDroppedColumnsKeyAdapter KeyAdapter = keyHexEncoderDecoder("/dm-master/shardddl-optimism/dropped-columns/") ) func keyAdapterKeysLen(s KeyAdapter) int { @@ -96,9 +96,10 @@ func keyAdapterKeysLen(s KeyAdapter) int { return 2 case ShardDDLOptimismInitSchemaKeyAdapter: return 3 - case ShardDDLOptimismInfoKeyAdapter, ShardDDLOptimismOperationKeyAdapter, ShardDDLOptimismDroppedColumnsKeyAdapter: + case ShardDDLOptimismInfoKeyAdapter, ShardDDLOptimismOperationKeyAdapter: return 4 - + case ShardDDLOptimismDroppedColumnsKeyAdapter: + return 7 } return -1 } diff --git a/dm/master/shardddl/optimist.go b/dm/master/shardddl/optimist.go index 09f8b4bdbc..81f25a1386 100644 --- a/dm/master/shardddl/optimist.go +++ b/dm/master/shardddl/optimist.go @@ -267,7 +267,7 @@ func (o *Optimist) rebuildLocks() (revSource, revInfo, revOperation int64, err e func (o *Optimist) recoverLocks( ifm map[string]map[string]map[string]map[string]optimism.Info, opm map[string]map[string]map[string]map[string]optimism.Operation, - colm map[string]map[string]interface{}) error { + colm map[string]map[string]map[string]map[string]map[string]interface{}) error { // construct locks based on the shard DDL info. o.lk.SetColumnMap(colm) defer o.lk.SetColumnMap(nil) diff --git a/go.sum b/go.sum index 40693e7455..e174fb820a 100644 --- a/go.sum +++ b/go.sum @@ -188,6 +188,7 @@ github.com/coocood/rtutil v0.0.0-20190304133409-c84515f646f2/go.mod h1:7qG7YFnOA 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 h1:pAWNwdf7QiT1zfaWyqCtNZQWCLByQyA3JrSQyuYAqnQ= 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= diff --git a/pkg/shardddl/optimism/column.go b/pkg/shardddl/optimism/column.go index cac067a404..347a585a46 100644 --- a/pkg/shardddl/optimism/column.go +++ b/pkg/shardddl/optimism/column.go @@ -21,8 +21,8 @@ import ( ) // GetAllDroppedColumns gets the all dropped columns. -func GetAllDroppedColumns(cli *clientv3.Client) (map[string]map[string]interface{}, int64, error) { - colm := make(map[string]map[string]interface{}) +func GetAllDroppedColumns(cli *clientv3.Client) (map[string]map[string]map[string]map[string]map[string]interface{}, int64, error) { + colm := make(map[string]map[string]map[string]map[string]map[string]interface{}) op := clientv3.OpGet(common.ShardDDLOptimismDroppedColumnsKeyAdapter.Path(), clientv3.WithPrefix()) respTxn, rev, err := etcdutil.DoOpsInOneTxnWithRetry(cli, op) if err != nil { @@ -40,20 +40,33 @@ func GetAllDroppedColumns(cli *clientv3.Client) (map[string]map[string]interface downSchema := keys[1] downTable := keys[2] column := keys[3] - info := Info{Task: task, DownSchema: downSchema, DownTable: downTable} + source := keys[4] + upSchema := keys[5] + upTable := keys[6] + info := NewInfo(task, source, upSchema, upTable, downSchema, downTable, nil, nil, nil) lockID := genDDLLockID(info) if _, ok := colm[lockID]; !ok { - colm[lockID] = make(map[string]interface{}) + colm[lockID] = make(map[string]map[string]map[string]map[string]interface{}) } - colm[lockID][column] = struct{}{} + if _, ok := colm[lockID][column]; !ok { + colm[lockID][column] = make(map[string]map[string]map[string]interface{}) + } + if _, ok := colm[lockID][column][source]; !ok { + colm[lockID][column][source] = make(map[string]map[string]interface{}) + } + if _, ok := colm[lockID][column][source][downSchema]; !ok { + colm[lockID][column][source][downSchema] = make(map[string]interface{}) + } + colm[lockID][column][source][downSchema][downTable] = struct{}{} } } return colm, rev, nil } // PutDroppedColumn puts the undropped column name into ectd. -func PutDroppedColumn(cli *clientv3.Client, task, downSchema, downTable, column string) (rev int64, putted bool, err error) { - key := common.ShardDDLOptimismDroppedColumnsKeyAdapter.Encode(task, downSchema, downTable, column) +func PutDroppedColumn(cli *clientv3.Client, info Info, column string) (rev int64, putted bool, err error) { + key := common.ShardDDLOptimismDroppedColumnsKeyAdapter.Encode( + info.Task, info.DownSchema, info.DownTable, column, info.Source, info.UpSchema, info.UpTable) op := clientv3.OpPut(key, "") @@ -68,7 +81,7 @@ func PutDroppedColumn(cli *clientv3.Client, task, downSchema, downTable, column func DeleteDroppedColumns(cli *clientv3.Client, task, downSchema, downTable string, columns ...string) (rev int64, deleted bool, err error) { ops := make([]clientv3.Op, 0, len(columns)) for _, col := range columns { - ops = append(ops, deleteDroppedColumnOp(task, downSchema, downTable, col)) + ops = append(ops, deleteDroppedColumnByColumnOp(task, downSchema, downTable, col)) } resp, rev, err := etcdutil.DoOpsInOneTxnWithRetry(cli, ops...) if err != nil { @@ -77,12 +90,12 @@ func DeleteDroppedColumns(cli *clientv3.Client, task, downSchema, downTable stri return rev, resp.Succeeded, nil } -// deleteDroppedColumnOp returns a DELETE etcd operation for init schema. -func deleteDroppedColumnOp(task, downSchema, downTable, column string) clientv3.Op { - return clientv3.OpDelete(common.ShardDDLOptimismDroppedColumnsKeyAdapter.Encode(task, downSchema, downTable, column)) +// deleteDroppedColumnOp returns a DELETE etcd operation for the specified task and column name. +func deleteDroppedColumnByColumnOp(task, downSchema, downTable, column string) clientv3.Op { + return clientv3.OpDelete(common.ShardDDLOptimismDroppedColumnsKeyAdapter.Encode(task, downSchema, downTable, column), clientv3.WithPrefix()) } -// deleteDroppedColumnOp returns a DELETE etcd operation for init schema. -func deleteDroppedColumnsOp(task, downSchema, downTable string) clientv3.Op { - return clientv3.OpDelete(common.ShardDDLOptimismDroppedColumnsKeyAdapter.Encode(task, downSchema, downTable), clientv3.WithPrefix()) +// deleteDroppedColumnsByLockOp returns a DELETE etcd operation for the specified task. +func deleteDroppedColumnsByLockOp(task, downSchema, downTable string) clientv3.Op { + return clientv3.OpDelete(common.ShardDDLOptimismDroppedColumnsKeyAdapter.Encode(task), clientv3.WithPrefix()) } diff --git a/pkg/shardddl/optimism/column_test.go b/pkg/shardddl/optimism/column_test.go index 3e8910f00d..8ffc124248 100644 --- a/pkg/shardddl/optimism/column_test.go +++ b/pkg/shardddl/optimism/column_test.go @@ -26,34 +26,54 @@ func (t *testColumn) TestColumnETCD(c *C) { task = "test" downSchema = "shardddl" downTable = "tb" - lockID = "test-`shardddl`.`tb`" + source1 = "mysql-replica-1" + source2 = "mysql-replica-2" + upSchema1 = "shardddl1" + upTable1 = "tb1" + upSchema2 = "shardddl2" + upTable2 = "tb2" + info1 = NewInfo(task, source1, upSchema1, upTable1, downSchema, downTable, nil, nil, nil) + info2 = NewInfo(task, source1, upSchema2, upTable2, downSchema, downTable, nil, nil, nil) + info3 = NewInfo(task, source2, upSchema1, upTable1, downSchema, downTable, nil, nil, nil) + lockID = genDDLLockID(info1) ) - rev1, putted, err := PutDroppedColumn(etcdTestCli, task, downSchema, downTable, "a") + rev1, putted, err := PutDroppedColumn(etcdTestCli, info1, "a") c.Assert(err, IsNil) c.Assert(putted, IsTrue) - rev2, putted, err := PutDroppedColumn(etcdTestCli, task, downSchema, downTable, "b") + rev2, putted, err := PutDroppedColumn(etcdTestCli, info1, "b") c.Assert(err, IsNil) c.Assert(putted, IsTrue) c.Assert(rev2, Greater, rev1) + rev3, putted, err := PutDroppedColumn(etcdTestCli, info2, "b") + c.Assert(err, IsNil) + c.Assert(putted, IsTrue) + c.Assert(rev3, Greater, rev2) + rev4, putted, err := PutDroppedColumn(etcdTestCli, info3, "b") + c.Assert(err, IsNil) + c.Assert(putted, IsTrue) + c.Assert(rev4, Greater, rev3) - expectedColm := map[string]map[string]interface{}{ + expectedColm := map[string]map[string]map[string]map[string]map[string]interface{}{ lockID: { - "a": struct{}{}, - "b": struct{}{}}, + "a": {source1: {upSchema1: {upTable1: struct{}{}}}}, + "b": {source1: {upSchema1: {upTable1: struct{}{}}, + upSchema2: {upTable2: struct{}{}}}, + source2: {upSchema1: {upTable1: struct{}{}}}}, + }, } - colm, rev3, err := GetAllDroppedColumns(etcdTestCli) + colm, rev5, err := GetAllDroppedColumns(etcdTestCli) c.Assert(err, IsNil) c.Assert(colm, DeepEquals, expectedColm) - c.Assert(rev3, Equals, rev2) + c.Assert(rev5, Equals, rev4) - rev4, deleted, err := DeleteDroppedColumns(etcdTestCli, task, downSchema, downTable, "b") + rev6, deleted, err := DeleteDroppedColumns(etcdTestCli, task, downSchema, downTable, "b") c.Assert(err, IsNil) c.Assert(deleted, IsTrue) - c.Assert(rev4, Greater, rev3) + c.Assert(rev6, Greater, rev5) delete(expectedColm[lockID], "b") - colm, rev5, err := GetAllDroppedColumns(etcdTestCli) + colm, rev7, err := GetAllDroppedColumns(etcdTestCli) c.Assert(err, IsNil) c.Assert(colm, DeepEquals, expectedColm) - c.Assert(rev5, Equals, rev4) + c.Assert(rev7, Equals, rev6) } diff --git a/pkg/shardddl/optimism/keeper.go b/pkg/shardddl/optimism/keeper.go index 067ac7b81d..5e8effcf34 100644 --- a/pkg/shardddl/optimism/keeper.go +++ b/pkg/shardddl/optimism/keeper.go @@ -26,8 +26,9 @@ import ( // The lock information do not need to be persistent, and can be re-constructed from the shard DDL info. type LockKeeper struct { mu sync.RWMutex - locks map[string]*Lock // lockID -> Lock - colm map[string]map[string]interface{} // lockID -> not fully dropped column name -> interface{} + locks map[string]*Lock // lockID -> Lock + // lockID -> column name -> source -> upSchema -> upTable -> interface{} + colm map[string]map[string]map[string]map[string]map[string]interface{} } // NewLockKeeper creates a new LockKeeper instance. @@ -58,7 +59,7 @@ func (lk *LockKeeper) TrySync(cli *clientv3.Client, info Info, tts []TargetTable } } - newDDLs, err := l.TrySync(info.Source, info.UpSchema, info.UpTable, info.DDLs, info.TableInfosAfter, tts, info.Version) + newDDLs, err := l.TrySync(info, tts) return lockID, newDDLs, err } @@ -112,7 +113,7 @@ func (lk *LockKeeper) Clear() { } // SetColumnMap sets the column map received from etcd -func (lk *LockKeeper) SetColumnMap(colm map[string]map[string]interface{}) { +func (lk *LockKeeper) SetColumnMap(colm map[string]map[string]map[string]map[string]map[string]interface{}) { lk.mu.Lock() defer lk.mu.Unlock() diff --git a/pkg/shardddl/optimism/lock.go b/pkg/shardddl/optimism/lock.go index a850aca0d4..f55b2d6456 100644 --- a/pkg/shardddl/optimism/lock.go +++ b/pkg/shardddl/optimism/lock.go @@ -61,8 +61,8 @@ type Lock struct { versions map[string]map[string]map[string]int64 // record the partially dropped columns - // column name -> interface{} - columns map[string]interface{} + // column name -> source -> upSchema -> upTable -> interface{} + columns map[string]map[string]map[string]map[string]interface{} } // NewLock creates a new Lock instance. @@ -79,7 +79,7 @@ func NewLock(cli *clientv3.Client, ID, task, downSchema, downTable string, ti *m done: make(map[string]map[string]map[string]bool), synced: true, versions: make(map[string]map[string]map[string]int64), - columns: make(map[string]interface{}), + columns: make(map[string]map[string]map[string]map[string]interface{}), } l.addTables(tts) metrics.ReportDDLPending(task, metrics.DDLPendingNone, metrics.DDLPendingSynced) @@ -100,8 +100,15 @@ func NewLock(cli *clientv3.Client, ID, task, downSchema, downTable string, ti *m // 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) (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 + ) l.mu.Lock() defer func() { if len(newDDLs) > 0 { @@ -205,7 +212,7 @@ func (l *Lock) TrySync(callerSource, callerSchema, callerTable string, if cmp, err = nextTable.Compare(oldJoined); err == nil && cmp == 0 { if col, err := GetColumnName(l.ID, ddls[idx], ast.AlterTableAddColumns); err != nil { return newDDLs, err - } else if _, ok := l.columns[col]; len(col) > 0 && ok { + } else if len(col) > 0 && l.IsDroppedColumn(info, col) { return newDDLs, terror.ErrShardDDLOptimismTrySyncFail.Generate( l.ID, fmt.Sprintf("add column %s that wasn't fully dropped in downstream. ddl: %s", col, ddls[idx])) } @@ -269,7 +276,7 @@ func (l *Lock) TrySync(callerSource, callerSchema, callerTable string, // check for add column with a smaller field len if col, err := AddDifferentFieldLenColumns(l.ID, ddls[idx], nextTable, newJoined); err != nil { return ddls, err - } else if _, ok := l.columns[col]; len(col) > 0 && ok { + } else if len(col) > 0 && l.IsDroppedColumn(info, col) { return ddls, terror.ErrShardDDLOptimismTrySyncFail.Generate( l.ID, fmt.Sprintf("add column %s that wasn't fully dropped in downstream. ddl: %s", col, ddls[idx])) } @@ -280,7 +287,7 @@ func (l *Lock) TrySync(callerSource, callerSchema, callerTable string, if col, err := GetColumnName(l.ID, ddls[idx], ast.AlterTableDropColumn); err != nil { return ddls, err } else if len(col) > 0 { - err = l.AddDroppedColumn(col) + err = l.AddDroppedColumn(info, col) if err != nil { log.L().Error("fail to add dropped column info in etcd", zap.Error(err)) } @@ -513,18 +520,46 @@ func (l *Lock) GetVersion(source string, schema string, table string) int64 { return l.versions[source][schema][table] } +func (l *Lock) IsDroppedColumn(info Info, col string) bool { + if _, ok := l.columns[col]; !ok { + return false + } + source, upSchema, upTable := info.Source, info.UpSchema, info.UpTable + if _, ok := l.columns[col][source]; !ok { + return false + } + if _, ok := l.columns[col][source][upSchema]; !ok { + return false + } + if _, ok := l.columns[col][source][upSchema][upTable]; !ok { + return false + } + return true +} + // AddDroppedColumn adds a dropped column name in both etcd and lock's column map -func (l *Lock) AddDroppedColumn(col string) error { - if _, ok := l.columns[col]; ok { +func (l *Lock) AddDroppedColumn(info Info, col string) error { + if l.IsDroppedColumn(info, col) { return nil } - log.L().Debug("add not fully dropped columns", zap.String("lockID", l.ID), zap.String("column", col)) + log.L().Debug("add not fully dropped columns", zap.Stringer("info", info), zap.String("column", col)) - _, _, err := PutDroppedColumn(l.cli, l.Task, l.DownSchema, l.DownTable, col) + source, upSchema, upTable := info.Source, info.UpSchema, info.UpTable + _, _, err := PutDroppedColumn(l.cli, info, col) if err != nil { return err } - l.columns[col] = struct{}{} + + if _, ok := l.columns[col]; !ok { + l.columns[col] = make(map[string]map[string]map[string]interface{}) + } + if _, ok := l.columns[col][source]; !ok { + l.columns[col][source] = make(map[string]map[string]interface{}) + } + if _, ok := l.columns[col][source][upSchema]; !ok { + l.columns[col][source][upSchema] = make(map[string]interface{}) + } + l.columns[col][source][upSchema][upTable] = struct{}{} return nil } @@ -551,6 +586,7 @@ func (l *Lock) DeleteColumnsByDDLs(ddls []string) error { if err != nil { return err } + for _, col := range colsToDelete { delete(l.columns, col) } diff --git a/pkg/shardddl/optimism/ops.go b/pkg/shardddl/optimism/ops.go index 56478887f1..b5424f47af 100644 --- a/pkg/shardddl/optimism/ops.go +++ b/pkg/shardddl/optimism/ops.go @@ -62,7 +62,7 @@ func DeleteInfosOperationsSchemaColumn(cli *clientv3.Client, infos []Info, ops [ opsDel = append(opsDel, deleteOperationOp(op)) } opsDel = append(opsDel, deleteInitSchemaOp(schema.Task, schema.DownSchema, schema.DownTable)) - opsDel = append(opsDel, deleteDroppedColumnsOp(schema.Task, schema.DownSchema, schema.DownTable)) + opsDel = append(opsDel, deleteDroppedColumnsByLockOp(schema.Task, schema.DownSchema, schema.DownTable)) resp, rev, err := etcdutil.DoOpsInOneCmpsTxnWithRetry(cli, cmps, opsDel, []clientv3.Op{}) if err != nil { return 0, false, err @@ -72,7 +72,7 @@ func DeleteInfosOperationsSchemaColumn(cli *clientv3.Client, infos []Info, ops [ // DeleteInfosOperationsTablesSchemasByTask deletes the shard DDL infos and operations in etcd. func DeleteInfosOperationsTablesSchemasByTask(cli *clientv3.Client, task string) (int64, error) { - opsDel := make([]clientv3.Op, 0, 3) + opsDel := make([]clientv3.Op, 0, 5) opsDel = append(opsDel, clientv3.OpDelete(common.ShardDDLOptimismInfoKeyAdapter.Encode(task), clientv3.WithPrefix())) opsDel = append(opsDel, clientv3.OpDelete(common.ShardDDLOptimismOperationKeyAdapter.Encode(task), clientv3.WithPrefix())) opsDel = append(opsDel, clientv3.OpDelete(common.ShardDDLOptimismSourceTablesKeyAdapter.Encode(task), clientv3.WithPrefix())) diff --git a/tests/sequence_sharding_optimistic/run.sh b/tests/sequence_sharding_optimistic/run.sh index 749c313f51..447289b785 100755 --- a/tests/sequence_sharding_optimistic/run.sh +++ b/tests/sequence_sharding_optimistic/run.sh @@ -156,8 +156,8 @@ run() { # try to get schema for the table, the latest schema got. curl -X PUT ${API_URL} -d '{"op":1, "task":"sequence_sharding_optimistic", "sources": ["mysql-replica-01"], "database":"sharding_seq_opt", "table":"t1"}' > ${WORK_DIR}/get_schema.log - # downstream does not enable alter-primary-key, so this is CLUSTERED index - check_log_contains ${WORK_DIR}/get_schema.log 'CREATE TABLE `t1` ( `id` bigint(20) NOT NULL, `c2` varchar(20) DEFAULT NULL, `c3` int(11) DEFAULT NULL, PRIMARY KEY (`id`) /\*T!\[clustered_index\] CLUSTERED \*/) ENGINE=InnoDB DEFAULT CHARSET=latin1 COLLATE=latin1_bin' 1 + # downstream does not enable alter-primary-key, so this is NONCLUSTERED index + check_log_contains ${WORK_DIR}/get_schema.log 'CREATE TABLE `t1` ( `id` bigint(20) NOT NULL, `c2` varchar(20) DEFAULT NULL, `c3` int(11) DEFAULT NULL, PRIMARY KEY (`id`) /*T![clustered_index] NONCLUSTERED */) ENGINE=InnoDB DEFAULT CHARSET=latin1 COLLATE=latin1_bin' 1 # drop the schema. curl -X PUT ${API_URL} -d '{"op":3, "task":"sequence_sharding_optimistic", "sources": ["mysql-replica-01"], "database":"sharding_seq_opt", "table":"t1"}' > ${WORK_DIR}/remove_schema.log From bc42b8380e9db7b403e61258716a323525f31f75 Mon Sep 17 00:00:00 2001 From: Chunzhu Li Date: Thu, 18 Mar 2021 11:14:49 +0800 Subject: [PATCH 07/23] merge master and resolve conflicts --- dm/ctl/master/operate_schema.go | 21 +- dm/master/server.go | 2 + dm/master/shardddl/optimist.go | 9 +- dm/pb/dmmaster.pb.go | 326 +++++++++++++-------- dm/pb/dmworker.pb.go | 327 ++++++++++++++-------- dm/pbmock/dmmaster.go | 35 +++ dm/proto/dmmaster.proto | 2 + dm/proto/dmworker.proto | 2 + dm/worker/server.go | 98 +------ dm/worker/server_test.go | 20 +- dm/worker/worker.go | 161 +++++++++-- dm/worker/worker_test.go | 54 ++-- go.sum | 246 ---------------- pkg/shardddl/optimism/info.go | 3 + pkg/shardddl/optimism/info_test.go | 2 +- pkg/shardddl/optimism/lock.go | 28 +- pkg/shardddl/optimism/lock_test.go | 283 ++++++++++--------- syncer/checkpoint.go | 41 +++ syncer/schema.go | 38 ++- tests/handle_error/run.sh | 4 + tests/sequence_sharding_optimistic/run.sh | 4 +- tests/shardddl1/conf/source1.yaml | 4 +- tests/shardddl1/conf/source2.yaml | 2 + tests/shardddl1/run.sh | 125 +++++++++ 24 files changed, 1045 insertions(+), 792 deletions(-) diff --git a/dm/ctl/master/operate_schema.go b/dm/ctl/master/operate_schema.go index 49c4fd30f6..0df828cdec 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 immediately") + 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 6fb4bcde87..7a45942074 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/master/shardddl/optimist.go b/dm/master/shardddl/optimist.go index 81f25a1386..a773e5e4cc 100644 --- a/dm/master/shardddl/optimist.go +++ b/dm/master/shardddl/optimist.go @@ -521,7 +521,10 @@ 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(o.cli, info, tts) var cfStage = optimism.ConflictNone - if err != nil { + 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)) + } 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)) @@ -558,6 +561,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, info.ModRevision) if err != nil { diff --git a/dm/pb/dmmaster.pb.go b/dm/pb/dmmaster.pb.go index 60b75b1a55..6072a79779 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"` @@ -2989,127 +3005,129 @@ func init() { func init() { proto.RegisterFile("dmmaster.proto", fileDescriptor_f9bef11f2a341f03) } var fileDescriptor_f9bef11f2a341f03 = []byte{ - // 1920 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, 0xad, 0x3c, 0xb6, 0x65, 0x86, 0xf1, 0x2a, 0xde, 0xe9, - 0x6e, 0x60, 0x18, 0x45, 0x8c, 0xb8, 0x3d, 0x2d, 0xb0, 0x05, 0x36, 0x56, 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, 0x7e, 0x90, 0x1e, 0x17, - 0xe8, 0xa5, 0xc7, 0x22, 0xe9, 0xd7, 0x28, 0x50, 0xcc, 0x9b, 0x21, 0x39, 0xfc, 0x23, 0xb7, 0x0a, - 0x50, 0xdf, 0xf8, 0xe6, 0x8d, 0xde, 0xfb, 0xbd, 0x3f, 0xf3, 0xe6, 0xbd, 0x11, 0x74, 0xc6, 0xb3, - 0x99, 0x15, 0xc5, 0x2c, 0x7c, 0x1c, 0x84, 0x7e, 0xec, 0x93, 0x7a, 0x30, 0x32, 0x3a, 0xe3, 0xd9, - 0xb5, 0x1f, 0x5e, 0x26, 0x6b, 0xc6, 0xde, 0xd4, 0xf7, 0xa7, 0x2e, 0x3b, 0xb2, 0x02, 0xe7, 0xc8, - 0xf2, 0x3c, 0x3f, 0xb6, 0x62, 0xc7, 0xf7, 0x22, 0xc1, 0xa5, 0xdf, 0x40, 0xf7, 0x3c, 0xb6, 0xc2, - 0xf8, 0xa5, 0x15, 0x5d, 0x9a, 0xec, 0xf5, 0x9c, 0x45, 0x31, 0x21, 0xd0, 0x8c, 0xad, 0xe8, 0x52, - 0xd7, 0xf6, 0xb5, 0x83, 0x35, 0x13, 0xbf, 0x89, 0x0e, 0xab, 0x91, 0x3f, 0x0f, 0x6d, 0x16, 0xe9, - 0xf5, 0xfd, 0xc6, 0xc1, 0x9a, 0x99, 0x90, 0xa4, 0x0f, 0x10, 0xb2, 0x99, 0x7f, 0xc5, 0x5e, 0xb0, - 0xd8, 0xd2, 0x1b, 0xfb, 0xda, 0x41, 0xcb, 0x54, 0x56, 0xe8, 0x6b, 0xd8, 0x54, 0x34, 0x44, 0x81, - 0xef, 0x45, 0x8c, 0xf4, 0x60, 0x25, 0x64, 0xd1, 0xdc, 0x8d, 0x51, 0x49, 0xcb, 0x94, 0x14, 0xe9, - 0x42, 0x63, 0x16, 0x4d, 0xf5, 0x3a, 0x6a, 0xe6, 0x9f, 0xe4, 0x38, 0x53, 0xdc, 0xd8, 0x6f, 0x1c, - 0xb4, 0x8f, 0xf5, 0xc7, 0xc1, 0xe8, 0xf1, 0x89, 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, 0x75, 0x8e, 0x81, 0x0b, 0xe1, 0xcc, 0xb3, 0xc0, 0xac, 0xfb, 0x01, 0x37, 0xd9, 0xb3, - 0x66, 0x4c, 0x2a, 0xc6, 0x6f, 0xd5, 0xe4, 0x46, 0xce, 0x64, 0xfa, 0x7b, 0x0d, 0xb6, 0x72, 0x0a, - 0xa4, 0x55, 0xb7, 0x69, 0xc8, 0x2c, 0xae, 0x57, 0x59, 0xdc, 0xa8, 0xb4, 0xb8, 0xf9, 0xbf, 0x5a, - 0xfc, 0x39, 0x6c, 0xbe, 0x0a, 0xc6, 0x05, 0x83, 0x97, 0x8a, 0x23, 0x0d, 0x81, 0xa8, 0x22, 0xee, - 0x24, 0x50, 0x5f, 0x40, 0xef, 0xe7, 0x73, 0x16, 0xde, 0x9c, 0xc7, 0x56, 0x3c, 0x8f, 0x86, 0x4e, - 0x14, 0x2b, 0xd8, 0x31, 0x20, 0x5a, 0x75, 0x40, 0x0a, 0xd8, 0xaf, 0x60, 0xb7, 0x24, 0x67, 0x69, - 0x03, 0x9e, 0x14, 0x0d, 0xd8, 0xe5, 0x06, 0x28, 0x72, 0xcb, 0xf8, 0x4f, 0x60, 0xeb, 0xfc, 0xc2, - 0xbf, 0x1e, 0x0c, 0x86, 0x43, 0xdf, 0xbe, 0x8c, 0xde, 0xcf, 0xf1, 0x7f, 0xd6, 0x60, 0x55, 0x4a, - 0x20, 0x1d, 0xa8, 0x9f, 0x0e, 0xe4, 0xef, 0xea, 0xa7, 0x83, 0x54, 0x52, 0x5d, 0x91, 0x44, 0xa0, - 0x39, 0xf3, 0xc7, 0x4c, 0xa6, 0x0c, 0x7e, 0x93, 0x6d, 0xb8, 0xe7, 0x5f, 0x7b, 0x2c, 0xd4, 0x9b, - 0xb8, 0x28, 0x08, 0xbe, 0x73, 0x30, 0x18, 0x46, 0xfa, 0x3d, 0x54, 0x88, 0xdf, 0xdc, 0x1f, 0xd1, - 0x8d, 0x67, 0xb3, 0xb1, 0xbe, 0x82, 0xab, 0x92, 0x22, 0x06, 0xb4, 0xe6, 0x9e, 0xe4, 0xac, 0x22, - 0x27, 0xa5, 0xa9, 0x0d, 0xdb, 0x79, 0x33, 0x97, 0xf6, 0xed, 0x47, 0x70, 0xcf, 0xe5, 0x3f, 0x95, - 0x9e, 0x6d, 0x73, 0xcf, 0x4a, 0x71, 0xa6, 0xe0, 0x50, 0x17, 0xb6, 0x5f, 0x79, 0xfc, 0x33, 0x59, - 0x97, 0xce, 0x2c, 0xba, 0x84, 0xc2, 0x7a, 0xc8, 0x02, 0xd7, 0xb2, 0xd9, 0x19, 0x5a, 0x2c, 0xb4, - 0xe4, 0xd6, 0xc8, 0x3e, 0xb4, 0x27, 0x7e, 0x68, 0x33, 0x13, 0xcb, 0x90, 0x2c, 0x4a, 0xea, 0x12, - 0xfd, 0x1c, 0x76, 0x0a, 0xda, 0x96, 0xb5, 0x89, 0x9a, 0x70, 0x5f, 0x16, 0x81, 0x24, 0xbd, 0x5d, - 0xeb, 0x26, 0x41, 0xfd, 0x40, 0x29, 0x05, 0x68, 0x2d, 0x72, 0x65, 0x2d, 0x58, 0x9c, 0x0b, 0xdf, - 0x69, 0x60, 0x54, 0x09, 0x95, 0xe0, 0x6e, 0x95, 0xfa, 0xff, 0xad, 0x30, 0xdf, 0x69, 0xb0, 0xfb, - 0xd5, 0x3c, 0x9c, 0x56, 0x19, 0xab, 0xd8, 0xa3, 0xe5, 0x2f, 0x07, 0x03, 0x5a, 0x8e, 0x67, 0xd9, - 0xb1, 0x73, 0xc5, 0x24, 0xaa, 0x94, 0xc6, 0xdc, 0x76, 0x66, 0x22, 0x3a, 0x0d, 0x13, 0xbf, 0xf9, - 0xfe, 0x89, 0xe3, 0x32, 0x3c, 0xfa, 0x22, 0x95, 0x53, 0x1a, 0x33, 0x77, 0x3e, 0x1a, 0x38, 0xa1, - 0x7e, 0x0f, 0x39, 0x92, 0xa2, 0xbf, 0x01, 0xbd, 0x0c, 0xec, 0x4e, 0xca, 0xd7, 0x23, 0xe8, 0x9e, - 0x5c, 0x30, 0xfb, 0xf2, 0xbf, 0x14, 0x5d, 0xfa, 0x19, 0x6c, 0x2a, 0xfb, 0x96, 0x4e, 0xb4, 0x0b, - 0xd8, 0x96, 0x39, 0x71, 0x8e, 0x8a, 0x13, 0x55, 0x7b, 0x4a, 0x36, 0xac, 0x73, 0xb4, 0x82, 0x9d, - 0xa5, 0x83, 0xed, 0x7b, 0x13, 0x67, 0x2a, 0x73, 0x4c, 0x52, 0xdc, 0xc5, 0x02, 0xff, 0xe9, 0x40, - 0xde, 0x6b, 0x29, 0x4d, 0xe7, 0xb0, 0x53, 0xd0, 0x74, 0x27, 0x7e, 0x7c, 0x06, 0x3b, 0x26, 0x9b, - 0x3a, 0xbc, 0x91, 0x49, 0xb6, 0xdc, 0x7a, 0x0b, 0x58, 0xe3, 0x71, 0xc8, 0xa2, 0x48, 0xaa, 0x4d, - 0x48, 0xfa, 0x14, 0x7a, 0x45, 0x31, 0x4b, 0xfb, 0xfa, 0x27, 0xb0, 0x7d, 0x36, 0x99, 0xb8, 0x8e, - 0xc7, 0x5e, 0xb0, 0xd9, 0x28, 0x87, 0x24, 0xbe, 0x09, 0x52, 0x24, 0xfc, 0xbb, 0xaa, 0x69, 0xe0, - 0x75, 0xa5, 0xf0, 0xfb, 0xa5, 0x21, 0xfc, 0x38, 0x0d, 0xf7, 0x90, 0x59, 0xe3, 0x0c, 0x42, 0x29, - 0xdc, 0x82, 0x2d, 0xc2, 0x8d, 0x8a, 0xf3, 0xbf, 0x5a, 0x5a, 0xf1, 0xef, 0x34, 0x80, 0x17, 0xd8, - 0x4e, 0x9e, 0x7a, 0x13, 0xbf, 0xd2, 0xf9, 0x06, 0xb4, 0x66, 0x68, 0xd7, 0xe9, 0x00, 0x7f, 0xd9, - 0x34, 0x53, 0x9a, 0xdf, 0x41, 0x96, 0xeb, 0xa4, 0xe5, 0x56, 0x10, 0xfc, 0x17, 0x01, 0x63, 0xe1, - 0x2b, 0x73, 0x28, 0x8a, 0xcd, 0x9a, 0x99, 0xd2, 0xbc, 0x75, 0xb4, 0x5d, 0x87, 0x79, 0x31, 0x72, - 0xc5, 0x2d, 0xa5, 0xac, 0xd0, 0x11, 0x80, 0x08, 0xe4, 0x42, 0x3c, 0x04, 0x9a, 0x3c, 0xfa, 0x49, - 0x08, 0xf8, 0x37, 0xc7, 0x11, 0xc5, 0xd6, 0x34, 0xb9, 0x20, 0x05, 0x81, 0xd5, 0x03, 0xd3, 0x4d, - 0xd6, 0x15, 0x49, 0xd1, 0x21, 0x74, 0x79, 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, 0x07, 0xb0, 0x2a, 0xda, 0x76, 0x51, 0xff, 0xdb, 0xc7, 0x1d, 0x1e, 0xce, 0xcc, 0xf5, 0x66, - 0xc2, 0x4e, 0xe4, 0x09, 0x2f, 0xdc, 0x26, 0x4f, 0xb4, 0xfc, 0x39, 0x79, 0x99, 0xeb, 0xcc, 0x84, - 0x4d, 0xff, 0xa2, 0xc1, 0xaa, 0x10, 0x13, 0x91, 0xc7, 0xb0, 0xe2, 0xa2, 0xd5, 0x28, 0xaa, 0x7d, - 0xbc, 0x8d, 0x39, 0x55, 0xf0, 0xc5, 0x97, 0x35, 0x53, 0xee, 0xe2, 0xfb, 0x05, 0x2c, 0xf4, 0x82, - 0xb2, 0x5f, 0xb5, 0x96, 0xef, 0x17, 0xbb, 0xf8, 0x7e, 0xa1, 0x16, 0x3d, 0xa4, 0xec, 0x57, 0xad, - 0xe1, 0xfb, 0xc5, 0xae, 0xa7, 0x2d, 0x58, 0x11, 0xb9, 0xc4, 0x47, 0x06, 0x94, 0x9b, 0x3b, 0x81, - 0xbd, 0x1c, 0xdc, 0x56, 0x0a, 0xab, 0x97, 0x83, 0xd5, 0x4a, 0xd5, 0xf7, 0x72, 0xea, 0x5b, 0x89, - 0x1a, 0x9e, 0x1e, 0x3c, 0x7c, 0x49, 0x36, 0x0a, 0x82, 0x32, 0x20, 0xaa, 0xca, 0xa5, 0xcb, 0xde, - 0x27, 0xb0, 0x2a, 0xc0, 0xe7, 0x5a, 0x1c, 0xe9, 0x6a, 0x33, 0xe1, 0xd1, 0xbf, 0x6a, 0x59, 0x2d, - 0xb7, 0x2f, 0xd8, 0xcc, 0x5a, 0x5c, 0xcb, 0x91, 0x9d, 0x8d, 0x27, 0xa5, 0x36, 0x70, 0xe1, 0x78, - 0xc2, 0x8f, 0xdc, 0xd8, 0x8a, 0xad, 0x91, 0x15, 0xa5, 0x97, 0x68, 0x42, 0x73, 0xeb, 0x63, 0x6b, - 0xe4, 0x32, 0x79, 0x87, 0x0a, 0x02, 0x0f, 0x07, 0xea, 0xd3, 0x57, 0xe4, 0xe1, 0x40, 0x4a, 0xbd, - 0x0f, 0x24, 0xda, 0x3b, 0xb9, 0x0f, 0x0e, 0x61, 0xfb, 0x39, 0x8b, 0xcf, 0xe7, 0x23, 0x7e, 0x61, - 0x9e, 0x4c, 0xa6, 0xb7, 0x5c, 0x07, 0xf4, 0x15, 0xec, 0x14, 0xf6, 0x2e, 0x0d, 0x91, 0x40, 0xd3, - 0x9e, 0x4c, 0x13, 0x37, 0xe2, 0x37, 0x1d, 0xc0, 0xc6, 0x73, 0x16, 0x2b, 0xba, 0x1f, 0x2a, 0x17, - 0x80, 0x6c, 0xbe, 0x4e, 0x26, 0xd3, 0x97, 0x37, 0x01, 0xbb, 0xe5, 0x36, 0x18, 0x42, 0x27, 0x91, - 0xb2, 0x34, 0xaa, 0x2e, 0x34, 0xec, 0x49, 0xda, 0xb6, 0xd9, 0x93, 0x29, 0xdd, 0x81, 0xad, 0xe7, - 0x4c, 0x9e, 0xb6, 0x0c, 0x19, 0x3d, 0x40, 0x6f, 0x29, 0xcb, 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, 0x1e, 0xac, 0x8d, 0x1c, 0xcf, 0xf5, 0xa7, 0x5f, 0xf9, 0x91, 0xcc, 0xbd, - 0x6c, 0x81, 0xcb, 0x8a, 0x5e, 0xbb, 0xe9, 0x3c, 0xc2, 0xbf, 0x69, 0x04, 0x5b, 0x39, 0x48, 0x77, - 0x92, 0x60, 0xcf, 0x61, 0xe7, 0x65, 0x68, 0x79, 0xd1, 0x84, 0x85, 0xf9, 0x96, 0x2a, 0xbb, 0x25, - 0x34, 0xf5, 0x96, 0x50, 0x8a, 0x89, 0xd0, 0x2c, 0x29, 0xde, 0x72, 0x14, 0x05, 0x2d, 0x6b, 0xc0, - 0xe1, 0x08, 0x5a, 0x49, 0xe3, 0x46, 0xb6, 0xe0, 0x83, 0x53, 0xef, 0xca, 0x72, 0x9d, 0x71, 0xb2, - 0xd4, 0xad, 0x91, 0x0f, 0xa0, 0x8d, 0x2f, 0x28, 0x62, 0xa9, 0xab, 0x91, 0x2e, 0xac, 0x8b, 0x51, - 0x5d, 0xae, 0xd4, 0x49, 0x07, 0xe0, 0x3c, 0xf6, 0x03, 0x49, 0x37, 0x90, 0xbe, 0xf0, 0xaf, 0x25, - 0xdd, 0x3c, 0xfc, 0x29, 0xb4, 0x92, 0x6e, 0x41, 0xd1, 0x91, 0x2c, 0x75, 0x6b, 0x64, 0x13, 0x36, - 0x9e, 0x5d, 0x39, 0x76, 0x9c, 0x2e, 0x69, 0x64, 0x17, 0xb6, 0x4e, 0x2c, 0xcf, 0x66, 0x6e, 0x9e, - 0x51, 0x3f, 0xfc, 0x25, 0xac, 0xca, 0xd4, 0xe7, 0xd0, 0xa4, 0x2c, 0x4e, 0x76, 0x6b, 0x64, 0x1d, - 0x5a, 0xfc, 0x20, 0x22, 0xa5, 0x71, 0x18, 0x22, 0x2f, 0x91, 0x46, 0x98, 0x22, 0x24, 0x48, 0x0b, - 0x98, 0x08, 0x11, 0xe9, 0xe6, 0xf1, 0xbf, 0x37, 0x60, 0x45, 0xfc, 0x80, 0x7c, 0x0d, 0x6b, 0xe9, - 0xb3, 0x11, 0xc1, 0xab, 0xa3, 0xf8, 0x4e, 0x65, 0xec, 0x14, 0x56, 0x85, 0xe7, 0xe9, 0xc3, 0xdf, - 0xfe, 0xfd, 0x5f, 0x7f, 0xaa, 0xdf, 0xa7, 0xdb, 0x47, 0x56, 0xe0, 0x44, 0x47, 0x57, 0x4f, 0x2c, - 0x37, 0xb8, 0xb0, 0x9e, 0x1c, 0xf1, 0xd4, 0x8d, 0x3e, 0xd5, 0x0e, 0xc9, 0x04, 0xda, 0xca, 0xeb, - 0x0d, 0xe9, 0x71, 0x31, 0xe5, 0xf7, 0x22, 0x63, 0xb7, 0xb4, 0x2e, 0x15, 0x3c, 0x42, 0x05, 0xfb, - 0xc6, 0x83, 0x2a, 0x05, 0x47, 0x6f, 0xf8, 0xc9, 0xff, 0x96, 0xeb, 0xf9, 0x0c, 0x20, 0x7b, 0x51, - 0x21, 0x88, 0xb6, 0xf4, 0x48, 0x63, 0xf4, 0x8a, 0xcb, 0x52, 0x49, 0x8d, 0xb8, 0xd0, 0x56, 0x1e, - 0x1f, 0x88, 0x51, 0x78, 0x8d, 0x50, 0x5e, 0x4b, 0x8c, 0x07, 0x95, 0x3c, 0x29, 0xe9, 0x63, 0x84, - 0xdb, 0x27, 0x7b, 0x05, 0xb8, 0x11, 0x6e, 0x95, 0x78, 0xc9, 0x09, 0xac, 0xab, 0x33, 0x3e, 0x41, - 0xeb, 0x2b, 0x1e, 0x37, 0x0c, 0xbd, 0xcc, 0x48, 0x21, 0x7f, 0x01, 0x1b, 0xb9, 0xa9, 0x9a, 0xe0, - 0xe6, 0xaa, 0xb1, 0xde, 0xb8, 0x5f, 0xc1, 0x49, 0xe5, 0x7c, 0x0d, 0xbd, 0xf2, 0x14, 0x8c, 0x5e, - 0xfc, 0x50, 0x09, 0x4a, 0x79, 0x12, 0x35, 0xfa, 0x8b, 0xd8, 0xa9, 0xe8, 0x33, 0xe8, 0x16, 0xa7, - 0x45, 0x82, 0xee, 0x5b, 0x30, 0xdc, 0x1a, 0x7b, 0xd5, 0xcc, 0x54, 0xe0, 0xa7, 0xb0, 0x96, 0x0e, - 0x77, 0x22, 0x51, 0x8b, 0x33, 0xa1, 0x48, 0xd4, 0xd2, 0x04, 0x48, 0x6b, 0x64, 0x0a, 0x1b, 0xb9, - 0x79, 0x4b, 0xf8, 0xab, 0x6a, 0xd8, 0x13, 0xfe, 0xaa, 0x1c, 0xce, 0xe8, 0x47, 0x18, 0xe0, 0x07, - 0x46, 0xaf, 0x18, 0x60, 0x51, 0xed, 0x78, 0x2a, 0x9e, 0x42, 0x27, 0x3f, 0x1a, 0x91, 0xfb, 0xe2, - 0xf9, 0xa0, 0x62, 0xea, 0x32, 0x8c, 0x2a, 0x56, 0x8a, 0x39, 0x84, 0x8d, 0xdc, 0x84, 0x23, 0x31, - 0x57, 0x0c, 0x4d, 0x12, 0x73, 0xd5, 0x38, 0x44, 0x7f, 0x88, 0x98, 0x1f, 0x1d, 0x7e, 0x5c, 0xc0, - 0x2c, 0x1b, 0xa5, 0xa3, 0x37, 0xfc, 0x4e, 0xfd, 0x36, 0x49, 0xce, 0xcb, 0xd4, 0x4f, 0xa2, 0x0c, - 0xe5, 0xfc, 0x94, 0x9b, 0x92, 0x72, 0x7e, 0xca, 0x4f, 0x42, 0xf4, 0x13, 0xd4, 0xf9, 0xd0, 0x30, - 0x0a, 0x3a, 0x45, 0x23, 0x79, 0xf4, 0xc6, 0x0f, 0xf0, 0xd8, 0xfe, 0x0a, 0x20, 0x6b, 0x05, 0xc5, - 0xb1, 0x2d, 0x75, 0xa3, 0xe2, 0xd8, 0x96, 0x3b, 0x46, 0xda, 0x47, 0x1d, 0x3a, 0xe9, 0x55, 0xdb, - 0x45, 0x26, 0x59, 0xc4, 0xb1, 0xa3, 0xca, 0x47, 0x5c, 0x6d, 0x09, 0xf3, 0x11, 0xcf, 0xb5, 0x5f, - 0x74, 0x1f, 0xb5, 0x18, 0xc6, 0x4e, 0x31, 0xe2, 0xb8, 0x8d, 0x1b, 0xe1, 0x62, 0xff, 0x92, 0xb5, - 0x45, 0x42, 0x4f, 0x55, 0x57, 0x25, 0xf4, 0x54, 0xf6, 0x50, 0x49, 0xa5, 0x23, 0xfd, 0xa2, 0x9e, - 0xf9, 0x48, 0x2d, 0x76, 0xe4, 0x25, 0xac, 0x88, 0x3e, 0x87, 0x6c, 0x4a, 0x61, 0x8a, 0x7c, 0xa2, - 0x2e, 0x49, 0xc1, 0x3f, 0x40, 0xc1, 0x1f, 0x92, 0xdb, 0x4a, 0x28, 0xf9, 0x06, 0xda, 0x4a, 0x6b, - 0x20, 0xea, 0x74, 0xb9, 0x7d, 0x11, 0x75, 0xba, 0xa2, 0x87, 0x58, 0xe8, 0x25, 0xc6, 0x77, 0xe1, - 0xb1, 0x38, 0x81, 0x75, 0xb5, 0x75, 0x12, 0x45, 0xaf, 0xa2, 0xc7, 0x32, 0xf4, 0x32, 0x23, 0x3d, - 0x10, 0xa7, 0xd0, 0xc9, 0xf7, 0x00, 0xe2, 0x6c, 0x55, 0x36, 0x18, 0xe2, 0x6c, 0x55, 0xb7, 0x0c, - 0xb4, 0xf6, 0x54, 0xff, 0xdb, 0xdb, 0xbe, 0xf6, 0xfd, 0xdb, 0xbe, 0xf6, 0xcf, 0xb7, 0x7d, 0xed, - 0x0f, 0xef, 0xfa, 0xb5, 0xef, 0xdf, 0xf5, 0x6b, 0xff, 0x78, 0xd7, 0xaf, 0x8d, 0x56, 0xf0, 0xef, - 0x9a, 0x1f, 0xfd, 0x27, 0x00, 0x00, 0xff, 0xff, 0xc0, 0x6b, 0xd1, 0x5e, 0xf2, 0x19, 0x00, 0x00, + // 1940 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, 0x4e, 0x5e, 0xdb, 0x32, 0xc3, 0xf8, 0x14, 0xdf, 0xf6, + 0x2e, 0x30, 0x8c, 0x22, 0x46, 0xdc, 0x3e, 0x1d, 0x70, 0x05, 0x2e, 0x56, 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, 0x65, 0xfa, 0x78, 0x40, + 0x81, 0xa2, 0x8f, 0x45, 0xd2, 0xaf, 0x51, 0xa0, 0xd8, 0xd9, 0x25, 0xb9, 0xfc, 0x23, 0xb7, 0x0a, + 0x50, 0xbf, 0x71, 0x76, 0x56, 0x33, 0xbf, 0xf9, 0xb3, 0xb3, 0x33, 0x2b, 0xe8, 0x8c, 0x67, 0x33, + 0x2b, 0x8a, 0x59, 0xf8, 0x38, 0x08, 0xfd, 0xd8, 0x27, 0xf5, 0x60, 0x64, 0x74, 0xc6, 0xb3, 0x6b, + 0x3f, 0xbc, 0x4c, 0xd6, 0x8c, 0xbd, 0xa9, 0xef, 0x4f, 0x5d, 0x76, 0x64, 0x05, 0xce, 0x91, 0xe5, + 0x79, 0x7e, 0x6c, 0xc5, 0x8e, 0xef, 0x45, 0x82, 0x4b, 0xbf, 0x81, 0xee, 0x79, 0x6c, 0x85, 0xf1, + 0x4b, 0x2b, 0xba, 0x34, 0xd9, 0xeb, 0x39, 0x8b, 0x62, 0x42, 0xa0, 0x19, 0x5b, 0xd1, 0xa5, 0xae, + 0xed, 0x6b, 0x07, 0x6b, 0x26, 0x7e, 0x13, 0x1d, 0x56, 0x23, 0x7f, 0x1e, 0xda, 0x2c, 0xd2, 0xeb, + 0xfb, 0x8d, 0x83, 0x35, 0x33, 0x21, 0x49, 0x1f, 0x20, 0x64, 0x33, 0xff, 0x8a, 0xbd, 0x60, 0xb1, + 0xa5, 0x37, 0xf6, 0xb5, 0x83, 0x96, 0xa9, 0xac, 0xd0, 0xd7, 0xb0, 0xa9, 0x68, 0x88, 0x02, 0xdf, + 0x8b, 0x18, 0xe9, 0xc1, 0x4a, 0xc8, 0xa2, 0xb9, 0x1b, 0xa3, 0x92, 0x96, 0x29, 0x29, 0xd2, 0x85, + 0xc6, 0x2c, 0x9a, 0xea, 0x75, 0xd4, 0xcc, 0x3f, 0xc9, 0x71, 0xa6, 0xb8, 0xb1, 0xdf, 0x38, 0x68, + 0x1f, 0xeb, 0x8f, 0x83, 0xd1, 0xe3, 0x13, 0x7f, 0x36, 0xf3, 0xbd, 0x5f, 0xa0, 0x9d, 0x89, 0xd0, + 0x14, 0x12, 0xfd, 0x35, 0x90, 0xb3, 0x80, 0x85, 0x56, 0xcc, 0x54, 0xb3, 0x0c, 0xa8, 0xfb, 0x01, + 0xea, 0xeb, 0x1c, 0x03, 0x17, 0xc2, 0x99, 0x67, 0x81, 0x59, 0xf7, 0x03, 0x6e, 0xb2, 0x67, 0xcd, + 0x98, 0x54, 0x8c, 0xdf, 0xaa, 0xc9, 0x8d, 0x9c, 0xc9, 0xf4, 0xf7, 0x1a, 0x6c, 0xe5, 0x14, 0x48, + 0xab, 0x6e, 0xd3, 0x90, 0x59, 0x5c, 0xaf, 0xb2, 0xb8, 0x51, 0x69, 0x71, 0xf3, 0x7f, 0xb5, 0xf8, + 0x73, 0xd8, 0x7c, 0x15, 0x8c, 0x0b, 0x06, 0x2f, 0x15, 0x47, 0x1a, 0x02, 0x51, 0x45, 0xdc, 0x49, + 0xa0, 0xbe, 0x80, 0xde, 0xcf, 0xe7, 0x2c, 0xbc, 0x39, 0x8f, 0xad, 0x78, 0x1e, 0x0d, 0x9d, 0x28, + 0x56, 0xb0, 0x63, 0x40, 0xb4, 0xea, 0x80, 0x14, 0xb0, 0x5f, 0xc1, 0x6e, 0x49, 0xce, 0xd2, 0x06, + 0x3c, 0x29, 0x1a, 0xb0, 0xcb, 0x0d, 0x50, 0xe4, 0x96, 0xf1, 0x9f, 0xc0, 0xd6, 0xf9, 0x85, 0x7f, + 0x3d, 0x18, 0x0c, 0x87, 0xbe, 0x7d, 0x19, 0xbd, 0x9f, 0xe3, 0xff, 0xac, 0xc1, 0xaa, 0x94, 0x40, + 0x3a, 0x50, 0x3f, 0x1d, 0xc8, 0xdf, 0xd5, 0x4f, 0x07, 0xa9, 0xa4, 0xba, 0x22, 0x89, 0x40, 0x73, + 0xe6, 0x8f, 0x99, 0x4c, 0x19, 0xfc, 0x26, 0xdb, 0x70, 0xcf, 0xbf, 0xf6, 0x58, 0xa8, 0x37, 0x71, + 0x51, 0x10, 0x7c, 0xe7, 0x60, 0x30, 0x8c, 0xf4, 0x7b, 0xa8, 0x10, 0xbf, 0xb9, 0x3f, 0xa2, 0x1b, + 0xcf, 0x66, 0x63, 0x7d, 0x05, 0x57, 0x25, 0x45, 0x0c, 0x68, 0xcd, 0x3d, 0xc9, 0x59, 0x45, 0x4e, + 0x4a, 0x53, 0x1b, 0xb6, 0xf3, 0x66, 0x2e, 0xed, 0xdb, 0x8f, 0xe0, 0x9e, 0xcb, 0x7f, 0x2a, 0x3d, + 0xdb, 0xe6, 0x9e, 0x95, 0xe2, 0x4c, 0xc1, 0xa1, 0x2e, 0x6c, 0xbf, 0xf2, 0xf8, 0x67, 0xb2, 0x2e, + 0x9d, 0x59, 0x74, 0x09, 0x85, 0xf5, 0x90, 0x05, 0xae, 0x65, 0xb3, 0x33, 0xb4, 0x58, 0x68, 0xc9, + 0xad, 0x91, 0x7d, 0x68, 0x4f, 0xfc, 0xd0, 0x66, 0x26, 0x96, 0x21, 0x59, 0x94, 0xd4, 0x25, 0xfa, + 0x39, 0xec, 0x14, 0xb4, 0x2d, 0x6b, 0x13, 0x35, 0xe1, 0xbe, 0x2c, 0x02, 0x49, 0x7a, 0xbb, 0xd6, + 0x4d, 0x82, 0xfa, 0x81, 0x52, 0x0a, 0xd0, 0x5a, 0xe4, 0xca, 0x5a, 0xb0, 0x38, 0x17, 0xbe, 0xd3, + 0xc0, 0xa8, 0x12, 0x2a, 0xc1, 0xdd, 0x2a, 0xf5, 0xff, 0x5b, 0x61, 0xbe, 0xd3, 0x60, 0xf7, 0xab, + 0x79, 0x38, 0xad, 0x32, 0x56, 0xb1, 0x47, 0xcb, 0x5f, 0x0e, 0x06, 0xb4, 0x1c, 0xcf, 0xb2, 0x63, + 0xe7, 0x8a, 0x49, 0x54, 0x29, 0x8d, 0xb9, 0xed, 0xcc, 0x44, 0x74, 0x1a, 0x26, 0x7e, 0xf3, 0xfd, + 0x13, 0xc7, 0x65, 0x78, 0xf4, 0x45, 0x2a, 0xa7, 0x34, 0x66, 0xee, 0x7c, 0x34, 0x70, 0x42, 0xfd, + 0x1e, 0x72, 0x24, 0x45, 0x7f, 0x03, 0x7a, 0x19, 0xd8, 0x9d, 0x94, 0xaf, 0x47, 0xd0, 0x3d, 0xb9, + 0x60, 0xf6, 0xe5, 0x7f, 0x29, 0xba, 0xf4, 0x33, 0xd8, 0x54, 0xf6, 0x2d, 0x9d, 0x68, 0x17, 0xb0, + 0x2d, 0x73, 0xe2, 0x1c, 0x15, 0x27, 0xaa, 0xf6, 0x94, 0x6c, 0x58, 0xe7, 0x68, 0x05, 0x3b, 0x4b, + 0x07, 0xdb, 0xf7, 0x26, 0xce, 0x54, 0xe6, 0x98, 0xa4, 0xb8, 0x8b, 0x05, 0xfe, 0xd3, 0x81, 0xbc, + 0xd7, 0x52, 0x9a, 0xce, 0x61, 0xa7, 0xa0, 0xe9, 0x4e, 0xfc, 0xf8, 0x0c, 0x76, 0x4c, 0x36, 0x75, + 0x78, 0x23, 0x93, 0x6c, 0xb9, 0xf5, 0x16, 0xb0, 0xc6, 0xe3, 0x90, 0x45, 0x91, 0x54, 0x9b, 0x90, + 0xf4, 0x29, 0xf4, 0x8a, 0x62, 0x96, 0xf6, 0xf5, 0x4f, 0x60, 0xfb, 0x6c, 0x32, 0x71, 0x1d, 0x8f, + 0xbd, 0x60, 0xb3, 0x51, 0x0e, 0x49, 0x7c, 0x13, 0xa4, 0x48, 0xf8, 0x77, 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, 0xbf, 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, + 0x5b, 0x4a, 0x59, 0xa1, 0x23, 0x00, 0x11, 0xc8, 0x85, 0x78, 0x08, 0x34, 0x79, 0xf4, 0x93, 0x10, + 0xf0, 0x6f, 0x8e, 0x23, 0x8a, 0xad, 0x69, 0x72, 0x41, 0x0a, 0x02, 0xab, 0x07, 0xa6, 0x9b, 0xac, + 0x2b, 0x92, 0xa2, 0x43, 0xe8, 0xf2, 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, 0x60, 0x55, 0xb4, 0xed, 0xa2, 0xfe, 0xb7, 0x8f, 0x3b, 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, 0x78, + 0x1b, 0x73, 0xaa, 0xe0, 0x8b, 0x2f, 0x6b, 0xa6, 0xdc, 0xc5, 0xf7, 0x0b, 0x58, 0xe8, 0x05, 0x65, + 0xbf, 0x6a, 0x2d, 0xdf, 0x2f, 0x76, 0xf1, 0xfd, 0x42, 0x2d, 0x7a, 0x48, 0xd9, 0xaf, 0x5a, 0xc3, + 0xf7, 0x8b, 0x5d, 0x4f, 0x5b, 0xb0, 0x22, 0x72, 0x89, 0x8f, 0x0c, 0x28, 0x37, 0x77, 0x02, 0x7b, + 0x39, 0xb8, 0xad, 0x14, 0x56, 0x2f, 0x07, 0xab, 0x95, 0xaa, 0xef, 0xe5, 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, 0x7f, 0xd7, 0xb2, 0x5a, 0x6e, + 0x5f, 0xb0, 0x99, 0xb5, 0xb8, 0x96, 0x23, 0x3b, 0x1b, 0x4f, 0x4a, 0x6d, 0xe0, 0xc2, 0xf1, 0x84, + 0x1f, 0xb9, 0xb1, 0x15, 0x5b, 0x23, 0x2b, 0x4a, 0x2f, 0xd1, 0x84, 0xe6, 0xd6, 0xc7, 0xd6, 0xc8, + 0x65, 0xf2, 0x0e, 0x15, 0x04, 0x1e, 0x0e, 0xd4, 0xa7, 0xaf, 0xc8, 0xc3, 0x81, 0x14, 0xdf, 0x3d, + 0x71, 0xe7, 0xd1, 0x85, 0xbe, 0x2a, 0x8e, 0x34, 0x12, 0x1c, 0x0d, 0x6f, 0x0c, 0xf5, 0x16, 0x2e, + 0xe2, 0xb7, 0x7a, 0x73, 0x48, 0xbb, 0xee, 0xe4, 0xe6, 0x38, 0x84, 0xed, 0xe7, 0x2c, 0x3e, 0x9f, + 0x8f, 0xf8, 0xd5, 0x7a, 0x32, 0x99, 0xde, 0x72, 0x71, 0xd0, 0x57, 0xb0, 0x53, 0xd8, 0xbb, 0x34, + 0x44, 0x02, 0x4d, 0x7b, 0x32, 0x4d, 0x1c, 0x8e, 0xdf, 0x74, 0x00, 0x1b, 0xcf, 0x59, 0xac, 0xe8, + 0x7e, 0xa8, 0x5c, 0x15, 0xb2, 0x4d, 0x3b, 0x99, 0x4c, 0x5f, 0xde, 0x04, 0xec, 0x96, 0x7b, 0x63, + 0x08, 0x9d, 0x44, 0xca, 0xd2, 0xa8, 0xba, 0xd0, 0xb0, 0x27, 0x69, 0x83, 0x67, 0x4f, 0xa6, 0x74, + 0x07, 0xb6, 0x9e, 0x33, 0x79, 0x2e, 0x33, 0x64, 0xf4, 0x00, 0xbd, 0xa5, 0x2c, 0x4b, 0x55, 0x52, + 0x80, 0x96, 0x09, 0xf8, 0xa3, 0x06, 0xe4, 0x4b, 0xcb, 0x1b, 0xbb, 0xec, 0x59, 0x18, 0xfa, 0xe1, + 0xc2, 0xae, 0x16, 0xb9, 0xef, 0x95, 0xa4, 0x7b, 0xb0, 0x36, 0x72, 0x3c, 0xd7, 0x9f, 0x7e, 0xe5, + 0x47, 0x32, 0x4b, 0xb3, 0x05, 0x4c, 0xb1, 0xd7, 0x6e, 0x3a, 0xb9, 0xf0, 0x6f, 0x1a, 0xc1, 0x56, + 0x0e, 0xd2, 0x9d, 0x24, 0xd8, 0x73, 0xd8, 0x79, 0x19, 0x5a, 0x5e, 0x34, 0x61, 0x61, 0xbe, 0xf9, + 0xca, 0xee, 0x13, 0x4d, 0xbd, 0x4f, 0x94, 0xb2, 0x23, 0x34, 0x4b, 0x8a, 0x37, 0x27, 0x45, 0x41, + 0xcb, 0x1a, 0x70, 0x38, 0x82, 0x56, 0xd2, 0xe2, 0x91, 0x2d, 0xf8, 0xe0, 0xd4, 0xbb, 0xb2, 0x5c, + 0x67, 0x9c, 0x2c, 0x75, 0x6b, 0xe4, 0x03, 0x68, 0xe3, 0x5b, 0x8b, 0x58, 0xea, 0x6a, 0xa4, 0x0b, + 0xeb, 0x62, 0xa8, 0x97, 0x2b, 0x75, 0xd2, 0x01, 0x38, 0x8f, 0xfd, 0x40, 0xd2, 0x0d, 0xa4, 0x2f, + 0xfc, 0x6b, 0x49, 0x37, 0x0f, 0x7f, 0x0a, 0xad, 0xa4, 0xaf, 0x50, 0x74, 0x24, 0x4b, 0xdd, 0x1a, + 0xd9, 0x84, 0x8d, 0x67, 0x57, 0x8e, 0x1d, 0xa7, 0x4b, 0x1a, 0xd9, 0x85, 0xad, 0x13, 0xcb, 0xb3, + 0x99, 0x9b, 0x67, 0xd4, 0x0f, 0x7f, 0x09, 0xab, 0x32, 0xf5, 0x39, 0x34, 0x29, 0x8b, 0x93, 0xdd, + 0x1a, 0x59, 0x87, 0x16, 0x3f, 0x88, 0x48, 0x69, 0x1c, 0x86, 0xc8, 0x4b, 0xa4, 0x11, 0xa6, 0x08, + 0x09, 0xd2, 0x02, 0x26, 0x42, 0x44, 0xba, 0x79, 0xfc, 0xef, 0x0d, 0x58, 0x11, 0x3f, 0x20, 0x5f, + 0xc3, 0x5a, 0xfa, 0xc0, 0x44, 0xf0, 0x92, 0x29, 0xbe, 0x68, 0x19, 0x3b, 0x85, 0x55, 0xe1, 0x79, + 0xfa, 0xf0, 0xb7, 0x7f, 0xfb, 0xd7, 0x9f, 0xea, 0xf7, 0xe9, 0xf6, 0x91, 0x15, 0x38, 0xd1, 0xd1, + 0xd5, 0x13, 0xcb, 0x0d, 0x2e, 0xac, 0x27, 0x47, 0x3c, 0x75, 0xa3, 0x4f, 0xb5, 0x43, 0x32, 0x81, + 0xb6, 0xf2, 0xce, 0x43, 0x7a, 0x5c, 0x4c, 0xf9, 0x65, 0xc9, 0xd8, 0x2d, 0xad, 0x4b, 0x05, 0x8f, + 0x50, 0xc1, 0xbe, 0xf1, 0xa0, 0x4a, 0xc1, 0xd1, 0x1b, 0x7e, 0xf2, 0xbf, 0xe5, 0x7a, 0x3e, 0x03, + 0xc8, 0xde, 0x5e, 0x08, 0xa2, 0x2d, 0x3d, 0xe7, 0x18, 0xbd, 0xe2, 0xb2, 0x54, 0x52, 0x23, 0x2e, + 0xb4, 0x95, 0x67, 0x0a, 0x62, 0x14, 0xde, 0x2d, 0x94, 0x77, 0x15, 0xe3, 0x41, 0x25, 0x4f, 0x4a, + 0xfa, 0x18, 0xe1, 0xf6, 0xc9, 0x5e, 0x01, 0x6e, 0x84, 0x5b, 0x25, 0x5e, 0x72, 0x02, 0xeb, 0xea, + 0x6b, 0x00, 0x41, 0xeb, 0x2b, 0x9e, 0x41, 0x0c, 0xbd, 0xcc, 0x48, 0x21, 0x7f, 0x01, 0x1b, 0xb9, + 0xf9, 0x9b, 0xe0, 0xe6, 0xaa, 0x07, 0x00, 0xe3, 0x7e, 0x05, 0x27, 0x95, 0xf3, 0x35, 0xf4, 0xca, + 0xf3, 0x32, 0x7a, 0xf1, 0x43, 0x25, 0x28, 0xe5, 0x99, 0xd5, 0xe8, 0x2f, 0x62, 0xa7, 0xa2, 0xcf, + 0xa0, 0x5b, 0x9c, 0x2b, 0x09, 0xba, 0x6f, 0xc1, 0x18, 0x6c, 0xec, 0x55, 0x33, 0x53, 0x81, 0x9f, + 0xc2, 0x5a, 0x3a, 0x06, 0x8a, 0x44, 0x2d, 0x4e, 0x8f, 0x22, 0x51, 0x4b, 0xb3, 0x22, 0xad, 0x91, + 0x29, 0x6c, 0xe4, 0x26, 0x33, 0xe1, 0xaf, 0xaa, 0xb1, 0x50, 0xf8, 0xab, 0x72, 0x8c, 0xa3, 0x1f, + 0x61, 0x80, 0x1f, 0x18, 0xbd, 0x62, 0x80, 0x45, 0xb5, 0xe3, 0xa9, 0x78, 0x0a, 0x9d, 0xfc, 0x10, + 0x45, 0xee, 0x8b, 0x87, 0x86, 0x8a, 0xf9, 0xcc, 0x30, 0xaa, 0x58, 0x29, 0xe6, 0x10, 0x36, 0x72, + 0xb3, 0x90, 0xc4, 0x5c, 0x31, 0x5e, 0x49, 0xcc, 0x55, 0x83, 0x13, 0xfd, 0x21, 0x62, 0x7e, 0x74, + 0xf8, 0x71, 0x01, 0xb3, 0x6c, 0xa9, 0x8e, 0xde, 0xf0, 0x3b, 0xf5, 0xdb, 0x24, 0x39, 0x2f, 0x53, + 0x3f, 0x89, 0x32, 0x94, 0xf3, 0x53, 0x6e, 0x9e, 0xca, 0xf9, 0x29, 0x3f, 0x33, 0xd1, 0x4f, 0x50, + 0xe7, 0x43, 0xc3, 0x28, 0xe8, 0x14, 0x2d, 0xe7, 0xd1, 0x1b, 0x3f, 0xc0, 0x63, 0xfb, 0x2b, 0x80, + 0xac, 0x69, 0x14, 0xc7, 0xb6, 0xd4, 0xb7, 0x8a, 0x63, 0x5b, 0xee, 0x2d, 0x69, 0x1f, 0x75, 0xe8, + 0xa4, 0x57, 0x6d, 0x17, 0x99, 0x64, 0x11, 0x17, 0xcd, 0x58, 0x2e, 0xe2, 0x6a, 0xf3, 0x98, 0x8f, + 0x78, 0xae, 0xfd, 0xa2, 0xfb, 0xa8, 0xc5, 0x30, 0x76, 0x8a, 0x11, 0xc7, 0x6d, 0xdc, 0x08, 0x17, + 0xfb, 0x97, 0xac, 0x2d, 0x12, 0x7a, 0xaa, 0xba, 0x2a, 0xa1, 0xa7, 0xb2, 0x87, 0x4a, 0x2a, 0x1d, + 0xe9, 0x17, 0xf5, 0xcc, 0x47, 0x6a, 0xb1, 0x23, 0x2f, 0x61, 0x45, 0xf4, 0x39, 0x64, 0x53, 0x0a, + 0x53, 0xe4, 0x13, 0x75, 0x49, 0x0a, 0xfe, 0x01, 0x0a, 0xfe, 0x90, 0xdc, 0x56, 0x42, 0xc9, 0x37, + 0xd0, 0x56, 0x5a, 0x03, 0x51, 0xa7, 0xcb, 0xed, 0x8b, 0xa8, 0xd3, 0x15, 0x3d, 0xc4, 0x42, 0x2f, + 0x31, 0xbe, 0x0b, 0x8f, 0xc5, 0x09, 0xac, 0xab, 0xad, 0x93, 0x28, 0x7a, 0x15, 0x3d, 0x96, 0xa1, + 0x97, 0x19, 0xe9, 0x81, 0x38, 0x85, 0x4e, 0xbe, 0x07, 0x10, 0x67, 0xab, 0xb2, 0xc1, 0x10, 0x67, + 0xab, 0xba, 0x65, 0xa0, 0xb5, 0xa7, 0xfa, 0x5f, 0xdf, 0xf6, 0xb5, 0xef, 0xdf, 0xf6, 0xb5, 0x7f, + 0xbe, 0xed, 0x6b, 0x7f, 0x78, 0xd7, 0xaf, 0x7d, 0xff, 0xae, 0x5f, 0xfb, 0xc7, 0xbb, 0x7e, 0x6d, + 0xb4, 0x82, 0x7f, 0xec, 0xfc, 0xe8, 0x3f, 0x01, 0x00, 0x00, 0xff, 0xff, 0x79, 0xf4, 0x4e, 0x15, + 0x1c, 0x1a, 0x00, 0x00, } // Reference imports to suppress errors if they are not otherwise used. @@ -5632,6 +5650,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) @@ -6943,6 +6981,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 } @@ -12092,6 +12136,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/pbmock/dmmaster.go b/dm/pbmock/dmmaster.go index b5d3c4885f..55c0174116 100644 --- a/dm/pbmock/dmmaster.go +++ b/dm/pbmock/dmmaster.go @@ -375,6 +375,26 @@ func (mr *MockMasterClientMockRecorder) StartTask(arg0, arg1 interface{}, arg2 . return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "StartTask", reflect.TypeOf((*MockMasterClient)(nil).StartTask), varargs...) } +// TransferSource mocks base method +func (m *MockMasterClient) TransferSource(arg0 context.Context, arg1 *pb.TransferSourceRequest, arg2 ...grpc.CallOption) (*pb.TransferSourceResponse, error) { + m.ctrl.T.Helper() + varargs := []interface{}{arg0, arg1} + for _, a := range arg2 { + varargs = append(varargs, a) + } + ret := m.ctrl.Call(m, "TransferSource", varargs...) + ret0, _ := ret[0].(*pb.TransferSourceResponse) + ret1, _ := ret[1].(error) + return ret0, ret1 +} + +// TransferSource indicates an expected call of TransferSource +func (mr *MockMasterClientMockRecorder) TransferSource(arg0, arg1 interface{}, arg2 ...interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + varargs := append([]interface{}{arg0, arg1}, arg2...) + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "TransferSource", reflect.TypeOf((*MockMasterClient)(nil).TransferSource), varargs...) +} + // UnlockDDLLock mocks base method func (m *MockMasterClient) UnlockDDLLock(arg0 context.Context, arg1 *pb.UnlockDDLLockRequest, arg2 ...grpc.CallOption) (*pb.UnlockDDLLockResponse, error) { m.ctrl.T.Helper() @@ -693,6 +713,21 @@ func (mr *MockMasterServerMockRecorder) StartTask(arg0, arg1 interface{}) *gomoc return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "StartTask", reflect.TypeOf((*MockMasterServer)(nil).StartTask), arg0, arg1) } +// TransferSource mocks base method +func (m *MockMasterServer) TransferSource(arg0 context.Context, arg1 *pb.TransferSourceRequest) (*pb.TransferSourceResponse, error) { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "TransferSource", arg0, arg1) + ret0, _ := ret[0].(*pb.TransferSourceResponse) + ret1, _ := ret[1].(error) + return ret0, ret1 +} + +// TransferSource indicates an expected call of TransferSource +func (mr *MockMasterServerMockRecorder) TransferSource(arg0, arg1 interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "TransferSource", reflect.TypeOf((*MockMasterServer)(nil).TransferSource), arg0, arg1) +} + // UnlockDDLLock mocks base method func (m *MockMasterServer) UnlockDDLLock(arg0 context.Context, arg1 *pb.UnlockDDLLockRequest) (*pb.UnlockDDLLockResponse, error) { m.ctrl.T.Helper() diff --git a/dm/proto/dmmaster.proto b/dm/proto/dmmaster.proto index 5121122c78..e6203e1160 100644 --- a/dm/proto/dmmaster.proto +++ b/dm/proto/dmmaster.proto @@ -368,6 +368,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/dm/worker/server.go b/dm/worker/server.go index 1588783561..1134674c02 100644 --- a/dm/worker/server.go +++ b/dm/worker/server.go @@ -557,73 +557,19 @@ func (s *Server) startWorker(cfg *config.SourceConfig) error { return terror.ErrWorkerAlreadyStart.Generate() } - // we get the newest subtask stages directly which will omit the subtask stage PUT/DELETE event - // because triggering these events is useless now - subTaskStages, subTaskCfgm, revSubTask, err := ha.GetSubTaskStageConfig(s.etcdClient, cfg.SourceID) - if err != nil { - return err - } - - subTaskCfgs := make([]*config.SubTaskConfig, 0, len(subTaskCfgm)) - for _, subTaskCfg := range subTaskCfgm { - subTaskCfg.LogLevel = s.cfg.LogLevel - subTaskCfg.LogFile = s.cfg.LogFile - subTaskCfg.LogFormat = s.cfg.LogFormat - subTaskCfgClone := subTaskCfg - if err = copyConfigFromSource(&subTaskCfgClone, cfg); err != nil { - return err - } - subTaskCfgs = append(subTaskCfgs, &subTaskCfgClone) - } - - if cfg.EnableRelay { - dctx, dcancel := context.WithTimeout(s.etcdClient.Ctx(), time.Duration(len(subTaskCfgs))*3*time.Second) - defer dcancel() - minLoc, err1 := getMinLocInAllSubTasks(dctx, subTaskCfgs) - if err1 != nil { - return err1 - } - - if minLoc != nil { - log.L().Info("get min location in all subtasks", zap.Stringer("location", *minLoc)) - cfg.RelayBinLogName = binlog.AdjustPosition(minLoc.Position).Name - cfg.RelayBinlogGTID = minLoc.GTIDSetStr() - // set UUIDSuffix when bound to a source - cfg.UUIDSuffix, err = binlog.ExtractSuffix(minLoc.Position.Name) - if err != nil { - return err - } - } else { - // set UUIDSuffix even not checkpoint exist - // so we will still remove relay dir - cfg.UUIDSuffix = binlog.MinUUIDSuffix - } - } - - log.L().Info("starting to handle mysql source", zap.String("sourceCfg", cfg.String()), zap.Reflect("subTasks", subTaskCfgs)) w, err := NewWorker(cfg, s.etcdClient, s.cfg.Name) if err != nil { return err } s.setWorker(w, false) - startRelay := false - var revRelay int64 if cfg.EnableRelay { - var relayStage ha.Stage - // we get the newest relay stages directly which will omit the relay stage PUT/DELETE event - // because triggering these events is useless now - relayStage, revRelay, err = ha.GetRelayStage(s.etcdClient, cfg.SourceID) - if err != nil { - // TODO: need retry - return err - } - startRelay = !relayStage.IsDeleted && relayStage.Expect == pb.Stage_Running s.UpdateKeepAliveTTL(s.cfg.RelayKeepAliveTTL) + if err2 := w.EnableRelay(); err2 != nil { + return err2 + } } - go func() { - w.Start(startRelay) - }() + go w.Start() isStarted := utils.WaitSomething(50, 100*time.Millisecond, func() bool { return w.closed.Get() == closedFalse @@ -633,37 +579,9 @@ func (s *Server) startWorker(cfg *config.SourceConfig) error { return terror.ErrWorkerNoStart } - for _, subTaskCfg := range subTaskCfgs { - expectStage := subTaskStages[subTaskCfg.Name] - if expectStage.IsDeleted { - continue - } - log.L().Info("start to create subtask", zap.String("sourceID", subTaskCfg.SourceID), zap.String("task", subTaskCfg.Name)) - if err := w.StartSubTask(subTaskCfg, expectStage.Expect); err != nil { - return err - } - } - - w.wg.Add(1) - go func() { - defer w.wg.Done() - // TODO: handle fatal error from observeSubtaskStage - //nolint:errcheck - w.observeSubtaskStage(w.ctx, s.etcdClient, revSubTask) - }() - - if cfg.EnableRelay { - w.wg.Add(1) - go func() { - defer w.wg.Done() - // TODO: handle fatal error from observeRelayStage - //nolint:errcheck - w.observeRelayStage(w.ctx, s.etcdClient, revRelay) - }() - } - + err = w.EnableHandleSubtasks() log.L().Info("started to handle mysql source", zap.String("sourceCfg", cfg.String())) - return nil + return err } func makeCommonWorkerResponse(reqErr error) *pb.CommonWorkerResponse { @@ -679,7 +597,7 @@ func makeCommonWorkerResponse(reqErr error) *pb.CommonWorkerResponse { // all subTask in subTaskCfgs should have same source // this function return the min location in all subtasks, used for relay's location -func getMinLocInAllSubTasks(ctx context.Context, subTaskCfgs []*config.SubTaskConfig) (minLoc *binlog.Location, err error) { +func getMinLocInAllSubTasks(ctx context.Context, subTaskCfgs map[string]config.SubTaskConfig) (minLoc *binlog.Location, err error) { for _, subTaskCfg := range subTaskCfgs { loc, err := getMinLocForSubTaskFunc(ctx, subTaskCfg) if err != nil { @@ -702,7 +620,7 @@ func getMinLocInAllSubTasks(ctx context.Context, subTaskCfgs []*config.SubTaskCo return minLoc, nil } -func getMinLocForSubTask(ctx context.Context, subTaskCfg *config.SubTaskConfig) (minLoc *binlog.Location, err error) { +func getMinLocForSubTask(ctx context.Context, subTaskCfg config.SubTaskConfig) (minLoc *binlog.Location, err error) { if subTaskCfg.Mode == config.ModeFull { return nil, nil } diff --git a/dm/worker/server_test.go b/dm/worker/server_test.go index 3a05543d39..a040770a9f 100644 --- a/dm/worker/server_test.go +++ b/dm/worker/server_test.go @@ -532,22 +532,20 @@ func (t *testServer) testStopWorkerWhenLostConnect(c *C, s *Server, ETCD *embed. } func (t *testServer) TestGetMinLocInAllSubTasks(c *C) { - subTaskCfg := []*config.SubTaskConfig{ - { - Name: "test2", - }, { - Name: "test3", - }, { - Name: "test1", - }, + + subTaskCfg := map[string]config.SubTaskConfig{ + "test2": {Name: "test2"}, + "test3": {Name: "test3"}, + "test1": {Name: "test1"}, } minLoc, err := getMinLocInAllSubTasks(context.Background(), subTaskCfg) c.Assert(err, IsNil) c.Assert(minLoc.Position.Name, Equals, "mysql-binlog.00001") c.Assert(minLoc.Position.Pos, Equals, uint32(12)) - for _, subtask := range subTaskCfg { - subtask.EnableGTID = true + for k, cfg := range subTaskCfg { + cfg.EnableGTID = true + subTaskCfg[k] = cfg } minLoc, err = getMinLocInAllSubTasks(context.Background(), subTaskCfg) @@ -666,7 +664,7 @@ func (t *testServer) TestUnifyMasterBinlogPos(c *C) { c.Assert(relay.RelayCatchUpMaster, IsTrue) } -func getFakeLocForSubTask(ctx context.Context, subTaskCfg *config.SubTaskConfig) (minLoc *binlog.Location, err error) { +func getFakeLocForSubTask(ctx context.Context, subTaskCfg config.SubTaskConfig) (minLoc *binlog.Location, err error) { gset1, _ := gtid.ParserGTID(mysql.MySQLFlavor, "ba8f633f-1f15-11eb-b1c7-0242ac110001:1-30") gset2, _ := gtid.ParserGTID(mysql.MySQLFlavor, "ba8f633f-1f15-11eb-b1c7-0242ac110001:1-50") gset3, _ := gtid.ParserGTID(mysql.MySQLFlavor, "ba8f633f-1f15-11eb-b1c7-0242ac110001:1-50,ba8f633f-1f15-11eb-b1c7-0242ac110002:1") diff --git a/dm/worker/worker.go b/dm/worker/worker.go index ec40e46cb9..4c66dd1404 100644 --- a/dm/worker/worker.go +++ b/dm/worker/worker.go @@ -28,6 +28,7 @@ import ( "github.com/pingcap/dm/dm/config" "github.com/pingcap/dm/dm/pb" + "github.com/pingcap/dm/pkg/binlog" "github.com/pingcap/dm/pkg/etcdutil" "github.com/pingcap/dm/pkg/ha" "github.com/pingcap/dm/pkg/log" @@ -68,7 +69,8 @@ type Worker struct { name string } -// NewWorker creates a new Worker +// NewWorker creates a new Worker. The functionality of relay and subtask is disabled by default, need call EnableRelay +// and EnableSubtask later func NewWorker(cfg *config.SourceConfig, etcdClient *clientv3.Client, name string) (w *Worker, err error) { w = &Worker{ cfg: cfg, @@ -89,18 +91,6 @@ func NewWorker(cfg *config.SourceConfig, etcdClient *clientv3.Client, name strin } }(w) - if cfg.EnableRelay { - // initial relay holder, the cfg's password need decrypt - w.relayHolder = NewRelayHolder(cfg) - purger1, err1 := w.relayHolder.Init([]purger.PurgeInterceptor{ - w, - }) - if err1 != nil { - return nil, err1 - } - w.relayPurger = purger1 - } - // initial task status checker if w.cfg.Checker.CheckEnable { tsc := NewTaskStatusChecker(w.cfg.Checker, w) @@ -119,17 +109,7 @@ func NewWorker(cfg *config.SourceConfig, etcdClient *clientv3.Client, name strin } // Start starts working -func (w *Worker) Start(startRelay bool) { - - if w.cfg.EnableRelay && startRelay { - log.L().Info("relay is started") - // start relay - w.relayHolder.Start() - - // start purger - w.relayPurger.Start() - } - +func (w *Worker) Start() { // start task status checker if w.cfg.Checker.CheckEnable { w.taskStatusChecker.Start() @@ -190,6 +170,122 @@ func (w *Worker) Close() { w.l.Info("Stop worker") } +// EnableRelay enables the functionality of start/watch/handle relay +func (w *Worker) EnableRelay() error { + // 1. adjust relay starting position, to the earliest of subtasks + _, subTaskCfgs, _, err := w.fetchSubTasksAndAdjust() + if err != nil { + return err + } + + dctx, dcancel := context.WithTimeout(w.etcdClient.Ctx(), time.Duration(len(subTaskCfgs))*3*time.Second) + defer dcancel() + minLoc, err1 := getMinLocInAllSubTasks(dctx, subTaskCfgs) + if err1 != nil { + return err1 + } + + if minLoc != nil { + log.L().Info("get min location in all subtasks", zap.Stringer("location", *minLoc)) + w.cfg.RelayBinLogName = binlog.AdjustPosition(minLoc.Position).Name + w.cfg.RelayBinlogGTID = minLoc.GTIDSetStr() + // set UUIDSuffix when bound to a source + w.cfg.UUIDSuffix, err = binlog.ExtractSuffix(minLoc.Position.Name) + if err != nil { + return err + } + } else { + // set UUIDSuffix even not checkpoint exist + // so we will still remove relay dir + w.cfg.UUIDSuffix = binlog.MinUUIDSuffix + } + + // 2. initial relay holder, the cfg's password need decrypt + w.relayHolder = NewRelayHolder(w.cfg) + relayPurger, err := w.relayHolder.Init([]purger.PurgeInterceptor{ + w, + }) + if err != nil { + return err + } + w.relayPurger = relayPurger + + // 3. get relay stage from etcd and check if need starting + // we get the newest relay stages directly which will omit the relay stage PUT/DELETE event + // because triggering these events is useless now + relayStage, revRelay, err := ha.GetRelayStage(w.etcdClient, w.cfg.SourceID) + if err != nil { + // TODO: need retry + return err + } + startImmediately := !relayStage.IsDeleted && relayStage.Expect == pb.Stage_Running + if startImmediately { + log.L().Info("relay is started") + w.relayHolder.Start() + w.relayPurger.Start() + } + + // 4. watch relay stage + w.wg.Add(1) + go func() { + defer w.wg.Done() + // TODO: handle fatal error from observeRelayStage + //nolint:errcheck + w.observeRelayStage(w.ctx, w.etcdClient, revRelay) + }() + return nil +} + +// EnableHandleSubtasks enables the functionality of start/watch/handle subtasks +func (w *Worker) EnableHandleSubtasks() error { + subTaskStages, subTaskCfgM, revSubTask, err := w.fetchSubTasksAndAdjust() + if err != nil { + return err + } + + log.L().Info("starting to handle mysql source", zap.String("sourceCfg", w.cfg.String()), zap.Any("subTasks", subTaskCfgM)) + + for _, subTaskCfg := range subTaskCfgM { + expectStage := subTaskStages[subTaskCfg.Name] + if expectStage.IsDeleted { + continue + } + log.L().Info("start to create subtask", zap.String("sourceID", subTaskCfg.SourceID), zap.String("task", subTaskCfg.Name)) + // for range of a map will use a same value-address, so we'd better not pass value-address to other function + clone := subTaskCfg + if err := w.StartSubTask(&clone, expectStage.Expect); err != nil { + return err + } + } + + w.wg.Add(1) + go func() { + defer w.wg.Done() + // TODO: handle fatal error from observeSubtaskStage + //nolint:errcheck + w.observeSubtaskStage(w.ctx, w.etcdClient, revSubTask) + }() + + return nil +} + +// fetchSubTasksAndAdjust gets source's subtask stages and configs, adjust some values by worker's config and status +// source **must not be empty** +// return map{task name -> subtask stage}, map{task name -> subtask config}, revision, error. +func (w *Worker) fetchSubTasksAndAdjust() (map[string]ha.Stage, map[string]config.SubTaskConfig, int64, error) { + // we get the newest subtask stages directly which will omit the subtask stage PUT/DELETE event + // because triggering these events is useless now + subTaskStages, subTaskCfgM, revSubTask, err := ha.GetSubTaskStageConfig(w.etcdClient, w.cfg.SourceID) + if err != nil { + return nil, nil, 0, err + } + + if err = copyConfigFromSourceForEach(subTaskCfgM, w.cfg); err != nil { + return nil, nil, 0, err + } + return subTaskStages, subTaskCfgM, revSubTask, nil +} + // StartSubTask creates a sub task an run it func (w *Worker) StartSubTask(cfg *config.SubTaskConfig, expectStage pb.Stage) error { w.Lock() @@ -298,8 +394,8 @@ func (w *Worker) QueryStatus(ctx context.Context, name string) []*pb.SubTaskStat return w.Status(ctx2, name) } -func (w *Worker) resetSubtaskStage(etcdCli *clientv3.Client) (int64, error) { - subTaskStages, subTaskCfgm, revSubTask, err := ha.GetSubTaskStageConfig(etcdCli, w.cfg.SourceID) +func (w *Worker) resetSubtaskStage() (int64, error) { + subTaskStages, subTaskCfgm, revSubTask, err := w.fetchSubTasksAndAdjust() if err != nil { return 0, err } @@ -361,7 +457,7 @@ func (w *Worker) observeSubtaskStage(ctx context.Context, etcdCli *clientv3.Clie case <-ctx.Done(): return nil case <-time.After(500 * time.Millisecond): - rev, err = w.resetSubtaskStage(etcdCli) + rev, err = w.resetSubtaskStage() if err != nil { log.L().Error("resetSubtaskStage is failed, will retry later", zap.Error(err), zap.Int("retryNum", retryNum)) } @@ -663,6 +759,17 @@ func copyConfigFromSource(cfg *config.SubTaskConfig, sourceCfg *config.SourceCon return nil } +// copyConfigFromSourceForEach do copyConfigFromSource for each value in subTaskCfgM and change subTaskCfgM in-place +func copyConfigFromSourceForEach(subTaskCfgM map[string]config.SubTaskConfig, sourceCfg *config.SourceConfig) error { + for k, subTaskCfg := range subTaskCfgM { + if err2 := copyConfigFromSource(&subTaskCfg, sourceCfg); err2 != nil { + return err2 + } + subTaskCfgM[k] = subTaskCfg + } + return nil +} + // getAllSubTaskStatus returns all subtask status of this worker, note the field // in subtask status is not completed, only includes `Name`, `Stage` and `Result` now func (w *Worker) getAllSubTaskStatus() map[string]*pb.SubTaskStatus { diff --git a/dm/worker/worker_test.go b/dm/worker/worker_test.go index 25b7074f43..35226e18cc 100644 --- a/dm/worker/worker_test.go +++ b/dm/worker/worker_test.go @@ -47,11 +47,34 @@ func (t *testServer) testWorker(c *C) { NewRelayHolder = NewRealRelayHolder }() - _, err := NewWorker(&cfg, nil, "") - c.Assert(err, ErrorMatches, "init error") + var ( + masterAddr = tempurl.Alloc()[len("http://"):] + keepAliveTTL = int64(1) + ) + etcdDir := c.MkDir() + ETCD, err := createMockETCD(etcdDir, "http://"+masterAddr) + c.Assert(err, IsNil) + defer ETCD.Close() + workerCfg := NewConfig() + c.Assert(workerCfg.Parse([]string{"-config=./dm-worker.toml"}), IsNil) + workerCfg.Join = masterAddr + workerCfg.KeepAliveTTL = keepAliveTTL + workerCfg.RelayKeepAliveTTL = keepAliveTTL + + etcdCli, err := clientv3.New(clientv3.Config{ + Endpoints: GetJoinURLs(workerCfg.Join), + DialTimeout: dialTimeout, + DialKeepAliveTime: keepaliveTime, + DialKeepAliveTimeout: keepaliveTimeout, + }) + c.Assert(err, IsNil) + + w, err := NewWorker(&cfg, etcdCli, "") + c.Assert(err, IsNil) + c.Assert(w.EnableRelay(), ErrorMatches, "init error") NewRelayHolder = NewDummyRelayHolder - w, err := NewWorker(&cfg, nil, "") + w, err = NewWorker(&cfg, etcdCli, "") c.Assert(err, IsNil) c.Assert(w.StatusJSON(context.Background(), ""), HasLen, emptyWorkerStatusInfoJSONLength) //c.Assert(w.closed.Get(), Equals, closedFalse) @@ -252,7 +275,7 @@ func (t *testWorkerEtcdCompact) TestWatchSubtaskStageEtcdCompact(c *C) { defer cancel() defer w.Close() go func() { - w.Start(false) + w.Start() }() c.Assert(utils.WaitSomething(50, 100*time.Millisecond, func() bool { return w.closed.Get() == closedFalse @@ -366,7 +389,8 @@ func (t *testWorkerEtcdCompact) TestWatchRelayStageEtcdCompact(c *C) { defer cancel() defer w.Close() go func() { - w.Start(true) + c.Assert(w.EnableRelay(), IsNil) + w.Start() }() c.Assert(utils.WaitSomething(50, 100*time.Millisecond, func() bool { return w.closed.Get() == closedFalse @@ -376,18 +400,18 @@ func (t *testWorkerEtcdCompact) TestWatchRelayStageEtcdCompact(c *C) { c.Assert(w.relayHolder, NotNil) _, err = ha.PutSourceCfg(etcdCli, sourceCfg) c.Assert(err, IsNil) - _, err = ha.PutRelayStageSourceBound(etcdCli, ha.NewRelayStage(pb.Stage_Running, sourceCfg.SourceID), + rev, err := ha.PutRelayStageSourceBound(etcdCli, ha.NewRelayStage(pb.Stage_Running, sourceCfg.SourceID), ha.NewSourceBound(sourceCfg.SourceID, cfg.Name)) c.Assert(err, IsNil) - rev, err := ha.DeleteSourceCfgRelayStageSourceBound(etcdCli, sourceCfg.SourceID, cfg.Name) - c.Assert(err, IsNil) // check relay stage, should be running c.Assert(utils.WaitSomething(30, 100*time.Millisecond, func() bool { return w.relayHolder.Stage() == pb.Stage_Running }), IsTrue) - // step 3: trigger etcd compaction and check whether we can receive it through watcher + // step 3: trigger etcd compaction and check whether we can receive it through watcher, then we delete relay stage _, err = etcdCli.Compact(ctx, rev) c.Assert(err, IsNil) + _, err = ha.DeleteSourceCfgRelayStageSourceBound(etcdCli, sourceCfg.SourceID, cfg.Name) + c.Assert(err, IsNil) relayStageCh := make(chan ha.Stage, 10) relayErrCh := make(chan error, 10) ha.WatchRelayStage(ctx, etcdCli, cfg.Name, startRev, relayStageCh, relayErrCh) @@ -397,19 +421,9 @@ func (t *testWorkerEtcdCompact) TestWatchRelayStageEtcdCompact(c *C) { case <-time.After(300 * time.Millisecond): c.Fatal("fail to get etcd error compacted") } - // step 4: watch relay stage from startRev - var wg sync.WaitGroup - ctx1, cancel1 := context.WithCancel(ctx) - wg.Add(1) - go func() { - defer wg.Done() - c.Assert(w.observeRelayStage(ctx1, etcdCli, startRev), IsNil) - }() - // step 5: should stop the running relay + // step 4: should stop the running relay because see deletion after compaction time.Sleep(time.Second) c.Assert(utils.WaitSomething(30, 100*time.Millisecond, func() bool { return w.relayHolder.Stage() == pb.Stage_Stopped }), IsTrue) - cancel1() - wg.Wait() } diff --git a/go.sum b/go.sum index e174fb820a..1763fb0d03 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,17 +170,14 @@ 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 h1:pAWNwdf7QiT1zfaWyqCtNZQWCLByQyA3JrSQyuYAqnQ= 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= @@ -213,10 +194,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= @@ -278,7 +257,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= @@ -289,12 +267,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= @@ -324,31 +300,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= @@ -367,9 +337,7 @@ 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= github.com/gogo/protobuf v1.3.2 h1:Ov1cvc58UF3b5XjBnZv7+opcTcQFZebYjWzi34vdm4Q= github.com/gogo/protobuf v1.3.2/go.mod h1:P1XiOD3dCwIKUDQYPy72D8LYyHL2YPYrpS2s69NZV8Q= @@ -380,38 +348,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= @@ -421,9 +379,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= @@ -432,7 +388,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= @@ -455,19 +410,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= @@ -477,7 +429,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= @@ -514,7 +465,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= @@ -527,9 +477,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= @@ -541,11 +489,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= @@ -554,7 +500,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= @@ -566,19 +511,15 @@ 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/errcheck v1.5.0/go.mod h1:pFxgyoBC7bSaBwPgfKdkLd5X25qrDl4LWUI2bnpBCr8= 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= @@ -586,20 +527,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= @@ -616,13 +553,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= @@ -632,14 +567,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= @@ -652,12 +585,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= @@ -669,7 +599,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= @@ -690,18 +619,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= @@ -713,23 +636,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= @@ -743,14 +662,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= @@ -762,11 +678,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= @@ -776,21 +689,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= @@ -800,35 +707,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= @@ -845,7 +740,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= @@ -853,29 +747,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= @@ -891,27 +775,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= @@ -931,9 +808,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= @@ -946,26 +821,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= @@ -974,7 +843,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= @@ -982,11 +850,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= @@ -995,7 +860,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= @@ -1004,7 +868,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= @@ -1019,7 +882,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= @@ -1028,59 +890,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= @@ -1091,7 +936,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= @@ -1104,7 +948,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= @@ -1116,11 +959,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= @@ -1132,9 +972,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= @@ -1146,13 +984,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= @@ -1161,9 +997,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= @@ -1173,26 +1007,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= @@ -1210,11 +1039,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= @@ -1237,7 +1063,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= @@ -1245,37 +1070,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= @@ -1284,37 +1101,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= @@ -1324,15 +1130,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= @@ -1345,7 +1147,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= @@ -1362,7 +1163,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= @@ -1372,10 +1172,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= @@ -1389,7 +1187,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= @@ -1398,7 +1195,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= @@ -1410,14 +1206,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= @@ -1425,19 +1218,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= @@ -1483,31 +1272,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= @@ -1522,7 +1304,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= @@ -1540,7 +1321,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= @@ -1560,26 +1340,21 @@ 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-20200619180055-7c47624df98f/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.0.0-20210106214847-113979e3529a/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= @@ -1596,14 +1371,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= @@ -1628,7 +1400,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= @@ -1644,27 +1415,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= @@ -1691,7 +1455,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= @@ -1705,13 +1468,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= @@ -1721,15 +1482,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= @@ -1750,12 +1506,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/info.go b/pkg/shardddl/optimism/info.go index 23b0319888..0d500bff95 100644 --- a/pkg/shardddl/optimism/info.go +++ b/pkg/shardddl/optimism/info.go @@ -54,6 +54,9 @@ type Info struct { // only set it when get/watch from etcd Version int64 `json:"-"` + // use to resolve conflict + IgnoreConflict bool `json:"ignore-conflict"` + // only set it when get/watch from etcd ModRevision int64 `json:"-"` } diff --git a/pkg/shardddl/optimism/info_test.go b/pkg/shardddl/optimism/info_test.go index 6831c53da9..d07edc097c 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/lock.go b/pkg/shardddl/optimism/lock.go index f55b2d6456..ab8384f167 100644 --- a/pkg/shardddl/optimism/lock.go +++ b/pkg/shardddl/optimism/lock.go @@ -102,12 +102,13 @@ func NewLock(cli *clientv3.Client, ID, task, downSchema, downTable string, ti *m // for intrusive, a DML prune or transform mechanism needed for two different schemas (before and after the conflict resolved). 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 + 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() { @@ -173,18 +174,23 @@ func (l *Lock) TrySync(info Info, tts []TargetTable) (newDDLs []string, err erro oldJoined := l.joined 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", l.tables[callerSource][callerSchema][callerTable]), zap.Stringer("to", lastTableInfo), zap.Strings("ddls", ddls)) + l.tables[callerSource][callerSchema][callerTable] = lastTableInfo + } + }() + lastJoined, err := joinTable(lastTableInfo) if err != nil { return emptyDDLs, err } 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/pkg/shardddl/optimism/lock_test.go b/pkg/shardddl/optimism/lock_test.go index 4d003a807d..e547b65028 100644 --- a/pkg/shardddl/optimism/lock_test.go +++ b/pkg/shardddl/optimism/lock_test.go @@ -102,8 +102,8 @@ 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 := 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) c.Assert(l.versions, DeepEquals, vers) @@ -121,8 +121,8 @@ func (t *testLock) TestLockTrySyncNormal(c *C) { t.checkLockNoDone(c, l) // 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 := 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) c.Assert(l.versions, DeepEquals, vers) @@ -131,8 +131,8 @@ 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 = 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) c.Assert(l.versions, DeepEquals, vers) @@ -141,8 +141,8 @@ 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]]++ - 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 = 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) c.Assert(l.versions, DeepEquals, vers) @@ -151,8 +151,8 @@ 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]]++ - 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 = 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]) c.Assert(l.versions, DeepEquals, vers) @@ -168,8 +168,8 @@ func (t *testLock) TestLockTrySyncNormal(c *C) { c.Assert(cmp, Equals, 1) // 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 = 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. c.Assert(l.versions, DeepEquals, vers) @@ -177,8 +177,8 @@ 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]]++ - 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 = 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]) c.Assert(l.versions, DeepEquals, vers) @@ -193,8 +193,8 @@ func (t *testLock) TestLockTrySyncNormal(c *C) { c.Assert(cmp, Equals, 0) // 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 = 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]) c.Assert(l.versions, DeepEquals, vers) @@ -232,8 +232,8 @@ func (t *testLock) TestLockTrySyncNormal(c *C) { for _, db := range dbs { 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 = 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) synced, remain = l.IsSynced() @@ -255,8 +255,8 @@ 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 = 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{}) c.Assert(l.versions, DeepEquals, vers) @@ -265,8 +265,8 @@ 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]]++ - 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 = 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]) c.Assert(l.versions, DeepEquals, vers) @@ -275,8 +275,8 @@ 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]]++ - 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 = 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) @@ -288,15 +288,15 @@ func (t *testLock) TestLockTrySyncNormal(c *C) { c.Assert(cmp, Equals, -1) // 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 = 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]]++ - 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 = 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{}) c.Assert(l.versions, DeepEquals, vers) @@ -308,8 +308,8 @@ func (t *testLock) TestLockTrySyncNormal(c *C) { c.Assert(cmp, Equals, 0) // 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 = 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{}) c.Assert(l.versions, DeepEquals, vers) @@ -320,7 +320,8 @@ 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 = 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) remain-- @@ -376,8 +377,8 @@ 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 := 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) @@ -387,8 +388,8 @@ 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]]++ - DDLs, err = l.TrySync(source, db, tbls[1], DDLs1, []*model.TableInfo{ti1}, tts, 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) @@ -396,8 +397,8 @@ 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 = 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 c.Assert(l.versions, DeepEquals, vers) @@ -407,8 +408,8 @@ 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]]++ - DDLs, err = l.TrySync(source, db, tbls[1], DDLs2, []*model.TableInfo{ti2}, tts, 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) c.Assert(l.versions, DeepEquals, vers) @@ -455,29 +456,29 @@ 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 := 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]]++ - DDLs, err = l.TrySync(source, db, tbls[1], DDLs1, []*model.TableInfo{ti1}, tts, 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]]++ - DDLs, err = l.TrySync(source, db, tbls[0], DDLs2, []*model.TableInfo{ti2}, tts, 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]]++ - DDLs, err = l.TrySync(source, db, tbls[1], DDLs2, []*model.TableInfo{ti2}, tts, 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) c.Assert(l.versions, DeepEquals, vers) @@ -521,15 +522,15 @@ 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]]++ - DDLs, err := l.TrySync(source, db, tbls[0], DDLs1, []*model.TableInfo{ti1}, tts, 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]]++ - DDLs, err = l.TrySync(source, db, tbls[1], DDLs1, []*model.TableInfo{ti1}, tts, 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) @@ -572,8 +573,8 @@ func (t *testLock) TestLockTrySyncNoDiff(c *C) { t.checkLockNoDone(c, l) // 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 := 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{}) c.Assert(l.versions, DeepEquals, vers) @@ -616,8 +617,8 @@ func (t *testLock) TestLockTrySyncNewTable(c *C) { t.checkLockNoDone(c, l) // 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 := 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) c.Assert(l.versions, DeepEquals, vers) @@ -640,8 +641,8 @@ func (t *testLock) TestLockTrySyncNewTable(c *C) { vers[source1][db1][tbl2] = 0 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 = 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) c.Assert(l.versions, DeepEquals, vers) @@ -708,8 +709,8 @@ 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 := 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) @@ -724,8 +725,8 @@ func (t *testLock) TestLockTrySyncRevert(c *C) { c.Assert(cmp, Equals, -1) // 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 = 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) @@ -734,8 +735,8 @@ 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 = 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) c.Assert(l.versions, DeepEquals, vers) @@ -750,8 +751,8 @@ func (t *testLock) TestLockTrySyncRevert(c *C) { c.Assert(cmp, Equals, -1) // 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 = 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) c.Assert(l.versions, DeepEquals, vers) @@ -765,8 +766,8 @@ func (t *testLock) TestLockTrySyncRevert(c *C) { c.Assert(cmp, Equals, -1) // 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 = 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) c.Assert(l.versions, DeepEquals, vers) @@ -775,8 +776,8 @@ 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 = 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) c.Assert(l.versions, DeepEquals, vers) @@ -790,8 +791,8 @@ func (t *testLock) TestLockTrySyncRevert(c *C) { c.Assert(cmp, Equals, -1) // 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 = 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) c.Assert(l.versions, DeepEquals, vers) @@ -805,8 +806,8 @@ func (t *testLock) TestLockTrySyncRevert(c *C) { c.Assert(cmp, Equals, -1) // 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 = 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) c.Assert(l.versions, DeepEquals, vers) @@ -854,8 +855,8 @@ func (t *testLock) TestLockTrySyncConflictNonIntrusive(c *C) { t.checkLockNoDone(c, l) // 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 := 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) @@ -870,8 +871,8 @@ 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]]++ - DDLs, err = l.TrySync(source, db, tbls[1], DDLs2, []*model.TableInfo{ti2_1, ti2}, tts, 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{}) c.Assert(l.versions, DeepEquals, vers) @@ -883,8 +884,8 @@ 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]]++ - DDLs, err = l.TrySync(source, db, tbls[0], DDLs3, []*model.TableInfo{ti3}, tts, 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) c.Assert(l.versions, DeepEquals, vers) @@ -899,8 +900,8 @@ func (t *testLock) TestLockTrySyncConflictNonIntrusive(c *C) { c.Assert(cmp, Equals, 0) // 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 = 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) c.Assert(l.versions, DeepEquals, vers) @@ -911,8 +912,8 @@ func (t *testLock) TestLockTrySyncConflictNonIntrusive(c *C) { c.Assert(ready[source][db][tbls[1]], IsTrue) // 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 = 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) c.Assert(l.versions, DeepEquals, vers) @@ -970,8 +971,8 @@ 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 := 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) @@ -986,8 +987,8 @@ 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]]++ - DDLs, err = l.TrySync(source, db, tbls[1], DDLs2, []*model.TableInfo{ti3, ti2}, tts, 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{}) c.Assert(l.versions, DeepEquals, vers) @@ -999,8 +1000,8 @@ func (t *testLock) TestLockTrySyncConflictIntrusive(c *C) { c.Assert(ready[source][db][tbls[1]], IsFalse) // 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 = 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{}) c.Assert(l.versions, DeepEquals, vers) @@ -1009,8 +1010,8 @@ 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]]++ - DDLs, err = l.TrySync(source, db, tbls[1], DDLs3, []*model.TableInfo{ti3}, tts, 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{}) c.Assert(l.versions, DeepEquals, vers) @@ -1021,8 +1022,8 @@ 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]]++ - DDLs, err = l.TrySync(source, db, tbls[1], DDLs4, []*model.TableInfo{ti4}, tts, 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{}) c.Assert(l.versions, DeepEquals, vers) @@ -1033,8 +1034,8 @@ 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]]++ - DDLs, err = l.TrySync(source, db, tbls[1], DDLs1, []*model.TableInfo{ti1}, tts, 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) c.Assert(l.versions, DeepEquals, vers) @@ -1046,8 +1047,8 @@ 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 = 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) c.Assert(l.versions, DeepEquals, vers) @@ -1062,8 +1063,8 @@ 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]]++ - DDLs, err = l.TrySync(source, db, tbls[1], DDLs6, []*model.TableInfo{ti6_1, ti6}, tts, 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{}) cmp, err = l.tables[source][db][tbls[1]].Compare(l.Joined()) @@ -1075,8 +1076,8 @@ 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 = 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. c.Assert(l.versions, DeepEquals, vers) @@ -1091,16 +1092,16 @@ func (t *testLock) TestLockTrySyncConflictIntrusive(c *C) { c.Assert(cmp, Equals, -1) // 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 = 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) ready = l.Ready() c.Assert(ready[source][db][tbls[0]], IsTrue) // 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 = 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) ready = l.Ready() @@ -1164,13 +1165,13 @@ func (t *testLock) TestLockTrySyncMultipleChangeDDL(c *C) { t.checkLockNoDone(c, l) // 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 := 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]]++ - DDLs, err = l.TrySync(sources[0], dbs[0], tbls[0], DDLs1, []*model.TableInfo{ti1}, tts, 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) @@ -1192,8 +1193,8 @@ func (t *testLock) TestLockTrySyncMultipleChangeDDL(c *C) { for _, source := range sources { 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 = 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]) c.Assert(l.versions, DeepEquals, vers) @@ -1211,8 +1212,8 @@ 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 = 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) c.Assert(l.versions, DeepEquals, vers) @@ -1234,8 +1235,8 @@ func (t *testLock) TestLockTrySyncMultipleChangeDDL(c *C) { for _, source := range sources { 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 = 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]) c.Assert(l.versions, DeepEquals, vers) @@ -1253,8 +1254,8 @@ 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 = 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:]) c.Assert(l.versions, DeepEquals, vers) @@ -1298,8 +1299,8 @@ 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 := 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) c.Assert(l.versions, DeepEquals, vers) @@ -1322,8 +1323,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]) + 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) c.Assert(l.versions, DeepEquals, vers) @@ -1387,8 +1389,8 @@ 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]]++ - DDLs, err := l.TrySync(source, db, tbls[0], DDLs1, []*model.TableInfo{ti1}, tts, 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) @@ -1402,8 +1404,8 @@ 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]]++ - DDLs, err = l.TrySync(source, db, tbls[1], DDLs2, []*model.TableInfo{ti1, ti2}, tts, 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) c.Assert(l.versions, DeepEquals, vers) @@ -1421,8 +1423,8 @@ 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]]++ - DDLs, err = l.TrySync(source, db, tbls[0], DDLs3, []*model.TableInfo{ti3}, tts, 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) c.Assert(l.versions, DeepEquals, vers) @@ -1501,8 +1503,8 @@ 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]]++ - DDLs, err := l.TrySync(source, db, tbls[0], DDLs1, []*model.TableInfo{ti1}, tts, 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) @@ -1510,8 +1512,8 @@ 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]]++ - DDLs, err = l.TrySync(source, db, tbls[1], DDLs2, []*model.TableInfo{ti2}, tts, 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, ErrorMatches, ".*add columns with different field lengths.*") c.Assert(DDLs, DeepEquals, DDLs2) c.Assert(l.versions, DeepEquals, vers) @@ -1521,7 +1523,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) @@ -1529,8 +1533,8 @@ 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]]++ - DDLs, err = l.TrySync(source, db, tbls[0], DDLs1, []*model.TableInfo{ti1}, tts, 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) c.Assert(l.versions, DeepEquals, vers) @@ -1665,7 +1669,8 @@ 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 := 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]) } @@ -1699,3 +1704,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 +} diff --git a/syncer/checkpoint.go b/syncer/checkpoint.go index ce87c1ba93..89c7ff4c84 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" @@ -211,6 +212,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, sourceSchema, sourceTable 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 +557,43 @@ func (cp *RemoteCheckPoint) FlushPointsExcept(tctx *tcontext.Context, exceptTabl return nil } +// FlushPointWithTableInfo implements CheckPoint.FlushPointWithTableInfo +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[sourceSchema][sourceTable] + + tiBytes, err := json.Marshal(ti) + if err != nil { + return terror.ErrSchemaTrackerCannotSerialize.Delegate(err, sourceSchema, sourceTable) + } + + location := point.MySQLLocation() + sql2, arg := cp.genUpdateSQL(sourceSchema, sourceTable, 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(utils.DefaultDBTimeout) + 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..6d0d3526fd 100644 --- a/syncer/schema.go +++ b/syncer/schema.go @@ -20,8 +20,12 @@ 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" + 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 +78,39 @@ func (s *Syncer) OperateSchema(ctx context.Context, req *pb.OperateWorkerSchemaR if err != nil { return "", terror.ErrSchemaTrackerCannotCreateTable.Delegate(err, req.Database, req.Table) } - return "", nil + + if !req.Flush && !req.Sync { + break + } + + ti, err := s.schemaTracker.GetTable(req.Database, req.Table) + 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 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("sync info with operate-schema", zap.Stringer("info", info)) + _, 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/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);" diff --git a/tests/sequence_sharding_optimistic/run.sh b/tests/sequence_sharding_optimistic/run.sh index 447289b785..7b071351a6 100755 --- a/tests/sequence_sharding_optimistic/run.sh +++ b/tests/sequence_sharding_optimistic/run.sh @@ -156,8 +156,8 @@ run() { # try to get schema for the table, the latest schema got. curl -X PUT ${API_URL} -d '{"op":1, "task":"sequence_sharding_optimistic", "sources": ["mysql-replica-01"], "database":"sharding_seq_opt", "table":"t1"}' > ${WORK_DIR}/get_schema.log - # downstream does not enable alter-primary-key, so this is NONCLUSTERED index - check_log_contains ${WORK_DIR}/get_schema.log 'CREATE TABLE `t1` ( `id` bigint(20) NOT NULL, `c2` varchar(20) DEFAULT NULL, `c3` int(11) DEFAULT NULL, PRIMARY KEY (`id`) /*T![clustered_index] NONCLUSTERED */) ENGINE=InnoDB DEFAULT CHARSET=latin1 COLLATE=latin1_bin' 1 + # downstream does not enable alter-primary-key, so this is CLUSTERED index + check_log_contains ${WORK_DIR}/get_schema.log 'CREATE TABLE `t1` ( `id` bigint(20) NOT NULL, `c2` varchar(20) DEFAULT NULL, `c3` int(11) DEFAULT NULL, PRIMARY KEY (`id`) /\*T!\[clustered_index\] NONCLUSTERED \*/) ENGINE=InnoDB DEFAULT CHARSET=latin1 COLLATE=latin1_bin' 1 # drop the schema. curl -X PUT ${API_URL} -d '{"op":3, "task":"sequence_sharding_optimistic", "sources": ["mysql-replica-01"], "database":"sharding_seq_opt", "table":"t1"}' > ${WORK_DIR}/remove_schema.log 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..faf12d7824 100644 --- a/tests/shardddl1/run.sh +++ b/tests/shardddl1/run.sh @@ -464,6 +464,130 @@ 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 --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 --flush --sync" \ + "\"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 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 --flush --sync" \ + "\"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_dm_ctl $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "query-status test" \ + "\"result\": true" 3 + + # 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} 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} 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_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "query-status test" \ + "\"result\": true" 3 + run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "show-ddl-locks" \ + "no DDL lock exists" 1 +} + +# 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 +601,7 @@ function run() { DM_${i} sleep 1 done + DM_RENAME_COLUMN_OPTIMISTIC } cleanup_data $shardddl From 9d165de869dc21bf797b13faaceb012996f1f766 Mon Sep 17 00:00:00 2001 From: Chunzhu Li Date: Thu, 18 Mar 2021 12:01:25 +0800 Subject: [PATCH 08/23] fix ut --- pkg/shardddl/optimism/column.go | 6 ++--- pkg/shardddl/optimism/lock_test.go | 35 ++++++++++++------------------ 2 files changed, 17 insertions(+), 24 deletions(-) diff --git a/pkg/shardddl/optimism/column.go b/pkg/shardddl/optimism/column.go index 347a585a46..1dd9a6a29f 100644 --- a/pkg/shardddl/optimism/column.go +++ b/pkg/shardddl/optimism/column.go @@ -54,10 +54,10 @@ func GetAllDroppedColumns(cli *clientv3.Client) (map[string]map[string]map[strin if _, ok := colm[lockID][column][source]; !ok { colm[lockID][column][source] = make(map[string]map[string]interface{}) } - if _, ok := colm[lockID][column][source][downSchema]; !ok { - colm[lockID][column][source][downSchema] = make(map[string]interface{}) + if _, ok := colm[lockID][column][source][upSchema]; !ok { + colm[lockID][column][source][upSchema] = make(map[string]interface{}) } - colm[lockID][column][source][downSchema][downTable] = struct{}{} + colm[lockID][column][source][upSchema][upTable] = struct{}{} } } return colm, rev, nil diff --git a/pkg/shardddl/optimism/lock_test.go b/pkg/shardddl/optimism/lock_test.go index e547b65028..812f989101 100644 --- a/pkg/shardddl/optimism/lock_test.go +++ b/pkg/shardddl/optimism/lock_test.go @@ -1573,15 +1573,15 @@ func (t *testLock) TestAddNotFullyDroppedColumns(c *C) { }, } - colm1 = map[string]map[string]interface{}{ + colm1 = map[string]map[string]map[string]map[string]map[string]interface{}{ ID: { - "b": struct{}{}, - "c": struct{}{}, + "b": {source: {db: {tbls[0]: struct{}{}}}}, + "c": {source: {db: {tbls[0]: struct{}{}}}}, }, } - colm2 = map[string]map[string]interface{}{ + colm2 = map[string]map[string]map[string]map[string]map[string]interface{}{ ID: { - "c": struct{}{}, + "c": {source: {db: {tbls[0]: struct{}{}}}}, }, } ) @@ -1597,17 +1597,15 @@ func (t *testLock) TestAddNotFullyDroppedColumns(c *C) { t.checkLockNoDone(c, l) c.Assert(l.IsResolved(), IsFalse) - // TrySync for the first table, drop the first column - vers[source][db][tbls[0]]++ - DDLs, err := l.TrySync(source, db, tbls[0], DDLs1, []*model.TableInfo{ti1}, tts, vers[source][db][tbls[0]]) + // TrySync for the first table, drop column c + DDLs, err := l.TrySync(newInfoWithVersion(task, source, db, tbls[0], downSchema, downTable, DDLs1, ti0, []*model.TableInfo{ti1}, vers), tts) c.Assert(err, IsNil) c.Assert(DDLs, DeepEquals, []string{}) c.Assert(l.versions, DeepEquals, vers) c.Assert(l.IsResolved(), IsFalse) // TrySync for the first table, drop column b - vers[source][db][tbls[0]]++ - DDLs, err = l.TrySync(source, db, tbls[0], DDLs2, []*model.TableInfo{ti2}, tts, vers[source][db][tbls[0]]) + DDLs, err = l.TrySync(newInfoWithVersion(task, source, db, tbls[0], downSchema, downTable, DDLs2, ti1, []*model.TableInfo{ti2}, vers), tts) c.Assert(err, IsNil) c.Assert(DDLs, DeepEquals, []string{}) c.Assert(l.versions, DeepEquals, vers) @@ -1618,8 +1616,7 @@ func (t *testLock) TestAddNotFullyDroppedColumns(c *C) { c.Assert(colm, DeepEquals, colm1) // TrySync for the second table, drop column b, this column should be dropped - vers[source][db][tbls[1]]++ - DDLs, err = l.TrySync(source, db, tbls[1], DDLs2, []*model.TableInfo{ti3}, tts, vers[source][db][tbls[1]]) + DDLs, err = l.TrySync(newInfoWithVersion(task, source, db, tbls[1], downSchema, downTable, DDLs2, ti0, []*model.TableInfo{ti3}, vers), tts) c.Assert(err, IsNil) c.Assert(DDLs, DeepEquals, DDLs2) c.Assert(l.versions, DeepEquals, vers) @@ -1632,22 +1629,19 @@ func (t *testLock) TestAddNotFullyDroppedColumns(c *C) { c.Assert(colm, DeepEquals, colm2) // TrySync for the first table, add column b, should succeed, because this column is fully dropped in the downstream - vers[source][db][tbls[0]]++ - DDLs, err = l.TrySync(source, db, tbls[0], DDLs3, []*model.TableInfo{ti1}, tts, vers[source][db][tbls[0]]) + DDLs, err = l.TrySync(newInfoWithVersion(task, source, db, tbls[0], downSchema, downTable, DDLs3, ti2, []*model.TableInfo{ti1}, vers), tts) c.Assert(err, IsNil) c.Assert(DDLs, DeepEquals, DDLs3) c.Assert(l.versions, DeepEquals, vers) c.Assert(l.IsResolved(), IsFalse) // TrySync for the first table, add column b, should fail, because this column isn't fully dropped in the downstream - vers[source][db][tbls[0]]++ - _, err = l.TrySync(source, db, tbls[0], DDLs4, []*model.TableInfo{ti0}, tts, vers[source][db][tbls[0]]) + _, err = l.TrySync(newInfoWithVersion(task, source, db, tbls[0], downSchema, downTable, DDLs4, ti1, []*model.TableInfo{ti0}, vers), tts) c.Assert(err, ErrorMatches, ".*add column c that wasn't fully dropped in downstream.*") c.Assert(l.IsResolved(), IsFalse) // TrySync for the second table, drop column b, this column should be dropped - vers[source][db][tbls[1]]++ - DDLs, err = l.TrySync(source, db, tbls[1], DDLs1, []*model.TableInfo{ti2}, tts, vers[source][db][tbls[1]]) + DDLs, err = l.TrySync(newInfoWithVersion(task, source, db, tbls[1], downSchema, downTable, DDLs1, ti3, []*model.TableInfo{ti2}, vers), tts) c.Assert(err, IsNil) c.Assert(DDLs, DeepEquals, DDLs1) c.Assert(l.versions, DeepEquals, vers) @@ -1655,9 +1649,8 @@ func (t *testLock) TestAddNotFullyDroppedColumns(c *C) { // Simulate watch done operation from dm-worker c.Assert(l.DeleteColumnsByDDLs(DDLs), IsNil) - // TrySync for the first table, add column b, should fail, because this column isn't fully dropped in the downstream - vers[source][db][tbls[0]]++ - DDLs, err = l.TrySync(source, db, tbls[0], DDLs4, []*model.TableInfo{ti0}, tts, vers[source][db][tbls[0]]) + // TrySync for the first table, add column b, should succeed, because this column is fully dropped in the downstream + DDLs, err = l.TrySync(newInfoWithVersion(task, source, db, tbls[0], downSchema, downTable, DDLs4, ti1, []*model.TableInfo{ti0}, vers), tts) c.Assert(err, IsNil) c.Assert(DDLs, DeepEquals, DDLs4) c.Assert(l.versions, DeepEquals, vers) From bd6cc216647ee26bc00ccec867923e57f1253285 Mon Sep 17 00:00:00 2001 From: Chunzhu Li Date: Thu, 18 Mar 2021 12:19:03 +0800 Subject: [PATCH 09/23] fix lint --- pkg/shardddl/optimism/lock.go | 1 + 1 file changed, 1 insertion(+) diff --git a/pkg/shardddl/optimism/lock.go b/pkg/shardddl/optimism/lock.go index ab8384f167..f0f6c4d9c9 100644 --- a/pkg/shardddl/optimism/lock.go +++ b/pkg/shardddl/optimism/lock.go @@ -526,6 +526,7 @@ func (l *Lock) GetVersion(source string, schema string, table string) int64 { return l.versions[source][schema][table] } +// IsDroppedColumn checks whether this column is a not fully dropped column for this lock func (l *Lock) IsDroppedColumn(info Info, col string) bool { if _, ok := l.columns[col]; !ok { return false From 2fdb868446fe130fbd5d911c597bdb53a6ad53b1 Mon Sep 17 00:00:00 2001 From: Chunzhu Li Date: Thu, 18 Mar 2021 15:05:42 +0800 Subject: [PATCH 10/23] fix test --- tests/shardddl3/run.sh | 3 +++ 1 file changed, 3 insertions(+) diff --git a/tests/shardddl3/run.sh b/tests/shardddl3/run.sh index 1521ef4cc6..55e8859cb8 100644 --- a/tests/shardddl3/run.sh +++ b/tests/shardddl3/run.sh @@ -1028,6 +1028,9 @@ function DM_DropAddColumn_CASE() { restart_master_on_pos $reset "3" + # make sure drop column c is synced + sleep 2 + run_sql_source1 "alter table ${shardddl1}.${tb1} add column c int;" run_sql_source1 "insert into ${shardddl1}.${tb1} values(6,6);" From fb5857aef167f500d7f5250f36cf77b11e3d889c Mon Sep 17 00:00:00 2001 From: Chunzhu Li Date: Thu, 18 Mar 2021 19:44:16 +0800 Subject: [PATCH 11/23] address more comments --- dm/common/common.go | 6 ++++++ dm/master/shardddl/optimist.go | 4 ++++ pkg/shardddl/optimism/column.go | 4 ++++ pkg/shardddl/optimism/lock_test.go | 8 ++++---- tests/shardddl3/run.sh | 5 +++-- 5 files changed, 21 insertions(+), 6 deletions(-) diff --git a/dm/common/common.go b/dm/common/common.go index 2d85ca7931..78212a31b9 100644 --- a/dm/common/common.go +++ b/dm/common/common.go @@ -81,6 +81,12 @@ var ( ShardDDLOptimismInitSchemaKeyAdapter KeyAdapter = keyHexEncoderDecoder("/dm-master/shardddl-optimism/init-schema/") // ShardDDLOptimismDroppedColumnsKeyAdapter is used to store the columns that are not fully dropped // k/v: Encode(task-name, downstream-schema-name, downstream-table-name, column-name, source-id, upstream-schema-name, upstream-table-name) -> empty + // If we don't identify different upstream tables, we may report an error for tb2 in the following case. + // Time series: (+a/-a means add/drop column a) + // older ----------------> newer + // tb1: +a +b +c -c + // tb2: +a +b +c + // tb3: +a +b +c ShardDDLOptimismDroppedColumnsKeyAdapter KeyAdapter = keyHexEncoderDecoder("/dm-master/shardddl-optimism/dropped-columns/") ) diff --git a/dm/master/shardddl/optimist.go b/dm/master/shardddl/optimist.go index a773e5e4cc..6a6d775a63 100644 --- a/dm/master/shardddl/optimist.go +++ b/dm/master/shardddl/optimist.go @@ -282,6 +282,10 @@ func (o *Optimist) recoverLocks( for _, ifSource := range ifTask { for _, ifSchema := range ifSource { for _, info := range ifSchema { + // We should return err after all infos are set up. + // If we stopped recovering locks once we meet an error, + // dm-master leader may not have the full information for the other "normal" locks, + // which will cause the sync error in dm-worker. err := o.handleInfo(info) setFirstErr(err) } diff --git a/pkg/shardddl/optimism/column.go b/pkg/shardddl/optimism/column.go index 1dd9a6a29f..52c347ccf5 100644 --- a/pkg/shardddl/optimism/column.go +++ b/pkg/shardddl/optimism/column.go @@ -64,6 +64,7 @@ func GetAllDroppedColumns(cli *clientv3.Client) (map[string]map[string]map[strin } // PutDroppedColumn puts the undropped column name into ectd. +// When we drop a column, we save this column's name in etcd. func PutDroppedColumn(cli *clientv3.Client, info Info, column string) (rev int64, putted bool, err error) { key := common.ShardDDLOptimismDroppedColumnsKeyAdapter.Encode( info.Task, info.DownSchema, info.DownTable, column, info.Source, info.UpSchema, info.UpTable) @@ -78,6 +79,9 @@ func PutDroppedColumn(cli *clientv3.Client, info Info, column string) (rev int64 } // DeleteDroppedColumns tries to delete the dropped columns for the specified lock ID. +// Only when this column is fully dropped in downstream database, +// in other words, **we receive a `Done` DDL group from dm-worker)**, +// we can delete this column's name from the etcd. func DeleteDroppedColumns(cli *clientv3.Client, task, downSchema, downTable string, columns ...string) (rev int64, deleted bool, err error) { ops := make([]clientv3.Op, 0, len(columns)) for _, col := range columns { diff --git a/pkg/shardddl/optimism/lock_test.go b/pkg/shardddl/optimism/lock_test.go index 812f989101..05cda1223f 100644 --- a/pkg/shardddl/optimism/lock_test.go +++ b/pkg/shardddl/optimism/lock_test.go @@ -1615,7 +1615,7 @@ func (t *testLock) TestAddNotFullyDroppedColumns(c *C) { c.Assert(err, IsNil) c.Assert(colm, DeepEquals, colm1) - // TrySync for the second table, drop column b, this column should be dropped + // TrySync for the second table, drop column b, this column should be fully dropped DDLs, err = l.TrySync(newInfoWithVersion(task, source, db, tbls[1], downSchema, downTable, DDLs2, ti0, []*model.TableInfo{ti3}, vers), tts) c.Assert(err, IsNil) c.Assert(DDLs, DeepEquals, DDLs2) @@ -1635,12 +1635,12 @@ func (t *testLock) TestAddNotFullyDroppedColumns(c *C) { c.Assert(l.versions, DeepEquals, vers) c.Assert(l.IsResolved(), IsFalse) - // TrySync for the first table, add column b, should fail, because this column isn't fully dropped in the downstream + // TrySync for the first table, add column c, should fail, because this column isn't fully dropped in the downstream _, err = l.TrySync(newInfoWithVersion(task, source, db, tbls[0], downSchema, downTable, DDLs4, ti1, []*model.TableInfo{ti0}, vers), tts) c.Assert(err, ErrorMatches, ".*add column c that wasn't fully dropped in downstream.*") c.Assert(l.IsResolved(), IsFalse) - // TrySync for the second table, drop column b, this column should be dropped + // TrySync for the second table, drop column c, this column should be fully dropped DDLs, err = l.TrySync(newInfoWithVersion(task, source, db, tbls[1], downSchema, downTable, DDLs1, ti3, []*model.TableInfo{ti2}, vers), tts) c.Assert(err, IsNil) c.Assert(DDLs, DeepEquals, DDLs1) @@ -1649,7 +1649,7 @@ func (t *testLock) TestAddNotFullyDroppedColumns(c *C) { // Simulate watch done operation from dm-worker c.Assert(l.DeleteColumnsByDDLs(DDLs), IsNil) - // TrySync for the first table, add column b, should succeed, because this column is fully dropped in the downstream + // TrySync for the first table, add column c, should succeed, because this column is fully dropped in the downstream DDLs, err = l.TrySync(newInfoWithVersion(task, source, db, tbls[0], downSchema, downTable, DDLs4, ti1, []*model.TableInfo{ti0}, vers), tts) c.Assert(err, IsNil) c.Assert(DDLs, DeepEquals, DDLs4) diff --git a/tests/shardddl3/run.sh b/tests/shardddl3/run.sh index 55e8859cb8..23d908cd69 100644 --- a/tests/shardddl3/run.sh +++ b/tests/shardddl3/run.sh @@ -1028,8 +1028,9 @@ function DM_DropAddColumn_CASE() { restart_master_on_pos $reset "3" - # make sure drop column c is synced - sleep 2 + # make sure column c is fully dropped in the downstream + check_log_contain_with_retry 'finish to handle ddls in optimistic shard mode' $WORK_DIR/worker1/log/dm-worker.log + check_log_contain_with_retry 'finish to handle ddls in optimistic shard mode' $WORK_DIR/worker2/log/dm-worker.log run_sql_source1 "alter table ${shardddl1}.${tb1} add column c int;" run_sql_source1 "insert into ${shardddl1}.${tb1} values(6,6);" From baee6facb5a910f73e6e9462091240a60d7800ec Mon Sep 17 00:00:00 2001 From: Chunzhu Li Date: Thu, 18 Mar 2021 19:47:17 +0800 Subject: [PATCH 12/23] address comment --- pkg/shardddl/optimism/lock.go | 10 ++++------ 1 file changed, 4 insertions(+), 6 deletions(-) diff --git a/pkg/shardddl/optimism/lock.go b/pkg/shardddl/optimism/lock.go index f0f6c4d9c9..fb22094c49 100644 --- a/pkg/shardddl/optimism/lock.go +++ b/pkg/shardddl/optimism/lock.go @@ -613,12 +613,10 @@ func AddDifferentFieldLenColumns(lockID, ddl string, oldJoined, newJoined schema newJoinedCols := schemacmp.DecodeColumnFieldTypes(newJoined) oldCol, ok1 := oldJoinedCols[col] newCol, ok2 := newJoinedCols[col] - if ok1 && ok2 { - if newCol.Flen != oldCol.Flen { - return col, terror.ErrShardDDLOptimismTrySyncFail.Generate( - lockID, fmt.Sprintf("add columns with different field lengths."+ - "ddl: %s, origLen: %d, newLen: %d", ddl, oldCol.Flen, newCol.Flen)) - } + if ok1 && ok2 && newCol.Flen != oldCol.Flen { + return col, terror.ErrShardDDLOptimismTrySyncFail.Generate( + lockID, fmt.Sprintf("add columns with different field lengths."+ + "ddl: %s, origLen: %d, newLen: %d", ddl, oldCol.Flen, newCol.Flen)) } } return col, nil From 10f3dcf5c37fd5d25621824103d07219810d1e86 Mon Sep 17 00:00:00 2001 From: Chunzhu Li Date: Fri, 19 Mar 2021 14:04:04 +0800 Subject: [PATCH 13/23] address comments --- dm/master/shardddl/optimist.go | 4 ++-- pkg/shardddl/optimism/column.go | 24 ++++++++++++------------ pkg/shardddl/optimism/keeper.go | 6 +++--- pkg/shardddl/optimism/lock.go | 14 ++++++++------ pkg/shardddl/optimism/lock_test.go | 3 +++ 5 files changed, 28 insertions(+), 23 deletions(-) diff --git a/dm/master/shardddl/optimist.go b/dm/master/shardddl/optimist.go index 6a6d775a63..e0874ed421 100644 --- a/dm/master/shardddl/optimist.go +++ b/dm/master/shardddl/optimist.go @@ -267,7 +267,7 @@ func (o *Optimist) rebuildLocks() (revSource, revInfo, revOperation int64, err e func (o *Optimist) recoverLocks( ifm map[string]map[string]map[string]map[string]optimism.Info, opm map[string]map[string]map[string]map[string]optimism.Operation, - colm map[string]map[string]map[string]map[string]map[string]interface{}) error { + colm map[string]map[string]map[string]map[string]map[string]struct{}) error { // construct locks based on the shard DDL info. o.lk.SetColumnMap(colm) defer o.lk.SetColumnMap(nil) @@ -315,7 +315,7 @@ func (o *Optimist) recoverLocks( } } } - return nil + return firstErr } // watchSourceInfoOperation watches the etcd operation for source tables, shard DDL infos and shard DDL operations. diff --git a/pkg/shardddl/optimism/column.go b/pkg/shardddl/optimism/column.go index 52c347ccf5..2044103729 100644 --- a/pkg/shardddl/optimism/column.go +++ b/pkg/shardddl/optimism/column.go @@ -20,9 +20,9 @@ import ( "github.com/pingcap/dm/pkg/etcdutil" ) -// GetAllDroppedColumns gets the all dropped columns. -func GetAllDroppedColumns(cli *clientv3.Client) (map[string]map[string]map[string]map[string]map[string]interface{}, int64, error) { - colm := make(map[string]map[string]map[string]map[string]map[string]interface{}) +// GetAllDroppedColumns gets the all partially dropped columns. +func GetAllDroppedColumns(cli *clientv3.Client) (map[string]map[string]map[string]map[string]map[string]struct{}, int64, error) { + colm := make(map[string]map[string]map[string]map[string]map[string]struct{}) op := clientv3.OpGet(common.ShardDDLOptimismDroppedColumnsKeyAdapter.Path(), clientv3.WithPrefix()) respTxn, rev, err := etcdutil.DoOpsInOneTxnWithRetry(cli, op) if err != nil { @@ -46,16 +46,16 @@ func GetAllDroppedColumns(cli *clientv3.Client) (map[string]map[string]map[strin info := NewInfo(task, source, upSchema, upTable, downSchema, downTable, nil, nil, nil) lockID := genDDLLockID(info) if _, ok := colm[lockID]; !ok { - colm[lockID] = make(map[string]map[string]map[string]map[string]interface{}) + colm[lockID] = make(map[string]map[string]map[string]map[string]struct{}) } if _, ok := colm[lockID][column]; !ok { - colm[lockID][column] = make(map[string]map[string]map[string]interface{}) + colm[lockID][column] = make(map[string]map[string]map[string]struct{}) } if _, ok := colm[lockID][column][source]; !ok { - colm[lockID][column][source] = make(map[string]map[string]interface{}) + colm[lockID][column][source] = make(map[string]map[string]struct{}) } if _, ok := colm[lockID][column][source][upSchema]; !ok { - colm[lockID][column][source][upSchema] = make(map[string]interface{}) + colm[lockID][column][source][upSchema] = make(map[string]struct{}) } colm[lockID][column][source][upSchema][upTable] = struct{}{} } @@ -63,7 +63,7 @@ func GetAllDroppedColumns(cli *clientv3.Client) (map[string]map[string]map[strin return colm, rev, nil } -// PutDroppedColumn puts the undropped column name into ectd. +// PutDroppedColumn puts the partially dropped column name into ectd. // When we drop a column, we save this column's name in etcd. func PutDroppedColumn(cli *clientv3.Client, info Info, column string) (rev int64, putted bool, err error) { key := common.ShardDDLOptimismDroppedColumnsKeyAdapter.Encode( @@ -78,9 +78,9 @@ func PutDroppedColumn(cli *clientv3.Client, info Info, column string) (rev int64 return rev, resp.Succeeded, nil } -// DeleteDroppedColumns tries to delete the dropped columns for the specified lock ID. +// DeleteDroppedColumns tries to delete the partially dropped columns for the specified lock ID. // Only when this column is fully dropped in downstream database, -// in other words, **we receive a `Done` DDL group from dm-worker)**, +// in other words, **we receive a `Done` operation from dm-worker**, // we can delete this column's name from the etcd. func DeleteDroppedColumns(cli *clientv3.Client, task, downSchema, downTable string, columns ...string) (rev int64, deleted bool, err error) { ops := make([]clientv3.Op, 0, len(columns)) @@ -99,7 +99,7 @@ func deleteDroppedColumnByColumnOp(task, downSchema, downTable, column string) c return clientv3.OpDelete(common.ShardDDLOptimismDroppedColumnsKeyAdapter.Encode(task, downSchema, downTable, column), clientv3.WithPrefix()) } -// deleteDroppedColumnsByLockOp returns a DELETE etcd operation for the specified task. +// deleteDroppedColumnsByLockOp returns a DELETE etcd operation for the specified lock. func deleteDroppedColumnsByLockOp(task, downSchema, downTable string) clientv3.Op { - return clientv3.OpDelete(common.ShardDDLOptimismDroppedColumnsKeyAdapter.Encode(task), clientv3.WithPrefix()) + return clientv3.OpDelete(common.ShardDDLOptimismDroppedColumnsKeyAdapter.Encode(task, downSchema, downTable), clientv3.WithPrefix()) } diff --git a/pkg/shardddl/optimism/keeper.go b/pkg/shardddl/optimism/keeper.go index 5e8effcf34..a0334bbf6b 100644 --- a/pkg/shardddl/optimism/keeper.go +++ b/pkg/shardddl/optimism/keeper.go @@ -27,8 +27,8 @@ import ( type LockKeeper struct { mu sync.RWMutex locks map[string]*Lock // lockID -> Lock - // lockID -> column name -> source -> upSchema -> upTable -> interface{} - colm map[string]map[string]map[string]map[string]map[string]interface{} + // lockID -> column name -> source -> upSchema -> upTable -> struct{} + colm map[string]map[string]map[string]map[string]map[string]struct{} } // NewLockKeeper creates a new LockKeeper instance. @@ -113,7 +113,7 @@ func (lk *LockKeeper) Clear() { } // SetColumnMap sets the column map received from etcd -func (lk *LockKeeper) SetColumnMap(colm map[string]map[string]map[string]map[string]map[string]interface{}) { +func (lk *LockKeeper) SetColumnMap(colm map[string]map[string]map[string]map[string]map[string]struct{}) { lk.mu.Lock() defer lk.mu.Unlock() diff --git a/pkg/shardddl/optimism/lock.go b/pkg/shardddl/optimism/lock.go index fb22094c49..d468e3690b 100644 --- a/pkg/shardddl/optimism/lock.go +++ b/pkg/shardddl/optimism/lock.go @@ -62,7 +62,7 @@ type Lock struct { // record the partially dropped columns // column name -> source -> upSchema -> upTable -> interface{} - columns map[string]map[string]map[string]map[string]interface{} + columns map[string]map[string]map[string]map[string]struct{} } // NewLock creates a new Lock instance. @@ -79,7 +79,7 @@ func NewLock(cli *clientv3.Client, ID, task, downSchema, downTable string, ti *m done: make(map[string]map[string]map[string]bool), synced: true, versions: make(map[string]map[string]map[string]int64), - columns: make(map[string]map[string]map[string]map[string]interface{}), + columns: make(map[string]map[string]map[string]map[string]struct{}), } l.addTables(tts) metrics.ReportDDLPending(task, metrics.DDLPendingNone, metrics.DDLPendingSynced) @@ -558,19 +558,21 @@ func (l *Lock) AddDroppedColumn(info Info, col string) error { } if _, ok := l.columns[col]; !ok { - l.columns[col] = make(map[string]map[string]map[string]interface{}) + l.columns[col] = make(map[string]map[string]map[string]struct{}) } if _, ok := l.columns[col][source]; !ok { - l.columns[col][source] = make(map[string]map[string]interface{}) + l.columns[col][source] = make(map[string]map[string]struct{}) } if _, ok := l.columns[col][source][upSchema]; !ok { - l.columns[col][source][upSchema] = make(map[string]interface{}) + l.columns[col][source][upSchema] = make(map[string]struct{}) } l.columns[col][source][upSchema][upTable] = struct{}{} return nil } -// DeleteColumnsByDDLs deletes the dropped columns by DDLs. +// DeleteColumnsByDDLs deletes the partially dropped columns that extracted from DDLs. +// We can remove columns from the partially dropped columns map unless this column is dropped in the downstream database, +// that is to say, op.Done is true and ddls contains drop column DDL. func (l *Lock) DeleteColumnsByDDLs(ddls []string) error { l.mu.Lock() defer l.mu.Unlock() diff --git a/pkg/shardddl/optimism/lock_test.go b/pkg/shardddl/optimism/lock_test.go index 05cda1223f..f27693da8b 100644 --- a/pkg/shardddl/optimism/lock_test.go +++ b/pkg/shardddl/optimism/lock_test.go @@ -31,6 +31,9 @@ var _ = Suite(&testLock{}) func (t *testLock) SetUpSuite(c *C) { c.Assert(log.InitLogger(&log.Config{}), IsNil) +} + +func (t *testLock) TearDownSuite(c *C) { clearTestInfoOperation(c) } From 62309ad72eb0c13a6a81336d4b2653eaa509f96f Mon Sep 17 00:00:00 2001 From: Chunzhu Li Date: Fri, 19 Mar 2021 14:56:50 +0800 Subject: [PATCH 14/23] fix uts --- pkg/shardddl/optimism/column_test.go | 2 +- pkg/shardddl/optimism/lock_test.go | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/pkg/shardddl/optimism/column_test.go b/pkg/shardddl/optimism/column_test.go index 8ffc124248..6f05e53606 100644 --- a/pkg/shardddl/optimism/column_test.go +++ b/pkg/shardddl/optimism/column_test.go @@ -53,7 +53,7 @@ func (t *testColumn) TestColumnETCD(c *C) { c.Assert(putted, IsTrue) c.Assert(rev4, Greater, rev3) - expectedColm := map[string]map[string]map[string]map[string]map[string]interface{}{ + expectedColm := map[string]map[string]map[string]map[string]map[string]struct{}{ lockID: { "a": {source1: {upSchema1: {upTable1: struct{}{}}}}, "b": {source1: {upSchema1: {upTable1: struct{}{}}, diff --git a/pkg/shardddl/optimism/lock_test.go b/pkg/shardddl/optimism/lock_test.go index f27693da8b..88d67fd5a9 100644 --- a/pkg/shardddl/optimism/lock_test.go +++ b/pkg/shardddl/optimism/lock_test.go @@ -1576,13 +1576,13 @@ func (t *testLock) TestAddNotFullyDroppedColumns(c *C) { }, } - colm1 = map[string]map[string]map[string]map[string]map[string]interface{}{ + colm1 = map[string]map[string]map[string]map[string]map[string]struct{}{ ID: { "b": {source: {db: {tbls[0]: struct{}{}}}}, "c": {source: {db: {tbls[0]: struct{}{}}}}, }, } - colm2 = map[string]map[string]map[string]map[string]map[string]interface{}{ + colm2 = map[string]map[string]map[string]map[string]map[string]struct{}{ ID: { "c": {source: {db: {tbls[0]: struct{}{}}}}, }, From 048f627fd6e50a4a5c6232c3ef0e65095d579126 Mon Sep 17 00:00:00 2001 From: Chunzhu Li Date: Mon, 22 Mar 2021 12:11:26 +0800 Subject: [PATCH 15/23] Apply suggestions from code review Co-authored-by: lance6716 --- pkg/shardddl/optimism/lock.go | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/pkg/shardddl/optimism/lock.go b/pkg/shardddl/optimism/lock.go index d468e3690b..1cce400c5d 100644 --- a/pkg/shardddl/optimism/lock.go +++ b/pkg/shardddl/optimism/lock.go @@ -61,7 +61,7 @@ type Lock struct { versions map[string]map[string]map[string]int64 // record the partially dropped columns - // column name -> source -> upSchema -> upTable -> interface{} + // column name -> source -> upSchema -> upTable -> struct{} columns map[string]map[string]map[string]map[string]struct{} } @@ -571,7 +571,7 @@ func (l *Lock) AddDroppedColumn(info Info, col string) error { } // DeleteColumnsByDDLs deletes the partially dropped columns that extracted from DDLs. -// We can remove columns from the partially dropped columns map unless this column is dropped in the downstream database, +// We can not remove columns from the partially dropped columns map unless this column is dropped in the downstream database, // that is to say, op.Done is true and ddls contains drop column DDL. func (l *Lock) DeleteColumnsByDDLs(ddls []string) error { l.mu.Lock() From 814ea09c45635668f5e9cfc010564fedf8f57157 Mon Sep 17 00:00:00 2001 From: Chunzhu Li Date: Wed, 24 Mar 2021 13:49:34 +0800 Subject: [PATCH 16/23] address comments --- dm/master/server_test.go | 2 +- dm/master/shardddl/optimist.go | 6 ++++-- dm/worker/task_checker_test.go | 2 +- pkg/shardddl/optimism/operation.go | 20 +++++++++++--------- pkg/shardddl/optimism/operation_test.go | 8 ++++---- pkg/shardddl/optimism/ops_test.go | 2 +- pkg/terror/error_list.go | 2 +- syncer/optimist.go | 2 +- syncer/shardddl/optimist_test.go | 5 +++-- tests/shardddl3/run.sh | 3 ++- 10 files changed, 29 insertions(+), 23 deletions(-) diff --git a/dm/master/server_test.go b/dm/master/server_test.go index 499c279117..6869ec504e 100644 --- a/dm/master/server_test.go +++ b/dm/master/server_test.go @@ -634,7 +634,7 @@ func (t *testMaster) TestStartTaskWithRemoveMeta(c *check.C) { tiBefore = createTableInfo(c, p, se, tblID, `CREATE TABLE bar (id INT PRIMARY KEY)`) tiAfter1 = createTableInfo(c, p, se, tblID, `CREATE TABLE bar (id INT PRIMARY KEY, c1 TEXT)`) info1 = optimism.NewInfo(taskName, sources[0], "foo-1", "bar-1", schema, table, DDLs1, tiBefore, []*model.TableInfo{tiAfter1}) - op1 = optimism.NewOperation(ID, taskName, sources[0], info1.UpSchema, info1.UpTable, DDLs1, optimism.ConflictNone, false) + op1 = optimism.NewOperation(ID, taskName, sources[0], info1.UpSchema, info1.UpTable, DDLs1, optimism.ConflictNone, "", false) ) _, err = optimism.PutSourceTables(etcdTestCli, st1) diff --git a/dm/master/shardddl/optimist.go b/dm/master/shardddl/optimist.go index e0874ed421..5b7ed22925 100644 --- a/dm/master/shardddl/optimist.go +++ b/dm/master/shardddl/optimist.go @@ -525,11 +525,13 @@ 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(o.cli, info, tts) var cfStage = optimism.ConflictNone + var cfMsg = "" 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)) } else if err != nil { cfStage = optimism.ConflictDetected // we treat any errors returned from `TrySync` as conflict detected now. + cfMsg = err.Error() 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)) } else { @@ -569,7 +571,7 @@ func (o *Optimist) handleLock(info optimism.Info, tts []optimism.TargetTable, sk return nil } - op := optimism.NewOperation(lockID, lock.Task, info.Source, info.UpSchema, info.UpTable, newDDLs, cfStage, false) + op := optimism.NewOperation(lockID, lock.Task, info.Source, info.UpSchema, info.UpTable, newDDLs, cfStage, cfMsg, false) rev, succ, err := optimism.PutOperation(o.cli, skipDone, op, info.ModRevision) if err != nil { return err @@ -641,7 +643,7 @@ func (o *Optimist) deleteInfosOps(lock *optimism.Lock) (bool, error) { info := optimism.NewInfo(lock.Task, source, schema, table, lock.DownSchema, lock.DownTable, nil, nil, nil) info.Version = lock.GetVersion(source, schema, table) infos = append(infos, info) - ops = append(ops, optimism.NewOperation(lock.ID, lock.Task, source, schema, table, nil, optimism.ConflictNone, false)) + ops = append(ops, optimism.NewOperation(lock.ID, lock.Task, source, schema, table, nil, optimism.ConflictNone, "", false)) } } } diff --git a/dm/worker/task_checker_test.go b/dm/worker/task_checker_test.go index bb2d95f5a0..b7bc9eb058 100644 --- a/dm/worker/task_checker_test.go +++ b/dm/worker/task_checker_test.go @@ -304,7 +304,7 @@ func (s *testTaskCheckerSuite) TestIsResumableError(c *check.C) { {terror.ErrParserParseRelayLog.New("parse relay log file bin.000018 from offset 500 in dir /home/tidb/deploy/relay_log/d2e831df-b4ec-11e9-9237-0242ac110008.000004: parse relay log file bin.000018 from offset 0 in dir /home/tidb/deploy/relay_log/d2e831df-b4ec-11e9-9237-0242ac110008.000004: parse relay log file /home/tidb/deploy/relay_log/d2e831df-b4ec-11e9-9237-0242ac110008.000004/bin.000018: get event err EOF, need 1567488104 but got 316323"), false}, // unresumable terror codes {terror.ErrSyncUnitDDLWrongSequence.Generate("wrong sequence", "right sequence"), false}, - {terror.ErrSyncerShardDDLConflict.Generate("conflict DDL"), false}, + {terror.ErrSyncerShardDDLConflict.Generate("conflict DDL", "conflict"), false}, // others {nil, true}, {errors.New("unknown error"), true}, diff --git a/pkg/shardddl/optimism/operation.go b/pkg/shardddl/optimism/operation.go index 41dfd9de45..ac8b8526fe 100644 --- a/pkg/shardddl/optimism/operation.go +++ b/pkg/shardddl/optimism/operation.go @@ -49,19 +49,20 @@ const ( // and is deleted when removing the lock by DM-master. // because we need the newest stage in Operation to recover the lock when restarting DM-master. type Operation struct { - ID string `json:"id"` // the corresponding DDL lock ID - Task string `json:"task"` // data migration task name - Source string `json:"source"` // upstream source ID - UpSchema string `json:"up-schema"` // upstream/source schema name, different sources can have the same schema name - UpTable string `json:"up-table"` // upstream/source table name, different sources can have the same table name - DDLs []string `json:"ddls"` // DDL statements need to apply to the downstream. - ConflictStage ConflictStage `json:"conflict-stage"` // current conflict stage. - Done bool `json:"done"` // whether the operation has done + ID string `json:"id"` // the corresponding DDL lock ID + Task string `json:"task"` // data migration task name + Source string `json:"source"` // upstream source ID + UpSchema string `json:"up-schema"` // upstream/source schema name, different sources can have the same schema name + UpTable string `json:"up-table"` // upstream/source table name, different sources can have the same table name + DDLs []string `json:"ddls"` // DDL statements need to apply to the downstream. + ConflictStage ConflictStage `json:"conflict-stage"` // current conflict stage. + ConflictMsg string `json:"conflict-message"` // current conflict message + Done bool `json:"done"` // whether the operation has done } // NewOperation creates a new Operation instance. func NewOperation(ID, task, source, upSchema, upTable string, - DDLs []string, conflictStage ConflictStage, done bool) Operation { + DDLs []string, conflictStage ConflictStage, conflictMsg string, done bool) Operation { return Operation{ ID: ID, Task: task, @@ -70,6 +71,7 @@ func NewOperation(ID, task, source, upSchema, upTable string, UpTable: upTable, DDLs: DDLs, ConflictStage: conflictStage, + ConflictMsg: conflictMsg, Done: done, } } diff --git a/pkg/shardddl/optimism/operation_test.go b/pkg/shardddl/optimism/operation_test.go index c223d484b8..e6832c3899 100644 --- a/pkg/shardddl/optimism/operation_test.go +++ b/pkg/shardddl/optimism/operation_test.go @@ -23,11 +23,11 @@ import ( func (t *testForEtcd) TestOperationJSON(c *C) { o1 := NewOperation("test-ID", "test", "mysql-replica-1", "db-1", "tbl-1", []string{ "ALTER TABLE tbl ADD COLUMN c1 INT", - }, ConflictDetected, true) + }, ConflictDetected, "conflict", true) j, err := o1.toJSON() c.Assert(err, IsNil) - c.Assert(j, Equals, `{"id":"test-ID","task":"test","source":"mysql-replica-1","up-schema":"db-1","up-table":"tbl-1","ddls":["ALTER TABLE tbl ADD COLUMN c1 INT"],"conflict-stage":"detected","done":true}`) + c.Assert(j, Equals, `{"id":"test-ID","task":"test","source":"mysql-replica-1","up-schema":"db-1","up-table":"tbl-1","ddls":["ALTER TABLE tbl ADD COLUMN c1 INT"],"conflict-stage":"detected","conflict-message":"conflict","done":true}`) c.Assert(j, Equals, o1.String()) o2, err := operationFromJSON(j) @@ -48,8 +48,8 @@ func (t *testForEtcd) TestOperationEtcd(c *C) { ID2 = "test2-`foo`.`bar`" source1 = "mysql-replica-1" DDLs = []string{"ALTER TABLE bar ADD COLUMN c1 INT"} - op11 = NewOperation(ID1, task1, source1, upSchema, upTable, DDLs, ConflictNone, false) - op21 = NewOperation(ID2, task2, source1, upSchema, upTable, DDLs, ConflictResolved, true) + op11 = NewOperation(ID1, task1, source1, upSchema, upTable, DDLs, ConflictNone, "", false) + op21 = NewOperation(ID2, task2, source1, upSchema, upTable, DDLs, ConflictResolved, "", true) ) // put the same keys twice. diff --git a/pkg/shardddl/optimism/ops_test.go b/pkg/shardddl/optimism/ops_test.go index 804747f18c..a190774dd2 100644 --- a/pkg/shardddl/optimism/ops_test.go +++ b/pkg/shardddl/optimism/ops_test.go @@ -29,7 +29,7 @@ func (t *testForEtcd) TestDeleteInfosOperationsSchema(c *C) { downTable = "bar" DDLs = []string{"ALTER TABLE bar ADD COLUMN c1 INT"} info = NewInfo(task, source, upSchema, upTable, downSchema, downTable, DDLs, nil, nil) - op = NewOperation("test-ID", task, source, upSchema, upTable, DDLs, ConflictResolved, false) + op = NewOperation("test-ID", task, source, upSchema, upTable, DDLs, ConflictResolved, "", false) is = NewInitSchema(task, downSchema, downTable, nil) ) diff --git a/pkg/terror/error_list.go b/pkg/terror/error_list.go index 7e1b0a5122..4d5f5560fc 100644 --- a/pkg/terror/error_list.go +++ b/pkg/terror/error_list.go @@ -992,7 +992,7 @@ var ( ErrSyncerUnitExecWithNoBlockingDDL = New(codeSyncerUnitExecWithNoBlockingDDL, ClassSyncUnit, ScopeInternal, LevelHigh, "process unit not waiting for sharding DDL to sync", "") ErrSyncerUnitGenBAList = New(codeSyncerUnitGenBAList, ClassSyncUnit, ScopeInternal, LevelHigh, "generate block allow list", "Please check the `block-allow-list` config in task configuration file.") ErrSyncerUnitHandleDDLFailed = New(codeSyncerUnitHandleDDLFailed, ClassSyncUnit, ScopeInternal, LevelHigh, "fail to handle ddl job for %s", "") - ErrSyncerShardDDLConflict = New(codeSyncerShardDDLConflict, ClassSyncUnit, ScopeInternal, LevelHigh, "fail to handle shard ddl %v in optimistic mode, because schema conflict detected", "Please use show-ddl-locks command for more details.") + ErrSyncerShardDDLConflict = New(codeSyncerShardDDLConflict, ClassSyncUnit, ScopeInternal, LevelHigh, "fail to handle shard ddl %v in optimistic mode, because schema conflict detected, conflict error: %s", "Please use show-ddl-locks command for more details.") ErrSyncerFailpoint = New(codeSyncerFailpoint, ClassSyncUnit, ScopeInternal, LevelLow, "failpoint specified error", "") ErrSyncerReplaceEvent = New(codeSyncerReplaceEvent, ClassSyncUnit, ScopeInternal, LevelHigh, "", "") ErrSyncerOperatorNotExist = New(codeSyncerOperatorNotExist, ClassSyncUnit, ScopeInternal, LevelLow, "error operator not exist, position: %s", "") diff --git a/syncer/optimist.go b/syncer/optimist.go index 5566036c19..955442104d 100644 --- a/syncer/optimist.go +++ b/syncer/optimist.go @@ -177,7 +177,7 @@ func (s *Syncer) handleQueryEventOptimistic( } if op.ConflictStage == optimism.ConflictDetected { - return terror.ErrSyncerShardDDLConflict.Generate(needHandleDDLs) + return terror.ErrSyncerShardDDLConflict.Generate(needHandleDDLs, op.ConflictMsg) } // updated needHandleDDLs to DDLs received from DM-master. diff --git a/syncer/shardddl/optimist_test.go b/syncer/shardddl/optimist_test.go index 0700f3d74e..102bc8f7df 100644 --- a/syncer/shardddl/optimist_test.go +++ b/syncer/shardddl/optimist_test.go @@ -27,6 +27,7 @@ import ( "github.com/pingcap/dm/pkg/log" "github.com/pingcap/dm/pkg/shardddl/optimism" + "github.com/pingcap/dm/pkg/terror" ) type testOptimist struct { @@ -82,9 +83,9 @@ func (t *testOptimist) TestOptimist(c *C) { tiAfter1 = createTableInfo(c, p, se, tblID, `CREATE TABLE bar (id INT PRIMARY KEY, c1 TEXT)`) tiAfter2 = createTableInfo(c, p, se, tblID, `CREATE TABLE bar (id INT PRIMARY KEY, c1 DATETIME)`) info1 = o.ConstructInfo("foo-1", "bar-1", downSchema, downTable, DDLs1, tiBefore, []*model.TableInfo{tiAfter1}) - op1 = optimism.NewOperation(ID, task, source, info1.UpSchema, info1.UpTable, DDLs1, optimism.ConflictNone, false) + op1 = optimism.NewOperation(ID, task, source, info1.UpSchema, info1.UpTable, DDLs1, optimism.ConflictNone, "", false) info2 = o.ConstructInfo("foo-1", "bar-2", downSchema, downTable, DDLs2, tiBefore, []*model.TableInfo{tiAfter2}) - op2 = optimism.NewOperation(ID, task, source, info2.UpSchema, info2.UpTable, DDLs2, optimism.ConflictDetected, false) + op2 = optimism.NewOperation(ID, task, source, info2.UpSchema, info2.UpTable, DDLs2, optimism.ConflictDetected, terror.ErrShardDDLOptimismTrySyncFail.Generate(ID, "conflict").Error(), false) infoCreate = o.ConstructInfo("foo-new", "bar-new", downSchema, downTable, []string{`CREATE TABLE bar (id INT PRIMARY KEY)`}, tiBefore, []*model.TableInfo{tiBefore}) // same table info. diff --git a/tests/shardddl3/run.sh b/tests/shardddl3/run.sh index 3275e8d7f0..10f94e531c 100644 --- a/tests/shardddl3/run.sh +++ b/tests/shardddl3/run.sh @@ -1040,7 +1040,8 @@ function DM_DropAddColumn_CASE() { run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" \ "query-status test" \ - "because schema conflict detected" 1 + "because schema conflict detected" 1 \ + "add column b that wasn't fully dropped in downstream" 1 check_sync_diff $WORK_DIR $cur/conf/diff_config.toml 3 'fail' From d982ac40bc54e6fd8e141045bb8dc98aeeac1dbb Mon Sep 17 00:00:00 2001 From: Chunzhu Li Date: Wed, 24 Mar 2021 14:20:06 +0800 Subject: [PATCH 17/23] fix terror --- _utils/terror_gen/errors_release.txt | 2 +- errors.toml | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/_utils/terror_gen/errors_release.txt b/_utils/terror_gen/errors_release.txt index 1c8f20f246..3aa5345956 100644 --- a/_utils/terror_gen/errors_release.txt +++ b/_utils/terror_gen/errors_release.txt @@ -312,7 +312,7 @@ ErrSyncerUnitUpdateConfigInSharding,[code=36058:class=sync-unit:scope=internal:l ErrSyncerUnitExecWithNoBlockingDDL,[code=36059:class=sync-unit:scope=internal:level=high], "Message: process unit not waiting for sharding DDL to sync" ErrSyncerUnitGenBAList,[code=36060:class=sync-unit:scope=internal:level=high], "Message: generate block allow list, Workaround: Please check the `block-allow-list` config in task configuration file." ErrSyncerUnitHandleDDLFailed,[code=36061:class=sync-unit:scope=internal:level=high], "Message: fail to handle ddl job for %s" -ErrSyncerShardDDLConflict,[code=36062:class=sync-unit:scope=internal:level=high], "Message: fail to handle shard ddl %v in optimistic mode, because schema conflict detected, Workaround: Please use show-ddl-locks command for more details." +ErrSyncerShardDDLConflict,[code=36062:class=sync-unit:scope=internal:level=high], "Message: fail to handle shard ddl %v in optimistic mode, because schema conflict detected, conflict error: %s, Workaround: Please use show-ddl-locks command for more details." ErrSyncerFailpoint,[code=36063:class=sync-unit:scope=internal:level=low], "Message: failpoint specified error" ErrSyncerReplaceEvent,[code=36064:class=sync-unit:scope=internal:level=high] ErrSyncerOperatorNotExist,[code=36065:class=sync-unit:scope=internal:level=low], "Message: error operator not exist, position: %s" diff --git a/errors.toml b/errors.toml index 4a77f15f7e..8cbcd53856 100644 --- a/errors.toml +++ b/errors.toml @@ -1883,7 +1883,7 @@ workaround = "" tags = ["internal", "high"] [error.DM-sync-unit-36062] -message = "fail to handle shard ddl %v in optimistic mode, because schema conflict detected" +message = "fail to handle shard ddl %v in optimistic mode, because schema conflict detected, conflict error: %s" description = "" workaround = "Please use show-ddl-locks command for more details." tags = ["internal", "high"] From 56fb28bc5aeca52061eba28fd43ef0e3e329a20c Mon Sep 17 00:00:00 2001 From: Chunzhu Li Date: Fri, 26 Mar 2021 11:14:33 +0800 Subject: [PATCH 18/23] change ModRevision to Revision --- dm/master/shardddl/optimist.go | 2 +- pkg/ha/bound.go | 2 +- pkg/ha/stage.go | 2 +- pkg/shardddl/optimism/info.go | 6 +++--- pkg/shardddl/optimism/info_test.go | 10 +++++----- pkg/shardddl/optimism/ops_test.go | 4 ++-- syncer/shardddl/optimist_test.go | 4 ++-- 7 files changed, 15 insertions(+), 15 deletions(-) diff --git a/dm/master/shardddl/optimist.go b/dm/master/shardddl/optimist.go index 5b7ed22925..a4f69df698 100644 --- a/dm/master/shardddl/optimist.go +++ b/dm/master/shardddl/optimist.go @@ -572,7 +572,7 @@ func (o *Optimist) handleLock(info optimism.Info, tts []optimism.TargetTable, sk } op := optimism.NewOperation(lockID, lock.Task, info.Source, info.UpSchema, info.UpTable, newDDLs, cfStage, cfMsg, false) - rev, succ, err := optimism.PutOperation(o.cli, skipDone, op, info.ModRevision) + rev, succ, err := optimism.PutOperation(o.cli, skipDone, op, info.Revision) if err != nil { return err } diff --git a/pkg/ha/bound.go b/pkg/ha/bound.go index 8b8e32dd28..e712600658 100644 --- a/pkg/ha/bound.go +++ b/pkg/ha/bound.go @@ -47,7 +47,7 @@ type SourceBound struct { // only used to report to the caller of the watcher, do not marsh it. // if it's true, it means the bound has been deleted in etcd. IsDeleted bool `json:"-"` - // record the etcd ModRevision of this bound + // record the etcd Revision of this bound Revision int64 `json:"-"` } diff --git a/pkg/ha/stage.go b/pkg/ha/stage.go index 584e61fb57..9cf39cea69 100644 --- a/pkg/ha/stage.go +++ b/pkg/ha/stage.go @@ -38,7 +38,7 @@ type Stage struct { // only used to report to the caller of the watcher, do not marsh it. // if it's true, it means the stage has been deleted in etcd. IsDeleted bool `json:"-"` - // record the etcd ModRevision of this Stage + // record the etcd Revision of this Stage Revision int64 `json:"-"` } diff --git a/pkg/shardddl/optimism/info.go b/pkg/shardddl/optimism/info.go index 0d500bff95..c71a193bb6 100644 --- a/pkg/shardddl/optimism/info.go +++ b/pkg/shardddl/optimism/info.go @@ -58,7 +58,7 @@ type Info struct { IgnoreConflict bool `json:"ignore-conflict"` // only set it when get/watch from etcd - ModRevision int64 `json:"-"` + Revision int64 `json:"-"` } // NewInfo creates a new Info instance. @@ -135,7 +135,7 @@ func GetAllInfo(cli *clientv3.Client) (map[string]map[string]map[string]map[stri return nil, 0, err2 } info.Version = kv.Version - info.ModRevision = kv.ModRevision + info.Revision = kv.ModRevision if _, ok := ifm[info.Task]; !ok { ifm[info.Task] = make(map[string]map[string]map[string]Info) @@ -186,7 +186,7 @@ func WatchInfo(ctx context.Context, cli *clientv3.Client, revision int64, case mvccpb.PUT: info, err = infoFromJSON(string(ev.Kv.Value)) info.Version = ev.Kv.Version - info.ModRevision = ev.Kv.ModRevision + info.Revision = ev.Kv.ModRevision case mvccpb.DELETE: info, err = infoFromJSON(string(ev.PrevKv.Value)) info.IsDeleted = true diff --git a/pkg/shardddl/optimism/info_test.go b/pkg/shardddl/optimism/info_test.go index d07edc097c..d9d9263f45 100644 --- a/pkg/shardddl/optimism/info_test.go +++ b/pkg/shardddl/optimism/info_test.go @@ -128,7 +128,7 @@ func (t *testForEtcd) TestInfoEtcd(c *C) { c.Assert(ifm[task1][source1][upSchema], HasLen, 1) i11WithVer := i11 i11WithVer.Version = 2 - i11WithVer.ModRevision = rev2 + i11WithVer.Revision = rev2 c.Assert(ifm[task1][source1][upSchema][upTable], DeepEquals, i11WithVer) // put another key and get again with 2 info. @@ -142,7 +142,7 @@ func (t *testForEtcd) TestInfoEtcd(c *C) { c.Assert(ifm[task1][source1][upSchema][upTable], DeepEquals, i11WithVer) i12WithVer := i12 i12WithVer.Version = 1 - i12WithVer.ModRevision = rev4 + i12WithVer.Revision = rev4 c.Assert(ifm[task1][source2][upSchema][upTable], DeepEquals, i12WithVer) // start the watcher. @@ -164,7 +164,7 @@ func (t *testForEtcd) TestInfoEtcd(c *C) { infoWithVer := <-wch i21WithVer := i21 i21WithVer.Version = 1 - i21WithVer.ModRevision = rev5 + i21WithVer.Revision = rev5 c.Assert(infoWithVer, DeepEquals, i21WithVer) c.Assert(len(ech), Equals, 0) @@ -174,7 +174,7 @@ func (t *testForEtcd) TestInfoEtcd(c *C) { c.Assert(err, IsNil) infoWithVer = <-wch i21WithVer.Version++ - i21WithVer.ModRevision = rev6 + i21WithVer.Revision = rev6 c.Assert(infoWithVer, DeepEquals, i21WithVer) c.Assert(len(ech), Equals, 0) @@ -195,7 +195,7 @@ func (t *testForEtcd) TestInfoEtcd(c *C) { c.Assert(err, IsNil) infoWithVer = <-wch i21WithVer.Version = 1 - i21WithVer.ModRevision = rev7 + i21WithVer.Revision = rev7 c.Assert(infoWithVer, DeepEquals, i21WithVer) c.Assert(len(ech), Equals, 0) diff --git a/pkg/shardddl/optimism/ops_test.go b/pkg/shardddl/optimism/ops_test.go index a190774dd2..1258a71100 100644 --- a/pkg/shardddl/optimism/ops_test.go +++ b/pkg/shardddl/optimism/ops_test.go @@ -41,7 +41,7 @@ func (t *testForEtcd) TestDeleteInfosOperationsSchema(c *C) { c.Assert(ifm, HasLen, 1) infoWithVer := info infoWithVer.Version = 1 - infoWithVer.ModRevision = rev + infoWithVer.Revision = rev c.Assert(ifm[task][source][upSchema][upTable], DeepEquals, infoWithVer) // put operation. @@ -134,7 +134,7 @@ func (t *testForEtcd) TestSourceTablesInfo(c *C) { c.Assert(ifm[task][source][upSchema], HasLen, 1) i11WithVer := i11 i11WithVer.Version = 1 - i11WithVer.ModRevision = rev3 + i11WithVer.Revision = rev3 c.Assert(ifm[task][source][upSchema][upTable], DeepEquals, i11WithVer) // put/update source tables and delete info. diff --git a/syncer/shardddl/optimist_test.go b/syncer/shardddl/optimist_test.go index 102bc8f7df..e727e2b914 100644 --- a/syncer/shardddl/optimist_test.go +++ b/syncer/shardddl/optimist_test.go @@ -147,7 +147,7 @@ func (t *testOptimist) TestOptimist(c *C) { c.Assert(ifm[task][source][info1.UpSchema], HasLen, 1) info1WithVer := info1 info1WithVer.Version = 1 - info1WithVer.ModRevision = rev1 + info1WithVer.Revision = rev1 c.Assert(ifm[task][source][info1.UpSchema][info1.UpTable], DeepEquals, info1WithVer) opc := op1c opc.Done = true @@ -171,7 +171,7 @@ func (t *testOptimist) TestOptimist(c *C) { c.Assert(err, IsNil) infoCreateWithVer := infoCreate infoCreateWithVer.Version = 1 - infoCreateWithVer.ModRevision = rev3 + infoCreateWithVer.Revision = rev3 c.Assert(ifm[task][source][infoCreate.UpSchema][infoCreate.UpTable], DeepEquals, infoCreateWithVer) c.Assert(o.tables.Tables[infoCreate.DownSchema][infoCreate.DownTable][infoCreate.UpSchema], HasKey, infoCreate.UpTable) From bfb7a17d03935d61dd503186c8749603517b4608 Mon Sep 17 00:00:00 2001 From: Chunzhu Li Date: Fri, 26 Mar 2021 14:04:41 +0800 Subject: [PATCH 19/23] delete colm in lockKeeper --- dm/master/shardddl/optimist.go | 4 +--- pkg/shardddl/optimism/column.go | 1 + pkg/shardddl/optimism/keeper.go | 19 ++++--------------- pkg/shardddl/optimism/keeper_test.go | 11 ++++++++++- 4 files changed, 16 insertions(+), 19 deletions(-) diff --git a/dm/master/shardddl/optimist.go b/dm/master/shardddl/optimist.go index ca421f002b..9fabc1494e 100644 --- a/dm/master/shardddl/optimist.go +++ b/dm/master/shardddl/optimist.go @@ -322,14 +322,12 @@ func (o *Optimist) recoverLocks( ifm map[string]map[string]map[string]map[string]optimism.Info, opm map[string]map[string]map[string]map[string]optimism.Operation, colm map[string]map[string]map[string]map[string]map[string]struct{}) error { - o.lk.SetColumnMap(colm) - defer o.lk.SetColumnMap(nil) // construct joined table based on the shard DDL info. o.logger.Info("build lock joined and tts") lockJoined, lockTTS := o.buildLockJoinedAndTTS(ifm) // build lock and restore table info o.logger.Info("rebuild locks and tables") - o.lk.RebuildLocksAndTables(o.cli, ifm, lockJoined, lockTTS) + o.lk.RebuildLocksAndTables(o.cli, ifm, colm, lockJoined, lockTTS) // sort infos by revision infos := sortInfos(ifm) var firstErr error diff --git a/pkg/shardddl/optimism/column.go b/pkg/shardddl/optimism/column.go index 2044103729..bd0f4e7f5f 100644 --- a/pkg/shardddl/optimism/column.go +++ b/pkg/shardddl/optimism/column.go @@ -21,6 +21,7 @@ import ( ) // GetAllDroppedColumns gets the all partially dropped columns. +// return lockID -> column-name -> source-id -> upstream-schema-name -> upstream-table-name func GetAllDroppedColumns(cli *clientv3.Client) (map[string]map[string]map[string]map[string]map[string]struct{}, int64, error) { colm := make(map[string]map[string]map[string]map[string]map[string]struct{}) op := clientv3.OpGet(common.ShardDDLOptimismDroppedColumnsKeyAdapter.Path(), clientv3.WithPrefix()) diff --git a/pkg/shardddl/optimism/keeper.go b/pkg/shardddl/optimism/keeper.go index 6fe59005d2..72cf2b40d3 100644 --- a/pkg/shardddl/optimism/keeper.go +++ b/pkg/shardddl/optimism/keeper.go @@ -29,8 +29,6 @@ import ( type LockKeeper struct { mu sync.RWMutex locks map[string]*Lock // lockID -> Lock - // lockID -> column name -> source -> upSchema -> upTable -> struct{} - colm map[string]map[string]map[string]map[string]map[string]struct{} } // NewLockKeeper creates a new LockKeeper instance. @@ -44,6 +42,7 @@ func NewLockKeeper() *LockKeeper { func (lk *LockKeeper) RebuildLocksAndTables( cli *clientv3.Client, ifm map[string]map[string]map[string]map[string]Info, + colm map[string]map[string]map[string]map[string]map[string]struct{}, lockJoined map[string]schemacmp.Table, lockTTS map[string][]TargetTable) { var ( @@ -60,6 +59,9 @@ func (lk *LockKeeper) RebuildLocksAndTables( lock = lk.locks[lockID] } lock.tables[info.Source][info.UpSchema][info.UpTable] = schemacmp.Encode(info.TableInfoBefore) + if columns, ok := colm[lockID]; ok { + lock.columns = columns + } } } } @@ -84,11 +86,6 @@ func (lk *LockKeeper) TrySync(cli *clientv3.Client, info Info, tts []TargetTable if l, ok = lk.locks[lockID]; !ok { lk.locks[lockID] = NewLock(cli, lockID, info.Task, info.DownSchema, info.DownTable, schemacmp.Encode(info.TableInfoBefore), tts) l = lk.locks[lockID] - if lk.colm != nil { - if columns, ok := lk.colm[lockID]; ok { - l.columns = columns - } - } } newDDLs, err := l.TrySync(info, tts) @@ -144,14 +141,6 @@ func (lk *LockKeeper) Clear() { lk.locks = make(map[string]*Lock) } -// SetColumnMap sets the column map received from etcd -func (lk *LockKeeper) SetColumnMap(colm map[string]map[string]map[string]map[string]map[string]struct{}) { - lk.mu.Lock() - defer lk.mu.Unlock() - - lk.colm = colm -} - // genDDLLockID generates DDL lock ID from its info. func genDDLLockID(info Info) string { return utils.GenDDLLockID(info.Task, info.DownSchema, info.DownTable) diff --git a/pkg/shardddl/optimism/keeper_test.go b/pkg/shardddl/optimism/keeper_test.go index 8a2ed1cb5e..c049197f3e 100644 --- a/pkg/shardddl/optimism/keeper_test.go +++ b/pkg/shardddl/optimism/keeper_test.go @@ -431,6 +431,14 @@ func (t *testKeeper) TestRebuildLocksAndTables(c *C) { source2: {upSchema: {upTable: i21}}, }, } + colm = map[string]map[string]map[string]map[string]map[string]struct{}{ + lockID: { + "c3": { + source1: {upSchema: {upTable: {}}}, + source2: {upSchema: {upTable: {}}}, + }, + }, + } lockJoined = map[string]schemacmp.Table{ lockID: schemacmp.Encode(ti2), } @@ -439,7 +447,7 @@ func (t *testKeeper) TestRebuildLocksAndTables(c *C) { } ) - lk.RebuildLocksAndTables(etcdTestCli, ifm, lockJoined, lockTTS) + lk.RebuildLocksAndTables(etcdTestCli, ifm, colm, lockJoined, lockTTS) locks := lk.Locks() c.Assert(len(locks), Equals, 1) lock, ok := locks[lockID] @@ -453,4 +461,5 @@ func (t *testKeeper) TestRebuildLocksAndTables(c *C) { cmp, err = lock.tables[source2][upSchema][upTable].Compare(schemacmp.Encode(ti2)) c.Assert(err, IsNil) c.Assert(cmp, Equals, 0) + c.Assert(lock.columns, DeepEquals, colm[lockID]) } From 235776f591033bdbf1b9bbd6226a712afbce978d Mon Sep 17 00:00:00 2001 From: Chunzhu Li Date: Fri, 26 Mar 2021 14:19:00 +0800 Subject: [PATCH 20/23] refine some logs --- pkg/shardddl/optimism/lock.go | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/pkg/shardddl/optimism/lock.go b/pkg/shardddl/optimism/lock.go index 98028cb387..0eb049b8f6 100644 --- a/pkg/shardddl/optimism/lock.go +++ b/pkg/shardddl/optimism/lock.go @@ -539,7 +539,7 @@ func (l *Lock) GetVersion(source string, schema string, table string) int64 { return l.versions[source][schema][table] } -// IsDroppedColumn checks whether this column is a not fully dropped column for this lock +// IsDroppedColumn checks whether this column is a partially dropped column for this lock func (l *Lock) IsDroppedColumn(info Info, col string) bool { if _, ok := l.columns[col]; !ok { return false @@ -562,7 +562,7 @@ func (l *Lock) AddDroppedColumn(info Info, col string) error { if l.IsDroppedColumn(info, col) { return nil } - log.L().Debug("add not fully dropped columns", zap.Stringer("info", info), zap.String("column", col)) + log.L().Debug("add partially dropped columns", zap.Stringer("info", info), zap.String("column", col)) source, upSchema, upTable := info.Source, info.UpSchema, info.UpTable _, _, err := PutDroppedColumn(l.cli, info, col) @@ -601,7 +601,7 @@ func (l *Lock) DeleteColumnsByDDLs(ddls []string) error { } } if len(colsToDelete) > 0 { - log.L().Debug("delete not fully dropped columns", + log.L().Debug("delete partially dropped columns", zap.String("lockID", l.ID), zap.Strings("columns", colsToDelete)) _, _, err := DeleteDroppedColumns(l.cli, l.Task, l.DownSchema, l.DownTable, colsToDelete...) From 2efb0cfc7f7aa9753c80d3e68ab5d94f4e6cf954 Mon Sep 17 00:00:00 2001 From: Chunzhu Li Date: Fri, 26 Mar 2021 14:57:10 +0800 Subject: [PATCH 21/23] Update pkg/shardddl/optimism/lock.go --- pkg/shardddl/optimism/lock.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pkg/shardddl/optimism/lock.go b/pkg/shardddl/optimism/lock.go index 0eb049b8f6..9c0062f5d6 100644 --- a/pkg/shardddl/optimism/lock.go +++ b/pkg/shardddl/optimism/lock.go @@ -562,7 +562,7 @@ func (l *Lock) AddDroppedColumn(info Info, col string) error { if l.IsDroppedColumn(info, col) { return nil } - log.L().Debug("add partially dropped columns", zap.Stringer("info", info), zap.String("column", col)) + log.L().Debug("add partially dropped columns", zap.String("column", col), zap.Stringer("info", info)) source, upSchema, upTable := info.Source, info.UpSchema, info.UpTable _, _, err := PutDroppedColumn(l.cli, info, col) From 05b2c207a66279a22c1cf4a5d1971ba5035195c3 Mon Sep 17 00:00:00 2001 From: Chunzhu Li Date: Fri, 26 Mar 2021 15:40:29 +0800 Subject: [PATCH 22/23] fix unit test etcd cluster race usage --- dm/master/shardddl/optimist_test.go | 74 +++++++++++++++------------- dm/master/shardddl/pessimist_test.go | 2 + 2 files changed, 43 insertions(+), 33 deletions(-) diff --git a/dm/master/shardddl/optimist_test.go b/dm/master/shardddl/optimist_test.go index 130b629f4f..01df88f860 100644 --- a/dm/master/shardddl/optimist_test.go +++ b/dm/master/shardddl/optimist_test.go @@ -27,6 +27,8 @@ import ( tiddl "github.com/pingcap/tidb/ddl" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/util/mock" + "go.etcd.io/etcd/clientv3" + "go.etcd.io/etcd/integration" "github.com/pingcap/dm/dm/config" "github.com/pingcap/dm/dm/pb" @@ -146,14 +148,20 @@ func (t *testOptimist) TestOptimistSourceTables(c *C) { } func (t *testOptimist) TestOptimist(c *C) { - t.testOptimist(c, noRestart) - t.testOptimist(c, restartOnly) - t.testOptimist(c, restartNewInstance) + cluster := integration.NewClusterV3(tt, &integration.ClusterConfig{Size: 1}) + defer cluster.Terminate(tt) + + cli := cluster.RandClient() + t.testOptimist(c, cli, noRestart) + t.testOptimist(c, cli, restartOnly) + t.testOptimist(c, cli, restartNewInstance) t.testSortInfos(c) } -func (t *testOptimist) testOptimist(c *C, restart int) { - defer clearOptimistTestSourceInfoOperation(c) +func (t *testOptimist) testOptimist(c *C, cli *clientv3.Client, restart int) { + defer func() { + c.Assert(optimism.ClearTestInfoOperationSchema(cli), IsNil) + }() var ( backOff = 30 @@ -166,11 +174,11 @@ func (t *testOptimist) testOptimist(c *C, restart int) { switch restart { case restartOnly: o.Close() - c.Assert(o.Start(ctx, etcdTestCli), IsNil) + c.Assert(o.Start(ctx, cli), IsNil) case restartNewInstance: o.Close() o = NewOptimist(&logger) - c.Assert(o.Start(ctx, etcdTestCli), IsNil) + c.Assert(o.Start(ctx, cli), IsNil) } } @@ -208,24 +216,24 @@ func (t *testOptimist) testOptimist(c *C, restart int) { st32.AddTable("foo-2", "bar-3", downSchema, downTable) // put source tables first. - _, err := optimism.PutSourceTables(etcdTestCli, st1) + _, err := optimism.PutSourceTables(cli, st1) c.Assert(err, IsNil) ctx, cancel := context.WithCancel(context.Background()) defer cancel() // CASE 1: start without any previous shard DDL info. - c.Assert(o.Start(ctx, etcdTestCli), IsNil) + c.Assert(o.Start(ctx, cli), IsNil) c.Assert(o.Locks(), HasLen, 0) o.Close() o.Close() // close multiple times. // CASE 2: start again without any previous shard DDL info. - c.Assert(o.Start(ctx, etcdTestCli), IsNil) + c.Assert(o.Start(ctx, cli), IsNil) c.Assert(o.Locks(), HasLen, 0) // PUT i11, will create a lock but not synced. - rev1, err := optimism.PutInfo(etcdTestCli, i11) + rev1, err := optimism.PutInfo(cli, i11) c.Assert(err, IsNil) c.Assert(utils.WaitSomething(backOff, waitTime, func() bool { return len(o.Locks()) == 1 @@ -257,7 +265,7 @@ func (t *testOptimist) testOptimist(c *C, restart int) { opCh := make(chan optimism.Operation, 10) errCh := make(chan error, 10) ctx2, cancel2 := context.WithCancel(ctx) - go optimism.WatchOperationPut(ctx2, etcdTestCli, i11.Task, i11.Source, i11.UpSchema, i11.UpTable, rev1, opCh, errCh) + go optimism.WatchOperationPut(ctx2, cli, i11.Task, i11.Source, i11.UpSchema, i11.UpTable, rev1, opCh, errCh) utils.WaitSomething(10, watchTimeout, func() bool { return len(opCh) != 0 }) @@ -274,7 +282,7 @@ func (t *testOptimist) testOptimist(c *C, restart int) { // mark op11 as done. op11c := op11 op11c.Done = true - _, putted, err := optimism.PutOperation(etcdTestCli, false, op11c, 0) + _, putted, err := optimism.PutOperation(cli, false, op11c, 0) c.Assert(err, IsNil) c.Assert(putted, IsTrue) c.Assert(utils.WaitSomething(backOff, waitTime, func() bool { @@ -288,7 +296,7 @@ func (t *testOptimist) testOptimist(c *C, restart int) { c.Assert(o.ShowLocks("", []string{}), DeepEquals, expectedLock) // PUT i12, the lock will be synced. - rev2, err := optimism.PutInfo(etcdTestCli, i12) + rev2, err := optimism.PutInfo(cli, i12) c.Assert(err, IsNil) c.Assert(utils.WaitSomething(backOff, waitTime, func() bool { synced, _ = o.Locks()[lockID].IsSynced() @@ -315,7 +323,7 @@ func (t *testOptimist) testOptimist(c *C, restart int) { opCh = make(chan optimism.Operation, 10) errCh = make(chan error, 10) ctx2, cancel2 = context.WithCancel(ctx) - go optimism.WatchOperationPut(ctx2, etcdTestCli, i12.Task, i12.Source, i12.UpSchema, i12.UpTable, rev2, opCh, errCh) + go optimism.WatchOperationPut(ctx2, cli, i12.Task, i12.Source, i12.UpSchema, i12.UpTable, rev2, opCh, errCh) utils.WaitSomething(10, watchTimeout, func() bool { return len(opCh) != 0 }) @@ -332,7 +340,7 @@ func (t *testOptimist) testOptimist(c *C, restart int) { // mark op12 as done, the lock should be resolved. op12c := op12 op12c.Done = true - _, putted, err = optimism.PutOperation(etcdTestCli, false, op12c, 0) + _, putted, err = optimism.PutOperation(cli, false, op12c, 0) c.Assert(err, IsNil) c.Assert(putted, IsTrue) c.Assert(utils.WaitSomething(backOff, waitTime, func() bool { @@ -343,15 +351,15 @@ func (t *testOptimist) testOptimist(c *C, restart int) { c.Assert(o.ShowLocks("", nil), HasLen, 0) // no shard DDL info or lock operation exists. - ifm, _, err := optimism.GetAllInfo(etcdTestCli) + ifm, _, err := optimism.GetAllInfo(cli) c.Assert(err, IsNil) c.Assert(ifm, HasLen, 0) - opm, _, err := optimism.GetAllOperations(etcdTestCli) + opm, _, err := optimism.GetAllOperations(cli) c.Assert(err, IsNil) c.Assert(opm, HasLen, 0) // put another table info. - rev1, err = optimism.PutInfo(etcdTestCli, i21) + rev1, err = optimism.PutInfo(cli, i21) c.Assert(err, IsNil) c.Assert(utils.WaitSomething(backOff, waitTime, func() bool { return len(o.Locks()) == 1 @@ -365,7 +373,7 @@ func (t *testOptimist) testOptimist(c *C, restart int) { opCh = make(chan optimism.Operation, 10) errCh = make(chan error, 10) ctx2, cancel2 = context.WithCancel(ctx) - go optimism.WatchOperationPut(ctx2, etcdTestCli, i21.Task, i21.Source, i21.UpSchema, i21.UpTable, rev1, opCh, errCh) + go optimism.WatchOperationPut(ctx2, cli, i21.Task, i21.Source, i21.UpSchema, i21.UpTable, rev1, opCh, errCh) utils.WaitSomething(10, watchTimeout, func() bool { return len(opCh) != 0 }) @@ -387,7 +395,7 @@ func (t *testOptimist) testOptimist(c *C, restart int) { c.Assert(remain, Equals, 1) // put table info for a new table (to simulate `CREATE TABLE`). - rev3, err := optimism.PutSourceTablesInfo(etcdTestCli, st32, i23) + rev3, err := optimism.PutSourceTablesInfo(cli, st32, i23) c.Assert(err, IsNil) c.Assert(utils.WaitSomething(backOff, waitTime, func() bool { ready := o.Locks()[lockID].Ready() @@ -432,7 +440,7 @@ func (t *testOptimist) testOptimist(c *C, restart int) { opCh = make(chan optimism.Operation, 10) errCh = make(chan error, 10) ctx2, cancel2 = context.WithCancel(ctx) - go optimism.WatchOperationPut(ctx2, etcdTestCli, i23.Task, i23.Source, i23.UpSchema, i23.UpTable, rev3, opCh, errCh) + go optimism.WatchOperationPut(ctx2, cli, i23.Task, i23.Source, i23.UpSchema, i23.UpTable, rev3, opCh, errCh) utils.WaitSomething(10, watchTimeout, func() bool { return len(opCh) != 0 }) @@ -446,13 +454,13 @@ func (t *testOptimist) testOptimist(c *C, restart int) { c.Assert(len(errCh), Equals, 0) // delete i12 for a table (to simulate `DROP TABLE`), the lock should become synced again. - rev2, err = optimism.PutInfo(etcdTestCli, i12) // put i12 first to trigger DELETE for i12. + rev2, err = optimism.PutInfo(cli, i12) // put i12 first to trigger DELETE for i12. c.Assert(err, IsNil) // wait until operation for i12 ready. opCh = make(chan optimism.Operation, 10) errCh = make(chan error, 10) ctx2, cancel2 = context.WithCancel(ctx) - go optimism.WatchOperationPut(ctx2, etcdTestCli, i12.Task, i12.Source, i12.UpSchema, i12.UpTable, rev2, opCh, errCh) + go optimism.WatchOperationPut(ctx2, cli, i12.Task, i12.Source, i12.UpSchema, i12.UpTable, rev2, opCh, errCh) utils.WaitSomething(10, watchTimeout, func() bool { return len(opCh) != 0 }) @@ -462,7 +470,7 @@ func (t *testOptimist) testOptimist(c *C, restart int) { c.Assert(len(opCh), Equals, 1) c.Assert(len(errCh), Equals, 0) - _, err = optimism.PutSourceTablesDeleteInfo(etcdTestCli, st31, i12) + _, err = optimism.PutSourceTablesDeleteInfo(cli, st31, i12) c.Assert(err, IsNil) c.Assert(utils.WaitSomething(backOff, waitTime, func() bool { synced, _ = o.Locks()[lockID].IsSynced() @@ -493,7 +501,7 @@ func (t *testOptimist) testOptimist(c *C, restart int) { // mark op21 as done. op21c := op21 op21c.Done = true - _, putted, err = optimism.PutOperation(etcdTestCli, false, op21c, 0) + _, putted, err = optimism.PutOperation(cli, false, op21c, 0) c.Assert(err, IsNil) c.Assert(putted, IsTrue) c.Assert(utils.WaitSomething(backOff, waitTime, func() bool { @@ -513,7 +521,7 @@ func (t *testOptimist) testOptimist(c *C, restart int) { // mark op23 as done. op23c := op23 op23c.Done = true - _, putted, err = optimism.PutOperation(etcdTestCli, false, op23c, 0) + _, putted, err = optimism.PutOperation(cli, false, op23c, 0) c.Assert(err, IsNil) c.Assert(putted, IsTrue) c.Assert(utils.WaitSomething(backOff, waitTime, func() bool { @@ -523,7 +531,7 @@ func (t *testOptimist) testOptimist(c *C, restart int) { c.Assert(o.Locks(), HasLen, 0) // PUT i31, will create a lock but not synced (to test `DROP COLUMN`) - rev1, err = optimism.PutInfo(etcdTestCli, i31) + rev1, err = optimism.PutInfo(cli, i31) c.Assert(err, IsNil) c.Assert(utils.WaitSomething(backOff, waitTime, func() bool { return len(o.Locks()) == 1 @@ -555,7 +563,7 @@ func (t *testOptimist) testOptimist(c *C, restart int) { opCh = make(chan optimism.Operation, 10) errCh = make(chan error, 10) ctx2, cancel2 = context.WithCancel(ctx) - go optimism.WatchOperationPut(ctx2, etcdTestCli, i31.Task, i31.Source, i31.UpSchema, i31.UpTable, rev1, opCh, errCh) + go optimism.WatchOperationPut(ctx2, cli, i31.Task, i31.Source, i31.UpSchema, i31.UpTable, rev1, opCh, errCh) utils.WaitSomething(10, watchTimeout, func() bool { return len(opCh) != 0 }) @@ -572,7 +580,7 @@ func (t *testOptimist) testOptimist(c *C, restart int) { // mark op31 as done. op31c := op31 op31c.Done = true - _, putted, err = optimism.PutOperation(etcdTestCli, false, op31c, 0) + _, putted, err = optimism.PutOperation(cli, false, op31c, 0) c.Assert(err, IsNil) c.Assert(putted, IsTrue) c.Assert(utils.WaitSomething(backOff, waitTime, func() bool { @@ -581,7 +589,7 @@ func (t *testOptimist) testOptimist(c *C, restart int) { c.Assert(o.ShowLocks("", []string{}), DeepEquals, expectedLock) // PUT i33, the lock will be synced. - rev3, err = optimism.PutInfo(etcdTestCli, i33) + rev3, err = optimism.PutInfo(cli, i33) c.Assert(err, IsNil) c.Assert(utils.WaitSomething(backOff, waitTime, func() bool { synced, _ = o.Locks()[lockID].IsSynced() @@ -608,7 +616,7 @@ func (t *testOptimist) testOptimist(c *C, restart int) { opCh = make(chan optimism.Operation, 10) errCh = make(chan error, 10) ctx2, cancel2 = context.WithCancel(ctx) - go optimism.WatchOperationPut(ctx2, etcdTestCli, i33.Task, i33.Source, i33.UpSchema, i33.UpTable, rev3, opCh, errCh) + go optimism.WatchOperationPut(ctx2, cli, i33.Task, i33.Source, i33.UpSchema, i33.UpTable, rev3, opCh, errCh) utils.WaitSomething(10, watchTimeout, func() bool { return len(opCh) != 0 }) @@ -625,7 +633,7 @@ func (t *testOptimist) testOptimist(c *C, restart int) { // mark op33 as done, the lock should be resolved. op33c := op33 op33c.Done = true - _, putted, err = optimism.PutOperation(etcdTestCli, false, op33c, 0) + _, putted, err = optimism.PutOperation(cli, false, op33c, 0) c.Assert(err, IsNil) c.Assert(putted, IsTrue) c.Assert(utils.WaitSomething(backOff, waitTime, func() bool { diff --git a/dm/master/shardddl/pessimist_test.go b/dm/master/shardddl/pessimist_test.go index 2498a9536a..8e796c0e3e 100644 --- a/dm/master/shardddl/pessimist_test.go +++ b/dm/master/shardddl/pessimist_test.go @@ -37,6 +37,7 @@ import ( var ( etcdTestCli *clientv3.Client etcdErrCompacted = v3rpc.ErrCompacted + tt *testing.T ) const ( @@ -54,6 +55,7 @@ func TestShardDDL(t *testing.T) { if err != nil { t.Fatal(err) } + tt = t mockCluster := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1}) defer mockCluster.Terminate(t) From 213e07f002d16e565120f8453bdf9ca144972490 Mon Sep 17 00:00:00 2001 From: Chunzhu Li Date: Fri, 26 Mar 2021 15:43:13 +0800 Subject: [PATCH 23/23] fix again --- dm/master/shardddl/optimist_test.go | 22 ++++++++++++---------- 1 file changed, 12 insertions(+), 10 deletions(-) diff --git a/dm/master/shardddl/optimist_test.go b/dm/master/shardddl/optimist_test.go index 01df88f860..f4dd405e0c 100644 --- a/dm/master/shardddl/optimist_test.go +++ b/dm/master/shardddl/optimist_test.go @@ -155,7 +155,7 @@ func (t *testOptimist) TestOptimist(c *C) { t.testOptimist(c, cli, noRestart) t.testOptimist(c, cli, restartOnly) t.testOptimist(c, cli, restartNewInstance) - t.testSortInfos(c) + t.testSortInfos(c, cli) } func (t *testOptimist) testOptimist(c *C, cli *clientv3.Client, restart int) { @@ -1040,8 +1040,10 @@ func (t *testOptimist) TestOptimistInitSchema(c *C) { c.Assert(is.TableInfo, DeepEquals, ti1) // the init schema is ti1 now. } -func (t *testOptimist) testSortInfos(c *C) { - defer clearOptimistTestSourceInfoOperation(c) +func (t *testOptimist) testSortInfos(c *C, cli *clientv3.Client) { + defer func() { + c.Assert(optimism.ClearTestInfoOperationSchema(cli), IsNil) + }() var ( task = "test-optimist-init-schema" @@ -1064,9 +1066,9 @@ func (t *testOptimist) testSortInfos(c *C) { i21 = optimism.NewInfo(task, sources[1], upSchema, upTables[1], downSchema, downTable, DDLs2, ti1, []*model.TableInfo{ti2}) ) - rev1, err := optimism.PutInfo(etcdTestCli, i11) + rev1, err := optimism.PutInfo(cli, i11) c.Assert(err, IsNil) - ifm, _, err := optimism.GetAllInfo(etcdTestCli) + ifm, _, err := optimism.GetAllInfo(cli) c.Assert(err, IsNil) infos := sortInfos(ifm) c.Assert(len(infos), Equals, 1) @@ -1074,9 +1076,9 @@ func (t *testOptimist) testSortInfos(c *C) { i11.Revision = rev1 c.Assert(infos[0], DeepEquals, i11) - rev2, err := optimism.PutInfo(etcdTestCli, i12) + rev2, err := optimism.PutInfo(cli, i12) c.Assert(err, IsNil) - ifm, _, err = optimism.GetAllInfo(etcdTestCli) + ifm, _, err = optimism.GetAllInfo(cli) c.Assert(err, IsNil) infos = sortInfos(ifm) c.Assert(len(infos), Equals, 2) @@ -1087,11 +1089,11 @@ func (t *testOptimist) testSortInfos(c *C) { c.Assert(infos[0], DeepEquals, i11) c.Assert(infos[1], DeepEquals, i12) - rev3, err := optimism.PutInfo(etcdTestCli, i21) + rev3, err := optimism.PutInfo(cli, i21) c.Assert(err, IsNil) - rev4, err := optimism.PutInfo(etcdTestCli, i11) + rev4, err := optimism.PutInfo(cli, i11) c.Assert(err, IsNil) - ifm, _, err = optimism.GetAllInfo(etcdTestCli) + ifm, _, err = optimism.GetAllInfo(cli) c.Assert(err, IsNil) infos = sortInfos(ifm) c.Assert(len(infos), Equals, 3)