diff --git a/cdc/capture.go b/cdc/capture.go index 3a2e1c4c1d..17062b440b 100644 --- a/cdc/capture.go +++ b/cdc/capture.go @@ -19,8 +19,6 @@ import ( "sync" "time" - "github.com/coreos/etcd/clientv3" - "github.com/coreos/etcd/mvcc" "github.com/google/uuid" "github.com/pingcap/errors" "github.com/pingcap/log" @@ -32,10 +30,13 @@ import ( tidbkv "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/store" "github.com/pingcap/tidb/store/tikv" + "go.etcd.io/etcd/clientv3" + "go.etcd.io/etcd/mvcc" "go.uber.org/zap" "golang.org/x/sync/errgroup" "golang.org/x/time/rate" "google.golang.org/grpc" + "google.golang.org/grpc/backoff" ) const ( @@ -62,7 +63,15 @@ func NewCapture(pdEndpoints []string) (c *Capture, err error) { Endpoints: pdEndpoints, DialTimeout: 5 * time.Second, DialOptions: []grpc.DialOption{ - grpc.WithBackoffMaxDelay(time.Second * 3), + grpc.WithConnectParams(grpc.ConnectParams{ + Backoff: backoff.Config{ + BaseDelay: time.Second, + Multiplier: 1.1, + Jitter: 0.1, + MaxDelay: 3 * time.Second, + }, + MinConnectTimeout: 3 * time.Second, + }), }, }) if err != nil { diff --git a/cdc/capture_info.go b/cdc/capture_info.go index d6707e06a4..3fa34661ed 100644 --- a/cdc/capture_info.go +++ b/cdc/capture_info.go @@ -3,14 +3,14 @@ package cdc import ( "context" - "github.com/coreos/etcd/clientv3" - "github.com/coreos/etcd/mvcc" - "github.com/coreos/etcd/mvcc/mvccpb" "github.com/pingcap/errors" "github.com/pingcap/failpoint" "github.com/pingcap/log" "github.com/pingcap/ticdc/cdc/kv" "github.com/pingcap/ticdc/cdc/model" + "go.etcd.io/etcd/clientv3" + "go.etcd.io/etcd/mvcc" + "go.etcd.io/etcd/mvcc/mvccpb" ) var captureEinfoKeyPrefix = kv.EtcdKeyBase + "/capture/info" diff --git a/cdc/capture_info_test.go b/cdc/capture_info_test.go index 8804a0bb18..1829c8166a 100644 --- a/cdc/capture_info_test.go +++ b/cdc/capture_info_test.go @@ -6,15 +6,15 @@ import ( "sync/atomic" "time" - "github.com/coreos/etcd/clientv3" - "github.com/coreos/etcd/embed" - "github.com/coreos/etcd/mvcc" "github.com/pingcap/check" "github.com/pingcap/errors" "github.com/pingcap/failpoint" "github.com/pingcap/ticdc/cdc/model" "github.com/pingcap/ticdc/pkg/etcd" "github.com/pingcap/ticdc/pkg/util" + "go.etcd.io/etcd/clientv3" + "go.etcd.io/etcd/embed" + "go.etcd.io/etcd/mvcc" "golang.org/x/sync/errgroup" ) diff --git a/cdc/entry/codec.go b/cdc/entry/codec.go index 1d381e76a7..30fbee0caf 100644 --- a/cdc/entry/codec.go +++ b/cdc/entry/codec.go @@ -20,8 +20,10 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/parser/mysql" + "github.com/pingcap/ticdc/cdc/schema" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/codec" + "github.com/pingcap/tidb/util/rowcodec" ) var ( @@ -180,14 +182,29 @@ func decodeMetaKey(ek []byte) (meta, error) { } // decodeRow decodes a byte slice into datums with a existing row map. +func decodeRow(b []byte, recordID int64, tableInfo *schema.TableInfo) (map[int64]types.Datum, error) { + if len(b) == 0 { + if tableInfo.PKIsHandle { + id, pkValue, err := fetchHandleValue(tableInfo, recordID) + if err != nil { + return nil, errors.Trace(err) + } + return map[int64]types.Datum{id: *pkValue}, nil + } + return map[int64]types.Datum{}, nil + } + if rowcodec.IsNewFormat(b) { + return decodeRowV2(b, recordID, tableInfo) + } + return decodeRowV1(b, recordID, tableInfo) +} + +// decodeRowV1 decodes value data using old encoding format. // Row layout: colID1, value1, colID2, value2, ..... -func decodeRow(b []byte) (map[int64]types.Datum, error) { +func decodeRowV1(b []byte, recordID int64, tableInfo *schema.TableInfo) (map[int64]types.Datum, error) { row := make(map[int64]types.Datum) - if b == nil { - return row, nil - } if len(b) == 1 && b[0] == codec.NilFlag { - return row, nil + b = b[1:] } var err error var data []byte @@ -213,11 +230,39 @@ func decodeRow(b []byte) (map[int64]types.Datum, error) { return nil, errors.Trace(err) } - row[id] = v + // unflatten value + colInfo, exist := tableInfo.GetColumnInfo(id) + if !exist { + // can not find column info, ignore this column because the column should be in WRITE ONLY state + continue + } + fieldType := &colInfo.FieldType + datum, err := unflatten(v, fieldType) + if err != nil { + return nil, errors.Trace(err) + } + row[id] = datum + } + + if tableInfo.PKIsHandle { + id, pkValue, err := fetchHandleValue(tableInfo, recordID) + if err != nil { + return nil, errors.Trace(err) + } + row[id] = *pkValue } return row, nil } +// decodeRowV2 decodes value data using new encoding format. +// Ref: https://github.com/pingcap/tidb/pull/12634 +// https://github.com/pingcap/tidb/blob/master/docs/design/2018-07-19-row-format.md +func decodeRowV2(data []byte, recordID int64, tableInfo *schema.TableInfo) (map[int64]types.Datum, error) { + handleColID, reqCols := tableInfo.GetRowColInfos() + decoder := rowcodec.NewDatumMapDecoder(reqCols, handleColID, time.UTC) + return decoder.DecodeToDatumMap(data, recordID, nil) +} + // unflatten converts a raw datum to a column datum. func unflatten(datum types.Datum, ft *types.FieldType) (types.Datum, error) { if datum.IsNull() { @@ -233,9 +278,7 @@ func unflatten(datum types.Datum, ft *types.FieldType) (types.Datum, error) { mysql.TypeString: return datum, nil case mysql.TypeDate, mysql.TypeDatetime, mysql.TypeTimestamp: - var t types.Time - t.Type = ft.Tp - t.Fsp = int8(ft.Decimal) + t := types.NewTime(types.ZeroCoreTime, ft.Tp, int8(ft.Decimal)) var err error err = t.FromPackedUint(datum.GetUint64()) if err != nil { diff --git a/cdc/entry/kv_entry.go b/cdc/entry/kv_entry.go index 43c7169ca2..a3e777b5f9 100644 --- a/cdc/entry/kv_entry.go +++ b/cdc/entry/kv_entry.go @@ -105,42 +105,29 @@ func isDistinct(index *timodel.IndexInfo, indexValue []types.Datum) bool { return false } -func (row *rowKVEntry) unflatten(tableInfo *schema.TableInfo) error { - if tableInfo.ID != row.TableID { - return errors.New("wrong table info in unflatten") - } - for colID, v := range row.Row { - colInfo, exist := tableInfo.GetColumnInfo(colID) - if !exist { - log.Info("can not find column info, ignore this column because the column should be in WRITE ONLY state", zap.Int64("colID", colID), zap.Uint64("ts", row.Ts)) - delete(row.Row, colID) - continue - } - fieldType := &colInfo.FieldType - datum, err := unflatten(v, fieldType) - if err != nil { - return errors.Trace(err) - } - row.Row[colID] = datum - } - return nil -} - -func unmarshal(raw *model.RawKVEntry) (kvEntry, error) { +func (m *Mounter) unmarshal(raw *model.RawKVEntry) (kvEntry, error) { switch { case bytes.HasPrefix(raw.Key, tablePrefix): - return unmarshalTableKVEntry(raw) + return m.unmarshalTableKVEntry(raw) case bytes.HasPrefix(raw.Key, metaPrefix) && raw.OpType == model.OpTypePut: - return unmarshalMetaKVEntry(raw) + return m.unmarshalMetaKVEntry(raw) } - return &unknownKVEntry{*raw}, nil + return nil, nil } -func unmarshalTableKVEntry(raw *model.RawKVEntry) (kvEntry, error) { +func (m *Mounter) unmarshalTableKVEntry(raw *model.RawKVEntry) (kvEntry, error) { key, tableID, err := decodeTableID(raw.Key) if err != nil { return nil, errors.Trace(err) } + tableInfo, exist := m.schemaStorage.TableByID(tableID) + if !exist { + if m.schemaStorage.IsTruncateTableID(tableID) { + log.Debug("skip the DML of truncated table", zap.Uint64("ts", raw.Ts), zap.Int64("tableID", tableID)) + return nil, nil + } + return nil, errors.NotFoundf("table in schema storage, id: %d", tableID) + } switch { case bytes.HasPrefix(key, recordPrefix): key, recordID, err := decodeRecordID(key) @@ -150,7 +137,7 @@ func unmarshalTableKVEntry(raw *model.RawKVEntry) (kvEntry, error) { if len(key) != 0 { return nil, errors.New("invalid record key") } - row, err := decodeRow(raw.Value) + row, err := decodeRow(raw.Value, recordID, tableInfo) if err != nil { return nil, errors.Trace(err) } @@ -196,7 +183,7 @@ const ( tableMetaPrefix = "Table:" ) -func unmarshalMetaKVEntry(raw *model.RawKVEntry) (kvEntry, error) { +func (m *Mounter) unmarshalMetaKVEntry(raw *model.RawKVEntry) (kvEntry, error) { meta, err := decodeMetaKey(raw.Key) if err != nil { return nil, errors.Trace(err) diff --git a/cdc/entry/kv_entry_test.go b/cdc/entry/kv_entry_test.go index d41aad7019..2d0a5809e2 100644 --- a/cdc/entry/kv_entry_test.go +++ b/cdc/entry/kv_entry_test.go @@ -3,7 +3,6 @@ package entry import ( "context" "reflect" - "time" "github.com/pingcap/check" "github.com/pingcap/errors" @@ -13,7 +12,6 @@ import ( "github.com/pingcap/ticdc/cdc/puller" "github.com/pingcap/ticdc/cdc/schema" "github.com/pingcap/ticdc/pkg/util" - "github.com/pingcap/tidb/store/tikv/oracle" "github.com/pingcap/tidb/types" ) @@ -22,9 +20,10 @@ type kvEntrySuite struct { var _ = check.Suite(&kvEntrySuite{}) -func (s *kvEntrySuite) TestCreateTable(c *check.C) { - pm, pmCtx, pmCancel := startPullerManager(c) - defer pmCancel() +func (s *kvEntrySuite) testCreateTable(c *check.C, newRowFormat bool) { + ctx, cancel := context.WithCancel(context.Background()) + pm, schema := setUpPullerAndSchema(ctx, c, newRowFormat) + m := NewTxnMounter(schema) pm.MustExec("create table test.test1(id varchar(255) primary key, a int, index i1 (a))") @@ -32,10 +31,10 @@ func (s *kvEntrySuite) TestCreateTable(c *check.C) { existDDLJobHistoryKVEntry := false // create another context with canceled, we can close this puller but not affect puller manager - plrCtx, plrCancel := context.WithCancel(pmCtx) + plrCtx, plrCancel := context.WithCancel(ctx) err := plr.CollectRawTxns(plrCtx, func(ctx context.Context, rawTxn model.RawTxn) error { for _, raw := range rawTxn.Entries { - entry, err := unmarshal(raw) + entry, err := m.unmarshal(raw) c.Assert(err, check.IsNil) if e, ok := entry.(*ddlJobKVEntry); ok { existDDLJobHistoryKVEntry = true @@ -73,17 +72,21 @@ func (s *kvEntrySuite) TestCreateTable(c *check.C) { }) c.Assert(errors.Cause(err), check.Equals, context.Canceled) c.Assert(existDDLJobHistoryKVEntry, check.IsTrue) + cancel() // create another puller, pull KVs from now - nowTso := oracle.EncodeTSO(time.Now().UnixNano() / int64(time.Millisecond)) - plr = pm.CreatePuller(nowTso, []util.Span{util.GetDDLSpan()}) + ctx, cancel = context.WithCancel(context.Background()) + pm, schema = setUpPullerAndSchema(ctx, c, newRowFormat) + m = NewTxnMounter(schema) + pm.MustExec("create table test.test2(id int primary key, b varchar(255) unique key)") + plr = pm.CreatePuller(0, []util.Span{util.GetDDLSpan()}) existDDLJobHistoryKVEntry = false - plrCtx, plrCancel = context.WithCancel(pmCtx) + plrCtx, plrCancel = context.WithCancel(ctx) err = plr.CollectRawTxns(plrCtx, func(ctx context.Context, rawTxn model.RawTxn) error { for _, raw := range rawTxn.Entries { - entry, err := unmarshal(raw) + entry, err := m.unmarshal(raw) c.Assert(err, check.IsNil) if e, ok := entry.(*ddlJobKVEntry); ok { existDDLJobHistoryKVEntry = true @@ -113,13 +116,17 @@ func (s *kvEntrySuite) TestCreateTable(c *check.C) { }) c.Assert(errors.Cause(err), check.Equals, context.Canceled) c.Assert(existDDLJobHistoryKVEntry, check.IsTrue) + cancel() } -func (s *kvEntrySuite) TestPkIsNotHandleDML(c *check.C) { - pm, pmCtx, pmCancel := startPullerManager(c) - defer pmCancel() +func (s *kvEntrySuite) testPkIsNotHandleDML(c *check.C, newRowFormat bool) { + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + pm, schema := setUpPullerAndSchema(ctx, c, newRowFormat, + "create table test.test1(id varchar(255) primary key, a int, index ci (a))", + ) + m := NewTxnMounter(schema) - pm.MustExec("create table test.test1(id varchar(255) primary key, a int, index ci (a))") tableInfo := pm.GetTableInfo("test", "test1") tableID := tableInfo.ID @@ -145,7 +152,7 @@ func (s *kvEntrySuite) TestPkIsNotHandleDML(c *check.C) { Delete: false, IndexValue: []types.Datum{types.NewBytesDatum([]byte("ttt"))}, }} - checkDMLKVEntries(pmCtx, c, tableInfo, plr, expect) + checkDMLKVEntries(ctx, c, tableInfo, m, plr, expect) pm.MustExec("update test.test1 set id = '777' where a = 666") expect = []kvEntry{ @@ -167,7 +174,7 @@ func (s *kvEntrySuite) TestPkIsNotHandleDML(c *check.C) { Delete: true, IndexValue: []types.Datum{types.NewBytesDatum([]byte("ttt"))}, }} - checkDMLKVEntries(pmCtx, c, tableInfo, plr, expect) + checkDMLKVEntries(ctx, c, tableInfo, m, plr, expect) pm.MustExec("delete from test.test1 where id = '777'") expect = []kvEntry{ @@ -189,14 +196,17 @@ func (s *kvEntrySuite) TestPkIsNotHandleDML(c *check.C) { Delete: true, IndexValue: []types.Datum{types.NewIntDatum(666)}, }} - checkDMLKVEntries(pmCtx, c, tableInfo, plr, expect) + checkDMLKVEntries(ctx, c, tableInfo, m, plr, expect) } -func (s *kvEntrySuite) TestPkIsHandleDML(c *check.C) { - pm, pmCtx, pmCancel := startPullerManager(c) - defer pmCancel() +func (s *kvEntrySuite) testPkIsHandleDML(c *check.C, newRowFormat bool) { + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + pm, schema := setUpPullerAndSchema(ctx, c, newRowFormat, + "create table test.test2(id int primary key, b varchar(255) unique key)", + ) + m := NewTxnMounter(schema) - pm.MustExec("create table test.test2(id int primary key, b varchar(255) unique key)") tableInfo := pm.GetTableInfo("test", "test2") tableID := tableInfo.ID @@ -208,7 +218,7 @@ func (s *kvEntrySuite) TestPkIsHandleDML(c *check.C) { TableID: tableInfo.ID, RecordID: 666, Delete: false, - Row: map[int64]types.Datum{2: types.NewBytesDatum([]byte("aaa"))}, + Row: map[int64]types.Datum{1: types.NewIntDatum(666), 2: types.NewBytesDatum([]byte("aaa"))}, }, &indexKVEntry{ TableID: tableInfo.ID, RecordID: 666, @@ -216,7 +226,7 @@ func (s *kvEntrySuite) TestPkIsHandleDML(c *check.C) { Delete: false, IndexValue: []types.Datum{types.NewBytesDatum([]byte("aaa"))}, }} - checkDMLKVEntries(pmCtx, c, tableInfo, plr, expect) + checkDMLKVEntries(ctx, c, tableInfo, m, plr, expect) pm.MustExec("update test.test2 set id = 888,b = 'bbb' where id = 666") expect = []kvEntry{ @@ -224,12 +234,12 @@ func (s *kvEntrySuite) TestPkIsHandleDML(c *check.C) { TableID: tableInfo.ID, RecordID: 666, Delete: true, - Row: map[int64]types.Datum{}, + Row: map[int64]types.Datum{1: types.NewIntDatum(666)}, }, &rowKVEntry{ TableID: tableInfo.ID, RecordID: 888, Delete: false, - Row: map[int64]types.Datum{2: types.NewBytesDatum([]byte("bbb"))}, + Row: map[int64]types.Datum{1: types.NewIntDatum(888), 2: types.NewBytesDatum([]byte("bbb"))}, }, &indexKVEntry{ TableID: tableInfo.ID, RecordID: 0, @@ -243,7 +253,7 @@ func (s *kvEntrySuite) TestPkIsHandleDML(c *check.C) { Delete: false, IndexValue: []types.Datum{types.NewBytesDatum([]byte("bbb"))}, }} - checkDMLKVEntries(pmCtx, c, tableInfo, plr, expect) + checkDMLKVEntries(ctx, c, tableInfo, m, plr, expect) pm.MustExec("delete from test.test2 where id = 888") expect = []kvEntry{ @@ -251,7 +261,7 @@ func (s *kvEntrySuite) TestPkIsHandleDML(c *check.C) { TableID: tableInfo.ID, RecordID: 888, Delete: true, - Row: map[int64]types.Datum{}, + Row: map[int64]types.Datum{1: types.NewIntDatum(888)}, }, &indexKVEntry{ TableID: tableInfo.ID, RecordID: 0, @@ -259,14 +269,17 @@ func (s *kvEntrySuite) TestPkIsHandleDML(c *check.C) { Delete: true, IndexValue: []types.Datum{types.NewBytesDatum([]byte("bbb"))}, }} - checkDMLKVEntries(pmCtx, c, tableInfo, plr, expect) + checkDMLKVEntries(ctx, c, tableInfo, m, plr, expect) } -func (s *kvEntrySuite) TestUkWithNull(c *check.C) { - pm, pmCtx, pmCancel := startPullerManager(c) - defer pmCancel() +func (s *kvEntrySuite) testUkWithNull(c *check.C, newRowFormat bool) { + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + pm, schema := setUpPullerAndSchema(ctx, c, newRowFormat, + "create table test.test2( a int, b varchar(255), c date, unique key(a,b,c))", + ) + m := NewTxnMounter(schema) - pm.MustExec("create table test.test2( a int, b varchar(255), c date, unique key(a,b,c))") tableInfo := pm.GetTableInfo("test", "test2") tableID := tableInfo.ID @@ -289,7 +302,7 @@ func (s *kvEntrySuite) TestUkWithNull(c *check.C) { Delete: false, IndexValue: []types.Datum{{}, types.NewBytesDatum([]byte("aa")), types.NewTimeDatum(time)}, }} - checkDMLKVEntries(pmCtx, c, tableInfo, plr, expect) + checkDMLKVEntries(ctx, c, tableInfo, m, plr, expect) pm.MustExec("insert into test.test2 values(null, null, '1996-11-20')") expect = []kvEntry{ @@ -306,7 +319,7 @@ func (s *kvEntrySuite) TestUkWithNull(c *check.C) { Delete: false, IndexValue: []types.Datum{{}, {}, types.NewTimeDatum(time)}, }} - checkDMLKVEntries(pmCtx, c, tableInfo, plr, expect) + checkDMLKVEntries(ctx, c, tableInfo, m, plr, expect) pm.MustExec("insert into test.test2 values(null, null, null)") expect = []kvEntry{ @@ -323,7 +336,7 @@ func (s *kvEntrySuite) TestUkWithNull(c *check.C) { Delete: false, IndexValue: []types.Datum{{}, {}, {}}, }} - checkDMLKVEntries(pmCtx, c, tableInfo, plr, expect) + checkDMLKVEntries(ctx, c, tableInfo, m, plr, expect) pm.MustExec("delete from test.test2 where c is null") expect = []kvEntry{ @@ -340,7 +353,7 @@ func (s *kvEntrySuite) TestUkWithNull(c *check.C) { Delete: true, IndexValue: []types.Datum{{}, {}, {}}, }} - checkDMLKVEntries(pmCtx, c, tableInfo, plr, expect) + checkDMLKVEntries(ctx, c, tableInfo, m, plr, expect) pm.MustExec("update test.test2 set a = 1, b = null where a is null and b is not null") expect = []kvEntry{ @@ -364,14 +377,17 @@ func (s *kvEntrySuite) TestUkWithNull(c *check.C) { Delete: true, IndexValue: []types.Datum{{}, types.NewBytesDatum([]byte("aa")), types.NewTimeDatum(time)}, }} - checkDMLKVEntries(pmCtx, c, tableInfo, plr, expect) + checkDMLKVEntries(ctx, c, tableInfo, m, plr, expect) } -func (s *kvEntrySuite) TestUkWithNoPk(c *check.C) { - pm, pmCtx, pmCancel := startPullerManager(c) - defer pmCancel() +func (s *kvEntrySuite) testUkWithNoPk(c *check.C, newRowFormat bool) { + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + pm, schema := setUpPullerAndSchema(ctx, c, newRowFormat, + "CREATE TABLE test.cdc_uk_with_no_pk (id INT, a1 INT, a3 INT, UNIQUE KEY dex1(a1, a3));", + ) + m := NewTxnMounter(schema) - pm.MustExec("CREATE TABLE test.cdc_uk_with_no_pk (id INT, a1 INT, a3 INT, UNIQUE KEY dex1(a1, a3));") tableInfo := pm.GetTableInfo("test", "cdc_uk_with_no_pk") tableID := tableInfo.ID @@ -392,7 +408,7 @@ func (s *kvEntrySuite) TestUkWithNoPk(c *check.C) { Delete: false, IndexValue: []types.Datum{types.NewIntDatum(6), {}}, }} - checkDMLKVEntries(pmCtx, c, tableInfo, plr, expect) + checkDMLKVEntries(ctx, c, tableInfo, m, plr, expect) pm.MustExec("UPDATE test.cdc_uk_with_no_pk SET id = 10 WHERE id = 5;") expect = []kvEntry{ @@ -402,7 +418,28 @@ func (s *kvEntrySuite) TestUkWithNoPk(c *check.C) { Delete: false, Row: map[int64]types.Datum{1: types.NewIntDatum(10), 2: types.NewIntDatum(6)}, }} - checkDMLKVEntries(pmCtx, c, tableInfo, plr, expect) + checkDMLKVEntries(ctx, c, tableInfo, m, plr, expect) +} + +func (s *kvEntrySuite) TestCreateTable(c *check.C) { + s.testCreateTable(c, true) + s.testCreateTable(c, false) +} +func (s *kvEntrySuite) TestPkIsNotHandleDML(c *check.C) { + s.testPkIsNotHandleDML(c, true) + s.testPkIsNotHandleDML(c, false) +} +func (s *kvEntrySuite) TestPkIsHandleDML(c *check.C) { + s.testPkIsHandleDML(c, true) + s.testPkIsHandleDML(c, false) +} +func (s *kvEntrySuite) TestUkWithNull(c *check.C) { + s.testUkWithNull(c, true) + s.testUkWithNull(c, false) +} +func (s *kvEntrySuite) TestUkWithNoPk(c *check.C) { + s.testUkWithNoPk(c, true) + s.testUkWithNoPk(c, false) } func assertIn(c *check.C, item kvEntry, expect []kvEntry) { @@ -414,24 +451,15 @@ func assertIn(c *check.C, item kvEntry, expect []kvEntry) { c.Fatalf("item {%#v} is not exist in expect {%#v}", item, expect) } -func startPullerManager(c *check.C) (*puller.MockPullerManager, context.Context, context.CancelFunc) { - // create and run mock puller manager - pm := puller.NewMockPullerManager(c) - pmCtx, pmCancel := context.WithCancel(context.Background()) - go pm.Run(pmCtx) - return pm, pmCtx, pmCancel -} - -func checkDMLKVEntries(ctx context.Context, c *check.C, tableInfo *schema.TableInfo, plr puller.Puller, expect []kvEntry) { +func checkDMLKVEntries(ctx context.Context, c *check.C, tableInfo *schema.TableInfo, m *Mounter, plr puller.Puller, expect []kvEntry) { ctx, cancel := context.WithCancel(ctx) err := plr.CollectRawTxns(ctx, func(ctx context.Context, rawTxn model.RawTxn) error { eventSum := 0 for _, raw := range rawTxn.Entries { - entry, err := unmarshal(raw) + entry, err := m.unmarshal(raw) c.Assert(err, check.IsNil) switch e := entry.(type) { case *rowKVEntry: - c.Assert(e.unflatten(tableInfo), check.IsNil) e.Ts = 0 assertIn(c, e, expect) eventSum++ diff --git a/cdc/entry/mounter.go b/cdc/entry/mounter.go index 23cb5c798c..1f77bdc7b1 100644 --- a/cdc/entry/mounter.go +++ b/cdc/entry/mounter.go @@ -29,7 +29,7 @@ func (m *Mounter) Mount(rawTxn model.RawTxn) (model.Txn, error) { } var replaceDMLs, deleteDMLs []*model.DML for _, raw := range rawTxn.Entries { - kvEntry, err := unmarshal(raw) + kvEntry, err := m.unmarshal(raw) if err != nil { return model.Txn{}, errors.Trace(err) } @@ -72,35 +72,18 @@ func (m *Mounter) Mount(rawTxn model.RawTxn) (model.Txn, error) { func (m *Mounter) mountRowKVEntry(row *rowKVEntry) (*model.DML, error) { tableInfo, tableName, exist := m.fetchTableInfo(row.TableID) if !exist { - if m.schemaStorage.IsTruncateTableID(row.TableID) { - log.Debug("skip the DML of truncated table", zap.Uint64("ts", row.Ts), zap.Int64("tableID", row.TableID)) - return nil, nil - } return nil, errors.NotFoundf("table in schema storage, id: %d", row.TableID) } - err := row.unflatten(tableInfo) - if err != nil { - return nil, errors.Trace(err) - } - - if row.Delete { - if tableInfo.PKIsHandle { - pkColName, pkValue, err := fetchHandleValue(tableInfo, row) - if err != nil { - return nil, errors.Trace(err) - } - return &model.DML{ - Database: tableName.Schema, - Table: tableName.Table, - Tp: model.DeleteDMLType, - Values: map[string]types.Datum{pkColName: *pkValue}, - }, nil - } + if row.Delete && !tableInfo.PKIsHandle { return nil, nil } - values := make(map[string]types.Datum, len(row.Row)+1) + datumsNum := 1 + if !row.Delete { + datumsNum = len(tableInfo.Columns) + } + values := make(map[string]types.Datum, datumsNum) for index, colValue := range row.Row { colInfo, exist := tableInfo.GetColumnInfo(index) if !exist { @@ -109,25 +92,24 @@ func (m *Mounter) mountRowKVEntry(row *rowKVEntry) (*model.DML, error) { colName := colInfo.Name.O values[colName] = colValue } - if tableInfo.PKIsHandle { - pkColName, pkValue, err := fetchHandleValue(tableInfo, row) - if err != nil { - return nil, errors.Trace(err) - } - values[pkColName] = *pkValue - } - for _, col := range tableInfo.Columns { - _, ok := values[col.Name.O] - if !ok { - values[col.Name.O] = getDefaultOrZeroValue(col) + var tp model.DMLType + if row.Delete { + tp = model.DeleteDMLType + } else { + tp = model.InsertDMLType + for _, col := range tableInfo.Columns { + _, ok := values[col.Name.O] + if !ok { + values[col.Name.O] = getDefaultOrZeroValue(col) + } } } return &model.DML{ Database: tableName.Schema, Table: tableName.Table, - Tp: model.InsertDMLType, + Tp: tp, Values: values, }, nil } @@ -201,7 +183,7 @@ func (m *Mounter) fetchTableInfo(tableID int64) (tableInfo *schema.TableInfo, ta return } -func fetchHandleValue(tableInfo *schema.TableInfo, row *rowKVEntry) (pkColName string, pkValue *types.Datum, err error) { +func fetchHandleValue(tableInfo *schema.TableInfo, recordID int64) (pkCoID int64, pkValue *types.Datum, err error) { handleColOffset := -1 for i, col := range tableInfo.Columns { if mysql.HasPriKeyFlag(col.Flag) { @@ -210,15 +192,15 @@ func fetchHandleValue(tableInfo *schema.TableInfo, row *rowKVEntry) (pkColName s } } if handleColOffset == -1 { - return "", nil, errors.New("can't find handle column, please check if the pk is handle") + return -1, nil, errors.New("can't find handle column, please check if the pk is handle") } handleCol := tableInfo.Columns[handleColOffset] - pkColName = handleCol.Name.O + pkCoID = handleCol.ID pkValue = &types.Datum{} if mysql.HasUnsignedFlag(handleCol.Flag) { - pkValue.SetUint64(uint64(row.RecordID)) + pkValue.SetUint64(uint64(recordID)) } else { - pkValue.SetInt64(row.RecordID) + pkValue.SetInt64(recordID) } return } diff --git a/cdc/entry/mounter_test.go b/cdc/entry/mounter_test.go index f777ee7d06..b67f7737a0 100644 --- a/cdc/entry/mounter_test.go +++ b/cdc/entry/mounter_test.go @@ -19,8 +19,8 @@ type mountTxnsSuite struct{} var _ = check.Suite(&mountTxnsSuite{}) -func setUpPullerAndSchema(ctx context.Context, c *check.C, sqls ...string) (*puller.MockPullerManager, *schema.Storage) { - pm := puller.NewMockPullerManager(c) +func setUpPullerAndSchema(ctx context.Context, c *check.C, newRowFormat bool, sqls ...string) (*puller.MockPullerManager, *schema.Storage) { + pm := puller.NewMockPullerManager(c, newRowFormat) go pm.Run(ctx) for _, sql := range sqls { pm.MustExec(sql) @@ -51,10 +51,10 @@ func getFirstRealTxn(ctx context.Context, c *check.C, plr puller.Puller) (result return } -func (cs *mountTxnsSuite) TestInsertPkNotHandle(c *check.C) { +func (cs *mountTxnsSuite) testInsertPkNotHandle(c *check.C, newRowFormat bool) { ctx, cancel := context.WithCancel(context.Background()) defer cancel() - pm, schema := setUpPullerAndSchema(ctx, c, + pm, schema := setUpPullerAndSchema(ctx, c, newRowFormat, "create database testDB", "create table testDB.test1(id varchar(255) primary key, a int, index ci (a))", ) @@ -128,10 +128,10 @@ func (cs *mountTxnsSuite) TestInsertPkNotHandle(c *check.C) { }) } -func (cs *mountTxnsSuite) TestIncompleteRow(c *check.C) { +func (cs *mountTxnsSuite) testIncompleteRow(c *check.C, newRowFormat bool) { ctx, cancel := context.WithCancel(context.Background()) defer cancel() - pm, schema := setUpPullerAndSchema(ctx, c, + pm, schema := setUpPullerAndSchema(ctx, c, newRowFormat, "create database testDB", "create table testDB.test1 (id int primary key, val int);", ) @@ -189,10 +189,10 @@ func (cs *mountTxnsSuite) TestIncompleteRow(c *check.C) { } -func (cs *mountTxnsSuite) TestInsertPkIsHandle(c *check.C) { +func (cs *mountTxnsSuite) testInsertPkIsHandle(c *check.C, newRowFormat bool) { ctx, cancel := context.WithCancel(context.Background()) defer cancel() - pm, schema := setUpPullerAndSchema(ctx, c, + pm, schema := setUpPullerAndSchema(ctx, c, newRowFormat, "create database testDB", "create table testDB.test1(id int primary key, a int unique key not null)", ) @@ -274,10 +274,10 @@ func (cs *mountTxnsSuite) TestInsertPkIsHandle(c *check.C) { }) } -func (cs *mountTxnsSuite) TestUk(c *check.C) { +func (cs *mountTxnsSuite) testUk(c *check.C, newRowFormat bool) { ctx, cancel := context.WithCancel(context.Background()) defer cancel() - pm, schema := setUpPullerAndSchema(ctx, c, + pm, schema := setUpPullerAndSchema(ctx, c, newRowFormat, "create database testDB", `create table testDB.test1( a int unique key not null, @@ -385,10 +385,10 @@ func (cs *mountTxnsSuite) TestUk(c *check.C) { }) } -func (cs *mountTxnsSuite) TestLargeInteger(c *check.C) { +func (cs *mountTxnsSuite) testLargeInteger(c *check.C, newRowFormat bool) { ctx, cancel := context.WithCancel(context.Background()) defer cancel() - pm, schema := setUpPullerAndSchema(ctx, c, + pm, schema := setUpPullerAndSchema(ctx, c, newRowFormat, "create database testDB", "CREATE TABLE testDB.large_int(id BIGINT UNSIGNED PRIMARY KEY, a int)", ) @@ -418,7 +418,7 @@ func (cs *mountTxnsSuite) TestLargeInteger(c *check.C) { ctx, cancel = context.WithCancel(context.Background()) defer cancel() - pm, schema = setUpPullerAndSchema(ctx, c, + pm, schema = setUpPullerAndSchema(ctx, c, newRowFormat, "create database testDB", "CREATE TABLE testDB.large_int(id BIGINT PRIMARY KEY, a int)", ) @@ -448,6 +448,27 @@ func (cs *mountTxnsSuite) TestLargeInteger(c *check.C) { } +func (cs *mountTxnsSuite) TestInsertPkNotHandle(c *check.C) { + cs.testInsertPkNotHandle(c, true) + cs.testInsertPkNotHandle(c, false) +} +func (cs *mountTxnsSuite) TestIncompleteRow(c *check.C) { + cs.testIncompleteRow(c, true) + cs.testIncompleteRow(c, false) +} +func (cs *mountTxnsSuite) TestInsertPkIsHandle(c *check.C) { + cs.testInsertPkIsHandle(c, true) + cs.testInsertPkIsHandle(c, false) +} +func (cs *mountTxnsSuite) TestUk(c *check.C) { + cs.testUk(c, true) + cs.testUk(c, false) +} +func (cs *mountTxnsSuite) TestLargeInteger(c *check.C) { + cs.testLargeInteger(c, true) + cs.testLargeInteger(c, false) +} + func (cs *mountTxnsSuite) assertTableTxnEquals(c *check.C, obtained, expected model.Txn) { obtainedDMLs := obtained.DMLs diff --git a/cdc/filter.go b/cdc/filter.go index fc726c6fc8..e68287bf91 100644 --- a/cdc/filter.go +++ b/cdc/filter.go @@ -12,12 +12,15 @@ type txnFilter struct { ignoreTxnCommitTs []uint64 } -func newTxnFilter(config *model.ReplicaConfig) *txnFilter { - filter := filter.New(config.FilterCaseSensitive, config.FilterRules) +func newTxnFilter(config *model.ReplicaConfig) (*txnFilter, error) { + filter, err := filter.New(config.FilterCaseSensitive, config.FilterRules) + if err != nil { + return nil, err + } return &txnFilter{ filter: filter, ignoreTxnCommitTs: config.IgnoreTxnCommitTs, - } + }, nil } // ShouldIgnoreTxn returns true is the given txn should be ignored diff --git a/cdc/filter_test.go b/cdc/filter_test.go index 5023ed4a2a..1b90c5c552 100644 --- a/cdc/filter_test.go +++ b/cdc/filter_test.go @@ -24,7 +24,8 @@ type filterSuite struct{} var _ = check.Suite(&filterSuite{}) func (s *filterSuite) TestShouldUseDefaultRules(c *check.C) { - filter := newTxnFilter(&model.ReplicaConfig{}) + filter, err := newTxnFilter(&model.ReplicaConfig{}) + c.Assert(err, check.IsNil) c.Assert(filter.ShouldIgnoreTable("information_schema", ""), check.IsTrue) c.Assert(filter.ShouldIgnoreTable("information_schema", "statistics"), check.IsTrue) c.Assert(filter.ShouldIgnoreTable("performance_schema", ""), check.IsTrue) @@ -38,7 +39,7 @@ func (s *filterSuite) TestShouldUseDefaultRules(c *check.C) { } func (s *filterSuite) TestShouldUseCustomRules(c *check.C) { - filter := newTxnFilter(&model.ReplicaConfig{ + filter, err := newTxnFilter(&model.ReplicaConfig{ FilterRules: &filter.Rules{ DoDBs: []string{"sns", "ecom"}, IgnoreTables: []*filter.Table{ @@ -47,6 +48,7 @@ func (s *filterSuite) TestShouldUseCustomRules(c *check.C) { }, }, }) + c.Assert(err, check.IsNil) assertIgnore := func(db, tbl string, boolCheck check.Checker) { c.Assert(filter.ShouldIgnoreTable(db, tbl), boolCheck) } @@ -76,9 +78,10 @@ func (s *filterSuite) TestShouldUseCustomRules(c *check.C) { } func (s *filterSuite) TestShouldIgnoreTxn(c *check.C) { - filter := newTxnFilter(&model.ReplicaConfig{ + filter, err := newTxnFilter(&model.ReplicaConfig{ IgnoreTxnCommitTs: []uint64{1, 3}, }) + c.Assert(err, check.IsNil) testCases := []struct { txn *model.Txn ignore bool diff --git a/cdc/http_handler.go b/cdc/http_handler.go index e8e5c401fc..83fe33bfa0 100644 --- a/cdc/http_handler.go +++ b/cdc/http_handler.go @@ -17,9 +17,9 @@ import ( "net/http" "strconv" - "github.com/coreos/etcd/clientv3/concurrency" "github.com/pingcap/errors" "github.com/pingcap/ticdc/cdc/model" + "go.etcd.io/etcd/clientv3/concurrency" ) const ( diff --git a/cdc/http_status.go b/cdc/http_status.go index dd21d281e0..8f8c04ed79 100644 --- a/cdc/http_status.go +++ b/cdc/http_status.go @@ -22,11 +22,11 @@ import ( "net/http/pprof" "os" - "github.com/coreos/etcd/clientv3" "github.com/pingcap/log" "github.com/pingcap/ticdc/cdc/kv" "github.com/prometheus/client_golang/prometheus" "github.com/prometheus/client_golang/prometheus/promhttp" + "go.etcd.io/etcd/clientv3" "go.uber.org/zap" ) diff --git a/cdc/kv/client.go b/cdc/kv/client.go index 3510a91b1b..e3830436a2 100644 --- a/cdc/kv/client.go +++ b/cdc/kv/client.go @@ -32,6 +32,7 @@ import ( "go.uber.org/zap" "golang.org/x/sync/errgroup" "google.golang.org/grpc" + gbackoff "google.golang.org/grpc/backoff" "google.golang.org/grpc/codes" "google.golang.org/grpc/keepalive" "google.golang.org/grpc/status" @@ -117,7 +118,15 @@ func (c *CDCClient) getConn( ctx, addr, grpc.WithInsecure(), - grpc.WithBackoffMaxDelay(time.Second), + grpc.WithConnectParams(grpc.ConnectParams{ + Backoff: gbackoff.Config{ + BaseDelay: time.Second, + Multiplier: 1.1, + Jitter: 0.1, + MaxDelay: 3 * time.Second, + }, + MinConnectTimeout: 3 * time.Second, + }), grpc.WithKeepaliveParams(keepalive.ClientParameters{ Time: 10 * time.Second, Timeout: 3 * time.Second, diff --git a/cdc/kv/etcd.go b/cdc/kv/etcd.go index 78c3046af2..98377ba63a 100644 --- a/cdc/kv/etcd.go +++ b/cdc/kv/etcd.go @@ -17,11 +17,11 @@ import ( "context" "fmt" - "github.com/coreos/etcd/clientv3" - "github.com/coreos/etcd/mvcc/mvccpb" "github.com/pingcap/errors" "github.com/pingcap/ticdc/cdc/model" "github.com/pingcap/ticdc/pkg/util" + "go.etcd.io/etcd/clientv3" + "go.etcd.io/etcd/mvcc/mvccpb" ) const ( diff --git a/cdc/kv/etcd_test.go b/cdc/kv/etcd_test.go index ea2609735c..e5e03b6a97 100644 --- a/cdc/kv/etcd_test.go +++ b/cdc/kv/etcd_test.go @@ -18,13 +18,13 @@ import ( "net/url" "time" - "github.com/coreos/etcd/clientv3" - "github.com/coreos/etcd/embed" "github.com/pingcap/check" "github.com/pingcap/errors" "github.com/pingcap/ticdc/cdc/model" "github.com/pingcap/ticdc/pkg/etcd" "github.com/pingcap/ticdc/pkg/util" + "go.etcd.io/etcd/clientv3" + "go.etcd.io/etcd/embed" "golang.org/x/sync/errgroup" ) diff --git a/cdc/kv/testing.go b/cdc/kv/testing.go index b7eee85b65..2ddb72e4e4 100644 --- a/cdc/kv/testing.go +++ b/cdc/kv/testing.go @@ -146,7 +146,7 @@ func TestSplit(t require.TestingT, pdCli pd.Client, storage kv.Storage) { regions := preRegions // In second loop try split every region. if i == 1 { - splitStore, ok := storage.(kv.SplitableStore) + splitStore, ok := storage.(kv.SplittableStore) require.True(t, ok) for _, r := range preRegions { splitKey := r.GetStartKey() diff --git a/cdc/owner.go b/cdc/owner.go index fae5fa9eb2..7c7d1a5f42 100644 --- a/cdc/owner.go +++ b/cdc/owner.go @@ -21,9 +21,6 @@ import ( "sync" "time" - "github.com/coreos/etcd/clientv3" - "github.com/coreos/etcd/clientv3/concurrency" - "github.com/coreos/etcd/mvcc" "github.com/pingcap/errors" "github.com/pingcap/log" pmodel "github.com/pingcap/parser/model" @@ -34,6 +31,9 @@ import ( "github.com/pingcap/ticdc/cdc/roles/storage" "github.com/pingcap/ticdc/cdc/schema" "github.com/pingcap/ticdc/pkg/util" + "go.etcd.io/etcd/clientv3" + "go.etcd.io/etcd/clientv3/concurrency" + "go.etcd.io/etcd/mvcc" "go.uber.org/zap" "golang.org/x/time/rate" ) @@ -492,7 +492,10 @@ func (o *ownerImpl) newChangeFeed(id model.ChangeFeedID, processorsInfos model.P } } - filter := newTxnFilter(info.GetConfig()) + filter, err := newTxnFilter(info.GetConfig()) + if err != nil { + return nil, errors.Trace(err) + } tables := make(map[uint64]schema.TableName) orphanTables := make(map[uint64]model.ProcessTableInfo) diff --git a/cdc/owner_test.go b/cdc/owner_test.go index edb375c4df..188052b4a2 100644 --- a/cdc/owner_test.go +++ b/cdc/owner_test.go @@ -7,9 +7,6 @@ import ( "sync" "time" - "github.com/coreos/etcd/clientv3" - "github.com/coreos/etcd/clientv3/concurrency" - "github.com/coreos/etcd/embed" "github.com/google/uuid" "github.com/pingcap/check" "github.com/pingcap/errors" @@ -22,6 +19,9 @@ import ( "github.com/pingcap/ticdc/cdc/schema" "github.com/pingcap/ticdc/pkg/etcd" "github.com/pingcap/ticdc/pkg/util" + "go.etcd.io/etcd/clientv3" + "go.etcd.io/etcd/clientv3/concurrency" + "go.etcd.io/etcd/embed" "golang.org/x/sync/errgroup" ) @@ -316,7 +316,8 @@ func (s *ownerSuite) TestDDL(c *check.C) { tables := map[uint64]schema.TableName{1: {Schema: "any"}} - filter := newTxnFilter(&model.ReplicaConfig{}) + filter, err := newTxnFilter(&model.ReplicaConfig{}) + c.Assert(err, check.IsNil) changeFeeds := map[model.ChangeFeedID]*changeFeed{ "test_change_feed": { tables: tables, @@ -335,7 +336,7 @@ func (s *ownerSuite) TestDDL(c *check.C) { } manager := roles.NewMockManager(uuid.New().String(), cancel) - err := manager.CampaignOwner(ctx) + err = manager.CampaignOwner(ctx) c.Assert(err, check.IsNil) owner := &ownerImpl{ cancelWatchCapture: cancel, diff --git a/cdc/processor.go b/cdc/processor.go index b5cf59b4df..4913f4910f 100644 --- a/cdc/processor.go +++ b/cdc/processor.go @@ -25,7 +25,6 @@ import ( "time" "github.com/cenkalti/backoff" - "github.com/coreos/etcd/clientv3" "github.com/pingcap/errors" "github.com/pingcap/log" timodel "github.com/pingcap/parser/model" @@ -43,9 +42,11 @@ import ( "github.com/pingcap/tidb/store/tikv" "github.com/pingcap/tidb/store/tikv/oracle" "github.com/pingcap/tidb/util/codec" + "go.etcd.io/etcd/clientv3" "go.uber.org/zap" "golang.org/x/sync/errgroup" "google.golang.org/grpc" + pbackoff "google.golang.org/grpc/backoff" ) const ( @@ -194,7 +195,15 @@ func NewProcessor(pdEndpoints []string, changefeed model.ChangeFeedInfo, changef Endpoints: pdEndpoints, DialTimeout: 5 * time.Second, DialOptions: []grpc.DialOption{ - grpc.WithBackoffMaxDelay(time.Second * 3), + grpc.WithConnectParams(grpc.ConnectParams{ + Backoff: pbackoff.Config{ + BaseDelay: time.Second, + Multiplier: 1.1, + Jitter: 0.1, + MaxDelay: 3 * time.Second, + }, + MinConnectTimeout: 3 * time.Second, + }), }, }) if err != nil { @@ -222,7 +231,10 @@ func NewProcessor(pdEndpoints []string, changefeed model.ChangeFeedInfo, changef return nil, err } - filter := newTxnFilter(changefeed.GetConfig()) + filter, err := newTxnFilter(changefeed.GetConfig()) + if err != nil { + return nil, errors.Trace(err) + } p := &processor{ captureID: captureID, diff --git a/cdc/processor_test.go b/cdc/processor_test.go index 9c97e5f253..b491f0a72c 100644 --- a/cdc/processor_test.go +++ b/cdc/processor_test.go @@ -19,7 +19,6 @@ import ( "sync" "time" - "github.com/coreos/etcd/clientv3" "github.com/pingcap/check" pd "github.com/pingcap/pd/client" "github.com/pingcap/ticdc/cdc/model" @@ -27,6 +26,7 @@ import ( "github.com/pingcap/ticdc/cdc/schema" "github.com/pingcap/ticdc/cdc/sink" "github.com/pingcap/ticdc/pkg/etcd" + "go.etcd.io/etcd/clientv3" ) type processorSuite struct{} @@ -128,7 +128,7 @@ func runCase(c *check.C, cases *processorTestCase) { return nil, nil } origFNewPD := fNewPDCli - fNewPDCli = func(pdAddrs []string, security pd.SecurityOption) (pd.Client, error) { + fNewPDCli = func(pdAddrs []string, security pd.SecurityOption, opts ...pd.ClientOption) (pd.Client, error) { return nil, nil } origFNewTsRw := fNewTsRWriter diff --git a/cdc/puller/mock_puller.go b/cdc/puller/mock_puller.go index b07ef8a414..e37f4ed253 100644 --- a/cdc/puller/mock_puller.go +++ b/cdc/puller/mock_puller.go @@ -162,18 +162,18 @@ func (p *mockPuller) Output() Buffer { } // NewMockPullerManager creates and sets up a mock puller manager -func NewMockPullerManager(c *check.C) *MockPullerManager { +func NewMockPullerManager(c *check.C, newRowFormat bool) *MockPullerManager { m := &MockPullerManager{ txnMap: make(map[uint64]*kvrpcpb.PrewriteRequest), rawTxnCh: make(chan model.RawTxn, 16), closeCh: make(chan struct{}), c: c, } - m.setUp() + m.setUp(newRowFormat) return m } -func (m *MockPullerManager) setUp() { +func (m *MockPullerManager) setUp(newRowFormat bool) { // avoid to print too many logs logLevel := log.GetLevel() log.SetLevel(zap.FatalLevel) @@ -205,6 +205,7 @@ func (m *MockPullerManager) setUp() { m.tidbKit = testkit.NewTestKit(m.c, m.store) m.MustExec("use test;") + m.tidbKit.Se.GetSessionVars().RowEncoder.Enable = newRowFormat mvccListener.registerPostPrewrite(m.postPrewrite) mvccListener.registerPostCommit(m.postCommit) diff --git a/cdc/puller/mock_puller_test.go b/cdc/puller/mock_puller_test.go index 39604b86e4..46fe9906da 100644 --- a/cdc/puller/mock_puller_test.go +++ b/cdc/puller/mock_puller_test.go @@ -19,7 +19,7 @@ type mockPullerSuite struct{} var _ = check.Suite(&mockPullerSuite{}) func (s *mockPullerSuite) TestTxnSort(c *check.C) { - pm := NewMockPullerManager(c) + pm := NewMockPullerManager(c, true) plr := pm.CreatePuller(0, []util.Span{util.Span{}.Hack()}) ctx, cancel := context.WithCancel(context.Background()) defer cancel() @@ -43,7 +43,7 @@ func (s *mockPullerSuite) TestTxnSort(c *check.C) { } func (s *mockPullerSuite) TestDDLPuller(c *check.C) { - pm := NewMockPullerManager(c) + pm := NewMockPullerManager(c, true) plr := pm.CreatePuller(0, []util.Span{util.GetDDLSpan()}) ctx, cancel := context.WithCancel(context.Background()) defer cancel() @@ -81,7 +81,7 @@ func (s *mockPullerSuite) TestDDLPuller(c *check.C) { } func (s *mockPullerSuite) TestStartTs(c *check.C) { - pm := NewMockPullerManager(c) + pm := NewMockPullerManager(c, true) plrA := pm.CreatePuller(0, []util.Span{util.Span{}.Hack()}) ctx, cancel := context.WithCancel(context.Background()) ts := uint64(0) diff --git a/cdc/roles/manager.go b/cdc/roles/manager.go index 3f16983342..226cff09e0 100644 --- a/cdc/roles/manager.go +++ b/cdc/roles/manager.go @@ -22,13 +22,13 @@ import ( "time" "unsafe" - "github.com/coreos/etcd/clientv3" - "github.com/coreos/etcd/clientv3/concurrency" - "github.com/coreos/etcd/etcdserver/api/v3rpc/rpctypes" - "github.com/coreos/etcd/mvcc/mvccpb" "github.com/pingcap/errors" "github.com/pingcap/log" "github.com/pingcap/parser/terror" + "go.etcd.io/etcd/clientv3" + "go.etcd.io/etcd/clientv3/concurrency" + "go.etcd.io/etcd/etcdserver/api/v3rpc/rpctypes" + "go.etcd.io/etcd/mvcc/mvccpb" "go.uber.org/zap" "google.golang.org/grpc/codes" "google.golang.org/grpc/status" diff --git a/cdc/roles/manager_test.go b/cdc/roles/manager_test.go index 5648db63e0..a388d4bd68 100644 --- a/cdc/roles/manager_test.go +++ b/cdc/roles/manager_test.go @@ -6,14 +6,14 @@ import ( "testing" "time" - "github.com/coreos/etcd/clientv3" - "github.com/coreos/etcd/clientv3/concurrency" - "github.com/coreos/etcd/embed" "github.com/pingcap/check" "github.com/pingcap/errors" "github.com/pingcap/ticdc/cdc/kv" "github.com/pingcap/ticdc/pkg/etcd" "github.com/pingcap/ticdc/pkg/util" + "go.etcd.io/etcd/clientv3" + "go.etcd.io/etcd/clientv3/concurrency" + "go.etcd.io/etcd/embed" "golang.org/x/sync/errgroup" ) diff --git a/cdc/roles/storage/etcd.go b/cdc/roles/storage/etcd.go index e7967ac1b5..726e115041 100644 --- a/cdc/roles/storage/etcd.go +++ b/cdc/roles/storage/etcd.go @@ -18,8 +18,6 @@ import ( "time" "github.com/cenkalti/backoff" - "github.com/coreos/etcd/clientv3" - "github.com/coreos/etcd/embed" "github.com/pingcap/errors" "github.com/pingcap/log" "github.com/pingcap/ticdc/cdc/kv" @@ -27,6 +25,8 @@ import ( "github.com/pingcap/ticdc/pkg/retry" "github.com/pingcap/ticdc/pkg/util" "github.com/pingcap/tidb/store/tikv/oracle" + "go.etcd.io/etcd/clientv3" + "go.etcd.io/etcd/embed" "go.uber.org/zap" ) diff --git a/cdc/roles/storage/etcd_test.go b/cdc/roles/storage/etcd_test.go index 6e39dbdf96..1883bacfd1 100644 --- a/cdc/roles/storage/etcd_test.go +++ b/cdc/roles/storage/etcd_test.go @@ -20,14 +20,14 @@ import ( "testing" "time" - "github.com/coreos/etcd/clientv3" - "github.com/coreos/etcd/embed" "github.com/pingcap/check" "github.com/pingcap/errors" "github.com/pingcap/ticdc/cdc/kv" "github.com/pingcap/ticdc/cdc/model" "github.com/pingcap/ticdc/pkg/etcd" "github.com/pingcap/ticdc/pkg/util" + "go.etcd.io/etcd/clientv3" + "go.etcd.io/etcd/embed" "golang.org/x/sync/errgroup" ) diff --git a/cdc/scheduler.go b/cdc/scheduler.go index 4f1e355d46..e7a82f5889 100644 --- a/cdc/scheduler.go +++ b/cdc/scheduler.go @@ -19,15 +19,15 @@ import ( "sync/atomic" "time" - "github.com/coreos/etcd/clientv3" - "github.com/coreos/etcd/mvcc" - "github.com/coreos/etcd/mvcc/mvccpb" "github.com/pingcap/errors" "github.com/pingcap/failpoint" "github.com/pingcap/log" "github.com/pingcap/ticdc/cdc/kv" "github.com/pingcap/ticdc/cdc/model" "github.com/pingcap/ticdc/pkg/util" + "go.etcd.io/etcd/clientv3" + "go.etcd.io/etcd/mvcc" + "go.etcd.io/etcd/mvcc/mvccpb" "go.uber.org/zap" "golang.org/x/time/rate" ) diff --git a/cdc/scheduler_test.go b/cdc/scheduler_test.go index 2e0f0b1dd4..ababd80e3b 100644 --- a/cdc/scheduler_test.go +++ b/cdc/scheduler_test.go @@ -20,9 +20,6 @@ import ( "sync/atomic" "time" - "github.com/coreos/etcd/clientv3" - "github.com/coreos/etcd/embed" - "github.com/coreos/etcd/mvcc" "github.com/pingcap/check" "github.com/pingcap/errors" "github.com/pingcap/failpoint" @@ -30,6 +27,9 @@ import ( "github.com/pingcap/ticdc/cdc/model" "github.com/pingcap/ticdc/pkg/etcd" "github.com/pingcap/ticdc/pkg/util" + "go.etcd.io/etcd/clientv3" + "go.etcd.io/etcd/embed" + "go.etcd.io/etcd/mvcc" "golang.org/x/sync/errgroup" ) diff --git a/cdc/schema/storage.go b/cdc/schema/storage.go index 84662d3917..74fbd7adfd 100644 --- a/cdc/schema/storage.go +++ b/cdc/schema/storage.go @@ -22,6 +22,7 @@ import ( "github.com/pingcap/log" "github.com/pingcap/parser/model" "github.com/pingcap/parser/mysql" + "github.com/pingcap/tidb/util/rowcodec" "go.uber.org/zap" ) @@ -61,6 +62,8 @@ type TableInfo struct { *model.TableInfo ColumnsOffset map[int64]int IndicesOffset map[int64]int + handleColID int64 + rowColInfos []rowcodec.ColInfo } // WrapTableInfo creates a TableInfo from a model.TableInfo @@ -98,6 +101,33 @@ func (ti *TableInfo) GetIndexInfo(indexID int64) (info *model.IndexInfo, exist b return ti.Indices[indexOffset], true } +// GetRowColInfos returns all column infos for rowcodec +func (ti *TableInfo) GetRowColInfos() (int64, []rowcodec.ColInfo) { + if len(ti.rowColInfos) != 0 { + return ti.handleColID, ti.rowColInfos + } + handleColID := int64(-1) + reqCols := make([]rowcodec.ColInfo, len(ti.Columns)) + for i, col := range ti.Columns { + isPK := (ti.PKIsHandle && mysql.HasPriKeyFlag(col.Flag)) || col.ID == model.ExtraHandleID + if isPK { + handleColID = col.ID + } + reqCols[i] = rowcodec.ColInfo{ + ID: col.ID, + Tp: int32(col.Tp), + Flag: int32(col.Flag), + Flen: col.Flen, + Decimal: col.Decimal, + Elems: col.Elems, + IsPKHandle: isPK, + } + } + ti.rowColInfos = reqCols + ti.handleColID = handleColID + return ti.handleColID, ti.rowColInfos +} + // WritableColumns returns all public and non-generated columns func (ti *TableInfo) WritableColumns() []*model.ColumnInfo { cols := make([]*model.ColumnInfo, 0, len(ti.Columns)) diff --git a/cmd/client.go b/cmd/client.go index 6c14f60505..9c949271d8 100644 --- a/cmd/client.go +++ b/cmd/client.go @@ -7,7 +7,6 @@ import ( "time" "github.com/BurntSushi/toml" - "github.com/coreos/etcd/clientv3" _ "github.com/go-sql-driver/mysql" // mysql driver "github.com/google/uuid" "github.com/pingcap/errors" @@ -16,7 +15,9 @@ import ( "github.com/pingcap/ticdc/cdc/model" "github.com/pingcap/tidb/store/tikv/oracle" "github.com/spf13/cobra" + "go.etcd.io/etcd/clientv3" "google.golang.org/grpc" + "google.golang.org/grpc/backoff" ) func init() { @@ -46,7 +47,15 @@ var cliCmd = &cobra.Command{ Endpoints: []string{pdAddress}, DialTimeout: 5 * time.Second, DialOptions: []grpc.DialOption{ - grpc.WithBackoffMaxDelay(time.Second * 3), + grpc.WithConnectParams(grpc.ConnectParams{ + Backoff: backoff.Config{ + BaseDelay: time.Second, + Multiplier: 1.1, + Jitter: 0.1, + MaxDelay: 3 * time.Second, + }, + MinConnectTimeout: 3 * time.Second, + }), }, }) if err != nil { diff --git a/cmd/ctrl.go b/cmd/ctrl.go index aaf7cd2aca..46dfd3da36 100644 --- a/cmd/ctrl.go +++ b/cmd/ctrl.go @@ -19,14 +19,15 @@ import ( "fmt" "time" - "github.com/coreos/etcd/clientv3" - "github.com/coreos/etcd/clientv3/concurrency" pd "github.com/pingcap/pd/client" "github.com/pingcap/ticdc/cdc/kv" "github.com/pingcap/ticdc/cdc/roles" "github.com/pingcap/tidb/store/tikv/oracle" "github.com/spf13/cobra" + "go.etcd.io/etcd/clientv3" + "go.etcd.io/etcd/clientv3/concurrency" "google.golang.org/grpc" + "google.golang.org/grpc/backoff" ) // command type @@ -92,7 +93,15 @@ var ctrlCmd = &cobra.Command{ Endpoints: []string{ctrlPdAddr}, DialTimeout: 5 * time.Second, DialOptions: []grpc.DialOption{ - grpc.WithBackoffMaxDelay(time.Second * 3), + grpc.WithConnectParams(grpc.ConnectParams{ + Backoff: backoff.Config{ + BaseDelay: time.Second, + Multiplier: 1.1, + Jitter: 0.1, + MaxDelay: 3 * time.Second, + }, + MinConnectTimeout: 3 * time.Second, + }), }, }) if err != nil { diff --git a/go.mod b/go.mod index d0344cdd86..d745e2ed57 100644 --- a/go.mod +++ b/go.mod @@ -7,27 +7,39 @@ require ( github.com/DATA-DOG/go-sqlmock v1.3.3 github.com/biogo/store v0.0.0-20190426020002-884f370e325d github.com/cenkalti/backoff v2.2.1+incompatible - github.com/coreos/etcd v3.3.13+incompatible github.com/go-sql-driver/mysql v1.4.1 + github.com/gogo/protobuf v1.3.1 // indirect + github.com/golang/groupcache v0.0.0-20191227052852-215e87163ea7 // indirect github.com/google/uuid v1.1.1 + github.com/gorilla/websocket v1.4.1 // indirect + github.com/grpc-ecosystem/go-grpc-middleware v1.1.0 // indirect + github.com/grpc-ecosystem/grpc-gateway v1.12.1 // indirect + github.com/json-iterator/go v1.1.9 // indirect github.com/phayes/freeport v0.0.0-20180830031419-95f893ade6f2 - github.com/pingcap/check v0.0.0-20190102082844-67f458068fc8 - github.com/pingcap/errors v0.11.4 + github.com/pingcap/check v0.0.0-20191216031241-8a5a85928f12 + github.com/pingcap/errors v0.11.5-0.20190809092503-95897b64e011 github.com/pingcap/failpoint v0.0.0-20200115060041-f2180fbf0df8 github.com/pingcap/kvproto v0.0.0-20200108025604-a4dc183d2af5 - github.com/pingcap/log v0.0.0-20191012051959-b742a5d432e9 - github.com/pingcap/parser v0.0.0-20191012071233-32876040fefb - github.com/pingcap/pd v1.1.0-beta.0.20190923032047-5c648dc365e0 - github.com/pingcap/tidb v1.1.0-beta.0.20191017064824-e1ba309148ad - github.com/pingcap/tidb-tools v2.1.3-0.20190321065848-1e8b48f5c168+incompatible - github.com/prometheus/client_golang v0.9.0 - github.com/remyoudompheng/bigfft v0.0.0-20190728182440-6a916e37a237 // indirect - github.com/sirupsen/logrus v1.4.2 // indirect - github.com/spf13/cobra v0.0.3 + github.com/pingcap/log v0.0.0-20200117041106-d28c14d3b1cd + github.com/pingcap/parser v0.0.0-20200120100653-1d87b3907217 + github.com/pingcap/pd v1.1.0-beta.0.20191219054547-4d65bbefbc6d + github.com/pingcap/tidb v1.1.0-beta.0.20200204134155-ebc6a2d39dd7 + github.com/pingcap/tidb-tools v3.1.0-beta.1.0.20200108061154-356b0e2e2282+incompatible + github.com/prometheus/client_golang v1.3.0 + github.com/prometheus/common v0.8.0 // indirect + github.com/spf13/cobra v0.0.5 + github.com/spf13/pflag v1.0.5 // indirect github.com/stretchr/testify v1.4.0 - go.uber.org/zap v1.10.0 - golang.org/x/sync v0.0.0-20190423024810-112230192c58 - golang.org/x/time v0.0.0-20190308202827-9d24e82272b4 + github.com/tmc/grpc-websocket-proxy v0.0.0-20190109142713-0ad062ec5ee5 // indirect + go.etcd.io/etcd v0.5.0-alpha.5.0.20191211224106-0dc78a144b31 + go.uber.org/zap v1.13.0 + golang.org/x/crypto v0.0.0-20200109152110-61a87790db17 // indirect + golang.org/x/net v0.0.0-20191209160850-c0dbc17a3553 // indirect + golang.org/x/sync v0.0.0-20190911185100-cd5d95a43a6e + golang.org/x/sys v0.0.0-20200113162924-86b910548bc1 // indirect + golang.org/x/time v0.0.0-20191024005414-555d28b269f0 google.golang.org/appengine v1.6.2 // indirect - google.golang.org/grpc v1.23.0 + google.golang.org/genproto v0.0.0-20200113173426-e1de0a7b01eb // indirect + google.golang.org/grpc v1.26.0 + gopkg.in/yaml.v2 v2.2.7 // indirect ) diff --git a/go.sum b/go.sum index f05b0554d7..551a087e2d 100644 --- a/go.sum +++ b/go.sum @@ -3,11 +3,20 @@ github.com/BurntSushi/toml v0.3.1 h1:WXkYYl6Yr3qBf1K79EBnL4mak0OimBfB0XUf9Vl28OQ github.com/BurntSushi/toml v0.3.1/go.mod h1:xHWCNGjB5oqiDr8zfno3MHue2Ht5sIBksp03qcyfWMU= github.com/DATA-DOG/go-sqlmock v1.3.3 h1:CWUqKXe0s8A2z6qCgkP4Kru7wC11YoAnoupUKFDnH08= github.com/DATA-DOG/go-sqlmock v1.3.3/go.mod h1:f/Ixk793poVmq4qj/V1dPUg2JEAKC73Q5eFN3EC/SaM= -github.com/StackExchange/wmi v0.0.0-20180725035823-b12b22c5341f h1:5ZfJxyXo8KyX8DgGXC5B7ILL8y51fci/qYz2B4j8iLY= -github.com/StackExchange/wmi v0.0.0-20180725035823-b12b22c5341f/go.mod h1:3eOhrUMpNV+6aFIbp5/iudMxNCF27Vw2OZgy4xEx0Fg= +github.com/OneOfOne/xxhash v1.2.2 h1:KMrpdQIwFcEqXDklaen+P1axHaj9BSKzvpUUfnHldSE= +github.com/OneOfOne/xxhash v1.2.2/go.mod h1:HSdplMjZKSmBqAxg5vPj2TmRDmfkzw+cTzAElWljhcU= +github.com/StackExchange/wmi v0.0.0-20190523213315-cbe66965904d h1:G0m3OIz70MZUWq3EgK3CesDbo8upS2Vm9/P3FtgI+Jk= +github.com/StackExchange/wmi v0.0.0-20190523213315-cbe66965904d/go.mod h1:3eOhrUMpNV+6aFIbp5/iudMxNCF27Vw2OZgy4xEx0Fg= +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/go.mod h1:ybxpYRFXyAe+OPACYpWeL0wqObRcbAqCMya13uyzqw0= -github.com/beorn7/perks v0.0.0-20180321164747-3a771d992973 h1:xJ4a3vCFaGF/jqvzLMYoU8P317H5OQ+Via4RmuPwCS0= +github.com/alecthomas/units v0.0.0-20190717042225-c3de453c63f4/go.mod h1:ybxpYRFXyAe+OPACYpWeL0wqObRcbAqCMya13uyzqw0= +github.com/antihax/optional v0.0.0-20180407024304-ca021399b1a6/go.mod h1:V8iCPQYkqmusNa815XgQio277wI47sdRh1dUOLdyC6Q= +github.com/armon/consul-api v0.0.0-20180202201655-eb2c6b5be1b6/go.mod h1:grANhF5doyWs3UAsr3K4I6qtAmlQcZDesFNEHPZAzj8= github.com/beorn7/perks v0.0.0-20180321164747-3a771d992973/go.mod h1:Dwedo/Wpr24TaqPxmxbtue+5NUziq4I4S80YR8gNf3Q= +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= github.com/bgentry/speakeasy v0.1.0/go.mod h1:+zsyZBPWlz7T6j88CTgSN5bM796AkVf0kBD4zp0CCIs= github.com/biogo/store v0.0.0-20190426020002-884f370e325d h1:vu2gsANkGtqYaQNXhmAAiJ7b1eKTbX3/aPAvDCasBgE= github.com/biogo/store v0.0.0-20190426020002-884f370e325d/go.mod h1:Iev9Q3MErcn+w3UOJD/DkEzllvugfdx7bGcMOFhvr/4= @@ -15,16 +24,21 @@ github.com/blacktear23/go-proxyprotocol v0.0.0-20180807104634-af7a81e8dd0d h1:rQ github.com/blacktear23/go-proxyprotocol v0.0.0-20180807104634-af7a81e8dd0d/go.mod h1:VKt7CNAQxpFpSDz3sXyj9hY/GbVsQCr0sB3w59nE7lU= github.com/cenkalti/backoff v2.2.1+incompatible h1:tNowT99t7UNflLxfYYSlKYsBpXdEet03Pg2g16Swow4= github.com/cenkalti/backoff v2.2.1+incompatible/go.mod h1:90ReRw6GdpyfrHakVjL/QHaoyV4aDUVVkXQJJJ3NXXM= +github.com/census-instrumentation/opencensus-proto v0.2.1/go.mod h1:f6KPmirojxKA12rnyqOA5BBL4O983OfeGPqjHWSTneU= +github.com/cespare/xxhash v1.1.0 h1:a6HrQnmkObjyL+Gs60czilIUGqrzKutQD6XZog3p+ko= +github.com/cespare/xxhash v1.1.0/go.mod h1:XrSqR1VqqWfGrhpAt58auRo0WTKS1nRRg3ghfAqPWnc= +github.com/cespare/xxhash/v2 v2.1.1 h1:6MnRN8NT7+YBpUIWxHtefFZOKTAPgGjpQSxqLNn0+qY= +github.com/cespare/xxhash/v2 v2.1.1/go.mod h1:VGX0DQ3Q6kWi7AoAeZDth3/j3BFtOZR5XLFGgcrjCOs= github.com/chzyer/logex v1.1.10/go.mod h1:+Ywpsq7O8HXn0nuIou7OrIPyXbp3wmkHB+jjWRnGsAI= github.com/chzyer/readline v0.0.0-20171208011716-f6d7a1f6fbf3/go.mod h1:nSuG5e5PlCu98SY8svDHJxuZscDgtXS6KTTbou5AhLI= github.com/chzyer/test v0.0.0-20180213035817-a1ea475d72b1/go.mod h1:Q3SI9o4m/ZMnBNeIyt5eFwwo7qiLfzFZmjNmxjkiQlU= github.com/client9/misspell v0.3.4/go.mod h1:qj6jICC3Q7zFZvVWo7KLAzC3yx5G7kyvSDkc90ppPyw= +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/coreos/bbolt v1.3.3 h1:n6AiVyVRKQFNb6mJlwESEvvLoDyiTzXX7ORAUlkeBdY= -github.com/coreos/bbolt v1.3.3/go.mod h1:iRUV2dpdMOn7Bo10OQBFzIJO9kkE559Wcmn+qkEiiKk= -github.com/coreos/etcd v3.3.13+incompatible h1:8F3hqu9fGYLBifCmRCJsicFqDx/D68Rt3q1JMazcgBQ= -github.com/coreos/etcd v3.3.13+incompatible/go.mod h1:uF7uidLiAD3TWHmW31ZFd/JWoc32PjwdhPthX9715RE= +github.com/coreos/etcd v3.3.10+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-systemd v0.0.0-20180511133405-39ca1b05acc7/go.mod h1:F5haX7vjVVG0kc13fIWeqUViNPyEJxv/OmvnBo0Yme4= @@ -33,60 +47,86 @@ github.com/coreos/go-systemd v0.0.0-20181031085051-9002847aa142/go.mod h1:F5haX7 github.com/coreos/pkg v0.0.0-20160727233714-3ac0863d7acf/go.mod h1:E3G3o1h8I7cfcXa63jLwjI0eiQQMgzzUDFVpN/nH/eA= github.com/coreos/pkg v0.0.0-20180928190104-399ea9e2e55f h1:lBNOc5arjvs8E5mO2tbpBpLoyyu8B6e44T7hJy6potg= github.com/coreos/pkg v0.0.0-20180928190104-399ea9e2e55f/go.mod h1:E3G3o1h8I7cfcXa63jLwjI0eiQQMgzzUDFVpN/nH/eA= +github.com/cpuguy83/go-md2man v1.0.10/go.mod h1:SmD6nW6nTyfqj6ABTjUi3V3JVMnlJmwcJI5acqYI6dE= +github.com/creack/pty v1.1.7/go.mod h1:lj5s0c3V2DBrqTV7llrYr5NG6My20zk30Fl46Y7DoTY= +github.com/cznic/golex v0.0.0-20181122101858-9c343928389c/go.mod h1:+bmmJDNmKlhWNG+gwWCkaBoTy39Fs+bzRxVBzoTQbIc= github.com/cznic/mathutil v0.0.0-20181122101859-297441e03548 h1:iwZdTE0PVqJCos1vaoKsclOGD3ADKpshg3SRtYBbwso= github.com/cznic/mathutil v0.0.0-20181122101859-297441e03548/go.mod h1:e6NPNENfs9mPDVNRekM7lKScauxd5kXTr1Mfyig6TDM= -github.com/cznic/sortutil v0.0.0-20150617083342-4c7342852e65 h1:hxuZop6tSoOi0sxFzoGGYdRqNrPubyaIf9KoBG9tPiE= -github.com/cznic/sortutil v0.0.0-20150617083342-4c7342852e65/go.mod h1:q2w6Bg5jeox1B+QkJ6Wp/+Vn0G/bo3f1uY7Fn3vivIQ= +github.com/cznic/parser v0.0.0-20160622100904-31edd927e5b1/go.mod h1:2B43mz36vGZNZEwkWi8ayRSSUXLfjL8OkbzwW4NcPMM= +github.com/cznic/sortutil v0.0.0-20181122101858-f5f958428db8 h1:LpMLYGyy67BoAFGda1NeOBQwqlv7nUXpm+rIVHGxZZ4= +github.com/cznic/sortutil v0.0.0-20181122101858-f5f958428db8/go.mod h1:q2w6Bg5jeox1B+QkJ6Wp/+Vn0G/bo3f1uY7Fn3vivIQ= +github.com/cznic/strutil v0.0.0-20171016134553-529a34b1c186/go.mod h1:AHHPPPXTw0h6pVabbcbyGRK1DckRn7r/STdZEeIDzZc= +github.com/cznic/y v0.0.0-20170802143616-045f81c6662a/go.mod h1:1rk5VM7oSnA4vjp+hrLQ3HWHa+Y4yPCa3/CsJrcNnvs= github.com/davecgh/go-spew v1.1.0/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= github.com/davecgh/go-spew v1.1.1 h1:vj9j/u1bqnvCEfJOwUhtlOARqs3+rkHYY13jYWTU97c= github.com/davecgh/go-spew v1.1.1/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= +github.com/dgraph-io/ristretto v0.0.1 h1:cJwdnj42uV8Jg4+KLrYovLiCgIfz9wtWm6E6KA+1tLs= +github.com/dgraph-io/ristretto v0.0.1/go.mod h1:T40EBc7CJke8TkpiYfGGKAeFjSaxuFXhuXRyumBd6RE= github.com/dgrijalva/jwt-go v3.2.0+incompatible h1:7qlOGliEKZXTDg6OTjfoBKDXWrumCAMpl/TFQ4/5kLM= github.com/dgrijalva/jwt-go v3.2.0+incompatible/go.mod h1:E3ru+11k8xSBh+hMPgOLZmtrrCbhqsmaPHjLKYnJCaQ= -github.com/dgryski/go-farm v0.0.0-20190104051053-3adb47b1fb0f h1:dDxpBYafY/GYpcl+LS4Bn3ziLPuEdGRkRjYAbSlWxSA= -github.com/dgryski/go-farm v0.0.0-20190104051053-3adb47b1fb0f/go.mod h1:SqUrOPUnsFjfmXRMNPybcSiG0BgUW2AuFH8PAnS2iTw= +github.com/dgryski/go-farm v0.0.0-20190423205320-6a90982ecee2 h1:tdlZCpZ/P9DhczCTSixgIKmwPv6+wP5DGjqLYw5SUiA= +github.com/dgryski/go-farm v0.0.0-20190423205320-6a90982ecee2/go.mod h1:SqUrOPUnsFjfmXRMNPybcSiG0BgUW2AuFH8PAnS2iTw= +github.com/docker/go-units v0.4.0 h1:3uh0PgVws3nIA0Q+MwDC8yjEPf9zjRfZZWXZYDct3Tw= +github.com/docker/go-units v0.4.0/go.mod h1:fgPhTUdO+D/Jk86RDLlptpiXQzgHJF7gydDDbaIK4Dk= github.com/dustin/go-humanize v0.0.0-20171111073723-bb3d318650d4/go.mod h1:HtrtbFcZ19U5GC7JDqmcUSB87Iq5E25KnS6fMYU6eOk= github.com/dustin/go-humanize v0.0.0-20180421182945-02af3965c54e/go.mod h1:HtrtbFcZ19U5GC7JDqmcUSB87Iq5E25KnS6fMYU6eOk= github.com/dustin/go-humanize v1.0.0 h1:VSnTsYCnlFHaM2/igO1h6X3HA71jcobQuxemgkq4zYo= github.com/dustin/go-humanize v1.0.0/go.mod h1:HtrtbFcZ19U5GC7JDqmcUSB87Iq5E25KnS6fMYU6eOk= github.com/eknkc/amber v0.0.0-20171010120322-cdade1c07385 h1:clC1lXBpe2kTj2VHdaIu9ajZQe4kcEY9j0NsnDDBZ3o= github.com/eknkc/amber v0.0.0-20171010120322-cdade1c07385/go.mod h1:0vRUJqYpeSZifjYj7uP3BG/gKcuzL9xWVV/Y+cK33KM= +github.com/elazarl/go-bindata-assetfs v1.0.0 h1:G/bYguwHIzWq9ZoyUQqrjTmJbbYn3j3CKKpKinvZLFk= +github.com/elazarl/go-bindata-assetfs v1.0.0/go.mod h1:v+YaWX3bdea5J/mo8dSETolEo7R71Vk1u8bnjau5yw4= +github.com/envoyproxy/go-control-plane v0.9.0/go.mod h1:YTl/9mNaCwkRvm6d1a2C3ymFceY/DCBVvsKhRF0iEA4= +github.com/envoyproxy/go-control-plane v0.9.1-0.20191026205805-5f8ba28d4473/go.mod h1:YTl/9mNaCwkRvm6d1a2C3ymFceY/DCBVvsKhRF0iEA4= +github.com/envoyproxy/protoc-gen-validate v0.1.0/go.mod h1:iSmxcyjqTsJpI2R4NaDN7+kN2VEUnK/pcBlmesArF7c= github.com/fatih/color v1.7.0/go.mod h1:Zm6kSWBoL9eyXnKyktHP6abPY2pDugNf5KwzbycvMj4= 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/ghodss/yaml v1.0.1-0.20190212211648-25d852aebe32 h1:Mn26/9ZMNWSw9C9ERFA1PUxfmGpolnw2v0bKOREu5ew= +github.com/ghodss/yaml v1.0.0/go.mod h1:4dBDuWmgqj2HViK6kFavaiC9ZROes6MMH2rRYeMEF04= github.com/ghodss/yaml v1.0.1-0.20190212211648-25d852aebe32/go.mod h1:GIjDIg/heH5DOkXY3YJ/wNhfHsQHoXGjl8G8amsYQ1I= -github.com/go-ole/go-ole v1.2.1 h1:2lOsA72HgjxAuMlKpFiCbHTvu44PIVkZ5hqm3RSdI/E= -github.com/go-ole/go-ole v1.2.1/go.mod h1:7FAglXiTm7HKlQRDeOQ6ZNUHidzCWXuZWq/1dTyBNF8= +github.com/go-kit/kit v0.8.0/go.mod h1:xBxKIO96dXMWWy0MnWVtmwkA9/13aqxPnvrjFYMA2as= +github.com/go-kit/kit v0.9.0/go.mod h1:xBxKIO96dXMWWy0MnWVtmwkA9/13aqxPnvrjFYMA2as= +github.com/go-logfmt/logfmt v0.3.0/go.mod h1:Qt1PoO58o5twSAckw1HlFXLmHsOX5/0LbT9GBnD5lWE= +github.com/go-logfmt/logfmt v0.4.0/go.mod h1:3RMwSq7FuexP4Kalkev3ejPJsZTpXXBr9+V4qmtdjCk= +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-playground/overalls v0.0.0-20180201144345-22ec1a223b7c/go.mod h1:UqxAgEOt89sCiXlrc/ycnx00LVvUO/eS8tMUkWX4R7w= github.com/go-sql-driver/mysql v0.0.0-20170715192408-3955978caca4/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-stack/stack v1.8.0/go.mod h1:v0f6uXyyMGvRgIKkXu+yp6POWl0qKG85gN/melR3HDY= github.com/gogo/protobuf v0.0.0-20180717141946-636bf0302bc9/go.mod h1:r8qH/GZQm5c6nD/R0oafs1akxWv10x8SbQlK7atdtwQ= github.com/gogo/protobuf v1.0.0/go.mod h1:r8qH/GZQm5c6nD/R0oafs1akxWv10x8SbQlK7atdtwQ= -github.com/gogo/protobuf v1.2.0 h1:xU6/SpYbvkNYiptHJYEDRseDLvYE7wSqhYYNy0QSUzI= -github.com/gogo/protobuf v1.2.0/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.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/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-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-20191227052852-215e87163ea7 h1:5ZkaAPbicIKTF2I64qf5Fh8Aa83Q/dnOafMYV0OMwjA= +github.com/golang/groupcache v0.0.0-20191227052852-215e87163ea7/go.mod h1:cIg4eruTrX1D+g88fzRXU5OdNfaM+9IcxsU14FzY7Hc= github.com/golang/mock v1.1.1/go.mod h1:oTYuIxOrZwtPieC+H1uAHpcLFnEyAGVDL/k47Jfbm0A= github.com/golang/protobuf v0.0.0-20180814211427-aa810b61a9c7/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= 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/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/google/btree v0.0.0-20180124185431-e89373fe6b4a/go.mod h1:lNA+9X1NB3Zf8V7Ke586lFgjr2dZNuvo3lPJSGZ5JPQ= -github.com/google/btree v0.0.0-20180813153112-4030bb1f1f0c h1:964Od4U6p2jUkFxvCydnIczKteheJEzHRToSGK3Bnlw= github.com/google/btree v0.0.0-20180813153112-4030bb1f1f0c/go.mod h1:lNA+9X1NB3Zf8V7Ke586lFgjr2dZNuvo3lPJSGZ5JPQ= github.com/google/btree v1.0.0 h1:0udJVsspx3VBr5FwtLhQQtuAsVc79tTq0ocGIPAU6qo= github.com/google/btree v1.0.0/go.mod h1:lNA+9X1NB3Zf8V7Ke586lFgjr2dZNuvo3lPJSGZ5JPQ= github.com/google/go-cmp v0.2.0/go.mod h1:oXzfMopK8JAjlY9xF4vHSVASa0yLyX7SntLO5aqRK0M= +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/gofuzz v1.0.0/go.mod h1:dBl0BpW6vV/+mYPU4Po3pmUjxk6FQPldtuIdl/M65Eg= +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/renameio v0.1.0/go.mod h1:KWCgfxg9yswjAJkECMjeO8J8rahYeXnNhOm40UhjYkI= github.com/google/shlex v0.0.0-20181106134648-c34317bd91bf/go.mod h1:RpwtwJQFrIEPstU94h88MWPXP2ektJZ8cZ0YntAmXiE= github.com/google/uuid v1.0.0/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= github.com/google/uuid v1.1.1 h1:Gkbcsh/GbpXz7lPftLA3P6TYMwjCLYm83jiFQZF/3gY= @@ -99,35 +139,51 @@ github.com/gorilla/mux v1.6.2 h1:Pgr17XVTNXAk3q/r4CpKzC5xBM/qW1uVLV+IhRZpIIk= github.com/gorilla/mux v1.6.2/go.mod h1:1lud6UwP+6orDFRuTfBEV8e9/aOM/c4fVVCaMa2zaAs= 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/grpc-ecosystem/go-grpc-middleware v1.0.1-0.20190118093823-f849b5445de4 h1:z53tR0945TRRQO/fLEVPI6SMv7ZflF0TEaTAoU7tOzg= +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.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= github.com/grpc-ecosystem/go-grpc-prometheus v1.2.0 h1:Ovs26xHkKqVztRpIrF/92BcuyuQ/YW4NSIpoGtfXNho= github.com/grpc-ecosystem/go-grpc-prometheus v1.2.0/go.mod h1:8NvIoxWQoOIhqOTXgfV/d3M/q6VIi02HzZEHgUlZvzk= github.com/grpc-ecosystem/grpc-gateway v1.4.1/go.mod h1:RSKVYQBd5MCa4OVpNdGskqpgL2+G+NZTnrVHpWWfpdw= -github.com/grpc-ecosystem/grpc-gateway v1.5.1 h1:3scN4iuXkNOyP98jF55Lv8a9j1o/IwvnDIZ0LHJK1nk= -github.com/grpc-ecosystem/grpc-gateway v1.5.1/go.mod h1:RSKVYQBd5MCa4OVpNdGskqpgL2+G+NZTnrVHpWWfpdw= +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/hashicorp/hcl v1.0.0/go.mod h1:E5yfLk+7swimpb2L/Alb/PJmXilQ/rhwaUYs4T20WEQ= github.com/hpcloud/tail v1.0.0 h1:nfCOvKYfkgYP8hkirhJocXT2+zOD8yUNjXaWfTlyFKI= github.com/hpcloud/tail v1.0.0/go.mod h1:ab1qPbhIpdTxEkNHXyeSf5vhxWSCs/tWer42PpOxQnU= github.com/inconshreveable/mousetrap v1.0.0 h1:Z8tu5sraLXCXIcARxBp/8cbvlwVa7Z1NHg9XEKhtSvM= github.com/inconshreveable/mousetrap v1.0.0/go.mod h1:PxqpIevigyE2G7u3NXJIT2ANytuPF1OarO4DADm73n8= +github.com/jeremywohl/flatten v0.0.0-20190921043622-d936035e55cf h1:Ut4tTtPNmInWiEWJRernsWm688R0RN6PFO8sZhwI0sk= +github.com/jeremywohl/flatten v0.0.0-20190921043622-d936035e55cf/go.mod h1:4AmD/VxjWcI5SRB0n6szE2A6s2fsNHDLO0nAlMHgfLQ= github.com/jonboulle/clockwork v0.1.0 h1:VKV+ZcuP6l3yW9doeqz6ziZGgcynBVQO+obU0+0hcPo= github.com/jonboulle/clockwork v0.1.0/go.mod h1:Ii8DK3G1RaLaWxj9trq07+26W01tbo22gdxWY5EU2bo= -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/go.mod h1:KdQUCv79m/52Kvf8AW2vK1V8akMuk1QjK/uOdHXbAo4= +github.com/json-iterator/go v1.1.8/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= github.com/juju/ratelimit v1.0.1 h1:+7AIFJVQ0EQgq/K9+0Krm7m530Du7tIz0METWzN0RgY= github.com/juju/ratelimit v1.0.1/go.mod h1:qapgC/Gy+xNh9UxzV13HGGl/6UXNN+ct+vwSgWNm/qk= +github.com/julienschmidt/httprouter v1.2.0/go.mod h1:SYymIcj16QtmaHHD7aYtjjsJG7VTCxuUUipMqKk8s4w= +github.com/kisielk/errcheck v1.1.0/go.mod h1:EZBBE59ingxPouuu3KfxchcWSUPOHkagtvWXihfKN4Q= +github.com/kisielk/errcheck v1.2.0/go.mod h1:/BMXB+zMLi60iA8Vv6Ksmxu/1UDYcXs4uQLJ+jE2L00= +github.com/kisielk/gotool v1.0.0/go.mod h1:XhKaO+MFFWcvkIS/tQcRk01m1F5IRFswLeQ+oQHNcck= github.com/klauspost/cpuid v0.0.0-20170728055534-ae7887de9fa5 h1:2U0HzY8BJ8hVwDKIzp7y4voR9CX/nvcfymLmg2UiOio= github.com/klauspost/cpuid v0.0.0-20170728055534-ae7887de9fa5/go.mod h1:Pj4uuM528wm8OyEC2QMXAi2YiTZ96dNQPGgoMS4s3ek= -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/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/pty v1.0.0/go.mod h1:pFQYn66WHrOpPYNljwOMqo10TkYh1fy3cYio2l3bCsQ= github.com/kr/pty v1.1.1/go.mod h1:pFQYn66WHrOpPYNljwOMqo10TkYh1fy3cYio2l3bCsQ= 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/magiconair/properties v1.8.0/go.mod h1:PppfXfuXeibc/6YijjN8zIbojt8czPbwD3XqdrwzmxQ= github.com/mattn/go-colorable v0.0.9/go.mod h1:9vuHe8Xs5qXnSaW/c/ABM9alt+Vo+STaOChaDxuIBZU= github.com/mattn/go-isatty v0.0.4/go.mod h1:M+lRXTBqGeGNdLjl/ufCoiOlB5xdOkqRJdNxMWT7Zi4= github.com/mattn/go-runewidth v0.0.2/go.mod h1:LwmH8dsx7+W8Uxz3IHJYH5QSwggIsqBzpuz5H//U1FU= @@ -135,13 +191,18 @@ github.com/mattn/go-shellwords v1.0.3/go.mod h1:3xCvwCdWdlDJUrvuMn7Wuy9eWs4pE8vq github.com/matttproud/golang_protobuf_extensions v1.0.0/go.mod h1:D8He9yQNgCq6Z5Ld7szi9bcBfOoFv/3dc6xSMkL2PC0= 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/mitchellh/go-homedir v1.1.0/go.mod h1:SfyaCUpYCn1Vlf4IUYiD9fPX4A5wJrkLzIz1N1q0pr0= +github.com/mitchellh/mapstructure v1.1.2/go.mod h1:FVVH3fgwuzCH5S8UJGiWEs2h04kUh9fWfEaFds41c1Y= +github.com/modern-go/concurrent v0.0.0-20180228061459-e0a39a4cb421/go.mod h1:6dJC0mAP4ikYIbvyc7fijjWJddQyLn8Ig3JB5CqoB9Q= github.com/modern-go/concurrent v0.0.0-20180306012644-bacd9c7ef1dd h1:TRLaZ9cD/w8PVh93nsPXa1VrQ6jlwL5oN8l14QlcNfg= github.com/modern-go/concurrent v0.0.0-20180306012644-bacd9c7ef1dd/go.mod h1:6dJC0mAP4ikYIbvyc7fijjWJddQyLn8Ig3JB5CqoB9Q= +github.com/modern-go/reflect2 v0.0.0-20180701023420-4b7aa43c6742/go.mod h1:bx2lNnkwVCuqBIxFjflWJWanXIb3RllmbCylyMrvgv0= 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/mwitkow/go-conntrack v0.0.0-20161129095857-cc309e4a2223/go.mod h1:qRWi+5nqEBWmkhHvq77mSJWrCKwh8bxhgT7d/eI7P4U= github.com/ngaut/pools v0.0.0-20180318154953-b7bc8c42aac7 h1:7KAv7KMGTTqSmYZtNdcNTgsos+vFzULLwyElndwn+5c= github.com/ngaut/pools v0.0.0-20180318154953-b7bc8c42aac7/go.mod h1:iWMfgwqYW+e8n5lC/jjNEhwcjbRDpl5NT7n2h+4UNcI= github.com/ngaut/sync2 v0.0.0-20141008032647-7a24ed77b2ef h1:K0Fn+DoFqNqktdZtdV3bPQ/0cuYh2H4rkg0tytX/07k= @@ -154,67 +215,95 @@ github.com/onsi/gomega v1.4.2 h1:3mYCb7aPxS/RU7TI1y4rkEn1oKmPRjNJLNEXgw7MH2I= github.com/onsi/gomega v1.4.2/go.mod h1:ex+gbHU/CVuBBDIJjb2X0qEXbFg53c61hWP/1CpauHY= 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.1.0 h1:pWlfV3Bxv7k65HYwkikxat0+s3pV4bsqf19k25Ur8rU= +github.com/opentracing/opentracing-go v1.1.0/go.mod h1:UkNAQd3GIcIGf0SeVgPpRdFStlNbqXla1AfSYxPUl2o= +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/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/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/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/errcode v0.0.0-20180921232412-a1a7271709d9 h1:KH4f4Si9XK6/IW50HtoaiLIFHGkapOM6w83za47UYik= github.com/pingcap/errcode v0.0.0-20180921232412-a1a7271709d9/go.mod h1:4b2X8xSqxIroj/IZ9MX/VGZhAwc11wB9wRIzHvz6SeM= github.com/pingcap/errors v0.11.0/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/failpoint v0.0.0-20190512135322-30cc7431d99c h1:hvQd3aOLKLF7xvRV6DzvPkKY4QXzfVbjU1BhW0d9yL8= -github.com/pingcap/failpoint v0.0.0-20190512135322-30cc7431d99c/go.mod h1:DNS3Qg7bEDhU6EXNHF+XSv/PGznQaMJ5FWvctpm6pQI= +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/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-20200115060041-f2180fbf0df8 h1:P1+sLSDI3Aw1UURnn8VOTY3kEv47a3MSnxUyBkwAcR8= github.com/pingcap/failpoint v0.0.0-20200115060041-f2180fbf0df8/go.mod h1:DNS3Qg7bEDhU6EXNHF+XSv/PGznQaMJ5FWvctpm6pQI= -github.com/pingcap/goleveldb v0.0.0-20171020122428-b9ff6c35079e h1:P73/4dPCL96rGrobssy1nVy2VaVpNCuLpCbr+FEaTA8= -github.com/pingcap/goleveldb v0.0.0-20171020122428-b9ff6c35079e/go.mod h1:O17XtbryoCJhkKGbT62+L2OlrniwqiGLSqrmdHCMzZw= -github.com/pingcap/kvproto v0.0.0-20190822090350-11ea838aedf7/go.mod h1:QMdbTAXCHzzygQzqcG9uVUgU2fKeSN1GmfMiykdSzzY= -github.com/pingcap/kvproto v0.0.0-20190910074005-0e61b6f435c1/go.mod h1:QMdbTAXCHzzygQzqcG9uVUgU2fKeSN1GmfMiykdSzzY= +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/goleveldb v0.0.0-20191226122134-f82aafb29989 h1:surzm05a8C9dN8dIUmo4Be2+pMRb6f55i+UIYrluu2E= +github.com/pingcap/goleveldb v0.0.0-20191226122134-f82aafb29989/go.mod h1:O17XtbryoCJhkKGbT62+L2OlrniwqiGLSqrmdHCMzZw= +github.com/pingcap/kvproto v0.0.0-20191211054548-3c6b38ea5107/go.mod h1:WWLmULLO7l8IOcQG+t+ItJ3fEcrL5FxF0Wu+HrMy26w= +github.com/pingcap/kvproto v0.0.0-20191213111810-93cb7c623c8b/go.mod h1:WWLmULLO7l8IOcQG+t+ItJ3fEcrL5FxF0Wu+HrMy26w= github.com/pingcap/kvproto v0.0.0-20200108025604-a4dc183d2af5 h1:RUxQExD5yubAjWGnw8kcxfO9abbiVHIE1rbuCyQCWDE= github.com/pingcap/kvproto v0.0.0-20200108025604-a4dc183d2af5/go.mod h1:WWLmULLO7l8IOcQG+t+ItJ3fEcrL5FxF0Wu+HrMy26w= -github.com/pingcap/log v0.0.0-20190715063458-479153f07ebd/go.mod h1:WpHUKhNZ18v116SvGrmjkA9CBhYmuUTKL+p8JC9ANEw= 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/parser v0.0.0-20191012071233-32876040fefb h1:okeNsbftvzQ8I9DseKukhZURRYJUHOpRSHwlSZC0g0g= -github.com/pingcap/parser v0.0.0-20191012071233-32876040fefb/go.mod h1:1FNvfp9+J0wvc4kl8eGNh7Rqrxveg15jJoWo/a0uHwA= -github.com/pingcap/pd v1.1.0-beta.0.20190923032047-5c648dc365e0 h1:GIEq+wZfrl2bcJxpuSrEH4H7/nlf5YdmpS+dU9lNIt8= -github.com/pingcap/pd v1.1.0-beta.0.20190923032047-5c648dc365e0/go.mod h1:G/6rJpnYwM0LKMec2rI82/5Kg6GaZMvlfB+e6/tvYmI= -github.com/pingcap/tidb v1.1.0-beta.0.20191017064824-e1ba309148ad h1:adqdkTB0/B0q6Hh+yO8OxZ93PguqTAjqixw4nxptV4w= -github.com/pingcap/tidb v1.1.0-beta.0.20191017064824-e1ba309148ad/go.mod h1:v9fRBNHEMlhqAY9nZTTQ2+rVccmldHhyL5rtvfM9a/I= -github.com/pingcap/tidb-tools v2.1.3-0.20190321065848-1e8b48f5c168+incompatible h1:MkWCxgZpJBgY2f4HtwWMMFzSBb3+JPzeJgF3VrXE/bU= -github.com/pingcap/tidb-tools v2.1.3-0.20190321065848-1e8b48f5c168+incompatible/go.mod h1:XGdcy9+yqlDSEMTpOXnwf3hiTeqrV6MN/u1se9N8yIM= -github.com/pingcap/tipb v0.0.0-20191015023537-709b39e7f8bb h1:miVtIZZhjpJ2bD+HZ99Mkdq8MuTr6QBHGmJtjqderBA= -github.com/pingcap/tipb v0.0.0-20191015023537-709b39e7f8bb/go.mod h1:RtkHW8WbcNxj8lsbzjaILci01CtYnYbIkQhjyZWrWVI= +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/parser v0.0.0-20200120100653-1d87b3907217 h1:UtieYveNGV84dIdb01UIXMQzGIyGLRiAoGXgLe9rJws= +github.com/pingcap/parser v0.0.0-20200120100653-1d87b3907217/go.mod h1:9v0Edh8IbgjGYW2ArJr19E+bvL8zKahsFp+ixWeId+4= +github.com/pingcap/pd v1.1.0-beta.0.20191219054547-4d65bbefbc6d h1:Ui80aiLTyd0EZD56o2tjFRYpHfhazBjtBdKeR8UoTFY= +github.com/pingcap/pd v1.1.0-beta.0.20191219054547-4d65bbefbc6d/go.mod h1:CML+b1JVjN+VbDijaIcUSmuPgpDjXEY7UiOx5yDP8eE= +github.com/pingcap/sysutil v0.0.0-20191216090214-5f9620d22b3b h1:EEyo/SCRswLGuSk+7SB86Ak1p8bS6HL1Mi4Dhyuv6zg= +github.com/pingcap/sysutil v0.0.0-20191216090214-5f9620d22b3b/go.mod h1:EB/852NMQ+aRKioCpToQ94Wl7fktV+FNnxf3CX/TTXI= +github.com/pingcap/tidb v1.1.0-beta.0.20200204134155-ebc6a2d39dd7 h1:tD1uPITXsdLnL0aqHD8eTmeWv1Jxy7ySmAfM8oPIuoE= +github.com/pingcap/tidb v1.1.0-beta.0.20200204134155-ebc6a2d39dd7/go.mod h1:46Dyz6ktuiBBhNbdUsvrjCtif9nyU3r7fXBYuTIXg78= +github.com/pingcap/tidb-tools v3.0.6-0.20191106033616-90632dda3863+incompatible h1:H1jg0aDWz2SLRh3hNBo2HFtnuHtudIUvBumU7syRkic= +github.com/pingcap/tidb-tools v3.0.6-0.20191106033616-90632dda3863+incompatible/go.mod h1:XGdcy9+yqlDSEMTpOXnwf3hiTeqrV6MN/u1se9N8yIM= +github.com/pingcap/tidb-tools v3.1.0-beta.1.0.20200108061154-356b0e2e2282+incompatible h1:HuvFPu3afgeirZka0oTHwymfoPYsXiXRlKjcAahXLNM= +github.com/pingcap/tidb-tools v3.1.0-beta.1.0.20200108061154-356b0e2e2282+incompatible/go.mod h1:XGdcy9+yqlDSEMTpOXnwf3hiTeqrV6MN/u1se9N8yIM= +github.com/pingcap/tipb v0.0.0-20190428032612-535e1abaa330/go.mod h1:RtkHW8WbcNxj8lsbzjaILci01CtYnYbIkQhjyZWrWVI= +github.com/pingcap/tipb v0.0.0-20200103084511-1d37e605f65d h1:ohGnm9xZ7pIysk7quOC7lZa8kOm9Pl5TMyjBThXqy2U= +github.com/pingcap/tipb v0.0.0-20200103084511-1d37e605f65d/go.mod h1:RtkHW8WbcNxj8lsbzjaILci01CtYnYbIkQhjyZWrWVI= 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/pmezard/go-difflib v1.0.0 h1:4DBwDE0NGyQoBHbLQYPwSUPoCMWR5BEzIk/f1lZbAQM= github.com/pmezard/go-difflib v1.0.0/go.mod h1:iKH77koFhYxTK1pcRnkKkqfTogsbg7gZNVY4sRDYZ/4= github.com/prometheus/client_golang v0.8.0/go.mod h1:7SWBe2y4D6OKWSNQJUaRYU/AaXPKyh/dDVn+NZz0KFw= -github.com/prometheus/client_golang v0.9.0 h1:tXuTFVHC03mW0D+Ua1Q2d1EAVqLTuggX50V0VLICCzY= -github.com/prometheus/client_golang v0.9.0/go.mod h1:7SWBe2y4D6OKWSNQJUaRYU/AaXPKyh/dDVn+NZz0KFw= +github.com/prometheus/client_golang v0.9.1/go.mod h1:7SWBe2y4D6OKWSNQJUaRYU/AaXPKyh/dDVn+NZz0KFw= +github.com/prometheus/client_golang v1.0.0/go.mod h1:db9x61etRT2tGnBNRi70OPL5FsnadC4Ky3P0J6CfImo= +github.com/prometheus/client_golang v1.3.0 h1:miYCvYqFXtl/J9FIy8eNpBfYthAEFg+Ys0XyUVEcDsc= +github.com/prometheus/client_golang v1.3.0/go.mod h1:hJaj2vgQTGQmVCsAACORcieXFeDPbaTKGT+JTgUa3og= github.com/prometheus/client_model v0.0.0-20170216185247-6f3806018612/go.mod h1:MbSGuTsp3dbXC40dX6PRTWyKYBIrTGTE9sqQNg2J8bo= -github.com/prometheus/client_model v0.0.0-20171117100541-99fa1f4be8e5/go.mod h1:MbSGuTsp3dbXC40dX6PRTWyKYBIrTGTE9sqQNg2J8bo= -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/go.mod h1:xMI15A0UPsDsEKsMN9yxemIoYk6Tm2C1GtYGdfGttqA= +github.com/prometheus/client_model v0.0.0-20190812154241-14fe0d1b01d4/go.mod h1:xMI15A0UPsDsEKsMN9yxemIoYk6Tm2C1GtYGdfGttqA= +github.com/prometheus/client_model v0.1.0 h1:ElTg5tNp4DqfV7UQjDqv2+RJlNzsDtvNAWccbItceIE= +github.com/prometheus/client_model v0.1.0/go.mod h1:xMI15A0UPsDsEKsMN9yxemIoYk6Tm2C1GtYGdfGttqA= github.com/prometheus/common v0.0.0-20180518154759-7600349dcfe1/go.mod h1:daVV7qP5qjZbuso7PdcryaAu0sAZbrN9i7WWcTMWvro= -github.com/prometheus/common v0.0.0-20181020173914-7e9e6cabbd39 h1:Cto4X6SVMWRPBkJ/3YHn1iDGDGc/Z+sW+AEMKHMVvN4= -github.com/prometheus/common v0.0.0-20181020173914-7e9e6cabbd39/go.mod h1:daVV7qP5qjZbuso7PdcryaAu0sAZbrN9i7WWcTMWvro= +github.com/prometheus/common v0.4.1/go.mod h1:TNfzLD0ON7rHzMJeJkieUDPYmFC7Snx/y86RQel1bk4= +github.com/prometheus/common v0.7.0/go.mod h1:DjGbpBbp5NYNiECxcL/VnbXCCaQpKd3tt26CguLLsqA= +github.com/prometheus/common v0.8.0 h1:bLkjvFe2ZRX1DpcgZcdf7j/+MnusEps5hktST/FHA34= +github.com/prometheus/common v0.8.0/go.mod h1:PC/OgXc+UN7B4ALwvn1yzVZmVwvhXp5JsbBv6wSv6i0= github.com/prometheus/procfs v0.0.0-20180612222113-7d6f385de8be/go.mod h1:c3At6R/oaqEKCNdg8wHV1ftS6bRYblBhIjjI8uT2IGk= -github.com/prometheus/procfs v0.0.0-20181005140218-185b4288413d h1:GoAlyOgbOEIFdaDqxJVlbOQ1DtGmZWs/Qau0hIlk+WQ= github.com/prometheus/procfs v0.0.0-20181005140218-185b4288413d/go.mod h1:c3At6R/oaqEKCNdg8wHV1ftS6bRYblBhIjjI8uT2IGk= -github.com/remyoudompheng/bigfft v0.0.0-20190512091148-babf20351dd7/go.mod h1:qqbHyh8v60DhA7CoWK5oRCqLrMHRGoxYCSS9EjAz6Eo= +github.com/prometheus/procfs v0.0.2/go.mod h1:TjEm7ze935MbeOT/UhFTIMYKhuLP4wbCsTZCD3I8kEA= +github.com/prometheus/procfs v0.0.8 h1:+fpWZdT24pJBiqJdAwYBjPSk+5YmQzYNPYzQsdzLkt8= +github.com/prometheus/procfs v0.0.8/go.mod h1:7Qr8sr6344vo1JqZ6HhLceV9o3AJ1Ff+GxbHq6oeK9A= 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/rogpeppe/fastuuid v0.0.0-20150106093220-6724a57986af/go.mod h1:XWv6SoW27p1b0cqNHllgS5HIMJraePCO15w5zCzIWYg= +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/russross/blackfriday v1.5.2/go.mod h1:JO/DiYxRf+HjHt06OyowR9PTA263kcR/rfWxYHBV53g= 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.18.10+incompatible h1:cy84jW6EVRPa5g9HAHrlbxMSIjBhDSX0OFYyMYminYs= -github.com/shirou/gopsutil v2.18.10+incompatible/go.mod h1:5b4v6he4MtMOwMlS0TUMTu2PcXUg8+E1lC7eC3UO/RA= +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/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/shurcooL/httpfs v0.0.0-20171119174359-809beceb2371 h1:SWV2fHctRpRrp49VXJ6UZja7gU9QLHwRpIPBN89SKEo= github.com/shurcooL/httpfs v0.0.0-20171119174359-809beceb2371/go.mod h1:ZY1cvUeJuFPAdZ/B6v7RHavJWZn2YPVFQ1OSXhCGOkg= +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/sirupsen/logrus v1.0.5/go.mod h1:pMByvHTf9Beacp5x1UXfOR9xyW/9antXMhjMPG0dEzc= github.com/sirupsen/logrus v1.2.0/go.mod h1:LxeOpSwHxABJmUn/MG1IvRgCAasNZTLOkJPxbbu5VWo= @@ -224,120 +313,161 @@ 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/spf13/cobra v0.0.3 h1:ZlrZ4XsMRm04Fr5pSFxBgfND2EBVa1nLpiy1stUsX/8= +github.com/spf13/afero v1.1.2/go.mod h1:j4pytiNVoe2o6bmDsKpLACNPDBIoEAkihy7loJ1B0CQ= +github.com/spf13/cast v1.3.0/go.mod h1:Qx5cxh0v+4UWYiBimWS+eyWzqEqokIECu5etghLkUJE= github.com/spf13/cobra v0.0.3/go.mod h1:1l0Ry5zgKvJasoi3XT1TypsSe7PqH0Sj9dhYf7v3XqQ= -github.com/spf13/pflag v1.0.1 h1:aCvUg6QPl3ibpQUxyLkrEkCHtPqYJL4x9AuhqVqFis4= +github.com/spf13/cobra v0.0.5 h1:f0B+LkLX6DtmRH1isoNA9VTtNUK9K8xYd28JNNfOv/s= +github.com/spf13/cobra v0.0.5/go.mod h1:3K3wKZymM7VvHMDS9+Akkh4K60UwM26emMESw8tLCHU= +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/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= +github.com/spf13/viper v1.3.2/go.mod h1:ZiWeW+zYFKm7srdB9IoDzzZXaJaI5eL9QjNiN/DMA2s= github.com/stretchr/objx v0.1.0/go.mod h1:HFkY916IF+rwdDfMAkV7OtwuqBVzrE8GR6GFx+wExME= -github.com/stretchr/objx v0.1.1 h1:2vfRuCMp5sSVIDSqO8oNnWJq7mPa6KVP3iPIwFBuy8A= github.com/stretchr/objx v0.1.1/go.mod h1:HFkY916IF+rwdDfMAkV7OtwuqBVzrE8GR6GFx+wExME= 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/struCoder/pidusage v0.1.2 h1:fFPTThlcWFQyizv3xKs5Lyq1lpG5lZ36arEGNhWz2Vs= -github.com/struCoder/pidusage v0.1.2/go.mod h1:pWBlW3YuSwRl6h7R5KbvA4N8oOqe9LjaKW5CwT1SPjI= github.com/syndtr/goleveldb v0.0.0-20180815032940-ae2bd5eed72d h1:4J9HCZVpvDmj2tiKGSTUnb3Ok/9CEQb9oqu9LHKQQpc= github.com/syndtr/goleveldb v0.0.0-20180815032940-ae2bd5eed72d/go.mod h1:Z4AUp2Km+PwemOoO/VB5AOx9XSsIItzFjoJlOSiYmn0= 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/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/uber-go/atomic v1.3.2 h1:Azu9lPBWRNKzYXSIwRfgRuDuS0YKsK4NFhiQv98gkxo= github.com/uber-go/atomic v1.3.2/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-lib v1.5.0 h1:OHbgr8l656Ub3Fw5k9SWnBfIEwvoHQ+W2y+Aa9D1Uyo= github.com/uber/jaeger-lib v1.5.0/go.mod h1:ComeNDZlWwrWnDv8aPp0Ba6+uUTzImX/AauajbLI56U= -github.com/ugorji/go v1.1.2 h1:JON3E2/GPW2iDNGoSAusl1KDf5TRQ8k8q7Tp097pZGs= github.com/ugorji/go v1.1.2/go.mod h1:hnLbHMwcvSihnDhEfx2/BzKp2xb0Y+ErdfYcrs9tkJQ= -github.com/ugorji/go/codec v0.0.0-20190204201341-e444a5086c43 h1:BasDe+IErOQKrMVXab7UayvSlIpiyGwRvuX3EKYY7UA= +github.com/ugorji/go/codec v0.0.0-20181204163529-d75b2dcb6bc8/go.mod h1:VFNgLljTbGfSG7qAOspJ7OScBnGdDN/yBr0sguwnwf0= github.com/ugorji/go/codec v0.0.0-20190204201341-e444a5086c43/go.mod h1:iT03XoTwV7xq/+UGwKO3UbC1nNNlopQiY61beSdrtOA= github.com/unrolled/render v0.0.0-20171102162132-65450fb6b2d3/go.mod h1:tu82oB5W2ykJRVioYsB+IQKcft7ryBr7w12qMBUPyXg= github.com/unrolled/render v0.0.0-20180914162206-b9786414de4d h1:ggUgChAeyge4NZ4QUw6lhHsVymzwSDJOZcE0s2X8S20= github.com/unrolled/render v0.0.0-20180914162206-b9786414de4d/go.mod h1:tu82oB5W2ykJRVioYsB+IQKcft7ryBr7w12qMBUPyXg= github.com/urfave/cli v1.20.0/go.mod h1:70zkFmudgCuE/ngEzBv17Jvp/497gISqfk5gWijbERA= +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/xiang90/probing v0.0.0-20190116061207-43a291ad63a2 h1:eY9dn8+vbi4tKz5Qo6v2eYzo7kUS51QINcR5jNpbZS8= github.com/xiang90/probing v0.0.0-20190116061207-43a291ad63a2/go.mod h1:UETIi67q53MR2AWcXfiuqkDkRtnGDLqkBTpCHuJHxtU= +github.com/xordataexchange/crypt v0.0.3-0.20170626215501-b2862e3d0a77/go.mod h1:aYKd//L2LvnjZzWKhF00oedf4jCCReLcmhLdhm1A27Q= github.com/yookoala/realpath v1.0.0/go.mod h1:gJJMA9wuX7AcqLy1+ffPatSCySA1FQ2S8Ya9AIoYBpE= 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/etcd v0.0.0-20190320044326-77d4b742cdbf h1:rmttwKPEgG/l4UscTDYtaJgeUsedKPKSyFfNQLI6q+I= go.etcd.io/etcd v0.0.0-20190320044326-77d4b742cdbf/go.mod h1:KSGwdbiFchh5KIC9My2+ZVl5/3ANcwohw50dpPwa2cw= -go.uber.org/atomic v1.3.2 h1:2Oa65PReHzfn29GpvgsYwloV9AVFHPDk8tYxt2c2tr4= +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.20191211224106-0dc78a144b31 h1:eyYvL5Nw6qK5WqiO4hBoxn0jtmbSANxqmezYGp0vuC0= +go.etcd.io/etcd v0.5.0-alpha.5.0.20191211224106-0dc78a144b31/go.mod h1:dnLIgRNXwCJa5e+c6mIZCrds/GIG4ncV9HhK5PX7jPg= go.uber.org/atomic v1.3.2/go.mod h1:gD2HeocX3+yG+ygLZcrzQJaqmWj9AIm7n08wl/qW/PE= -go.uber.org/atomic v1.4.0 h1:cxzIVoETapQEqDhQu3QfnvXAV4AlzcvUCxkVUFw3+EU= go.uber.org/atomic v1.4.0/go.mod h1:gD2HeocX3+yG+ygLZcrzQJaqmWj9AIm7n08wl/qW/PE= -go.uber.org/multierr v1.1.0 h1:HoEmRHQPVSqub6w2z2d2EOVs2fjyFRGyofhKuyDq0QI= +go.uber.org/atomic v1.5.0/go.mod h1:sABNBOSYdrvTF6hTgEIbc7YasKWGhgEQZyfxyTvoXHQ= +go.uber.org/atomic v1.5.1 h1:rsqfU5vBkVknbhUGbAUwQKR2H4ItV8tjJ+6kJX4cxHM= +go.uber.org/atomic v1.5.1/go.mod h1:sABNBOSYdrvTF6hTgEIbc7YasKWGhgEQZyfxyTvoXHQ= +go.uber.org/automaxprocs v1.2.0/go.mod h1:YfO3fm683kQpzETxlTGZhGIVmXAhaw3gxeBADbpZtnU= +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/go.mod h1:wR5kodmAFQ0UK8QlbwjlSNy0Z68gJhDJUG5sjR94q/0= -go.uber.org/multierr v1.2.0 h1:6I+W7f5VwC5SV9dNrZ3qXrDB9mD0dyGOi/ZJmYw03T4= -go.uber.org/multierr v1.2.0/go.mod h1:wR5kodmAFQ0UK8QlbwjlSNy0Z68gJhDJUG5sjR94q/0= -go.uber.org/zap v1.9.1 h1:XCJQEf3W6eZaVwhRBof6ImoYGJSITeKWsyeh3HFu/5o= +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/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.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/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= golang.org/x/crypto v0.0.0-20180608092829-8ac0e0d97ce4/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/go.mod h1:djNgcEr1/C05ACkg1iLfiJU5Ep61QUkGW8qpdssI0+w= -golang.org/x/crypto v0.0.0-20190605123033-f99c8df09eb5 h1:58fnuSXlxZmFdJyvtTFVmVhcMLU6v5fEb/ok4wyqtNU= +golang.org/x/crypto v0.0.0-20190510104115-cbcb75029529/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= golang.org/x/crypto v0.0.0-20190605123033-f99c8df09eb5/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= golang.org/x/crypto v0.0.0-20190909091759-094676da4a83/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= -golang.org/x/crypto v0.0.0-20190911031432-227b76d455e7 h1:0hQKqeLdqlt5iIwVOBErRisrHJAN57yOiPRQItI20fU= -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-20191206172530-e9b2fee46413/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto= +golang.org/x/crypto v0.0.0-20200109152110-61a87790db17 h1:nVJ3guKA9qdkEQ3TUdXI9QSINo2CUPM/cySEvw2w8I0= +golang.org/x/crypto v0.0.0-20200109152110-61a87790db17/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto= golang.org/x/exp v0.0.0-20190121172915-509febef88a4/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= golang.org/x/lint v0.0.0-20181026193005-c67002cb31c3/go.mod h1:UVdnD1Gm6xHRNCYTkRU2/jEulfH38KcIWyp/GAMgvoE= golang.org/x/lint v0.0.0-20190227174305-5b3e6a55c961/go.mod h1:wehouNa3lNwaWXcvxsM5YxQ5yQlVC4a0KAMCusXpPoU= golang.org/x/lint v0.0.0-20190313153728-d0100b6bd8b3/go.mod h1:6SW0HCj/g11FgYtHlgUYUwCkIfeOF89ocIRzGO/8vkc= +golang.org/x/lint v0.0.0-20190930215403-16217165b5de/go.mod h1:6SW0HCj/g11FgYtHlgUYUwCkIfeOF89ocIRzGO/8vkc= +golang.org/x/lint v0.0.0-20191125180803-fdd1cda4f05f h1:J5lckAjkw6qYlOZNj90mLYNTEKDvWeuc1yieZ8qUzUE= +golang.org/x/lint v0.0.0-20191125180803-fdd1cda4f05f/go.mod h1:5qLYkcX4OjUUV8bRuDixDT3tpyyb+LUpUlRWLxfhWrs= +golang.org/x/mod v0.0.0-20190513183733-4bf6d317e70e/go.mod h1:mXi4GBBbnImb6dmsKGUJ2LatrhH/nqhxcFungHvyanc= +golang.org/x/mod v0.1.1-0.20191105210325-c90efee705ee/go.mod h1:QqPTAvyqsEbceGzBzNggFXnrqF1CaUcvgkdR5Ot7KZg= golang.org/x/net v0.0.0-20180724234803-3673e40ba225/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= golang.org/x/net v0.0.0-20180826012351-8a410e7b638d/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= golang.org/x/net v0.0.0-20180906233101-161cd47e91fd/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= golang.org/x/net v0.0.0-20181005035420-146acd28ed58/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= +golang.org/x/net v0.0.0-20181114220301-adae6a3d119a/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= +golang.org/x/net v0.0.0-20181220203305-927f97764cc3/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/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg= golang.org/x/net v0.0.0-20190404232315-eb5bcb51f2a3/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg= -golang.org/x/net v0.0.0-20190603091049-60506f45cf65 h1:+rhAzEzT3f4JtomfC371qB+0Ola2caSKcY69NUBZrRQ= golang.org/x/net v0.0.0-20190603091049-60506f45cf65/go.mod h1:HSz+uSET+XFnRR8LxR5pz3Of3rY3CfYBVs4xY44aLks= +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/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= -golang.org/x/net v0.0.0-20190909003024-a7b16738d86b h1:XfVGCX+0T4WOStkaOsJRllbsiImhB2jgVBGc9L0lPGc= +golang.org/x/net v0.0.0-20190813141303-74dc4d7220e7/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= golang.org/x/net v0.0.0-20190909003024-a7b16738d86b/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= +golang.org/x/net v0.0.0-20191002035440-2ec189313ef0/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= +golang.org/x/net v0.0.0-20191209160850-c0dbc17a3553 h1:efeOvDhwQ29Dj3SdAV/MJf8oukgn+8D8WgaCaRMchF8= +golang.org/x/net v0.0.0-20191209160850-c0dbc17a3553/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= golang.org/x/oauth2 v0.0.0-20180821212333-d2e6202438be/go.mod h1:N/0e6XlmueqKjAGxoOufVs8QHGRruUQn6yWY3a++T0U= 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/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= -golang.org/x/sync v0.0.0-20190423024810-112230192c58 h1:8gQV6CLnAEikrhgkHFbMAEhagSSnXWGV915qUMm9mrU= 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/sys v0.0.0-20180830151530-49385e6e1522/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20180905080454-ebe1bf3edb33/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20180909124046-d0be0721c37e/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= +golang.org/x/sys v0.0.0-20181107165924-66b7b1311ac8/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= +golang.org/x/sys v0.0.0-20181116152217-5ac8a444bdc5/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= +golang.org/x/sys v0.0.0-20181205085412-a5c9d58dba9a/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20190215142949-d0b11bdaac8a/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20190412213103-97732733099d/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20190422165155-953cdadca894/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20190606165138-5da285871e9c h1:+EXw7AwNOKzPFXMZ1yNjO40aWCh3PIquJB2fYlv9wcs= golang.org/x/sys v0.0.0-20190606165138-5da285871e9c/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20190826190057-c7b8b68b1456/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20190909082730-f460065e899a/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20190910064555-bbd175535a8b h1:3S2h5FadpNr0zUUCVZjlKIEYF+KaX/OBplTGo89CYHI= -golang.org/x/sys v0.0.0-20190910064555-bbd175535a8b/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20191210023423-ac6580df4449/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20191220142924-d4481acd189f/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20200113162924-86b910548bc1 h1:gZpLHxUX5BdYLA08Lj4YCJNN/jk7KtquiArPoeX0WvA= +golang.org/x/sys v0.0.0-20200113162924-86b910548bc1/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= 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/time v0.0.0-20180412165947-fbb02b2291d2 h1:+DCIGbF/swA92ohVg0//6X2IVY3KZs6p9mix0ziNYJM= golang.org/x/time v0.0.0-20180412165947-fbb02b2291d2/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= +golang.org/x/tools v0.0.0-20180221164845-07fd8470d635/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= golang.org/x/tools v0.0.0-20180917221912-90fa682c2a6e/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= +golang.org/x/tools v0.0.0-20181030221726-6c7e314b6563/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= golang.org/x/tools v0.0.0-20190114222345-bf090417da8b/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= golang.org/x/tools v0.0.0-20190226205152-f727befe758c/go.mod h1:9Yl7xja0Znq3iFh3HoIrodX9oNMXvdceNzlUR8zjMvY= golang.org/x/tools v0.0.0-20190311212946-11955173bddd/go.mod h1:LCzVGOaR6xXOjkQ3onu1FJEFr0SW1gC7cKk1uF8kGRs= golang.org/x/tools v0.0.0-20190524140312-2c0ae7006135/go.mod h1:RgjU9mgBXZiqYHBnxXauZ1Gv1EHHAz9KjViQ78xBX0Q= -golang.org/x/tools v0.0.0-20190606124116-d0a3d012864b h1:mSUCVIwDx4hfXJfWsOPfdzEHxzb2Xjl6BQ8YgPnazQA= golang.org/x/tools v0.0.0-20190606124116-d0a3d012864b/go.mod h1:/rFqwRUd4F7ZHNgwSSTFct+R/Kf4OFW1sUzUTQQTgfc= -golang.org/x/tools v0.0.0-20190911022129-16c5e0f7d110 h1:6S6bidS7O4yAwA5ORRbRIjvNQ9tGbLd5e+LRIaTeVDQ= -golang.org/x/tools v0.0.0-20190911022129-16c5e0f7d110/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= +golang.org/x/tools v0.0.0-20190621195816-6e04913cbbac/go.mod h1:/rFqwRUd4F7ZHNgwSSTFct+R/Kf4OFW1sUzUTQQTgfc= +golang.org/x/tools v0.0.0-20191029041327-9cc4af7d6b2c/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= +golang.org/x/tools v0.0.0-20191029190741-b9c20aec41a5/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= +golang.org/x/tools v0.0.0-20191107010934-f79515f33823/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= +golang.org/x/tools v0.0.0-20191125144606-a911d9008d1f/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= +golang.org/x/tools v0.0.0-20200119215504-eb0d8dd85bcc h1:ZA7KFRdqWZkBr0/YbHm1h08vDJ5gQdjVG/8L153z5c4= +golang.org/x/tools v0.0.0-20200119215504-eb0d8dd85bcc/go.mod h1:TB2adYChydJhpapKDTa4BR/hXlZSLoq2Wpct/0txZ28= golang.org/x/xerrors v0.0.0-20190717185122-a985d3407aa7/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= +golang.org/x/xerrors v0.0.0-20191011141410-1b5146add898/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= 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.6.2 h1:j8RI1yW0SkI+paT6uGwMlrMI/6zwYA6/CFil8rxOzGI= @@ -345,34 +475,51 @@ google.golang.org/appengine v1.6.2/go.mod h1:i06prIuMbXzDqacNJfV5OdTW448YApPu5ww google.golang.org/genproto v0.0.0-20180608181217-32ee49c4dd80/go.mod h1:JiN7NxoALGmiZfu7CAH4rXhgtRTLTxftemlI0sWmxmc= google.golang.org/genproto v0.0.0-20180817151627-c66870c02cf8/go.mod h1:JiN7NxoALGmiZfu7CAH4rXhgtRTLTxftemlI0sWmxmc= google.golang.org/genproto v0.0.0-20181004005441-af9cb2a35e7f/go.mod h1:JiN7NxoALGmiZfu7CAH4rXhgtRTLTxftemlI0sWmxmc= -google.golang.org/genproto v0.0.0-20190905072037-92dd089d5514 h1:oFSK4421fpCKRrpzIpybyBVWyht05NegY9+L/3TLAZs= +google.golang.org/genproto v0.0.0-20190819201941-24fa4b261c55/go.mod h1:DMBHOl98Agz4BDEuKkezgsaosCRResVns1a3J2ZsMNc= google.golang.org/genproto v0.0.0-20190905072037-92dd089d5514/go.mod h1:DMBHOl98Agz4BDEuKkezgsaosCRResVns1a3J2ZsMNc= +google.golang.org/genproto v0.0.0-20190927181202-20e1ac93f88c/go.mod h1:IbNlFCBrqXvoKpeg0TB2l7cyZUmoaFKYIwrEpbDKLA8= +google.golang.org/genproto v0.0.0-20200113173426-e1de0a7b01eb h1:EsMpWw4S8DM2QYm5idfmmWsv2N57GWi2tx3p96Gpja4= +google.golang.org/genproto v0.0.0-20200113173426-e1de0a7b01eb/go.mod h1:n3cpQtvxv34hfy77yVDNjmbRyujviMdxYliBSkLhpCc= google.golang.org/grpc v0.0.0-20180607172857-7a6a684ca69e/go.mod h1:yo6s7OP7yaDglbqo1J04qKzAhqBH6lvTonzMVmEdcZw= google.golang.org/grpc v1.14.0/go.mod h1:yo6s7OP7yaDglbqo1J04qKzAhqBH6lvTonzMVmEdcZw= google.golang.org/grpc v1.19.0/go.mod h1:mqu4LbDTu4XGKhr4mRzUsmM4RtVoemTSY81AxZiDr8c= -google.golang.org/grpc v1.23.0 h1:AzbTB6ux+okLTzP8Ru1Xs41C303zdcfEht7MQnYJt5A= google.golang.org/grpc v1.23.0/go.mod h1:Y5yQAOtifL1yxbo5wqy6BxZv8vAUGQwXBOALyacEbxg= +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/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= gopkg.in/airbrake/gobrake.v2 v2.0.9/go.mod h1:/h5ZAUhDkGaJfjzjKLSjv6zCL6O0LLBxU4K+aSYdM/U= gopkg.in/alecthomas/gometalinter.v2 v2.0.12/go.mod h1:NDRytsqEZyolNuAgTzJkZMkSQM7FIKyzVzGhjB/qfYo= +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/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/cheggaaa/pb.v1 v1.0.25/go.mod h1:V/YB90LKu/1FcN3WVnfiiE5oMCibMjukxqG/qStrOgw= +gopkg.in/errgo.v2 v2.1.0/go.mod h1:hNsd1EY+bozCKY1Ytp96fpM3vjJbqLJn88ws8XvfDNI= gopkg.in/fsnotify.v1 v1.4.7 h1:xOHLXZwVvI9hhs+cLKq5+I5onOuwQLhQwiu63xxlHs4= gopkg.in/fsnotify.v1 v1.4.7/go.mod h1:Tz8NjZHkW78fSQdbUxIjBTcgA1z1m8ZHf0WmKUhAMys= gopkg.in/gemnasium/logrus-airbrake-hook.v2 v2.1.2/go.mod h1:Xk6kEKp8OKb+X14hQBKWaSkCsqBpgog8nAV2xsGOxlo= gopkg.in/natefinch/lumberjack.v2 v2.0.0 h1:1Lc07Kr7qY4U2YPouBjpCLxpiyxIVoxqXgkXLknAOE8= gopkg.in/natefinch/lumberjack.v2 v2.0.0/go.mod h1:l0ndWWf7gzL7RNwBG7wST/UCcT4T24xpD6X8LsfU/+k= +gopkg.in/resty.v1 v1.12.0/go.mod h1:mDo4pnntr5jdWRML875a/NmxYqAlA73dVijT2AXvQQo= gopkg.in/tomb.v1 v1.0.0-20141024135613-dd632973f1e7 h1:uRGJdciOHaEIrze2W8Q3AKkepLTh2hOroT7a+7czfdQ= gopkg.in/tomb.v1 v1.0.0-20141024135613-dd632973f1e7/go.mod h1:dt/ZhP58zS4L8KSrWDmTeBkI65Dw0HsyUHuEVlX15mw= +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.7 h1:VUgggvou5XRW9mHwD/yXxIYSMtY0zoKQf/v226p2nyo= +gopkg.in/yaml.v2 v2.2.7/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= honnef.co/go/tools v0.0.0-20190102054323-c2f93a96b099/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= +sigs.k8s.io/yaml v1.1.0 h1:4A07+ZFc2wgJwo8YNlQpr1rVlgUDlxXHhPJciaPY5gs= +sigs.k8s.io/yaml v1.1.0/go.mod h1:UJmg0vDUVViEyp3mgSv9WPwZCDxu4rQW1olrI1uml+o= 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-data v0.0.0-20151005221446-73f23eafcf67 h1:e1sMhtVq9AfcEy8AXNb8eSg6gbzfdpYhoNqnPJa+GzI= diff --git a/pkg/etcd/etcd.go b/pkg/etcd/etcd.go index 8f6db75fdd..45240a820c 100644 --- a/pkg/etcd/etcd.go +++ b/pkg/etcd/etcd.go @@ -18,9 +18,9 @@ import ( "strconv" "time" - "github.com/coreos/etcd/embed" "github.com/phayes/freeport" "github.com/pingcap/errors" + "go.etcd.io/etcd/embed" ) // getFreeListenURLs get free ports and localhost as url. @@ -54,6 +54,7 @@ func SetupEmbedEtcd(dir string) (clientURL *url.URL, e *embed.Etcd, err error) { } cfg.LPUrls = []url.URL{*urls[0]} cfg.LCUrls = []url.URL{*urls[1]} + cfg.Logger = "zap" clientURL = urls[1] e, err = embed.StartEtcd(cfg) diff --git a/pkg/etcd/etcd_test.go b/pkg/etcd/etcd_test.go index 57052d9242..bbf982bfcb 100644 --- a/pkg/etcd/etcd_test.go +++ b/pkg/etcd/etcd_test.go @@ -19,9 +19,9 @@ import ( "testing" "time" - "github.com/coreos/etcd/clientv3" - "github.com/coreos/etcd/embed" "github.com/pingcap/check" + "go.etcd.io/etcd/clientv3" + "go.etcd.io/etcd/embed" ) func Test(t *testing.T) { check.TestingT(t) } diff --git a/tests/util/db.go b/tests/util/db.go index 4284e169ca..860dcde9dc 100644 --- a/tests/util/db.go +++ b/tests/util/db.go @@ -104,6 +104,7 @@ func CheckSyncState(sourceDB, targetDB *sql.DB, schema string) bool { SourceTables: []*diff.TableInstance{sourceTableInstance}, TargetTable: targetTableInstance, UseChecksum: true, + CpDB: targetDB, } structEqual, dataEqual, err := tableDiff.Equal(context.Background(), func(sql string) error { log.Print(sql)