diff --git a/.gitignore b/.gitignore index 2da0ef7a133..d73deeda7b3 100644 --- a/.gitignore +++ b/.gitignore @@ -37,7 +37,6 @@ cscope.* cmd/cdc/cdc # Files generated when testing -vendor/ tiflash-config-preprocessed.toml # Files generated when running docker-compose diff --git a/Dockerfile b/Dockerfile index b04ec56945d..6364cd476d9 100644 --- a/Dockerfile +++ b/Dockerfile @@ -2,10 +2,11 @@ FROM golang:1.14-alpine as builder RUN apk add --no-cache git make bash WORKDIR /go/src/github.com/pingcap/ticdc COPY . . +ENV CDC_ENABLE_VENDOR=1 RUN make FROM alpine:3.12 -RUN apk add --no-cache tzdata bash curl +RUN apk add --no-cache tzdata bash curl socat COPY --from=builder /go/src/github.com/pingcap/ticdc/bin/cdc /cdc EXPOSE 8300 CMD [ "/cdc" ] diff --git a/Makefile b/Makefile index 75b1fb54adb..3bed941d464 100644 --- a/Makefile +++ b/Makefile @@ -14,7 +14,11 @@ TEST_DIR := /tmp/tidb_cdc_test SHELL := /usr/bin/env bash GO := GO111MODULE=on go -GOBUILD := CGO_ENABLED=0 $(GO) build $(BUILD_FLAG) -trimpath +ifeq (${CDC_ENABLE_VENDOR}, 1) +GOVENDORFLAG := -mod=vendor +endif + +GOBUILD := CGO_ENABLED=0 $(GO) build $(BUILD_FLAG) -trimpath $(GOVENDORFLAG) ifeq ($(GOVERSION114), 1) GOTEST := CGO_ENABLED=1 $(GO) test -p 3 --race -gcflags=all=-d=checkptr=0 else @@ -24,7 +28,7 @@ endif ARCH := "`uname -s`" LINUX := "Linux" MAC := "Darwin" -PACKAGE_LIST := go list ./...| grep -vE 'vendor|proto|ticdc\/tests' +PACKAGE_LIST := go list ./...| grep -vE 'vendor|proto|ticdc\/tests|integration' PACKAGES := $$($(PACKAGE_LIST)) PACKAGE_DIRECTORIES := $(PACKAGE_LIST) | sed 's|github.com/pingcap/$(PROJECT)/||' FILES := $$(find . -name '*.go' -type f | grep -vE 'vendor') @@ -84,8 +88,10 @@ check_third_party_binary: @which bin/go-ycsb @which bin/etcdctl @which bin/jq + @which bin/minio integration_test_build: check_failpoint_ctl + ./scripts/fix_lib_zstd.sh $(FAILPOINT_ENABLE) $(GOTEST) -c -cover -covemode=atomic \ -coverpkg=github.com/pingcap/ticdc/... \ @@ -124,7 +130,7 @@ tidy: check: check-copyright fmt lint check-static tidy coverage: - GO111MODULE=off go get github.com/zhouqiang-cl/gocovmerge + GO111MODULE=off go get github.com/wadey/gocovmerge gocovmerge "$(TEST_DIR)"/cov.* | grep -vE ".*.pb.go|$(CDC_PKG)/cdc/kv/testing.go|.*.__failpoint_binding__.go" > "$(TEST_DIR)/all_cov.out" grep -vE ".*.pb.go|$(CDC_PKG)/cdc/kv/testing.go|.*.__failpoint_binding__.go" "$(TEST_DIR)/cov.unit.out" > "$(TEST_DIR)/unit_cov.out" ifeq ("$(JenkinsCI)", "1") diff --git a/cdc/changefeed.go b/cdc/changefeed.go index 7c000c4e717..7239857b0d4 100644 --- a/cdc/changefeed.go +++ b/cdc/changefeed.go @@ -345,7 +345,7 @@ func (c *changeFeed) balanceOrphanTables(ctx context.Context, captures map[model } for captureID, funcs := range updateFuncs { - newStatus, err := c.etcdCli.AtomicPutTaskStatus(ctx, c.id, captureID, funcs...) + newStatus, _, err := c.etcdCli.AtomicPutTaskStatus(ctx, c.id, captureID, funcs...) if err != nil { return errors.Trace(err) } @@ -365,7 +365,7 @@ func (c *changeFeed) balanceOrphanTables(ctx context.Context, captures map[model func (c *changeFeed) updateTaskStatus(ctx context.Context, taskStatus map[model.CaptureID]*model.TaskStatus) error { for captureID, status := range taskStatus { - newStatus, err := c.etcdCli.AtomicPutTaskStatus(ctx, c.id, captureID, func(modRevision int64, taskStatus *model.TaskStatus) (bool, error) { + newStatus, _, err := c.etcdCli.AtomicPutTaskStatus(ctx, c.id, captureID, func(modRevision int64, taskStatus *model.TaskStatus) (bool, error) { if taskStatus.SomeOperationsUnapplied() { log.Error("unexpected task status, there are operations unapplied in this status", zap.Any("status", taskStatus)) return false, errors.Errorf("waiting to processor handle the operation finished time out") @@ -789,7 +789,7 @@ func (c *changeFeed) calcResolvedTs(ctx context.Context) error { for len(c.ddlJobHistory) > 0 && c.ddlJobHistory[0].BinlogInfo.FinishedTS <= c.ddlExecutedTs { c.ddlJobHistory = c.ddlJobHistory[1:] } - if len(c.ddlJobHistory) > 0 && minResolvedTs > c.ddlJobHistory[0].BinlogInfo.FinishedTS { + if len(c.ddlJobHistory) > 0 && minResolvedTs >= c.ddlJobHistory[0].BinlogInfo.FinishedTS { minResolvedTs = c.ddlJobHistory[0].BinlogInfo.FinishedTS c.ddlState = model.ChangeFeedWaitToExecDDL } diff --git a/cdc/entry/codec.go b/cdc/entry/codec.go index e552d5e0c26..504bcb7f3e6 100644 --- a/cdc/entry/codec.go +++ b/cdc/entry/codec.go @@ -22,7 +22,6 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/parser/mysql" - "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/codec" "github.com/pingcap/tidb/util/rowcodec" @@ -186,13 +185,6 @@ 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 *model.TableInfo, tz *time.Location) (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) { @@ -261,8 +253,8 @@ func decodeRowV1(b []byte, recordID int64, tableInfo *model.TableInfo, tz *time. // https://github.com/pingcap/tidb/blob/master/docs/design/2018-07-19-row-format.md func decodeRowV2(data []byte, recordID int64, tableInfo *model.TableInfo, tz *time.Location) (map[int64]types.Datum, error) { handleColID, reqCols := tableInfo.GetRowColInfos() - decoder := rowcodec.NewDatumMapDecoder(reqCols, []int64{handleColID}, tz) - return decoder.DecodeToDatumMap(data, kv.IntHandle(recordID), nil) + decoder := rowcodec.NewDatumMapDecoder(reqCols, handleColID, tz) + return decoder.DecodeToDatumMap(data, recordID, nil) } // unflatten converts a raw datum to a column datum. diff --git a/cdc/entry/codec_test.go b/cdc/entry/codec_test.go index 84183aa715f..757dda484ec 100644 --- a/cdc/entry/codec_test.go +++ b/cdc/entry/codec_test.go @@ -17,7 +17,6 @@ import ( "testing" "github.com/pingcap/check" - "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/tablecodec" "github.com/pingcap/tidb/util/codec" ) @@ -31,7 +30,7 @@ var _ = check.Suite(&codecSuite{}) func (s *codecSuite) TestDecodeRecordKey(c *check.C) { recordPrefix := tablecodec.GenTableRecordPrefix(12345) - key := tablecodec.EncodeRecordKey(recordPrefix, kv.IntHandle(67890)) + key := tablecodec.EncodeRecordKey(recordPrefix, 67890) key, tableID, err := decodeTableID(key) c.Assert(err, check.IsNil) c.Assert(tableID, check.Equals, int64(12345)) diff --git a/cdc/entry/mounter.go b/cdc/entry/mounter.go index e1f079499fb..05493def461 100644 --- a/cdc/entry/mounter.go +++ b/cdc/entry/mounter.go @@ -19,7 +19,6 @@ import ( "encoding/binary" "encoding/json" "math/rand" - "strings" "time" "github.com/pingcap/errors" @@ -32,7 +31,6 @@ import ( "golang.org/x/sync/errgroup" "github.com/pingcap/ticdc/cdc/model" - "github.com/pingcap/ticdc/pkg/quotes" "github.com/pingcap/ticdc/pkg/util" ) @@ -54,6 +52,12 @@ type rowKVEntry struct { baseKVEntry Row map[int64]types.Datum PreRow map[int64]types.Datum + + // In some cases, row data may exist but not contain any Datum, + // use this RowExist/PreRowExist variable to distinguish between row data that does not exist + // or row data that does not contain any Datum. + RowExist bool + PreRowExist bool } type indexKVEntry struct { @@ -253,7 +257,7 @@ func (m *mounterImpl) unmarshalAndMountRowChanged(ctx context.Context, raw *mode if rowKV == nil { return nil, nil } - return m.mountRowKVEntry(tableInfo, rowKV) + return m.mountRowKVEntry(tableInfo, rowKV, raw.ApproximateSize()) case bytes.HasPrefix(key, indexPrefix): indexKV, err := m.unmarshalIndexKVEntry(key, raw.Value, raw.OldValue, baseInfo) if err != nil { @@ -262,7 +266,7 @@ func (m *mounterImpl) unmarshalAndMountRowChanged(ctx context.Context, raw *mode if indexKV == nil { return nil, nil } - return m.mountIndexKVEntry(tableInfo, indexKV) + return m.mountIndexKVEntry(tableInfo, indexKV, raw.ApproximateSize()) } return nil, nil }() @@ -281,22 +285,42 @@ func (m *mounterImpl) unmarshalRowKVEntry(tableInfo *model.TableInfo, restKey [] if len(key) != 0 { return nil, errors.New("invalid record key") } - row, err := decodeRow(rawValue, recordID, tableInfo, m.tz) + decodeRow := func(rawColValue []byte) (map[int64]types.Datum, bool, error) { + if len(rawColValue) == 0 { + return nil, false, nil + } + row, err := decodeRow(rawColValue, recordID, tableInfo, m.tz) + if err != nil { + return nil, false, errors.Trace(err) + } + return row, true, nil + } + + row, rowExist, err := decodeRow(rawValue) + if err != nil { + return nil, errors.Trace(err) + } + preRow, preRowExist, err := decodeRow(rawOldValue) if err != nil { return nil, errors.Trace(err) } - var preRow map[int64]types.Datum - if rawOldValue != nil { - preRow, err = decodeRow(rawOldValue, recordID, tableInfo, m.tz) + + if base.Delete && !m.enableOldValue && tableInfo.PKIsHandle { + id, pkValue, err := fetchHandleValue(tableInfo, recordID) if err != nil { return nil, errors.Trace(err) } + preRow = map[int64]types.Datum{id: *pkValue} + preRowExist = true } + base.RecordID = recordID return &rowKVEntry{ baseKVEntry: base, Row: row, PreRow: preRow, + RowExist: rowExist, + PreRowExist: preRowExist, }, nil } @@ -365,62 +389,37 @@ func UnmarshalDDL(raw *model.RawKVEntry) (*timodel.Job, error) { return job, nil } -func datum2Column(tableInfo *model.TableInfo, datums map[int64]types.Datum, fillWithDefaultValue bool) (map[string]*model.Column, error) { - estimateLen := len(datums) - if fillWithDefaultValue { - estimateLen = len(tableInfo.Columns) - } - cols := make(map[string]*model.Column, estimateLen) - for index, colValue := range datums { - colInfo, exist := tableInfo.GetColumnInfo(index) - if !exist { - return nil, errors.NotFoundf("column info, colID: %d", index) - } - if !tableInfo.IsColCDCVisible(colInfo) { +func datum2Column(tableInfo *model.TableInfo, datums map[int64]types.Datum, fillWithDefaultValue bool) ([]*model.Column, error) { + cols := make([]*model.Column, len(tableInfo.RowColumnsOffset)) + for _, colInfo := range tableInfo.Columns { + if !model.IsColCDCVisible(colInfo) { continue } colName := colInfo.Name.O - value, err := formatColVal(colValue, colInfo.Tp) - if err != nil { - return nil, errors.Trace(err) + colDatums, exist := datums[colInfo.ID] + var colValue interface{} + if exist { + var err error + colValue, err = formatColVal(colDatums, colInfo.Tp) + if err != nil { + return nil, errors.Trace(err) + } + } else if fillWithDefaultValue { + colValue = getDefaultOrZeroValue(colInfo) + } else { + continue } - col := &model.Column{ + cols[tableInfo.RowColumnsOffset[colInfo.ID]] = &model.Column{ + Name: colName, Type: colInfo.Tp, - Value: value, - Flag: transColumnFlag(colInfo), + Value: colValue, + Flag: tableInfo.ColumnsFlag[colInfo.ID], } - if tableInfo.IsColumnUnique(colInfo.ID) { - whereHandle := true - col.WhereHandle = &whereHandle - } - cols[colName] = col - } - if fillWithDefaultValue { - for _, col := range tableInfo.Columns { - _, ok := cols[col.Name.O] - if !ok && tableInfo.IsColCDCVisible(col) { - column := &model.Column{ - Type: col.Tp, - Value: getDefaultOrZeroValue(col), - Flag: transColumnFlag(col), - } - if tableInfo.IsColumnUnique(col.ID) { - whereHandle := true - column.WhereHandle = &whereHandle - } - cols[col.Name.O] = column - } - } - } - - err := setHandleKeyFlag(tableInfo, cols) - if err != nil { - return nil, errors.Trace(err) } return cols, nil } -func (m *mounterImpl) mountRowKVEntry(tableInfo *model.TableInfo, row *rowKVEntry) (*model.RowChangedEvent, error) { +func (m *mounterImpl) mountRowKVEntry(tableInfo *model.TableInfo, row *rowKVEntry, dataSize int64) (*model.RowChangedEvent, error) { // if m.enableOldValue == true, go into this function // if m.enableNewValue == false and row.Delete == false, go into this function // if m.enableNewValue == false and row.Delete == true and tableInfo.PKIsHandle = true, go into this function @@ -431,30 +430,22 @@ func (m *mounterImpl) mountRowKVEntry(tableInfo *model.TableInfo, row *rowKVEntr var err error // Decode previous columns. - var preCols map[string]*model.Column - if len(row.PreRow) != 0 { + var preCols []*model.Column + if row.PreRowExist { // FIXME(leoppro): using pre table info to mounter pre column datum // the pre column and current column in one event may using different table info - preCols, err = datum2Column(tableInfo, row.PreRow, true) + preCols, err = datum2Column(tableInfo, row.PreRow, m.enableOldValue) if err != nil { return nil, errors.Trace(err) } } - var cols map[string]*model.Column - oldValueDisabledAndRowIsDelete := !m.enableOldValue && row.Delete - cols, err = datum2Column(tableInfo, row.Row, !oldValueDisabledAndRowIsDelete) - if err != nil { - return nil, errors.Trace(err) - } - if oldValueDisabledAndRowIsDelete { - preCols = cols - cols = nil - } - - var partitionID int64 - if tableInfo.GetPartitionInfo() != nil { - partitionID = row.PhysicalTableID + var cols []*model.Column + if row.RowExist { + cols, err = datum2Column(tableInfo, row.Row, true) + if err != nil { + return nil, errors.Trace(err) + } } schemaName := tableInfo.TableName.Schema @@ -465,48 +456,19 @@ func (m *mounterImpl) mountRowKVEntry(tableInfo *model.TableInfo, row *rowKVEntr RowID: row.RecordID, TableInfoVersion: tableInfo.TableInfoVersion, Table: &model.TableName{ - Schema: schemaName, - Table: tableName, - Partition: partitionID, + Schema: schemaName, + Table: tableName, + TableID: row.PhysicalTableID, + IsPartition: tableInfo.GetPartitionInfo() != nil, }, - IndieMarkCol: tableInfo.IndieMarkCol, - Delete: row.Delete, - Columns: cols, - PreColumns: preCols, - // FIXME(leoppor): Correctness of conflict detection with old values - Keys: genMultipleKeys(tableInfo.TableInfo, preCols, cols, quotes.QuoteSchema(schemaName, tableName)), + Columns: cols, + PreColumns: preCols, + IndexColumns: tableInfo.IndexColumnsOffset, + ApproximateSize: dataSize, }, nil } -func setHandleKeyFlag(tableInfo *model.TableInfo, colValues map[string]*model.Column) error { - switch tableInfo.HandleIndexID { - case model.HandleIndexTableIneligible: - log.Fatal("this table is not eligible", zap.Int64("tableID", tableInfo.ID)) - case model.HandleIndexPKIsHandle: - // pk is handle - if !tableInfo.PKIsHandle { - log.Fatal("the pk of this table is not handle", zap.Int64("tableID", tableInfo.ID)) - } - for _, colInfo := range tableInfo.Columns { - if mysql.HasPriKeyFlag(colInfo.Flag) { - colValues[colInfo.Name.O].Flag.SetIsHandleKey() - break - } - } - default: - handleIndexInfo, exist := tableInfo.GetIndexInfo(tableInfo.HandleIndexID) - if !exist { - return errors.NotFoundf("handle index info(%d) in table(%d)", tableInfo.HandleIndexID, tableInfo.ID) - } - for _, colInfo := range handleIndexInfo.Columns { - colName := tableInfo.Columns[colInfo.Offset].Name.O - colValues[colName].Flag.SetIsHandleKey() - } - } - return nil -} - -func (m *mounterImpl) mountIndexKVEntry(tableInfo *model.TableInfo, idx *indexKVEntry) (*model.RowChangedEvent, error) { +func (m *mounterImpl) mountIndexKVEntry(tableInfo *model.TableInfo, idx *indexKVEntry, dataSize int64) (*model.RowChangedEvent, error) { // skip set index KV if !idx.Delete || m.enableOldValue { return nil, nil @@ -531,39 +493,42 @@ func (m *mounterImpl) mountIndexKVEntry(tableInfo *model.TableInfo, idx *indexKV return nil, errors.Trace(err) } - preCols := make(map[string]*model.Column, len(idx.IndexValue)) + preCols := make([]*model.Column, len(tableInfo.RowColumnsOffset)) for i, idxCol := range indexInfo.Columns { - value, err := formatColVal(idx.IndexValue[i], tableInfo.Columns[idxCol.Offset].Tp) + colInfo := tableInfo.Columns[idxCol.Offset] + value, err := formatColVal(idx.IndexValue[i], colInfo.Tp) if err != nil { return nil, errors.Trace(err) } - whereHandle := true - col := &model.Column{ - Type: tableInfo.Columns[idxCol.Offset].Tp, - WhereHandle: &whereHandle, - Value: value, - Flag: transColumnFlag(tableInfo.Columns[idxCol.Offset]), + preCols[tableInfo.RowColumnsOffset[colInfo.ID]] = &model.Column{ + Name: colInfo.Name.O, + Type: colInfo.Tp, + Value: value, + Flag: tableInfo.ColumnsFlag[colInfo.ID], } - col.Flag.SetIsHandleKey() - preCols[idxCol.Name.O] = col } return &model.RowChangedEvent{ StartTs: idx.StartTs, CommitTs: idx.CRTs, RowID: idx.RecordID, Table: &model.TableName{ - Schema: tableInfo.TableName.Schema, - Table: tableInfo.TableName.Table, + Schema: tableInfo.TableName.Schema, + Table: tableInfo.TableName.Table, + TableID: idx.PhysicalTableID, + IsPartition: tableInfo.GetPartitionInfo() != nil, }, - IndieMarkCol: tableInfo.IndieMarkCol, - Delete: true, - PreColumns: preCols, - Keys: genMultipleKeys(tableInfo.TableInfo, preCols, nil, quotes.QuoteSchema(tableInfo.TableName.Schema, tableInfo.TableName.Table)), + PreColumns: preCols, + IndexColumns: tableInfo.IndexColumnsOffset, + ApproximateSize: dataSize, }, nil } -func formatColVal(datum types.Datum, tp byte) (interface{}, error) { +var emptyBytes = make([]byte, 0) +func formatColVal(datum types.Datum, tp byte) (interface{}, error) { + if datum.IsNull() { + return nil, nil + } switch tp { case mysql.TypeDate, mysql.TypeDatetime, mysql.TypeNewDate, mysql.TypeTimestamp: return datum.GetMysqlTime().String(), nil @@ -584,6 +549,12 @@ func formatColVal(datum types.Datum, tp byte) (interface{}, error) { case mysql.TypeBit: // Encode bits as integers to avoid pingcap/tidb#10988 (which also affects MySQL itself) return datum.GetBinaryLiteral().ToInt(nil) + case mysql.TypeString, mysql.TypeVarString, mysql.TypeVarchar: + b := datum.GetBytes() + if b == nil { + b = emptyBytes + } + return b, nil default: return datum.GetValue(), nil } @@ -602,12 +573,14 @@ func getDefaultOrZeroValue(col *timodel.ColumnInfo) interface{} { d := types.NewDatum(col.GetDefaultValue()) return d.GetValue() } - - if col.Tp == mysql.TypeEnum { + switch col.Tp { + case mysql.TypeEnum: // For enum type, if no default value and not null is set, // the default value is the first element of the enum list d := types.NewDatum(col.FieldType.Elems[0]) return d.GetValue() + case mysql.TypeString, mysql.TypeVarString, mysql.TypeVarchar: + return emptyBytes } d := table.GetZeroValue(col) @@ -635,113 +608,3 @@ func fetchHandleValue(tableInfo *model.TableInfo, recordID int64) (pkCoID int64, } return } - -func genMultipleKeys(ti *timodel.TableInfo, preCols, cols map[string]*model.Column, table string) []string { - estimateLen := len(ti.Indices) + 1 - if len(preCols) != 0 && len(cols) != 0 { - estimateLen *= 2 - } - multipleKeys := make([]string, 0, estimateLen) - buildKeys := func(colValues map[string]*model.Column) { - if len(colValues) == 0 { - return - } - if ti.PKIsHandle { - if pk := ti.GetPkColInfo(); pk != nil && !pk.IsGenerated() { - cols := []*timodel.ColumnInfo{pk} - - key := genKeyList(table, cols, colValues) - if len(key) > 0 { // ignore `null` value. - multipleKeys = append(multipleKeys, key) - } else { - log.L().Debug("ignore empty primary key", zap.String("table", table)) - } - } - } - - for _, indexCols := range ti.Indices { - if !indexCols.Unique { - continue - } - cols := getIndexColumns(ti.Columns, indexCols) - key := genKeyList(table, cols, colValues) - if len(key) > 0 { // ignore `null` value. - noGeneratedColumn := true - for _, col := range cols { - if col.IsGenerated() { - noGeneratedColumn = false - break - } - } - // If the index contain generated column, we can't use this key to detect conflict with other DML, - // Because such as insert can't specified the generated value. - if noGeneratedColumn { - multipleKeys = append(multipleKeys, key) - } - } else { - log.L().Debug("ignore empty index key", zap.String("table", table)) - } - } - } - buildKeys(preCols) - buildKeys(cols) - - if len(multipleKeys) == 0 { - // use table name as key if no key generated (no PK/UK), - // no concurrence for rows in the same table. - log.L().Debug("use table name as the key", zap.String("table", table)) - multipleKeys = append(multipleKeys, table) - } - - return multipleKeys -} - -func transColumnFlag(col *timodel.ColumnInfo) model.ColumnFlagType { - var flag model.ColumnFlagType - if col.Charset == "binary" { - flag.SetIsBinary() - } - if col.IsGenerated() { - flag.SetIsGeneratedColumn() - } - if mysql.HasPriKeyFlag(col.Flag) { - flag.SetIsPrimaryKey() - } - if mysql.HasUniKeyFlag(col.Flag) { - flag.SetIsUniqueKey() - } - if !mysql.HasNotNullFlag(col.Flag) { - flag.SetIsNullable() - } - if mysql.HasMultipleKeyFlag(col.Flag) { - flag.SetIsMultipleKey() - } - return flag -} - -func genKeyList(table string, columns []*timodel.ColumnInfo, values map[string]*model.Column) string { - var buf strings.Builder - for _, col := range columns { - val, ok := values[col.Name.O] - if !ok || val.Value == nil { - log.L().Debug("ignore null value", zap.String("column", col.Name.O), zap.String("table", table)) - continue // ignore `null` value. - } - buf.WriteString(model.ColumnValueString(val.Value)) - } - if buf.Len() == 0 { - log.L().Debug("all value are nil, no key generated", zap.String("table", table)) - return "" // all values are `null`. - } - - buf.WriteString(table) - return buf.String() -} - -func getIndexColumns(columns []*timodel.ColumnInfo, indexColumns *timodel.IndexInfo) []*timodel.ColumnInfo { - cols := make([]*timodel.ColumnInfo, 0, len(indexColumns.Columns)) - for _, column := range indexColumns.Columns { - cols = append(cols, columns[column.Offset]) - } - return cols -} diff --git a/cdc/entry/schema_storage.go b/cdc/entry/schema_storage.go index 26d4b162ce9..c8cc5caf77b 100644 --- a/cdc/entry/schema_storage.go +++ b/cdc/entry/schema_storage.go @@ -24,12 +24,13 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/log" timodel "github.com/pingcap/parser/model" - "github.com/pingcap/ticdc/cdc/model" - "github.com/pingcap/ticdc/pkg/filter" - "github.com/pingcap/ticdc/pkg/retry" timeta "github.com/pingcap/tidb/meta" "go.uber.org/zap" "go.uber.org/zap/zapcore" + + "github.com/pingcap/ticdc/cdc/model" + "github.com/pingcap/ticdc/pkg/filter" + "github.com/pingcap/ticdc/pkg/retry" ) // schemaSnapshot stores the source TiDB all schema information diff --git a/cdc/entry/schema_storage_test.go b/cdc/entry/schema_storage_test.go index c37833a202a..b72c98d60b1 100644 --- a/cdc/entry/schema_storage_test.go +++ b/cdc/entry/schema_storage_test.go @@ -26,7 +26,6 @@ import ( "github.com/pingcap/ticdc/cdc/model" "github.com/pingcap/tidb/session" "github.com/pingcap/tidb/store/mockstore" - "github.com/pingcap/tidb/store/mockstore/cluster" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/testkit" ) @@ -647,11 +646,7 @@ func (t *schemaSuite) TestMultiVersionStorage(c *C) { } func (t *schemaSuite) TestCreateSnapFromMeta(c *C) { - store, err := mockstore.NewMockStore( - mockstore.WithClusterInspector(func(c cluster.Cluster) { - mockstore.BootstrapWithSingleStore(c) - }), - ) + store, err := mockstore.NewMockTikvStore() c.Assert(err, IsNil) session.SetSchemaLease(0) diff --git a/cdc/http_handler.go b/cdc/http_handler.go index 380a855c8ca..70383ba113a 100644 --- a/cdc/http_handler.go +++ b/cdc/http_handler.go @@ -33,6 +33,8 @@ const ( APIOpVarTargetCaptureID = "target-cp-id" // APIOpVarTableID is the key of table ID in HTTP API APIOpVarTableID = "table-id" + // APIOpForceRemoveChangefeed is used when remove a changefeed + APIOpForceRemoveChangefeed = "force-remove" ) type commonResp struct { @@ -113,9 +115,20 @@ func (s *Server) handleChangefeedAdmin(w http.ResponseWriter, req *http.Request) writeError(w, http.StatusBadRequest, errors.Errorf("invalid admin job type: %s", typeStr)) return } + opts := &model.AdminJobOption{} + if forceRemoveStr := req.Form.Get(APIOpForceRemoveChangefeed); forceRemoveStr != "" { + forceRemoveOpt, err := strconv.ParseBool(forceRemoveStr) + if err != nil { + writeError(w, http.StatusBadRequest, + errors.Errorf("invalid force remove option: %s", forceRemoveStr)) + return + } + opts.ForceRemove = forceRemoveOpt + } job := model.AdminJob{ CfID: req.Form.Get(APIOpVarChangefeedID), Type: model.AdminJobType(typ), + Opts: opts, } err = s.owner.EnqueueJob(job) handleOwnerResp(w, err) diff --git a/cdc/kv/client.go b/cdc/kv/client.go index 0f0e4315033..52833d5b112 100644 --- a/cdc/kv/client.go +++ b/cdc/kv/client.go @@ -237,6 +237,32 @@ func (a *connArray) Close() { } } +type regionEventFeedLimiters struct { + sync.Mutex + // TODO replace with a LRU cache. + limiters map[uint64]*rate.Limiter +} + +var defaultRegionEventFeedLimiters *regionEventFeedLimiters = ®ionEventFeedLimiters{ + limiters: make(map[uint64]*rate.Limiter), +} + +func (rl *regionEventFeedLimiters) getLimiter(regionID uint64) *rate.Limiter { + var limiter *rate.Limiter + var ok bool + + rl.Lock() + limiter, ok = rl.limiters[regionID] + if !ok { + // In most cases, region replica count is 3. + replicaCount := 3 + limiter = rate.NewLimiter(rate.Every(100*time.Millisecond), replicaCount) + rl.limiters[regionID] = limiter + } + rl.Unlock() + return limiter +} + // CDCClient to get events from TiKV type CDCClient struct { pd pd.Client @@ -251,6 +277,8 @@ type CDCClient struct { regionCache *tikv.RegionCache kvStorage tikv.Storage + + regionLimiters *regionEventFeedLimiters } // NewCDCClient creates a CDCClient instance @@ -270,6 +298,7 @@ func NewCDCClient(ctx context.Context, pd pd.Client, kvStorage tikv.Storage, cre }{ conns: make(map[string]*connArray), }, + regionLimiters: defaultRegionEventFeedLimiters, } return } @@ -301,6 +330,10 @@ func (c *CDCClient) getConn(ctx context.Context, addr string) (*grpc.ClientConn, return ca.Get(), nil } +func (c *CDCClient) getRegionLimiter(regionID uint64) *rate.Limiter { + return c.regionLimiters.getLimiter(regionID) +} + func (c *CDCClient) newStream(ctx context.Context, addr string, storeID uint64) (stream cdcpb.ChangeData_EventFeedClient, err error) { err = retry.Run(50*time.Millisecond, 3, func() error { conn, err := c.getConn(ctx, addr) @@ -612,11 +645,12 @@ MainLoop: extraOp = kvrpcpb.ExtraOp_ReadOldValue } + regionID := rpcCtx.Meta.GetId() req := &cdcpb.ChangeDataRequest{ Header: &cdcpb.Header{ ClusterId: s.client.clusterID, }, - RegionId: rpcCtx.Meta.GetId(), + RegionId: regionID, RequestId: requestID, RegionEpoch: rpcCtx.Meta.RegionEpoch, CheckpointTs: sri.ts, @@ -671,8 +705,9 @@ MainLoop: } streams[rpcCtx.Addr] = stream + limiter := s.client.getRegionLimiter(regionID) g.Go(func() error { - return s.receiveFromStream(ctx, g, rpcCtx.Addr, getStoreID(rpcCtx), stream, pendingRegions) + return s.receiveFromStream(ctx, g, rpcCtx.Addr, getStoreID(rpcCtx), stream, pendingRegions, limiter) }) } @@ -745,6 +780,7 @@ func needReloadRegion(failStoreIDs map[uint64]struct{}, rpcCtx *tikv.RPCContext) func (s *eventFeedSession) partialRegionFeed( ctx context.Context, state *regionFeedState, + limiter *rate.Limiter, ) error { receiver := state.regionEventCh defer func() { @@ -762,12 +798,6 @@ func (s *eventFeedSession) partialRegionFeed( }() ts := state.sri.ts - rl := rate.NewLimiter(0.1, 5) - - if !rl.Allow() { - return errors.New("partialRegionFeed exceeds rate limit") - } - maxTs, err := s.singleEventFeed(ctx, state.sri.verID.GetID(), state.sri.span, state.sri.ts, receiver) log.Debug("singleEventFeed quit") @@ -779,8 +809,9 @@ func (s *eventFeedSession) partialRegionFeed( ts = maxTs } + regionID := state.sri.verID.GetID() log.Info("EventFeed disconnected", - zap.Reflect("regionID", state.sri.verID.GetID()), + zap.Reflect("regionID", regionID), zap.Stringer("span", state.sri.span), zap.Uint64("checkpoint", ts), zap.String("error", err.Error())) @@ -789,6 +820,22 @@ func (s *eventFeedSession) partialRegionFeed( // We need to ensure when the error is handled, `isStopped` must be set. So set it before sending the error. state.markStopped() + + now := time.Now() + delay := limiter.ReserveN(now, 1).Delay() + if delay != 0 { + log.Info("EventFeed retry rate limited", + zap.Duration("delay", delay), zap.Reflect("regionID", regionID)) + t := time.NewTimer(delay) + defer t.Stop() + select { + case <-t.C: + // We can proceed. + case <-ctx.Done(): + return ctx.Err() + } + } + return s.onRegionFail(ctx, regionErrorInfo{ singleRegionInfo: state.sri, err: err, @@ -930,6 +977,7 @@ func (s *eventFeedSession) receiveFromStream( storeID uint64, stream cdcpb.ChangeData_EventFeedClient, pendingRegions *syncRegionFeedStateMap, + limiter *rate.Limiter, ) error { // Cancel the pending regions if the stream failed. Otherwise it will remain unhandled in the pendingRegions list // however not registered in the new reconnected stream. @@ -1035,7 +1083,7 @@ func (s *eventFeedSession) receiveFromStream( regionStates[event.RegionId] = state g.Go(func() error { - return s.partialRegionFeed(ctx, state) + return s.partialRegionFeed(ctx, state, limiter) }) } else if state.isStopped() { log.Warn("drop event due to region feed stopped", @@ -1194,11 +1242,12 @@ func (s *eventFeedSession) singleEventFeed( revent := &model.RegionFeedEvent{ Val: &model.RawKVEntry{ - OpType: opType, - Key: entry.Key, - Value: entry.GetValue(), - StartTs: entry.StartTs, - CRTs: entry.CommitTs, + OpType: opType, + Key: entry.Key, + Value: entry.GetValue(), + OldValue: entry.GetOldValue(), + StartTs: entry.StartTs, + CRTs: entry.CommitTs, }, } diff --git a/cdc/kv/client_test.go b/cdc/kv/client_test.go index d90dd92a15a..fc7d0608f66 100644 --- a/cdc/kv/client_test.go +++ b/cdc/kv/client_test.go @@ -43,9 +43,7 @@ type clientSuite struct { var _ = check.Suite(&clientSuite{}) func (s *clientSuite) TestNewClose(c *check.C) { - mvccStore, err := mocktikv.NewMVCCLevelDB("") - c.Assert(err, check.IsNil) - cluster := mocktikv.NewCluster(mvccStore) + cluster := mocktikv.NewCluster() pdCli := mocktikv.NewPDClient(cluster) cli, err := NewCDCClient(context.Background(), pdCli, nil, &security.Credential{}) @@ -134,7 +132,9 @@ func (s *etcdSuite) TestConnectOfflineTiKV(c *check.C) { wg.Wait() }() - rpcClient, cluster, pdClient, err := mocktikv.NewTiKVAndPDClient("") + cluster := mocktikv.NewCluster() + mvccStore := mocktikv.MustNewMVCCStore() + rpcClient, pdClient, err := mocktikv.NewTiKVAndPDClient(cluster, mvccStore, "") c.Assert(err, check.IsNil) pdClient = &mockPDClient{Client: pdClient, version: util.MinTiKVVersion.String()} kvStorage, err := tikv.NewTestTiKVStore(rpcClient, pdClient, nil, nil, 0) @@ -198,7 +198,9 @@ func (s *etcdSuite) TestConnectOfflineTiKV(c *check.C) { // TODO enable the test func (s *etcdSuite) TodoTestIncompatibleTiKV(c *check.C) { - rpcClient, cluster, pdClient, err := mocktikv.NewTiKVAndPDClient("") + cluster := mocktikv.NewCluster() + mvccStore := mocktikv.MustNewMVCCStore() + rpcClient, pdClient, err := mocktikv.NewTiKVAndPDClient(cluster, mvccStore, "") c.Assert(err, check.IsNil) pdClient = &mockPDClient{Client: pdClient, version: "v2.1.0" /* CDC is not compatible with 2.1.0 */} kvStorage, err := tikv.NewTestTiKVStore(rpcClient, pdClient, nil, nil, 0) diff --git a/cdc/kv/etcd.go b/cdc/kv/etcd.go index 3f47ea153d9..f2762ef356a 100644 --- a/cdc/kv/etcd.go +++ b/cdc/kv/etcd.go @@ -120,6 +120,17 @@ func (c CDCEtcdClient) ClearAllCDCInfo(ctx context.Context) error { return errors.Trace(err) } +// RevokeAllLeases revokes all leases passed from parameter +func (c CDCEtcdClient) RevokeAllLeases(ctx context.Context, leases map[string]int64) error { + for _, lease := range leases { + _, err := c.Client.Revoke(ctx, clientv3.LeaseID(lease)) + if err != nil { + return errors.Trace(err) + } + } + return nil +} + // GetChangeFeeds returns kv revision and a map mapping from changefeedID to changefeed detail mvccpb.KeyValue func (c CDCEtcdClient) GetChangeFeeds(ctx context.Context) (int64, map[string]*mvccpb.KeyValue, error) { key := GetEtcdKeyChangeFeedList() @@ -162,6 +173,29 @@ func (c CDCEtcdClient) DeleteChangeFeedInfo(ctx context.Context, id string) erro return errors.Trace(err) } +// GetAllChangeFeedStatus queries all changefeed job status +func (c CDCEtcdClient) GetAllChangeFeedStatus(ctx context.Context) (map[string]*model.ChangeFeedStatus, error) { + key := JobKeyPrefix + resp, err := c.Client.Get(ctx, key, clientv3.WithPrefix()) + if err != nil { + return nil, errors.Trace(err) + } + statuses := make(map[string]*model.ChangeFeedStatus, resp.Count) + for _, rawKv := range resp.Kvs { + changefeedID, err := model.ExtractKeySuffix(string(rawKv.Key)) + if err != nil { + return nil, err + } + status := &model.ChangeFeedStatus{} + err = status.Unmarshal(rawKv.Value) + if err != nil { + return nil, errors.Trace(err) + } + statuses[changefeedID] = status + } + return statuses, nil +} + // GetChangeFeedStatus queries the checkpointTs and resovledTs of a given changefeed func (c CDCEtcdClient) GetChangeFeedStatus(ctx context.Context, id string) (*model.ChangeFeedStatus, int64, error) { key := GetEtcdKeyJob(id) @@ -198,20 +232,41 @@ func (c CDCEtcdClient) GetCaptures(ctx context.Context) (int64, []*model.Capture return revision, infos, nil } +// GetCaptureLeases returns a map mapping from capture ID to its lease +func (c CDCEtcdClient) GetCaptureLeases(ctx context.Context) (map[string]int64, error) { + key := CaptureInfoKeyPrefix + + resp, err := c.Client.Get(ctx, key, clientv3.WithPrefix()) + if err != nil { + return nil, errors.Trace(err) + } + leases := make(map[string]int64, resp.Count) + for _, kv := range resp.Kvs { + captureID, err := model.ExtractKeySuffix(string(kv.Key)) + if err != nil { + return nil, err + } + leases[captureID] = kv.Lease + } + return leases, nil +} + // CreateChangefeedInfo creates a change feed info into etcd and fails if it is already exists. func (c CDCEtcdClient) CreateChangefeedInfo(ctx context.Context, info *model.ChangeFeedInfo, changeFeedID string) error { if err := model.ValidateChangefeedID(changeFeedID); err != nil { return err } - key := GetEtcdKeyChangeFeedInfo(changeFeedID) + infoKey := GetEtcdKeyChangeFeedInfo(changeFeedID) + jobKey := GetEtcdKeyJob(changeFeedID) value, err := info.Marshal() if err != nil { return errors.Trace(err) } resp, err := c.Client.Txn(ctx).If( - clientv3.Compare(clientv3.ModRevision(key), "=", 0), + clientv3.Compare(clientv3.ModRevision(infoKey), "=", 0), + clientv3.Compare(clientv3.ModRevision(jobKey), "=", 0), ).Then( - clientv3.OpPut(key, value), + clientv3.OpPut(infoKey, value), ).Commit() if err != nil { return errors.Trace(err) @@ -219,7 +274,7 @@ func (c CDCEtcdClient) CreateChangefeedInfo(ctx context.Context, info *model.Cha if !resp.Succeeded { log.Warn("changefeed already exists, ignore create changefeed", zap.String("changefeedid", changeFeedID)) - return errors.Annotatef(model.ErrChangeFeedAlreadyExists, "key: %s", key) + return errors.Trace(model.ErrChangeFeedAlreadyExists) } return errors.Trace(err) } @@ -520,8 +575,9 @@ func (c CDCEtcdClient) AtomicPutTaskStatus( changefeedID string, captureID string, updateFuncs ...UpdateTaskStatusFunc, -) (*model.TaskStatus, error) { +) (*model.TaskStatus, int64, error) { var status *model.TaskStatus + var newModRevision int64 err := retry.Run(100*time.Millisecond, 3, func() error { select { case <-ctx.Done(): @@ -570,12 +626,13 @@ func (c CDCEtcdClient) AtomicPutTaskStatus( log.Info("outdated table infos, ignore update taskStatus") return errors.Annotatef(model.ErrWriteTsConflict, "key: %s", key) } + newModRevision = resp.Header.GetRevision() return nil }) if err != nil { - return nil, errors.Trace(err) + return nil, newModRevision, errors.Trace(err) } - return status, nil + return status, newModRevision, nil } // GetTaskPosition queries task process from etcd, returns @@ -629,6 +686,16 @@ func (c CDCEtcdClient) DeleteTaskPosition(ctx context.Context, changefeedID stri return errors.Trace(err) } +// RemoveChangeFeedStatus removes changefeed job status from etcd +func (c CDCEtcdClient) RemoveChangeFeedStatus( + ctx context.Context, + changefeedID string, +) error { + key := GetEtcdKeyJob(changefeedID) + _, err := c.Client.Delete(ctx, key) + return errors.Trace(err) +} + // PutChangeFeedStatus puts changefeed synchronization status into etcd func (c CDCEtcdClient) PutChangeFeedStatus( ctx context.Context, diff --git a/cdc/kv/etcd_test.go b/cdc/kv/etcd_test.go index 4feb4e3ad44..a6dca4b905e 100644 --- a/cdc/kv/etcd_test.go +++ b/cdc/kv/etcd_test.go @@ -17,18 +17,19 @@ import ( "context" "fmt" "net/url" + "sort" "time" - "github.com/pingcap/log" - "go.uber.org/zap" - "github.com/pingcap/check" "github.com/pingcap/errors" + "github.com/pingcap/log" "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/clientv3/concurrency" "go.etcd.io/etcd/embed" + "go.uber.org/zap" "golang.org/x/sync/errgroup" ) @@ -253,6 +254,43 @@ func (s *etcdSuite) TestPutAllChangeFeedStatus(c *check.C) { } } +func (s etcdSuite) TestGetAllChangeFeedStatus(c *check.C) { + var ( + changefeeds = map[model.ChangeFeedID]*model.ChangeFeedStatus{ + "cf1": { + ResolvedTs: 100, + CheckpointTs: 90, + }, + "cf2": { + ResolvedTs: 100, + CheckpointTs: 70, + }, + } + ) + err := s.client.PutAllChangeFeedStatus(context.Background(), changefeeds) + c.Assert(err, check.IsNil) + statuses, err := s.client.GetAllChangeFeedStatus(context.Background()) + c.Assert(err, check.IsNil) + c.Assert(statuses, check.DeepEquals, changefeeds) +} + +func (s *etcdSuite) TestRemoveChangeFeedStatus(c *check.C) { + ctx := context.Background() + changefeedID := "test-remove-changefeed-status" + status := &model.ChangeFeedStatus{ + ResolvedTs: 1, + } + err := s.client.PutChangeFeedStatus(ctx, changefeedID, status) + c.Assert(err, check.IsNil) + status, _, err = s.client.GetChangeFeedStatus(ctx, changefeedID) + c.Assert(err, check.IsNil) + c.Assert(status, check.DeepEquals, status) + err = s.client.RemoveChangeFeedStatus(ctx, changefeedID) + c.Assert(err, check.IsNil) + _, _, err = s.client.GetChangeFeedStatus(ctx, changefeedID) + c.Assert(errors.Cause(err), check.Equals, model.ErrChangeFeedNotExists) +} + func (s *etcdSuite) TestSetChangeFeedStatusTTL(c *check.C) { ctx := context.Background() err := s.client.PutChangeFeedStatus(ctx, "test1", &model.ChangeFeedStatus{ @@ -354,3 +392,52 @@ func (s *etcdSuite) TestCreateChangefeed(c *check.C) { err = s.client.CreateChangefeedInfo(ctx, detail, "test-id") c.Assert(errors.Cause(err), check.Equals, model.ErrChangeFeedAlreadyExists) } + +type Captures []*model.CaptureInfo + +func (c Captures) Len() int { return len(c) } +func (c Captures) Less(i, j int) bool { return c[i].ID < c[j].ID } +func (c Captures) Swap(i, j int) { c[i], c[j] = c[j], c[i] } + +func (s *etcdSuite) TestGetAllCaptureLeases(c *check.C) { + ctx := context.Background() + testCases := []*model.CaptureInfo{ + { + ID: "a3f41a6a-3c31-44f4-aa27-344c1b8cd658", + AdvertiseAddr: "127.0.0.1:8301", + }, + { + ID: "cdb041d9-ccdd-480d-9975-e97d7adb1185", + AdvertiseAddr: "127.0.0.1:8302", + }, + { + ID: "e05e5d34-96ea-44af-812d-ca72aa19e1e5", + AdvertiseAddr: "127.0.0.1:8303", + }, + } + leases := make(map[string]int64) + + for _, cinfo := range testCases { + sess, err := concurrency.NewSession(s.client.Client, concurrency.WithTTL(10)) + c.Assert(err, check.IsNil) + err = s.client.PutCaptureInfo(ctx, cinfo, sess.Lease()) + c.Assert(err, check.IsNil) + leases[cinfo.ID] = int64(sess.Lease()) + } + + _, captures, err := s.client.GetCaptures(ctx) + c.Assert(err, check.IsNil) + c.Assert(captures, check.HasLen, len(testCases)) + sort.Sort(Captures(captures)) + c.Assert(captures, check.DeepEquals, testCases) + + queryLeases, err := s.client.GetCaptureLeases(ctx) + c.Assert(err, check.IsNil) + c.Check(queryLeases, check.DeepEquals, leases) + + err = s.client.RevokeAllLeases(ctx, leases) + c.Assert(err, check.IsNil) + queryLeases, err = s.client.GetCaptureLeases(ctx) + c.Assert(err, check.IsNil) + c.Check(queryLeases, check.DeepEquals, map[string]int64{}) +} diff --git a/cdc/kv/matcher.go b/cdc/kv/matcher.go index 1d60157f084..b7728ef587a 100644 --- a/cdc/kv/matcher.go +++ b/cdc/kv/matcher.go @@ -46,14 +46,15 @@ func newMatcher() *matcher { func (m *matcher) putPrewriteRow(row *cdcpb.Event_Row) { key := newMatchKey(row) value := row.GetValue() - // tikv may send a prewrite event with empty value + oldvalue := row.GetOldValue() + // tikv may send a prewrite event with empty value (txn heartbeat) // here we need to avoid the invalid prewrite event overwrite the value if _, exist := m.unmatchedValue[key]; exist && len(value) == 0 { return } m.unmatchedValue[key] = &pendingValue{ value: value, - oldValue: row.GetOldValue(), + oldValue: oldvalue, } } diff --git a/cdc/metrics_processor.go b/cdc/metrics_processor.go index c09b29d1a22..1a582fc5e2d 100644 --- a/cdc/metrics_processor.go +++ b/cdc/metrics_processor.go @@ -97,6 +97,14 @@ var ( Name: "exit_with_error_count", Help: "counter for processor exits with error", }, []string{"changefeed", "capture"}) + sinkFlushRowChangedDuration = prometheus.NewHistogramVec( + prometheus.HistogramOpts{ + Namespace: "ticdc", + Subsystem: "processor", + Name: "flush_event_duration_seconds", + Help: "Bucketed histogram of processing time (s) of flushing events in processor", + Buckets: prometheus.ExponentialBuckets(0.002 /* 2ms */, 2, 20), + }, []string{"changefeed", "capture"}) ) // initProcessorMetrics registers all metrics used in processor @@ -112,4 +120,5 @@ func initProcessorMetrics(registry *prometheus.Registry) { registry.MustRegister(tableOutputChanSizeGauge) registry.MustRegister(waitEventPrepareDuration) registry.MustRegister(processorErrorCounter) + registry.MustRegister(sinkFlushRowChangedDuration) } diff --git a/cdc/model/changefeed.go b/cdc/model/changefeed.go index d04692a07ee..b1447f8638b 100644 --- a/cdc/model/changefeed.go +++ b/cdc/model/changefeed.go @@ -21,9 +21,11 @@ import ( "time" "github.com/pingcap/errors" + "github.com/pingcap/log" "github.com/pingcap/ticdc/pkg/config" "github.com/pingcap/ticdc/pkg/cyclic/mark" "github.com/pingcap/tidb/store/tikv/oracle" + "go.uber.org/zap" ) // SortEngine is the sorter engine @@ -91,6 +93,28 @@ func ValidateChangefeedID(changefeedID string) error { return nil } +// String implements fmt.Stringer interface, but hide some sensitive information +func (info *ChangeFeedInfo) String() (str string) { + var err error + str, err = info.Marshal() + if err != nil { + log.Error("failed to marshal changefeed info", zap.Error(err)) + return + } + clone := new(ChangeFeedInfo) + err = clone.Unmarshal([]byte(str)) + if err != nil { + log.Error("failed to unmarshal changefeed info", zap.Error(err)) + return + } + clone.SinkURI = "***" + str, err = clone.Marshal() + if err != nil { + log.Error("failed to marshal changefeed info", zap.Error(err)) + } + return +} + // GetStartTs returns StartTs if it's specified or using the CreateTime of changefeed. func (info *ChangeFeedInfo) GetStartTs() uint64 { if info.StartTs > 0 { diff --git a/cdc/model/changefeed_test.go b/cdc/model/changefeed_test.go index 824f562635a..d19af65ef87 100644 --- a/cdc/model/changefeed_test.go +++ b/cdc/model/changefeed_test.go @@ -207,3 +207,12 @@ func (s *changefeedSuite) TestCheckErrorHistory(c *check.C) { c.Assert(needSave, check.IsFalse) c.Assert(canInit, check.IsFalse) } + +func (s *changefeedSuite) TestChangefeedInfoStringer(c *check.C) { + info := &ChangeFeedInfo{ + SinkURI: "blackhole://", + StartTs: 418881574869139457, + } + str := info.String() + c.Check(str, check.Matches, ".*sink-uri\":\"\\*\\*\\*\".*") +} diff --git a/cdc/model/kv.go b/cdc/model/kv.go index e2fd3af965b..3722e1b8cec 100644 --- a/cdc/model/kv.go +++ b/cdc/model/kv.go @@ -72,3 +72,8 @@ func (v *RawKVEntry) String() string { return fmt.Sprintf("OpType: %v, Key: %s, Value: %s, StartTs: %d, CRTs: %d", v.OpType, string(v.Key), string(v.Value), v.StartTs, v.CRTs) } + +// ApproximateSize calculate the approximate size of this event +func (v *RawKVEntry) ApproximateSize() int64 { + return int64(len(v.Key) + len(v.Value) + len(v.OldValue)) +} diff --git a/cdc/model/owner.go b/cdc/model/owner.go index 4c85e4bfbc0..20584160d32 100644 --- a/cdc/model/owner.go +++ b/cdc/model/owner.go @@ -24,10 +24,16 @@ import ( // AdminJobType represents for admin job type, both used in owner and processor type AdminJobType int +// AdminJobOption records addition options of an admin job +type AdminJobOption struct { + ForceRemove bool +} + // AdminJob holds an admin job type AdminJob struct { CfID string Type AdminJobType + Opts *AdminJobOption Error *RunningError } diff --git a/cdc/model/schema_storage.go b/cdc/model/schema_storage.go index 8b0912a39ee..ad7b0e9f645 100644 --- a/cdc/model/schema_storage.go +++ b/cdc/model/schema_storage.go @@ -16,6 +16,10 @@ package model import ( "fmt" + "github.com/pingcap/log" + + "go.uber.org/zap" + "github.com/pingcap/parser/model" "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/util/rowcodec" @@ -38,6 +42,11 @@ type TableInfo struct { indicesOffset map[int64]int uniqueColumns map[int64]struct{} + // It's a mapping from ColumnID to the offset of the columns in row changed events. + RowColumnsOffset map[int64]int + + ColumnsFlag map[int64]ColumnFlagType + // only for new row format decoder handleColID int64 @@ -47,10 +56,8 @@ type TableInfo struct { // HandleIndexTableIneligible(-2) : the table is not eligible HandleIndexID int64 - // if the table of this row only has one unique index(includes primary key), - // IndieMarkCol will be set to the name of the unique index - IndieMarkCol string - rowColInfos []rowcodec.ColInfo + IndexColumnsOffset [][]int + rowColInfos []rowcodec.ColInfo } // WrapTableInfo creates a TableInfo from a timodel.TableInfo @@ -63,26 +70,39 @@ func WrapTableInfo(schemaID int64, schemaName string, version uint64, info *mode columnsOffset: make(map[int64]int, len(info.Columns)), indicesOffset: make(map[int64]int, len(info.Indices)), uniqueColumns: make(map[int64]struct{}), + RowColumnsOffset: make(map[int64]int, len(info.Columns)), + ColumnsFlag: make(map[int64]ColumnFlagType, len(info.Columns)), handleColID: -1, HandleIndexID: HandleIndexTableIneligible, rowColInfos: make([]rowcodec.ColInfo, len(info.Columns)), } - uniqueIndexNum := 0 + rowColumnsCurrentOffset := 0 for i, col := range ti.Columns { ti.columnsOffset[col.ID] = i - isPK := (ti.PKIsHandle && mysql.HasPriKeyFlag(col.Flag)) || col.ID == model.ExtraHandleID - if isPK { - ti.handleColID = col.ID - ti.HandleIndexID = HandleIndexPKIsHandle - ti.uniqueColumns[col.ID] = struct{}{} - uniqueIndexNum++ + isPK := false + if IsColCDCVisible(col) { + ti.RowColumnsOffset[col.ID] = rowColumnsCurrentOffset + rowColumnsCurrentOffset++ + isPK = (ti.PKIsHandle && mysql.HasPriKeyFlag(col.Flag)) || col.ID == model.ExtraHandleID + if isPK { + // pk is handle + ti.handleColID = col.ID + ti.HandleIndexID = HandleIndexPKIsHandle + ti.uniqueColumns[col.ID] = struct{}{} + ti.IndexColumnsOffset = append(ti.IndexColumnsOffset, []int{ti.RowColumnsOffset[col.ID]}) + } } ti.rowColInfos[i] = rowcodec.ColInfo{ ID: col.ID, IsPKHandle: isPK, - Ft: rowcodec.FieldTypeFromModelColumn(col), + Tp: int32(col.Tp), + Flag: int32(col.Flag), + Flen: col.Flen, + Decimal: col.Decimal, + Elems: col.Elems, + Collate: col.Collate, } } @@ -94,20 +114,22 @@ func WrapTableInfo(schemaID int64, schemaName string, version uint64, info *mode } } if idx.Primary || idx.Unique { - uniqueIndexNum++ - } - } - - // this table has only one unique column - if uniqueIndexNum == 1 && len(ti.uniqueColumns) == 1 { - for col := range ti.uniqueColumns { - info, _ := ti.GetColumnInfo(col) - if !info.IsGenerated() { - ti.IndieMarkCol = info.Name.O + indexColOffset := make([]int, 0, len(idx.Columns)) + for _, idxCol := range idx.Columns { + colInfo := ti.Columns[idxCol.Offset] + if IsColCDCVisible(colInfo) { + indexColOffset = append(indexColOffset, ti.RowColumnsOffset[colInfo.ID]) + } + } + if len(indexColOffset) > 0 { + ti.IndexColumnsOffset = append(ti.IndexColumnsOffset, indexColOffset) } } } + ti.findHandleIndex() + ti.initColumnsFlag() + log.Debug("warpped table info", zap.Reflect("tableInfo", ti)) return ti } @@ -140,6 +162,64 @@ func (ti *TableInfo) findHandleIndex() { } } +func (ti *TableInfo) initColumnsFlag() { + for _, colInfo := range ti.Columns { + var flag ColumnFlagType + if colInfo.Charset == "binary" { + flag.SetIsBinary() + } + if colInfo.IsGenerated() { + flag.SetIsGeneratedColumn() + } + if mysql.HasPriKeyFlag(colInfo.Flag) { + flag.SetIsPrimaryKey() + if ti.HandleIndexID == HandleIndexPKIsHandle { + flag.SetIsHandleKey() + } + } + if mysql.HasUniKeyFlag(colInfo.Flag) { + flag.SetIsUniqueKey() + } + if !mysql.HasNotNullFlag(colInfo.Flag) { + flag.SetIsNullable() + } + if mysql.HasMultipleKeyFlag(colInfo.Flag) { + flag.SetIsMultipleKey() + } + if mysql.HasUnsignedFlag(colInfo.Flag) { + flag.SetIsUnsigned() + } + ti.ColumnsFlag[colInfo.ID] = flag + } + + // In TiDB, just as in MySQL, only the first column of an index can be marked as "multiple key" or "unique key", + // and only the first column of a unique index may be marked as "unique key". + // See https://dev.mysql.com/doc/refman/5.7/en/show-columns.html. + // Yet if an index has multiple columns, we would like to easily determine that all those columns are indexed, + // which is crucial for the completeness of the information we pass to the downstream. + // Therefore, instead of using the MySql standard, + // we made our own decision to mark all columns in an index with the appropriate flag(s). + for _, idxInfo := range ti.Indices { + for _, idxCol := range idxInfo.Columns { + colInfo := ti.Columns[idxCol.Offset] + flag := ti.ColumnsFlag[colInfo.ID] + if idxInfo.Primary { + flag.SetIsPrimaryKey() + } + if idxInfo.Unique { + flag.SetIsUniqueKey() + } + if len(idxInfo.Columns) > 1 { + flag.SetIsMultipleKey() + } + if idxInfo.ID == ti.HandleIndexID && ti.HandleIndexID >= 0 { + flag.SetIsHandleKey() + } + ti.ColumnsFlag[colInfo.ID] = flag + } + } +} + // GetColumnInfo returns the column info by ID func (ti *TableInfo) GetColumnInfo(colID int64) (info *model.ColumnInfo, exist bool) { colOffset, exist := ti.columnsOffset[colID] @@ -168,7 +248,7 @@ func (ti *TableInfo) GetRowColInfos() (int64, []rowcodec.ColInfo) { } // IsColCDCVisible returns whether the col is visible for CDC -func (ti *TableInfo) IsColCDCVisible(col *model.ColumnInfo) bool { +func IsColCDCVisible(col *model.ColumnInfo) bool { // this column is a virtual generated column if col.IsGenerated() && !col.GeneratedStored { return false diff --git a/cdc/model/sink.go b/cdc/model/sink.go index cd6b65b6403..d4e091169d2 100644 --- a/cdc/model/sink.go +++ b/cdc/model/sink.go @@ -55,118 +55,136 @@ const ( MultipleKeyFlag // NullableFlag means the column is nullable NullableFlag + // UnsignedFlag means the column stores an unsigned integer + UnsignedFlag ) -//SetIsBinary set BinaryFlag +//SetIsBinary sets BinaryFlag func (b *ColumnFlagType) SetIsBinary() { (*util.Flag)(b).Add(util.Flag(BinaryFlag)) } -//UnsetIsBinary unset BinaryFlag +//UnsetIsBinary unsets BinaryFlag func (b *ColumnFlagType) UnsetIsBinary() { (*util.Flag)(b).Remove(util.Flag(BinaryFlag)) } -//IsBinary show whether BinaryFlag is set +//IsBinary shows whether BinaryFlag is set func (b *ColumnFlagType) IsBinary() bool { return (*util.Flag)(b).HasAll(util.Flag(BinaryFlag)) } -//SetIsHandleKey set HandleKey +//SetIsHandleKey sets HandleKey func (b *ColumnFlagType) SetIsHandleKey() { (*util.Flag)(b).Add(util.Flag(HandleKeyFlag)) } -//UnsetIsHandleKey unset HandleKey +//UnsetIsHandleKey unsets HandleKey func (b *ColumnFlagType) UnsetIsHandleKey() { (*util.Flag)(b).Remove(util.Flag(HandleKeyFlag)) } -//IsHandleKey show whether HandleKey is set +//IsHandleKey shows whether HandleKey is set func (b *ColumnFlagType) IsHandleKey() bool { return (*util.Flag)(b).HasAll(util.Flag(HandleKeyFlag)) } -//SetIsGeneratedColumn set GeneratedColumn +//SetIsGeneratedColumn sets GeneratedColumn func (b *ColumnFlagType) SetIsGeneratedColumn() { (*util.Flag)(b).Add(util.Flag(GeneratedColumnFlag)) } -//UnsetIsGeneratedColumn unset GeneratedColumn +//UnsetIsGeneratedColumn unsets GeneratedColumn func (b *ColumnFlagType) UnsetIsGeneratedColumn() { (*util.Flag)(b).Remove(util.Flag(GeneratedColumnFlag)) } -//IsGeneratedColumn show whether GeneratedColumn is set +//IsGeneratedColumn shows whether GeneratedColumn is set func (b *ColumnFlagType) IsGeneratedColumn() bool { return (*util.Flag)(b).HasAll(util.Flag(GeneratedColumnFlag)) } -//SetIsPrimaryKey set PrimaryKeyFlag +//SetIsPrimaryKey sets PrimaryKeyFlag func (b *ColumnFlagType) SetIsPrimaryKey() { (*util.Flag)(b).Add(util.Flag(PrimaryKeyFlag)) } -//UnsetIsPrimaryKey unset PrimaryKeyFlag +//UnsetIsPrimaryKey unsets PrimaryKeyFlag func (b *ColumnFlagType) UnsetIsPrimaryKey() { (*util.Flag)(b).Remove(util.Flag(PrimaryKeyFlag)) } -//IsPrimaryKey show whether PrimaryKeyFlag is set +//IsPrimaryKey shows whether PrimaryKeyFlag is set func (b *ColumnFlagType) IsPrimaryKey() bool { return (*util.Flag)(b).HasAll(util.Flag(PrimaryKeyFlag)) } -//SetIsUniqueKey set UniqueKeyFlag +//SetIsUniqueKey sets UniqueKeyFlag func (b *ColumnFlagType) SetIsUniqueKey() { (*util.Flag)(b).Add(util.Flag(UniqueKeyFlag)) } -//UnsetIsUniqueKey unset UniqueKeyFlag +//UnsetIsUniqueKey unsets UniqueKeyFlag func (b *ColumnFlagType) UnsetIsUniqueKey() { (*util.Flag)(b).Remove(util.Flag(UniqueKeyFlag)) } -//IsUniqueKey show whether UniqueKeyFlag is set +//IsUniqueKey shows whether UniqueKeyFlag is set func (b *ColumnFlagType) IsUniqueKey() bool { return (*util.Flag)(b).HasAll(util.Flag(UniqueKeyFlag)) } -//IsMultipleKey show whether MultipleKeyFlag is set +//IsMultipleKey shows whether MultipleKeyFlag is set func (b *ColumnFlagType) IsMultipleKey() bool { return (*util.Flag)(b).HasAll(util.Flag(MultipleKeyFlag)) } -//SetIsMultipleKey set MultipleKeyFlag +//SetIsMultipleKey sets MultipleKeyFlag func (b *ColumnFlagType) SetIsMultipleKey() { (*util.Flag)(b).Add(util.Flag(MultipleKeyFlag)) } -//UnsetIsMultipleKey unset MultipleKeyFlag +//UnsetIsMultipleKey unsets MultipleKeyFlag func (b *ColumnFlagType) UnsetIsMultipleKey() { (*util.Flag)(b).Remove(util.Flag(MultipleKeyFlag)) } -//IsNullable show whether NullableFlag is set +//IsNullable shows whether NullableFlag is set func (b *ColumnFlagType) IsNullable() bool { return (*util.Flag)(b).HasAll(util.Flag(NullableFlag)) } -//SetIsNullable set NullableFlag +//SetIsNullable sets NullableFlag func (b *ColumnFlagType) SetIsNullable() { (*util.Flag)(b).Add(util.Flag(NullableFlag)) } -//UnsetIsNullable unset NullableFlag +//UnsetIsNullable unsets NullableFlag func (b *ColumnFlagType) UnsetIsNullable() { (*util.Flag)(b).Remove(util.Flag(NullableFlag)) } +//IsUnsigned shows whether UnsignedFlag is set +func (b *ColumnFlagType) IsUnsigned() bool { + return (*util.Flag)(b).HasAll(util.Flag(UnsignedFlag)) +} + +//SetIsUnsigned sets UnsignedFlag +func (b *ColumnFlagType) SetIsUnsigned() { + (*util.Flag)(b).Add(util.Flag(UnsignedFlag)) +} + +//UnsetIsUnsigned unsets UnsignedFlag +func (b *ColumnFlagType) UnsetIsUnsigned() { + (*util.Flag)(b).Remove(util.Flag(UnsignedFlag)) +} + // TableName represents name of a table, includes table name and schema name. type TableName struct { - Schema string `toml:"db-name" json:"db-name"` - Table string `toml:"tbl-name" json:"tbl-name"` - Partition int64 `json:"partition"` + Schema string `toml:"db-name" json:"db-name"` + Table string `toml:"tbl-name" json:"tbl-name"` + TableID int64 `toml:"tbl-id" json:"tbl-"` + IsPartition bool `toml:"is-partition" json:"is-partition"` } // String implements fmt.Stringer interface. @@ -189,6 +207,11 @@ func (t *TableName) GetTable() string { return t.Table } +// GetTableID returns table ID. +func (t *TableName) GetTableID() int64 { + return t.TableID +} + // RowChangedEvent represents a row changed event type RowChangedEvent struct { StartTs uint64 `json:"start-ts"` @@ -198,26 +221,27 @@ type RowChangedEvent struct { Table *TableName `json:"table"` - Delete bool `json:"delete"` - TableInfoVersion uint64 `json:"table-info-version,omitempty"` - // if the table of this row only has one unique index(includes primary key), - // IndieMarkCol will be set to the name of the unique index - IndieMarkCol string `json:"indie-mark-col"` - Columns map[string]*Column `json:"columns"` - PreColumns map[string]*Column `json:"pre-columns"` - Keys []string `json:"keys"` + Columns []*Column `json:"columns"` + PreColumns []*Column `json:"pre-columns"` + IndexColumns [][]int + + // approximate size of this event, calculate by tikv proto bytes size + ApproximateSize int64 +} + +// IsDelete returns true if the row is a delete event +func (r *RowChangedEvent) IsDelete() bool { + return len(r.PreColumns) != 0 && len(r.Columns) == 0 } // Column represents a column value in row changed event type Column struct { - Type byte `json:"t"` - // WhereHandle is deprecation - // WhereHandle is replaced by HandleKey in Flag - WhereHandle *bool `json:"h,omitempty"` - Flag ColumnFlagType `json:"f"` - Value interface{} `json:"v"` + Name string `json:"name"` + Type byte `json:"type"` + Flag ColumnFlagType `json:"flag"` + Value interface{} `json:"value"` } // ColumnValueString returns the string representation of the column value @@ -281,7 +305,9 @@ type SimpleTableInfo struct { // db name Schema string // table name - Table string + Table string + // table ID + TableID int64 ColumnInfo []*ColumnInfo } @@ -296,68 +322,63 @@ type DDLEvent struct { } // FromJob fills the values of DDLEvent from DDL job -func (e *DDLEvent) FromJob(job *model.Job, preTableInfo *TableInfo) { - e.TableInfo = new(SimpleTableInfo) - e.TableInfo.Schema = job.SchemaName - e.StartTs = job.StartTS - e.CommitTs = job.BinlogInfo.FinishedTS - e.Query = job.Query - e.Type = job.Type +func (d *DDLEvent) FromJob(job *model.Job, preTableInfo *TableInfo) { + d.TableInfo = new(SimpleTableInfo) + d.TableInfo.Schema = job.SchemaName + d.StartTs = job.StartTS + d.CommitTs = job.BinlogInfo.FinishedTS + d.Query = job.Query + d.Type = job.Type if job.BinlogInfo.TableInfo != nil { tableName := job.BinlogInfo.TableInfo.Name.O tableInfo := job.BinlogInfo.TableInfo - e.TableInfo.ColumnInfo = make([]*ColumnInfo, len(tableInfo.Columns)) + d.TableInfo.ColumnInfo = make([]*ColumnInfo, len(tableInfo.Columns)) for i, colInfo := range tableInfo.Columns { - e.TableInfo.ColumnInfo[i] = new(ColumnInfo) - e.TableInfo.ColumnInfo[i].FromTiColumnInfo(colInfo) + d.TableInfo.ColumnInfo[i] = new(ColumnInfo) + d.TableInfo.ColumnInfo[i].FromTiColumnInfo(colInfo) } - e.TableInfo.Table = tableName + d.TableInfo.Table = tableName + d.TableInfo.TableID = job.TableID } - e.fillPreTableInfo(preTableInfo) + d.fillPreTableInfo(preTableInfo) } -func (e *DDLEvent) fillPreTableInfo(preTableInfo *TableInfo) { +func (d *DDLEvent) fillPreTableInfo(preTableInfo *TableInfo) { if preTableInfo == nil { return } - e.PreTableInfo = new(SimpleTableInfo) - e.PreTableInfo.Schema = preTableInfo.TableName.Schema - e.PreTableInfo.Table = preTableInfo.TableName.Table + d.PreTableInfo = new(SimpleTableInfo) + d.PreTableInfo.Schema = preTableInfo.TableName.Schema + d.PreTableInfo.Table = preTableInfo.TableName.Table + d.PreTableInfo.TableID = preTableInfo.ID - e.PreTableInfo.ColumnInfo = make([]*ColumnInfo, len(preTableInfo.Columns)) + d.PreTableInfo.ColumnInfo = make([]*ColumnInfo, len(preTableInfo.Columns)) for i, colInfo := range preTableInfo.Columns { - e.PreTableInfo.ColumnInfo[i] = new(ColumnInfo) - e.PreTableInfo.ColumnInfo[i].FromTiColumnInfo(colInfo) + d.PreTableInfo.ColumnInfo[i] = new(ColumnInfo) + d.PreTableInfo.ColumnInfo[i].FromTiColumnInfo(colInfo) } } -// Txn represents a transaction which includes many row events -type Txn struct { +// SingleTableTxn represents a transaction which includes many row events in a single table +type SingleTableTxn struct { + Table *TableName StartTs uint64 CommitTs uint64 Rows []*RowChangedEvent - Keys []string ReplicaID uint64 } -// Append adds a row changed event into Txn -func (t *Txn) Append(row *RowChangedEvent) { - if row.StartTs != t.StartTs || row.CommitTs != t.CommitTs { +// Append adds a row changed event into SingleTableTxn +func (t *SingleTableTxn) Append(row *RowChangedEvent) { + if row.StartTs != t.StartTs || row.CommitTs != t.CommitTs || row.Table.TableID != t.Table.TableID { log.Fatal("unexpected row change event", zap.Uint64("startTs of txn", t.StartTs), zap.Uint64("commitTs of txn", t.CommitTs), - zap.Uint64("startTs of row", row.StartTs), - zap.Uint64("commitTs of row", row.CommitTs)) + zap.Any("table of txn", t.Table), + zap.Any("row", row)) } t.Rows = append(t.Rows, row) - if len(row.Keys) == 0 { - if len(t.Keys) == 0 { - t.Keys = []string{QuoteSchema(row.Table.Schema, row.Table.Table)} - } - } else { - t.Keys = append(t.Keys, row.Keys...) - } } diff --git a/cdc/owner.go b/cdc/owner.go index 2b776ed62ae..7ef4eac1b6c 100644 --- a/cdc/owner.go +++ b/cdc/owner.go @@ -76,12 +76,16 @@ type Owner struct { // gcTTL is the ttl of cdc gc safepoint ttl. gcTTL int64 - // whether gc safepoint is set in pd - gcSafePointSet bool + // last update gc safepoint time. zero time means has not updated or cleared + gcSafepointLastUpdate time.Time } -// CDCServiceSafePointID is the ID of CDC service in pd.UpdateServiceGCSafePoint. -const CDCServiceSafePointID = "ticdc" +const ( + // CDCServiceSafePointID is the ID of CDC service in pd.UpdateServiceGCSafePoint. + CDCServiceSafePointID = "ticdc" + // GCSafepointUpdateInterval is the minimual interval that CDC can update gc safepoint + GCSafepointUpdateInterval = time.Duration(2 * time.Second) +) // NewOwner creates a new Owner instance func NewOwner(pdClient pd.Client, credential *security.Credential, sess *concurrency.Session, gcTTL int64) (*Owner, error) { @@ -178,7 +182,7 @@ func (o *Owner) newChangeFeed( taskPositions map[string]*model.TaskPosition, info *model.ChangeFeedInfo, checkpointTs uint64) (*changeFeed, error) { - log.Info("Find new changefeed", zap.Reflect("info", info), + log.Info("Find new changefeed", zap.Stringer("info", info), zap.String("id", id), zap.Uint64("checkpoint ts", checkpointTs)) failpoint.Inject("NewChangefeedNoRetryError", func() { @@ -296,6 +300,7 @@ func (o *Owner) newChangeFeed( } sinkTableInfo[j-1] = new(model.SimpleTableInfo) + sinkTableInfo[j-1].TableID = tid sinkTableInfo[j-1].ColumnInfo = make([]*model.ColumnInfo, len(tblInfo.Cols())) for i, colInfo := range tblInfo.Cols() { @@ -544,12 +549,12 @@ func (o *Owner) balanceTables(ctx context.Context) error { func (o *Owner) flushChangeFeedInfos(ctx context.Context) error { // no running or stopped changefeed, clear gc safepoint. if len(o.changeFeeds) == 0 && len(o.stoppedFeeds) == 0 { - if o.gcSafePointSet { + if !o.gcSafepointLastUpdate.IsZero() { _, err := o.pdClient.UpdateServiceGCSafePoint(ctx, CDCServiceSafePointID, 0, 0) if err != nil { return errors.Trace(err) } - o.gcSafePointSet = false + o.gcSafepointLastUpdate = *new(time.Time) } return nil } @@ -573,12 +578,14 @@ func (o *Owner) flushChangeFeedInfos(ctx context.Context) error { minCheckpointTs = status.CheckpointTs } } - _, err := o.pdClient.UpdateServiceGCSafePoint(ctx, CDCServiceSafePointID, o.gcTTL, minCheckpointTs) - if err != nil { - log.Info("failed to update service safe point", zap.Error(err)) - return errors.Trace(err) + if time.Since(o.gcSafepointLastUpdate) > GCSafepointUpdateInterval { + _, err := o.pdClient.UpdateServiceGCSafePoint(ctx, CDCServiceSafePointID, o.gcTTL, minCheckpointTs) + if err != nil { + log.Info("failed to update service safe point", zap.Error(err)) + return errors.Trace(err) + } + o.gcSafepointLastUpdate = time.Now() } - o.gcSafePointSet = true return nil } @@ -621,7 +628,7 @@ func (o *Owner) dispatchJob(ctx context.Context, job model.AdminJob) error { return errors.Errorf("changefeed %s not found in owner cache", job.CfID) } for captureID := range cf.taskStatus { - newStatus, err := cf.etcdCli.AtomicPutTaskStatus(ctx, cf.id, captureID, func(modRevision int64, taskStatus *model.TaskStatus) (bool, error) { + newStatus, _, err := cf.etcdCli.AtomicPutTaskStatus(ctx, cf.id, captureID, func(modRevision int64, taskStatus *model.TaskStatus) (bool, error) { taskStatus.AdminJobType = job.Type return true, nil }) @@ -785,7 +792,14 @@ func (o *Owner) handleAdminJob(ctx context.Context) error { switch feedState { case model.StateRemoved, model.StateFinished: // remove a removed or finished changefeed - log.Info("changefeed has been removed or finished, remove command will do nothing") + if job.Opts != nil && job.Opts.ForceRemove { + err := o.etcdClient.RemoveChangeFeedStatus(ctx, job.CfID) + if err != nil { + return errors.Trace(err) + } + } else { + log.Info("changefeed has been removed or finished, remove command will do nothing") + } continue case model.StateStopped, model.StateFailed: // remove a paused or failed changefeed @@ -804,10 +818,18 @@ func (o *Owner) handleAdminJob(ctx context.Context) error { if err != nil { return errors.Trace(err) } - // set ttl to changefeed status - err = o.etcdClient.SetChangeFeedStatusTTL(ctx, job.CfID, 24*3600 /*24 hours*/) - if err != nil { - return errors.Trace(err) + if job.Opts != nil && job.Opts.ForceRemove { + // if `ForceRemove` is enabled, remove all information related to this changefeed + err := o.etcdClient.RemoveChangeFeedStatus(ctx, job.CfID) + if err != nil { + return errors.Trace(err) + } + } else { + // set ttl to changefeed status + err = o.etcdClient.SetChangeFeedStatusTTL(ctx, job.CfID, 24*3600 /*24 hours*/) + if err != nil { + return errors.Trace(err) + } } case model.AdminResume: // resume changefeed must read checkpoint from ChangeFeedStatus diff --git a/cdc/owner_test.go b/cdc/owner_test.go index fc27a98b76b..51250695b0c 100644 --- a/cdc/owner_test.go +++ b/cdc/owner_test.go @@ -89,7 +89,7 @@ func (h *handlerForPrueDMLTest) PullDDL() (resolvedTs uint64, ddl []*model.DDL, return uint64(math.MaxUint64), nil, nil } -func (h *handlerForPrueDMLTest) ExecDDL(context.Context, string, map[string]string, model.Txn) error { +func (h *handlerForPrueDMLTest) ExecDDL(context.Context, string, map[string]string, model.SingleTableTxn) error { panic("unreachable") } @@ -240,7 +240,7 @@ func (h *handlerForDDLTest) PullDDL() (resolvedTs uint64, jobs []*model.DDL, err return h.ddlResolvedTs[h.ddlIndex], []*model.DDL{h.ddls[h.ddlIndex]}, nil } -func (h *handlerForDDLTest) ExecDDL(ctx context.Context, sinkURI string, _ map[string]string, txn model.Txn) error { +func (h *handlerForDDLTest) ExecDDL(ctx context.Context, sinkURI string, _ map[string]string, txn model.SingleTableTxn) error { h.mu.Lock() defer h.mu.Unlock() h.ddlExpectIndex++ @@ -578,7 +578,7 @@ func (s *ownerSuite) TestChangefeedApplyDDLJob(c *check.C) { Name: timodel.NewCIStr("t1"), PKIsHandle: true, Columns: []*timodel.ColumnInfo{ - {ID: 1, FieldType: types.FieldType{Flag: mysql.PriKeyFlag}}, + {ID: 1, FieldType: types.FieldType{Flag: mysql.PriKeyFlag}, State: timodel.StatePublic}, }, }, }, @@ -600,7 +600,7 @@ func (s *ownerSuite) TestChangefeedApplyDDLJob(c *check.C) { Name: timodel.NewCIStr("t2"), PKIsHandle: true, Columns: []*timodel.ColumnInfo{ - {ID: 1, FieldType: types.FieldType{Flag: mysql.PriKeyFlag}}, + {ID: 1, FieldType: types.FieldType{Flag: mysql.PriKeyFlag}, State: timodel.StatePublic}, }, }, }, @@ -642,7 +642,7 @@ func (s *ownerSuite) TestChangefeedApplyDDLJob(c *check.C) { Name: timodel.NewCIStr("t1"), PKIsHandle: true, Columns: []*timodel.ColumnInfo{ - {ID: 1, FieldType: types.FieldType{Flag: mysql.PriKeyFlag}}, + {ID: 1, FieldType: types.FieldType{Flag: mysql.PriKeyFlag}, State: timodel.StatePublic}, }, }, }, @@ -705,7 +705,7 @@ func (s *ownerSuite) TestChangefeedApplyDDLJob(c *check.C) { f, err := filter.NewFilter(config.GetDefaultReplicaConfig()) c.Assert(err, check.IsNil) - store, err := mockstore.NewMockStore() + store, err := mockstore.NewMockTikvStore() c.Assert(err, check.IsNil) defer func() { _ = store.Close() diff --git a/cdc/processor.go b/cdc/processor.go index b0398709351..3dcfab58b83 100644 --- a/cdc/processor.go +++ b/cdc/processor.go @@ -51,8 +51,6 @@ import ( ) const ( - resolveTsInterval = time.Millisecond * 500 - // TODO: processor output chan size, the accumulated data is determined by // the count of sorted data and unmounted data. In current benchmark a single // processor can reach 50k-100k QPS, and accumulated data is around @@ -63,6 +61,8 @@ const ( defaultMemBufferCapacity int64 = 10 * 1024 * 1024 * 1024 // 10G defaultSyncResolvedBatch = 1024 + + defaultFlushTaskPositionInterval = 200 * time.Millisecond ) var ( @@ -87,6 +87,7 @@ type processor struct { sinkEmittedResolvedTs uint64 globalResolvedTs uint64 + localResolvedTs uint64 checkpointTs uint64 ddlPuller puller.Puller @@ -107,9 +108,12 @@ type processor struct { sinkEmittedResolvedReceiver *notify.Receiver localResolvedNotifier *notify.Notifier localResolvedReceiver *notify.Receiver + localCheckpointTsNotifier *notify.Notifier + localCheckpointTsReceiver *notify.Receiver - wg *errgroup.Group - errCh chan<- error + wg *errgroup.Group + errCh chan<- error + opDoneCh chan int64 } type tableInfo struct { @@ -121,6 +125,10 @@ type tableInfo struct { sorter *puller.Rectifier workload model.WorkloadInfo cancel context.CancelFunc + // isDying shows that the table is being removed. + // In the case the same table is added back before safe removal is finished, + // this flag is used to tell whether it's safe to kill the table. + isDying uint32 } func (t *tableInfo) loadResolvedTs() uint64 { @@ -136,6 +144,7 @@ func (t *tableInfo) loadResolvedTs() uint64 { // safeStop will stop the table change feed safety func (t *tableInfo) safeStop() (stopped bool, checkpointTs model.Ts) { + atomic.StoreUint32(&t.isDying, 1) t.sorter.SafeStop() status := t.sorter.GetStatus() if status != model.SorterStatusStopped && status != model.SorterStatusFinished { @@ -183,6 +192,7 @@ func newProcessor( sinkEmittedResolvedNotifier := new(notify.Notifier) localResolvedNotifier := new(notify.Notifier) + localCheckpointTsNotifier := new(notify.Notifier) p := &processor{ id: uuid.New().String(), limitter: limitter, @@ -209,8 +219,13 @@ func newProcessor( localResolvedNotifier: localResolvedNotifier, localResolvedReceiver: localResolvedNotifier.NewReceiver(50 * time.Millisecond), + localCheckpointTsNotifier: localCheckpointTsNotifier, + localCheckpointTsReceiver: localCheckpointTsNotifier.NewReceiver(50 * time.Millisecond), + tables: make(map[int64]*tableInfo), markTableIDs: make(map[int64]struct{}), + + opDoneCh: make(chan int64, 256), } modRevision, status, err := p.etcdCli.GetTaskStatus(ctx, p.changefeedID, p.captureInfo.ID) if err != nil { @@ -309,15 +324,18 @@ func (p *processor) writeDebugInfo(w io.Writer) { // 3, sync TaskStatus between in memory and storage. // 4, check admin command in TaskStatus and apply corresponding command func (p *processor) positionWorker(ctx context.Context) error { - checkpointTsTick := time.NewTicker(resolveTsInterval) - lastUpdateInfoTime := time.Now() - updateInfo := func() error { + lastFlushTime := time.Now() + retryFlushTaskStatusAndPosition := func() error { t0Update := time.Now() err := retry.Run(500*time.Millisecond, 3, func() error { - inErr := p.updateInfo(ctx) + inErr := p.flushTaskStatusAndPosition(ctx) if inErr != nil { if errors.Cause(inErr) != context.Canceled { - log.Error( + logError := log.Error + if errors.Cause(inErr) == model.ErrAdminStopProcessor { + logError = log.Warn + } + logError( "update info failed", zap.String("changefeed", p.changefeedID), zap.Error(inErr), ) @@ -334,16 +352,15 @@ func (p *processor) positionWorker(ctx context.Context) error { if err != nil { return errors.Annotate(err, "failed to update info") } - lastUpdateInfoTime = time.Now() return nil } defer func() { - checkpointTsTick.Stop() p.localResolvedReceiver.Stop() + p.localCheckpointTsReceiver.Stop() if !p.isStopped() { - err := updateInfo() + err := retryFlushTaskStatusAndPosition() if err != nil && errors.Cause(err) != context.Canceled { log.Warn("failed to update info before exit", zap.Error(err)) } @@ -356,8 +373,6 @@ func (p *processor) positionWorker(ctx context.Context) error { metricResolvedTsLagGauge := resolvedTsLagGauge.WithLabelValues(p.changefeedID, p.captureInfo.AdvertiseAddr) checkpointTsGauge := checkpointTsGauge.WithLabelValues(p.changefeedID, p.captureInfo.AdvertiseAddr) metricCheckpointTsLagGauge := checkpointTsLagGauge.WithLabelValues(p.changefeedID, p.captureInfo.AdvertiseAddr) - checkUpdateInfo := time.NewTicker(500 * time.Millisecond) - defer checkUpdateInfo.Stop() for { select { case <-ctx.Done(): @@ -373,23 +388,19 @@ func (p *processor) positionWorker(ctx context.Context) error { } } p.stateMu.Unlock() + atomic.StoreUint64(&p.localResolvedTs, minResolvedTs) phyTs := oracle.ExtractPhysical(minResolvedTs) // It is more accurate to get tso from PD, but in most cases we have // deployed NTP service, a little bias is acceptable here. metricResolvedTsLagGauge.Set(float64(oracle.GetPhysical(time.Now())-phyTs) / 1e3) - if minResolvedTs == p.position.ResolvedTs { - continue - } - p.position.ResolvedTs = minResolvedTs resolvedTsGauge.Set(float64(phyTs)) - if err := updateInfo(); err != nil { + if err := retryFlushTaskStatusAndPosition(); err != nil { return errors.Trace(err) } - - case <-checkpointTsTick.C: + case <-p.localCheckpointTsReceiver.C: checkpointTs := atomic.LoadUint64(&p.checkpointTs) phyTs := oracle.ExtractPhysical(checkpointTs) // It is more accurate to get tso from PD, but in most cases we have @@ -399,18 +410,16 @@ func (p *processor) positionWorker(ctx context.Context) error { if p.position.CheckPointTs >= checkpointTs { continue } - p.position.CheckPointTs = checkpointTs - checkpointTsGauge.Set(float64(phyTs)) - if err := updateInfo(); err != nil { - return errors.Trace(err) - } - case <-checkUpdateInfo.C: - if time.Since(lastUpdateInfoTime) < time.Second { + if time.Since(lastFlushTime) < defaultFlushTaskPositionInterval { continue } - if err := updateInfo(); err != nil { + + p.position.CheckPointTs = checkpointTs + checkpointTsGauge.Set(float64(phyTs)) + if err := retryFlushTaskStatusAndPosition(); err != nil { return errors.Trace(err) } + lastFlushTime = time.Now() } } } @@ -472,33 +481,40 @@ func (p *processor) workloadWorker(ctx context.Context) error { } } -func (p *processor) updateInfo(ctx context.Context) error { +func (p *processor) flushTaskPosition(ctx context.Context) error { + failpoint.Inject("ProcessorUpdatePositionDelaying", func() { + time.Sleep(1 * time.Second) + }) if p.isStopped() { return errors.Trace(model.ErrAdminStopProcessor) } - updatePosition := func() error { - failpoint.Inject("ProcessorUpdatePositionDelaying", func() { - time.Sleep(1 * time.Second) - }) - //p.position.Count = p.sink.Count() - err := p.etcdCli.PutTaskPosition(ctx, p.changefeedID, p.captureInfo.ID, p.position) - if err != nil { - log.Error("failed to update position", zap.Error(err)) - return errors.Trace(err) - } - log.Debug("update task position", zap.Stringer("position", p.position)) - return nil + //p.position.Count = p.sink.Count() + err := p.etcdCli.PutTaskPosition(ctx, p.changefeedID, p.captureInfo.ID, p.position) + if err == nil { + log.Debug("flushed task position", zap.Stringer("position", p.position)) + } else if errors.Cause(err) != context.Canceled { + log.Error("failed to flush task position", zap.Error(err)) + return errors.Trace(err) + } + return nil +} + +// First try to synchronize task status from etcd. +// If local cached task status is outdated (caused by new table scheduling), +// update it to latest value, and force update task position, since add new +// tables may cause checkpoint ts fallback in processor. +func (p *processor) flushTaskStatusAndPosition(ctx context.Context) error { + if p.isStopped() { + return errors.Trace(model.ErrAdminStopProcessor) } - newModRevision := p.statusModRevision var tablesToRemove []model.TableID - newTaskStatus, err := p.etcdCli.AtomicPutTaskStatus(ctx, p.changefeedID, p.captureInfo.ID, + newTaskStatus, newModRevision, err := p.etcdCli.AtomicPutTaskStatus(ctx, p.changefeedID, p.captureInfo.ID, func(modRevision int64, taskStatus *model.TaskStatus) (bool, error) { // if the task status is not changed and not operation to handle // we need not to change the task status if p.statusModRevision == modRevision && !taskStatus.SomeOperationsUnapplied() { return false, nil } - newModRevision = modRevision if taskStatus.AdminJobType.IsStopState() { err := p.stop(ctx) if err != nil { @@ -511,7 +527,7 @@ func (p *processor) updateInfo(ctx context.Context) error { if err != nil { return false, backoff.Permanent(errors.Trace(err)) } - err = updatePosition() + err = p.flushTaskPosition(ctx) if err != nil { return true, errors.Trace(err) } @@ -520,7 +536,7 @@ func (p *processor) updateInfo(ctx context.Context) error { if err != nil { // not need to check error //nolint:errcheck - updatePosition() + p.flushTaskPosition(ctx) return errors.Trace(err) } for _, tableID := range tablesToRemove { @@ -531,7 +547,8 @@ func (p *processor) updateInfo(ctx context.Context) error { syncTableNumGauge. WithLabelValues(p.changefeedID, p.captureInfo.AdvertiseAddr). Set(float64(len(p.status.Tables))) - return nil + + return p.flushTaskPosition(ctx) } func (p *processor) removeTable(tableID int64) { @@ -546,6 +563,9 @@ func (p *processor) removeTable(tableID int64) { return } + if atomic.SwapUint32(&table.isDying, 0) == 0 { + return + } table.cancel() delete(p.tables, tableID) if table.markTableID != 0 { @@ -570,6 +590,8 @@ func (p *processor) handleTables(ctx context.Context, status *model.TaskStatus) continue } stopped, checkpointTs := table.safeStop() + log.Debug("safeStop table", zap.Int64("tableID", tableID), + zap.Bool("stopped", stopped), zap.Uint64("checkpointTs", checkpointTs)) if stopped { opt.BoundaryTs = checkpointTs if checkpointTs <= p.position.CheckPointTs { @@ -587,9 +609,27 @@ func (p *processor) handleTables(ctx context.Context, status *model.TaskStatus) return tablesToRemove, errors.NotValidf("normal table(%d) and mark table not match ", tableID) } p.addTable(ctx, tableID, replicaInfo) - opt.Done = true } } + + for { + select { + case <-ctx.Done(): + return nil, ctx.Err() + case tableID := <-p.opDoneCh: + log.Debug("Operation done signal received", + zap.Int64("tableID", tableID), + zap.Reflect("operation", status.Operation[tableID])) + if status.Operation[tableID] == nil { + log.Debug("TableID does not exist, probably a mark table, ignore", zap.Int64("tableID", tableID)) + continue + } + status.Operation[tableID].Done = true + default: + goto done + } + } +done: if !status.SomeOperationsUnapplied() { status.Operation = nil } @@ -601,12 +641,15 @@ func (p *processor) globalStatusWorker(ctx context.Context) error { log.Info("Global status worker started") var ( - changefeedStatus *model.ChangeFeedStatus - statusRev int64 - lastCheckPointTs uint64 - lastResolvedTs uint64 - watchKey = kv.GetEtcdKeyJob(p.changefeedID) + changefeedStatus *model.ChangeFeedStatus + statusRev int64 + lastCheckPointTs uint64 + lastResolvedTs uint64 + watchKey = kv.GetEtcdKeyJob(p.changefeedID) + globalResolvedTsNotifier = new(notify.Notifier) + globalResolvedTsReceiver = globalResolvedTsNotifier.NewReceiver(1 * time.Second) ) + updateStatus := func(changefeedStatus *model.ChangeFeedStatus) error { if lastResolvedTs == changefeedStatus.ResolvedTs && lastCheckPointTs == changefeedStatus.CheckpointTs { @@ -619,14 +662,34 @@ func (p *processor) globalStatusWorker(ctx context.Context) error { if lastResolvedTs < changefeedStatus.ResolvedTs { lastResolvedTs = changefeedStatus.ResolvedTs atomic.StoreUint64(&p.globalResolvedTs, lastResolvedTs) + log.Debug("Update globalResolvedTs", zap.Uint64("globalResolvedTs", lastResolvedTs)) + globalResolvedTsNotifier.Notify() + } + return nil + } + + go func() { + for { select { case <-ctx.Done(): - return ctx.Err() - case p.output <- model.NewResolvedPolymorphicEvent(lastResolvedTs): + return + case <-globalResolvedTsReceiver.C: + globalResolvedTs := atomic.LoadUint64(&p.globalResolvedTs) + localResolvedTs := atomic.LoadUint64(&p.localResolvedTs) + if globalResolvedTs > localResolvedTs { + log.Warn("globalResolvedTs too large", zap.Uint64("globalResolvedTs", globalResolvedTs), + zap.Uint64("localResolvedTs", localResolvedTs)) + // we do not issue resolved events if globalResolvedTs > localResolvedTs. + continue + } + select { + case <-ctx.Done(): + return + case p.output <- model.NewResolvedPolymorphicEvent(lastResolvedTs): + } } } - return nil - } + }() retryCfg := backoff.WithMaxRetries( backoff.WithContext( @@ -682,6 +745,7 @@ func (p *processor) globalStatusWorker(ctx context.Context) error { } func (p *processor) sinkDriver(ctx context.Context) error { + metricFlushDuration := sinkFlushRowChangedDuration.WithLabelValues(p.changefeedID, p.captureInfo.AdvertiseAddr) for { select { case <-ctx.Done(): @@ -695,14 +759,25 @@ func (p *processor) sinkDriver(ctx context.Context) error { } else { minTs = globalResolvedTs } - if minTs == 0 { + if minTs == 0 || atomic.LoadUint64(&p.checkpointTs) == minTs { continue } - err := p.sink.FlushRowChangedEvents(ctx, minTs) + start := time.Now() + + checkpointTs, err := p.sink.FlushRowChangedEvents(ctx, minTs) if err != nil { return errors.Trace(err) } - atomic.StoreUint64(&p.checkpointTs, minTs) + if checkpointTs != 0 { + atomic.StoreUint64(&p.checkpointTs, checkpointTs) + p.localCheckpointTsNotifier.Notify() + } + + dur := time.Since(start) + metricFlushDuration.Observe(dur.Seconds()) + if dur > 3*time.Second { + log.Warn("flush row changed events too slow", zap.Duration("duration", dur)) + } } } } @@ -728,6 +803,7 @@ func (p *processor) syncResolved(ctx context.Context) error { } rows = append(rows, ev.Row) } + failpoint.Inject("ProcessorSyncResolvedPreEmit", func() {}) err := p.sink.EmitRowChangedEvents(ctx, rows...) if err != nil { return errors.Trace(err) @@ -821,9 +897,14 @@ func (p *processor) addTable(ctx context.Context, tableID int64, replicaInfo *mo tableName = strconv.Itoa(int(tableID)) } - if _, ok := p.tables[tableID]; ok { - log.Warn("Ignore existing table", zap.Int64("ID", tableID)) - return + if table, ok := p.tables[tableID]; ok { + if atomic.SwapUint32(&table.isDying, 0) == 1 { + log.Warn("The same table exists but is dying. Cancel it and continue.", zap.Int64("ID", tableID)) + table.cancel() + } else { + log.Warn("Ignore existing table", zap.Int64("ID", tableID)) + return + } } globalResolvedTs := atomic.LoadUint64(&p.sinkEmittedResolvedTs) log.Debug("Add table", zap.Int64("tableID", tableID), @@ -887,9 +968,40 @@ func (p *processor) addTable(ctx context.Context, tableID int64, replicaInfo *mo p.errCh <- err } }() + + var lastResolvedTs uint64 go func() { + opDone := false resolvedTsGauge := tableResolvedTsGauge.WithLabelValues(p.changefeedID, p.captureInfo.AdvertiseAddr, table.name) - var lastResolvedTs uint64 + checkDoneTicker := time.NewTicker(1 * time.Second) + checkDone := func() { + localResolvedTs := atomic.LoadUint64(&p.localResolvedTs) + globalResolvedTs := atomic.LoadUint64(&p.globalResolvedTs) + if !opDone && lastResolvedTs >= localResolvedTs && localResolvedTs >= globalResolvedTs { + log.Debug("localResolvedTs >= globalResolvedTs, sending operation done signal", + zap.Uint64("localResolvedTs", localResolvedTs), zap.Uint64("globalResolvedTs", globalResolvedTs), + zap.Int64("tableID", tableID)) + + opDone = true + checkDoneTicker.Stop() + select { + case <-ctx.Done(): + if errors.Cause(ctx.Err()) != context.Canceled { + p.errCh <- ctx.Err() + } + return + case p.opDoneCh <- tableID: + } + } + if !opDone { + log.Debug("addTable not done", + zap.Uint64("tableResolvedTs", lastResolvedTs), + zap.Uint64("localResolvedTs", localResolvedTs), + zap.Uint64("globalResolvedTs", globalResolvedTs), + zap.Int64("tableID", tableID)) + } + } + for { select { case <-ctx.Done(): @@ -920,6 +1032,9 @@ func (p *processor) addTable(ctx context.Context, tableID int64, replicaInfo *mo lastResolvedTs = pEvent.CRTs p.localResolvedNotifier.Notify() resolvedTsGauge.Set(float64(oracle.ExtractPhysical(pEvent.CRTs))) + if !opDone { + checkDone() + } continue } sinkResolvedTs := atomic.LoadUint64(&p.sinkEmittedResolvedTs) @@ -940,23 +1055,25 @@ func (p *processor) addTable(ctx context.Context, tableID int64, replicaInfo *mo return case p.output <- pEvent: } + case <-checkDoneTicker.C: + if !opDone { + checkDone() + } } } }() return sorter } - table.sorter = startPuller(tableID, &table.resolvedTs) - if p.changefeed.Config.Cyclic.IsEnabled() && replicaInfo.MarkTableID != 0 { mTableID := replicaInfo.MarkTableID // we should to make sure a mark table is only listened once. if _, exist := p.markTableIDs[mTableID]; !exist { p.markTableIDs[mTableID] = struct{}{} - startPuller(mTableID, &table.mResolvedTs) - table.markTableID = mTableID table.mResolvedTs = replicaInfo.StartTs + + startPuller(mTableID, &table.mResolvedTs) } } @@ -967,6 +1084,9 @@ func (p *processor) addTable(ctx context.Context, tableID int64, replicaInfo *mo if p.position.ResolvedTs > replicaInfo.StartTs { p.position.ResolvedTs = replicaInfo.StartTs } + + table.sorter = startPuller(tableID, &table.resolvedTs) + syncTableNumGauge.WithLabelValues(p.changefeedID, p.captureInfo.AdvertiseAddr).Inc() } diff --git a/cdc/puller/file_sorter.go b/cdc/puller/file_sorter.go index 6c9356c6b40..2584348e597 100644 --- a/cdc/puller/file_sorter.go +++ b/cdc/puller/file_sorter.go @@ -32,17 +32,16 @@ import ( "github.com/google/uuid" "github.com/pingcap/errors" "github.com/pingcap/log" + "github.com/pingcap/ticdc/cdc/model" "github.com/vmihailenco/msgpack/v5" "go.uber.org/zap" "golang.org/x/sync/errgroup" - - "github.com/pingcap/ticdc/cdc/model" ) var ( defaultSorterBufferSize = 1000 defaultAutoResolvedRows = 1000 - defaultInitFileCount = 10 + defaultInitFileCount = 3 defaultFileSizeLimit uint64 = 1 << 31 // 2GB per file at most ) @@ -108,12 +107,23 @@ func (cache *fileCache) increase(idx, size int) { } } -func (cache *fileCache) gc() { - // TODO: control gc running time, in case of the delete operation of - // some large files blocks sorting +func (cache *fileCache) gc(maxRunDuration time.Duration) { cache.fileLock.Lock() - defer cache.fileLock.Unlock() - for _, f := range cache.toRemoveFiles { + index := 0 + defer func() { + cache.toRemoveFiles = cache.toRemoveFiles[index:len(cache.toRemoveFiles)] + cache.fileLock.Unlock() + }() + start := time.Now() + for i, f := range cache.toRemoveFiles { + duration := time.Since(start) + if duration > maxRunDuration { + log.Warn("gc runs execeeds max run duration", + zap.Duration("duration", duration), + zap.Duration("maxRunDuration", maxRunDuration), + ) + return + } fpath := filepath.Join(cache.dir, f) if _, err := os.Stat(fpath); err == nil { err2 := os.Remove(fpath) @@ -121,8 +131,8 @@ func (cache *fileCache) gc() { log.Warn("remove file failed", zap.Error(err2)) } } + index = i + 1 } - cache.toRemoveFiles = cache.toRemoveFiles[:0] } // prepareSorting checks whether the file cache can start a new sorting round @@ -531,9 +541,10 @@ func (fs *FileSorter) gcRemovedFiles(ctx context.Context) error { for { select { case <-ctx.Done(): + fs.cache.gc(time.Second * 3) return errors.Trace(ctx.Err()) case <-ticker.C: - fs.cache.gc() + fs.cache.gc(time.Second * 10) } } } diff --git a/cdc/puller/mock_puller.go b/cdc/puller/mock_puller.go index 143e3f9d3d5..1938ceb7977 100644 --- a/cdc/puller/mock_puller.go +++ b/cdc/puller/mock_puller.go @@ -28,7 +28,6 @@ import ( tidbkv "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/session" "github.com/pingcap/tidb/store/mockstore" - "github.com/pingcap/tidb/store/mockstore/cluster" "github.com/pingcap/tidb/store/mockstore/mocktikv" "github.com/pingcap/tidb/util/testkit" "go.uber.org/zap" @@ -106,7 +105,6 @@ func (l *mvccListener) registerPostRollback(fn func(keys [][]byte, startTs uint6 // MockPullerManager keeps track of transactions for mock pullers type MockPullerManager struct { - cluster cluster.Cluster mvccStore mocktikv.MVCCStore store tidbkv.Storage domain *domain.Domain @@ -198,12 +196,7 @@ func (m *MockPullerManager) setUp(newRowFormat bool) { mvccListener := newMVCCListener(mocktikv.MustNewMVCCStore()) m.mvccStore = mvccListener - store, err := mockstore.NewMockStore( - mockstore.WithClusterInspector(func(c cluster.Cluster) { - mockstore.BootstrapWithSingleStore(c) - m.cluster = c - }), - ) + store, err := mockstore.NewMockTikvStore() if err != nil { log.Fatal("create mock puller failed", zap.Error(err)) } diff --git a/cdc/sink/black_hole.go b/cdc/sink/black_hole.go index 8ad63f1d805..5e91e2353b4 100644 --- a/cdc/sink/black_hole.go +++ b/cdc/sink/black_hole.go @@ -52,7 +52,7 @@ func (b *blackHoleSink) EmitRowChangedEvents(ctx context.Context, rows ...*model return nil } -func (b *blackHoleSink) FlushRowChangedEvents(ctx context.Context, resolvedTs uint64) error { +func (b *blackHoleSink) FlushRowChangedEvents(ctx context.Context, resolvedTs uint64) (uint64, error) { log.Debug("BlockHoleSink: FlushRowChangedEvents", zap.Uint64("resolvedTs", resolvedTs)) err := b.statistics.RecordBatchExecution(func() (int, error) { // TODO: add some random replication latency @@ -63,7 +63,7 @@ func (b *blackHoleSink) FlushRowChangedEvents(ctx context.Context, resolvedTs ui }) b.statistics.PrintStatus() atomic.StoreUint64(&b.checkpointTs, resolvedTs) - return err + return resolvedTs, err } func (b *blackHoleSink) EmitCheckpointTs(ctx context.Context, ts uint64) error { diff --git a/cdc/sink/causality.go b/cdc/sink/causality.go index 3f1cbb2189d..0a9377c36e2 100644 --- a/cdc/sink/causality.go +++ b/cdc/sink/causality.go @@ -13,6 +13,15 @@ package sink +import ( + "encoding/binary" + + "github.com/pingcap/log" + "go.uber.org/zap" + + "github.com/pingcap/ticdc/cdc/model" +) + // causality provides a simple mechanism to improve the concurrency of SQLs execution under the premise of ensuring correctness. // causality groups sqls that maybe contain causal relationships, and syncer executes them linearly. // if some conflicts exist in more than one groups, then syncer waits all SQLs that are grouped be executed and reset causality. @@ -27,13 +36,13 @@ func newCausality() *causality { } } -func (c *causality) add(keys []string, idx int) { +func (c *causality) add(keys [][]byte, idx int) { if len(keys) == 0 { return } for _, key := range keys { - c.relations[key] = idx + c.relations[string(key)] = idx } } @@ -42,14 +51,14 @@ func (c *causality) reset() { } // detectConflict detects whether there is a conflict -func (c *causality) detectConflict(keys []string) (bool, int) { +func (c *causality) detectConflict(keys [][]byte) (bool, int) { if len(keys) == 0 { return false, 0 } firstIdx := -1 for _, key := range keys { - if idx, ok := c.relations[key]; ok { + if idx, ok := c.relations[string(key)]; ok { if firstIdx == -1 { firstIdx = idx } else if firstIdx != idx { @@ -60,3 +69,74 @@ func (c *causality) detectConflict(keys []string) (bool, int) { return firstIdx != -1, firstIdx } + +func genTxnKeys(txn *model.SingleTableTxn) [][]byte { + if len(txn.Rows) == 0 { + return nil + } + keysSet := make(map[string]struct{}, len(txn.Rows)) + for _, row := range txn.Rows { + rowKeys := genRowKeys(row) + for _, key := range rowKeys { + keysSet[string(key)] = struct{}{} + } + } + keys := make([][]byte, 0, len(keysSet)) + for key := range keysSet { + keys = append(keys, []byte(key)) + } + return keys +} + +func genRowKeys(row *model.RowChangedEvent) [][]byte { + var keys [][]byte + if len(row.Columns) != 0 { + for iIdx, idxCol := range row.IndexColumns { + key := genKeyList(row.Columns, iIdx, idxCol, row.Table.TableID) + if len(key) == 0 { + continue + } + keys = append(keys, key) + } + } + if len(row.PreColumns) != 0 { + for iIdx, idxCol := range row.IndexColumns { + key := genKeyList(row.PreColumns, iIdx, idxCol, row.Table.TableID) + if len(key) == 0 { + continue + } + keys = append(keys, key) + } + } + if len(keys) == 0 { + // use table ID as key if no key generated (no PK/UK), + // no concurrence for rows in the same table. + log.Debug("use table id as the key", zap.Int64("tableID", row.Table.TableID)) + tableKey := make([]byte, 8) + binary.BigEndian.PutUint64(tableKey, uint64(row.Table.TableID)) + keys = [][]byte{tableKey} + } + return keys +} + +func genKeyList(columns []*model.Column, iIdx int, colIdx []int, tableID int64) []byte { + var key []byte + for _, i := range colIdx { + // if a column value is null, we can ignore this index + // If the index contain generated column, we can't use this key to detect conflict with other DML, + // Because such as insert can't specified the generated value. + if columns[i] == nil || columns[i].Value == nil || columns[i].Flag.IsGeneratedColumn() { + return nil + } + key = append(key, []byte(model.ColumnValueString(columns[i].Value))...) + key = append(key, 0) + } + if len(key) == 0 { + return nil + } + tableKey := make([]byte, 16) + binary.BigEndian.PutUint64(tableKey[:8], uint64(iIdx)) + binary.BigEndian.PutUint64(tableKey[8:], uint64(tableID)) + key = append(key, tableKey...) + return key +} diff --git a/cdc/sink/causality_test.go b/cdc/sink/causality_test.go index c062f5d1792..8d70ab1760a 100644 --- a/cdc/sink/causality_test.go +++ b/cdc/sink/causality_test.go @@ -14,7 +14,12 @@ package sink import ( + "bytes" + "sort" + "github.com/pingcap/check" + "github.com/pingcap/parser/mysql" + "github.com/pingcap/ticdc/cdc/model" ) type testCausalitySuite struct{} @@ -22,10 +27,10 @@ type testCausalitySuite struct{} var _ = check.Suite(&testCausalitySuite{}) func (s *testCausalitySuite) TestCausality(c *check.C) { - rows := [][]string{ - {"a"}, - {"b"}, - {"c"}, + rows := [][][]byte{ + {[]byte("a")}, + {[]byte("b")}, + {[]byte("c")}, } ca := newCausality() for i, row := range rows { @@ -40,20 +45,20 @@ func (s *testCausalitySuite) TestCausality(c *check.C) { } c.Assert(len(ca.relations), check.Equals, 3) cases := []struct { - keys []string + keys [][]byte conflict bool idx int }{ // Test for single key index conflict. - {[]string{"a", "ab"}, true, 0}, - {[]string{"b", "ba"}, true, 1}, - {[]string{"a", "a"}, true, 0}, - {[]string{"b", "b"}, true, 1}, - {[]string{"c", "c"}, true, 2}, + {[][]byte{[]byte("a"), []byte("ab")}, true, 0}, + {[][]byte{[]byte("b"), []byte("ba")}, true, 1}, + {[][]byte{[]byte("a"), []byte("a")}, true, 0}, + {[][]byte{[]byte("b"), []byte("b")}, true, 1}, + {[][]byte{[]byte("c"), []byte("c")}, true, 2}, // Test for multi-key index conflict. - {[]string{"a", "b"}, true, -1}, - {[]string{"b", "a"}, true, -1}, - {[]string{"b", "c"}, true, -1}, + {[][]byte{[]byte("a"), []byte("b")}, true, -1}, + {[][]byte{[]byte("b"), []byte("a")}, true, -1}, + {[][]byte{[]byte("b"), []byte("c")}, true, -1}, } for _, cas := range cases { conflict, idx := ca.detectConflict(cas.keys) @@ -64,3 +69,149 @@ func (s *testCausalitySuite) TestCausality(c *check.C) { ca.reset() c.Assert(len(ca.relations), check.Equals, 0) } + +func (s *testCausalitySuite) TestGenKeys(c *check.C) { + testCases := []struct { + txn *model.SingleTableTxn + expected [][]byte + }{{ + txn: &model.SingleTableTxn{}, + expected: nil, + }, { + txn: &model.SingleTableTxn{ + Rows: []*model.RowChangedEvent{ + { + StartTs: 418658114257813514, + CommitTs: 418658114257813515, + Table: &model.TableName{Schema: "common_1", Table: "uk_without_pk", TableID: 47}, + PreColumns: []*model.Column{nil, { + Name: "a1", + Type: mysql.TypeLong, + Flag: model.BinaryFlag | model.MultipleKeyFlag | model.HandleKeyFlag, + Value: 12, + }, { + Name: "a3", + Type: mysql.TypeLong, + Flag: model.BinaryFlag | model.MultipleKeyFlag | model.HandleKeyFlag, + Value: 1, + }}, + IndexColumns: [][]int{{1, 2}}, + }, { + StartTs: 418658114257813514, + CommitTs: 418658114257813515, + Table: &model.TableName{Schema: "common_1", Table: "uk_without_pk", TableID: 47}, + PreColumns: []*model.Column{nil, { + Name: "a1", + Type: mysql.TypeLong, + Flag: model.BinaryFlag | model.MultipleKeyFlag | model.HandleKeyFlag, + Value: 1, + }, { + Name: "a3", + Type: mysql.TypeLong, + Flag: model.BinaryFlag | model.MultipleKeyFlag | model.HandleKeyFlag, + Value: 21, + }}, + IndexColumns: [][]int{{1, 2}}, + }, + }, + }, + expected: [][]byte{ + {'1', '2', 0x0, '1', 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 47}, + {'1', 0x0, '2', '1', 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 47}, + }, + }, { + txn: &model.SingleTableTxn{ + Rows: []*model.RowChangedEvent{ + { + StartTs: 418658114257813514, + CommitTs: 418658114257813515, + Table: &model.TableName{Schema: "common_1", Table: "uk_without_pk", TableID: 47}, + PreColumns: []*model.Column{nil, { + Name: "a1", + Type: mysql.TypeLong, + Flag: model.BinaryFlag | model.HandleKeyFlag, + Value: 12, + }, { + Name: "a3", + Type: mysql.TypeLong, + Flag: model.BinaryFlag | model.HandleKeyFlag, + Value: 1, + }}, + IndexColumns: [][]int{{1}, {2}}, + }, { + StartTs: 418658114257813514, + CommitTs: 418658114257813515, + Table: &model.TableName{Schema: "common_1", Table: "uk_without_pk", TableID: 47}, + PreColumns: []*model.Column{nil, { + Name: "a1", + Type: mysql.TypeLong, + Flag: model.BinaryFlag | model.HandleKeyFlag, + Value: 1, + }, { + Name: "a3", + Type: mysql.TypeLong, + Flag: model.BinaryFlag | model.HandleKeyFlag, + Value: 21, + }}, + IndexColumns: [][]int{{1}, {2}}, + }, + }, + }, + expected: [][]byte{ + {'2', '1', 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 1, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 47}, + {'1', '2', 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 47}, + {'1', 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 1, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 47}, + {'1', 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 47}, + }, + }, { + txn: &model.SingleTableTxn{ + Rows: []*model.RowChangedEvent{ + { + StartTs: 418658114257813514, + CommitTs: 418658114257813515, + Table: &model.TableName{Schema: "common_1", Table: "uk_without_pk", TableID: 47}, + PreColumns: []*model.Column{nil, { + Name: "a1", + Type: mysql.TypeLong, + Flag: model.BinaryFlag | model.NullableFlag, + Value: nil, + }, { + Name: "a3", + Type: mysql.TypeLong, + Flag: model.BinaryFlag | model.NullableFlag, + Value: nil, + }}, + IndexColumns: [][]int{{1}, {2}}, + }, { + StartTs: 418658114257813514, + CommitTs: 418658114257813515, + Table: &model.TableName{Schema: "common_1", Table: "uk_without_pk", TableID: 47}, + PreColumns: []*model.Column{nil, { + Name: "a1", + Type: mysql.TypeLong, + Flag: model.BinaryFlag | model.HandleKeyFlag, + Value: 1, + }, { + Name: "a3", + Type: mysql.TypeLong, + Flag: model.BinaryFlag | model.HandleKeyFlag, + Value: 21, + }}, + IndexColumns: [][]int{{1}, {2}}, + }, + }, + }, + expected: [][]uint8{ + {'2', '1', 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 1, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 47}, + {'1', 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 47}, + {0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 47}, + }, + }} + for _, tc := range testCases { + keys := genTxnKeys(tc.txn) + sort.Slice(keys, func(i, j int) bool { + return bytes.Compare(keys[i], keys[j]) > 0 + }) + c.Assert(keys, check.DeepEquals, tc.expected) + } +} diff --git a/cdc/sink/cdclog/file.go b/cdc/sink/cdclog/file.go new file mode 100644 index 00000000000..872a863a1a0 --- /dev/null +++ b/cdc/sink/cdclog/file.go @@ -0,0 +1,385 @@ +// Copyright 2020 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package cdclog + +import ( + "context" + "net/url" + "os" + "path/filepath" + "sync" + "time" + + "github.com/pingcap/errors" + "github.com/pingcap/log" + "github.com/uber-go/atomic" + "go.uber.org/zap" + "golang.org/x/sync/errgroup" + + parsemodel "github.com/pingcap/parser/model" + + "github.com/pingcap/ticdc/cdc/model" + "github.com/pingcap/ticdc/cdc/sink/codec" +) + +const ( + defaultDirMode = 0755 + defaultFileMode = 0644 + + defaultFileName = "cdclog" + + maxRowFileSize = 10 << 20 // TODO update +) + +type logPath struct { + root string + ddl string + meta string +} + +type tableStream struct { + dataCh chan *model.RowChangedEvent + rowFile *os.File + + encoder codec.EventBatchEncoder + + tableID int64 + sendEvents *atomic.Int64 + sendSize *atomic.Int64 +} + +func newTableStream(tableID int64) *tableStream { + return &tableStream{ + tableID: tableID, + dataCh: make(chan *model.RowChangedEvent, defaultBufferChanSize), + + sendEvents: atomic.NewInt64(0), + sendSize: atomic.NewInt64(0), + } +} + +type fileSink struct { + logMeta *logMeta + logPath *logPath + + ddlFile *os.File + encoder func() codec.EventBatchEncoder + + ddlEncoder codec.EventBatchEncoder + hashMap sync.Map + tableStreams []*tableStream +} + +func (f *fileSink) flushLogMeta(ctx context.Context) error { + data, err := f.logMeta.Marshal() + if err != nil { + return errors.Annotate(err, "marshal meta to json failed") + } + // FIXME: if initialize succeed, O_WRONLY is enough, but sometimes it will failed + file, err := os.OpenFile(f.logPath.meta, os.O_CREATE|os.O_WRONLY, defaultFileMode) + if err != nil { + return err + } + _, err = file.Write(data) + return err +} + +func (f *fileSink) flushTableStreams(ctx context.Context) error { + // TODO use a fixed worker pool + eg, _ := errgroup.WithContext(ctx) + for _, ts := range f.tableStreams { + tsReplica := ts + eg.Go(func() error { + var fileName string + flushedEvents := tsReplica.sendEvents.Load() + flushedSize := tsReplica.sendSize.Load() + firstCreated := false + if tsReplica.encoder == nil { + // create encoder for each file + tsReplica.encoder = f.encoder() + firstCreated = true + } + for event := int64(0); event < flushedEvents; event++ { + row := <-tsReplica.dataCh + if event == flushedEvents-1 { + // the last event + fileName = makeTableFileName(row.CommitTs) + } + _, err := tsReplica.encoder.AppendRowChangedEvent(row) + if err != nil { + return err + } + } + rowDatas := tsReplica.encoder.MixedBuild(firstCreated) + defer func() { + if tsReplica.encoder != nil { + tsReplica.encoder.Reset() + } + }() + + log.Debug("[flushTableStreams] build cdc log data", + zap.Int64("table id", tsReplica.tableID), + zap.Int64("flushed size", flushedSize), + zap.Int64("flushed event", flushedEvents), + zap.Int("encode size", len(rowDatas)), + zap.String("file name", fileName), + ) + + tableDir := filepath.Join(f.logPath.root, makeTableDirectoryName(tsReplica.tableID)) + + if tsReplica.rowFile == nil { + // create new file to append data + err := os.MkdirAll(tableDir, defaultDirMode) + if err != nil { + return err + } + file, err := os.OpenFile(filepath.Join(tableDir, defaultFileName), os.O_CREATE|os.O_WRONLY|os.O_APPEND, defaultFileMode) + if err != nil { + return err + } + tsReplica.rowFile = file + } + + stat, err := tsReplica.rowFile.Stat() + if err != nil { + return err + } + + if stat.Size() > maxRowFileSize { + // rotate file + err := tsReplica.rowFile.Close() + if err != nil { + return err + } + oldPath := filepath.Join(tableDir, defaultFileName) + newPath := filepath.Join(tableDir, fileName) + err = os.Rename(oldPath, newPath) + if err != nil { + return err + } + file, err := os.OpenFile(filepath.Join(tableDir, defaultFileName), os.O_CREATE|os.O_WRONLY|os.O_APPEND, defaultFileMode) + if err != nil { + return err + } + tsReplica.rowFile = file + tsReplica.encoder = nil + } + _, err = tsReplica.rowFile.Write(rowDatas) + if err != nil { + return err + } + + tsReplica.sendEvents.Sub(flushedEvents) + tsReplica.sendSize.Sub(flushedSize) + return nil + }) + } + return eg.Wait() +} + +func (f *fileSink) createDDLFile(commitTs uint64) (*os.File, error) { + fileName := makeDDLFileName(commitTs) + file, err := os.OpenFile(filepath.Join(f.logPath.ddl, fileName), os.O_CREATE|os.O_WRONLY|os.O_APPEND, defaultFileMode) + if err != nil { + log.Error("[EmitDDLEvent] create ddl file failed", zap.Error(err)) + return nil, err + } + return file, err +} + +func (f *fileSink) EmitRowChangedEvents(ctx context.Context, rows ...*model.RowChangedEvent) error { + for _, row := range rows { + // dispatch row event by tableID + tableID := row.Table.GetTableID() + var ( + ok bool + item interface{} + hash int + ) + if item, ok = f.hashMap.Load(tableID); !ok { + // found new tableID + f.tableStreams = append(f.tableStreams, newTableStream(tableID)) + hash = len(f.tableStreams) - 1 + f.hashMap.Store(tableID, hash) + } else { + hash = item.(int) + } + select { + case <-ctx.Done(): + return ctx.Err() + case f.tableStreams[hash].dataCh <- row: + f.tableStreams[hash].sendEvents.Inc() + f.tableStreams[hash].sendSize.Add(row.ApproximateSize) + } + } + return nil +} + +func (f *fileSink) FlushRowChangedEvents(ctx context.Context, resolvedTs uint64) (uint64, error) { + log.Debug("[FlushRowChangedEvents] enter", zap.Uint64("ts", resolvedTs)) + // TODO update flush policy with size + select { + case <-ctx.Done(): + return 0, ctx.Err() + case <-time.After(defaultFlushRowChangedEventDuration): + return resolvedTs, f.flushTableStreams(ctx) + } +} + +func (f *fileSink) EmitCheckpointTs(ctx context.Context, ts uint64) error { + log.Debug("[EmitCheckpointTs]", zap.Uint64("ts", ts)) + f.logMeta.GlobalResolvedTS = ts + return f.flushLogMeta(ctx) +} + +func (f *fileSink) EmitDDLEvent(ctx context.Context, ddl *model.DDLEvent) error { + switch ddl.Type { + case parsemodel.ActionCreateTable: + f.logMeta.Names[ddl.TableInfo.TableID] = model.QuoteSchema(ddl.TableInfo.Schema, ddl.TableInfo.Table) + err := f.flushLogMeta(ctx) + if err != nil { + return err + } + case parsemodel.ActionRenameTable: + delete(f.logMeta.Names, ddl.PreTableInfo.TableID) + f.logMeta.Names[ddl.TableInfo.TableID] = model.QuoteSchema(ddl.TableInfo.Schema, ddl.TableInfo.Table) + err := f.flushLogMeta(ctx) + if err != nil { + return err + } + } + firstCreated := false + if f.ddlEncoder == nil { + // create ddl encoder once for each ddl log file + f.ddlEncoder = f.encoder() + firstCreated = true + } + _, err := f.ddlEncoder.AppendDDLEvent(ddl) + if err != nil { + return err + } + data := f.ddlEncoder.MixedBuild(firstCreated) + + defer func() { + if f.ddlEncoder != nil { + f.ddlEncoder.Reset() + } + }() + + if f.ddlFile == nil { + // create file stream + file, err := f.createDDLFile(ddl.CommitTs) + if err != nil { + return err + } + f.ddlFile = file + } + + stat, err := f.ddlFile.Stat() + if err != nil { + return err + } + + log.Debug("[EmitDDLEvent] current file stats", + zap.String("name", stat.Name()), + zap.Int64("size", stat.Size()), + zap.Int("data size", len(data)), + ) + + if stat.Size() > maxDDLFlushSize { + // rotate file + err = f.ddlFile.Close() + if err != nil { + return err + } + file, err := f.createDDLFile(ddl.CommitTs) + if err != nil { + return err + } + f.ddlFile = file + // reset ddl encoder for new file + f.ddlEncoder = nil + } + + _, err = f.ddlFile.Write(data) + if err != nil { + return err + } + return nil +} + +func (f *fileSink) Initialize(ctx context.Context, tableInfo []*model.SimpleTableInfo) error { + if tableInfo != nil { + for _, table := range tableInfo { + if table != nil { + name := makeTableDirectoryName(table.TableID) + err := os.MkdirAll(filepath.Join(f.logPath.root, name), defaultDirMode) + if err != nil { + return errors.Annotatef(err, "create table directory for %s on failed", name) + } + } + } + // update log meta to record the relationship about tableName and tableID + f.logMeta = makeLogMetaContent(tableInfo) + data, err := f.logMeta.Marshal() + if err != nil { + return errors.Annotate(err, "marshal meta to json failed") + } + filePath := f.logPath.meta + if _, err := os.Stat(filePath); !os.IsNotExist(err) { + return errors.Annotate(err, "meta file already exists, please change the path of this sink") + } + file, err := os.Create(filePath) + if err != nil { + return err + } + _, err = file.Write(data) + if err != nil { + return err + } + } + return nil +} + +func (f *fileSink) Close() error { + return nil +} + +// NewLocalFileSink support log data to file. +func NewLocalFileSink(sinkURI *url.URL) (*fileSink, error) { + log.Info("[NewLocalFileSink]", + zap.String("host", sinkURI.Host), + zap.String("path", sinkURI.Path), + ) + rootPath := sinkURI.Path + "/" + logPath := &logPath{ + root: rootPath, + meta: rootPath + logMetaFile, + ddl: rootPath + ddlEventsDir, + } + err := os.MkdirAll(logPath.ddl, defaultDirMode) + if err != nil { + log.Error("create ddl path failed", + zap.String("ddl path", logPath.ddl), + zap.Error(err)) + return nil, err + } + return &fileSink{ + logMeta: newLogMeta(), + logPath: logPath, + encoder: codec.NewJSONEventBatchEncoder, + + tableStreams: make([]*tableStream, 0), + }, nil +} diff --git a/cdc/sink/cdclog/s3.go b/cdc/sink/cdclog/s3.go new file mode 100644 index 00000000000..fbfe5c50333 --- /dev/null +++ b/cdc/sink/cdclog/s3.go @@ -0,0 +1,417 @@ +// Copyright 2020 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package cdclog + +import ( + "context" + "net/url" + "strings" + "sync" + "time" + + "github.com/pingcap/br/pkg/storage" + "github.com/pingcap/errors" + "github.com/pingcap/kvproto/pkg/backup" + "github.com/pingcap/log" + parsemodel "github.com/pingcap/parser/model" + "github.com/uber-go/atomic" + "go.uber.org/zap" + "golang.org/x/sync/errgroup" + + "github.com/pingcap/ticdc/cdc/model" + "github.com/pingcap/ticdc/cdc/sink/codec" +) + +const ( + maxNotifySize = 15 << 20 // trigger flush if one table has reached 16Mb data size in memory + maxPartFlushSize = 5 << 20 // The minimal multipart upload size is 5Mb. + maxCompletePartSize = 100 << 20 // rotate row changed event file if one complete file larger than 100Mb + maxDDLFlushSize = 10 << 20 // rotate ddl event file if one complete file larger than 10Mb + + defaultBufferChanSize = 20480 + defaultFlushRowChangedEventDuration = 5 * time.Second // TODO make it as a config +) + +type tableBuffer struct { + // for log + tableID int64 + dataCh chan *model.RowChangedEvent + sendSize *atomic.Int64 + sendEvents *atomic.Int64 + + encoder codec.EventBatchEncoder + + uploadParts struct { + uploader storage.Uploader + uploadNum int + byteSize int64 + } +} + +func (tb *tableBuffer) IsEmpty() bool { + return tb.sendEvents.Load() == 0 && tb.uploadParts.uploadNum == 0 +} + +func (tb *tableBuffer) flush(ctx context.Context, s *s3Sink) error { + hashPart := tb.uploadParts + sendEvents := tb.sendEvents.Load() + if sendEvents == 0 && hashPart.uploadNum == 0 { + log.Info("nothing to flush", zap.Int64("tableID", tb.tableID)) + return nil + } + + firstCreated := false + if tb.encoder == nil { + // create encoder for each file + tb.encoder = s.encoder() + firstCreated = true + } + + var newFileName string + flushedSize := int64(0) + for event := int64(0); event < sendEvents; event++ { + row := <-tb.dataCh + flushedSize += row.ApproximateSize + if event == sendEvents-1 { + // if last event, we record ts as new rotate file name + newFileName = makeTableFileObject(row.Table.TableID, row.CommitTs) + } + _, err := tb.encoder.AppendRowChangedEvent(row) + if err != nil { + return err + } + } + rowDatas := tb.encoder.MixedBuild(firstCreated) + // reset encoder buf for next round append + defer func() { + if tb.encoder != nil { + tb.encoder.Reset() + } + }() + + log.Debug("[FlushRowChangedEvents[Debug]] flush table buffer", + zap.Int64("table", tb.tableID), + zap.Int64("event size", sendEvents), + zap.Int("row data size", len(rowDatas)), + zap.Int("upload num", hashPart.uploadNum), + zap.Int64("upload byte size", hashPart.byteSize), + // zap.ByteString("rowDatas", rowDatas), + ) + + if len(rowDatas) > 0 { + if len(rowDatas) > maxPartFlushSize || hashPart.uploadNum > 0 { + // S3 multi-upload need every chunk(except the last one) is greater than 5Mb + // so, if this batch data size is greater than 5Mb or it has uploadPart already + // we will use multi-upload this batch data + if hashPart.uploader == nil { + uploader, err := s.storage.CreateUploader(ctx, newFileName) + if err != nil { + return err + } + hashPart.uploader = uploader + } + + err := hashPart.uploader.UploadPart(ctx, rowDatas) + if err != nil { + return err + } + + hashPart.byteSize += int64(len(rowDatas)) + hashPart.uploadNum++ + + if hashPart.byteSize > maxCompletePartSize || len(rowDatas) <= maxPartFlushSize { + // we need do complete when total upload size is greater than 100Mb + // or this part data is less than 5Mb to avoid meet EntityTooSmall error + log.Info("[FlushRowChangedEvents] complete file", zap.Int64("tableID", tb.tableID)) + err = hashPart.uploader.CompleteUpload(ctx) + if err != nil { + return err + } + hashPart.byteSize = 0 + hashPart.uploadNum = 0 + hashPart.uploader = nil + tb.encoder = nil + } + } else { + // generate normal file because S3 multi-upload need every part at least 5Mb. + log.Info("[FlushRowChangedEvents] normal upload file", zap.Int64("tableID", tb.tableID)) + err := s.storage.Write(ctx, newFileName, rowDatas) + if err != nil { + return err + } + tb.encoder = nil + } + } + + tb.sendEvents.Sub(sendEvents) + tb.sendSize.Sub(flushedSize) + tb.uploadParts = hashPart + return nil +} + +func newTableBuffer(tableID int64) *tableBuffer { + return &tableBuffer{ + tableID: tableID, + dataCh: make(chan *model.RowChangedEvent, defaultBufferChanSize), + sendSize: atomic.NewInt64(0), + sendEvents: atomic.NewInt64(0), + uploadParts: struct { + uploader storage.Uploader + uploadNum int + byteSize int64 + }{ + uploader: nil, + uploadNum: 0, + byteSize: 0, + }, + } +} + +type s3Sink struct { + prefix string + + storage *storage.S3Storage + + logMeta *logMeta + encoder func() codec.EventBatchEncoder + + // hold encoder for ddl event log + ddlEncoder codec.EventBatchEncoder + hashMap sync.Map + tableBuffers []*tableBuffer + notifyChan chan struct{} +} + +func (s *s3Sink) EmitRowChangedEvents(ctx context.Context, rows ...*model.RowChangedEvent) error { + shouldFlush := false + for _, row := range rows { + // dispatch row event by tableID + tableID := row.Table.GetTableID() + var ( + ok bool + item interface{} + hash int + ) + if item, ok = s.hashMap.Load(tableID); !ok { + // found new tableID + s.tableBuffers = append(s.tableBuffers, newTableBuffer(tableID)) + hash = len(s.tableBuffers) - 1 + s.hashMap.Store(tableID, hash) + } else { + hash = item.(int) + } + select { + case <-ctx.Done(): + return ctx.Err() + case s.tableBuffers[hash].dataCh <- row: + s.tableBuffers[hash].sendSize.Add(row.ApproximateSize) + if s.tableBuffers[hash].sendSize.Load() > maxNotifySize { + // trigger flush when a table has maxNotifySize + shouldFlush = true + } + s.tableBuffers[hash].sendEvents.Inc() + } + } + if shouldFlush { + // should not block here + select { + case s.notifyChan <- struct{}{}: + default: + } + } + return nil +} + +func (s *s3Sink) flushLogMeta(ctx context.Context) error { + data, err := s.logMeta.Marshal() + if err != nil { + return errors.Annotate(err, "marshal meta to json failed") + } + return s.storage.Write(ctx, logMetaFile, data) +} + +func (s *s3Sink) flushTableBuffers(ctx context.Context) error { + // TODO use a fixed worker pool + eg, ectx := errgroup.WithContext(ctx) + for _, tb := range s.tableBuffers { + if tb.IsEmpty() { + continue + } + tbReplica := tb + eg.Go(func() error { + log.Info("[FlushRowChangedEvents] flush specify row changed event", + zap.Int64("table", tbReplica.tableID), + zap.Int64("event size", tbReplica.sendEvents.Load())) + return tbReplica.flush(ectx, s) + }) + } + return eg.Wait() +} + +func (s *s3Sink) FlushRowChangedEvents(ctx context.Context, resolvedTs uint64) (uint64, error) { + // we should flush all events before resolvedTs, there are two kind of flush policy + // 1. flush row events to a s3 chunk: if the event size is not enough, + // TODO: when cdc crashed, we should repair these chunks to a complete file + // 2. flush row events to a complete s3 file: if the event size is enough + select { + case <-ctx.Done(): + return 0, ctx.Err() + case <-s.notifyChan: + return resolvedTs, s.flushTableBuffers(ctx) + + case <-time.After(defaultFlushRowChangedEventDuration): + // cannot accumulate enough row events in 10 second + // flush all tables' row events to s3 + return resolvedTs, s.flushTableBuffers(ctx) + } +} + +// EmitCheckpointTs update the global resolved ts in log meta +// sleep 5 seconds to avoid update too frequently +func (s *s3Sink) EmitCheckpointTs(ctx context.Context, ts uint64) error { + s.logMeta.GlobalResolvedTS = ts + return s.flushLogMeta(ctx) +} + +// EmitDDLEvent write ddl event to S3 directory, all events split by '\n' +// Because S3 doesn't support append-like write. +// we choose a hack way to read origin file then write in place. +func (s *s3Sink) EmitDDLEvent(ctx context.Context, ddl *model.DDLEvent) error { + switch ddl.Type { + case parsemodel.ActionCreateTable: + s.logMeta.Names[ddl.TableInfo.TableID] = model.QuoteSchema(ddl.TableInfo.Schema, ddl.TableInfo.Table) + err := s.flushLogMeta(ctx) + if err != nil { + return err + } + case parsemodel.ActionRenameTable: + delete(s.logMeta.Names, ddl.PreTableInfo.TableID) + s.logMeta.Names[ddl.TableInfo.TableID] = model.QuoteSchema(ddl.TableInfo.Schema, ddl.TableInfo.Table) + err := s.flushLogMeta(ctx) + if err != nil { + return err + } + } + firstCreated := false + if s.ddlEncoder == nil { + s.ddlEncoder = s.encoder() + firstCreated = true + } + _, err := s.ddlEncoder.AppendDDLEvent(ddl) + if err != nil { + return err + } + data := s.ddlEncoder.MixedBuild(firstCreated) + // reset encoder buf for next round append + defer s.ddlEncoder.Reset() + + var ( + name string + size int64 + fileData []byte + ) + err = s.storage.WalkDir(ctx, ddlEventsDir, 1, func(key string, fileSize int64) error { + log.Debug("[EmitDDLEvent] list content from s3", + zap.String("key", key), + zap.Int64("size", size), + zap.Any("ddl", ddl)) + name = strings.ReplaceAll(key, s.prefix, "") + size = fileSize + return nil + }) + if err != nil { + return err + } + if size == 0 || size > maxDDLFlushSize { + // no ddl file exists or + // exists file is oversized. we should generate a new file + fileData = data + name = makeDDLFileObject(ddl.CommitTs) + log.Debug("[EmitDDLEvent] create first or rotate ddl log", + zap.String("name", name), zap.Any("ddl", ddl)) + if size > maxDDLFlushSize { + // reset ddl encoder for new file + s.ddlEncoder = nil + } + } else { + // hack way: append data to old file + log.Debug("[EmitDDLEvent] append ddl to origin log", + zap.String("name", name), zap.Any("ddl", ddl)) + fileData, err = s.storage.Read(ctx, name) + if err != nil { + return err + } + fileData = append(fileData, data...) + } + return s.storage.Write(ctx, name, fileData) +} + +func (s *s3Sink) Initialize(ctx context.Context, tableInfo []*model.SimpleTableInfo) error { + if tableInfo != nil { + for _, table := range tableInfo { + if table != nil { + err := s.storage.Write(ctx, makeTableDirectoryName(table.TableID), nil) + if err != nil { + return errors.Annotate(err, "create table directory on s3 failed") + } + } + } + // update log meta to record the relationship about tableName and tableID + s.logMeta = makeLogMetaContent(tableInfo) + + data, err := s.logMeta.Marshal() + if err != nil { + return errors.Annotate(err, "marshal meta to json failed") + } + return s.storage.Write(ctx, logMetaFile, data) + } + return nil +} + +func (s *s3Sink) Close() error { + return nil +} + +// NewS3Sink creates new sink support log data to s3 directly +func NewS3Sink(sinkURI *url.URL) (*s3Sink, error) { + if len(sinkURI.Host) == 0 { + return nil, errors.Errorf("please specify the bucket for s3 in %s", sinkURI) + } + prefix := strings.Trim(sinkURI.Path, "/") + s3 := &backup.S3{Bucket: sinkURI.Host, Prefix: prefix} + options := &storage.BackendOptions{} + storage.ExtractQueryParameters(sinkURI, &options.S3) + if err := options.S3.Apply(s3); err != nil { + return nil, err + } + // we should set this to true, since br set it by default in parseBackend + s3.ForcePathStyle = true + + s3storage, err := storage.NewS3Storage(s3, false) + if err != nil { + return nil, err + } + + notifyChan := make(chan struct{}) + tableBuffers := make([]*tableBuffer, 0) + return &s3Sink{ + prefix: prefix, + storage: s3storage, + logMeta: newLogMeta(), + encoder: codec.NewJSONEventBatchEncoder, + + tableBuffers: tableBuffers, + notifyChan: notifyChan, + }, nil +} diff --git a/cdc/sink/cdclog/utils.go b/cdc/sink/cdclog/utils.go new file mode 100644 index 00000000000..e79cf41013b --- /dev/null +++ b/cdc/sink/cdclog/utils.go @@ -0,0 +1,83 @@ +// Copyright 2020 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package cdclog + +import ( + "encoding/json" + "fmt" + + "github.com/pingcap/log" + "go.uber.org/zap" + + "github.com/pingcap/ticdc/cdc/model" +) + +const ( + tablePrefix = "t_" + logMetaFile = "log.meta" + + ddlEventsDir = "ddls" + ddlEventsPrefix = "ddl" + + maxUint64 = ^uint64(0) +) + +type logMeta struct { + Names map[int64]string `json:"names"` + GlobalResolvedTS uint64 `json:"global_resolved_ts"` +} + +func newLogMeta() *logMeta { + return &logMeta{ + Names: make(map[int64]string), + } +} + +// Marshal saves logMeta +func (l *logMeta) Marshal() ([]byte, error) { + return json.Marshal(l) +} + +func makeTableDirectoryName(tableID int64) string { + return fmt.Sprintf("%s%d", tablePrefix, tableID) +} + +func makeTableFileObject(tableID int64, commitTS uint64) string { + return fmt.Sprintf("%s%d/%s", tablePrefix, tableID, makeTableFileName(commitTS)) +} + +func makeTableFileName(commitTS uint64) string { + return fmt.Sprintf("cdclog.%d", commitTS) +} + +func makeLogMetaContent(tableInfos []*model.SimpleTableInfo) *logMeta { + meta := new(logMeta) + names := make(map[int64]string) + for _, table := range tableInfos { + if table != nil { + log.Info("[makeLogMetaContent]", zap.Reflect("table", table)) + names[table.TableID] = model.QuoteSchema(table.Schema, table.Table) + } + } + meta.Names = names + return meta +} + +func makeDDLFileObject(commitTS uint64) string { + return fmt.Sprintf("%s/%s", ddlEventsDir, makeDDLFileName(commitTS)) +} + +func makeDDLFileName(commitTS uint64) string { + return fmt.Sprintf("%s.%d", ddlEventsPrefix, maxUint64-commitTS) +} diff --git a/cdc/sink/codec/avro.go b/cdc/sink/codec/avro.go index 4e66de50fbb..04975edfa29 100644 --- a/cdc/sink/codec/avro.go +++ b/cdc/sink/codec/avro.go @@ -20,11 +20,10 @@ import ( "encoding/json" "fmt" "math" - "math/rand" + "math/big" "strconv" "time" - "github.com/linkedin/goavro/v2" "github.com/pingcap/errors" "github.com/pingcap/log" "github.com/pingcap/parser/mysql" @@ -36,8 +35,7 @@ import ( // AvroEventBatchEncoder converts the events to binary Avro data type AvroEventBatchEncoder struct { - // TODO use Avro for Kafka keys - // keySchemaManager *AvroSchemaManager + keySchemaManager *AvroSchemaManager valueSchemaManager *AvroSchemaManager keyBuf []byte valueBuf []byte @@ -52,6 +50,7 @@ type avroEncodeResult struct { func NewAvroEventBatchEncoder() EventBatchEncoder { return &AvroEventBatchEncoder{ valueSchemaManager: nil, + keySchemaManager: nil, keyBuf: nil, valueBuf: nil, } @@ -67,14 +66,48 @@ func (a *AvroEventBatchEncoder) GetValueSchemaManager() *AvroSchemaManager { return a.valueSchemaManager } +// SetKeySchemaManager sets the value schema manager for an Avro encoder +func (a *AvroEventBatchEncoder) SetKeySchemaManager(manager *AvroSchemaManager) { + a.keySchemaManager = manager +} + +// GetKeySchemaManager gets the value schema manager for an Avro encoder +func (a *AvroEventBatchEncoder) GetKeySchemaManager() *AvroSchemaManager { + return a.keySchemaManager +} + // AppendRowChangedEvent appends a row change event to the encoder // NOTE: the encoder can only store one RowChangedEvent! func (a *AvroEventBatchEncoder) AppendRowChangedEvent(e *model.RowChangedEvent) (EncoderResult, error) { if a.keyBuf != nil || a.valueBuf != nil { return EncoderNoOperation, errors.New("Fatal sink bug. Batch size must be 1") } + if !e.IsDelete() { + res, err := avroEncode(e.Table, a.valueSchemaManager, e.TableInfoVersion, e.Columns) + if err != nil { + log.Warn("AppendRowChangedEvent: avro encoding failed", zap.String("table", e.Table.String())) + return EncoderNoOperation, errors.Annotate(err, "AppendRowChangedEvent could not encode to Avro") + } + + evlp, err := res.toEnvelope() + if err != nil { + log.Warn("AppendRowChangedEvent: could not construct Avro envelope", zap.String("table", e.Table.String())) + return EncoderNoOperation, errors.Annotate(err, "AppendRowChangedEvent could not construct Avro envelope") + } + + a.valueBuf = evlp + } else { + a.valueBuf = nil + } + + pkeyCols := make([]*model.Column, 0) + for _, col := range e.Columns { + if col.Flag.IsHandleKey() { + pkeyCols = append(pkeyCols, col) + } + } - res, err := a.avroEncode(e.Table, e.TableInfoVersion, e.Columns) + res, err := avroEncode(e.Table, a.keySchemaManager, e.TableInfoVersion, pkeyCols) if err != nil { log.Warn("AppendRowChangedEvent: avro encoding failed", zap.String("table", e.Table.String())) return EncoderNoOperation, errors.Annotate(err, "AppendRowChangedEvent could not encode to Avro") @@ -86,9 +119,7 @@ func (a *AvroEventBatchEncoder) AppendRowChangedEvent(e *model.RowChangedEvent) return EncoderNoOperation, errors.Annotate(err, "AppendRowChangedEvent could not construct Avro envelope") } - a.valueBuf = evlp - // TODO use primary key(s) as kafka key - a.keyBuf = []byte(strconv.FormatInt(e.RowID, 10)) + a.keyBuf = evlp return EncoderNeedAsyncWrite, nil } @@ -99,33 +130,8 @@ func (a *AvroEventBatchEncoder) AppendResolvedEvent(ts uint64) (EncoderResult, e return EncoderNoOperation, nil } -// AppendDDLEvent generates new schema and registers it to the Registry +// AppendDDLEvent is no-op now func (a *AvroEventBatchEncoder) AppendDDLEvent(e *model.DDLEvent) (EncoderResult, error) { - if e.TableInfo == nil { - log.Info("AppendDDLEvent: no schema generation needed, skip") - return EncoderNoOperation, nil - } - - schemaStr, err := ColumnInfoToAvroSchema(e.TableInfo.Table, e.TableInfo.ColumnInfo) - if err != nil { - return EncoderNoOperation, errors.Annotate(err, "AppendDDLEvent failed") - } - log.Info("AppendDDLEvent: new schema generated", zap.String("schema_str", schemaStr)) - - avroCodec, err := goavro.NewCodec(schemaStr) - if err != nil { - return EncoderNoOperation, errors.Annotate(err, "AppendDDLEvent failed: could not verify schema, probably bug") - } - - err = a.valueSchemaManager.Register(context.Background(), model.TableName{ - Schema: e.TableInfo.Schema, - Table: e.TableInfo.Table, - }, avroCodec) - - if err != nil { - return EncoderNoOperation, errors.Annotate(err, "AppendDDLEvent failed: could not register schema") - } - return EncoderNoOperation, nil } @@ -138,6 +144,16 @@ func (a *AvroEventBatchEncoder) Build() (key []byte, value []byte) { return k, v } +// MixedBuild implements the EventBatchEncoder interface +func (a *AvroEventBatchEncoder) MixedBuild(withVersion bool) []byte { + panic("Mixed Build only use for JsonEncoder") +} + +// Reset implements the EventBatchEncoder interface +func (a *AvroEventBatchEncoder) Reset() { + panic("Reset only used for JsonEncoder") +} + // Size is always 0 or 1 func (a *AvroEventBatchEncoder) Size() int { if a.valueBuf == nil { @@ -146,10 +162,19 @@ func (a *AvroEventBatchEncoder) Size() int { return 1 } -func (a *AvroEventBatchEncoder) avroEncode(table *model.TableName, tableVersion uint64, cols map[string]*model.Column) (*avroEncodeResult, error) { - avroCodec, registryID, err := a.valueSchemaManager.Lookup(context.Background(), *table, tableVersion) +func avroEncode(table *model.TableName, manager *AvroSchemaManager, tableVersion uint64, cols []*model.Column) (*avroEncodeResult, error) { + schemaGen := func() (string, error) { + schema, err := ColumnInfoToAvroSchema(table.Table, cols) + if err != nil { + return "", errors.Annotate(err, "AvroEventBatchEncoder: generating schema failed") + } + return schema, nil + } + + // TODO pass ctx from the upper function. Need to modify the EventBatchEncoder interface. + avroCodec, registryID, err := manager.GetCachedOrRegister(context.Background(), *table, tableVersion, schemaGen) if err != nil { - return nil, errors.Annotate(err, "AvroEventBatchEncoder: lookup failed") + return nil, errors.Annotate(err, "AvroEventBatchEncoder: get-or-register failed") } native, err := rowToAvroNativeData(cols) @@ -169,16 +194,9 @@ func (a *AvroEventBatchEncoder) avroEncode(table *model.TableName, tableVersion } type avroSchemaTop struct { - Tp string `json:"type"` - Name string `json:"name"` - Fields []avroSchemaField `json:"fields"` -} - -type avroSchemaField struct { - Name string `json:"name"` - // Tp can be a string or an avroLogicalType - Tp []interface{} `json:"type"` - DefaultValue interface{} `json:"default"` + Tp string `json:"type"` + Name string `json:"name"` + Fields []map[string]interface{} `json:"fields"` } type logicalType string @@ -186,32 +204,38 @@ type logicalType string type avroLogicalType struct { Type string `json:"type"` LogicalType logicalType `json:"logicalType"` + Precision interface{} `json:"precision,omitempty"` + Scale interface{} `json:"scale,omitempty"` } const ( timestampMillis logicalType = "timestamp-millis" - timeMicros logicalType = "time-micros" + timeMillis logicalType = "time-millis" + decimalType logicalType = "decimal" ) // ColumnInfoToAvroSchema generates the Avro schema JSON for the corresponding columns -func ColumnInfoToAvroSchema(name string, columnInfo []*model.ColumnInfo) (string, error) { +func ColumnInfoToAvroSchema(name string, columnInfo []*model.Column) (string, error) { top := avroSchemaTop{ Tp: "record", - Name: name + "_" + strconv.FormatInt(rand.Int63(), 10), + Name: name, Fields: nil, } for _, col := range columnInfo { - avroType, err := getAvroDataTypeNameMysql(col.Type) + avroType, err := getAvroDataTypeFromColumn(col) if err != nil { return "", err } - - field := avroSchemaField{ - Name: col.Name, - Tp: []interface{}{"null", avroType}, - DefaultValue: nil, + field := make(map[string]interface{}) + field["name"] = col.Name + if col.Flag.IsHandleKey() { + field["type"] = avroType + } else { + field["type"] = []interface{}{"null", avroType} + field["default"] = nil } + top.Fields = append(top.Fields, field) } @@ -219,25 +243,33 @@ func ColumnInfoToAvroSchema(name string, columnInfo []*model.ColumnInfo) (string if err != nil { return "", errors.Annotate(err, "ColumnInfoToAvroSchema: failed to generate json") } + log.Debug("Avro Schema JSON generated", zap.ByteString("schema", str)) return string(str), nil } -func rowToAvroNativeData(cols map[string]*model.Column) (interface{}, error) { +func rowToAvroNativeData(cols []*model.Column) (interface{}, error) { ret := make(map[string]interface{}, len(cols)) - for key, col := range cols { + for _, col := range cols { + if col == nil { + continue + } data, str, err := columnToAvroNativeData(col) if err != nil { return nil, err } + if col.Flag.IsHandleKey() { + ret[col.Name] = data + continue + } union := make(map[string]interface{}, 1) union[str] = data - ret[key] = union + ret[col.Name] = union } return ret, nil } -func getAvroDataTypeName(v interface{}) (string, error) { +func getAvroDataTypeFallback(v interface{}) (string, error) { switch v.(type) { case bool: return "boolean", nil @@ -255,18 +287,22 @@ func getAvroDataTypeName(v interface{}) (string, error) { return "null", nil case string: return "string", nil - case time.Duration: - return "long", nil - case time.Time: - return "long", nil default: - log.Warn("getAvroDataTypeName: unknown type") + log.Warn("getAvroDataTypeFallback: unknown type") return "", errors.New("unknown type for Avro") } } -func getAvroDataTypeNameMysql(tp byte) (interface{}, error) { - switch tp { +var unsignedLongAvroType = avroLogicalType{ + Type: "bytes", + LogicalType: decimalType, + Precision: 8, + Scale: 0, +} + +func getAvroDataTypeFromColumn(col *model.Column) (interface{}, error) { + + switch col.Type { case mysql.TypeFloat: return "float", nil case mysql.TypeDouble: @@ -278,22 +314,30 @@ func getAvroDataTypeNameMysql(tp byte) (interface{}, error) { Type: "long", LogicalType: timestampMillis, }, nil - case mysql.TypeDuration: //duration should read fsp from column meta data + case mysql.TypeDuration: return avroLogicalType{ - Type: "long", - LogicalType: timeMicros, + Type: "int", + LogicalType: timeMillis, }, nil case mysql.TypeEnum: - return "long", nil + return unsignedLongAvroType, nil case mysql.TypeSet: - return "long", nil + return unsignedLongAvroType, nil case mysql.TypeBit: - return "long", nil + return unsignedLongAvroType, nil case mysql.TypeNewDecimal: return "string", nil case mysql.TypeTiny, mysql.TypeShort, mysql.TypeInt24: return "int", nil - case mysql.TypeLong, mysql.TypeLonglong: + case mysql.TypeLong: + if col.Flag.IsUnsigned() { + return "long", nil + } + return "int", nil + case mysql.TypeLonglong: + if col.Flag.IsUnsigned() { + return unsignedLongAvroType, nil + } return "long", nil case mysql.TypeNull: return "null", nil @@ -301,20 +345,30 @@ func getAvroDataTypeNameMysql(tp byte) (interface{}, error) { return "string", nil case mysql.TypeTinyBlob, mysql.TypeMediumBlob, mysql.TypeLongBlob, mysql.TypeBlob: return "bytes", nil + case mysql.TypeYear: + return "long", nil default: + log.Fatal("Unknown MySql type", zap.Reflect("mysql-type", col.Type)) return "", errors.New("Unknown Mysql type") } } func columnToAvroNativeData(col *model.Column) (interface{}, string, error) { - if v, ok := col.Value.(int); ok { - col.Value = int64(v) - } - if col.Value == nil { return nil, "null", nil } + handleUnsignedInt64 := func() (interface{}, string, error) { + var retVal interface{} + switch v := col.Value.(type) { + case uint64: + retVal = big.NewRat(0, 1).SetUint64(v) + case int64: + retVal = big.NewRat(0, 1).SetInt64(v) + } + return retVal, string("bytes." + decimalType), nil + } + switch col.Type { case mysql.TypeDate, mysql.TypeDatetime, mysql.TypeNewDate, mysql.TypeTimestamp: str := col.Value.(string) @@ -360,26 +414,34 @@ func columnToAvroNativeData(col *model.Column) (interface{}, string, error) { fracInt = int64(float64(fracInt) * math.Pow10(6-fsp)) d := types.NewDuration(hours, minutes, seconds, int(fracInt), int8(fsp)).Duration - const fullType = "long." + timeMicros + const fullType = "int." + timeMillis return d, string(fullType), nil + case mysql.TypeYear: + return col.Value.(int64), "long", nil case mysql.TypeJSON: return col.Value.(tijson.BinaryJSON).String(), "string", nil case mysql.TypeNewDecimal: - dec := col.Value.(*types.MyDecimal) - if dec == nil { - return nil, "null", nil - } - return dec.String(), "string", nil + return col.Value.(string), "string", nil case mysql.TypeEnum: - return col.Value.(types.Enum).Value, "long", nil + return handleUnsignedInt64() case mysql.TypeSet: - return col.Value.(types.Set).Value, "long", nil + return handleUnsignedInt64() case mysql.TypeBit: - return col.Value.(uint64), "long", nil - case mysql.TypeTiny: - return int32(col.Value.(uint8)), "int", nil + return handleUnsignedInt64() + case mysql.TypeTiny, mysql.TypeShort, mysql.TypeInt24: + return int32(col.Value.(int64)), "int", nil + case mysql.TypeLong: + if col.Flag.IsUnsigned() { + return int64(col.Value.(uint64)), "long", nil + } + return col.Value.(int64), "int", nil + case mysql.TypeLonglong: + if col.Flag.IsUnsigned() { + return handleUnsignedInt64() + } + return col.Value.(int64), "long", nil default: - avroType, err := getAvroDataTypeName(col.Value) + avroType, err := getAvroDataTypeFallback(col.Value) if err != nil { return nil, "", err } diff --git a/cdc/sink/codec/avro_test.go b/cdc/sink/codec/avro_test.go index 52bdd050945..399f58804e9 100644 --- a/cdc/sink/codec/avro_test.go +++ b/cdc/sink/codec/avro_test.go @@ -40,11 +40,15 @@ var _ = check.Suite(&avroBatchEncoderSuite{}) func (s *avroBatchEncoderSuite) SetUpSuite(c *check.C) { startHTTPInterceptForTestingRegistry(c) - manager, err := NewAvroSchemaManager(context.Background(), &security.Credential{}, "http://127.0.0.1:8081", "-value") + keyManager, err := NewAvroSchemaManager(context.Background(), &security.Credential{}, "http://127.0.0.1:8081", "-key") + c.Assert(err, check.IsNil) + + valueManager, err := NewAvroSchemaManager(context.Background(), &security.Credential{}, "http://127.0.0.1:8081", "-value") c.Assert(err, check.IsNil) s.encoder = &AvroEventBatchEncoder{ - valueSchemaManager: manager, + valueSchemaManager: valueManager, + keySchemaManager: keyManager, keyBuf: nil, valueBuf: nil, } @@ -72,21 +76,17 @@ func (s *avroBatchEncoderSuite) TestAvroEncodeOnly(c *check.C) { c.Assert(err, check.IsNil) table := model.TableName{ - Schema: "testdb", - Table: "test1", - Partition: 0, + Schema: "testdb", + Table: "test1", } - err = s.encoder.valueSchemaManager.Register(context.Background(), table, avroCodec) - c.Assert(err, check.IsNil) - - r, err := s.encoder.avroEncode(&table, 1, map[string]*model.Column{ - "id": {Value: int32(1), Type: mysql.TypeLong}, - "myint": {Value: int32(2), Type: mysql.TypeLong}, - "mybool": {Value: uint8(1), Type: mysql.TypeTiny}, - "myfloat": {Value: float32(3.14), Type: mysql.TypeFloat}, - "mybytes": {Value: []byte("Hello World"), Type: mysql.TypeBlob}, - "ts": {Value: time.Now().Format(types.TimeFSPFormat), Type: mysql.TypeTimestamp}, + r, err := avroEncode(&table, s.encoder.valueSchemaManager, 1, []*model.Column{ + {Name: "id", Value: int64(1), Type: mysql.TypeLong}, + {Name: "myint", Value: int64(2), Type: mysql.TypeLong}, + {Name: "mybool", Value: int64(1), Type: mysql.TypeTiny}, + {Name: "myfloat", Value: float32(3.14), Type: mysql.TypeFloat}, + {Name: "mybytes", Value: []byte("Hello World"), Type: mysql.TypeBlob}, + {Name: "ts", Value: time.Now().Format(types.TimeFSPFormat), Type: mysql.TypeTimestamp}, }) c.Assert(err, check.IsNil) @@ -138,19 +138,17 @@ func (s *avroBatchEncoderSuite) TestAvroEnvelope(c *check.C) { } func (s *avroBatchEncoderSuite) TestAvroEncode(c *check.C) { - trueVar := true testCaseUpdate := &model.RowChangedEvent{ CommitTs: 417318403368288260, Table: &model.TableName{ Schema: "test", Table: "person", }, - Delete: false, - Columns: map[string]*model.Column{ - "id": {Type: mysql.TypeLong, WhereHandle: &trueVar, Value: 1}, - "name": {Type: mysql.TypeVarchar, Value: "Bob"}, - "tiny": {Type: mysql.TypeTiny, Value: uint8(255)}, - "comment": {Type: mysql.TypeBlob, Value: []byte("测试")}, + Columns: []*model.Column{ + {Name: "id", Type: mysql.TypeLong, Flag: model.HandleKeyFlag, Value: int64(1)}, + {Name: "name", Type: mysql.TypeVarchar, Value: "Bob"}, + {Name: "tiny", Type: mysql.TypeTiny, Value: int64(255)}, + {Name: "comment", Type: mysql.TypeBlob, Value: []byte("测试")}, }, } diff --git a/cdc/sink/codec/canal.go b/cdc/sink/codec/canal.go index 2147bf33a4b..0e70a48b177 100644 --- a/cdc/sink/codec/canal.go +++ b/cdc/sink/codec/canal.go @@ -44,7 +44,7 @@ func convertToCanalTs(commitTs uint64) int64 { // get the canal EventType according to the RowChangedEvent func convertRowEventType(e *model.RowChangedEvent) canal.EventType { - if e.Delete { + if e.IsDelete() { return canal.EventType_DELETE } return canal.EventType_UPDATE @@ -212,16 +212,22 @@ func (b *canalEntryBuilder) buildColumn(c *model.Column, colName string, updated // build the RowData of a canal entry func (b *canalEntryBuilder) buildRowData(e *model.RowChangedEvent) (*canal.RowData, error) { var columns []*canal.Column - for name, column := range e.Columns { - c, err := b.buildColumn(column, name, !e.Delete) + for _, column := range e.Columns { + if e == nil { + continue + } + c, err := b.buildColumn(column, column.Name, !e.IsDelete()) if err != nil { return nil, errors.Trace(err) } columns = append(columns, c) } var preColumns []*canal.Column - for name, column := range e.PreColumns { - c, err := b.buildColumn(column, name, !e.Delete) + for _, column := range e.PreColumns { + if e == nil { + continue + } + c, err := b.buildColumn(column, column.Name, !e.IsDelete()) if err != nil { return nil, errors.Trace(err) } @@ -351,6 +357,11 @@ func (d *CanalEventBatchEncoder) Build() (key []byte, value []byte) { return nil, value } +// MixedBuild implements the EventBatchEncoder interface +func (d *CanalEventBatchEncoder) MixedBuild(withVersion bool) []byte { + panic("Mixed Build only use for JsonEncoder") +} + // Size implements the EventBatchEncoder interface func (d *CanalEventBatchEncoder) Size() int { // TODO: avoid marshaling the messages every time for calculating the size of the packet @@ -361,6 +372,11 @@ func (d *CanalEventBatchEncoder) Size() int { return proto.Size(d.packet) } +// Reset implements the EventBatchEncoder interface +func (d *CanalEventBatchEncoder) Reset() { + panic("Reset only used for JsonEncoder") +} + // refreshPacketBody() marshals the messages to the packet body func (d *CanalEventBatchEncoder) refreshPacketBody() error { oldSize := len(d.packet.Body) diff --git a/cdc/sink/codec/canal_test.go b/cdc/sink/codec/canal_test.go index 6a7f143287a..c9dd8983336 100644 --- a/cdc/sink/codec/canal_test.go +++ b/cdc/sink/codec/canal_test.go @@ -33,23 +33,23 @@ var _ = check.Suite(&canalBatchSuite{ rowCases: [][]*model.RowChangedEvent{{{ CommitTs: 1, Table: &model.TableName{Schema: "a", Table: "b"}, - Columns: map[string]*model.Column{"col1": {Type: 1, Value: "aa"}}, + Columns: []*model.Column{{Name: "col1", Type: 1, Value: "aa"}}, }}, {{ CommitTs: 1, Table: &model.TableName{Schema: "a", Table: "b"}, - Columns: map[string]*model.Column{"col1": {Type: 1, Value: "aa"}}, + Columns: []*model.Column{{Name: "col1", Type: 1, Value: "aa"}}, }, { CommitTs: 2, Table: &model.TableName{Schema: "a", Table: "b"}, - Columns: map[string]*model.Column{"col1": {Type: 1, Value: "bb"}}, + Columns: []*model.Column{{Name: "col1", Type: 1, Value: "bb"}}, }, { CommitTs: 3, Table: &model.TableName{Schema: "a", Table: "b"}, - Columns: map[string]*model.Column{"col1": {Type: 1, Value: "bb"}}, + Columns: []*model.Column{{Name: "col1", Type: 1, Value: "bb"}}, }, { CommitTs: 4, - Table: &model.TableName{Schema: "a", Table: "c", Partition: 6}, - Columns: map[string]*model.Column{"col1": {Type: 1, Value: "cc"}}, + Table: &model.TableName{Schema: "a", Table: "c", TableID: 6, IsPartition: true}, + Columns: []*model.Column{{Name: "col1", Type: 1, Value: "cc"}}, }}, {}}, ddlCases: [][]*model.DDLEvent{{{ CommitTs: 1, @@ -135,13 +135,12 @@ func (s *canalEntrySuite) TestConvertEntry(c *check.C) { Schema: "cdc", Table: "person", }, - Delete: false, - Columns: map[string]*model.Column{ - "id": {Type: mysql.TypeLong, Flag: model.PrimaryKeyFlag, Value: 1}, - "name": {Type: mysql.TypeVarchar, Value: "Bob"}, - "tiny": {Type: mysql.TypeTiny, Value: 255}, - "comment": {Type: mysql.TypeBlob, Value: []byte("测试")}, - "blob": {Type: mysql.TypeBlob, Value: []byte("测试blob"), Flag: model.BinaryFlag}, + Columns: []*model.Column{ + {Name: "id", Type: mysql.TypeLong, Flag: model.PrimaryKeyFlag, Value: 1}, + {Name: "name", Type: mysql.TypeVarchar, Value: "Bob"}, + {Name: "tiny", Type: mysql.TypeTiny, Value: 255}, + {Name: "comment", Type: mysql.TypeBlob, Value: []byte("测试")}, + {Name: "blob", Type: mysql.TypeBlob, Value: []byte("测试blob"), Flag: model.BinaryFlag}, }, } testCaseDelete := &model.RowChangedEvent{ @@ -150,9 +149,8 @@ func (s *canalEntrySuite) TestConvertEntry(c *check.C) { Schema: "cdc", Table: "person", }, - Delete: true, - PreColumns: map[string]*model.Column{ - "id": {Type: mysql.TypeLong, Flag: model.PrimaryKeyFlag, Value: 1}, + PreColumns: []*model.Column{ + {Name: "id", Type: mysql.TypeLong, Flag: model.PrimaryKeyFlag, Value: 1}, }, } testCaseDdl := &model.DDLEvent{ diff --git a/cdc/sink/codec/interface.go b/cdc/sink/codec/interface.go index 6818a175495..a88ddf1fa99 100644 --- a/cdc/sink/codec/interface.go +++ b/cdc/sink/codec/interface.go @@ -31,8 +31,15 @@ type EventBatchEncoder interface { AppendDDLEvent(e *model.DDLEvent) (EncoderResult, error) // Build builds the batch and returns the bytes of key and value. Build() (key []byte, value []byte) + // MixedBuild builds the batch and returns the bytes of mixed keys and values. + // This is used for cdc log, to merge key and value into one byte slice + // when first create file, we should set withVersion to true, to tell us that + // the first 8 byte represents the encoder version + MixedBuild(withVersion bool) []byte // Size returns the size of the batch(bytes) Size() int + // Reset reset the kv buffer + Reset() } // EventBatchDecoder is an abstraction for events decoder diff --git a/cdc/sink/codec/json.go b/cdc/sink/codec/json.go index aa80f611c95..d5757e8ff69 100644 --- a/cdc/sink/codec/json.go +++ b/cdc/sink/codec/json.go @@ -18,13 +18,17 @@ import ( "encoding/base64" "encoding/binary" "encoding/json" + "sort" + "strconv" + "strings" "github.com/pingcap/errors" "github.com/pingcap/log" timodel "github.com/pingcap/parser/model" "github.com/pingcap/parser/mysql" - "github.com/pingcap/ticdc/cdc/model" "go.uber.org/zap" + + "github.com/pingcap/ticdc/cdc/model" ) const ( @@ -32,9 +36,67 @@ const ( BatchVersion1 uint64 = 1 ) -type column = model.Column +type column struct { + Type byte `json:"t"` + + // WhereHandle is deprecation + // WhereHandle is replaced by HandleKey in Flag + WhereHandle *bool `json:"h,omitempty"` + Flag model.ColumnFlagType `json:"f"` + Value interface{} `json:"v"` +} -func formatColumnVal(c *column) { +func (c *column) FromSinkColumn(col *model.Column) { + c.Type = col.Type + c.Flag = col.Flag + if c.Flag.IsHandleKey() { + whereHandle := true + c.WhereHandle = &whereHandle + } + if col.Value == nil { + c.Value = nil + return + } + switch col.Type { + case mysql.TypeString, mysql.TypeVarString, mysql.TypeVarchar: + str := string(col.Value.([]byte)) + if c.Flag.IsBinary() { + str = strconv.Quote(str) + str = str[1 : len(str)-1] + } + c.Value = str + default: + c.Value = col.Value + } +} + +func (c *column) ToSinkColumn(name string) *model.Column { + col := new(model.Column) + col.Type = c.Type + col.Flag = c.Flag + col.Name = name + col.Value = c.Value + if c.Value == nil { + return col + } + switch col.Type { + case mysql.TypeString, mysql.TypeVarString, mysql.TypeVarchar: + str := col.Value.(string) + var err error + if c.Flag.IsBinary() { + str, err = strconv.Unquote("\"" + str + "\"") + if err != nil { + log.Fatal("invalid column value, please report a bug", zap.Any("col", c), zap.Error(err)) + } + } + col.Value = []byte(str) + default: + col.Value = c.Value + } + return col +} + +func formatColumnVal(c column) column { switch c.Type { case mysql.TypeTinyBlob, mysql.TypeMediumBlob, mysql.TypeLongBlob, mysql.TypeBlob: @@ -54,6 +116,7 @@ func formatColumnVal(c *column) { c.Value = uint64(intNum) } } + return c } type mqMessageKey struct { @@ -61,6 +124,7 @@ type mqMessageKey struct { Ts uint64 `json:"ts"` Schema string `json:"scm,omitempty"` Table string `json:"tbl,omitempty"` + RowID int64 `json:"rid,omitempty"` Partition *int64 `json:"ptn,omitempty"` Type model.MqMessageType `json:"t"` } @@ -74,9 +138,9 @@ func (m *mqMessageKey) Decode(data []byte) error { } type mqMessageRow struct { - Update map[string]*column `json:"u,omitempty"` - PreColumns map[string]*column `json:"p,omitempty"` - Delete map[string]*column `json:"d,omitempty"` + Update map[string]column `json:"u,omitempty"` + PreColumns map[string]column `json:"p,omitempty"` + Delete map[string]column `json:"d,omitempty"` } func (m *mqMessageRow) Encode() ([]byte, error) { @@ -90,11 +154,14 @@ func (m *mqMessageRow) Decode(data []byte) error { if err != nil { return errors.Trace(err) } - for _, column := range m.Update { - formatColumnVal(column) + for colName, column := range m.Update { + m.Update[colName] = formatColumnVal(column) } - for _, column := range m.Delete { - formatColumnVal(column) + for colName, column := range m.Delete { + m.Delete[colName] = formatColumnVal(column) + } + for colName, column := range m.PreColumns { + m.PreColumns[colName] = formatColumnVal(column) } return nil } @@ -121,26 +188,58 @@ func newResolvedMessage(ts uint64) *mqMessageKey { func rowEventToMqMessage(e *model.RowChangedEvent) (*mqMessageKey, *mqMessageRow) { var partition *int64 - if e.Table.Partition != 0 { - partition = &e.Table.Partition + if e.Table.IsPartition { + partition = &e.Table.TableID } key := &mqMessageKey{ Ts: e.CommitTs, Schema: e.Table.Schema, Table: e.Table.Table, + RowID: e.RowID, Partition: partition, Type: model.MqMessageTypeRow, } value := &mqMessageRow{} - if e.Delete { - value.Delete = e.PreColumns + if e.IsDelete() { + value.Delete = sinkColumns2JsonColumns(e.PreColumns) } else { - value.Update = e.Columns - value.PreColumns = e.PreColumns + value.Update = sinkColumns2JsonColumns(e.Columns) + value.PreColumns = sinkColumns2JsonColumns(e.PreColumns) } return key, value } +func sinkColumns2JsonColumns(cols []*model.Column) map[string]column { + jsonCols := make(map[string]column, len(cols)) + for _, col := range cols { + if col == nil { + continue + } + c := column{} + c.FromSinkColumn(col) + jsonCols[col.Name] = c + } + if len(jsonCols) == 0 { + return nil + } + return jsonCols +} + +func jsonColumns2SinkColumns(cols map[string]column) []*model.Column { + sinkCols := make([]*model.Column, 0, len(cols)) + for name, col := range cols { + c := col.ToSinkColumn(name) + sinkCols = append(sinkCols, c) + } + if len(sinkCols) == 0 { + return nil + } + sort.Slice(sinkCols, func(i, j int) bool { + return strings.Compare(sinkCols[i].Name, sinkCols[j].Name) > 0 + }) + return sinkCols +} + func mqMessageToRowEvent(key *mqMessageKey, value *mqMessageRow) *model.RowChangedEvent { e := new(model.RowChangedEvent) // TODO: we lost the startTs from kafka message @@ -150,17 +249,17 @@ func mqMessageToRowEvent(key *mqMessageKey, value *mqMessageRow) *model.RowChang Schema: key.Schema, Table: key.Table, } + // TODO: we lost the tableID from kafka message if key.Partition != nil { - e.Table.Partition = *key.Partition + e.Table.TableID = *key.Partition + e.Table.IsPartition = true } if len(value.Delete) != 0 { - e.Delete = true - e.PreColumns = value.Delete + e.PreColumns = jsonColumns2SinkColumns(value.Delete) } else { - e.Delete = false - e.Columns = value.Update - e.PreColumns = value.PreColumns + e.Columns = jsonColumns2SinkColumns(value.Update) + e.PreColumns = jsonColumns2SinkColumns(value.PreColumns) } return e } @@ -273,11 +372,54 @@ func (d *JSONEventBatchEncoder) Build() (key []byte, value []byte) { return d.keyBuf.Bytes(), d.valueBuf.Bytes() } +// MixedBuild implements the EventBatchEncoder interface +func (d *JSONEventBatchEncoder) MixedBuild(withVersion bool) []byte { + keyBytes := d.keyBuf.Bytes() + valueBytes := d.valueBuf.Bytes() + mixedBytes := make([]byte, len(keyBytes)+len(valueBytes)) + + index := uint64(0) + keyIndex := uint64(0) + valueIndex := uint64(0) + + if withVersion { + // the first 8 bytes is the version, we should copy directly + // then skip 8 bytes for next round key value parse + copy(mixedBytes[:8], keyBytes[:8]) + index = uint64(8) // skip version + keyIndex = uint64(8) // skip version + } + + for { + if keyIndex >= uint64(len(keyBytes)) { + break + } + keyLen := binary.BigEndian.Uint64(keyBytes[keyIndex : keyIndex+8]) + offset := keyLen + 8 + copy(mixedBytes[index:index+offset], keyBytes[keyIndex:keyIndex+offset]) + keyIndex += offset + index += offset + + valueLen := binary.BigEndian.Uint64(valueBytes[valueIndex : valueIndex+8]) + offset = valueLen + 8 + copy(mixedBytes[index:index+offset], valueBytes[valueIndex:valueIndex+offset]) + valueIndex += offset + index += offset + } + return mixedBytes +} + // Size implements the EventBatchEncoder interface func (d *JSONEventBatchEncoder) Size() int { return d.keyBuf.Len() + d.valueBuf.Len() } +// Reset implements the EventBatchEncoder interface +func (d *JSONEventBatchEncoder) Reset() { + d.keyBuf.Reset() + d.valueBuf.Reset() +} + // NewJSONEventBatchEncoder creates a new JSONEventBatchEncoder. func NewJSONEventBatchEncoder() EventBatchEncoder { batch := &JSONEventBatchEncoder{ @@ -290,6 +432,106 @@ func NewJSONEventBatchEncoder() EventBatchEncoder { return batch } +// JSONEventBatchMixedDecoder decodes the byte of a batch into the original messages. +type JSONEventBatchMixedDecoder struct { + mixedBytes []byte + nextKey *mqMessageKey + nextKeyLen uint64 +} + +// HasNext implements the EventBatchDecoder interface +func (b *JSONEventBatchMixedDecoder) HasNext() (model.MqMessageType, bool, error) { + if !b.hasNext() { + return 0, false, nil + } + if err := b.decodeNextKey(); err != nil { + return 0, false, err + } + return b.nextKey.Type, true, nil +} + +// NextResolvedEvent implements the EventBatchDecoder interface +func (b *JSONEventBatchMixedDecoder) NextResolvedEvent() (uint64, error) { + if b.nextKey == nil { + if err := b.decodeNextKey(); err != nil { + return 0, err + } + } + b.mixedBytes = b.mixedBytes[b.nextKeyLen+8:] + if b.nextKey.Type != model.MqMessageTypeResolved { + return 0, errors.NotFoundf("not found resolved event message") + } + valueLen := binary.BigEndian.Uint64(b.mixedBytes[:8]) + b.mixedBytes = b.mixedBytes[valueLen+8:] + resolvedTs := b.nextKey.Ts + b.nextKey = nil + return resolvedTs, nil +} + +// NextRowChangedEvent implements the EventBatchDecoder interface +func (b *JSONEventBatchMixedDecoder) NextRowChangedEvent() (*model.RowChangedEvent, error) { + if b.nextKey == nil { + if err := b.decodeNextKey(); err != nil { + return nil, err + } + } + b.mixedBytes = b.mixedBytes[b.nextKeyLen+8:] + if b.nextKey.Type != model.MqMessageTypeRow { + return nil, errors.NotFoundf("not found row event message") + } + valueLen := binary.BigEndian.Uint64(b.mixedBytes[:8]) + value := b.mixedBytes[8 : valueLen+8] + b.mixedBytes = b.mixedBytes[valueLen+8:] + rowMsg := new(mqMessageRow) + if err := rowMsg.Decode(value); err != nil { + return nil, errors.Trace(err) + } + rowEvent := mqMessageToRowEvent(b.nextKey, rowMsg) + b.nextKey = nil + return rowEvent, nil +} + +// NextDDLEvent implements the EventBatchDecoder interface +func (b *JSONEventBatchMixedDecoder) NextDDLEvent() (*model.DDLEvent, error) { + if b.nextKey == nil { + if err := b.decodeNextKey(); err != nil { + return nil, err + } + } + b.mixedBytes = b.mixedBytes[b.nextKeyLen+8:] + if b.nextKey.Type != model.MqMessageTypeDDL { + return nil, errors.NotFoundf("not found ddl event message") + } + valueLen := binary.BigEndian.Uint64(b.mixedBytes[:8]) + value := b.mixedBytes[8 : valueLen+8] + b.mixedBytes = b.mixedBytes[valueLen+8:] + ddlMsg := new(mqMessageDDL) + if err := ddlMsg.Decode(value); err != nil { + return nil, errors.Trace(err) + } + ddlEvent := mqMessageToDDLEvent(b.nextKey, ddlMsg) + b.nextKey = nil + return ddlEvent, nil +} + +func (b *JSONEventBatchMixedDecoder) hasNext() bool { + return len(b.mixedBytes) > 0 +} + +func (b *JSONEventBatchMixedDecoder) decodeNextKey() error { + keyLen := binary.BigEndian.Uint64(b.mixedBytes[:8]) + key := b.mixedBytes[8 : keyLen+8] + // drop value bytes + msgKey := new(mqMessageKey) + err := msgKey.Decode(key) + if err != nil { + return errors.Trace(err) + } + b.nextKey = msgKey + b.nextKeyLen = keyLen + return nil +} + // JSONEventBatchDecoder decodes the byte of a batch into the original messages. type JSONEventBatchDecoder struct { keyBytes []byte @@ -397,6 +639,12 @@ func NewJSONEventBatchDecoder(key []byte, value []byte) (EventBatchDecoder, erro if version != BatchVersion1 { return nil, errors.New("unexpected key format version") } + // if only decode one byte slice, we choose MixedDecoder + if len(key) > 0 && len(value) == 0 { + return &JSONEventBatchMixedDecoder{ + mixedBytes: key, + }, nil + } return &JSONEventBatchDecoder{ keyBytes: key, valueBytes: value, diff --git a/cdc/sink/codec/json_test.go b/cdc/sink/codec/json_test.go index 21a1e9ac9ae..933c814325f 100644 --- a/cdc/sink/codec/json_test.go +++ b/cdc/sink/codec/json_test.go @@ -33,23 +33,23 @@ var _ = check.Suite(&batchSuite{ rowCases: [][]*model.RowChangedEvent{{{ CommitTs: 1, Table: &model.TableName{Schema: "a", Table: "b"}, - Columns: map[string]*model.Column{"col1": {Type: 1, Value: "aa"}}, + Columns: []*model.Column{{Name: "col1", Type: 1, Value: "aa"}}, }}, {{ CommitTs: 1, Table: &model.TableName{Schema: "a", Table: "b"}, - Columns: map[string]*model.Column{"col1": {Type: 1, Value: "aa"}}, + Columns: []*model.Column{{Name: "col1", Type: 1, Value: "aa"}}, }, { CommitTs: 2, Table: &model.TableName{Schema: "a", Table: "b"}, - Columns: map[string]*model.Column{"col1": {Type: 1, Value: "bb"}}, + Columns: []*model.Column{{Name: "col1", Type: 1, Value: "bb"}}, }, { CommitTs: 3, Table: &model.TableName{Schema: "a", Table: "b"}, - Columns: map[string]*model.Column{"col1": {Type: 1, Value: "bb"}}, + Columns: []*model.Column{{Name: "col1", Type: 1, Value: "bb"}}, }, { CommitTs: 4, - Table: &model.TableName{Schema: "a", Table: "c", Partition: 6}, - Columns: map[string]*model.Column{"col1": {Type: 1, Value: "cc"}}, + Table: &model.TableName{Schema: "a", Table: "c", TableID: 6, IsPartition: true}, + Columns: []*model.Column{{Name: "col1", Type: 1, Value: "cc"}}, }}, {}}, ddlCases: [][]*model.DDLEvent{{{ CommitTs: 1, @@ -84,16 +84,7 @@ var _ = check.Suite(&batchSuite{ }) func (s *batchSuite) testBatchCodec(c *check.C, newEncoder func() EventBatchEncoder, newDecoder func(key []byte, value []byte) (EventBatchDecoder, error)) { - for _, cs := range s.rowCases { - encoder := newEncoder() - for _, row := range cs { - _, err := encoder.AppendRowChangedEvent(row) - c.Assert(err, check.IsNil) - } - key, value := encoder.Build() - c.Assert(len(key)+len(value), check.Equals, encoder.Size()) - decoder, err := newDecoder(key, value) - c.Assert(err, check.IsNil) + checkRowDecoder := func(decoder EventBatchDecoder, cs []*model.RowChangedEvent) { index := 0 for { tp, hasNext, err := decoder.HasNext() @@ -108,17 +99,7 @@ func (s *batchSuite) testBatchCodec(c *check.C, newEncoder func() EventBatchEnco index++ } } - - for _, cs := range s.ddlCases { - encoder := newEncoder() - for _, ddl := range cs { - _, err := encoder.AppendDDLEvent(ddl) - c.Assert(err, check.IsNil) - } - key, value := encoder.Build() - c.Assert(len(key)+len(value), check.Equals, encoder.Size()) - decoder, err := newDecoder(key, value) - c.Assert(err, check.IsNil) + checkDDLDecoder := func(decoder EventBatchDecoder, cs []*model.DDLEvent) { index := 0 for { tp, hasNext, err := decoder.HasNext() @@ -133,17 +114,7 @@ func (s *batchSuite) testBatchCodec(c *check.C, newEncoder func() EventBatchEnco index++ } } - - for _, cs := range s.resolvedTsCases { - encoder := newEncoder() - for _, ts := range cs { - _, err := encoder.AppendResolvedEvent(ts) - c.Assert(err, check.IsNil) - } - key, value := encoder.Build() - c.Assert(len(key)+len(value), check.Equals, encoder.Size()) - decoder, err := newDecoder(key, value) - c.Assert(err, check.IsNil) + checkTSDecoder := func(decoder EventBatchDecoder, cs []uint64) { index := 0 for { tp, hasNext, err := decoder.HasNext() @@ -158,6 +129,69 @@ func (s *batchSuite) testBatchCodec(c *check.C, newEncoder func() EventBatchEnco index++ } } + + for _, cs := range s.rowCases { + encoder := newEncoder() + for _, row := range cs { + _, err := encoder.AppendRowChangedEvent(row) + c.Assert(err, check.IsNil) + } + // test mixed decode + mixed := encoder.MixedBuild(true) + c.Assert(len(mixed), check.Equals, encoder.Size()) + mixedDecoder, err := newDecoder(mixed, nil) + c.Assert(err, check.IsNil) + checkRowDecoder(mixedDecoder, cs) + // test normal decode + key, value := encoder.Build() + c.Assert(len(key)+len(value), check.Equals, encoder.Size()) + decoder, err := newDecoder(key, value) + c.Assert(err, check.IsNil) + checkRowDecoder(decoder, cs) + } + + for _, cs := range s.ddlCases { + encoder := newEncoder() + for _, ddl := range cs { + _, err := encoder.AppendDDLEvent(ddl) + c.Assert(err, check.IsNil) + } + // test mixed encode + mixed := encoder.MixedBuild(true) + c.Assert(len(mixed), check.Equals, encoder.Size()) + mixedDecoder, err := newDecoder(mixed, nil) + c.Assert(err, check.IsNil) + checkDDLDecoder(mixedDecoder, cs) + + // test normal encode + key, value := encoder.Build() + c.Assert(len(key)+len(value), check.Equals, encoder.Size()) + decoder, err := newDecoder(key, value) + c.Assert(err, check.IsNil) + checkDDLDecoder(decoder, cs) + } + + for _, cs := range s.resolvedTsCases { + encoder := newEncoder() + for _, ts := range cs { + _, err := encoder.AppendResolvedEvent(ts) + c.Assert(err, check.IsNil) + } + + // test mixed encode + mixed := encoder.MixedBuild(true) + c.Assert(len(mixed), check.Equals, encoder.Size()) + mixedDecoder, err := newDecoder(mixed, nil) + c.Assert(err, check.IsNil) + checkTSDecoder(mixedDecoder, cs) + + // test normal encode + key, value := encoder.Build() + c.Assert(len(key)+len(value), check.Equals, encoder.Size()) + decoder, err := newDecoder(key, value) + c.Assert(err, check.IsNil) + checkTSDecoder(decoder, cs) + } } func (s *batchSuite) TestDefaultEventBatchCodec(c *check.C) { @@ -169,7 +203,7 @@ var _ = check.Suite(&columnSuite{}) type columnSuite struct{} func (s *columnSuite) TestFormatCol(c *check.C) { - row := &mqMessageRow{Update: map[string]*column{"test": { + row := &mqMessageRow{Update: map[string]column{"test": { Type: mysql.TypeString, Value: "测", }}} @@ -180,7 +214,7 @@ func (s *columnSuite) TestFormatCol(c *check.C) { c.Assert(err, check.IsNil) c.Assert(row2, check.DeepEquals, row) - row = &mqMessageRow{Update: map[string]*column{"test": { + row = &mqMessageRow{Update: map[string]column{"test": { Type: mysql.TypeBlob, Value: []byte("测"), }}} @@ -191,3 +225,24 @@ func (s *columnSuite) TestFormatCol(c *check.C) { c.Assert(err, check.IsNil) c.Assert(row2, check.DeepEquals, row) } + +func (s *columnSuite) TestVarBinaryCol(c *check.C) { + col := &model.Column{ + Name: "test", + Type: mysql.TypeString, + Flag: model.BinaryFlag, + Value: []byte{0x89, 0x50, 0x4E, 0x47, 0x0D, 0x0A, 0x1A, 0x0A}, + } + jsonCol := column{} + jsonCol.FromSinkColumn(col) + row := &mqMessageRow{Update: map[string]column{"test": jsonCol}} + rowEncode, err := row.Encode() + c.Assert(err, check.IsNil) + row2 := new(mqMessageRow) + err = row2.Decode(rowEncode) + c.Assert(err, check.IsNil) + c.Assert(row2, check.DeepEquals, row) + jsonCol2 := row2.Update["test"] + col2 := jsonCol2.ToSinkColumn("test") + c.Assert(col2, check.DeepEquals, col) +} diff --git a/cdc/sink/codec/schema_registry.go b/cdc/sink/codec/schema_registry.go index b5b302c4dc3..533516bc7f7 100644 --- a/cdc/sink/codec/schema_registry.go +++ b/cdc/sink/codec/schema_registry.go @@ -110,7 +110,8 @@ func NewAvroSchemaManager( var regexRemoveSpaces = regexp.MustCompile(`\s`) // Register the latest schema for a table to the Registry, by passing in a Codec -func (m *AvroSchemaManager) Register(ctx context.Context, tableName model.TableName, codec *goavro.Codec) error { +// Returns the Schema's ID and err +func (m *AvroSchemaManager) Register(ctx context.Context, tableName model.TableName, codec *goavro.Codec) (int, error) { // The Schema Registry expects the JSON to be without newline characters reqBody := registerRequest{ Schema: regexRemoveSpaces.ReplaceAllString(codec.Schema(), ""), @@ -119,25 +120,25 @@ func (m *AvroSchemaManager) Register(ctx context.Context, tableName model.TableN } payload, err := json.Marshal(&reqBody) if err != nil { - return errors.Annotate(err, "Could not marshal request to the Registry") + return 0, errors.Annotate(err, "Could not marshal request to the Registry") } uri := m.registryURL + "/subjects/" + url.QueryEscape(m.tableNameToSchemaSubject(tableName)) + "/versions" log.Debug("Registering schema", zap.String("uri", uri), zap.ByteString("payload", payload)) req, err := http.NewRequestWithContext(ctx, "POST", uri, bytes.NewReader(payload)) if err != nil { - return err + return 0, err } req.Header.Add("Accept", "application/vnd.schemaregistry.v1+json") resp, err := httpRetry(ctx, m.credential, req, false) if err != nil { - return err + return 0, err } defer resp.Body.Close() body, err := ioutil.ReadAll(resp.Body) if err != nil { - return errors.Annotate(err, "Failed to read response from Registry") + return 0, errors.Annotate(err, "Failed to read response from Registry") } if resp.StatusCode != 200 { @@ -146,18 +147,18 @@ func (m *AvroSchemaManager) Register(ctx context.Context, tableName model.TableN zap.String("uri", uri), zap.ByteString("requestBody", payload), zap.ByteString("responseBody", body)) - return errors.New("Failed to register schema to the Registry, HTTP error") + return 0, errors.New("Failed to register schema to the Registry, HTTP error") } var jsonResp registerResponse err = json.Unmarshal(body, &jsonResp) if err != nil { - return errors.Annotate(err, "Failed to parse result from Registry") + return 0, errors.Annotate(err, "Failed to parse result from Registry") } if jsonResp.ID == 0 { - return errors.Errorf("Illegal schema ID returned from Registry %d", jsonResp.ID) + return 0, errors.Errorf("Illegal schema ID returned from Registry %d", jsonResp.ID) } log.Info("Registered schema successfully", @@ -165,7 +166,7 @@ func (m *AvroSchemaManager) Register(ctx context.Context, tableName model.TableN zap.String("uri", uri), zap.ByteString("body", body)) - return nil + return jsonResp.ID, nil } // Lookup the latest schema and the Registry designated ID for that schema. @@ -195,7 +196,7 @@ func (m *AvroSchemaManager) Lookup(ctx context.Context, tableName model.TableNam } req.Header.Add("Accept", "application/vnd.schemaregistry.v1+json, application/vnd.schemaregistry+json, application/json") - resp, err := httpRetry(ctx, m.credential, req, false) + resp, err := httpRetry(ctx, m.credential, req, true) if err != nil { return nil, 0, err } @@ -246,6 +247,55 @@ func (m *AvroSchemaManager) Lookup(ctx context.Context, tableName model.TableNam return cacheEntry.codec, cacheEntry.registryID, nil } +// SchemaGenerator represents a function that returns an Avro schema in JSON. +// Used for lazy evaluation +type SchemaGenerator func() (string, error) + +// GetCachedOrRegister checks if the suitable Avro schema has been cached. +// If not, a new schema is generated, registered and cached. +func (m *AvroSchemaManager) GetCachedOrRegister(ctx context.Context, tableName model.TableName, tiSchemaID uint64, schemaGen SchemaGenerator) (*goavro.Codec, int, error) { + key := m.tableNameToSchemaSubject(tableName) + if entry, exists := m.cache[key]; exists && entry.tiSchemaID == tiSchemaID { + log.Info("Avro schema GetCachedOrRegister cache hit", + zap.String("key", key), + zap.Uint64("tiSchemaID", tiSchemaID), + zap.Int("registryID", entry.registryID)) + return entry.codec, entry.registryID, nil + } + + log.Info("Avro schema lookup cache miss", + zap.String("key", key), + zap.Uint64("tiSchemaID", tiSchemaID)) + + schema, err := schemaGen() + if err != nil { + return nil, 0, errors.Annotate(err, "GetCachedOrRegister: SchemaGen failed") + } + + codec, err := goavro.NewCodec(schema) + if err != nil { + return nil, 0, errors.Annotate(err, "GetCachedOrRegister: Could not make goavro codec") + } + + id, err := m.Register(ctx, tableName, codec) + if err != nil { + return nil, 0, errors.Annotate(err, "GetCachedOrRegister: Could not register schema") + } + + cacheEntry := new(schemaCacheEntry) + cacheEntry.codec = codec + cacheEntry.registryID = id + cacheEntry.tiSchemaID = tiSchemaID + m.cache[m.tableNameToSchemaSubject(tableName)] = cacheEntry + + log.Info("Avro schema GetCachedOrRegister successful with cache miss", + zap.Uint64("tiSchemaID", cacheEntry.tiSchemaID), + zap.Int("registryID", cacheEntry.registryID), + zap.String("schema", cacheEntry.codec.Schema())) + + return codec, id, nil +} + // ClearRegistry clears the Registry subject for the given table. Should be idempotent. // Exported for testing. func (m *AvroSchemaManager) ClearRegistry(ctx context.Context, tableName model.TableName) error { diff --git a/cdc/sink/codec/schema_registry_test.go b/cdc/sink/codec/schema_registry_test.go index 994a2d5d25f..2835759ff0c 100644 --- a/cdc/sink/codec/schema_registry_test.go +++ b/cdc/sink/codec/schema_registry_test.go @@ -174,9 +174,8 @@ func getTestingContext() context.Context { func (s *AvroSchemaRegistrySuite) TestSchemaRegistry(c *check.C) { table := model.TableName{ - Schema: "testdb", - Table: "test1", - Partition: 0, + Schema: "testdb", + Table: "test1", } manager, err := NewAvroSchemaManager(getTestingContext(), &security.Credential{}, "http://127.0.0.1:8081", "-value") @@ -186,7 +185,7 @@ func (s *AvroSchemaRegistrySuite) TestSchemaRegistry(c *check.C) { c.Assert(err, check.IsNil) _, _, err = manager.Lookup(getTestingContext(), table, 1) - c.Assert(err, check.ErrorMatches, `.*cancelled.*`) + c.Assert(err, check.ErrorMatches, `.*not\sfound.*`) codec, err := goavro.NewCodec(`{ "type": "record", @@ -201,7 +200,7 @@ func (s *AvroSchemaRegistrySuite) TestSchemaRegistry(c *check.C) { }`) c.Assert(err, check.IsNil) - err = manager.Register(getTestingContext(), table, codec) + _, err = manager.Register(getTestingContext(), table, codec) c.Assert(err, check.IsNil) var id int @@ -231,7 +230,7 @@ func (s *AvroSchemaRegistrySuite) TestSchemaRegistry(c *check.C) { ] }`) c.Assert(err, check.IsNil) - err = manager.Register(getTestingContext(), table, codec) + _, err = manager.Register(getTestingContext(), table, codec) c.Assert(err, check.IsNil) codec2, id2, err := manager.Lookup(getTestingContext(), table, 999) @@ -250,9 +249,8 @@ func (s *AvroSchemaRegistrySuite) TestSchemaRegistryBad(c *check.C) { func (s *AvroSchemaRegistrySuite) TestSchemaRegistryIdempotent(c *check.C) { table := model.TableName{ - Schema: "testdb", - Table: "test1", - Partition: 0, + Schema: "testdb", + Table: "test1", } manager, err := NewAvroSchemaManager(getTestingContext(), &security.Credential{}, "http://127.0.0.1:8081", "-value") @@ -282,9 +280,12 @@ func (s *AvroSchemaRegistrySuite) TestSchemaRegistryIdempotent(c *check.C) { }`) c.Assert(err, check.IsNil) + id := 0 for i := 0; i < 20; i++ { - err = manager.Register(getTestingContext(), table, codec) + id1, err := manager.Register(getTestingContext(), table, codec) c.Assert(err, check.IsNil) + c.Assert(id == 0 || id == id1, check.IsTrue) + id = id1 } } diff --git a/cdc/sink/common/common.go b/cdc/sink/common/common.go index dd0f81eb05e..065c763f9ba 100644 --- a/cdc/sink/common/common.go +++ b/cdc/sink/common/common.go @@ -27,14 +27,14 @@ import ( // UnresolvedTxnCache caches unresolved txns type UnresolvedTxnCache struct { unresolvedTxnsMu sync.Mutex - unresolvedTxns map[model.TableName][]*model.Txn + unresolvedTxns map[model.TableID][]*model.SingleTableTxn checkpointTs uint64 } // NewUnresolvedTxnCache returns a new UnresolvedTxnCache func NewUnresolvedTxnCache() *UnresolvedTxnCache { return &UnresolvedTxnCache{ - unresolvedTxns: make(map[model.TableName][]*model.Txn), + unresolvedTxns: make(map[model.TableID][]*model.SingleTableTxn), } } @@ -48,8 +48,7 @@ func (c *UnresolvedTxnCache) Append(filter *filter.Filter, rows ...*model.RowCha log.Info("Row changed event ignored", zap.Uint64("start-ts", row.StartTs)) continue } - key := *row.Table - txns := c.unresolvedTxns[key] + txns := c.unresolvedTxns[row.Table.TableID] if len(txns) == 0 || txns[len(txns)-1].StartTs != row.StartTs { // fail-fast check if len(txns) != 0 && txns[len(txns)-1].CommitTs > row.CommitTs { @@ -60,11 +59,12 @@ func (c *UnresolvedTxnCache) Append(filter *filter.Filter, rows ...*model.RowCha zap.Uint64("last received row startTs", txns[len(txns)-1].StartTs), zap.Uint64("last received row commitTs", txns[len(txns)-1].CommitTs)) } - txns = append(txns, &model.Txn{ + txns = append(txns, &model.SingleTableTxn{ StartTs: row.StartTs, CommitTs: row.CommitTs, + Table: row.Table, }) - c.unresolvedTxns[key] = txns + c.unresolvedTxns[row.Table.TableID] = txns } txns[len(txns)-1].Append(row) appendRows++ @@ -73,7 +73,7 @@ func (c *UnresolvedTxnCache) Append(filter *filter.Filter, rows ...*model.RowCha } // Resolved returns resolved txns according to resolvedTs -func (c *UnresolvedTxnCache) Resolved(resolvedTs uint64) map[model.TableName][]*model.Txn { +func (c *UnresolvedTxnCache) Resolved(resolvedTs uint64) map[model.TableID][]*model.SingleTableTxn { if resolvedTs <= atomic.LoadUint64(&c.checkpointTs) { return nil } @@ -89,7 +89,7 @@ func (c *UnresolvedTxnCache) Resolved(resolvedTs uint64) map[model.TableName][]* } // Unresolved returns unresolved txns -func (c *UnresolvedTxnCache) Unresolved() map[model.TableName][]*model.Txn { +func (c *UnresolvedTxnCache) Unresolved() map[model.TableID][]*model.SingleTableTxn { return c.unresolvedTxns } @@ -99,26 +99,26 @@ func (c *UnresolvedTxnCache) UpdateCheckpoint(checkpointTs uint64) { } func splitResolvedTxn( - resolvedTs uint64, unresolvedTxns map[model.TableName][]*model.Txn, -) (minTs uint64, resolvedRowsMap map[model.TableName][]*model.Txn) { - resolvedRowsMap = make(map[model.TableName][]*model.Txn, len(unresolvedTxns)) + resolvedTs uint64, unresolvedTxns map[model.TableID][]*model.SingleTableTxn, +) (minTs uint64, resolvedRowsMap map[model.TableID][]*model.SingleTableTxn) { + resolvedRowsMap = make(map[model.TableID][]*model.SingleTableTxn, len(unresolvedTxns)) minTs = resolvedTs - for key, txns := range unresolvedTxns { + for tableID, txns := range unresolvedTxns { i := sort.Search(len(txns), func(i int) bool { return txns[i].CommitTs > resolvedTs }) if i == 0 { continue } - var resolvedTxns []*model.Txn + var resolvedTxns []*model.SingleTableTxn if i == len(txns) { resolvedTxns = txns - delete(unresolvedTxns, key) + delete(unresolvedTxns, tableID) } else { resolvedTxns = txns[:i] - unresolvedTxns[key] = txns[i:] + unresolvedTxns[tableID] = txns[i:] } - resolvedRowsMap[key] = resolvedTxns + resolvedRowsMap[tableID] = resolvedTxns if len(resolvedTxns) > 0 && resolvedTxns[0].CommitTs < minTs { minTs = resolvedTxns[0].CommitTs diff --git a/cdc/sink/common/common_test.go b/cdc/sink/common/common_test.go index 26352d0b005..6ceeb185b3d 100644 --- a/cdc/sink/common/common_test.go +++ b/cdc/sink/common/common_test.go @@ -28,63 +28,63 @@ var _ = check.Suite(&SinkCommonSuite{}) func (s SinkCommonSuite) TestSplitResolvedTxn(c *check.C) { testCases := []struct { - unresolvedTxns map[model.TableName][]*model.Txn + unresolvedTxns map[model.TableID][]*model.SingleTableTxn resolvedTs uint64 - expectedResolvedTxns map[model.TableName][]*model.Txn - expectedUnresolvedTxns map[model.TableName][]*model.Txn + expectedResolvedTxns map[model.TableID][]*model.SingleTableTxn + expectedUnresolvedTxns map[model.TableID][]*model.SingleTableTxn expectedMinTs uint64 }{{ - unresolvedTxns: map[model.TableName][]*model.Txn{ - {Table: "t1"}: {{CommitTs: 11}, {CommitTs: 21}, {CommitTs: 21}, {CommitTs: 23}, {CommitTs: 33}, {CommitTs: 34}}, - {Table: "t2"}: {{CommitTs: 23}, {CommitTs: 24}, {CommitTs: 26}, {CommitTs: 26}, {CommitTs: 26}, {CommitTs: 29}}, + unresolvedTxns: map[model.TableID][]*model.SingleTableTxn{ + 1: {{CommitTs: 11}, {CommitTs: 21}, {CommitTs: 21}, {CommitTs: 23}, {CommitTs: 33}, {CommitTs: 34}}, + 2: {{CommitTs: 23}, {CommitTs: 24}, {CommitTs: 26}, {CommitTs: 26}, {CommitTs: 26}, {CommitTs: 29}}, }, resolvedTs: 5, - expectedResolvedTxns: map[model.TableName][]*model.Txn{}, - expectedUnresolvedTxns: map[model.TableName][]*model.Txn{ - {Table: "t1"}: {{CommitTs: 11}, {CommitTs: 21}, {CommitTs: 21}, {CommitTs: 23}, {CommitTs: 33}, {CommitTs: 34}}, - {Table: "t2"}: {{CommitTs: 23}, {CommitTs: 24}, {CommitTs: 26}, {CommitTs: 26}, {CommitTs: 26}, {CommitTs: 29}}, + expectedResolvedTxns: map[model.TableID][]*model.SingleTableTxn{}, + expectedUnresolvedTxns: map[model.TableID][]*model.SingleTableTxn{ + 1: {{CommitTs: 11}, {CommitTs: 21}, {CommitTs: 21}, {CommitTs: 23}, {CommitTs: 33}, {CommitTs: 34}}, + 2: {{CommitTs: 23}, {CommitTs: 24}, {CommitTs: 26}, {CommitTs: 26}, {CommitTs: 26}, {CommitTs: 29}}, }, expectedMinTs: 5, }, { - unresolvedTxns: map[model.TableName][]*model.Txn{ - {Table: "t1"}: {{CommitTs: 11}, {CommitTs: 21}, {CommitTs: 21}, {CommitTs: 23}, {CommitTs: 33}, {CommitTs: 34}}, - {Table: "t2"}: {{CommitTs: 23}, {CommitTs: 24}, {CommitTs: 26}, {CommitTs: 26}, {CommitTs: 26}, {CommitTs: 29}}, + unresolvedTxns: map[model.TableID][]*model.SingleTableTxn{ + 1: {{CommitTs: 11}, {CommitTs: 21}, {CommitTs: 21}, {CommitTs: 23}, {CommitTs: 33}, {CommitTs: 34}}, + 2: {{CommitTs: 23}, {CommitTs: 24}, {CommitTs: 26}, {CommitTs: 26}, {CommitTs: 26}, {CommitTs: 29}}, }, resolvedTs: 23, - expectedResolvedTxns: map[model.TableName][]*model.Txn{ - {Table: "t1"}: {{CommitTs: 11}, {CommitTs: 21}, {CommitTs: 21}, {CommitTs: 23}}, - {Table: "t2"}: {{CommitTs: 23}}, + expectedResolvedTxns: map[model.TableID][]*model.SingleTableTxn{ + 1: {{CommitTs: 11}, {CommitTs: 21}, {CommitTs: 21}, {CommitTs: 23}}, + 2: {{CommitTs: 23}}, }, - expectedUnresolvedTxns: map[model.TableName][]*model.Txn{ - {Table: "t1"}: {{CommitTs: 33}, {CommitTs: 34}}, - {Table: "t2"}: {{CommitTs: 24}, {CommitTs: 26}, {CommitTs: 26}, {CommitTs: 26}, {CommitTs: 29}}, + expectedUnresolvedTxns: map[model.TableID][]*model.SingleTableTxn{ + 1: {{CommitTs: 33}, {CommitTs: 34}}, + 2: {{CommitTs: 24}, {CommitTs: 26}, {CommitTs: 26}, {CommitTs: 26}, {CommitTs: 29}}, }, expectedMinTs: 11, }, { - unresolvedTxns: map[model.TableName][]*model.Txn{ - {Table: "t1"}: {{CommitTs: 11}, {CommitTs: 21}, {CommitTs: 21}, {CommitTs: 23}, {CommitTs: 33}, {CommitTs: 34}}, - {Table: "t2"}: {{CommitTs: 23}, {CommitTs: 24}, {CommitTs: 26}, {CommitTs: 26}, {CommitTs: 26}, {CommitTs: 29}}, + unresolvedTxns: map[model.TableID][]*model.SingleTableTxn{ + 1: {{CommitTs: 11}, {CommitTs: 21}, {CommitTs: 21}, {CommitTs: 23}, {CommitTs: 33}, {CommitTs: 34}}, + 2: {{CommitTs: 23}, {CommitTs: 24}, {CommitTs: 26}, {CommitTs: 26}, {CommitTs: 26}, {CommitTs: 29}}, }, resolvedTs: 30, - expectedResolvedTxns: map[model.TableName][]*model.Txn{ - {Table: "t1"}: {{CommitTs: 11}, {CommitTs: 21}, {CommitTs: 21}, {CommitTs: 23}}, - {Table: "t2"}: {{CommitTs: 23}, {CommitTs: 24}, {CommitTs: 26}, {CommitTs: 26}, {CommitTs: 26}, {CommitTs: 29}}, + expectedResolvedTxns: map[model.TableID][]*model.SingleTableTxn{ + 1: {{CommitTs: 11}, {CommitTs: 21}, {CommitTs: 21}, {CommitTs: 23}}, + 2: {{CommitTs: 23}, {CommitTs: 24}, {CommitTs: 26}, {CommitTs: 26}, {CommitTs: 26}, {CommitTs: 29}}, }, - expectedUnresolvedTxns: map[model.TableName][]*model.Txn{ - {Table: "t1"}: {{CommitTs: 33}, {CommitTs: 34}}, + expectedUnresolvedTxns: map[model.TableID][]*model.SingleTableTxn{ + 1: {{CommitTs: 33}, {CommitTs: 34}}, }, expectedMinTs: 11, }, { - unresolvedTxns: map[model.TableName][]*model.Txn{ - {Table: "t1"}: {{CommitTs: 11}, {CommitTs: 21}, {CommitTs: 21}, {CommitTs: 23}, {CommitTs: 33}, {CommitTs: 34}}, - {Table: "t2"}: {{CommitTs: 23}, {CommitTs: 24}, {CommitTs: 26}, {CommitTs: 26}, {CommitTs: 26}, {CommitTs: 29}}, + unresolvedTxns: map[model.TableID][]*model.SingleTableTxn{ + 1: {{CommitTs: 11}, {CommitTs: 21}, {CommitTs: 21}, {CommitTs: 23}, {CommitTs: 33}, {CommitTs: 34}}, + 2: {{CommitTs: 23}, {CommitTs: 24}, {CommitTs: 26}, {CommitTs: 26}, {CommitTs: 26}, {CommitTs: 29}}, }, resolvedTs: 40, - expectedResolvedTxns: map[model.TableName][]*model.Txn{ - {Table: "t1"}: {{CommitTs: 11}, {CommitTs: 21}, {CommitTs: 21}, {CommitTs: 23}, {CommitTs: 33}, {CommitTs: 34}}, - {Table: "t2"}: {{CommitTs: 23}, {CommitTs: 24}, {CommitTs: 26}, {CommitTs: 26}, {CommitTs: 26}, {CommitTs: 29}}, + expectedResolvedTxns: map[model.TableID][]*model.SingleTableTxn{ + 1: {{CommitTs: 11}, {CommitTs: 21}, {CommitTs: 21}, {CommitTs: 23}, {CommitTs: 33}, {CommitTs: 34}}, + 2: {{CommitTs: 23}, {CommitTs: 24}, {CommitTs: 26}, {CommitTs: 26}, {CommitTs: 26}, {CommitTs: 29}}, }, - expectedUnresolvedTxns: map[model.TableName][]*model.Txn{}, + expectedUnresolvedTxns: map[model.TableID][]*model.SingleTableTxn{}, expectedMinTs: 11, }} for _, tc := range testCases { diff --git a/cdc/sink/dispatcher/default.go b/cdc/sink/dispatcher/default.go index 8a04196b587..741f0cad3f2 100644 --- a/cdc/sink/dispatcher/default.go +++ b/cdc/sink/dispatcher/default.go @@ -18,21 +18,26 @@ import ( ) type defaultDispatcher struct { - partitionNum int32 - tbd *tableDispatcher - ivd *indexValueDispatcher + partitionNum int32 + tbd *tableDispatcher + ivd *indexValueDispatcher + enableOldValue bool } -func newDefaultDispatcher(partitionNum int32) *defaultDispatcher { +func newDefaultDispatcher(partitionNum int32, enableOldValue bool) *defaultDispatcher { return &defaultDispatcher{ - partitionNum: partitionNum, - tbd: newTableDispatcher(partitionNum), - ivd: newIndexValueDispatcher(partitionNum), + partitionNum: partitionNum, + tbd: newTableDispatcher(partitionNum), + ivd: newIndexValueDispatcher(partitionNum), + enableOldValue: enableOldValue, } } func (d *defaultDispatcher) Dispatch(row *model.RowChangedEvent) int32 { - if len(row.IndieMarkCol) == 0 { + if d.enableOldValue { + return d.tbd.Dispatch(row) + } + if len(row.IndexColumns) != 1 { return d.tbd.Dispatch(row) } return d.ivd.Dispatch(row) diff --git a/cdc/sink/dispatcher/default_test.go b/cdc/sink/dispatcher/default_test.go index 7c6c0b241cd..b80306643a0 100644 --- a/cdc/sink/dispatcher/default_test.go +++ b/cdc/sink/dispatcher/default_test.go @@ -32,139 +32,168 @@ func (s DefaultDispatcherSuite) TestDefaultDispatcher(c *check.C) { Schema: "test", Table: "t1", }, - IndieMarkCol: "id", - Columns: map[string]*model.Column{ - "id": { + Columns: []*model.Column{ + { + Name: "id", Value: 1, - Flag: model.HandleKeyFlag, + Flag: model.HandleKeyFlag | model.PrimaryKeyFlag, }, }, + IndexColumns: [][]int{{0}}, }, exceptPartition: 11}, {row: &model.RowChangedEvent{ Table: &model.TableName{ Schema: "test", Table: "t1", }, - IndieMarkCol: "id", - Columns: map[string]*model.Column{ - "id": { + Columns: []*model.Column{ + { + Name: "id", Value: 2, - Flag: model.HandleKeyFlag, + Flag: model.HandleKeyFlag | model.PrimaryKeyFlag, }, }, + IndexColumns: [][]int{{0}}, }, exceptPartition: 1}, {row: &model.RowChangedEvent{ Table: &model.TableName{ Schema: "test", Table: "t1", }, - IndieMarkCol: "id", - Columns: map[string]*model.Column{ - "id": { + Columns: []*model.Column{ + { + Name: "id", Value: 3, - Flag: model.HandleKeyFlag, + Flag: model.HandleKeyFlag | model.PrimaryKeyFlag, }, }, + IndexColumns: [][]int{{0}}, }, exceptPartition: 7}, {row: &model.RowChangedEvent{ Table: &model.TableName{ Schema: "test", Table: "t2", }, - IndieMarkCol: "id", - Columns: map[string]*model.Column{ - "id": { + Columns: []*model.Column{ + { + Name: "id", Value: 1, - Flag: model.HandleKeyFlag, - }, - "a": { + Flag: model.HandleKeyFlag | model.PrimaryKeyFlag, + }, { + Name: "a", Value: 1, }, }, + IndexColumns: [][]int{{0}}, }, exceptPartition: 1}, {row: &model.RowChangedEvent{ Table: &model.TableName{ Schema: "test", Table: "t2", }, - IndieMarkCol: "id", - Columns: map[string]*model.Column{ - "id": { + Columns: []*model.Column{ + { + Name: "id", Value: 2, - Flag: model.HandleKeyFlag, - }, - "a": { + Flag: model.HandleKeyFlag | model.PrimaryKeyFlag, + }, { + Name: "a", Value: 2, }, }, + IndexColumns: [][]int{{0}}, }, exceptPartition: 11}, {row: &model.RowChangedEvent{ Table: &model.TableName{ Schema: "test", Table: "t2", }, - IndieMarkCol: "id", - Columns: map[string]*model.Column{ - "id": { + Columns: []*model.Column{ + { + Name: "id", Value: 3, - Flag: model.HandleKeyFlag, - }, - "a": { + Flag: model.HandleKeyFlag | model.PrimaryKeyFlag, + }, { + Name: "a", Value: 3, }, }, + IndexColumns: [][]int{{0}}, }, exceptPartition: 13}, {row: &model.RowChangedEvent{ Table: &model.TableName{ Schema: "test", Table: "t2", }, - IndieMarkCol: "id", - Columns: map[string]*model.Column{ - "id": { + Columns: []*model.Column{ + { + Name: "id", Value: 3, - Flag: model.HandleKeyFlag, - }, - "a": { + Flag: model.HandleKeyFlag | model.PrimaryKeyFlag, + }, { + Name: "a", Value: 4, }, }, + IndexColumns: [][]int{{0}}, }, exceptPartition: 13}, {row: &model.RowChangedEvent{ Table: &model.TableName{ Schema: "test", Table: "t3", }, - Columns: map[string]*model.Column{ - "id": { + Columns: []*model.Column{ + { + Name: "id", Value: 1, + Flag: model.HandleKeyFlag | model.PrimaryKeyFlag, + }, + { + Name: "a", + Value: 2, + Flag: model.UniqueKeyFlag, }, }, + IndexColumns: [][]int{{0}, {1}}, }, exceptPartition: 3}, {row: &model.RowChangedEvent{ Table: &model.TableName{ Schema: "test", Table: "t3", }, - Columns: map[string]*model.Column{ - "id": { + Columns: []*model.Column{ + { + Name: "id", Value: 2, + Flag: model.HandleKeyFlag | model.PrimaryKeyFlag, + }, { + Name: "a", + Value: 3, + Flag: model.UniqueKeyFlag, }, }, + IndexColumns: [][]int{{0}, {1}}, }, exceptPartition: 3}, {row: &model.RowChangedEvent{ Table: &model.TableName{ Schema: "test", Table: "t3", }, - Columns: map[string]*model.Column{ - "id": { + Columns: []*model.Column{ + { + Name: "id", Value: 3, + Flag: model.HandleKeyFlag | model.PrimaryKeyFlag, + }, { + Name: "a", + Value: 4, + Flag: model.UniqueKeyFlag, }, }, + IndexColumns: [][]int{{0}, {1}}, }, exceptPartition: 3}, } - p := newDefaultDispatcher(16) + p := newDefaultDispatcher(16, false) for _, tc := range testCases { c.Assert(p.Dispatch(tc.row), check.Equals, tc.exceptPartition) } diff --git a/cdc/sink/dispatcher/index_value.go b/cdc/sink/dispatcher/index_value.go index bdc463021df..e9824fc2808 100644 --- a/cdc/sink/dispatcher/index_value.go +++ b/cdc/sink/dispatcher/index_value.go @@ -41,9 +41,12 @@ func (r *indexValueDispatcher) Dispatch(row *model.RowChangedEvent) int32 { if len(row.Columns) == 0 { dispatchCols = row.PreColumns } - for name, col := range dispatchCols { + for _, col := range dispatchCols { + if col == nil { + continue + } if col.Flag.IsHandleKey() { - r.hasher.Write([]byte(name), []byte(model.ColumnValueString(col.Value))) + r.hasher.Write([]byte(col.Name), []byte(model.ColumnValueString(col.Value))) } } return int32(r.hasher.Sum32() % uint32(r.partitionNum)) diff --git a/cdc/sink/dispatcher/index_value_test.go b/cdc/sink/dispatcher/index_value_test.go index da2f56d80fa..6bde84ea382 100644 --- a/cdc/sink/dispatcher/index_value_test.go +++ b/cdc/sink/dispatcher/index_value_test.go @@ -32,12 +32,13 @@ func (s IndexValueDispatcherSuite) TestIndexValueDispatcher(c *check.C) { Schema: "test", Table: "t1", }, - Columns: map[string]*model.Column{ - "a": { + Columns: []*model.Column{ + { + Name: "a", Value: 11, Flag: model.HandleKeyFlag, - }, - "b": { + }, { + Name: "b", Value: 22, Flag: 0, }, @@ -48,12 +49,13 @@ func (s IndexValueDispatcherSuite) TestIndexValueDispatcher(c *check.C) { Schema: "test", Table: "t1", }, - Columns: map[string]*model.Column{ - "a": { + Columns: []*model.Column{ + { + Name: "a", Value: 22, Flag: model.HandleKeyFlag, - }, - "b": { + }, { + Name: "b", Value: 22, Flag: 0, }, @@ -64,12 +66,13 @@ func (s IndexValueDispatcherSuite) TestIndexValueDispatcher(c *check.C) { Schema: "test", Table: "t1", }, - Columns: map[string]*model.Column{ - "a": { + Columns: []*model.Column{ + { + Name: "a", Value: 11, Flag: model.HandleKeyFlag, - }, - "b": { + }, { + Name: "b", Value: 33, Flag: 0, }, @@ -80,12 +83,13 @@ func (s IndexValueDispatcherSuite) TestIndexValueDispatcher(c *check.C) { Schema: "test", Table: "t2", }, - Columns: map[string]*model.Column{ - "a": { + Columns: []*model.Column{ + { + Name: "a", Value: 11, Flag: model.HandleKeyFlag, - }, - "b": { + }, { + Name: "b", Value: 22, Flag: model.HandleKeyFlag, }, @@ -96,12 +100,13 @@ func (s IndexValueDispatcherSuite) TestIndexValueDispatcher(c *check.C) { Schema: "test", Table: "t2", }, - Columns: map[string]*model.Column{ - "b": { + Columns: []*model.Column{ + { + Name: "b", Value: 22, Flag: model.HandleKeyFlag, - }, - "a": { + }, { + Name: "a", Value: 11, Flag: model.HandleKeyFlag, }, @@ -112,12 +117,13 @@ func (s IndexValueDispatcherSuite) TestIndexValueDispatcher(c *check.C) { Schema: "test", Table: "t2", }, - Columns: map[string]*model.Column{ - "a": { + Columns: []*model.Column{ + { + Name: "a", Value: 11, Flag: model.HandleKeyFlag, - }, - "b": { + }, { + Name: "b", Value: 0, Flag: model.HandleKeyFlag, }, @@ -128,12 +134,13 @@ func (s IndexValueDispatcherSuite) TestIndexValueDispatcher(c *check.C) { Schema: "test", Table: "t2", }, - Columns: map[string]*model.Column{ - "a": { + Columns: []*model.Column{ + { + Name: "a", Value: 11, Flag: model.HandleKeyFlag, - }, - "b": { + }, { + Name: "b", Value: 33, Flag: model.HandleKeyFlag, }, diff --git a/cdc/sink/dispatcher/interface.go b/cdc/sink/dispatcher/interface.go index d021281ca20..51be7aa1a4a 100644 --- a/cdc/sink/dispatcher/interface.go +++ b/cdc/sink/dispatcher/interface.go @@ -103,13 +103,18 @@ func NewDispatcher(cfg *config.ReplicaConfig, partitionNum int32) (Dispatcher, e rule.fromString(ruleConfig.Dispatcher) switch rule { case dispatchRuleRowID, dispatchRuleIndexValue: + if cfg.EnableOldValue { + log.Warn("This index-value distribution mode " + + "does not guarantee row-level orderliness when " + + "switching on the old value, so please use caution!") + } d = newIndexValueDispatcher(partitionNum) case dispatchRuleTS: d = newTsDispatcher(partitionNum) case dispatchRuleTable: - d = newTsDispatcher(partitionNum) + d = newTableDispatcher(partitionNum) case dispatchRuleDefault: - d = newDefaultDispatcher(partitionNum) + d = newDefaultDispatcher(partitionNum, cfg.EnableOldValue) } rules = append(rules, struct { Dispatcher diff --git a/cdc/sink/dispatcher/switcher_test.go b/cdc/sink/dispatcher/switcher_test.go index b7f18f55e9a..a5645b4b5fb 100644 --- a/cdc/sink/dispatcher/switcher_test.go +++ b/cdc/sink/dispatcher/switcher_test.go @@ -35,6 +35,9 @@ func (s SwitcherSuite) TestSwitcher(c *check.C) { d, err = NewDispatcher(&config.ReplicaConfig{ Sink: &config.SinkConfig{ DispatchRules: []*config.DispatchRule{ + {Matcher: []string{"test_default.*"}, Dispatcher: "default"}, + {Matcher: []string{"test_table.*"}, Dispatcher: "table"}, + {Matcher: []string{"test_index_value.*"}, Dispatcher: "index-value"}, {Matcher: []string{"test.*"}, Dispatcher: "rowid"}, {Matcher: []string{"*.*", "!*.test"}, Dispatcher: "ts"}, }, @@ -56,4 +59,19 @@ func (s SwitcherSuite) TestSwitcher(c *check.C) { Schema: "sbs", Table: "test", }, }), check.FitsTypeOf, &defaultDispatcher{}) + c.Assert(d.(*dispatcherSwitcher).matchDispatcher(&model.RowChangedEvent{ + Table: &model.TableName{ + Schema: "test_default", Table: "test", + }, + }), check.FitsTypeOf, &defaultDispatcher{}) + c.Assert(d.(*dispatcherSwitcher).matchDispatcher(&model.RowChangedEvent{ + Table: &model.TableName{ + Schema: "test_table", Table: "test", + }, + }), check.FitsTypeOf, &tableDispatcher{}) + c.Assert(d.(*dispatcherSwitcher).matchDispatcher(&model.RowChangedEvent{ + Table: &model.TableName{ + Schema: "test_index_value", Table: "test", + }, + }), check.FitsTypeOf, &indexValueDispatcher{}) } diff --git a/cdc/sink/mq.go b/cdc/sink/mq.go index 22849d9ce40..cb5e554150e 100644 --- a/cdc/sink/mq.go +++ b/cdc/sink/mq.go @@ -21,14 +21,14 @@ import ( "sync/atomic" "time" - "github.com/linkedin/goavro/v2" "github.com/pingcap/errors" "github.com/pingcap/log" "github.com/pingcap/ticdc/cdc/model" "github.com/pingcap/ticdc/cdc/sink/codec" "github.com/pingcap/ticdc/cdc/sink/dispatcher" - "github.com/pingcap/ticdc/cdc/sink/mqProducer" - "github.com/pingcap/ticdc/cdc/sink/pulsar" + "github.com/pingcap/ticdc/cdc/sink/producer" + "github.com/pingcap/ticdc/cdc/sink/producer/kafka" + "github.com/pingcap/ticdc/cdc/sink/producer/pulsar" "github.com/pingcap/ticdc/pkg/config" "github.com/pingcap/ticdc/pkg/filter" "github.com/pingcap/ticdc/pkg/notify" @@ -38,7 +38,7 @@ import ( ) type mqSink struct { - mqProducer mqProducer.Producer + mqProducer producer.Producer dispatcher dispatcher.Dispatcher newEncoder func() codec.EventBatchEncoder filter *filter.Filter @@ -58,7 +58,7 @@ type mqSink struct { } func newMqSink( - ctx context.Context, credential *security.Credential, mqProducer mqProducer.Producer, + ctx context.Context, credential *security.Credential, mqProducer producer.Producer, filter *filter.Filter, config *config.ReplicaConfig, opts map[string]string, errCh chan error, ) (*mqSink, error) { partitionNum := mqProducer.GetPartitionNum() @@ -86,14 +86,19 @@ func newMqSink( if !ok { return nil, errors.New(`Avro protocol requires parameter "registry"`) } - schemaManager, err := codec.NewAvroSchemaManager(ctx, credential, registryURI, "-value") + keySchemaManager, err := codec.NewAvroSchemaManager(ctx, credential, registryURI, "-key") if err != nil { - return nil, errors.Annotate(err, "Could not create Avro schema manager") + return nil, errors.Annotate(err, "Could not create Avro schema manager for message keys") + } + valueSchemaManager, err := codec.NewAvroSchemaManager(ctx, credential, registryURI, "-value") + if err != nil { + return nil, errors.Annotate(err, "Could not create Avro schema manager for message values") } newEncoder1 := newEncoder newEncoder = func() codec.EventBatchEncoder { avroEncoder := newEncoder1().(*codec.AvroEventBatchEncoder) - avroEncoder.SetValueSchemaManager(schemaManager) + avroEncoder.SetKeySchemaManager(keySchemaManager) + avroEncoder.SetValueSchemaManager(valueSchemaManager) return avroEncoder } } @@ -148,15 +153,15 @@ func (k *mqSink) EmitRowChangedEvents(ctx context.Context, rows ...*model.RowCha return nil } -func (k *mqSink) FlushRowChangedEvents(ctx context.Context, resolvedTs uint64) error { +func (k *mqSink) FlushRowChangedEvents(ctx context.Context, resolvedTs uint64) (uint64, error) { if resolvedTs <= k.checkpointTs { - return nil + return k.checkpointTs, nil } for i := 0; i < int(k.partitionNum); i++ { select { case <-ctx.Done(): - return ctx.Err() + return 0, ctx.Err() case k.partitionInput[i] <- struct { row *model.RowChangedEvent resolvedTs uint64 @@ -169,7 +174,7 @@ flushLoop: for { select { case <-ctx.Done(): - return ctx.Err() + return 0, ctx.Err() case <-k.resolvedReceiver.C: for i := 0; i < int(k.partitionNum); i++ { if resolvedTs > atomic.LoadUint64(&k.partitionResolvedTs[i]) { @@ -181,11 +186,11 @@ flushLoop: } err := k.mqProducer.Flush(ctx) if err != nil { - return errors.Trace(err) + return 0, errors.Trace(err) } k.checkpointTs = resolvedTs k.statistics.PrintStatus() - return nil + return k.checkpointTs, nil } func (k *mqSink) EmitCheckpointTs(ctx context.Context, ts uint64) error { @@ -206,7 +211,7 @@ func (k *mqSink) EmitCheckpointTs(ctx context.Context, ts uint64) error { } func (k *mqSink) EmitDDLEvent(ctx context.Context, ddl *model.DDLEvent) error { - if k.filter.ShouldIgnoreDDLEvent(ddl.StartTs, ddl.TableInfo.Schema, ddl.TableInfo.Table) { + if k.filter.ShouldIgnoreDDLEvent(ddl.StartTs, ddl.Type, ddl.TableInfo.Schema, ddl.TableInfo.Table) { log.Info( "DDL event ignored", zap.String("query", ddl.Query), @@ -236,43 +241,7 @@ func (k *mqSink) EmitDDLEvent(ctx context.Context, ddl *model.DDLEvent) error { // Initialize registers Avro schemas for all tables func (k *mqSink) Initialize(ctx context.Context, tableInfo []*model.SimpleTableInfo) error { - if k.protocol == codec.ProtocolAvro && tableInfo != nil { - avroEncoder := k.newEncoder().(*codec.AvroEventBatchEncoder) - manager := avroEncoder.GetValueSchemaManager() - if manager == nil { - return errors.New("No schema manager in Avro encoder, probably bug") - } - - for _, info := range tableInfo { - if info == nil { - continue - } - - if k.filter.ShouldIgnoreTable(info.Schema, info.Table) { - log.Info("Skip creating schema for table", zap.String("table-name", info.Table)) - continue - } - - str, err := codec.ColumnInfoToAvroSchema(info.Table, info.ColumnInfo) - if err != nil { - return errors.Annotate(err, "Error in Initialize") - } - - avroCodec, err := goavro.NewCodec(str) - if err != nil { - return errors.Annotate(err, "Initialize failed: could not verify schema, probably bug") - } - - err = manager.Register(context.Background(), model.TableName{ - Schema: info.Schema, - Table: info.Table, - }, avroCodec) - - if err != nil { - return errors.Annotate(err, "Initialize failed: could not register schema") - } - } - } + // No longer need it for now return nil } @@ -389,11 +358,11 @@ func (k *mqSink) writeToProducer(ctx context.Context, key []byte, value []byte, } func newKafkaSaramaSink(ctx context.Context, sinkURI *url.URL, filter *filter.Filter, replicaConfig *config.ReplicaConfig, opts map[string]string, errCh chan error) (*mqSink, error) { - config := mqProducer.NewKafkaConfig() + config := kafka.NewKafkaConfig() scheme := strings.ToLower(sinkURI.Scheme) - if scheme != "kafka" { - return nil, errors.New("can not create MQ sink with unsupported scheme") + if scheme != "kafka" && scheme != "kafka+ssl" { + return nil, errors.Errorf("can't create MQ sink with unsupported scheme: %s", scheme) } s := sinkURI.Query().Get("partition-num") if s != "" { @@ -457,7 +426,7 @@ func newKafkaSaramaSink(ctx context.Context, sinkURI *url.URL, filter *filter.Fi topic := strings.TrimFunc(sinkURI.Path, func(r rune) bool { return r == '/' }) - producer, err := mqProducer.NewKafkaSaramaProducer(ctx, sinkURI.Host, topic, config, errCh) + producer, err := kafka.NewKafkaSaramaProducer(ctx, sinkURI.Host, topic, config, errCh) if err != nil { return nil, errors.Trace(err) } diff --git a/cdc/sink/mysql.go b/cdc/sink/mysql.go index d39f0b8b0ea..e042b96b09d 100644 --- a/cdc/sink/mysql.go +++ b/cdc/sink/mysql.go @@ -22,6 +22,7 @@ import ( "strconv" "strings" "sync" + "sync/atomic" "time" "github.com/cenkalti/backoff" @@ -48,16 +49,26 @@ import ( "github.com/pingcap/tidb/infoschema" "github.com/prometheus/client_golang/prometheus" "go.uber.org/zap" - "golang.org/x/sync/errgroup" ) const ( - defaultWorkerCount = 16 - defaultMaxTxnRow = 256 - defaultDMLMaxRetryTime = 8 - defaultDDLMaxRetryTime = 20 - defaultTiDBTxnMode = "optimistic" - defaultFlushInterval = time.Millisecond * 50 + defaultWorkerCount = 16 + defaultMaxTxnRow = 256 + defaultDMLMaxRetryTime = 8 + defaultDDLMaxRetryTime = 20 + defaultTiDBTxnMode = "optimistic" + defaultFlushInterval = time.Millisecond * 50 + defaultBatchReplaceEnabled = true + defaultBatchReplaceSize = 20 +) + +var ( + validSchemes = map[string]bool{ + "mysql": true, + "mysql+ssl": true, + "tidb": true, + "tidb+ssl": true, + } ) type mysqlSink struct { @@ -67,10 +78,13 @@ type mysqlSink struct { filter *filter.Filter cyclic *cyclic.Cyclic - txnCache *common.UnresolvedTxnCache - workers []*mysqlSinkWorker - notifier *notify.Notifier - errCh chan error + txnCache *common.UnresolvedTxnCache + workers []*mysqlSinkWorker + resolvedTs uint64 + + execWaitNotifier *notify.Notifier + resolvedNotifier *notify.Notifier + errCh chan error statistics *Statistics @@ -85,23 +99,57 @@ func (s *mysqlSink) EmitRowChangedEvents(ctx context.Context, rows ...*model.Row return nil } -func (s *mysqlSink) FlushRowChangedEvents(ctx context.Context, resolvedTs uint64) error { - resolvedTxnsMap := s.txnCache.Resolved(resolvedTs) - if len(resolvedTxnsMap) == 0 { - s.txnCache.UpdateCheckpoint(resolvedTs) - return nil +func (s *mysqlSink) FlushRowChangedEvents(ctx context.Context, resolvedTs uint64) (uint64, error) { + atomic.StoreUint64(&s.resolvedTs, resolvedTs) + s.resolvedNotifier.Notify() + + // check and throw error + select { + case err := <-s.errCh: + return 0, err + default: } - if s.cyclic != nil { - // Filter rows if it is origined from downstream. - cyclic.FilterAndReduceTxns(resolvedTxnsMap, s.cyclic.FilterReplicaID(), s.cyclic.ReplicaID()) + checkpointTs := resolvedTs + for _, worker := range s.workers { + workerCheckpointTs := atomic.LoadUint64(&worker.checkpointTs) + if workerCheckpointTs < checkpointTs { + checkpointTs = workerCheckpointTs + } } + return checkpointTs, nil +} - if err := s.concurrentExec(ctx, resolvedTxnsMap); err != nil { - return errors.Trace(err) +func (s *mysqlSink) flushRowChangedEvents(ctx context.Context) { + receiver := s.resolvedNotifier.NewReceiver(50 * time.Millisecond) + for { + select { + case <-ctx.Done(): + return + case <-receiver.C: + } + resolvedTs := atomic.LoadUint64(&s.resolvedTs) + resolvedTxnsMap := s.txnCache.Resolved(resolvedTs) + if len(resolvedTxnsMap) == 0 { + for _, worker := range s.workers { + atomic.StoreUint64(&worker.checkpointTs, resolvedTs) + } + s.txnCache.UpdateCheckpoint(resolvedTs) + continue + } + + if s.cyclic != nil { + // Filter rows if it is origined from downstream. + skippedRowCount := cyclic.FilterAndReduceTxns( + resolvedTxnsMap, s.cyclic.FilterReplicaID(), s.cyclic.ReplicaID()) + s.statistics.SubRowsCount(skippedRowCount) + } + s.dispatchAndExecTxns(ctx, resolvedTxnsMap) + for _, worker := range s.workers { + atomic.StoreUint64(&worker.checkpointTs, resolvedTs) + } + s.txnCache.UpdateCheckpoint(resolvedTs) } - s.txnCache.UpdateCheckpoint(resolvedTs) - return nil } func (s *mysqlSink) EmitCheckpointTs(ctx context.Context, ts uint64) error { @@ -110,7 +158,7 @@ func (s *mysqlSink) EmitCheckpointTs(ctx context.Context, ts uint64) error { } func (s *mysqlSink) EmitDDLEvent(ctx context.Context, ddl *model.DDLEvent) error { - if s.filter.ShouldIgnoreDDLEvent(ddl.StartTs, ddl.TableInfo.Schema, ddl.TableInfo.Table) { + if s.filter.ShouldIgnoreDDLEvent(ddl.StartTs, ddl.Type, ddl.TableInfo.Schema, ddl.TableInfo.Table) { log.Info( "DDL event ignored", zap.String("query", ddl.Query), @@ -149,6 +197,15 @@ func (s *mysqlSink) execDDLWithMaxRetries(ctx context.Context, ddl *model.DDLEve func (s *mysqlSink) execDDL(ctx context.Context, ddl *model.DDLEvent) error { shouldSwitchDB := len(ddl.TableInfo.Schema) > 0 && ddl.Type != timodel.ActionCreateSchema + failpoint.Inject("MySQLSinkExecDDLDelay", func() { + select { + case <-ctx.Done(): + failpoint.Return(ctx.Err()) + case <-time.After(time.Hour): + } + failpoint.Return(nil) + }) + tx, err := s.db.BeginTx(ctx, nil) if err != nil { return errors.Trace(err) @@ -205,17 +262,21 @@ func (s *mysqlSink) adjustSQLMode(ctx context.Context) error { var _ Sink = &mysqlSink{} type sinkParams struct { - workerCount int - maxTxnRow int - tidbTxnMode string - changefeedID string - captureAddr string + workerCount int + maxTxnRow int + tidbTxnMode string + changefeedID string + captureAddr string + batchReplaceEnabled bool + batchReplaceSize int } var defaultParams = &sinkParams{ - workerCount: defaultWorkerCount, - maxTxnRow: defaultMaxTxnRow, - tidbTxnMode: defaultTiDBTxnMode, + workerCount: defaultWorkerCount, + maxTxnRow: defaultMaxTxnRow, + tidbTxnMode: defaultTiDBTxnMode, + batchReplaceEnabled: defaultBatchReplaceEnabled, + batchReplaceSize: defaultBatchReplaceSize, } func configureSinkURI(ctx context.Context, dsnCfg *dmysql.Config, tz *time.Location, params *sinkParams) (string, error) { @@ -280,8 +341,8 @@ func newMySQLSink(ctx context.Context, changefeedID model.ChangeFeedID, sinkURI return nil, errors.New("fail to open MySQL sink, empty URL") } scheme := strings.ToLower(sinkURI.Scheme) - if scheme != "mysql" && scheme != "tidb" { - return nil, errors.New("can create mysql sink with unsupported scheme") + if _, ok := validSchemes[scheme]; !ok { + return nil, errors.Errorf("can't create mysql sink with unsupported scheme: %s", scheme) } s := sinkURI.Query().Get("worker-count") if s != "" { @@ -327,6 +388,23 @@ func newMySQLSink(ctx context.Context, changefeedID model.ChangeFeedID, sinkURI } tlsParam = "?tls=" + name } + + s = sinkURI.Query().Get("batch-replace-enable") + if s != "" { + enable, err := strconv.ParseBool(s) + if err != nil { + return nil, errors.Trace(err) + } + params.batchReplaceEnabled = enable + } + if params.batchReplaceEnabled && sinkURI.Query().Get("batch-replace-size") != "" { + size, err := strconv.Atoi(sinkURI.Query().Get("batch-replace-size")) + if err != nil { + return nil, errors.Trace(err) + } + params.batchReplaceSize = size + } + // dsn format of the driver: // [username[:password]@][protocol[(address)]]/dbname[?param1=value1&...¶mN=valueN] username := sinkURI.User.Username() @@ -395,16 +473,19 @@ func newMySQLSink(ctx context.Context, changefeedID model.ChangeFeedID, sinkURI } } - sink.notifier = new(notify.Notifier) + sink.execWaitNotifier = new(notify.Notifier) + sink.resolvedNotifier = new(notify.Notifier) sink.createSinkWorkers(ctx) + go sink.flushRowChangedEvents(ctx) + return sink, nil } func (s *mysqlSink) createSinkWorkers(ctx context.Context) { s.workers = make([]*mysqlSinkWorker, s.params.workerCount) for i := range s.workers { - receiver := s.notifier.NewReceiver(defaultFlushInterval) + receiver := s.execWaitNotifier.NewReceiver(defaultFlushInterval) worker := newMySQLSinkWorker( s.params.maxTxnRow, i, s.metricBucketSizeCounters[i], receiver, s.execDMLs) s.workers[i] = worker @@ -421,7 +502,7 @@ func (s *mysqlSink) createSinkWorkers(ctx context.Context) { } func (s *mysqlSink) notifyAndWaitExec(ctx context.Context) { - s.notifier.Notify() + s.execWaitNotifier.Notify() done := make(chan struct{}) go func() { for _, w := range s.workers { @@ -439,67 +520,47 @@ func (s *mysqlSink) notifyAndWaitExec(ctx context.Context) { } } -func (s *mysqlSink) concurrentExec(ctx context.Context, txnsGroup map[model.TableName][]*model.Txn) error { - errg, ctx := errgroup.WithContext(ctx) - ch := make(chan struct{}, 1) - errg.Go(func() error { - s.dispatchAndExecTxns(ctx, txnsGroup) - ch <- struct{}{} - return nil - }) - errg.Go(func() error { - select { - case <-ctx.Done(): - return nil - case <-ch: - return nil - case err := <-s.errCh: - return err - } - }) - return errg.Wait() -} - -func (s *mysqlSink) dispatchAndExecTxns(ctx context.Context, txnsGroup map[model.TableName][]*model.Txn) { +func (s *mysqlSink) dispatchAndExecTxns(ctx context.Context, txnsGroup map[model.TableID][]*model.SingleTableTxn) { nWorkers := s.params.workerCount causality := newCausality() rowsChIdx := 0 - sendFn := func(txn *model.Txn, idx int) { - causality.add(txn.Keys, idx) + sendFn := func(txn *model.SingleTableTxn, keys [][]byte, idx int) { + causality.add(keys, idx) s.workers[idx].appendTxn(ctx, txn) } - resolveConflict := func(txn *model.Txn) { - if conflict, idx := causality.detectConflict(txn.Keys); conflict { + resolveConflict := func(txn *model.SingleTableTxn) { + keys := genTxnKeys(txn) + if conflict, idx := causality.detectConflict(keys); conflict { if idx >= 0 { - sendFn(txn, idx) + sendFn(txn, keys, idx) return } s.notifyAndWaitExec(ctx) causality.reset() } - sendFn(txn, rowsChIdx) + sendFn(txn, keys, rowsChIdx) rowsChIdx++ rowsChIdx = rowsChIdx % nWorkers } - for _, txns := range txnsGroup { - for _, txn := range txns { - startTime := time.Now() - resolveConflict(txn) - s.metricConflictDetectDurationHis.Observe(time.Since(startTime).Seconds()) - } - } + h := newTxnsHeap(txnsGroup) + h.iter(func(txn *model.SingleTableTxn) { + startTime := time.Now() + resolveConflict(txn) + s.metricConflictDetectDurationHis.Observe(time.Since(startTime).Seconds()) + }) s.notifyAndWaitExec(ctx) } type mysqlSinkWorker struct { - txnCh chan *model.Txn + txnCh chan *model.SingleTableTxn txnWg sync.WaitGroup maxTxnRow int bucket int execDMLs func(context.Context, []*model.RowChangedEvent, uint64, int) error metricBucketSize prometheus.Counter receiver *notify.Receiver + checkpointTs uint64 } func newMySQLSinkWorker( @@ -510,7 +571,7 @@ func newMySQLSinkWorker( execDMLs func(context.Context, []*model.RowChangedEvent, uint64, int) error, ) *mysqlSinkWorker { return &mysqlSinkWorker{ - txnCh: make(chan *model.Txn, 1024), + txnCh: make(chan *model.SingleTableTxn, 1024), maxTxnRow: maxTxnRow, bucket: bucket, metricBucketSize: metricBucketSize, @@ -523,7 +584,7 @@ func (w *mysqlSinkWorker) waitAllTxnsExecuted() { w.txnWg.Wait() } -func (w *mysqlSinkWorker) appendTxn(ctx context.Context, txn *model.Txn) { +func (w *mysqlSinkWorker) appendTxn(ctx context.Context, txn *model.SingleTableTxn) { if txn == nil { return } @@ -537,9 +598,10 @@ func (w *mysqlSinkWorker) appendTxn(ctx context.Context, txn *model.Txn) { func (w *mysqlSinkWorker) run(ctx context.Context) (err error) { var ( - toExecRows []*model.RowChangedEvent - replicaID uint64 - txnNum int + toExecRows []*model.RowChangedEvent + replicaID uint64 + txnNum int + lastCommitTs uint64 ) defer func() { @@ -565,6 +627,7 @@ func (w *mysqlSinkWorker) run(ctx context.Context) (err error) { txnNum = 0 return err } + atomic.StoreUint64(&w.checkpointTs, lastCommitTs) toExecRows = toExecRows[:0] w.metricBucketSize.Add(float64(txnNum)) w.txnWg.Add(-1 * txnNum) @@ -587,6 +650,7 @@ func (w *mysqlSinkWorker) run(ctx context.Context) (err error) { } replicaID = txn.ReplicaID toExecRows = append(toExecRows, txn.Rows...) + lastCommitTs = txn.CommitTs txnNum++ case <-w.receiver.C: if err := flushRows(); err != nil { @@ -597,7 +661,8 @@ func (w *mysqlSinkWorker) run(ctx context.Context) (err error) { } func (s *mysqlSink) Close() error { - s.notifier.Close() + s.execWaitNotifier.Close() + s.resolvedNotifier.Close() return s.db.Close() } @@ -645,14 +710,14 @@ func (s *mysqlSink) execDMLWithMaxRetries( if err = tx.Commit(); err != nil { return 0, checkTxnErr(errors.Trace(err)) } - return len(dmls.sqls), nil + return dmls.rowCount, nil }) if err != nil { return errors.Trace(err) } log.Debug("Exec Rows succeeded", zap.String("changefeed", s.params.changefeedID), - zap.Int("num of Rows", len(dmls.sqls)), + zap.Int("num of Rows", dmls.rowCount), zap.Int("bucket", bucket)) return nil }, @@ -660,37 +725,65 @@ func (s *mysqlSink) execDMLWithMaxRetries( } type preparedDMLs struct { - sqls []string - values [][]interface{} - markSQL string + sqls []string + values [][]interface{} + markSQL string + rowCount int } // prepareDMLs converts model.RowChangedEvent list to query string list and args list -func (s *mysqlSink) prepareDMLs(rows []*model.RowChangedEvent, replicaID uint64, bucket int) (*preparedDMLs, error) { +func (s *mysqlSink) prepareDMLs(rows []*model.RowChangedEvent, replicaID uint64, bucket int) *preparedDMLs { sqls := make([]string, 0, len(rows)) values := make([][]interface{}, 0, len(rows)) + replaces := make(map[string][][]interface{}) + rowCount := 0 for _, row := range rows { var query string var args []interface{} - var err error + quoteTable := quotes.QuoteSchema(row.Table.Schema, row.Table.Table) // TODO(leoppro): using `UPDATE` instead of `REPLACE` if the old value is enabled if len(row.PreColumns) != 0 { - query, args, err = prepareDelete(row.Table.Schema, row.Table.Table, row.PreColumns) - if err != nil { - return nil, errors.Trace(err) + // flush cached batch replace, we must keep the sequence of DMLs + if s.params.batchReplaceEnabled && len(replaces) > 0 { + replaceSqls, replaceValues := reduceReplace(replaces, s.params.batchReplaceSize) + sqls = append(sqls, replaceSqls...) + values = append(values, replaceValues...) + replaces = make(map[string][][]interface{}) + } + query, args = prepareDelete(quoteTable, row.PreColumns) + if query != "" { + sqls = append(sqls, query) + values = append(values, args) + rowCount++ } - sqls = append(sqls, query) - values = append(values, args) } if len(row.Columns) != 0 { - query, args, err = prepareReplace(row.Table.Schema, row.Table.Table, row.Columns) - if err != nil { - return nil, errors.Trace(err) + if s.params.batchReplaceEnabled { + query, args = prepareReplace(quoteTable, row.Columns, false /* appendPlaceHolder */) + if query != "" { + if _, ok := replaces[query]; !ok { + replaces[query] = make([][]interface{}, 0) + } + replaces[query] = append(replaces[query], args) + rowCount++ + } + } else { + query, args = prepareReplace(quoteTable, row.Columns, true /* appendPlaceHolder */) + sqls = append(sqls, query) + values = append(values, args) + if query != "" { + sqls = append(sqls, query) + values = append(values, args) + rowCount++ + } } - sqls = append(sqls, query) - values = append(values, args) } } + if s.params.batchReplaceEnabled { + replaceSqls, replaceValues := reduceReplace(replaces, s.params.batchReplaceSize) + sqls = append(sqls, replaceSqls...) + values = append(values, replaceValues...) + } dmls := &preparedDMLs{ sqls: sqls, values: values, @@ -701,18 +794,21 @@ func (s *mysqlSink) prepareDMLs(rows []*model.RowChangedEvent, replicaID uint64, updateMark := s.cyclic.UdpateSourceTableCyclicMark( row.Table.Schema, row.Table.Table, uint64(bucket), replicaID, row.StartTs) dmls.markSQL = updateMark + // rowCount is used in statistics, and for simplicity, + // we do not count mark table rows in rowCount. } - return dmls, nil + dmls.rowCount = rowCount + return dmls } func (s *mysqlSink) execDMLs(ctx context.Context, rows []*model.RowChangedEvent, replicaID uint64, bucket int) error { failpoint.Inject("MySQLSinkExecDMLError", func() { + // Add a delay to ensure the sink worker with `MySQLSinkHangLongTime` + // failpoint injected is executed first. + time.Sleep(time.Second * 2) failpoint.Return(errors.Trace(dmysql.ErrInvalidConn)) }) - dmls, err := s.prepareDMLs(rows, replicaID, bucket) - if err != nil { - return errors.Trace(err) - } + dmls := s.prepareDMLs(rows, replicaID, bucket) log.Debug("prepare DMLs", zap.Any("rows", rows), zap.Strings("sqls", dmls.sqls), zap.Any("values", dmls.values)) if err := s.execDMLWithMaxRetries(ctx, dmls, defaultDMLMaxRetryTime, bucket); err != nil { ts := make([]uint64, 0, len(rows)) @@ -727,31 +823,76 @@ func (s *mysqlSink) execDMLs(ctx context.Context, rows []*model.RowChangedEvent, return nil } -func prepareReplace(schema, table string, cols map[string]*model.Column) (string, []interface{}, error) { +func prepareReplace(quoteTable string, cols []*model.Column, appendPlaceHolder bool) (string, []interface{}) { var builder strings.Builder columnNames := make([]string, 0, len(cols)) args := make([]interface{}, 0, len(cols)) - for k, v := range cols { - if v.Flag.IsGeneratedColumn() { + for _, col := range cols { + if col == nil || col.Flag.IsGeneratedColumn() { continue } - columnNames = append(columnNames, k) - args = append(args, v.Value) + columnNames = append(columnNames, col.Name) + args = append(args, col.Value) + } + if len(args) == 0 { + return "", nil } colList := "(" + buildColumnList(columnNames) + ")" - tblName := quotes.QuoteSchema(schema, table) - builder.WriteString("REPLACE INTO " + tblName + colList + " VALUES ") - builder.WriteString("(" + model.HolderString(len(columnNames)) + ");") + builder.WriteString("REPLACE INTO " + quoteTable + colList + " VALUES ") + if appendPlaceHolder { + builder.WriteString("(" + model.HolderString(len(columnNames)) + ");") + } - return builder.String(), args, nil + return builder.String(), args } -func prepareDelete(schema, table string, cols map[string]*model.Column) (string, []interface{}, error) { +// reduceReplace groups SQLs with the same replace statement format, as following +// sql: `REPLACE INTO `test`.`t` (`a`,`b`) VALUES (?,?,?,?,?,?)` +// args: (1,"",2,"2",3,"") +func reduceReplace(replaces map[string][][]interface{}, batchSize int) ([]string, [][]interface{}) { + nextHolderString := func(query string, valueNum int, last bool) string { + query += "(" + model.HolderString(valueNum) + ")" + if !last { + query += "," + } + return query + } + sqls := make([]string, 0) + args := make([][]interface{}, 0) + for replace, vals := range replaces { + query := replace + cacheCount := 0 + cacheArgs := make([]interface{}, 0) + last := false + for i, val := range vals { + cacheCount += 1 + if i == len(vals)-1 || cacheCount >= batchSize { + last = true + } + query = nextHolderString(query, len(val), last) + cacheArgs = append(cacheArgs, val...) + if last { + sqls = append(sqls, query) + args = append(args, cacheArgs) + query = replace + cacheCount = 0 + cacheArgs = make([]interface{}, 0, len(cacheArgs)) + last = false + } + } + } + return sqls, args +} + +func prepareDelete(quoteTable string, cols []*model.Column) (string, []interface{}) { var builder strings.Builder - builder.WriteString("DELETE FROM " + quotes.QuoteSchema(schema, table) + " WHERE") + builder.WriteString("DELETE FROM " + quoteTable + " WHERE ") colNames, wargs := whereSlice(cols) + if len(wargs) == 0 { + return "", nil + } args := make([]interface{}, 0, len(wargs)) for i := 0; i < len(colNames); i++ { if i > 0 { @@ -766,16 +907,16 @@ func prepareDelete(schema, table string, cols map[string]*model.Column) (string, } builder.WriteString(" LIMIT 1;") sql := builder.String() - return sql, args, nil + return sql, args } -func whereSlice(cols map[string]*model.Column) (colNames []string, args []interface{}) { +func whereSlice(cols []*model.Column) (colNames []string, args []interface{}) { // Try to use unique key values when available - for colName, col := range cols { - if !col.Flag.IsHandleKey() { + for _, col := range cols { + if col == nil || !col.Flag.IsHandleKey() { continue } - colNames = append(colNames, colName) + colNames = append(colNames, col.Name) args = append(args, col.Value) } return diff --git a/cdc/sink/mysql_test.go b/cdc/sink/mysql_test.go index c5a06d5533d..479d5d5d994 100644 --- a/cdc/sink/mysql_test.go +++ b/cdc/sink/mysql_test.go @@ -15,11 +15,14 @@ package sink import ( "context" + "fmt" + "sort" "testing" "time" "github.com/davecgh/go-spew/spew" "github.com/pingcap/check" + "github.com/pingcap/parser/mysql" "github.com/pingcap/ticdc/cdc/model" "github.com/pingcap/ticdc/cdc/sink/common" "github.com/pingcap/ticdc/pkg/config" @@ -37,102 +40,99 @@ var _ = check.Suite(&MySQLSinkSuite{}) func newMySQLSink4Test(c *check.C) *mysqlSink { f, err := filter.NewFilter(config.GetDefaultReplicaConfig()) c.Assert(err, check.IsNil) + params := defaultParams + params.batchReplaceEnabled = false return &mysqlSink{ txnCache: common.NewUnresolvedTxnCache(), filter: f, statistics: NewStatistics(context.TODO(), "test", make(map[string]string)), + params: params, } } func (s MySQLSinkSuite) TestEmitRowChangedEvents(c *check.C) { testCases := []struct { input []*model.RowChangedEvent - expected map[model.TableName][]*model.Txn + expected map[model.TableID][]*model.SingleTableTxn }{{ input: []*model.RowChangedEvent{}, - expected: map[model.TableName][]*model.Txn{}, + expected: map[model.TableID][]*model.SingleTableTxn{}, }, { input: []*model.RowChangedEvent{ { StartTs: 1, CommitTs: 2, - Table: &model.TableName{Schema: "s1", Table: "t1"}, - Keys: []string{"a", "b"}, + Table: &model.TableName{Schema: "s1", Table: "t1", TableID: 1}, }, { StartTs: 1, CommitTs: 2, - Table: &model.TableName{Schema: "s1", Table: "t1"}, - Keys: []string{"b", "c"}, + Table: &model.TableName{Schema: "s1", Table: "t1", TableID: 1}, }, { StartTs: 1, CommitTs: 2, - Table: &model.TableName{Schema: "s1", Table: "t1"}, - Keys: []string{"a", "b"}, + Table: &model.TableName{Schema: "s1", Table: "t1", TableID: 1}, }, { StartTs: 3, CommitTs: 4, - Table: &model.TableName{Schema: "s1", Table: "t1"}, + Table: &model.TableName{Schema: "s1", Table: "t1", TableID: 1}, }, { StartTs: 3, CommitTs: 4, - Table: &model.TableName{Schema: "s1", Table: "t1"}, + Table: &model.TableName{Schema: "s1", Table: "t1", TableID: 1}, }, { StartTs: 3, CommitTs: 4, - Table: &model.TableName{Schema: "s1", Table: "t1"}, + Table: &model.TableName{Schema: "s1", Table: "t1", TableID: 1}, }, }, - expected: map[model.TableName][]*model.Txn{ - {Schema: "s1", Table: "t1"}: { + expected: map[model.TableID][]*model.SingleTableTxn{ + 1: { { + Table: &model.TableName{Schema: "s1", Table: "t1", TableID: 1}, StartTs: 1, CommitTs: 2, Rows: []*model.RowChangedEvent{ { StartTs: 1, CommitTs: 2, - Table: &model.TableName{Schema: "s1", Table: "t1"}, - Keys: []string{"a", "b"}, + Table: &model.TableName{Schema: "s1", Table: "t1", TableID: 1}, }, { StartTs: 1, CommitTs: 2, - Table: &model.TableName{Schema: "s1", Table: "t1"}, - Keys: []string{"b", "c"}, + Table: &model.TableName{Schema: "s1", Table: "t1", TableID: 1}, }, { StartTs: 1, CommitTs: 2, - Table: &model.TableName{Schema: "s1", Table: "t1"}, - Keys: []string{"a", "b"}, + Table: &model.TableName{Schema: "s1", Table: "t1", TableID: 1}, }}, - Keys: []string{"a", "b", "b", "c", "a", "b"}, }, { + Table: &model.TableName{Schema: "s1", Table: "t1", TableID: 1}, StartTs: 3, CommitTs: 4, Rows: []*model.RowChangedEvent{ { StartTs: 3, CommitTs: 4, - Table: &model.TableName{Schema: "s1", Table: "t1"}, + Table: &model.TableName{Schema: "s1", Table: "t1", TableID: 1}, }, { StartTs: 3, CommitTs: 4, - Table: &model.TableName{Schema: "s1", Table: "t1"}, + Table: &model.TableName{Schema: "s1", Table: "t1", TableID: 1}, }, { StartTs: 3, CommitTs: 4, - Table: &model.TableName{Schema: "s1", Table: "t1"}, + Table: &model.TableName{Schema: "s1", Table: "t1", TableID: 1}, }}, - Keys: []string{"`s1`.`t1`"}, }, }, }, @@ -141,103 +141,103 @@ func (s MySQLSinkSuite) TestEmitRowChangedEvents(c *check.C) { { StartTs: 1, CommitTs: 2, - Table: &model.TableName{Schema: "s1", Table: "t1"}, + Table: &model.TableName{Schema: "s1", Table: "t1", TableID: 1}, }, { StartTs: 3, CommitTs: 4, - Table: &model.TableName{Schema: "s1", Table: "t1"}, + Table: &model.TableName{Schema: "s1", Table: "t1", TableID: 1}, }, { StartTs: 5, CommitTs: 6, - Table: &model.TableName{Schema: "s1", Table: "t1"}, + Table: &model.TableName{Schema: "s1", Table: "t1", TableID: 1}, }, { StartTs: 1, CommitTs: 2, - Table: &model.TableName{Schema: "s1", Table: "t2"}, + Table: &model.TableName{Schema: "s1", Table: "t2", TableID: 2}, }, { StartTs: 3, CommitTs: 4, - Table: &model.TableName{Schema: "s1", Table: "t2"}, + Table: &model.TableName{Schema: "s1", Table: "t2", TableID: 2}, }, { StartTs: 5, CommitTs: 6, - Table: &model.TableName{Schema: "s1", Table: "t2"}, + Table: &model.TableName{Schema: "s1", Table: "t2", TableID: 2}, }, }, - expected: map[model.TableName][]*model.Txn{ - {Schema: "s1", Table: "t1"}: { + expected: map[model.TableID][]*model.SingleTableTxn{ + 1: { { + Table: &model.TableName{Schema: "s1", Table: "t1", TableID: 1}, StartTs: 1, CommitTs: 2, Rows: []*model.RowChangedEvent{ { StartTs: 1, CommitTs: 2, - Table: &model.TableName{Schema: "s1", Table: "t1"}, + Table: &model.TableName{Schema: "s1", Table: "t1", TableID: 1}, }}, - Keys: []string{"`s1`.`t1`"}, }, { + Table: &model.TableName{Schema: "s1", Table: "t1", TableID: 1}, StartTs: 3, CommitTs: 4, Rows: []*model.RowChangedEvent{ { StartTs: 3, CommitTs: 4, - Table: &model.TableName{Schema: "s1", Table: "t1"}, + Table: &model.TableName{Schema: "s1", Table: "t1", TableID: 1}, }}, - Keys: []string{"`s1`.`t1`"}, }, { + Table: &model.TableName{Schema: "s1", Table: "t1", TableID: 1}, StartTs: 5, CommitTs: 6, Rows: []*model.RowChangedEvent{ { StartTs: 5, CommitTs: 6, - Table: &model.TableName{Schema: "s1", Table: "t1"}, + Table: &model.TableName{Schema: "s1", Table: "t1", TableID: 1}, }}, - Keys: []string{"`s1`.`t1`"}, }, }, - {Schema: "s1", Table: "t2"}: { + 2: { { + Table: &model.TableName{Schema: "s1", Table: "t2", TableID: 2}, StartTs: 1, CommitTs: 2, Rows: []*model.RowChangedEvent{ { StartTs: 1, CommitTs: 2, - Table: &model.TableName{Schema: "s1", Table: "t2"}, + Table: &model.TableName{Schema: "s1", Table: "t2", TableID: 2}, }}, - Keys: []string{"`s1`.`t2`"}, }, { + Table: &model.TableName{Schema: "s1", Table: "t2", TableID: 2}, StartTs: 3, CommitTs: 4, Rows: []*model.RowChangedEvent{ { StartTs: 3, CommitTs: 4, - Table: &model.TableName{Schema: "s1", Table: "t2"}, + Table: &model.TableName{Schema: "s1", Table: "t2", TableID: 2}, }}, - Keys: []string{"`s1`.`t2`"}, }, { + Table: &model.TableName{Schema: "s1", Table: "t2", TableID: 2}, StartTs: 5, CommitTs: 6, Rows: []*model.RowChangedEvent{ { StartTs: 5, CommitTs: 6, - Table: &model.TableName{Schema: "s1", Table: "t2"}, + Table: &model.TableName{Schema: "s1", Table: "t2", TableID: 2}, }}, - Keys: []string{"`s1`.`t2`"}, }, }, }, @@ -254,16 +254,16 @@ func (s MySQLSinkSuite) TestEmitRowChangedEvents(c *check.C) { func (s MySQLSinkSuite) TestMysqlSinkWorker(c *check.C) { testCases := []struct { - txns []*model.Txn + txns []*model.SingleTableTxn expectedOutputRows [][]*model.RowChangedEvent exportedOutputReplicaIDs []uint64 maxTxnRow int }{ { - txns: []*model.Txn{}, + txns: []*model.SingleTableTxn{}, maxTxnRow: 4, }, { - txns: []*model.Txn{ + txns: []*model.SingleTableTxn{ { CommitTs: 1, Rows: []*model.RowChangedEvent{{CommitTs: 1}}, @@ -274,7 +274,7 @@ func (s MySQLSinkSuite) TestMysqlSinkWorker(c *check.C) { exportedOutputReplicaIDs: []uint64{1}, maxTxnRow: 2, }, { - txns: []*model.Txn{ + txns: []*model.SingleTableTxn{ { CommitTs: 1, Rows: []*model.RowChangedEvent{{CommitTs: 1}, {CommitTs: 1}, {CommitTs: 1}}, @@ -287,7 +287,7 @@ func (s MySQLSinkSuite) TestMysqlSinkWorker(c *check.C) { exportedOutputReplicaIDs: []uint64{1}, maxTxnRow: 2, }, { - txns: []*model.Txn{ + txns: []*model.SingleTableTxn{ { CommitTs: 1, Rows: []*model.RowChangedEvent{{CommitTs: 1}, {CommitTs: 1}}, @@ -311,7 +311,7 @@ func (s MySQLSinkSuite) TestMysqlSinkWorker(c *check.C) { exportedOutputReplicaIDs: []uint64{1, 1}, maxTxnRow: 4, }, { - txns: []*model.Txn{ + txns: []*model.SingleTableTxn{ { CommitTs: 1, Rows: []*model.RowChangedEvent{{CommitTs: 1}}, @@ -336,7 +336,7 @@ func (s MySQLSinkSuite) TestMysqlSinkWorker(c *check.C) { exportedOutputReplicaIDs: []uint64{1, 2, 3}, maxTxnRow: 4, }, { - txns: []*model.Txn{ + txns: []*model.SingleTableTxn{ { CommitTs: 1, Rows: []*model.RowChangedEvent{{CommitTs: 1}}, @@ -401,6 +401,202 @@ func (s MySQLSinkSuite) TestMysqlSinkWorker(c *check.C) { } } +func (s MySQLSinkSuite) TestPrepareDML(c *check.C) { + testCases := []struct { + input []*model.RowChangedEvent + expected *preparedDMLs + }{{ + input: []*model.RowChangedEvent{}, + expected: &preparedDMLs{sqls: []string{}, values: [][]interface{}{}}, + }, { + input: []*model.RowChangedEvent{ + { + StartTs: 418658114257813514, + CommitTs: 418658114257813515, + Table: &model.TableName{Schema: "common_1", Table: "uk_without_pk"}, + PreColumns: []*model.Column{nil, { + Name: "a1", + Type: mysql.TypeLong, + Flag: model.BinaryFlag | model.MultipleKeyFlag | model.HandleKeyFlag, + Value: 1, + }, { + Name: "a3", + Type: mysql.TypeLong, + Flag: model.BinaryFlag | model.MultipleKeyFlag | model.HandleKeyFlag, + Value: 1, + }}, + IndexColumns: [][]int{{1, 2}}, + }, + }, + expected: &preparedDMLs{ + sqls: []string{"DELETE FROM `common_1`.`uk_without_pk` WHERE `a1` = ? AND `a3` = ? LIMIT 1;"}, + values: [][]interface{}{{1, 1}}, + rowCount: 1, + }, + }} + ms := newMySQLSink4Test(c) + for i, tc := range testCases { + dmls := ms.prepareDMLs(tc.input, 0, 0) + c.Assert(dmls, check.DeepEquals, tc.expected, check.Commentf("%d", i)) + } +} + +func (s MySQLSinkSuite) TestMapReplace(c *check.C) { + testCases := []struct { + quoteTable string + cols []*model.Column + expectedQuery string + expectedArgs []interface{} + }{ + { + quoteTable: "`test`.`t1`", + cols: []*model.Column{ + {Name: "a", Type: mysql.TypeLong, Value: 1}, + {Name: "b", Type: mysql.TypeVarchar, Value: "varchar"}, + {Name: "c", Type: mysql.TypeLong, Value: 1, Flag: model.GeneratedColumnFlag}, + {Name: "d", Type: mysql.TypeTiny, Value: uint8(255)}, + }, + expectedQuery: "REPLACE INTO `test`.`t1`(`a`,`b`,`d`) VALUES ", + expectedArgs: []interface{}{1, "varchar", uint8(255)}, + }, + { + quoteTable: "`test`.`t1`", + cols: []*model.Column{ + {Name: "a", Type: mysql.TypeLong, Value: 1}, + {Name: "b", Type: mysql.TypeVarchar, Value: "varchar"}, + {Name: "c", Type: mysql.TypeLong, Value: 1}, + {Name: "d", Type: mysql.TypeTiny, Value: uint8(255)}, + }, + expectedQuery: "REPLACE INTO `test`.`t1`(`a`,`b`,`c`,`d`) VALUES ", + expectedArgs: []interface{}{1, "varchar", 1, uint8(255)}, + }, + } + for _, tc := range testCases { + // multiple times to verify the stability of column sequence in query string + for i := 0; i < 10; i++ { + query, args := prepareReplace(tc.quoteTable, tc.cols, false) + c.Assert(query, check.Equals, tc.expectedQuery) + c.Assert(args, check.DeepEquals, tc.expectedArgs) + } + } +} + +type sqlArgs [][]interface{} + +func (a sqlArgs) Len() int { return len(a) } +func (a sqlArgs) Less(i, j int) bool { return fmt.Sprintf("%s", a[i]) < fmt.Sprintf("%s", a[j]) } +func (a sqlArgs) Swap(i, j int) { a[i], a[j] = a[j], a[i] } + +func (s MySQLSinkSuite) TestReduceReplace(c *check.C) { + testCases := []struct { + replaces map[string][][]interface{} + batchSize int + sort bool + expectSQLs []string + expectArgs [][]interface{} + }{ + { + replaces: map[string][][]interface{}{ + "REPLACE INTO `test`.`t1`(`a`,`b`) VALUES ": { + []interface{}{1, "1"}, + []interface{}{2, "2"}, + []interface{}{3, "3"}, + }, + }, + batchSize: 1, + sort: false, + expectSQLs: []string{ + "REPLACE INTO `test`.`t1`(`a`,`b`) VALUES (?,?)", + "REPLACE INTO `test`.`t1`(`a`,`b`) VALUES (?,?)", + "REPLACE INTO `test`.`t1`(`a`,`b`) VALUES (?,?)", + }, + expectArgs: [][]interface{}{ + {1, "1"}, + {2, "2"}, + {3, "3"}, + }, + }, + { + replaces: map[string][][]interface{}{ + "REPLACE INTO `test`.`t1`(`a`,`b`) VALUES ": { + []interface{}{1, "1"}, + []interface{}{2, "2"}, + []interface{}{3, "3"}, + []interface{}{4, "3"}, + []interface{}{5, "5"}, + }, + }, + batchSize: 3, + sort: false, + expectSQLs: []string{ + "REPLACE INTO `test`.`t1`(`a`,`b`) VALUES (?,?),(?,?),(?,?)", + "REPLACE INTO `test`.`t1`(`a`,`b`) VALUES (?,?),(?,?)", + }, + expectArgs: [][]interface{}{ + {1, "1", 2, "2", 3, "3"}, + {4, "3", 5, "5"}, + }, + }, + { + replaces: map[string][][]interface{}{ + "REPLACE INTO `test`.`t1`(`a`,`b`) VALUES ": { + []interface{}{1, "1"}, + []interface{}{2, "2"}, + []interface{}{3, "3"}, + []interface{}{4, "3"}, + []interface{}{5, "5"}, + }, + }, + batchSize: 10, + sort: false, + expectSQLs: []string{ + "REPLACE INTO `test`.`t1`(`a`,`b`) VALUES (?,?),(?,?),(?,?),(?,?),(?,?)", + }, + expectArgs: [][]interface{}{ + {1, "1", 2, "2", 3, "3", 4, "3", 5, "5"}, + }, + }, + { + replaces: map[string][][]interface{}{ + "REPLACE INTO `test`.`t1`(`a`,`b`) VALUES ": { + []interface{}{1, "1"}, + []interface{}{2, "2"}, + []interface{}{3, "3"}, + []interface{}{4, "3"}, + []interface{}{5, "5"}, + []interface{}{6, "6"}, + }, + "REPLACE INTO `test`.`t2`(`a`,`b`) VALUES ": { + []interface{}{7, ""}, + []interface{}{8, ""}, + []interface{}{9, ""}, + }, + }, + batchSize: 3, + sort: true, + expectSQLs: []string{ + "REPLACE INTO `test`.`t1`(`a`,`b`) VALUES (?,?),(?,?),(?,?)", + "REPLACE INTO `test`.`t1`(`a`,`b`) VALUES (?,?),(?,?),(?,?)", + "REPLACE INTO `test`.`t2`(`a`,`b`) VALUES (?,?),(?,?),(?,?)", + }, + expectArgs: [][]interface{}{ + {1, "1", 2, "2", 3, "3"}, + {4, "3", 5, "5", 6, "6"}, + {7, "", 8, "", 9, ""}, + }, + }, + } + for _, tc := range testCases { + sqls, args := reduceReplace(tc.replaces, tc.batchSize) + if tc.sort { + sort.Strings(sqls) + sort.Sort(sqlArgs(args)) + } + c.Assert(sqls, check.DeepEquals, tc.expectSQLs) + c.Assert(args, check.DeepEquals, tc.expectArgs) + } +} + /* import ( "context" @@ -435,7 +631,7 @@ func (s MySQLSinkSuite) TestMysqlSinkWorker(c *check.C) { db: db, } - t := model.Txn{ + t := model.SingleTableTxn{ DDL: &model.DDL{ Database: "test", Table: "user", @@ -468,7 +664,7 @@ func (s MySQLSinkSuite) TestMysqlSinkWorker(c *check.C) { db: db, } - t := model.Txn{ + t := model.SingleTableTxn{ DDL: &model.DDL{ Database: "test", Table: "user", @@ -541,7 +737,7 @@ func (s MySQLSinkSuite) TestMysqlSinkWorker(c *check.C) { infoGetter: &helper, } - t := model.Txn{ + t := model.SingleTableTxn{ DMLs: []*model.DML{ { Database: "test", @@ -581,7 +777,7 @@ func (s MySQLSinkSuite) TestMysqlSinkWorker(c *check.C) { infoGetter: &helper, } - t := model.Txn{ + t := model.SingleTableTxn{ DMLs: []*model.DML{ { Database: "test", diff --git a/cdc/sink/mqProducer/kafka.go b/cdc/sink/producer/kafka/kafka.go similarity index 96% rename from cdc/sink/mqProducer/kafka.go rename to cdc/sink/producer/kafka/kafka.go index 505658165d1..99c760b6d83 100644 --- a/cdc/sink/mqProducer/kafka.go +++ b/cdc/sink/producer/kafka/kafka.go @@ -11,7 +11,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -package mqProducer +package kafka import ( "context" @@ -30,8 +30,8 @@ import ( "go.uber.org/zap" ) -// KafkaConfig stores the Kafka configuration -type KafkaConfig struct { +// Config stores the Kafka configuration +type Config struct { PartitionNum int32 ReplicationFactor int16 @@ -44,8 +44,8 @@ type KafkaConfig struct { } // NewKafkaConfig returns a default Kafka configuration -func NewKafkaConfig() KafkaConfig { - return KafkaConfig{ +func NewKafkaConfig() Config { + return Config{ Version: "2.4.0", MaxMessageBytes: 512 * 1024 * 1024, // 512M ReplicationFactor: 1, @@ -192,13 +192,13 @@ func (k *kafkaSaramaProducer) run(ctx context.Context) error { } // NewKafkaSaramaProducer creates a kafka sarama producer -func NewKafkaSaramaProducer(ctx context.Context, address string, topic string, config KafkaConfig, errCh chan error) (*kafkaSaramaProducer, error) { +func NewKafkaSaramaProducer(ctx context.Context, address string, topic string, config Config, errCh chan error) (*kafkaSaramaProducer, error) { log.Info("Starting kafka sarama producer ...", zap.Reflect("config", config)) cfg, err := newSaramaConfig(ctx, config) if err != nil { return nil, err } - if config.PartitionNum < 0 { + if config.PartitionNum <= 0 { return nil, errors.NotValidf("partition num %d", config.PartitionNum) } asyncClient, err := sarama.NewAsyncProducer(strings.Split(address, ","), cfg) @@ -298,7 +298,7 @@ func kafkaClientID(role, captureAddr, changefeedID, configuredClientID string) ( } // NewSaramaConfig return the default config and set the according version and metrics -func newSaramaConfig(ctx context.Context, c KafkaConfig) (*sarama.Config, error) { +func newSaramaConfig(ctx context.Context, c Config) (*sarama.Config, error) { config := sarama.NewConfig() version, err := sarama.ParseKafkaVersion(c.Version) diff --git a/cdc/sink/mqProducer/kafka_test.go b/cdc/sink/producer/kafka/kafka_test.go similarity index 98% rename from cdc/sink/mqProducer/kafka_test.go rename to cdc/sink/producer/kafka/kafka_test.go index 95ad938445b..40bc4a582da 100644 --- a/cdc/sink/mqProducer/kafka_test.go +++ b/cdc/sink/producer/kafka/kafka_test.go @@ -11,7 +11,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -package mqProducer +package kafka import ( "testing" diff --git a/cdc/sink/mqProducer/mq_producer.go b/cdc/sink/producer/mq_producer.go similarity index 97% rename from cdc/sink/mqProducer/mq_producer.go rename to cdc/sink/producer/mq_producer.go index 7a7f4e5f105..524c1029b88 100644 --- a/cdc/sink/mqProducer/mq_producer.go +++ b/cdc/sink/producer/mq_producer.go @@ -11,7 +11,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -package mqProducer +package producer import ( "context" diff --git a/cdc/sink/pulsar/doc.go b/cdc/sink/producer/pulsar/doc.go similarity index 100% rename from cdc/sink/pulsar/doc.go rename to cdc/sink/producer/pulsar/doc.go diff --git a/cdc/sink/pulsar/option.go b/cdc/sink/producer/pulsar/option.go similarity index 100% rename from cdc/sink/pulsar/option.go rename to cdc/sink/producer/pulsar/option.go diff --git a/cdc/sink/pulsar/producer.go b/cdc/sink/producer/pulsar/producer.go similarity index 100% rename from cdc/sink/pulsar/producer.go rename to cdc/sink/producer/pulsar/producer.go diff --git a/cdc/sink/sink.go b/cdc/sink/sink.go index b5897084098..f47e8c5e9e0 100644 --- a/cdc/sink/sink.go +++ b/cdc/sink/sink.go @@ -20,6 +20,7 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/ticdc/cdc/model" + "github.com/pingcap/ticdc/cdc/sink/cdclog" "github.com/pingcap/ticdc/pkg/config" "github.com/pingcap/ticdc/pkg/filter" ) @@ -44,7 +45,7 @@ type Sink interface { // FlushRowChangedEvents flushes each row which of commitTs less than or equal to `resolvedTs` into downstream. // TiCDC guarantees that all of Event which of commitTs less than or equal to `resolvedTs` are sent to Sink through `EmitRowChangedEvents` - FlushRowChangedEvents(ctx context.Context, resolvedTs uint64) error + FlushRowChangedEvents(ctx context.Context, resolvedTs uint64) (uint64, error) // EmitCheckpointTs sends CheckpointTs to Sink // TiCDC guarantees that all Events **in the cluster** which of commitTs less than or equal `checkpointTs` are sent to downstream successfully. @@ -66,10 +67,14 @@ func NewSink(ctx context.Context, changefeedID model.ChangeFeedID, sinkURIStr st return newBlackHoleSink(ctx, opts), nil case "mysql", "tidb", "mysql+ssl", "tidb+ssl": return newMySQLSink(ctx, changefeedID, sinkURI, filter, opts) - case "kafka": + case "kafka", "kafka+ssl": return newKafkaSaramaSink(ctx, sinkURI, filter, config, opts, errCh) case "pulsar", "pulsar+ssl": return newPulsarSink(ctx, sinkURI, filter, config, opts, errCh) + case "local": + return cdclog.NewLocalFileSink(sinkURI) + case "s3": + return cdclog.NewS3Sink(sinkURI) default: return nil, errors.Errorf("the sink scheme (%s) is not supported", sinkURI.Scheme) } diff --git a/cdc/sink/statistics.go b/cdc/sink/statistics.go index d370ae43fb3..727eb9cb5e3 100644 --- a/cdc/sink/statistics.go +++ b/cdc/sink/statistics.go @@ -80,6 +80,11 @@ func (b *Statistics) AddRowsCount(count int) { atomic.AddUint64(&b.totalRows, uint64(count)) } +// SubRowsCount records total number of rows needs to flush +func (b *Statistics) SubRowsCount(count int) { + atomic.AddUint64(&b.totalRows, ^uint64(count-1)) +} + // RecordBatchExecution records the cost time of batch execution and batch size func (b *Statistics) RecordBatchExecution(executer func() (int, error)) error { startTime := time.Now() diff --git a/cdc/sink/txns_heap.go b/cdc/sink/txns_heap.go new file mode 100644 index 00000000000..a514a05dd38 --- /dev/null +++ b/cdc/sink/txns_heap.go @@ -0,0 +1,85 @@ +// Copyright 2020 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package sink + +import ( + "container/heap" + + "github.com/pingcap/ticdc/cdc/model" +) + +type innerTxnsHeap []innerHeapEntry + +type innerHeapEntry struct { + ts uint64 + bucket int +} + +func (h innerTxnsHeap) Len() int { return len(h) } +func (h innerTxnsHeap) Less(i, j int) bool { return h[i].ts < h[j].ts } +func (h innerTxnsHeap) Swap(i, j int) { h[i], h[j] = h[j], h[i] } + +func (h *innerTxnsHeap) Push(x interface{}) { + // Push and Pop use pointer receivers because they modify the slice's length, + // not just its contents. + *h = append(*h, x.(innerHeapEntry)) +} + +func (h *innerTxnsHeap) Pop() interface{} { + old := *h + n := len(old) + x := old[n-1] + *h = old[0 : n-1] + return x +} + +type txnsHeap struct { + inner *innerTxnsHeap + txnsGroup [][]*model.SingleTableTxn +} + +func newTxnsHeap(txnsMap map[model.TableID][]*model.SingleTableTxn) *txnsHeap { + txnsGroup := make([][]*model.SingleTableTxn, 0, len(txnsMap)) + for _, txns := range txnsMap { + txnsGroup = append(txnsGroup, txns) + } + inner := make(innerTxnsHeap, 0, len(txnsGroup)) + heap.Init(&inner) + for bucket, txns := range txnsGroup { + if len(txns) == 0 { + continue + } + entry := innerHeapEntry{ts: txns[0].CommitTs, bucket: bucket} + heap.Push(&inner, entry) + } + return &txnsHeap{inner: &inner, txnsGroup: txnsGroup} +} + +func (h *txnsHeap) iter(fn func(txn *model.SingleTableTxn)) { + for { + if h.inner.Len() == 0 { + break + } + minEntry := heap.Pop(h.inner).(innerHeapEntry) + bucket := minEntry.bucket + fn(h.txnsGroup[bucket][0]) + h.txnsGroup[bucket] = h.txnsGroup[bucket][1:] + if len(h.txnsGroup[bucket]) > 0 { + heap.Push(h.inner, innerHeapEntry{ + ts: h.txnsGroup[bucket][0].CommitTs, + bucket: bucket, + }) + } + } +} diff --git a/cdc/sink/txns_heap_test.go b/cdc/sink/txns_heap_test.go new file mode 100644 index 00000000000..35f212f7918 --- /dev/null +++ b/cdc/sink/txns_heap_test.go @@ -0,0 +1,56 @@ +// Copyright 2020 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package sink + +import ( + "github.com/pingcap/check" + "github.com/pingcap/ticdc/cdc/model" +) + +type TxnsHeapSuite struct{} + +var _ = check.Suite(&TxnsHeapSuite{}) + +func (s TxnsHeapSuite) TestTxnsHeap(c *check.C) { + testCases := []struct { + txnsMap map[model.TableID][]*model.SingleTableTxn + expected []*model.SingleTableTxn + }{{ + txnsMap: nil, + expected: nil, + }, { + txnsMap: map[model.TableID][]*model.SingleTableTxn{ + 1: { + {CommitTs: 1}, {CommitTs: 3}, {CommitTs: 5}, {CommitTs: 7}, {CommitTs: 9}, + }, + 2: { + {CommitTs: 1}, {CommitTs: 10}, {CommitTs: 15}, {CommitTs: 15}, {CommitTs: 15}, + }, + 3: { + {CommitTs: 1}, {CommitTs: 1}, {CommitTs: 1}, {CommitTs: 2}, {CommitTs: 3}, + }, + }, + expected: []*model.SingleTableTxn{{CommitTs: 1}, {CommitTs: 1}, {CommitTs: 1}, {CommitTs: 1}, {CommitTs: 1}, {CommitTs: 2}, + {CommitTs: 3}, {CommitTs: 3}, {CommitTs: 5}, {CommitTs: 7}, {CommitTs: 9}, {CommitTs: 10}, {CommitTs: 15}, {CommitTs: 15}, {CommitTs: 15}}, + }} + + for _, tc := range testCases { + h := newTxnsHeap(tc.txnsMap) + i := 0 + h.iter(func(txn *model.SingleTableTxn) { + c.Assert(txn, check.DeepEquals, tc.expected[i]) + i++ + }) + } +} diff --git a/cmd/client.go b/cmd/client.go index 4d936e7ee5e..c5a948df7fb 100644 --- a/cmd/client.go +++ b/cmd/client.go @@ -38,7 +38,7 @@ import ( func init() { cliCmd := newCliCommand() - cliCmd.PersistentFlags().StringVar(&cliPdAddr, "pd", "http://127.0.0.1:2379", "PD address") + cliCmd.PersistentFlags().StringVar(&cliPdAddr, "pd", "http://127.0.0.1:2379", "PD address, use ',' to separate multiple PDs") cliCmd.PersistentFlags().BoolVarP(&interact, "interact", "i", false, "Run cdc cli with readline") cliCmd.PersistentFlags().StringVar(&cliLogLevel, "log-level", "warn", "log level (etc: debug|info|warn|error)") addSecurityFlags(cliCmd.PersistentFlags(), false /* isServer */) @@ -64,14 +64,17 @@ var ( cdcEtcdCli kv.CDCEtcdClient pdCli pd.Client - interact bool - simplified bool - cliLogLevel string + interact bool + simplified bool + cliLogLevel string + changefeedListAll bool changefeedID string captureID string interval uint + optForceRemove bool + defaultContext context.Context ) @@ -136,9 +139,11 @@ func newCliCommand() *cobra.Command { if err != nil { return errors.Annotate(err, "fail to validate TLS settings") } + + pdEndpoints := strings.Split(cliPdAddr, ",") etcdCli, err := clientv3.New(clientv3.Config{ Context: defaultContext, - Endpoints: []string{cliPdAddr}, + Endpoints: pdEndpoints, TLS: tlsConfig, DialTimeout: 30 * time.Second, DialOptions: []grpc.DialOption{ @@ -161,7 +166,7 @@ func newCliCommand() *cobra.Command { } cdcEtcdCli = kv.NewCDCEtcdClient(etcdCli) pdCli, err = pd.NewClientWithContext( - defaultContext, []string{cliPdAddr}, credential.PDSecurityOption(), + defaultContext, pdEndpoints, credential.PDSecurityOption(), pd.WithGRPCDialOptions( grpcTLSOption, grpc.WithBlock(), @@ -180,7 +185,7 @@ func newCliCommand() *cobra.Command { } ctx := defaultContext errorTiKVIncompatible := true // Error if TiKV is incompatible. - err = util.CheckClusterVersion(ctx, pdCli, cliPdAddr, credential, errorTiKVIncompatible) + err = util.CheckClusterVersion(ctx, pdCli, pdEndpoints[0], credential, errorTiKVIncompatible) if err != nil { return err } @@ -197,7 +202,7 @@ func newCliCommand() *cobra.Command { newCaptureCommand(), newChangefeedCommand(), newProcessorCommand(), - newMetadataCommand(), + newUnsafeCommand(), newTsoCommand(), ) diff --git a/cmd/client_changefeed.go b/cmd/client_changefeed.go index 9d025a8922f..ac52347d02f 100644 --- a/cmd/client_changefeed.go +++ b/cmd/client_changefeed.go @@ -67,7 +67,7 @@ func newAdminChangefeedCommand() []*cobra.Command { CfID: changefeedID, Type: model.AdminStop, } - return applyAdminChangefeed(ctx, job) + return applyAdminChangefeed(ctx, job, getCredential()) }, }, { @@ -79,7 +79,7 @@ func newAdminChangefeedCommand() []*cobra.Command { CfID: changefeedID, Type: model.AdminResume, } - return applyAdminChangefeed(ctx, job) + return applyAdminChangefeed(ctx, job, getCredential()) }, }, { @@ -90,8 +90,11 @@ func newAdminChangefeedCommand() []*cobra.Command { job := model.AdminJob{ CfID: changefeedID, Type: model.AdminRemove, + Opts: &model.AdminJobOption{ + ForceRemove: optForceRemove, + }, } - return applyAdminChangefeed(ctx, job) + return applyAdminChangefeed(ctx, job, getCredential()) }, }, } @@ -99,6 +102,9 @@ func newAdminChangefeedCommand() []*cobra.Command { for _, cmd := range cmds { cmd.PersistentFlags().StringVarP(&changefeedID, "changefeed-id", "c", "", "Replication task (changefeed) ID") _ = cmd.MarkPersistentFlagRequired("changefeed-id") + if cmd.Use == "remove" { + cmd.PersistentFlags().BoolVarP(&optForceRemove, "force", "f", false, "remove all information of the changefeed") + } } return cmds } @@ -113,10 +119,23 @@ func newListChangefeedCommand() *cobra.Command { if err != nil { return err } - cfs := make([]*changefeedCommonInfo, 0, len(raw)) + changefeedIDs := make(map[string]struct{}, len(raw)) for id := range raw { + changefeedIDs[id] = struct{}{} + } + if changefeedListAll { + statuses, err := cdcEtcdCli.GetAllChangeFeedStatus(ctx) + if err != nil { + return err + } + for cid := range statuses { + changefeedIDs[cid] = struct{}{} + } + } + cfs := make([]*changefeedCommonInfo, 0, len(changefeedIDs)) + for id := range changefeedIDs { cfci := &changefeedCommonInfo{ID: id} - resp, err := applyOwnerChangefeedQuery(ctx, id) + resp, err := applyOwnerChangefeedQuery(ctx, id, getCredential()) if err != nil { // if no capture is available, the query will fail, just add a warning here log.Warn("query changefeed info failed", zap.String("error", err.Error())) @@ -133,6 +152,7 @@ func newListChangefeedCommand() *cobra.Command { return jsonPrint(cmd, cfs) }, } + command.PersistentFlags().BoolVarP(&changefeedListAll, "all", "a", false, "List all replication tasks(including removed and finished)") return command } @@ -144,7 +164,7 @@ func newQueryChangefeedCommand() *cobra.Command { ctx := defaultContext if simplified { - resp, err := applyOwnerChangefeedQuery(ctx, changefeedID) + resp, err := applyOwnerChangefeedQuery(ctx, changefeedID, getCredential()) if err != nil { return err } @@ -229,6 +249,17 @@ func verifyChangefeedParamers(ctx context.Context, cmd *cobra.Command, isCreate // TODO(neil) enable ID bucket. } } + + for _, rules := range cfg.Sink.DispatchRules { + switch strings.ToLower(rules.Dispatcher) { + case "rowid", "index-value": + if cfg.EnableOldValue { + cmd.Printf("[WARN] This index-value distribution mode "+ + "does not guarantee row-level orderliness when "+ + "switching on the old value, so please use caution! dispatch-rules: %#v", rules) + } + } + } info := &model.ChangeFeedInfo{ SinkURI: sinkURI, Opts: make(map[string]string), @@ -370,6 +401,20 @@ func newUpdateChangefeedCommand() *cobra.Command { // Fix some fields that can't be updated. info.CreateTime = old.CreateTime info.AdminJobType = old.AdminJobType + info.StartTs = old.StartTs + info.ErrorHis = old.ErrorHis + info.Error = old.Error + + resp, err := applyOwnerChangefeedQuery(ctx, changefeedID, getCredential()) + // if no cdc owner exists, allow user to update changefeed config + if err != nil && errors.Cause(err) != errOwnerNotFound { + return err + } + // Note that the correctness of the logic here depends on the return value of `/capture/owner/changefeed/query` interface. + // TODO: Using error codes instead of string containing judgments + if err == nil && !strings.Contains(resp, `"state": "stopped"`) { + return errors.Errorf("can only update changefeed config when it is stopped\nstatus: %s", resp) + } changelog, err := diff.Diff(old, info) if err != nil { diff --git a/cmd/client_meta.go b/cmd/client_unsafe.go similarity index 65% rename from cmd/client_meta.go rename to cmd/client_unsafe.go index eba17f1880c..f5024903e03 100644 --- a/cmd/client_meta.go +++ b/cmd/client_unsafe.go @@ -22,46 +22,61 @@ import ( "github.com/spf13/cobra" ) -func newMetadataCommand() *cobra.Command { +func newUnsafeCommand() *cobra.Command { command := &cobra.Command{ - Use: "meta", - Short: "Manage metadata stored in PD", + Use: "unsafe", + Hidden: true, } command.AddCommand( - newDeleteMetaCommand(), - newDeleteGCTTLCommand(), + newDeleteServiceGcSafepointCommand(), + newResetCommand(), ) command.PersistentFlags().BoolVar(&noConfirm, "no-confirm", false, "Don't ask user whether to confirm executing meta command") return command } -func newDeleteMetaCommand() *cobra.Command { +func newResetCommand() *cobra.Command { command := &cobra.Command{ - Use: "delete", - Short: "Delete all meta data in etcd, confirm that you know what this command will do and use it at your own risk", + Use: "reset", + Short: "Reset the status of the TiCDC cluster, delete all meta data in etcd, confirm that you know what this command will do and use it at your own risk", RunE: func(cmd *cobra.Command, args []string) error { if err := confirmMetaDelete(cmd); err != nil { return err } ctx := defaultContext - err := cdcEtcdCli.ClearAllCDCInfo(ctx) + + leases, err := cdcEtcdCli.GetCaptureLeases(ctx) + if err != nil { + return errors.Trace(err) + } + + err = cdcEtcdCli.ClearAllCDCInfo(ctx) + if err != nil { + return errors.Trace(err) + } + + err = cdcEtcdCli.RevokeAllLeases(ctx, leases) if err != nil { return errors.Trace(err) } + _, err = pdCli.UpdateServiceGCSafePoint(ctx, cdc.CDCServiceSafePointID, 0, 0) - if err == nil { - cmd.Println("all metadata truncated in PD!") + if err != nil { + return errors.Trace(err) } - return errors.Trace(err) + + cmd.Println("reset and all metadata truncated in PD!") + + return nil }, } return command } -func newDeleteGCTTLCommand() *cobra.Command { +func newDeleteServiceGcSafepointCommand() *cobra.Command { command := &cobra.Command{ - Use: "delete-gc-ttl", - Short: "Delete CDC GC TTL in PD, confirm that you know what this command will do and use it at your own risk", + Use: "delete-service-gc-safepoint", + Short: "Delete CDC service GC safepoint in PD, confirm that you know what this command will do and use it at your own risk", RunE: func(cmd *cobra.Command, args []string) error { if err := confirmMetaDelete(cmd); err != nil { return err @@ -69,7 +84,7 @@ func newDeleteGCTTLCommand() *cobra.Command { ctx := defaultContext _, err := pdCli.UpdateServiceGCSafePoint(ctx, cdc.CDCServiceSafePointID, 0, 0) if err == nil { - cmd.Println("CDC GC TTL truncated in PD!") + cmd.Println("CDC service GC safepoint truncated in PD!") } return errors.Trace(err) }, diff --git a/cmd/util.go b/cmd/util.go index cb2249e8f61..db8fe5a43bb 100644 --- a/cmd/util.go +++ b/cmd/util.go @@ -16,9 +16,9 @@ package cmd import ( "context" "encoding/json" + liberrors "errors" "fmt" "io/ioutil" - "net/http" "net/url" "os" "os/signal" @@ -36,6 +36,7 @@ import ( "github.com/pingcap/ticdc/cdc/sink" "github.com/pingcap/ticdc/pkg/config" "github.com/pingcap/ticdc/pkg/filter" + "github.com/pingcap/ticdc/pkg/httputil" "github.com/pingcap/ticdc/pkg/security" "github.com/pingcap/ticdc/pkg/util" "github.com/pingcap/tidb/store/tikv" @@ -52,6 +53,10 @@ var ( allowedCertCN string ) +var ( + errOwnerNotFound = liberrors.New("owner not found") +) + func addSecurityFlags(flags *pflag.FlagSet, isServer bool) { flags.StringVar(&caPath, "ca", "", "CA certificate path for TLS connection") flags.StringVar(&certPath, "cert", "", "Certificate path for TLS connection") @@ -130,18 +135,31 @@ func getOwnerCapture(ctx context.Context) (*capture, error) { return c, nil } } - return nil, errors.NotFoundf("owner") + return nil, errors.Trace(errOwnerNotFound) } -func applyAdminChangefeed(ctx context.Context, job model.AdminJob) error { +func applyAdminChangefeed(ctx context.Context, job model.AdminJob, credential *security.Credential) error { owner, err := getOwnerCapture(ctx) if err != nil { return err } - addr := fmt.Sprintf("http://%s/capture/owner/admin", owner.AdvertiseAddr) - resp, err := http.PostForm(addr, url.Values(map[string][]string{ - cdc.APIOpVarAdminJob: {fmt.Sprint(int(job.Type))}, - cdc.APIOpVarChangefeedID: {job.CfID}, + scheme := "http" + if credential.IsTLSEnabled() { + scheme = "https" + } + addr := fmt.Sprintf("%s://%s/capture/owner/admin", scheme, owner.AdvertiseAddr) + cli, err := httputil.NewClient(credential) + if err != nil { + return err + } + forceRemoveOpt := "false" + if job.Opts != nil && job.Opts.ForceRemove { + forceRemoveOpt = "true" + } + resp, err := cli.PostForm(addr, url.Values(map[string][]string{ + cdc.APIOpVarAdminJob: {fmt.Sprint(int(job.Type))}, + cdc.APIOpVarChangefeedID: {job.CfID}, + cdc.APIOpForceRemoveChangefeed: {forceRemoveOpt}, })) if err != nil { return err @@ -156,13 +174,23 @@ func applyAdminChangefeed(ctx context.Context, job model.AdminJob) error { return nil } -func applyOwnerChangefeedQuery(ctx context.Context, cid model.ChangeFeedID) (string, error) { +func applyOwnerChangefeedQuery( + ctx context.Context, cid model.ChangeFeedID, credential *security.Credential, +) (string, error) { owner, err := getOwnerCapture(ctx) if err != nil { return "", err } - addr := fmt.Sprintf("http://%s/capture/owner/changefeed/query", owner.AdvertiseAddr) - resp, err := http.PostForm(addr, url.Values(map[string][]string{ + scheme := "http" + if credential.IsTLSEnabled() { + scheme = "https" + } + addr := fmt.Sprintf("%s://%s/capture/owner/changefeed/query", scheme, owner.AdvertiseAddr) + cli, err := httputil.NewClient(credential) + if err != nil { + return "", err + } + resp, err := cli.PostForm(addr, url.Values(map[string][]string{ cdc.APIOpVarChangefeedID: {cid}, })) if err != nil { diff --git a/docker-compose-avro.yml b/docker-compose-avro.yml index cda5b91aa78..3ec605d1ece 100644 --- a/docker-compose-avro.yml +++ b/docker-compose-avro.yml @@ -12,9 +12,12 @@ services: - ./docker/logs:/logs - ./docker/config:/config command: - - /bin/bash - - -c - - /usr/bin/tail -f /dev/null + - /usr/bin/socat + - -v + - tcp-l:1234,fork + - exec:'/bin/cat' + ports: + - "1234:1234" depends_on: - "upstream-pd" - "schema-registry" @@ -31,7 +34,7 @@ services: context: . dockerfile: ./Dockerfile volumes: - - ./docker/data:/data + - /data - ./docker/logs:/logs entrypoint: "/cdc server" command: @@ -51,7 +54,7 @@ services: context: . dockerfile: ./Dockerfile volumes: - - ./docker/data:/data + - /data - ./docker/logs:/logs entrypoint: "/cdc server" command: @@ -71,7 +74,7 @@ services: context: . dockerfile: ./Dockerfile volumes: - - ./docker/data:/data + - /data - ./docker/logs:/logs entrypoint: "/cdc server" command: @@ -91,7 +94,7 @@ services: - "2379:2379" volumes: - ./docker/config/pd.toml:/pd.toml:ro - - ./docker/data:/data + - /data - ./docker/logs:/logs command: - --name=upstream-pd @@ -109,7 +112,7 @@ services: image: pingcap/tikv:nightly volumes: - ./docker/config/tikv.toml:/tikv.toml:ro - - ./docker/data:/data + - /data - ./docker/logs:/logs command: - --addr=0.0.0.0:20160 @@ -143,7 +146,7 @@ services: image: pingcap/tikv:nightly volumes: - ./docker/config/tikv.toml:/tikv.toml:ro - - ./docker/data:/data + - /data - ./docker/logs:/logs command: - --addr=0.0.0.0:20160 @@ -182,7 +185,7 @@ services: - "3379:2379" volumes: - ./docker/config/pd.toml:/pd.toml:ro - - ./docker/data:/data + - /data - ./docker/logs:/logs command: - --name=downstream-pd @@ -200,7 +203,7 @@ services: image: pingcap/tikv:nightly volumes: - ./docker/config/tikv.toml:/tikv.toml:ro - - ./docker/data:/data + - /data - ./docker/logs:/logs command: - --addr=0.0.0.0:20160 @@ -217,7 +220,7 @@ services: image: pingcap/tikv:nightly volumes: - ./docker/config/tikv.toml:/tikv.toml:ro - - ./docker/data:/data + - /data - ./docker/logs:/logs command: - --addr=0.0.0.0:20160 @@ -234,7 +237,7 @@ services: image: pingcap/tikv:nightly volumes: - ./docker/config/tikv.toml:/tikv.toml:ro - - ./docker/data:/data + - /data - ./docker/logs:/logs command: - --addr=0.0.0.0:20160 @@ -333,7 +336,8 @@ services: CONNECT_CONFIG_STORAGE_TOPIC: docker-connect-configs CONNECT_OFFSET_STORAGE_TOPIC: docker-connect-offsets CONNECT_STATUS_STORAGE_TOPIC: docker-connect-status - CONNECT_KEY_CONVERTER: org.apache.kafka.connect.storage.StringConverter + CONNECT_KEY_CONVERTER: io.confluent.connect.avro.AvroConverter + CONNECT_KEY_CONVERTER_SCHEMA_REGISTRY_URL: 'http://schema-registry:8081' CONNECT_VALUE_CONVERTER: io.confluent.connect.avro.AvroConverter CONNECT_VALUE_CONVERTER_SCHEMA_REGISTRY_URL: 'http://schema-registry:8081' CONNECT_INTERNAL_KEY_CONVERTER: "org.apache.kafka.connect.json.JsonConverter" @@ -358,3 +362,13 @@ services: # Now launch Kafka Connect sleep infinity & /etc/confluent/docker/run + + kafka-connect-healthcheck: + image: devshawn/kafka-connect-healthcheck:0.1.0 + container_name: kafka-connect-healthcheck + depends_on: + - kafka-connect-01 + ports: + - 18083:18083 + environment: + HEALTHCHECK_CONNECT_URL: 'http://kafka-connect-01:8083' \ No newline at end of file diff --git a/docs/media/cdc_architecture.svg b/docs/media/cdc_architecture.svg index 55d13a9fcbd..e5c2cdfda7e 100644 --- a/docs/media/cdc_architecture.svg +++ b/docs/media/cdc_architecture.svg @@ -1 +1,84 @@ - + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + diff --git a/go.mod b/go.mod index 43053e0bba7..7f0482427a5 100644 --- a/go.mod +++ b/go.mod @@ -18,31 +18,36 @@ require ( 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/integralist/go-findroot v0.0.0-20160518114804-ac90681525dc github.com/jarcoal/httpmock v1.0.5 + github.com/jmoiron/sqlx v1.2.0 github.com/linkedin/goavro/v2 v2.9.7 github.com/mattn/go-shellwords v1.0.3 github.com/phayes/freeport v0.0.0-20180830031419-95f893ade6f2 + github.com/pingcap/br v0.0.0-20200821071758-97ac20a3f6b0 github.com/pingcap/check v0.0.0-20200212061837-5e12011dc712 - github.com/pingcap/errors v0.11.5-0.20190809092503-95897b64e011 - github.com/pingcap/failpoint v0.0.0-20200603062251-b230c36c413c - github.com/pingcap/kvproto v0.0.0-20200706115936-1e0910aabe6c + github.com/pingcap/errors v0.11.5-0.20200729012136-4e113ddee29e + github.com/pingcap/failpoint v0.0.0-20200702092429-9f69995143ce + github.com/pingcap/kvproto v0.0.0-20200818080353-7aaed8998596 github.com/pingcap/log v0.0.0-20200511115504-543df19646ad - github.com/pingcap/parser v0.0.0-20200731033026-84f62115187c - github.com/pingcap/pd/v4 v4.0.0-rc.2.0.20200714122454-1a64f969cb3c - github.com/pingcap/tidb v1.1.0-beta.0.20200731004449-a63fa79d90c5 - github.com/pingcap/tidb-tools v4.0.5-0.20200731060919-6a36d482e3f9+incompatible + github.com/pingcap/parser v0.0.0-20200803072748-fdf66528323d + github.com/pingcap/pd/v4 v4.0.5-0.20200817114353-e465cafe8a91 + github.com/pingcap/tidb v1.1.0-beta.0.20200820092836-c5b7658b0896 + github.com/pingcap/tidb-tools v4.0.5-0.20200817064459-ba61a7376547+incompatible github.com/prometheus/client_golang v1.5.1 github.com/r3labs/diff v1.1.0 github.com/spf13/cobra v1.0.0 github.com/spf13/pflag v1.0.5 github.com/stretchr/testify v1.5.1 + github.com/uber-go/atomic v1.3.2 github.com/vmihailenco/msgpack/v5 v5.0.0-beta.1 go.etcd.io/etcd v0.5.0-alpha.5.0.20200425165423-262c93980547 go.uber.org/zap v1.15.0 - golang.org/x/crypto v0.0.0-20200302210943-78000ba7a073 // indirect - golang.org/x/sync v0.0.0-20190911185100-cd5d95a43a6e - golang.org/x/text v0.3.2 + golang.org/x/sync v0.0.0-20200625203802-6e8e738ad208 + golang.org/x/text v0.3.3 golang.org/x/time v0.0.0-20191024005414-555d28b269f0 google.golang.org/genproto v0.0.0-20200113173426-e1de0a7b01eb // indirect google.golang.org/grpc v1.26.0 + gopkg.in/mgo.v2 v2.0.0-20190816093944-a6b53ec6cb22 // indirect + upper.io/db.v3 v3.7.1+incompatible ) diff --git a/go.sum b/go.sum index 5d44098ca7f..3a23db89849 100644 --- a/go.sum +++ b/go.sum @@ -4,22 +4,18 @@ cloud.google.com/go v0.38.0/go.mod h1:990N+gfupTy94rShfmMCWGDn0LpTmnzTp2qbd1dvSR cloud.google.com/go v0.44.1/go.mod h1:iSa0KzasP4Uvy3f1mN/7PiObzGgflwredwwASm/v6AU= cloud.google.com/go v0.44.2/go.mod h1:60680Gw3Yr4ikxnPRS/oxxkBccT6SA1yMk63TGekxKY= cloud.google.com/go v0.45.1/go.mod h1:RpBamKRgapWJb87xiFSdk4g1CME7QZg3uwTez+TSTjc= -cloud.google.com/go v0.46.3 h1:AVXDdKsrtX33oR9fbCMu/+c1o8Ofjq6Ku/MInaLVg5Y= cloud.google.com/go v0.46.3/go.mod h1:a6bKKbmY7er1mI7TEI4lsAkts/mkhTSZK8w33B4RAg0= cloud.google.com/go v0.50.0 h1:0E3eE8MX426vUOs7aHfI7aN1BrIzzzf4ccKCSfSjGmc= cloud.google.com/go v0.50.0/go.mod h1:r9sluTvynVuxRIOHXQEHMFffphuXHOMZMycpNR5e6To= -cloud.google.com/go/bigquery v1.0.1 h1:hL+ycaJpVE9M7nLoiXb/Pn10ENE2u+oddxbD8uu0ZVU= cloud.google.com/go/bigquery v1.0.1/go.mod h1:i/xbL2UlR5RvWAURpBYZTtm/cXjCha9lbfbpx4poX+o= cloud.google.com/go/bigquery v1.3.0 h1:sAbMqjY1PEQKZBWfbu6Y6bsupJ9c4QdHnzg/VvYTLcE= cloud.google.com/go/bigquery v1.3.0/go.mod h1:PjpwJnslEMmckchkHFfq+HTD2DmtT67aNFKH1/VBDHE= cloud.google.com/go/datastore v1.0.0 h1:Kt+gOPPp2LEPWp8CSfxhsM8ik9CcyE/gYu+0r+RnZvM= cloud.google.com/go/datastore v1.0.0/go.mod h1:LXYbyblFSglQ5pkeyhO+Qmw7ukd3C+pD7TKLgZqpHYE= -cloud.google.com/go/pubsub v1.0.1 h1:W9tAK3E57P75u0XLLR82LZyw8VpAnhmyTOxW9qzmyj8= cloud.google.com/go/pubsub v1.0.1/go.mod h1:R0Gpsv3s54REJCy4fxDixWD93lHJMoZTyQ2kNxGRt3I= cloud.google.com/go/pubsub v1.1.0 h1:9/vpR43S4aJaROxqQHQ3nH9lfyKKV0dC3vOmnw8ebQQ= cloud.google.com/go/pubsub v1.1.0/go.mod h1:EwwdRX2sKPjnvnqCa270oGRyludottCI76h+R3AArQw= cloud.google.com/go/storage v1.0.0/go.mod h1:IhtSnM/ZTZV8YYJWCY8RULGVqBDmpoyjwiyrjsg+URw= -cloud.google.com/go/storage v1.4.0 h1:KDdqY5VTXBTqpSbctVTt0mVvfanP6JZzNzLE0qNY100= cloud.google.com/go/storage v1.4.0/go.mod h1:ZusYJWlOshgSBGbt6K3GnB3MT3H1xs2id9+TCl4fDBA= cloud.google.com/go/storage v1.5.0 h1:RPUcBvDeYgQFMfQu1eBMq6piD1SXmLH+vK3qjewZPus= cloud.google.com/go/storage v1.5.0/go.mod h1:tpKbwo567HUNpVclU5sGELwQWBDZ8gh0ZeosJ0Rtdos= @@ -31,14 +27,11 @@ github.com/DATA-DOG/go-sqlmock v1.3.3 h1:CWUqKXe0s8A2z6qCgkP4Kru7wC11YoAnoupUKFD github.com/DATA-DOG/go-sqlmock v1.3.3/go.mod h1:f/Ixk793poVmq4qj/V1dPUg2JEAKC73Q5eFN3EC/SaM= github.com/Jeffail/gabs/v2 v2.5.1 h1:ANfZYjpMlfTTKebycu4X1AgkVWumFVDYQl7JwOr4mDk= github.com/Jeffail/gabs/v2 v2.5.1/go.mod h1:xCn81vdHKxFUuWWAaD5jCTQDNPBMh5pPs9IJ+NcziBI= -github.com/KyleBanks/depth v1.2.1 h1:5h8fQADFrWtarTdtDudMmGsC7GPbOAu6RVB3ffsVFHc= github.com/KyleBanks/depth v1.2.1/go.mod h1:jzSb9d0L43HxTQfT+oSA1EEp2q+ne2uh6XgeJcm8brE= github.com/OneOfOne/xxhash v1.2.2 h1:KMrpdQIwFcEqXDklaen+P1axHaj9BSKzvpUUfnHldSE= github.com/OneOfOne/xxhash v1.2.2/go.mod h1:HSdplMjZKSmBqAxg5vPj2TmRDmfkzw+cTzAElWljhcU= github.com/PuerkitoBio/purell v1.1.0/go.mod h1:c11w/QuzBsJSee3cPx9rAFu61PvFxuPbtSwDGJws/X0= -github.com/PuerkitoBio/purell v1.1.1 h1:WEQqlqaGbrPkxLJWfBwQmfEAE1Z7ONdDLqrN38tNFfI= github.com/PuerkitoBio/purell v1.1.1/go.mod h1:c11w/QuzBsJSee3cPx9rAFu61PvFxuPbtSwDGJws/X0= -github.com/PuerkitoBio/urlesc v0.0.0-20170810143723-de5bf2ad4578 h1:d+Bc7a5rLufV/sSk/8dngufqelfh6jnri85riMAaF/M= github.com/PuerkitoBio/urlesc v0.0.0-20170810143723-de5bf2ad4578/go.mod h1:uGdkoq3SwY9Y+13GIhn11/XLaGBb4BfwItxLd5jeuXE= github.com/Shopify/sarama v1.26.1 h1:3jnfWKD7gVwbB1KSy/lE0szA9duPuSFLViK0o/d3DgA= github.com/Shopify/sarama v1.26.1/go.mod h1:NbSGBSSndYaIhRcBtY9V0U7AyH+x71bG668AuWys/yU= @@ -61,7 +54,6 @@ github.com/ardielle/ardielle-go v1.5.2 h1:TilHTpHIQJ27R1Tl/iITBzMwiUGSlVfiVhwDNG github.com/ardielle/ardielle-go v1.5.2/go.mod h1:I4hy1n795cUhaVt/ojz83SNVCYIGsAFAONtv2Dr7HUI= github.com/ardielle/ardielle-tools v1.5.4/go.mod h1:oZN+JRMnqGiIhrzkRN9l26Cej9dEx4jeNG6A+AdkShk= github.com/armon/consul-api v0.0.0-20180202201655-eb2c6b5be1b6/go.mod h1:grANhF5doyWs3UAsr3K4I6qtAmlQcZDesFNEHPZAzj8= -github.com/aws/aws-sdk-go v1.26.1 h1:JGQggXhOiNJIqsmbYUl3cYtJZUffeOWlHtxfzGK7WPI= github.com/aws/aws-sdk-go v1.26.1/go.mod h1:KmX6BPdI08NWTb3/sm4ZGu5ShLoqVDhKgpiN924inxo= github.com/aws/aws-sdk-go v1.30.24 h1:y3JPD51VuEmVqN3BEDVm4amGpDma2cKJcDPuAU1OR58= github.com/aws/aws-sdk-go v1.30.24/go.mod h1:5zCpMtNQVjRREroY7sYe8lOMRSxkhG6MZveU8YkpAk0= @@ -83,13 +75,11 @@ 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/cheggaaa/pb/v3 v3.0.1 h1:m0BngUk2LuSRYdx4fujDKNRXNDpbNCfptPfVT2m6OJY= github.com/cheggaaa/pb/v3 v3.0.1/go.mod h1:SqqeMF/pMOIu3xgGoxtPYhMNQP258xE4x/XRTYua+KU= github.com/cheggaaa/pb/v3 v3.0.4 h1:QZEPYOj2ix6d5oEg63fbHmpolrnNiwjUsk+h74Yt4bM= github.com/cheggaaa/pb/v3 v3.0.4/go.mod h1:7rgWxLrAUcFMkvJuv09+DYi7mMUYi8nO9iOWcvGJPfw= github.com/chzyer/logex v1.1.10 h1:Swpa1K6QvQznwJRcfTfQJmTE72DqScAa40E+fbHEXEE= github.com/chzyer/logex v1.1.10/go.mod h1:+Ywpsq7O8HXn0nuIou7OrIPyXbp3wmkHB+jjWRnGsAI= -github.com/chzyer/readline v0.0.0-20171208011716-f6d7a1f6fbf3 h1:T7Bw4H6z3WAZ2khw+gfKdYmbKHyy5xiHtk9IHfZqm7g= github.com/chzyer/readline v0.0.0-20171208011716-f6d7a1f6fbf3/go.mod h1:nSuG5e5PlCu98SY8svDHJxuZscDgtXS6KTTbou5AhLI= github.com/chzyer/readline v0.0.0-20180603132655-2972be24d48e h1:fY5BOSpyZCqRo5OhCuC+XN+r/bBCmeuuJtjz+bCNIf8= github.com/chzyer/readline v0.0.0-20180603132655-2972be24d48e/go.mod h1:nSuG5e5PlCu98SY8svDHJxuZscDgtXS6KTTbou5AhLI= @@ -100,22 +90,13 @@ github.com/cockroachdb/datadriven v0.0.0-20190809214429-80d97fb3cbaa h1:OaNxuTZr 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/coocood/badger v1.5.1-0.20200515070411-e02af0688441/go.mod h1:klY8SfH2lNZ/23/SIxwHoJw+T6wYGB12YPCF9MUoiu0= -github.com/coocood/badger v1.5.1-0.20200528065104-c02ac3616d04 h1:prWWMbnhC2+Iw/dWM0imCzaU2xjaIMQZS9V+BZ1DoUE= -github.com/coocood/badger v1.5.1-0.20200528065104-c02ac3616d04/go.mod h1:klY8SfH2lNZ/23/SIxwHoJw+T6wYGB12YPCF9MUoiu0= -github.com/coocood/bbloom v0.0.0-20190830030839-58deb6228d64 h1:W1SHiII3e0jVwvaQFglwu3kS9NLxOeTpvik7MbKCyuQ= -github.com/coocood/bbloom v0.0.0-20190830030839-58deb6228d64/go.mod h1:F86k/6c7aDUdwSUevnLpHS/3Q9hzYCE99jGk2xsHnt0= -github.com/coocood/rtutil v0.0.0-20190304133409-c84515f646f2 h1:NnLfQ77q0G4k2Of2c1ceQ0ec6MkLQyDp+IGdVM0D8XM= -github.com/coocood/rtutil v0.0.0-20190304133409-c84515f646f2/go.mod h1:7qG7YFnOALvsx6tKTNmQot8d7cGFXM9TidzvRFLWYwM= github.com/coreos/bbolt v1.3.2/go.mod h1:iRUV2dpdMOn7Bo10OQBFzIJO9kkE559Wcmn+qkEiiKk= github.com/coreos/etcd v3.3.10+incompatible/go.mod h1:uF7uidLiAD3TWHmW31ZFd/JWoc32PjwdhPthX9715RE= github.com/coreos/go-etcd v2.0.0+incompatible/go.mod h1:Jez6KQU2B/sWsbdaef3ED8NzMklzPG4d5KIOhIy30Tk= -github.com/coreos/go-semver v0.2.0 h1:3Jm3tLmsgAYcjC+4Up7hJrFBPr+n7rAqYeSw/SZazuY= github.com/coreos/go-semver v0.2.0/go.mod h1:nnelYz7RCh+5ahJtPPxZlU+153eP4D4r3EedlOD2RNk= github.com/coreos/go-semver v0.3.0 h1:wkHLiw0WNATZnSG7epLsujiMCgPAc9xhjJ4tgnAxmfM= github.com/coreos/go-semver v0.3.0/go.mod h1:nnelYz7RCh+5ahJtPPxZlU+153eP4D4r3EedlOD2RNk= github.com/coreos/go-systemd v0.0.0-20180511133405-39ca1b05acc7/go.mod h1:F5haX7vjVVG0kc13fIWeqUViNPyEJxv/OmvnBo0Yme4= -github.com/coreos/go-systemd v0.0.0-20181031085051-9002847aa142 h1:3jFq2xL4ZajGK4aZY8jz+DAF0FHjI51BXjjSwCzS1Dk= github.com/coreos/go-systemd v0.0.0-20181031085051-9002847aa142/go.mod h1:F5haX7vjVVG0kc13fIWeqUViNPyEJxv/OmvnBo0Yme4= github.com/coreos/go-systemd v0.0.0-20190321100706-95778dfbb74e/go.mod h1:F5haX7vjVVG0kc13fIWeqUViNPyEJxv/OmvnBo0Yme4= github.com/coreos/go-systemd v0.0.0-20190719114852-fd7a80b32e1f h1:JOrtw2xFKzlg+cbHpyrpLDmnN1HqhBfnX7WDiW7eG2c= @@ -123,12 +104,9 @@ github.com/coreos/go-systemd v0.0.0-20190719114852-fd7a80b32e1f/go.mod h1:F5haX7 github.com/coreos/go-systemd/v22 v22.0.0 h1:XJIw/+VlJ+87J+doOxznsAWIdmWuViOVhkQamW5YV28= github.com/coreos/go-systemd/v22 v22.0.0/go.mod h1:xO0FLkIi5MaZafQlIrOotqXZ90ih+1atmu1JpKERPPk= 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/corona10/goimagehash v1.0.2/go.mod h1:/l9umBhvcHQXVtQO1V6Gp1yD20STawkhRnnX0D1bvVI= -github.com/cpuguy83/go-md2man v1.0.10 h1:BSKMNlYxDvnunlTymqtgONjNnaRV1sTpcovwwjF22jk= github.com/cpuguy83/go-md2man v1.0.10/go.mod h1:SmD6nW6nTyfqj6ABTjUi3V3JVMnlJmwcJI5acqYI6dE= -github.com/cpuguy83/go-md2man/v2 v2.0.0-20190314233015-f79a8a8ca69d h1:U+s90UTSYgptZMwQh2aRr3LuazLJIa+Pg3Kc1ylSYVY= github.com/cpuguy83/go-md2man/v2 v2.0.0-20190314233015-f79a8a8ca69d/go.mod h1:maD7wRr/U5Z6m/iR4s+kqSMx2CaBsrgA7czyZG/E6dU= github.com/cpuguy83/go-md2man/v2 v2.0.0/go.mod h1:maD7wRr/U5Z6m/iR4s+kqSMx2CaBsrgA7czyZG/E6dU= github.com/creack/pty v1.1.7/go.mod h1:lj5s0c3V2DBrqTV7llrYr5NG6My20zk30Fl46Y7DoTY= @@ -140,6 +118,7 @@ github.com/cznic/sortutil v0.0.0-20181122101858-f5f958428db8 h1:LpMLYGyy67BoAFGd 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/danjacques/gofslock v0.0.0-20191023191349-0a45f885bc37 h1:X6mKGhCFOxrKeeHAjv/3UvT6e5RRxW6wRdlqlV6/H4w= github.com/danjacques/gofslock v0.0.0-20191023191349-0a45f885bc37/go.mod h1:DC3JtzuG7kxMvJ6dZmf2ymjNyoXwgtklr7FN+Um2B0U= 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= @@ -149,7 +128,6 @@ github.com/dgraph-io/ristretto v0.0.1 h1:cJwdnj42uV8Jg4+KLrYovLiCgIfz9wtWm6E6KA+ 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/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/dgryski/go-sip13 v0.0.0-20181026042036-e10d5fee7954/go.mod h1:vAd38F8PWV+bWy6jNmig1y/TA+kYO4g3RSRF0IAv0no= @@ -169,9 +147,7 @@ github.com/eapache/queue v1.1.0 h1:YOEu7KNc61ntiQlcEeUIoDTJ2o8mQznoNvUhiigpIqc= github.com/eapache/queue v1.1.0/go.mod h1:6eCeP0CKFpHLu8blIFXhExK/dRa7WDZfr6jVFPTqq+I= github.com/edwingeng/deque v0.0.0-20191220032131-8596380dee17 h1:8i9x3Q4hW1kLE4ScsOtUlwVHT76LKhkmOw9zbDxnyUc= github.com/edwingeng/deque v0.0.0-20191220032131-8596380dee17/go.mod h1:3Ys1pJhyVaB6iWigv4o2r6Ug1GZmfDWqvqmO6bjojg0= -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= @@ -180,7 +156,6 @@ github.com/erikstmartin/go-testdb v0.0.0-20160219214506-8d10e4a1bae5/go.mod h1:a github.com/fatih/color v1.7.0/go.mod h1:Zm6kSWBoL9eyXnKyktHP6abPY2pDugNf5KwzbycvMj4= github.com/fatih/color v1.9.0 h1:8xPHl4/q1VyqGIPif1F+1V3Y3lSmrq01EabUW3CoW5s= github.com/fatih/color v1.9.0/go.mod h1:eQcE1qtQxscV5RaZvpXrrb8Drkc3/DdQ+uUYCNjL+zU= -github.com/fatih/structtag v1.2.0 h1:/OdNE99OxoI/PqaW/SuSK9uxxT3f/tcSZgon/ssNSx4= github.com/fatih/structtag v1.2.0/go.mod h1:mBJUNpUnHmRKrKlQQlmCrh5PuhftFbNv8Ys4/aAZl94= github.com/fogleman/gg v1.3.0/go.mod h1:R/bRT+9gY/C5z7JzPU0zXsXHKM4/ayA+zqcVNZzPa1k= github.com/fortytw2/leaktest v1.3.0 h1:u8491cBMTQ8ft8aeV+adlcytMZylmA5nnwwkRZjI8vw= @@ -189,12 +164,10 @@ github.com/frankban/quicktest v1.7.2 h1:2QxQoC1TS09S7fhCPsrvqYdvP1H5M1P1ih5ABm3B github.com/frankban/quicktest v1.7.2/go.mod h1:jaStnuzAqU1AJdCO0l53JDCJrVDKcS03DbaAcR7Ks/o= github.com/fsnotify/fsnotify v1.4.7 h1:IXs+QLmnXW2CcXuY+8Mzv/fWEsPGWxqefPtCP5CnV9I= github.com/fsnotify/fsnotify v1.4.7/go.mod h1:jwhsz4b93w/PPRr/qN1Yymfu8t87LnFCMoQvtojpjFo= -github.com/fsouza/fake-gcs-server v1.15.0 h1:ss/ztlt10Y64A5qslmxZKsiqW/i28t5DkRtv6qSFaLQ= github.com/fsouza/fake-gcs-server v1.15.0/go.mod h1:HNxAJ/+FY/XSsxuwz8iIYdp2GtMmPbJ8WQjjGMxd6Qk= github.com/fsouza/fake-gcs-server v1.17.0 h1:OeH75kBZcZa3ZE+zz/mFdJ2btt9FgqfjI7gIh9+5fvk= github.com/fsouza/fake-gcs-server v1.17.0/go.mod h1:D1rTE4YCyHFNa99oyJJ5HyclvN/0uQR+pM/VdlL83bw= github.com/ghodss/yaml v1.0.0/go.mod h1:4dBDuWmgqj2HViK6kFavaiC9ZROes6MMH2rRYeMEF04= -github.com/ghodss/yaml v1.0.1-0.20190212211648-25d852aebe32 h1:Mn26/9ZMNWSw9C9ERFA1PUxfmGpolnw2v0bKOREu5ew= github.com/ghodss/yaml v1.0.1-0.20190212211648-25d852aebe32/go.mod h1:GIjDIg/heH5DOkXY3YJ/wNhfHsQHoXGjl8G8amsYQ1I= github.com/gin-contrib/gzip v0.0.1/go.mod h1:fGBJBCdt6qCZuCAOwWuFhBB4OOq9EFqlo5dEaFhhu5w= github.com/gin-contrib/sse v0.0.0-20170109093832-22d885f9ecc7/go.mod h1:VJ0WA2NBN22VlZ2dKZQPAPnyWw5XTlK1KymzLKsr59s= @@ -216,29 +189,24 @@ github.com/go-ole/go-ole v1.2.4 h1:nNBDSCOigTSiarFpYE9J/KtEA1IOW4CNeqT9TQDqCxI= github.com/go-ole/go-ole v1.2.4/go.mod h1:XCwSNxSkXRo4vlyPy93sltvi/qJq0jqQhjqQNIwKuxM= github.com/go-openapi/jsonpointer v0.17.0/go.mod h1:cOnomiV+CVVwFLk0A/MExoFMjwdsUdVpsRhURCKh+3M= github.com/go-openapi/jsonpointer v0.19.2/go.mod h1:3akKfEdA7DF1sugOqz1dVQHBcuDBPKZGEoHC/NkiQRg= -github.com/go-openapi/jsonpointer v0.19.3 h1:gihV7YNZK1iK6Tgwwsxo2rJbD1GTbdm72325Bq8FI3w= github.com/go-openapi/jsonpointer v0.19.3/go.mod h1:Pl9vOtqEWErmShwVjC8pYs9cog34VGT37dQOVbmoatg= github.com/go-openapi/jsonreference v0.17.0/go.mod h1:g4xxGn04lDIRh0GJb5QlpE3HfopLOL6uZrK/VgnsK9I= github.com/go-openapi/jsonreference v0.19.0/go.mod h1:g4xxGn04lDIRh0GJb5QlpE3HfopLOL6uZrK/VgnsK9I= github.com/go-openapi/jsonreference v0.19.2/go.mod h1:jMjeRr2HHw6nAVajTXJ4eiUwohSTlpa0o73RUL1owJc= -github.com/go-openapi/jsonreference v0.19.3 h1:5cxNfTy0UVC3X8JL5ymxzyoUZmo8iZb+jeTWn7tUa8o= github.com/go-openapi/jsonreference v0.19.3/go.mod h1:rjx6GuL8TTa9VaixXglHmQmIL98+wF9xc8zWvFonSJ8= github.com/go-openapi/spec v0.19.0/go.mod h1:XkF/MOi14NmjsfZ8VtAKf8pIlbZzyoTvZsdfssdxcBI= github.com/go-openapi/spec v0.19.4/go.mod h1:FpwSN1ksY1eteniUU7X0N/BgJ7a4WvBFVA8Lj9mJglo= -github.com/go-openapi/spec v0.19.7 h1:0xWSeMd35y5avQAThZR2PkEuqSosoS5t6gDH4L8n11M= github.com/go-openapi/spec v0.19.7/go.mod h1:Hm2Jr4jv8G1ciIAo+frC/Ft+rR2kQDh8JHKHb3gWUSk= github.com/go-openapi/swag v0.17.0/go.mod h1:AByQ+nYG6gQg71GINrmuDXCPWdL640yX49/kXLo40Tg= github.com/go-openapi/swag v0.19.2/go.mod h1:POnQmlKehdgb5mhVOsnJFsivZCEZ/vjK9gh66Z9tfKk= github.com/go-openapi/swag v0.19.5/go.mod h1:POnQmlKehdgb5mhVOsnJFsivZCEZ/vjK9gh66Z9tfKk= -github.com/go-openapi/swag v0.19.8 h1:vfK6jLhs7OI4tAXkvkooviaE1JEPcw3mutyegLHHjmk= github.com/go-openapi/swag v0.19.8/go.mod h1:ao+8BpOPyKdpQz3AOJfbeEVpLmWAvlT1IfTe5McPyhY= github.com/go-playground/locales v0.12.1/go.mod h1:IUMDtCfWo/w/mtMfIE/IG2K+Ey3ygWanZIBtBW0W2TM= github.com/go-playground/locales v0.13.0/go.mod h1:taPMhCMXrRLJO55olJkUXHZBHCxTMfnGwq/HNwmWNS8= -github.com/go-playground/overalls v0.0.0-20180201144345-22ec1a223b7c h1:3bjbKXoj7jBYdHpQFbKL2546c4dtltTHzjo+5i4CHBU= github.com/go-playground/overalls v0.0.0-20180201144345-22ec1a223b7c/go.mod h1:UqxAgEOt89sCiXlrc/ycnx00LVvUO/eS8tMUkWX4R7w= github.com/go-playground/universal-translator v0.16.0/go.mod h1:1AnU7NaIRDWWzGEKwgtJRd2xk99HeFyHw3yid4rvQIY= github.com/go-playground/universal-translator v0.17.0/go.mod h1:UkSxE5sNxxRwHyU+Scu5vgOQjsIJAF8j9muTVoKLVtA= -github.com/go-sql-driver/mysql v1.4.1 h1:g24URVg0OFbNUTx9qqY1IRZ9D9z3iPyi5zKhQZpNwpA= +github.com/go-sql-driver/mysql v1.4.0/go.mod h1:zAC/RDZ24gD3HViQzih4MyKcchzm+sOG5ZlKdlhCg5w= github.com/go-sql-driver/mysql v1.4.1/go.mod h1:zAC/RDZ24gD3HViQzih4MyKcchzm+sOG5ZlKdlhCg5w= github.com/go-sql-driver/mysql v1.5.0 h1:ozyZYNQW3x3HtqT1jira07DN2PArx2v7/mN66gGcHOs= github.com/go-sql-driver/mysql v1.5.0/go.mod h1:DCzpHaOWr8IXmIStZouvnhqoel9Qv2LBy8hT2VhHyBg= @@ -266,11 +234,9 @@ github.com/golang/mock v1.3.1/go.mod h1:sBzyDLLjw3U8JLTeZvSv8jJB+tU5PVekmnlKIyFU 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/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= -github.com/golang/protobuf v1.3.2 h1:6nsPYzhq5kReh6QImI3k5qWzO4PEbvbIW2cwSfR/6xs= github.com/golang/protobuf v1.3.2/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= github.com/golang/protobuf v1.3.4 h1:87PNWwrRvUSnqS4dlcBU/ftvOIBep4sYuBLlh6rX2wk= github.com/golang/protobuf v1.3.4/go.mod h1:vzj43D7+SQXF/4pzW/hwtAqwc6iTitCiVSaWz5lYuqw= -github.com/golang/snappy v0.0.0-20180518054509-2e65f85255db h1:woRePGFeVFfLKN/pOkfl+p/TAqKOfFu+7KPlMVpok/w= github.com/golang/snappy v0.0.0-20180518054509-2e65f85255db/go.mod h1:/XxbfmMg8lxefKM7IXC3fBNl/7bRcc72aCRzEWrmP2Q= github.com/golang/snappy v0.0.1 h1:Qgr9rKW7uDUkrbSmQeiDsGa8SjGyCOGtuasMWwvp2P4= github.com/golang/snappy v0.0.1/go.mod h1:/XxbfmMg8lxefKM7IXC3fBNl/7bRcc72aCRzEWrmP2Q= @@ -279,7 +245,6 @@ 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.0/go.mod h1:8QqcDgzrUqlUb/G2PQTWiueGozuR1884gddMywk6iLU= -github.com/google/go-cmp v0.3.1 h1:Xye71clBPdm5HgqGwUkwhbynsUJZhDbS20FvLhQ2izg= github.com/google/go-cmp v0.3.1/go.mod h1:8QqcDgzrUqlUb/G2PQTWiueGozuR1884gddMywk6iLU= github.com/google/go-cmp v0.4.0 h1:xsAVV57WRhGj6kEIi8ReJzQlHHqcBYCElAvkovg3B/4= github.com/google/go-cmp v0.4.0/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= @@ -288,7 +253,6 @@ github.com/google/martian v2.1.0+incompatible h1:/CP5g8u/VJHijgedC/Legn3BAbAaWPg github.com/google/martian v2.1.0+incompatible/go.mod h1:9I4somxYTbIHy5NJKHRl3wXiIaQGbYVAs8BPL6v8lEs= github.com/google/pprof v0.0.0-20181206194817-3ea8567a2e57/go.mod h1:zfwlbNMJ+OItoe0UupaVj+oy1omPYYDuagoSzA8v9mc= github.com/google/pprof v0.0.0-20190515194954-54271f7e092f/go.mod h1:zfwlbNMJ+OItoe0UupaVj+oy1omPYYDuagoSzA8v9mc= -github.com/google/pprof v0.0.0-20190930153522-6ce02741cba3 h1:3CYI9xg87xNAD+es02gZxbX/ky4KQeoFBsNOzuoAQZg= github.com/google/pprof v0.0.0-20190930153522-6ce02741cba3/go.mod h1:zfwlbNMJ+OItoe0UupaVj+oy1omPYYDuagoSzA8v9mc= github.com/google/pprof v0.0.0-20200407044318-7d83b28da2e9 h1:K+lX49/3eURCE1IjlaZN//u6c+9nfDAMnyQ9E2dsJbY= github.com/google/pprof v0.0.0-20200407044318-7d83b28da2e9/go.mod h1:ZgVRPoUq/hfqzAqh7sHMqb3I9Rq5C59dIz2SbBwJ4eM= @@ -318,7 +282,6 @@ github.com/grpc-ecosystem/go-grpc-prometheus v1.2.0 h1:Ovs26xHkKqVztRpIrF/92Bcuy github.com/grpc-ecosystem/go-grpc-prometheus v1.2.0/go.mod h1:8NvIoxWQoOIhqOTXgfV/d3M/q6VIi02HzZEHgUlZvzk= github.com/grpc-ecosystem/grpc-gateway v1.9.0/go.mod h1:vNeuVxBJEsws4ogUvrchl83t/GYV9WGTSLVdBhOQFDY= github.com/grpc-ecosystem/grpc-gateway v1.9.5/go.mod h1:vNeuVxBJEsws4ogUvrchl83t/GYV9WGTSLVdBhOQFDY= -github.com/grpc-ecosystem/grpc-gateway v1.12.1 h1:zCy2xE9ablevUOrUZc3Dl72Dt+ya2FNAvC2yLYMHzi4= github.com/grpc-ecosystem/grpc-gateway v1.12.1/go.mod h1:8XEsbTttt/W+VvjtQhLACqCisSPWTxCZ7sBRjU6iH9c= github.com/grpc-ecosystem/grpc-gateway v1.14.3 h1:OCJlWkOUoTnl0neNGlf4fUm3TmbEtguw7vR+nGtnDjY= github.com/grpc-ecosystem/grpc-gateway v1.14.3/go.mod h1:6CwZWGDSPRJidgKAtJVvND6soZe6fT7iteq8wDPdhb0= @@ -327,7 +290,6 @@ github.com/hashicorp/go-uuid v1.0.2 h1:cfejS+Tpcp13yd5nYHWDI6qVCny6wyX2Mt5SGur2I github.com/hashicorp/go-uuid v1.0.2/go.mod h1:6SBZvOh/SIDV7/2o3Jml5SYk/TvGqwFJ/bN7x4byOro= github.com/hashicorp/golang-lru v0.5.0/go.mod h1:/m3WP610KZHVQ1SGc6re/UDhFvYD7pJ4Ao+sR/qLZy8= github.com/hashicorp/golang-lru v0.5.1/go.mod h1:/m3WP610KZHVQ1SGc6re/UDhFvYD7pJ4Ao+sR/qLZy8= -github.com/hashicorp/hcl v1.0.0 h1:0Anlzjpi4vEasTeNFn2mLJgTSwt0+6sfsiTG8qcWGx4= 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= @@ -337,6 +299,8 @@ github.com/iancoleman/strcase v0.0.0-20191112232945-16388991a334/go.mod h1:SK73t github.com/ianlancetaylor/demangle v0.0.0-20181102032728-5e5cf60278f6/go.mod h1:aSSvb/t6k1mPoxDqO4vJh6VOCGPwU4O0C2/Eqndh1Sc= github.com/inconshreveable/mousetrap v1.0.0 h1:Z8tu5sraLXCXIcARxBp/8cbvlwVa7Z1NHg9XEKhtSvM= github.com/inconshreveable/mousetrap v1.0.0/go.mod h1:PxqpIevigyE2G7u3NXJIT2ANytuPF1OarO4DADm73n8= +github.com/integralist/go-findroot v0.0.0-20160518114804-ac90681525dc h1:4IZpk3M4m6ypx0IlRoEyEyY1gAdicWLMQ0NcG/gBnnA= +github.com/integralist/go-findroot v0.0.0-20160518114804-ac90681525dc/go.mod h1:UlaC6ndby46IJz9m/03cZPKKkR9ykeIVBBDE3UDBdJk= github.com/jarcoal/httpmock v1.0.5 h1:cHtVEcTxRSX4J0je7mWPfc9BpDpqzXSJ5HbymZmyHck= github.com/jarcoal/httpmock v1.0.5/go.mod h1:ATjnClrvW/3tijVmpL/va5Z3aAyGvqU3gCT8nX0Txik= github.com/jawher/mow.cli v1.0.4/go.mod h1:5hQj2V8g+qYmLUVWqu4Wuja1pI57M83EChYLVZ0sMKk= @@ -347,10 +311,11 @@ github.com/jessevdk/go-flags v1.4.0/go.mod h1:4FA24M0QyGHXBuZZK/XkWh8h0e1EYbRYJS github.com/jinzhu/gorm v1.9.12/go.mod h1:vhTjlKSJUTWNtcbQtrMBFCxy7eXTzeCAzfL5fBZT/Qs= github.com/jinzhu/inflection v1.0.0/go.mod h1:h+uFLlag+Qp1Va5pdKtLDYj+kHp5pxUVkryuEj+Srlc= github.com/jinzhu/now v1.0.1/go.mod h1:d3SSVoowX0Lcu0IBviAWJpolVfI5UJVZZ7cO71lE/z8= -github.com/jmespath/go-jmespath v0.0.0-20180206201540-c2b33e8439af h1:pmfjZENx5imkbgOkpRUYLnmbU7UEFbjtDA2hxJ1ichM= github.com/jmespath/go-jmespath v0.0.0-20180206201540-c2b33e8439af/go.mod h1:Nht3zPeWKUH0NzdCt2Blrr5ys8VGpn0CEB0cQHVjt7k= github.com/jmespath/go-jmespath v0.3.0 h1:OS12ieG61fsCg5+qLJ+SsW9NicxNkg3b25OyT2yCeUc= github.com/jmespath/go-jmespath v0.3.0/go.mod h1:9QtRXoHjLGCJ5IBSaohpXITPlowMeeYCZ7fLUTSywik= +github.com/jmoiron/sqlx v1.2.0 h1:41Ip0zITnmWNR/vHV+S4m+VoUivnWY5E4OJfLZjCJMA= +github.com/jmoiron/sqlx v1.2.0/go.mod h1:1FEQNm3xlJgrMD+FBdI9+xvCksHtbpVBBw5dYhBSsks= github.com/joho/godotenv v1.3.0/go.mod h1:7hK45KPybAkOC6peb+G5yklZfMxEjkZhHbwpqxOKXbg= github.com/jonboulle/clockwork v0.1.0 h1:VKV+ZcuP6l3yW9doeqz6ziZGgcynBVQO+obU0+0hcPo= github.com/jonboulle/clockwork v0.1.0/go.mod h1:Ii8DK3G1RaLaWxj9trq07+26W01tbo22gdxWY5EU2bo= @@ -360,49 +325,28 @@ github.com/json-iterator/go v1.1.6/go.mod h1:+SdeFBvtyEkXs7REEP0seUULqWtbJapLOCV github.com/json-iterator/go v1.1.7/go.mod h1:KdQUCv79m/52Kvf8AW2vK1V8akMuk1QjK/uOdHXbAo4= github.com/json-iterator/go v1.1.9 h1:9yzud/Ht36ygwatGx56VwCZtlI/2AD15T1X2sjSuGns= github.com/json-iterator/go v1.1.9/go.mod h1:KdQUCv79m/52Kvf8AW2vK1V8akMuk1QjK/uOdHXbAo4= -github.com/jstemmer/go-junit-report v0.0.0-20190106144839-af01ea7f8024 h1:rBMNdlhTLzJjJSDIjNEXX1Pz3Hmwmz91v+zycvx9PJc= github.com/jstemmer/go-junit-report v0.0.0-20190106144839-af01ea7f8024/go.mod h1:6v2b51hI/fHJwM22ozAgKL4VKDeJcHhJFhtBdhmNjmU= github.com/jstemmer/go-junit-report v0.9.1 h1:6QPYqodiu3GuPL+7mfx+NwDdp2eTkp9IfEUpgAwUN0o= github.com/jstemmer/go-junit-report v0.9.1/go.mod h1:Brl9GWCQeLvo8nXZwPNNblvFj/XSXhF0NWZEnDohbsk= -github.com/juju/clock v0.0.0-20180524022203-d293bb356ca4/go.mod h1:nD0vlnrUjcjJhqN5WuCWZyzfd5AHZAC9/ajvbSx69xA= -github.com/juju/errors v0.0.0-20150916125642-1b5e39b83d18/go.mod h1:W54LbzXuIE0boCoNJfwqpmkKJ1O4TCTZMetAt6jGk7Q= -github.com/juju/errors v0.0.0-20181118221551-089d3ea4e4d5 h1:rhqTjzJlm7EbkELJDKMTU7udov+Se0xZkWmugr6zGok= -github.com/juju/errors v0.0.0-20181118221551-089d3ea4e4d5/go.mod h1:W54LbzXuIE0boCoNJfwqpmkKJ1O4TCTZMetAt6jGk7Q= -github.com/juju/loggo v0.0.0-20170605014607-8232ab8918d9 h1:Y+lzErDTURqeXqlqYi4YBYbDd7ycU74gW1ADt57/bgY= -github.com/juju/loggo v0.0.0-20170605014607-8232ab8918d9/go.mod h1:vgyd7OREkbtVEN/8IXZe5Ooef3LQePvuBm9UWj6ZL8U= github.com/juju/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/juju/retry v0.0.0-20160928201858-1998d01ba1c3/go.mod h1:OohPQGsr4pnxwD5YljhQ+TZnuVRYpa5irjugL1Yuif4= -github.com/juju/testing v0.0.0-20200510222523-6c8c298c77a0 h1:+WWUkhnTjV6RNOxkcwk79qrjeyHEHvBzlneueBsatX4= -github.com/juju/testing v0.0.0-20200510222523-6c8c298c77a0/go.mod h1:hpGvhGHPVbNBraRLZEhoQwFLMrjK8PSlO4D3nDjKYXo= -github.com/juju/testing v0.0.0-20200608005635-e4eedbc6f7aa h1:v1ZEHRVaUgTIkxzYaT78fJ+3bV3vjxj9jfNJcYzi9pY= -github.com/juju/testing v0.0.0-20200608005635-e4eedbc6f7aa/go.mod h1:hpGvhGHPVbNBraRLZEhoQwFLMrjK8PSlO4D3nDjKYXo= -github.com/juju/utils v0.0.0-20180808125547-9dfc6dbfb02b/go.mod h1:6/KLg8Wz/y2KVGWEpkK9vMNGkOnu4k/cqs8Z1fKjTOk= -github.com/juju/version v0.0.0-20161031051906-1f41e27e54f2/go.mod h1:kE8gK5X0CImdr7qpSKl3xB2PmpySSmfj7zVbkZFs81U= github.com/julienschmidt/httprouter v1.2.0/go.mod h1:SYymIcj16QtmaHHD7aYtjjsJG7VTCxuUUipMqKk8s4w= -github.com/kevinburke/go-bindata v3.18.0+incompatible h1:NfOP49jFW7KyBl7UwTg0xkhSfHjESEwe2VMrcnSHG20= github.com/kevinburke/go-bindata v3.18.0+incompatible/go.mod h1:/pEEZ72flUW2p0yi30bslSp9YqD9pysLxunQDdb2CPM= github.com/kisielk/errcheck v1.1.0/go.mod h1:EZBBE59ingxPouuu3KfxchcWSUPOHkagtvWXihfKN4Q= github.com/kisielk/errcheck v1.2.0/go.mod h1:/BMXB+zMLi60iA8Vv6Ksmxu/1UDYcXs4uQLJ+jE2L00= github.com/kisielk/gotool v1.0.0/go.mod h1:XhKaO+MFFWcvkIS/tQcRk01m1F5IRFswLeQ+oQHNcck= -github.com/klauspost/compress v1.9.5/go.mod h1:RyIbtBH6LamlWaDj8nUwkbUhJ87Yi3uG0guNDohfE1A= -github.com/klauspost/compress v1.9.8 h1:VMAMUUOh+gaxKTMk+zqbjsSjsIcUcL/LF4o63i82QyA= github.com/klauspost/compress v1.9.8/go.mod h1:RyIbtBH6LamlWaDj8nUwkbUhJ87Yi3uG0guNDohfE1A= github.com/klauspost/compress v1.10.8 h1:eLeJ3dr/Y9+XRfJT4l+8ZjmtB5RPJhucH2HeCV5+IZY= github.com/klauspost/compress v1.10.8/go.mod h1:aoV0uJVorq1K+umq18yTdKaF57EivdYsUV+/s2qKfXs= -github.com/klauspost/cpuid v0.0.0-20170728055534-ae7887de9fa5 h1:2U0HzY8BJ8hVwDKIzp7y4voR9CX/nvcfymLmg2UiOio= github.com/klauspost/cpuid v0.0.0-20170728055534-ae7887de9fa5/go.mod h1:Pj4uuM528wm8OyEC2QMXAi2YiTZ96dNQPGgoMS4s3ek= -github.com/klauspost/cpuid v1.2.0 h1:NMpwD2G9JSFOE1/TJjGSo5zG7Yb2bTe7eq1jH+irmeE= github.com/klauspost/cpuid v1.2.0/go.mod h1:Pj4uuM528wm8OyEC2QMXAi2YiTZ96dNQPGgoMS4s3ek= github.com/klauspost/cpuid v1.2.1 h1:vJi+O/nMdFt0vqm8NZBI6wzALWdA2X+egi0ogNyrC/w= github.com/klauspost/cpuid v1.2.1/go.mod h1:Pj4uuM528wm8OyEC2QMXAi2YiTZ96dNQPGgoMS4s3ek= github.com/konsorten/go-windows-terminal-sequences v1.0.1/go.mod h1:T0+1ngSBFLxvqU3pZ+m/2kptfBszLMUkC4ZK/EgS/cQ= -github.com/konsorten/go-windows-terminal-sequences v1.0.2 h1:DB17ag19krx9CFsz4o3enTrPXyIXCl+2iCXH/aMAp9s= github.com/konsorten/go-windows-terminal-sequences v1.0.2/go.mod h1:T0+1ngSBFLxvqU3pZ+m/2kptfBszLMUkC4ZK/EgS/cQ= github.com/konsorten/go-windows-terminal-sequences v1.0.3 h1:CE8S1cTafDpPvMhIxNJKvHsGVBgn1xWYf1NbHQhywc8= github.com/konsorten/go-windows-terminal-sequences v1.0.3/go.mod h1:T0+1ngSBFLxvqU3pZ+m/2kptfBszLMUkC4ZK/EgS/cQ= github.com/kr/logfmt v0.0.0-20140226030751-b84e30acd515/go.mod h1:+0opPa2QZZtGFBFZlji/RkVcI2GknAs/DXo4wKdlNEc= -github.com/kr/pretty v0.1.0 h1:L/CwN0zerZDmRFUapSPitk6f+Q3+0za1rQkzVuMiMFI= github.com/kr/pretty v0.1.0/go.mod h1:dAy3ld7l9f0ibDNOQOHHMYYIIbhfbHSm3C4ZsoJORNo= github.com/kr/pretty v0.2.0 h1:s5hAObm+yFO5uHYt5dYjxi2rXrsnmRpJx4OYvIWUaQs= github.com/kr/pretty v0.2.0/go.mod h1:ipq/a2n7PKx3OHsz4KJII5eveXtPO4qwEXGdVfWzfnI= @@ -412,43 +356,44 @@ 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/leodido/go-urn v1.1.0/go.mod h1:+cyI34gQWZcE1eQU7NVgKkkzdXDQHr1dBMtdAPozLkw= github.com/leodido/go-urn v1.2.0/go.mod h1:+8+nEpDfqqsY+g338gtMEUOtuK+4dEMhiQEgxpxOKII= +github.com/lib/pq v1.0.0/go.mod h1:5WUZQaWbwv1U+lTReE5YruASi9Al49XbQIvNi/34Woo= +github.com/lib/pq v1.1.1 h1:sJZmqHoEaY7f+NPP8pgLB/WxulyR3fewgCM2qaSlBb4= github.com/lib/pq v1.1.1/go.mod h1:5WUZQaWbwv1U+lTReE5YruASi9Al49XbQIvNi/34Woo= github.com/linkedin/goavro/v2 v2.9.7 h1:Vd++Rb/RKcmNJjM0HP/JJFMEWa21eUBVKPYlKehOGrM= github.com/linkedin/goavro/v2 v2.9.7/go.mod h1:UgQUb2N/pmueQYH9bfqFioWxzYCZXSfF8Jw03O5sjqA= -github.com/magiconair/properties v1.8.0 h1:LLgXmsheXeRoUOBOjtwPQCWIYqM/LU1ayDtDePerRcY= github.com/magiconair/properties v1.8.0/go.mod h1:PppfXfuXeibc/6YijjN8zIbojt8czPbwD3XqdrwzmxQ= github.com/mailru/easyjson v0.0.0-20180823135443-60711f1a8329/go.mod h1:C1wdFJiN94OJF2b5HbByQZoLdCWB1Yqtg26g4irojpc= github.com/mailru/easyjson v0.0.0-20190614124828-94de47d64c63/go.mod h1:C1wdFJiN94OJF2b5HbByQZoLdCWB1Yqtg26g4irojpc= github.com/mailru/easyjson v0.0.0-20190626092158-b2ccc519800e/go.mod h1:C1wdFJiN94OJF2b5HbByQZoLdCWB1Yqtg26g4irojpc= -github.com/mailru/easyjson v0.7.1 h1:mdxE1MF9o53iCb2Ghj1VfWvh7ZOwHpnVG/xwXrV90U8= github.com/mailru/easyjson v0.7.1/go.mod h1:KAzv3t3aY1NaHWoQz1+4F1ccyAH66Jk7yos7ldAVICs= github.com/mattn/go-colorable v0.0.9/go.mod h1:9vuHe8Xs5qXnSaW/c/ABM9alt+Vo+STaOChaDxuIBZU= github.com/mattn/go-colorable v0.1.2/go.mod h1:U0ppj6V5qS13XJ6of8GYAs25YV2eR4EVcfRqFIhoBtE= -github.com/mattn/go-colorable v0.1.4 h1:snbPLB8fVfU9iwbbo30TPtbLRzwWu6aJS6Xh4eaaviA= github.com/mattn/go-colorable v0.1.4/go.mod h1:U0ppj6V5qS13XJ6of8GYAs25YV2eR4EVcfRqFIhoBtE= +github.com/mattn/go-colorable v0.1.7 h1:bQGKb3vps/j0E9GfJQ03JyhRuxsvdAanXlT9BTw3mdw= +github.com/mattn/go-colorable v0.1.7/go.mod h1:u6P/XSegPjTcexA+o6vUJrdnUu04hMope9wVRipJSqc= github.com/mattn/go-isatty v0.0.4/go.mod h1:M+lRXTBqGeGNdLjl/ufCoiOlB5xdOkqRJdNxMWT7Zi4= github.com/mattn/go-isatty v0.0.7/go.mod h1:Iq45c/XA43vh69/j3iqttzPXn0bhXyGjM0Hdxcsrc5s= github.com/mattn/go-isatty v0.0.8/go.mod h1:Iq45c/XA43vh69/j3iqttzPXn0bhXyGjM0Hdxcsrc5s= github.com/mattn/go-isatty v0.0.9/go.mod h1:YNRxwqDuOph6SZLI9vUUz6OYw3QyUt7WiY2yME+cCiQ= github.com/mattn/go-isatty v0.0.10/go.mod h1:qgIWMr58cqv1PHHyhnkY9lrL7etaEgOFcMEpPG5Rm84= -github.com/mattn/go-isatty v0.0.11 h1:FxPOTFNqGkuDUGi3H/qkUbQO4ZiBa2brKq5r0l8TGeM= github.com/mattn/go-isatty v0.0.11/go.mod h1:PhnuNfih5lzO57/f3n+odYbM4JtupLOxQOAqxQCu2WE= +github.com/mattn/go-isatty v0.0.12 h1:wuysRhFDzyxgEmMf5xjvJ2M9dZoWAXNNr5LSBS7uHXY= +github.com/mattn/go-isatty v0.0.12/go.mod h1:cbi8OIDigv2wuxKPP5vlRcQ1OAZbq2CE4Kysco4FUpU= github.com/mattn/go-runewidth v0.0.2/go.mod h1:LwmH8dsx7+W8Uxz3IHJYH5QSwggIsqBzpuz5H//U1FU= github.com/mattn/go-runewidth v0.0.4/go.mod h1:LwmH8dsx7+W8Uxz3IHJYH5QSwggIsqBzpuz5H//U1FU= -github.com/mattn/go-runewidth v0.0.7 h1:Ei8KR0497xHyKJPAv59M1dkC+rOZCMBJ+t3fZ+twI54= github.com/mattn/go-runewidth v0.0.7/go.mod h1:H031xJmbD/WCDINGzjvQ9THkh0rPKHF+m2gUSrubnMI= +github.com/mattn/go-runewidth v0.0.9 h1:Lm995f3rfxdpd6TSmuVCHVb/QhupuXlYr8sCI/QdE+0= +github.com/mattn/go-runewidth v0.0.9/go.mod h1:H031xJmbD/WCDINGzjvQ9THkh0rPKHF+m2gUSrubnMI= github.com/mattn/go-shellwords v1.0.3 h1:K/VxK7SZ+cvuPgFSLKi5QPI9Vr/ipOf4C1gN+ntueUk= github.com/mattn/go-shellwords v1.0.3/go.mod h1:3xCvwCdWdlDJUrvuMn7Wuy9eWs4pE8vqg+NOMyg4B2o= +github.com/mattn/go-sqlite3 v1.9.0/go.mod h1:FPy6KqzDD04eiIsT53CuJW3U88zkxoIYsOqkbpncsNc= +github.com/mattn/go-sqlite3 v2.0.1+incompatible h1:xQ15muvnzGBHpIpdrNi1DA5x0+TcBZzsIDwmw9uTHzw= github.com/mattn/go-sqlite3 v2.0.1+incompatible/go.mod h1:FPy6KqzDD04eiIsT53CuJW3U88zkxoIYsOqkbpncsNc= github.com/matttproud/golang_protobuf_extensions v1.0.1 h1:4hp9jkHxhMHkqkrB3Ix0jegS5sx/RkqARlsWZ6pIwiU= github.com/matttproud/golang_protobuf_extensions v1.0.1/go.mod h1:D8He9yQNgCq6Z5Ld7szi9bcBfOoFv/3dc6xSMkL2PC0= -github.com/mgechev/dots v0.0.0-20190921121421-c36f7dcfbb81 h1:QASJXOGm2RZ5Ardbc86qNFvby9AqkLDibfChMtAg5QM= github.com/mgechev/dots v0.0.0-20190921121421-c36f7dcfbb81/go.mod h1:KQ7+USdGKfpPjXk4Ga+5XxQM4Lm4e3gAogrreFAYpOg= -github.com/mgechev/revive v1.0.2 h1:v0NxxQ7fSFz/u1NQydPo6EGdq7va0J1BtsZmae6kzUg= github.com/mgechev/revive v1.0.2/go.mod h1:rb0dQy1LVAxW9SWy5R3LPUjevzUbUS316U5MFySA2lo= -github.com/mitchellh/go-homedir v1.1.0 h1:lukF9ziXFxDFPkA1vsr5zpc1XuPDn/wFntq5mG+4E0Y= github.com/mitchellh/go-homedir v1.1.0/go.mod h1:SfyaCUpYCn1Vlf4IUYiD9fPX4A5wJrkLzIz1N1q0pr0= -github.com/mitchellh/mapstructure v1.1.2 h1:fmNYVwqnSfB9mZU6OS2O6GsXM+wcskZDuKQzvN1EDeE= 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= @@ -457,36 +402,23 @@ github.com/modern-go/reflect2 v0.0.0-20180701023420-4b7aa43c6742/go.mod h1:bx2lN github.com/modern-go/reflect2 v1.0.1 h1:9f412s+6RmYXLWZSEzVVgPGK7C2PphHj5RJrvfx9AWI= github.com/modern-go/reflect2 v1.0.1/go.mod h1:bx2lNnkwVCuqBIxFjflWJWanXIb3RllmbCylyMrvgv0= github.com/montanaflynn/stats v0.0.0-20151014174947-eeaced052adb/go.mod h1:wL8QJuTMNUDYhXwkmfOly8iTdp5TEcJFWZD2D7SIkUc= -github.com/montanaflynn/stats v0.0.0-20180911141734-db72e6cae808 h1:pmpDGKLw4n82EtrNiLqB+xSz/JQwFOaZuMALYUHwX5s= github.com/montanaflynn/stats v0.0.0-20180911141734-db72e6cae808/go.mod h1:wL8QJuTMNUDYhXwkmfOly8iTdp5TEcJFWZD2D7SIkUc= github.com/montanaflynn/stats v0.5.0 h1:2EkzeTSqBB4V4bJwWrt5gIIrZmpJBcoIRGS2kWLgzmk= github.com/montanaflynn/stats v0.5.0/go.mod h1:wL8QJuTMNUDYhXwkmfOly8iTdp5TEcJFWZD2D7SIkUc= github.com/mwitkow/go-conntrack v0.0.0-20161129095857-cc309e4a2223/go.mod h1:qRWi+5nqEBWmkhHvq77mSJWrCKwh8bxhgT7d/eI7P4U= -github.com/ncw/directio v1.0.4 h1:CojwI07mCEmRkajgx42Pf8jyCwTs1ji9/Ij9/PJG12k= -github.com/ncw/directio v1.0.4/go.mod h1:CKGdcN7StAaqjT7Qack3lAXeX4pjnyc46YeqZH1yWVY= github.com/nfnt/resize v0.0.0-20160724205520-891127d8d1b5/go.mod h1:jpp1/29i3P1S/RLdc7JQKbRpFeM1dOBd8T9ki5s+AY8= 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= github.com/ngaut/sync2 v0.0.0-20141008032647-7a24ed77b2ef/go.mod h1:7WjlapSfwQyo6LNmIvEWzsW1hbBQfpUO4JWnuQRmva8= -github.com/ngaut/unistore v0.0.0-20200521040325-2af94f1b0c83/go.mod h1:odn0MiR+DNxnxOiCskG4wWacBIW2GBmJOf/TAnMZfWE= -github.com/ngaut/unistore v0.0.0-20200603091253-e0b717679796 h1:mhOUY30ysXVcRuftTiFgbw//GERhoM/65u5jvr9LJ7s= -github.com/ngaut/unistore v0.0.0-20200603091253-e0b717679796/go.mod h1:9mpqZeS1CkNlgZwJ0LZXb+Qd7xVO5o55ngys7T1/oH8= -github.com/ngaut/unistore v0.0.0-20200604043635-5004cdad650f/go.mod h1:5Vec+R2BwOyugVQ8Id8uDmlIYbqodCvykM50IpaAjk4= -github.com/ngaut/unistore v0.0.0-20200604061006-d8e9dc0ad154/go.mod h1:YGQzxn9cVy0q2puXVt1X8l5OohRHv2djR/ziz1k14XQ= -github.com/ngaut/unistore v0.0.0-20200630072006-0c4035925f69 h1:tn65bmj+EBa8gXYKQHPRlGcDdo+tDEjUK88gO4QK11s= -github.com/ngaut/unistore v0.0.0-20200630072006-0c4035925f69/go.mod h1:Hxlp5VAoPOHwcXLUw/E+P3XjJX1EP38NWjXPpc4nuOE= github.com/nicksnyder/go-i18n v1.10.0/go.mod h1:HrK7VCrbOvQoUAQ7Vpy7i87N7JZZZ7R2xBGjv0j365Q= github.com/oklog/ulid v1.3.1/go.mod h1:CirwcVhetQ6Lv90oh/F+FBtV6XMibvdAFo93nm5qn4U= github.com/olekukonko/tablewriter v0.0.0-20170122224234-a0225b3f23b5/go.mod h1:vsDQFd/mU46D+Z4whnwzcISnGGzXWMclvtLoiIKAKIo= -github.com/olekukonko/tablewriter v0.0.4 h1:vHD/YYe1Wolo78koG299f7V/VAS08c6IpCLn+Ejf/w8= github.com/olekukonko/tablewriter v0.0.4/go.mod h1:zq6QwlOf5SlnkVbMSr5EoBv3636FWnp+qbPhuoO21uA= -github.com/onsi/ginkgo v1.6.0 h1:Ix8l273rp3QzYgXSR+c8d1fTG7UPgYkOSELPhiY/YGw= github.com/onsi/ginkgo v1.6.0/go.mod h1:lLunBs/Ym6LB5Z9jYTR76FiuTmxDTDusOGeTQH+WWjE= github.com/onsi/ginkgo v1.7.0/go.mod h1:lLunBs/Ym6LB5Z9jYTR76FiuTmxDTDusOGeTQH+WWjE= github.com/onsi/ginkgo v1.11.0 h1:JAKSXpt1YjtLA7YpPiqO9ss6sNXEsPfSGdwN0UHqzrw= github.com/onsi/ginkgo v1.11.0/go.mod h1:lLunBs/Ym6LB5Z9jYTR76FiuTmxDTDusOGeTQH+WWjE= -github.com/onsi/gomega v1.4.2 h1:3mYCb7aPxS/RU7TI1y4rkEn1oKmPRjNJLNEXgw7MH2I= github.com/onsi/gomega v1.4.2/go.mod h1:ex+gbHU/CVuBBDIJjb2X0qEXbFg53c61hWP/1CpauHY= github.com/onsi/gomega v1.4.3/go.mod h1:ex+gbHU/CVuBBDIJjb2X0qEXbFg53c61hWP/1CpauHY= github.com/onsi/gomega v1.8.1 h1:C5Dqfs/LeauYDX0jJXIe2SWmwCbGzx9yF8C8xy3Lh34= @@ -497,9 +429,7 @@ github.com/opentracing/opentracing-go v1.0.2/go.mod h1:UkNAQd3GIcIGf0SeVgPpRdFSt 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 h1:e5+lF2E4Y2WCIxBefVowBuB0iHrUH4HZ8q+6mGF7fJc= github.com/pelletier/go-toml v1.3.0/go.mod h1:PN7xzY2wHTK0K9p34ErDQMlFxa51Fk0OUruD3k1mMwo= -github.com/petermattis/goid v0.0.0-20180202154549-b0b1615b78e5 h1:q2e307iGHPdTGp0hoxKjt1H5pDo6utceo3dQVK3I5XQ= github.com/petermattis/goid v0.0.0-20180202154549-b0b1615b78e5/go.mod h1:jvVRKCrJTQWu0XVbaOlby/2lO20uSCHEMzzplHXte1o= github.com/phayes/freeport v0.0.0-20180830031419-95f893ade6f2 h1:JhzVVoYvbOACxoUmOs6V/G4D5nPVUW73rKvXxP4XUJc= github.com/phayes/freeport v0.0.0-20180830031419-95f893ade6f2/go.mod h1:iIss55rKnNBTvrwdmkUpLnDpZoAHvWaiq5+iMmen4AE= @@ -510,20 +440,16 @@ github.com/pierrec/lz4 v2.4.1+incompatible h1:mFe7ttWaflA46Mhqh+jUfjp2qTbPYxLB2/ github.com/pierrec/lz4 v2.4.1+incompatible/go.mod h1:pdkljMzZIN41W+lC3N2tnIh5sFi+IEE17M5jbnwPHcY= github.com/pingcap-incubator/tidb-dashboard v0.0.0-20200407064406-b2b8ad403d01/go.mod h1:77fCh8d3oKzC5ceOJWeZXAS/mLzVgdZ7rKniwmOyFuo= github.com/pingcap-incubator/tidb-dashboard v0.0.0-20200514075710-eecc9a4525b5/go.mod h1:8q+yDx0STBPri8xS4A2duS1dAf+xO0cMtjwe0t6MWJk= -github.com/pingcap-incubator/tidb-dashboard v0.0.0-20200710045508-523e95bc5ec9/go.mod h1:9yaAM77sPfa5/f6sdxr3jSkKfIz463KRHyiFHiGjdes= -github.com/pingcap/badger v1.5.1-0.20200604041313-19c397305fcc h1:aL83XYtYRGJHGruiw5Kk/vJiJl2xFiWfqCgnvHin7ek= -github.com/pingcap/badger v1.5.1-0.20200604041313-19c397305fcc/go.mod h1:LyrqUOHZrUDf9oGi1yoz1+qw9ckSIhQb5eMa1acOLNQ= -github.com/pingcap/br v0.0.0-20200426093517-dd11ae28b885 h1:gI14HoGBF9UyECMgqSRZx1ONhREtrZe8JCmZ/6OFilw= +github.com/pingcap-incubator/tidb-dashboard v0.0.0-20200807020752-01f0abe88e93/go.mod h1:9yaAM77sPfa5/f6sdxr3jSkKfIz463KRHyiFHiGjdes= github.com/pingcap/br v0.0.0-20200426093517-dd11ae28b885/go.mod h1:4w3meMnk7HDNpNgjuRAxavruTeKJvUiXxoEWTjzXPnA= -github.com/pingcap/br v0.0.0-20200521085655-53201addd4ad h1:nptiQT0kWdIUghh49OyaTBYb4DtdxJmsLHOxbU25kW4= github.com/pingcap/br v0.0.0-20200521085655-53201addd4ad/go.mod h1:SlSUHWY7QUoooiYxOKuJ8kUh2KjI29ogBh89YXz2dLA= -github.com/pingcap/br v0.0.0-20200610051721-b057d65ff579/go.mod h1:Gq6o66nDReG0fMxqSdl8JHLZhMnxKhVxtHxqubUWAyg= -github.com/pingcap/br v0.0.0-20200617120402-56e151ad8b67/go.mod h1:/3QzpDG7YTPrDsrg8i1lwdYUrplJ0jVD+9pxhh19+k4= -github.com/pingcap/br v0.0.0-20200727092753-a475692725db h1:LoS2dSe764tdHEnspPGpeMVDTP0vJHCqUlZ6zxhnuO8= -github.com/pingcap/br v0.0.0-20200727092753-a475692725db/go.mod h1:4iTqZAMbEPmjBggYixqIg2FwIHBQtyImTM/QYlpTBGk= +github.com/pingcap/br v0.0.0-20200623060633-439a1c2b2bfd/go.mod h1:NGee2H9vXLunFIBXGb3uFsWRpw3BBo822sY4dyXepqo= +github.com/pingcap/br v0.0.0-20200716021245-f1df51c11469/go.mod h1:Ft2Vuvj6XJkbjQvflDOesJTy+9bui0saz0UonIgipAw= +github.com/pingcap/br v0.0.0-20200805095214-09dcc7534821/go.mod h1:JVsPK6Ibo2RBkTC2l1bzcRBVuHte/tJRERgay5gsBb8= +github.com/pingcap/br v0.0.0-20200821071758-97ac20a3f6b0 h1:QKBRsTJtdCXealFX9w/TOT6HxJz6mOyfrR7w5jfV7+A= +github.com/pingcap/br v0.0.0-20200821071758-97ac20a3f6b0/go.mod h1:w16YTn749niFWD2MC0DNGO9qp4OHTBC//D0Qb1ms2sk= 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/check v0.0.0-20200212061837-5e12011dc712 h1:R8gStypOBmpnHEx1qi//SaqxJVI4inOqljg/Aj5/390= github.com/pingcap/check v0.0.0-20200212061837-5e12011dc712/go.mod h1:PYMCGwN0JHjoqGr3HrZoD+b8Tgx8bKnArhSq8YVzUMc= @@ -531,99 +457,74 @@ github.com/pingcap/errcode v0.0.0-20180921232412-a1a7271709d9 h1:KH4f4Si9XK6/IW5 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/go.mod h1:Oi8TUi2kEtXXLMJk9l1cGmz20kV3TaQ0usTwv5KuLY8= -github.com/pingcap/errors v0.11.5-0.20190809092503-95897b64e011 h1:58naV4XMEqm0hl9LcYo6cZoGBGiLtefMQMF/vo3XLgQ= github.com/pingcap/errors v0.11.5-0.20190809092503-95897b64e011/go.mod h1:Oi8TUi2kEtXXLMJk9l1cGmz20kV3TaQ0usTwv5KuLY8= -github.com/pingcap/failpoint v0.0.0-20191029060244-12f4ac2fd11d h1:F8vp38kTAckN+v8Jlc98uMBvKIzr1a+UhnLyVYn8Q5Q= +github.com/pingcap/errors v0.11.5-0.20200729012136-4e113ddee29e h1:/EGWHNOyEgizEBuAujWsb9vXrPZtt1b7ooDPyjEkjDw= +github.com/pingcap/errors v0.11.5-0.20200729012136-4e113ddee29e/go.mod h1:g4vx//d6VakjJ0mk7iLBlKA8LFavV/sAVINT/1PFxeQ= github.com/pingcap/failpoint v0.0.0-20191029060244-12f4ac2fd11d/go.mod h1:DNS3Qg7bEDhU6EXNHF+XSv/PGznQaMJ5FWvctpm6pQI= -github.com/pingcap/failpoint v0.0.0-20200210140405-f8f9fb234798 h1:6DMbRqPI1qzQ8N1xc3+nKY8IxSACd9VqQKkRVvbyoIg= github.com/pingcap/failpoint v0.0.0-20200210140405-f8f9fb234798/go.mod h1:DNS3Qg7bEDhU6EXNHF+XSv/PGznQaMJ5FWvctpm6pQI= -github.com/pingcap/failpoint v0.0.0-20200506114213-c17f16071c53 h1:8sC8OLinmaw24xLeeJlYBFvUBsOiOYBtNqTuVOTnynQ= github.com/pingcap/failpoint v0.0.0-20200506114213-c17f16071c53/go.mod h1:w4PEZ5y16LeofeeGwdgZB4ddv9bLyDuIX+ljstgKZyk= -github.com/pingcap/failpoint v0.0.0-20200603062251-b230c36c413c h1:cm0zAj+Tab94mp4OH+VoLJiSNQvZO4pWDGJ8KEk2a0c= github.com/pingcap/failpoint v0.0.0-20200603062251-b230c36c413c/go.mod h1:w4PEZ5y16LeofeeGwdgZB4ddv9bLyDuIX+ljstgKZyk= +github.com/pingcap/failpoint v0.0.0-20200702092429-9f69995143ce h1:Y1kCxlCtlPTMtVcOkjUcuQKh+YrluSo7+7YMCQSzy30= +github.com/pingcap/failpoint v0.0.0-20200702092429-9f69995143ce/go.mod h1:w4PEZ5y16LeofeeGwdgZB4ddv9bLyDuIX+ljstgKZyk= github.com/pingcap/fn v0.0.0-20191016082858-07623b84a47d h1:rCmRK0lCRrHMUbS99BKFYhK9YxJDNw0xB033cQbYo0s= github.com/pingcap/fn v0.0.0-20191016082858-07623b84a47d/go.mod h1:fMRU1BA1y+r89AxUoaAar4JjrhUkVDt0o0Np6V8XbDQ= github.com/pingcap/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-20190227013052-e71ca0165a5f/go.mod h1:QMdbTAXCHzzygQzqcG9uVUgU2fKeSN1GmfMiykdSzzY= github.com/pingcap/kvproto v0.0.0-20191211054548-3c6b38ea5107/go.mod h1:WWLmULLO7l8IOcQG+t+ItJ3fEcrL5FxF0Wu+HrMy26w= github.com/pingcap/kvproto v0.0.0-20200214064158-62d31900d88e/go.mod h1:IOdRDPLyda8GX2hE/jO7gqaCV/PNFh8BZQCQZXfIOqI= github.com/pingcap/kvproto v0.0.0-20200411081810-b85805c9476c/go.mod h1:IOdRDPLyda8GX2hE/jO7gqaCV/PNFh8BZQCQZXfIOqI= github.com/pingcap/kvproto v0.0.0-20200417092353-efbe03bcffbd/go.mod h1:IOdRDPLyda8GX2hE/jO7gqaCV/PNFh8BZQCQZXfIOqI= -github.com/pingcap/kvproto v0.0.0-20200420075417-e0c6e8842f22 h1:D5EBGKd6o4A0PV0sUaUduPSCShiNi0OwFJmf+xRzpuI= github.com/pingcap/kvproto v0.0.0-20200420075417-e0c6e8842f22/go.mod h1:IOdRDPLyda8GX2hE/jO7gqaCV/PNFh8BZQCQZXfIOqI= github.com/pingcap/kvproto v0.0.0-20200423020121-038e31959c2a/go.mod h1:IOdRDPLyda8GX2hE/jO7gqaCV/PNFh8BZQCQZXfIOqI= github.com/pingcap/kvproto v0.0.0-20200424032552-6650270c39c3/go.mod h1:IOdRDPLyda8GX2hE/jO7gqaCV/PNFh8BZQCQZXfIOqI= -github.com/pingcap/kvproto v0.0.0-20200428135407-0f5ffe459677 h1:90pbLYmkk7bXLgyaYZj22QQLouVxqTZrswi+7DNMSRQ= -github.com/pingcap/kvproto v0.0.0-20200428135407-0f5ffe459677/go.mod h1:IOdRDPLyda8GX2hE/jO7gqaCV/PNFh8BZQCQZXfIOqI= -github.com/pingcap/kvproto v0.0.0-20200518112156-d4aeb467de29 h1:NpW1OuYrIl+IQrSsVbtyHpHpazmSCHy+ysrOixY0xY4= github.com/pingcap/kvproto v0.0.0-20200518112156-d4aeb467de29/go.mod h1:IOdRDPLyda8GX2hE/jO7gqaCV/PNFh8BZQCQZXfIOqI= -github.com/pingcap/kvproto v0.0.0-20200608081027-d02a6f65e956/go.mod h1:IOdRDPLyda8GX2hE/jO7gqaCV/PNFh8BZQCQZXfIOqI= -github.com/pingcap/kvproto v0.0.0-20200701055533-4ef28cac01f8/go.mod h1:IOdRDPLyda8GX2hE/jO7gqaCV/PNFh8BZQCQZXfIOqI= -github.com/pingcap/kvproto v0.0.0-20200706115936-1e0910aabe6c h1:VnLpCAxMAeDxc7HXTetwDQB+/MtDQjHAOBsd4QnGVwA= github.com/pingcap/kvproto v0.0.0-20200706115936-1e0910aabe6c/go.mod h1:IOdRDPLyda8GX2hE/jO7gqaCV/PNFh8BZQCQZXfIOqI= -github.com/pingcap/log v0.0.0-20191012051959-b742a5d432e9 h1:AJD9pZYm72vMgPcQDww9rkZ1DnWfl0pXV3BOWlkYIjA= +github.com/pingcap/kvproto v0.0.0-20200818080353-7aaed8998596 h1:1cRjX7+yHQiE4pV/xwB8XcbZXV9sHshWMNTd5I6SS2o= +github.com/pingcap/kvproto v0.0.0-20200818080353-7aaed8998596/go.mod h1:IOdRDPLyda8GX2hE/jO7gqaCV/PNFh8BZQCQZXfIOqI= github.com/pingcap/log v0.0.0-20191012051959-b742a5d432e9/go.mod h1:4rbK1p9ILyIfb6hU7OG2CiWSqMXnp3JMbiaVJ6mvoY8= -github.com/pingcap/log v0.0.0-20200117041106-d28c14d3b1cd h1:CV3VsP3Z02MVtdpTMfEgRJ4T9NGgGTxdHpJerent7rM= github.com/pingcap/log v0.0.0-20200117041106-d28c14d3b1cd/go.mod h1:4rbK1p9ILyIfb6hU7OG2CiWSqMXnp3JMbiaVJ6mvoY8= github.com/pingcap/log v0.0.0-20200511115504-543df19646ad h1:SveG82rmu/GFxYanffxsSF503SiQV+2JLnWEiGiF+Tc= github.com/pingcap/log v0.0.0-20200511115504-543df19646ad/go.mod h1:4rbK1p9ILyIfb6hU7OG2CiWSqMXnp3JMbiaVJ6mvoY8= github.com/pingcap/parser v0.0.0-20200424075042-8222d8b724a4/go.mod h1:9v0Edh8IbgjGYW2ArJr19E+bvL8zKahsFp+ixWeId+4= github.com/pingcap/parser v0.0.0-20200507022230-f3bf29096657/go.mod h1:9v0Edh8IbgjGYW2ArJr19E+bvL8zKahsFp+ixWeId+4= -github.com/pingcap/parser v0.0.0-20200518090819-ec1e13b948b1/go.mod h1:vQdbJqobJAgFyiRNNtXahpMoGWwPEuWciVEK5A20NS0= -github.com/pingcap/parser v0.0.0-20200522094936-3b720a0512a6 h1:sa3NjjIEU9JOY/P8bhn+SJupcKZ497b3sfPVfvyCm2A= -github.com/pingcap/parser v0.0.0-20200522094936-3b720a0512a6/go.mod h1:vQdbJqobJAgFyiRNNtXahpMoGWwPEuWciVEK5A20NS0= -github.com/pingcap/parser v0.0.0-20200609110328-c65941b9fbb3/go.mod h1:vQdbJqobJAgFyiRNNtXahpMoGWwPEuWciVEK5A20NS0= -github.com/pingcap/parser v0.0.0-20200623082809-b74301ac298b/go.mod h1:vQdbJqobJAgFyiRNNtXahpMoGWwPEuWciVEK5A20NS0= -github.com/pingcap/parser v0.0.0-20200730092557-34a468e9b774 h1:ch3VED+NQdsARfJUmNciIynhVbOot3be1pQcmftIrI4= -github.com/pingcap/parser v0.0.0-20200730092557-34a468e9b774/go.mod h1:vQdbJqobJAgFyiRNNtXahpMoGWwPEuWciVEK5A20NS0= -github.com/pingcap/parser v0.0.0-20200731033026-84f62115187c h1:UG7JwE9UbEpiTOJeUXHQBJFfMrU+yAtdpzEbLRiAnkI= -github.com/pingcap/parser v0.0.0-20200731033026-84f62115187c/go.mod h1:vQdbJqobJAgFyiRNNtXahpMoGWwPEuWciVEK5A20NS0= -github.com/pingcap/pd/v4 v4.0.0-rc.1.0.20200422143320-428acd53eba2 h1:JTzYYukREvxVSKW/ncrzNjFitd8snoQ/Xz32pw8i+s8= +github.com/pingcap/parser v0.0.0-20200603032439-c4ecb4508d2f/go.mod h1:9v0Edh8IbgjGYW2ArJr19E+bvL8zKahsFp+ixWeId+4= +github.com/pingcap/parser v0.0.0-20200623164729-3a18f1e5dceb/go.mod h1:vQdbJqobJAgFyiRNNtXahpMoGWwPEuWciVEK5A20NS0= +github.com/pingcap/parser v0.0.0-20200803072748-fdf66528323d h1:QQMAWm/b/8EyCrqqcjdO4DcACS06tx8IhKGWC4PTqiQ= +github.com/pingcap/parser v0.0.0-20200803072748-fdf66528323d/go.mod h1:vQdbJqobJAgFyiRNNtXahpMoGWwPEuWciVEK5A20NS0= github.com/pingcap/pd/v4 v4.0.0-rc.1.0.20200422143320-428acd53eba2/go.mod h1:s+utZtXDznOiL24VK0qGmtoHjjXNsscJx3m1n8cC56s= -github.com/pingcap/pd/v4 v4.0.0-rc.2.0.20200520083007-2c251bd8f181 h1:FM+PzdoR3fmWAJx3ug+p5aOgs5aZYwFkoDL7Potdsz0= github.com/pingcap/pd/v4 v4.0.0-rc.2.0.20200520083007-2c251bd8f181/go.mod h1:q4HTx/bA8aKBa4S7L+SQKHvjRPXCRV0tA0yRw0qkZSA= -github.com/pingcap/pd/v4 v4.0.0-rc.2.0.20200714122454-1a64f969cb3c h1:aOiyGetA256/LUkdmhny0Q/PWTBQiF/TPNhJuJMGRSY= -github.com/pingcap/pd/v4 v4.0.0-rc.2.0.20200714122454-1a64f969cb3c/go.mod h1:v/dY4mVkt3dh/Liphhk0E4ScOkZpIk0m0GvWJ9FapDs= -github.com/pingcap/sysutil v0.0.0-20200206130906-2bfa6dc40bcd h1:k7CIHMFVKjHsda3PKkiN4zv++NEnexlUwiJEhryWpG0= +github.com/pingcap/pd/v4 v4.0.5-0.20200817114353-e465cafe8a91 h1:zCOWP+kIzM6ZsXdu2QoM/W6+3vFZj04MYboMP2Obc0E= +github.com/pingcap/pd/v4 v4.0.5-0.20200817114353-e465cafe8a91/go.mod h1:m9OEkKoPMQWjrbJ9pqjjeCqzqxraZrPEuWa1OI6Wcek= github.com/pingcap/sysutil v0.0.0-20200206130906-2bfa6dc40bcd/go.mod h1:EB/852NMQ+aRKioCpToQ94Wl7fktV+FNnxf3CX/TTXI= -github.com/pingcap/sysutil v0.0.0-20200408114249-ed3bd6f7fdb1 h1:PI8YpTl45F8ilNkrPtT4IdbcZB1SCEa+gK/U5GJYl3E= github.com/pingcap/sysutil v0.0.0-20200408114249-ed3bd6f7fdb1/go.mod h1:EB/852NMQ+aRKioCpToQ94Wl7fktV+FNnxf3CX/TTXI= github.com/pingcap/sysutil v0.0.0-20200715082929-4c47bcac246a h1:i2RElJ2aykSqZKeY+3SK18NHhajil8cQdG77wHe+P1Y= github.com/pingcap/sysutil v0.0.0-20200715082929-4c47bcac246a/go.mod h1:EB/852NMQ+aRKioCpToQ94Wl7fktV+FNnxf3CX/TTXI= github.com/pingcap/tidb v1.1.0-beta.0.20200424154252-5ede18f10eed/go.mod h1:m2VDlJDbUeHPCXAfKPajqLmB1uLvWpkKk3zALNqDYdw= github.com/pingcap/tidb v1.1.0-beta.0.20200509133407-a9dc72cf2558/go.mod h1:cXNbVSQAkwwmjFQmEnEPI00Z2/Y/KOhouttUPERiInE= -github.com/pingcap/tidb v1.1.0-beta.0.20200513065557-5a0787dfa915/go.mod h1:khS9Z9YlbtxsaZsSsSahelgh5L16EtP30QADFmPiI/I= -github.com/pingcap/tidb v1.1.0-beta.0.20200521154755-134e691d6f5f/go.mod h1:HobvT8ySGq9gkwPoyz/+V6TKWMRGEzFQobJOvLc2Oy8= -github.com/pingcap/tidb v1.1.0-beta.0.20200603101356-552e7709de0d h1:U4Bm4KdtCfOIH1L5QbWFeck32gjZhLb7dCoTkEc1vPY= -github.com/pingcap/tidb v1.1.0-beta.0.20200603101356-552e7709de0d/go.mod h1:wgu4vP3wq+x/l1X3ckOZFvyGKcVIBkq30NQVh0Y+qYA= -github.com/pingcap/tidb v1.1.0-beta.0.20200604055950-efc1c154d098/go.mod h1:UMxsNE326wyfgFJCx6aerPRLj1/tGPYDBKS9T9NOHI8= -github.com/pingcap/tidb v1.1.0-beta.0.20200610060912-f12cdc42010f/go.mod h1:jyXOvS9k0PTxYHju2OgySOe9FtydA52TiQ5bXAaKyQE= -github.com/pingcap/tidb v1.1.0-beta.0.20200721005019-f5c6e59f0daf/go.mod h1:dYCOFXJsoqBumpxAyBqCG3WZriIY7JgeBZHgvfARDa8= -github.com/pingcap/tidb v1.1.0-beta.0.20200731004449-a63fa79d90c5 h1:9I/TzJreppQ/QB/XsgDRidpGVj3Q+FYRYPI/PjXW7MM= -github.com/pingcap/tidb v1.1.0-beta.0.20200731004449-a63fa79d90c5/go.mod h1:tGL4bibikUVvAbzS9rdcqBg2AIVRttZYmGCl0+wAUQk= +github.com/pingcap/tidb v1.1.0-beta.0.20200606093724-b5b4da0e6a90/go.mod h1:aaBBi3OJmYjENWY31YYOY8K6UoZZYgjZVZH56D0QIdE= +github.com/pingcap/tidb v1.1.0-beta.0.20200715100003-b4da443a3c4c/go.mod h1:TplKBs1sevRvK11aT7ro0ntTCalyh1fMaWACp03dQf4= +github.com/pingcap/tidb v1.1.0-beta.0.20200716023258-b10faca6ff89/go.mod h1:hDlQ5BJ4rLLCOUlvXqW3skyYEjyymzeTA3eXpNEDx38= +github.com/pingcap/tidb v1.1.0-beta.0.20200820092836-c5b7658b0896 h1:l2UJF9cFxwaMMNMjguqrfiC7sFZrEqbtEmAAWFyHx9w= +github.com/pingcap/tidb v1.1.0-beta.0.20200820092836-c5b7658b0896/go.mod h1:IAStISSVhEI9Gp/sE4w6Ms0WxpdBJ9qNTczNyskvd5A= github.com/pingcap/tidb-tools v4.0.0-beta.1.0.20200306084441-875bd09aa3d5+incompatible/go.mod h1:XGdcy9+yqlDSEMTpOXnwf3hiTeqrV6MN/u1se9N8yIM= github.com/pingcap/tidb-tools v4.0.0-rc.1.0.20200421113014-507d2bb3a15e+incompatible/go.mod h1:XGdcy9+yqlDSEMTpOXnwf3hiTeqrV6MN/u1se9N8yIM= github.com/pingcap/tidb-tools v4.0.0-rc.1.0.20200514040632-f76b3e428e19+incompatible/go.mod h1:XGdcy9+yqlDSEMTpOXnwf3hiTeqrV6MN/u1se9N8yIM= -github.com/pingcap/tidb-tools v4.0.0+incompatible/go.mod h1:XGdcy9+yqlDSEMTpOXnwf3hiTeqrV6MN/u1se9N8yIM= +github.com/pingcap/tidb-tools v4.0.0-rc.2.0.20200521050818-6dd445d83fe0+incompatible/go.mod h1:XGdcy9+yqlDSEMTpOXnwf3hiTeqrV6MN/u1se9N8yIM= +github.com/pingcap/tidb-tools v4.0.1-0.20200530144555-cdec43635625+incompatible/go.mod h1:XGdcy9+yqlDSEMTpOXnwf3hiTeqrV6MN/u1se9N8yIM= github.com/pingcap/tidb-tools v4.0.1+incompatible/go.mod h1:XGdcy9+yqlDSEMTpOXnwf3hiTeqrV6MN/u1se9N8yIM= -github.com/pingcap/tidb-tools v4.0.5-0.20200731060919-6a36d482e3f9+incompatible h1:daZx8wC4HBhCc3WZ35l3qAUs4B/OMiIkHNltS5mCQVU= -github.com/pingcap/tidb-tools v4.0.5-0.20200731060919-6a36d482e3f9+incompatible/go.mod h1:XGdcy9+yqlDSEMTpOXnwf3hiTeqrV6MN/u1se9N8yIM= +github.com/pingcap/tidb-tools v4.0.5-0.20200817064459-ba61a7376547+incompatible h1:KGkKSxJ4t5F3Ys0ox144M5vGWm8NuHkXFyETFtakKW4= +github.com/pingcap/tidb-tools v4.0.5-0.20200817064459-ba61a7376547+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-20200417094153-7316d94df1ee h1:XJQ6/LGzOSc/jo33AD8t7jtc4GohxcyODsYnb+kZXJM= github.com/pingcap/tipb v0.0.0-20200417094153-7316d94df1ee/go.mod h1:RtkHW8WbcNxj8lsbzjaILci01CtYnYbIkQhjyZWrWVI= -github.com/pingcap/tipb v0.0.0-20200604070248-508f03b0b342/go.mod h1:RtkHW8WbcNxj8lsbzjaILci01CtYnYbIkQhjyZWrWVI= -github.com/pingcap/tipb v0.0.0-20200615034523-dcfcea0b5965/go.mod h1:RtkHW8WbcNxj8lsbzjaILci01CtYnYbIkQhjyZWrWVI= +github.com/pingcap/tipb v0.0.0-20200522051215-f31a15d98fce/go.mod h1:RtkHW8WbcNxj8lsbzjaILci01CtYnYbIkQhjyZWrWVI= github.com/pingcap/tipb v0.0.0-20200618092958-4fad48b4c8c3 h1:ESL3eIt1kUt8IMvR1011ejZlAyDcOzw89ARvVHvpD5k= github.com/pingcap/tipb v0.0.0-20200618092958-4fad48b4c8c3/go.mod h1:RtkHW8WbcNxj8lsbzjaILci01CtYnYbIkQhjyZWrWVI= github.com/pkg/errors v0.8.0/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0= -github.com/pkg/errors v0.8.1 h1:iURUrRGxPUNPdy5/HRSm+Yj6okJ6UtLINN0Q9M4+h3I= github.com/pkg/errors v0.8.1/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0= github.com/pkg/errors v0.9.1 h1:FEBLx1zS214owpjy7qsBeixbURkuhQAwrK5UwLGTwt4= github.com/pkg/errors v0.9.1/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0= 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.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 v0.9.3/go.mod h1:/TN21ttK/J9q6uSwhBd54HahCDft0ttaMvbicHlPoso= github.com/prometheus/client_golang v1.0.0/go.mod h1:db9x61etRT2tGnBNRi70OPL5FsnadC4Ky3P0J6CfImo= @@ -634,7 +535,6 @@ github.com/prometheus/client_model v0.0.0-20190129233127-fd36f4220a90/go.mod h1: github.com/prometheus/client_model v0.0.0-20190812154241-14fe0d1b01d4/go.mod h1:xMI15A0UPsDsEKsMN9yxemIoYk6Tm2C1GtYGdfGttqA= github.com/prometheus/client_model v0.2.0 h1:uq5h0d+GuxiXLJLNABMgp2qUWDPiLvgCzz2dUR+/W/M= github.com/prometheus/client_model v0.2.0/go.mod h1:xMI15A0UPsDsEKsMN9yxemIoYk6Tm2C1GtYGdfGttqA= -github.com/prometheus/common v0.0.0-20181020173914-7e9e6cabbd39/go.mod h1:daVV7qP5qjZbuso7PdcryaAu0sAZbrN9i7WWcTMWvro= github.com/prometheus/common v0.0.0-20181113130724-41aa239b4cce/go.mod h1:daVV7qP5qjZbuso7PdcryaAu0sAZbrN9i7WWcTMWvro= github.com/prometheus/common v0.4.0/go.mod h1:TNfzLD0ON7rHzMJeJkieUDPYmFC7Snx/y86RQel1bk4= github.com/prometheus/common v0.4.1/go.mod h1:TNfzLD0ON7rHzMJeJkieUDPYmFC7Snx/y86RQel1bk4= @@ -654,17 +554,12 @@ github.com/remyoudompheng/bigfft v0.0.0-20190728182440-6a916e37a237 h1:HQagqIiBm 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 h1:RR9dF3JtopPvtkroDZuVD7qquD0bnHlKSqaQhgwt8yk= github.com/rogpeppe/go-internal v1.3.0/go.mod h1:M8bDsm7K2OlrFYOpmOWEs/qY81heoFRclV5y23lUDJ4= github.com/rs/cors v1.7.0/go.mod h1:gFx+x8UowdsKA9AchylcLynDq+nNFfI8FkUZdN/jGCU= -github.com/russross/blackfriday v1.5.2 h1:HyvC0ARfnZBqnXwABFeSZHpKvJHJJfPz81GNueLj0oo= github.com/russross/blackfriday v1.5.2/go.mod h1:JO/DiYxRf+HjHt06OyowR9PTA263kcR/rfWxYHBV53g= -github.com/russross/blackfriday/v2 v2.0.1 h1:lPqVAte+HuHNfhJ/0LC98ESWRz8afy9tM/0RK8m9o+Q= github.com/russross/blackfriday/v2 v2.0.1/go.mod h1:+Rmxgy9KzJVeS9/2gXHxylqXiyQDYRxCVz55jmeOWTM= -github.com/sasha-s/go-deadlock v0.2.0 h1:lMqc+fUb7RrFS3gQLtoQsJ7/6TV/pAIFvBsqX73DK8Y= github.com/sasha-s/go-deadlock v0.2.0/go.mod h1:StQn567HiB1fF2yJ44N9au7wOhrPS3iZqiDbRupzT10= github.com/satori/go.uuid v1.2.0/go.mod h1:dA0hQrYB0VpLJoorglMZABFdXlWrHn1NEOzdhQKdks0= -github.com/sergi/go-diff v1.0.1-0.20180205163309-da645544ed44 h1:tB9NOR21++IjLyVx3/PCPhWMwqGNCMQEH96A6dMZ/gc= github.com/sergi/go-diff v1.0.1-0.20180205163309-da645544ed44/go.mod h1:0CfEIISq7TuYL3j771MWULgwwjU+GofnZX9QAmXWZgo= github.com/shirou/gopsutil v2.19.10+incompatible h1:lA4Pi29JEVIQIgATSeftHSY0rMGI9CLrl2ZvDLiahto= github.com/shirou/gopsutil v2.19.10+incompatible/go.mod h1:5b4v6he4MtMOwMlS0TUMTu2PcXUg8+E1lC7eC3UO/RA= @@ -673,40 +568,31 @@ github.com/shirou/w32 v0.0.0-20160930032740-bb4de0191aa4/go.mod h1:qsXQc7+bwAM3Q github.com/shurcooL/httpfs v0.0.0-20190707220628-8d4bc4ba7749 h1:bUGsEnyNbVPw06Bs80sCeARAlK8lhwqGyi6UT8ymuGk= github.com/shurcooL/httpfs v0.0.0-20190707220628-8d4bc4ba7749/go.mod h1:ZY1cvUeJuFPAdZ/B6v7RHavJWZn2YPVFQ1OSXhCGOkg= github.com/shurcooL/httpgzip v0.0.0-20190720172056-320755c1c1b0/go.mod h1:919LwcH0M7/W4fcZ0/jy0qGght1GIhqyS/EgWGH2j5Q= -github.com/shurcooL/sanitized_anchor_name v1.0.0 h1:PdmoCO6wvbs+7yrJyMORt4/BmY5IYyJwS/kOiWx8mHo= github.com/shurcooL/sanitized_anchor_name v1.0.0/go.mod h1:1NzhyTcUVG4SuEtjjoZeVRXNmyL/1OwPU0+IJeTBvfc= -github.com/shurcooL/vfsgen v0.0.0-20181020040650-a97a25d856ca h1:3fECS8atRjByijiI8yYiuwLwQ2ZxXobW7ua/8GRB3pI= github.com/shurcooL/vfsgen v0.0.0-20181020040650-a97a25d856ca/go.mod h1:TrYk7fJVaAttu97ZZKrO9UbRa8izdowaMIZcxYMbVaw= github.com/shurcooL/vfsgen v0.0.0-20181202132449-6a9ea43bcacd h1:ug7PpSOB5RBPK1Kg6qskGBoP3Vnj/aNYFTznWvlkGo0= github.com/shurcooL/vfsgen v0.0.0-20181202132449-6a9ea43bcacd/go.mod h1:TrYk7fJVaAttu97ZZKrO9UbRa8izdowaMIZcxYMbVaw= github.com/sirupsen/logrus v1.2.0/go.mod h1:LxeOpSwHxABJmUn/MG1IvRgCAasNZTLOkJPxbbu5VWo= github.com/sirupsen/logrus v1.4.1/go.mod h1:ni0Sbl8bgC9z8RoU9G6nDWqqs/fq4eDPysMBDgk/93Q= -github.com/sirupsen/logrus v1.4.2 h1:SPIRibHv4MatM3XXNO2BJeFLZwZ2LvZgfQ5+UNI2im4= github.com/sirupsen/logrus v1.4.2/go.mod h1:tLMulIdttU9McNUspp0xgXVQah82FyeX6MwdIuYE2rE= github.com/sirupsen/logrus v1.6.0 h1:UBcNElsrwanuuMsnGSlYmtmgbb23qDR5dG+6X6Oo89I= github.com/sirupsen/logrus v1.6.0/go.mod h1:7uNnSEd1DgxDLC74fIahvMZmmYsHGZGEOFrfsX/uA88= github.com/soheilhy/cmux v0.1.4 h1:0HKaf1o97UwFjHH9o5XsHUOF+tqmdA7KEzXLpiyaw0E= github.com/soheilhy/cmux v0.1.4/go.mod h1:IM3LyeVVIOuxMH7sFAkER9+bJ4dT7Ms6E4xg4kGIyLM= -github.com/spaolacci/murmur3 v0.0.0-20180118202830-f09979ecbc72 h1:qLC7fQah7D6K1B0ujays3HV9gkFtllcxhzImRR7ArPQ= github.com/spaolacci/murmur3 v0.0.0-20180118202830-f09979ecbc72/go.mod h1:JwIasOWyU6f++ZhiEuf87xNszmSA2myDM2Kzu9HwQUA= github.com/spaolacci/murmur3 v1.1.0 h1:7c1g84S4BPRrfL5Xrdp6fOJ206sU9y293DDHaoy0bLI= github.com/spaolacci/murmur3 v1.1.0/go.mod h1:JwIasOWyU6f++ZhiEuf87xNszmSA2myDM2Kzu9HwQUA= -github.com/spf13/afero v1.1.2 h1:m8/z1t7/fwjysjQRYbP0RD+bUIF/8tJwPdEZsI83ACI= github.com/spf13/afero v1.1.2/go.mod h1:j4pytiNVoe2o6bmDsKpLACNPDBIoEAkihy7loJ1B0CQ= -github.com/spf13/cast v1.3.0 h1:oget//CVOEoFewqQxwr0Ej5yjygnqGkvggSE/gB35Q8= 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/cobra v0.0.5 h1:f0B+LkLX6DtmRH1isoNA9VTtNUK9K8xYd28JNNfOv/s= github.com/spf13/cobra v0.0.5/go.mod h1:3K3wKZymM7VvHMDS9+Akkh4K60UwM26emMESw8tLCHU= github.com/spf13/cobra v1.0.0 h1:6m/oheQuQ13N9ks4hubMG6BnvwOeaJrqSPLahSnczz8= github.com/spf13/cobra v1.0.0/go.mod h1:/6GTrnGXV9HjY+aR4k0oJ5tcvakLuG6EuKReYlHNrgE= -github.com/spf13/jwalterweatherman v1.0.0 h1:XHEdyB+EcvlqZamSM4ZOMGlc93t6AcsBEu9Gc1vn7yk= 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 h1:VUFqw5KcqRf7i70GOzW7N+Q7+gxVBkSSqiXB12+JQ4M= github.com/spf13/viper v1.3.2/go.mod h1:ZiWeW+zYFKm7srdB9IoDzzZXaJaI5eL9QjNiN/DMA2s= github.com/spf13/viper v1.4.0/go.mod h1:PTJ7Z/lr49W6bUbkmS1V3by4uWynFiR9p7+dSq/yZzE= github.com/stretchr/objx v0.1.0/go.mod h1:HFkY916IF+rwdDfMAkV7OtwuqBVzrE8GR6GFx+wExME= @@ -715,7 +601,6 @@ github.com/stretchr/objx v0.2.0 h1:Hbg2NidpLE8veEBkEZTL3CvlkUIVzuU9jDplZO54c48= github.com/stretchr/objx v0.2.0/go.mod h1:qt09Ya8vawLte6SNmTgCsAVtYtaKzEcn8ATUoHMkEqE= github.com/stretchr/testify v1.2.2/go.mod h1:a8OnRcib4nhh0OaRAV+Yts87kKdq0PP7pXfy6kDkUVs= github.com/stretchr/testify v1.3.0/go.mod h1:M5WIy9Dh21IEIfnGCwXGc5bZfKNJtfHm1UVUgZn+9EI= -github.com/stretchr/testify v1.4.0 h1:2E4SXV/wtOkTonXsotYi4li6zVWxYlZuYNCXe9XRJyk= github.com/stretchr/testify v1.4.0/go.mod h1:j7eGeouHqKxXV5pUuKE4zz7dFj8WfuZ+81PSLYec5m4= github.com/stretchr/testify v1.5.1 h1:nOGnQDM7FYENwehXlg/kFVnos3rEvtKTjRvOWSzb6H4= github.com/stretchr/testify v1.5.1/go.mod h1:5W2xD1RspED5o8YsWQXVCued0rvSQ+mT+I5cxcmMvtA= @@ -726,10 +611,8 @@ github.com/swaggo/http-swagger v0.0.0-20200308142732-58ac5e232fba/go.mod h1:O1lA github.com/swaggo/swag v1.5.1/go.mod h1:1Bl9F/ZBpVWh22nY0zmYyASPO1lI/zIwRDrpZU+tv8Y= github.com/swaggo/swag v1.6.3/go.mod h1:wcc83tB4Mb2aNiL/HP4MFeQdpHUrca+Rp/DRNgWAUio= github.com/swaggo/swag v1.6.5/go.mod h1:Y7ZLSS0d0DdxhWGVhQdu+Bu1QhaF5k0RD7FKdiAykeY= -github.com/swaggo/swag v1.6.6-0.20200323071853-8e21f4cefeea h1:e4navjNdMYbOJeqjea2DcUC00Djp1ewI7sJqmp1xd+I= github.com/swaggo/swag v1.6.6-0.20200323071853-8e21f4cefeea/go.mod h1:xDhTyuFIujYiN3DKWC/H/83xcfHp+UE/IzWWampG7Zc= github.com/swaggo/swag v1.6.6-0.20200529100950-7c765ddd0476/go.mod h1:xDhTyuFIujYiN3DKWC/H/83xcfHp+UE/IzWWampG7Zc= -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/syndtr/goleveldb v1.0.1-0.20190625010220-02440ea7a285 h1:uSDYjYejelKyceA6DiCsngFof9jAyeaSyX9XC5a1a7Q= github.com/syndtr/goleveldb v1.0.1-0.20190625010220-02440ea7a285/go.mod h1:9OrXJhf154huy1nPWmuSrkgjPUtUNhA+Zmy+6AESzuA= @@ -742,15 +625,11 @@ github.com/tmc/grpc-websocket-proxy v0.0.0-20170815181823-89b8d40f7ca8/go.mod h1 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/twmb/murmur3 v1.1.3 h1:D83U0XYKcHRYwYIpBKf3Pks91Z0Byda/9SJ8B6EMRcA= -github.com/twmb/murmur3 v1.1.3/go.mod h1:Qq/R7NUyOfr65zD+6Q5IHKsJLwP7exErjN6lyyq3OSQ= github.com/uber-go/atomic v1.3.2 h1:Azu9lPBWRNKzYXSIwRfgRuDuS0YKsK4NFhiQv98gkxo= github.com/uber-go/atomic v1.3.2/go.mod h1:/Ct5t2lcmbJ4OSe/waGBoaVvVqtO0bmtfVNex1PFV8g= -github.com/uber/jaeger-client-go v2.15.0+incompatible h1:NP3qsSqNxh8VYr956ur1N/1C1PjvOJnJykCzcD5QHbk= github.com/uber/jaeger-client-go v2.15.0+incompatible/go.mod h1:WVhlPFC8FDjOFMMWRy2pZqQJSXxYSwNYOkTr/Z6d3Kk= github.com/uber/jaeger-client-go v2.22.1+incompatible h1:NHcubEkVbahf9t3p75TOCR83gdUHXjRJvjoBh1yACsM= github.com/uber/jaeger-client-go v2.22.1+incompatible/go.mod h1:WVhlPFC8FDjOFMMWRy2pZqQJSXxYSwNYOkTr/Z6d3Kk= -github.com/uber/jaeger-lib v1.5.0 h1:OHbgr8l656Ub3Fw5k9SWnBfIEwvoHQ+W2y+Aa9D1Uyo= github.com/uber/jaeger-lib v1.5.0/go.mod h1:ComeNDZlWwrWnDv8aPp0Ba6+uUTzImX/AauajbLI56U= github.com/uber/jaeger-lib v2.2.0+incompatible h1:MxZXOiR2JuoANZ3J6DE/U0kSFv/eJ/GfSYVCjK7dyaw= github.com/uber/jaeger-lib v2.2.0+incompatible/go.mod h1:ComeNDZlWwrWnDv8aPp0Ba6+uUTzImX/AauajbLI56U= @@ -761,14 +640,10 @@ github.com/ugorji/go/codec v0.0.0-20181022190402-e5e69e061d4f/go.mod h1:VFNgLljT github.com/ugorji/go/codec v0.0.0-20181204163529-d75b2dcb6bc8/go.mod h1:VFNgLljTbGfSG7qAOspJ7OScBnGdDN/yBr0sguwnwf0= github.com/ugorji/go/codec v1.1.5-pre/go.mod h1:tULtS6Gy1AE1yCENaw4Vb//HLH5njI2tfCQDUqRd8fI= github.com/ugorji/go/codec v1.1.7/go.mod h1:Ax+UKWsSmolVDwsd+7N3ZtXu+yMGCf907BLYF3GoBXY= -github.com/unrolled/render v0.0.0-20171102162132-65450fb6b2d3 h1:ZsIlNwu/G0zbChIZaWOeZ2TPGNmKMt46jZLXi3e8LFc= github.com/unrolled/render v0.0.0-20171102162132-65450fb6b2d3/go.mod h1:tu82oB5W2ykJRVioYsB+IQKcft7ryBr7w12qMBUPyXg= github.com/urfave/cli v1.20.0/go.mod h1:70zkFmudgCuE/ngEzBv17Jvp/497gISqfk5gWijbERA= -github.com/urfave/cli v1.22.2 h1:gsqYFH8bb9ekPA12kRo0hfjngWQjkJPlN9R0N78BoUo= github.com/urfave/cli v1.22.2/go.mod h1:Gos4lmkARVdJ6EkW0WaNv/tZAAMe9V7XWyB60NtXRu0= -github.com/urfave/cli/v2 v2.1.1 h1:Qt8FeAtxE/vfdrLmR3rxR6JRE0RoVmbXu8+6kZtYU4k= github.com/urfave/cli/v2 v2.1.1/go.mod h1:SE9GqnLQmjVa0iPEY0f1w3ygNIYcIJ0OKPMoW2caLfQ= -github.com/urfave/negroni v0.3.0 h1:PaXOb61mWeZJxc1Ji2xJjpVg9QfPo0rrB+lHyBxGNSU= github.com/urfave/negroni v0.3.0/go.mod h1:Meg73S6kFm/4PpbYdq35yYWoCZ9mS/YSx+lKnmiohz4= github.com/valyala/gozstd v1.7.0 h1:Ljh5c9zboqLhwTI33al32R72iCZfn0mCbVGcFWbGwRQ= github.com/valyala/gozstd v1.7.0/go.mod h1:y5Ew47GLlP37EkTB+B4s7r6A5rdaeB7ftbl9zoYiIPQ= @@ -784,16 +659,12 @@ github.com/xiang90/probing v0.0.0-20190116061207-43a291ad63a2/go.mod h1:UETIi67q github.com/xordataexchange/crypt v0.0.3-0.20170626215501-b2862e3d0a77/go.mod h1:aYKd//L2LvnjZzWKhF00oedf4jCCReLcmhLdhm1A27Q= github.com/yahoo/athenz v1.8.55 h1:xGhxN3yLq334APyn0Zvcc+aqu78Q7BBhYJevM3EtTW0= github.com/yahoo/athenz v1.8.55/go.mod h1:G7LLFUH7Z/r4QAB7FfudfuA7Am/eCzO1GlzBhDL6Kv0= -github.com/yookoala/realpath v1.0.0 h1:7OA9pj4FZd+oZDsyvXWQvjn5oBdcHRTV44PpdMSuImQ= github.com/yookoala/realpath v1.0.0/go.mod h1:gJJMA9wuX7AcqLy1+ffPatSCySA1FQ2S8Ya9AIoYBpE= github.com/yuin/goldmark v1.1.25/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= github.com/yuin/goldmark v1.1.27/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= -github.com/zhangjinpeng1987/raft v0.0.0-20190624145930-deeb32d6553d h1:rGkexfPDxNuTCObUwTbsRUlti+evR/Ksb4dKy6esXW0= -github.com/zhangjinpeng1987/raft v0.0.0-20190624145930-deeb32d6553d/go.mod h1:1KDQ09J8MRHEtHze4at7BJZDW/doUAgkJ8w9KjEUhSo= +github.com/yuin/goldmark v1.1.32/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= go.etcd.io/bbolt v1.3.2/go.mod h1:IbVyRI1SCnLcuJnV2u8VeU0CEYM7e686BmAb1XKL+uU= -go.etcd.io/bbolt v1.3.3 h1:MUGmc65QhB3pIlaQ5bB4LwqSj6GIonVJXpZiaKNyaKk= go.etcd.io/bbolt v1.3.3/go.mod h1:IbVyRI1SCnLcuJnV2u8VeU0CEYM7e686BmAb1XKL+uU= -go.etcd.io/bbolt v1.3.4 h1:hi1bXHMVrlQh6WwxAy+qZCV/SYIlqo+Ushwdpa4tAKg= go.etcd.io/bbolt v1.3.4/go.mod h1:G5EMThwa9y8QZGBClrRx5EY+Yw9kAhnjy3bSjsnlVTQ= go.etcd.io/bbolt v1.3.5 h1:XAzx9gjCb0Rxj7EoqcClPD1d5ZBxZJk0jbuoPHenBt0= go.etcd.io/bbolt v1.3.5/go.mod h1:G5EMThwa9y8QZGBClrRx5EY+Yw9kAhnjy3bSjsnlVTQ= @@ -807,7 +678,6 @@ go.opencensus.io v0.22.2 h1:75k/FF0Q2YM8QYo07VPddOLBslDt1MZOdEslOHvmzAs= go.opencensus.io v0.22.2/go.mod h1:yxeiOL68Rb0Xd1ddK5vPZ/oVn4vY4Ynel7k9FzqtOIw= go.uber.org/atomic v1.3.2/go.mod h1:gD2HeocX3+yG+ygLZcrzQJaqmWj9AIm7n08wl/qW/PE= go.uber.org/atomic v1.4.0/go.mod h1:gD2HeocX3+yG+ygLZcrzQJaqmWj9AIm7n08wl/qW/PE= -go.uber.org/atomic v1.5.0 h1:OI5t8sDa1Or+q8AeE+yKeB/SDYioSHAgcVljj9JIETY= go.uber.org/atomic v1.5.0/go.mod h1:sABNBOSYdrvTF6hTgEIbc7YasKWGhgEQZyfxyTvoXHQ= go.uber.org/atomic v1.6.0 h1:Ezj3JGmsOnG1MoRWQkPBsKLe9DwWD9QeXzTRzzldNVk= go.uber.org/atomic v1.6.0/go.mod h1:sABNBOSYdrvTF6hTgEIbc7YasKWGhgEQZyfxyTvoXHQ= @@ -818,7 +688,6 @@ 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.3.0/go.mod h1:VgVr7evmIr6uPjLBxg28wmKNXyqE9akIJ5XnfpiKl+4= -go.uber.org/multierr v1.4.0 h1:f3WCSC2KzAcBXGATIxAB1E2XuCpNU255wNKZ505qi3E= go.uber.org/multierr v1.4.0/go.mod h1:VgVr7evmIr6uPjLBxg28wmKNXyqE9akIJ5XnfpiKl+4= go.uber.org/multierr v1.5.0 h1:KCa4XfM8CWFCpxXRGok+Q0SS/0XBhMDbHHGABQLvD2A= go.uber.org/multierr v1.5.0/go.mod h1:FeouvMocqHpRaaGuG9EjoKcStLC43Zu/fmqdUMPcKYU= @@ -828,13 +697,10 @@ go.uber.org/zap v1.8.0/go.mod h1:vwi/ZaCAaUcBkycHslxD9B2zi4UTXhF60s6SWpuDF0Q= go.uber.org/zap v1.9.1/go.mod h1:vwi/ZaCAaUcBkycHslxD9B2zi4UTXhF60s6SWpuDF0Q= 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= -go.uber.org/zap v1.14.1 h1:nYDKopTbvAPq/NrUVZwT15y2lpROBiLLyoRTbXOYWOo= go.uber.org/zap v1.14.1/go.mod h1:Mb2vm2krFEG5DV0W9qcHBYFtp/Wku1cvYaqPsS/WYfc= go.uber.org/zap v1.15.0 h1:ZZCA22JRF2gQE5FoNmhmrf7jeJJ2uhqDUNRYKm8dvmM= go.uber.org/zap v1.15.0/go.mod h1:Mb2vm2krFEG5DV0W9qcHBYFtp/Wku1cvYaqPsS/WYfc= -golang.org/x/crypto v0.0.0-20180214000028-650f4a345ab4/go.mod h1:6SG95UA2DQfeDnfUPMdvaQW0Q7yPrPDi9nlGo2tz2b4= golang.org/x/crypto v0.0.0-20180904163835-0709b304e793/go.mod h1:6SG95UA2DQfeDnfUPMdvaQW0Q7yPrPDi9nlGo2tz2b4= golang.org/x/crypto v0.0.0-20181203042331-505ab145d0a9/go.mod h1:6SG95UA2DQfeDnfUPMdvaQW0Q7yPrPDi9nlGo2tz2b4= golang.org/x/crypto v0.0.0-20190308221718-c2843e01d9a2/go.mod h1:djNgcEr1/C05ACkg1iLfiJU5Ep61QUkGW8qpdssI0+w= @@ -847,15 +713,13 @@ golang.org/x/crypto v0.0.0-20191002192127-34f69633bfdc/go.mod h1:yigFU9vqHzYiE8U golang.org/x/crypto v0.0.0-20191011191535-87dc89f01550/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= golang.org/x/crypto v0.0.0-20191205180655-e7c4368fe9dd/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto= golang.org/x/crypto v0.0.0-20191206172530-e9b2fee46413/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto= -golang.org/x/crypto v0.0.0-20200204104054-c9f3fb736b72 h1:+ELyKg6m8UBf0nPFSqD0mi7zUfwPyXo23HNjMnXPz7w= golang.org/x/crypto v0.0.0-20200204104054-c9f3fb736b72/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto= -golang.org/x/crypto v0.0.0-20200302210943-78000ba7a073 h1:xMPOj6Pz6UipU1wXLkrtqpHbR0AVFnyPEQq/wRWz9lM= -golang.org/x/crypto v0.0.0-20200302210943-78000ba7a073/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto= +golang.org/x/crypto v0.0.0-20200622213623-75b288015ac9 h1:psW17arqaxU48Z5kZ0CQnkZWQJsqcURM6tKiBApRjXI= +golang.org/x/crypto v0.0.0-20200622213623-75b288015ac9/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto= golang.org/x/exp v0.0.0-20190121172915-509febef88a4/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= golang.org/x/exp v0.0.0-20190306152737-a1d7652674e8/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= golang.org/x/exp v0.0.0-20190510132918-efd6b22b2522/go.mod h1:ZjyILWgesfNpC6sMxTJOJm9Kp84zZh5NQWvqDGG3Qr8= golang.org/x/exp v0.0.0-20190829153037-c13cbed26979/go.mod h1:86+5VVa7VpoJ4kLfm080zCjGlMRFzhUhsZKEZO7MGek= -golang.org/x/exp v0.0.0-20191030013958-a1ab85dbe136 h1:A1gGSx58LAGVHUUsOf7IiR0u8Xb6W51gRwfDBhkdcaw= golang.org/x/exp v0.0.0-20191030013958-a1ab85dbe136/go.mod h1:JXzH8nQsPlswgeRAPE3MuO9GYsAcnJvJ4vnMwN/5qkY= golang.org/x/exp v0.0.0-20191129062945-2f5052295587/go.mod h1:2RIsYlXP63K8oxa1u096TMicItID8zy7Y6sNkU49FU4= golang.org/x/exp v0.0.0-20191227195350-da58074b4299 h1:zQpM52jfKHG6II1ISZY1ZcpygvuSFZpLwfluuF89XOg= @@ -877,11 +741,10 @@ golang.org/x/mobile v0.0.0-20190312151609-d3739f865fa6/go.mod h1:z+o9i4GpDbdi3rU golang.org/x/mobile v0.0.0-20190719004257-d2bd2a29d028/go.mod h1:E/iHnbuqvinMTCcRqshq8CkpyQDoeVncDDYHnLhea+o= golang.org/x/mod v0.0.0-20190513183733-4bf6d317e70e/go.mod h1:mXi4GBBbnImb6dmsKGUJ2LatrhH/nqhxcFungHvyanc= golang.org/x/mod v0.1.0/go.mod h1:0QHyrYULN0/3qlju5TqG8bIK38QM8yzMo5ekMj3DlcY= -golang.org/x/mod v0.1.1-0.20191105210325-c90efee705ee h1:WG0RUwxtNT4qqaXX3DPA8zHFNm/D9xaBpxzHt1WcA/E= golang.org/x/mod v0.1.1-0.20191105210325-c90efee705ee/go.mod h1:QqPTAvyqsEbceGzBzNggFXnrqF1CaUcvgkdR5Ot7KZg= -golang.org/x/mod v0.2.0 h1:KU7oHjnv3XNWfa5COkzUifxZmxp1TyI7ImMXqFxLwvQ= golang.org/x/mod v0.2.0/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA= -golang.org/x/net v0.0.0-20180406214816-61147c48b25b/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= +golang.org/x/mod v0.3.0 h1:RM4zey1++hCTbCVQfnWeKs9/IEsaBLA8vTkd0WVtmH4= +golang.org/x/mod v0.3.0/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA= 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= @@ -904,17 +767,16 @@ golang.org/x/net v0.0.0-20190813141303-74dc4d7220e7/go.mod h1:z5CRVTTTmAJ677TzLL golang.org/x/net v0.0.0-20190827160401-ba9fcec4b297/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/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= -golang.org/x/net v0.0.0-20200202094626-16171245cfb2 h1:CCH4IOTTfewWjGOlSp+zGcjutRKlBEZQ6wTn8ozI/nI= golang.org/x/net v0.0.0-20200202094626-16171245cfb2/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= golang.org/x/net v0.0.0-20200226121028-0de0cce0169b/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= golang.org/x/net v0.0.0-20200301022130-244492dfa37a/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= -golang.org/x/net v0.0.0-20200324143707-d3edc9973b7e h1:3G+cUijn7XD+S4eJFddp53Pv7+slrESplyjG25HgL+k= golang.org/x/net v0.0.0-20200324143707-d3edc9973b7e/go.mod h1:qpuaurCH72eLCgpAm/N6yyVIVM9cpaDIP3A8BGJEC5A= -golang.org/x/net v0.0.0-20200520182314-0ba52f642ac2 h1:eDrdRpKgkcCqKZQwyZRyeFZgfqt37SL7Kv3tok06cKE= golang.org/x/net v0.0.0-20200520182314-0ba52f642ac2/go.mod h1:qpuaurCH72eLCgpAm/N6yyVIVM9cpaDIP3A8BGJEC5A= +golang.org/x/net v0.0.0-20200625001655-4c5254603344/go.mod h1:/O7V0waA8r7cgGh81Ro3o1hOxt32SMVPicZroKQ2sZA= +golang.org/x/net v0.0.0-20200813134508-3edf25e44fcc h1:zK/HqS5bZxDptfPJNq8v7vJfXtkU7r9TLIoSr1bXaP4= +golang.org/x/net v0.0.0-20200813134508-3edf25e44fcc/go.mod h1:/O7V0waA8r7cgGh81Ro3o1hOxt32SMVPicZroKQ2sZA= golang.org/x/oauth2 v0.0.0-20180821212333-d2e6202438be/go.mod h1:N/0e6XlmueqKjAGxoOufVs8QHGRruUQn6yWY3a++T0U= golang.org/x/oauth2 v0.0.0-20190226205417-e64efc72b421/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw= -golang.org/x/oauth2 v0.0.0-20190604053449-0f29369cfe45 h1:SVwTIAaPC2U/AvvLNZ2a7OVsmBpC8L5BlwK1whH3hm0= golang.org/x/oauth2 v0.0.0-20190604053449-0f29369cfe45/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw= golang.org/x/oauth2 v0.0.0-20191202225959-858c2ad4c8b6 h1:pE8b58s1HRDMi8RDc79m0HISf9D4TzseP40cEA6IGfs= golang.org/x/oauth2 v0.0.0-20191202225959-858c2ad4c8b6/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw= @@ -923,8 +785,9 @@ golang.org/x/sync v0.0.0-20181108010431-42b317875d0f/go.mod h1:RxMgew5VJxzue5/jJ golang.org/x/sync v0.0.0-20181221193216-37e7f081c4d4/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20190227155943-e225da77a7e6/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20190423024810-112230192c58/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= -golang.org/x/sync v0.0.0-20190911185100-cd5d95a43a6e h1:vcxGaoTs7kV8m5Np9uUNQin4BrLOthgV7252N8V+FwY= golang.org/x/sync v0.0.0-20190911185100-cd5d95a43a6e/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= +golang.org/x/sync v0.0.0-20200625203802-6e8e738ad208 h1:qwRHBd0NqMbJxfbotnDhm2ByMI1Shq4Y6oRJo21SGJA= +golang.org/x/sync v0.0.0-20200625203802-6e8e738ad208/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/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= @@ -943,7 +806,6 @@ golang.org/x/sys v0.0.0-20190606165138-5da285871e9c/go.mod h1:h1NjWce9XRLGQEsW7w golang.org/x/sys v0.0.0-20190610200419-93c9922d18ae/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20190616124812-15dcb6c0061f/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20190624142023-c5567b49c5d0/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20190626221950-04f50cda93cb/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20190726091711-fc99dfbffb4e/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20190804053845-51ab0e2deafa/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20190813064441-fde4db37ae7a/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= @@ -953,14 +815,18 @@ golang.org/x/sys v0.0.0-20191026070338-33540a1f6037/go.mod h1:h1NjWce9XRLGQEsW7w golang.org/x/sys v0.0.0-20191128015809-6d18c012aee9/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20191204072324-ce4227a45e2e/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20191228213918-04cbcbbfeed8/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20200116001909-b77594299b42/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20200122134326-e047566fdf82/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20200202164722-d101bd2416d5/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20200323222414-85ca7c5b95cd h1:xhmwyvizuTgC2qz7ZlMluP20uW+C3Rm0FD/WLDX8884= +golang.org/x/sys v0.0.0-20200223170610-d5e6a3e2c0ae/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20200323222414-85ca7c5b95cd/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20200819171115-d785dc25833f h1:KJuwZVtZBVzDmEDtB2zro9CXkD9O0dpCv4o2LHbQIAw= +golang.org/x/sys v0.0.0-20200819171115-d785dc25833f/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/text v0.3.3 h1:cokOdA+Jmi5PJGXLlLllQSgYigAEfHXJAERHVMaCc2k= +golang.org/x/text v0.3.3/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ= golang.org/x/time v0.0.0-20180412165947-fbb02b2291d2/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= golang.org/x/time v0.0.0-20181108054448-85acf8d2951c/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= golang.org/x/time v0.0.0-20190308202827-9d24e82272b4/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= @@ -990,7 +856,6 @@ golang.org/x/tools v0.0.0-20191012152004-8de300cfc20a/go.mod h1:b+2E5dAYhXwXZwtn 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-20191030062658-86caa796c7ab/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= -golang.org/x/tools v0.0.0-20191107010934-f79515f33823 h1:akkRBeitX2EZP59KdtKw310CI4WGPCNPyrLbE7WZA8Y= golang.org/x/tools v0.0.0-20191107010934-f79515f33823/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= golang.org/x/tools v0.0.0-20191113191852-77e3bb0ad9e7/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= golang.org/x/tools v0.0.0-20191114200427-caa0b0f7d508/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= @@ -1003,22 +868,21 @@ golang.org/x/tools v0.0.0-20191227053925-7b8e75db28f4/go.mod h1:TB2adYChydJhpapK golang.org/x/tools v0.0.0-20200130002326-2f3ba24bd6e7/go.mod h1:TB2adYChydJhpapKDTa4BR/hXlZSLoq2Wpct/0txZ28= golang.org/x/tools v0.0.0-20200225230052-807dcd883420/go.mod h1:TB2adYChydJhpapKDTa4BR/hXlZSLoq2Wpct/0txZ28= golang.org/x/tools v0.0.0-20200325010219-a49f79bcc224/go.mod h1:Sl4aGygMT6LrqrWclx+PTx3U+LnKx/seiNR+3G19Ar8= -golang.org/x/tools v0.0.0-20200325203130-f53864d0dba1 h1:odiryKYJy7CjdrZxhrcE1Z8L9+kGyGZOnfpuauvdCeU= golang.org/x/tools v0.0.0-20200325203130-f53864d0dba1/go.mod h1:Sl4aGygMT6LrqrWclx+PTx3U+LnKx/seiNR+3G19Ar8= golang.org/x/tools v0.0.0-20200521211927-2b542361a4fc/go.mod h1:EkVYQZoAsY45+roYkvgYkIh4xh/qjgUK9TdY2XT94GE= -golang.org/x/tools v0.0.0-20200527183253-8e7acdbce89d h1:SR+e35rACZFBohNb4Om1ibX6N3iO0FtdbwqGSuD9dBU= golang.org/x/tools v0.0.0-20200527183253-8e7acdbce89d/go.mod h1:EkVYQZoAsY45+roYkvgYkIh4xh/qjgUK9TdY2XT94GE= +golang.org/x/tools v0.0.0-20200820010801-b793a1359eac h1:DugppSxw0LSF8lcjaODPJZoDzq0ElTGskTst3ZaBkHI= +golang.org/x/tools v0.0.0-20200820010801-b793a1359eac/go.mod h1:njjCfa9FT2d7l9Bc6FUM5FLjQPp3cFF28FI3qnDFljA= golang.org/x/xerrors v0.0.0-20190717185122-a985d3407aa7/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= -golang.org/x/xerrors v0.0.0-20191011141410-1b5146add898 h1:/atklqdjdhuosWIl6AIbOeHJjicWYPqR9bpxqxYG2pA= golang.org/x/xerrors v0.0.0-20191011141410-1b5146add898/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= -golang.org/x/xerrors v0.0.0-20191204190536-9bdfabe68543 h1:E7g+9GITq07hpfrRu66IVDexMakfv52eLZ2CXBWiKr4= golang.org/x/xerrors v0.0.0-20191204190536-9bdfabe68543/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= +golang.org/x/xerrors v0.0.0-20200804184101-5ec99f83aff1 h1:go1bK/D/BFZV2I8cIQd1NKEZ+0owSTG1fDTci4IqFcE= +golang.org/x/xerrors v0.0.0-20200804184101-5ec99f83aff1/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= google.golang.org/api v0.4.0/go.mod h1:8k5glujaEP+g9n7WNsDg8QP6cUVNI86fCNMcbazEtwE= google.golang.org/api v0.7.0/go.mod h1:WtwebWUNSVBH/HAw79HIFXZNqEvBhG+Ra+ax0hx3E3M= google.golang.org/api v0.8.0/go.mod h1:o4eAsZoiT+ibD93RtjEohWalFOjRDx6CVaqeizhEnKg= google.golang.org/api v0.9.0/go.mod h1:o4eAsZoiT+ibD93RtjEohWalFOjRDx6CVaqeizhEnKg= google.golang.org/api v0.13.0/go.mod h1:iLdEw5Ide6rF15KTC1Kkl0iskquN2gFfn9o9XIsbkAI= -google.golang.org/api v0.14.0 h1:uMf5uLi4eQMRrMKhCplNik4U4H8Z6C1br3zOtAa/aDE= google.golang.org/api v0.14.0/go.mod h1:iLdEw5Ide6rF15KTC1Kkl0iskquN2gFfn9o9XIsbkAI= google.golang.org/api v0.15.0/go.mod h1:iLdEw5Ide6rF15KTC1Kkl0iskquN2gFfn9o9XIsbkAI= google.golang.org/api v0.15.1 h1:5mMS6mYvK5LVB8+ujVBC33Y8gltBo/kT6HBm6kU80G4= @@ -1060,7 +924,6 @@ gopkg.in/alecthomas/gometalinter.v2 v2.0.12/go.mod h1:NDRytsqEZyolNuAgTzJkZMkSQM 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-20160105164936-4f90aeace3a2/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= 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= @@ -1082,15 +945,14 @@ gopkg.in/jcmturner/gokrb5.v7 v7.5.0 h1:a9tsXlIDD9SKxotJMK3niV7rPZAJeX2aD/0yg3qlI gopkg.in/jcmturner/gokrb5.v7 v7.5.0/go.mod h1:l8VISx+WGYp+Fp7KRbsiUuXTTOnxIc3Tuvyavf11/WM= gopkg.in/jcmturner/rpc.v1 v1.1.0 h1:QHIUxTX1ISuAv9dD2wJ9HWQVuWDX/Zc0PfeC2tjc4rU= gopkg.in/jcmturner/rpc.v1 v1.1.0/go.mod h1:YIdkC4XfD6GXbzje11McwsDuOlZQSb9W4vfLvuNnlv8= -gopkg.in/mgo.v2 v2.0.0-20160818015218-f2b6f6c918c4 h1:hILp2hNrRnYjZpmIbx70psAHbBSEcQ1NIzDcUbJ1b6g= -gopkg.in/mgo.v2 v2.0.0-20160818015218-f2b6f6c918c4/go.mod h1:yeKp02qBN3iKW1OzL3MGk2IdtZzaj7SFntXj72NppTA= +gopkg.in/mgo.v2 v2.0.0-20190816093944-a6b53ec6cb22 h1:VpOs+IwYnYBaFnrNAeB8UUWtL3vEUnzSCL1nVjPhqrw= +gopkg.in/mgo.v2 v2.0.0-20190816093944-a6b53ec6cb22/go.mod h1:yeKp02qBN3iKW1OzL3MGk2IdtZzaj7SFntXj72NppTA= 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/square/go-jose.v2 v2.4.1/go.mod h1:M9dMgbHiYLoDGQrXy7OpJDJWiKiU//h+vD76mk0e1AI= 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-20170712054546-1be3d31502d6/go.mod h1:JAlM8MvJe8wmxCU4Bli9HhUf9+ttbYbLASfIpnQbh74= gopkg.in/yaml.v2 v2.0.0-20170812160011-eb3733d160e7/go.mod h1:JAlM8MvJe8wmxCU4Bli9HhUf9+ttbYbLASfIpnQbh74= gopkg.in/yaml.v2 v2.2.1/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= gopkg.in/yaml.v2 v2.2.2/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= @@ -1098,18 +960,18 @@ gopkg.in/yaml.v2 v2.2.3/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= gopkg.in/yaml.v2 v2.2.4/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= gopkg.in/yaml.v2 v2.2.5/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= gopkg.in/yaml.v2 v2.2.7/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= -gopkg.in/yaml.v2 v2.2.8 h1:obN1ZagJSUGI0Ek/LBmuj4SNLPfIny3KsKFopxRdj10= gopkg.in/yaml.v2 v2.2.8/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= +gopkg.in/yaml.v2 v2.3.0 h1:clyUAQHOM3G0M3f5vQj7LuJrETvjVot3Z5el9nffUtU= +gopkg.in/yaml.v2 v2.3.0/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= honnef.co/go/tools v0.0.0-20190102054323-c2f93a96b099/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4= honnef.co/go/tools v0.0.0-20190106161140-3f1c8253044a/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4= honnef.co/go/tools v0.0.0-20190418001031-e561f6794a2a/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4= honnef.co/go/tools v0.0.0-20190523083050-ea95bdfd59fc/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4= -honnef.co/go/tools v0.0.1-2019.2.3 h1:3JgtbtFHMiCmsznwGVTUWbgGov+pVqnlf1dEJTNAXeM= honnef.co/go/tools v0.0.1-2019.2.3/go.mod h1:a3bituU0lyd329TUQxRnasdCoJDkEUEAqEt0JzvZhAg= -honnef.co/go/tools v0.0.1-2020.1.3 h1:sXmLre5bzIR6ypkjXCDI3jHPssRhc8KD/Ome589sc3U= honnef.co/go/tools v0.0.1-2020.1.3/go.mod h1:X/FiERA/W4tHapMX5mGpAtMSVEeEUOyHaw9vFzvIQ3k= -honnef.co/go/tools v0.0.1-2020.1.4 h1:UoveltGrhghAA7ePc+e+QYDHXrBps2PqFZiHkGR/xK8= honnef.co/go/tools v0.0.1-2020.1.4/go.mod h1:X/FiERA/W4tHapMX5mGpAtMSVEeEUOyHaw9vFzvIQ3k= +honnef.co/go/tools v0.0.1-2020.1.5 h1:nI5egYTGJakVyOryqLs1cQO5dO0ksin5XXs2pspk75k= +honnef.co/go/tools v0.0.1-2020.1.5/go.mod h1:X/FiERA/W4tHapMX5mGpAtMSVEeEUOyHaw9vFzvIQ3k= k8s.io/klog v1.0.0/go.mod h1:4Bi6QPql/J/LkTDqv7R/cd3hPo4k2DG6Ptcz060Ez5I= rsc.io/binaryregexp v0.2.0/go.mod h1:qTv7/COck+e2FymRvadv62gMdZztPaShugOCi3I+8D8= sigs.k8s.io/yaml v1.1.0 h1:4A07+ZFc2wgJwo8YNlQpr1rVlgUDlxXHhPJciaPY5gs= @@ -1118,3 +980,5 @@ sourcegraph.com/sourcegraph/appdash v0.0.0-20180531100431-4c381bd170b4 h1:VO9oZb 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= sourcegraph.com/sourcegraph/appdash-data v0.0.0-20151005221446-73f23eafcf67/go.mod h1:L5q+DGLGOQFpo1snNEkLOJT2d1YTW66rWNzatr3He1k= +upper.io/db.v3 v3.7.1+incompatible h1:GiK/NmDUClH3LrZd54qj5OQsz8brGFv652QXyRXtg2U= +upper.io/db.v3 v3.7.1+incompatible/go.mod h1:FgTdD24eBjJAbPKsQSiHUNgXjOR4Lub3u1UMHSIh82Y= diff --git a/integration/README.md b/integration/README.md new file mode 100644 index 00000000000..0b67244fd1c --- /dev/null +++ b/integration/README.md @@ -0,0 +1,72 @@ +# Integration Framework + +## Introduction +The **Integration Framework** is designed to provide a flexible way for contributors to write integration tests for new sinks or MQ protocols. The core of the framework is stored in `{ticdc_root}/integration/framework`, and test cases should be stored in `{ticdc_root}/integration`. Currently, although the Framework is still under active development, it is capable of helping test Avro support and it is the only officially supported way for developers to run integration tests with Kafka connect. + +## Quick Start +To create a test case, you need to: +- create a struct that implements the `Task` interface, +- and ask the Environment to run the task in the `main` function in `integration.go`. +Note that the second step will be automated soon. + +```go +// Task represents a single test case +type Task interface { + Name() string + GetCDCProfile() *CDCProfile + Prepare(taskContext *TaskContext) error + Run(taskContext *TaskContext) error +} +``` + +For the time being, if you would like to write Avro tests, it is recommended to embed `framework.AvroSingleTableTask` in your own struct, which executes the necessary setup steps, including creating the Kafka Connect sink and creating the changefeed with appropriate configurations. + + +Example: +```go +type myCase struct { + framework.AvroSingleTableTask +} + +func newMyCase() *myCase{ + myCase := new(myCase) + myCase.AvroSingleTableTask.TableName = "test" + return myCase +} + +func (c *myCase) Name() string { + return "My Case" +} + +func (c *alterCase) Run(ctx *framework.TaskContext) error { + _, err := ctx.Upstream.ExecContext(ctx.Ctx, "create table test (id int primary key, value int)") + if err != nil { + return err + } + + // Get a handle of an existing table + table := ctx.SQLHelper().GetTable("test") + // Create an SQL request, send it to the upstream, wait for completion and check the correctness of replication + err = table.Insert(map[string]interface{}{ + "id": 0, + "value": 0, + }).Send().Wait().Check() + if err != nil { + return errors.AddStack(err) + } + + // To wait on a batch of SQL requests, create a slice of Awaitables + reqs := make([]framework.Awaitable, 0) + for i := 1; i < 1000; i++ { + // Only send, do not wait + req := table.Insert(map[string]interface{}{ + "id": i, + "value": i, + }).Send() + reqs = append(reqs, req) + } + + // Wait on SQL requests in batch and check the correctness + return framework.All(ctx.SQLHelper(), reqs).Wait().Check() +} +``` diff --git a/integration/case_alter.go b/integration/case_alter.go new file mode 100644 index 00000000000..b6940856e80 --- /dev/null +++ b/integration/case_alter.go @@ -0,0 +1,70 @@ +// Copyright 2020 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package main + +import ( + "fmt" + "math/rand" + + "github.com/pingcap/errors" + "github.com/pingcap/ticdc/integration/framework" +) + +//nolint:unused +type alterCase struct { + framework.AvroSingleTableTask +} + +func newAlterCase() *alterCase { + alterCase := new(alterCase) + alterCase.AvroSingleTableTask.TableName = "test" + return alterCase +} + +func (c *alterCase) Name() string { + return "Alter" +} + +func (c *alterCase) Run(ctx *framework.TaskContext) error { + _, err := ctx.Upstream.ExecContext(ctx.Ctx, "create table test (id int primary key)") + if err != nil { + return err + } + + for i := 0; i < 20; i++ { + _, err := ctx.Upstream.ExecContext(ctx.Ctx, fmt.Sprintf("alter table test add column (value%d int)", i)) + if err != nil { + return err + } + + table := ctx.SQLHelper().GetTable("test") + reqs := make([]framework.Awaitable, 0) + for j := 0; j < 1000; j++ { + rowData := make(map[string]interface{}, i+1) + rowData["id"] = i*1000 + j + for k := 0; k <= i; k++ { + rowData[fmt.Sprintf("value%d", k)] = rand.Int31() + } + awaitable := table.Insert(rowData).Send() + reqs = append(reqs, awaitable) + } + + err = framework.All(ctx.SQLHelper(), reqs).Wait().Check() + if err != nil { + return errors.AddStack(err) + } + } + + return nil +} diff --git a/integration/case_composite_pkey.go b/integration/case_composite_pkey.go new file mode 100644 index 00000000000..df2b811bf25 --- /dev/null +++ b/integration/case_composite_pkey.go @@ -0,0 +1,68 @@ +// Copyright 2020 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package main + +import ( + "github.com/pingcap/errors" + "github.com/pingcap/ticdc/integration/framework" +) + +//nolint:unused +type compositePKeyCase struct { + framework.AvroSingleTableTask +} + +func newCompositePKeyCase() *compositePKeyCase { + compositePKeyCase := new(compositePKeyCase) + compositePKeyCase.AvroSingleTableTask.TableName = "test" + return compositePKeyCase +} + +func (s *compositePKeyCase) Name() string { + return "Composite Primary Key" +} + +func (s *compositePKeyCase) Run(ctx *framework.TaskContext) error { + _, err := ctx.Upstream.ExecContext(ctx.Ctx, "create table test (id1 int, id2 int, value int, primary key (id1, id2))") + if err != nil { + return err + } + + // Get a handle of an existing table + table := ctx.SQLHelper().GetTable("test") + // Create an SQL request, send it to the upstream, wait for completion and check the correctness of replication + err = table.Insert(map[string]interface{}{ + "id1": 0, + "id2": 1, + "value": 0, + }).Send().Wait().Check() + if err != nil { + return errors.AddStack(err) + } + + err = table.Upsert(map[string]interface{}{ + "id1": 0, + "id2": 1, + "value": 1, + }).Send().Wait().Check() + if err != nil { + return err + } + + err = table.Delete(map[string]interface{}{ + "id1": 0, + "id2": 1, + }).Send().Wait().Check() + return err +} diff --git a/integration/case_delete.go b/integration/case_delete.go new file mode 100644 index 00000000000..79661ecd434 --- /dev/null +++ b/integration/case_delete.go @@ -0,0 +1,90 @@ +// Copyright 2020 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package main + +import ( + "errors" + + "github.com/pingcap/log" + "github.com/pingcap/ticdc/integration/framework" + "go.uber.org/zap" +) + +//nolint:unused +type deleteCase struct { + framework.AvroSingleTableTask +} + +func newDeleteCase() *deleteCase { + deleteCase := new(deleteCase) + deleteCase.AvroSingleTableTask.TableName = "test" + return deleteCase +} + +func (c *deleteCase) Name() string { + return "Delete" +} + +func (c *deleteCase) Run(ctx *framework.TaskContext) error { + _, err := ctx.Upstream.ExecContext(ctx.Ctx, "create table test (id int primary key, value int)") + if err != nil { + return err + } + + table := ctx.SQLHelper().GetTable("test") + + // To wait on a batch of SQL requests, create a slice of Awaitables + reqs := make([]framework.Awaitable, 0) + for i := 0; i < 1000; i++ { + // Only send, do not wait + req := table.Insert(map[string]interface{}{ + "id": i, + "value": i, + }).Send() + reqs = append(reqs, req) + } + + err = framework.All(ctx.SQLHelper(), reqs).Wait().Check() + if err != nil { + return err + } + + deletes := make([]framework.Awaitable, 0, 1000) + for i := 0; i < 1000; i++ { + req := table.Delete(map[string]interface{}{ + "id": i, + }).Send() + deletes = append(deletes, req) + } + + for _, req := range deletes { + err := req.Wait().Check() + if err != nil { + return err + } + } + + count := 0 + err = ctx.Downstream.QueryRowContext(ctx.Ctx, "select count(*) from test").Scan(&count) + if err != nil { + return err + } + + if count != 0 { + log.Warn("table is not empty", zap.Int("count", count)) + return errors.New("table is not empty") + } + + return nil +} diff --git a/integration/case_many_types.go b/integration/case_many_types.go new file mode 100644 index 00000000000..480c6376c2d --- /dev/null +++ b/integration/case_many_types.go @@ -0,0 +1,99 @@ +// Copyright 2020 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package main + +import ( + "math" + "time" + + "github.com/pingcap/ticdc/integration/framework" +) + +//nolint:unused +type manyTypesCase struct { + framework.AvroSingleTableTask +} + +func newManyTypesCase() *manyTypesCase { + manyTypesCase := new(manyTypesCase) + manyTypesCase.AvroSingleTableTask.TableName = "test" + return manyTypesCase +} + +func (s *manyTypesCase) Name() string { + return "Many Types" +} + +func (s *manyTypesCase) Run(ctx *framework.TaskContext) error { + createDBQuery := `create table test ( + id INT, + t_boolean BOOLEAN, + t_bigint BIGINT, + t_double DOUBLE, + t_decimal DECIMAL(38, 19), + t_bit BIT(64), + t_date DATE, + t_datetime DATETIME, + t_timestamp TIMESTAMP NULL, + t_time TIME, + t_year YEAR, + t_char CHAR, + t_varchar VARCHAR(10), + t_blob BLOB, + t_text TEXT, + t_enum ENUM ('enum1', 'enum2', 'enum3'), + t_set SET ('a', 'b', 'c'), + t_json JSON, + PRIMARY KEY (id) + ) + ` + _, err := ctx.Upstream.ExecContext(ctx.Ctx, createDBQuery) + if err != nil { + return err + } + + _, err = ctx.Downstream.ExecContext(ctx.Ctx, "drop table if exists test") + if err != nil { + return err + } + + _, err = ctx.Downstream.ExecContext(ctx.Ctx, createDBQuery) + if err != nil { + return err + } + + // Get a handle of an existing table + table := ctx.SQLHelper().GetTable("test") + return table.Insert(map[string]interface{}{ + "id": 0, + "t_boolean": true, + "t_bigint": math.MaxInt64, + "t_double": 1.01234, + "t_decimal": "12345.6789", + "t_bit": 0b1001001, + "t_date": time.Date(2020, 1, 1, 0, 0, 0, 0, time.UTC), + "t_datetime": time.Now(), + "t_timestamp": time.Now(), + "t_time": "23:59:59", + "t_year": 2019, + "t_char": "a", + "t_varchar": "测试varchar", + "t_blob": []byte{0x1, 0x2, 0x0, 0x3, 0x4}, + "t_text": "测试text", + "t_enum": "enum2", + "t_set": "a,b", + "t_json": nil, + }).Send().Wait().Check() + +} diff --git a/integration/case_simple.go b/integration/case_simple.go new file mode 100644 index 00000000000..278fc9c32af --- /dev/null +++ b/integration/case_simple.go @@ -0,0 +1,82 @@ +// Copyright 2020 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package main + +import ( + "github.com/pingcap/errors" + "github.com/pingcap/ticdc/integration/framework" +) + +//nolint:unused +type simpleCase struct { + framework.AvroSingleTableTask +} + +func newSimpleCase() *simpleCase { + simpleCase := new(simpleCase) + simpleCase.AvroSingleTableTask.TableName = "test" + return simpleCase +} + +func (s *simpleCase) Name() string { + return "Simple" +} + +func (s *simpleCase) Run(ctx *framework.TaskContext) error { + _, err := ctx.Upstream.ExecContext(ctx.Ctx, "create table test (id int primary key, value int)") + if err != nil { + return err + } + + // Get a handle of an existing table + table := ctx.SQLHelper().GetTable("test") + // Create an SQL request, send it to the upstream, wait for completion and check the correctness of replication + err = table.Insert(map[string]interface{}{ + "id": 0, + "value": 0, + }).Send().Wait().Check() + if err != nil { + return errors.AddStack(err) + } + + // To wait on a batch of SQL requests, create a slice of Awaitables + reqs := make([]framework.Awaitable, 0) + for i := 1; i < 1000; i++ { + // Only send, do not wait + req := table.Insert(map[string]interface{}{ + "id": i, + "value": i, + }).Send() + reqs = append(reqs, req) + } + + // Wait on SQL requests in batch and check the correctness + err = framework.All(ctx.SQLHelper(), reqs).Wait().Check() + if err != nil { + return err + } + + err = table.Upsert(map[string]interface{}{ + "id": 0, + "value": 1, + }).Send().Wait().Check() + if err != nil { + return err + } + + err = table.Delete(map[string]interface{}{ + "id": 0, + }).Send().Wait().Check() + return err +} diff --git a/integration/case_unsigned.go b/integration/case_unsigned.go new file mode 100644 index 00000000000..1d5acb53b64 --- /dev/null +++ b/integration/case_unsigned.go @@ -0,0 +1,68 @@ +// Copyright 2020 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package main + +import ( + "github.com/pingcap/ticdc/integration/framework" +) + +//nolint:unused +type unsignedCase struct { + framework.AvroSingleTableTask +} + +func newUnsignedCase() *unsignedCase { + unsignedCase := new(unsignedCase) + unsignedCase.AvroSingleTableTask.TableName = "test" + return unsignedCase +} + +func (s *unsignedCase) Name() string { + return "Many Types" +} + +func (s *unsignedCase) Run(ctx *framework.TaskContext) error { + createDBQuery := `create table test ( + id INT, + t_int INT UNSIGNED, + t_bigint BIGINT UNSIGNED, + t_bit BIT(64), + PRIMARY KEY (id) + ) + ` + _, err := ctx.Upstream.ExecContext(ctx.Ctx, createDBQuery) + if err != nil { + return err + } + + _, err = ctx.Downstream.ExecContext(ctx.Ctx, "drop table if exists test") + if err != nil { + return err + } + + _, err = ctx.Downstream.ExecContext(ctx.Ctx, createDBQuery) + if err != nil { + return err + } + + // Get a handle of an existing table + table := ctx.SQLHelper().GetTable("test") + return table.Insert(map[string]interface{}{ + "id": 0, + "t_int": 0xFEEDBEEF, + "t_bigint": uint64(0xFEEDBEEFFEEDBEEF), + "t_bit": uint64(0xFFFFFFFFFFFFFFFA), + }).Send().Wait().Check() + +} diff --git a/integration/framework/avro_kafka_docker_env.go b/integration/framework/avro_kafka_docker_env.go new file mode 100644 index 00000000000..0a65be05673 --- /dev/null +++ b/integration/framework/avro_kafka_docker_env.go @@ -0,0 +1,154 @@ +// Copyright 2020 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package framework + +import ( + "context" + "database/sql" + "encoding/json" + "errors" + "io/ioutil" + "net/http" + "os/exec" + "time" + + "github.com/integralist/go-findroot/find" + "github.com/pingcap/log" + "github.com/pingcap/ticdc/pkg/retry" + "go.uber.org/zap" +) + +const ( + healthCheckURI = "http://127.0.0.1:18083" + dockerComposeFilePath = "/docker-compose-avro.yml" + controllerContainerName = "ticdc_controller_1" + upstreamDSN = "root@tcp(127.0.0.1:4000)/" + downstreamDSN = "root@tcp(127.0.0.1:5000)/" +) + +// AvroKafkaDockerEnv represents the docker-compose service defined in docker-compose-avro.yml +type AvroKafkaDockerEnv struct { + dockerComposeOperator +} + +// NewAvroKafkaDockerEnv creates a new AvroKafkaDockerEnv +func NewAvroKafkaDockerEnv(dockerComposeFile string) *AvroKafkaDockerEnv { + healthChecker := func() error { + resp, err := http.Get(healthCheckURI) + if err != nil { + return err + } + + if resp.Body == nil { + return errors.New("kafka Connect HealthCheck returns empty body") + } + defer func() { _ = resp.Body.Close() }() + + bytes, err := ioutil.ReadAll(resp.Body) + if err != nil { + return err + } + + m := make(map[string]interface{}) + err = json.Unmarshal(bytes, &m) + if err != nil { + return err + } + + healthy, ok := m["healthy"] + if !ok { + return errors.New("kafka connect healthcheck did not return health info") + } + + if !healthy.(bool) { + return errors.New("kafka connect not healthy") + } + + return nil + } + + var file string + if dockerComposeFile == "" { + st, err := find.Repo() + if err != nil { + log.Fatal("Could not find git repo root", zap.Error(err)) + } + file = st.Path + dockerComposeFilePath + } else { + file = dockerComposeFile + } + + return &AvroKafkaDockerEnv{dockerComposeOperator{ + fileName: file, + controller: controllerContainerName, + healthChecker: healthChecker, + }} +} + +// Reset implements Environment +func (e *AvroKafkaDockerEnv) Reset() { + e.TearDown() + e.Setup() +} + +// RunTest implements Environment +func (e *AvroKafkaDockerEnv) RunTest(task Task) { + cmdLine := "/cdc " + task.GetCDCProfile().String() + bytes, err := e.ExecInController(cmdLine) + if err != nil { + log.Fatal("RunTest failed: cannot setup changefeed", + zap.Error(err), + zap.ByteString("stdout", bytes), + zap.ByteString("stderr", err.(*exec.ExitError).Stderr)) + } + + upstream, err := sql.Open("mysql", upstreamDSN) + if err != nil { + log.Fatal("RunTest: cannot connect to upstream database", zap.Error(err)) + } + + downstream, err := sql.Open("mysql", downstreamDSN) + if err != nil { + log.Fatal("RunTest: cannot connect to downstream database", zap.Error(err)) + } + + taskCtx := &TaskContext{ + Upstream: upstream, + Downstream: downstream, + env: e, + waitForReady: func() error { + return retry.Run(time.Second, 120, e.healthChecker) + }, + Ctx: context.Background(), + } + + err = task.Prepare(taskCtx) + if err != nil { + e.TearDown() + log.Fatal("RunTest: task preparation failed", zap.String("name", task.Name()), zap.Error(err)) + } + + log.Info("Start running task", zap.String("name", task.Name())) + err = task.Run(taskCtx) + if err != nil { + e.TearDown() + log.Fatal("RunTest: task failed", zap.String("name", task.Name()), zap.Error(err)) + } + log.Info("Finished running task", zap.String("name", task.Name())) +} + +// SetListener implements Environment. Currently unfinished, will be used to monitor Kafka output +func (e *AvroKafkaDockerEnv) SetListener(states interface{}, listener MqListener) { + // TODO +} diff --git a/integration/framework/avro_kafka_docker_env_test.go b/integration/framework/avro_kafka_docker_env_test.go new file mode 100644 index 00000000000..9d5c741d1f0 --- /dev/null +++ b/integration/framework/avro_kafka_docker_env_test.go @@ -0,0 +1,85 @@ +// Copyright 2020 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package framework + +import ( + "os/exec" + "testing" + + "github.com/pingcap/log" + "github.com/stretchr/testify/require" +) + +func TestAvroKafkaDockerEnv_Basic(t *testing.T) { + env := NewAvroKafkaDockerEnv("") + require.NotNil(t, env) + + env.Setup() + + bytes, err := env.ExecInController("echo test") + require.NoErrorf(t, err, "Execution returned error", func() string { + switch err := err.(type) { + case *exec.ExitError: + return string(err.Stderr) + default: + return "" + } + }()) + require.Equal(t, "test\n", string(bytes)) + + env.TearDown() +} + +type dummyTask struct { + test *testing.T +} + +func (t *dummyTask) Prepare(taskContext *TaskContext) error { + return nil +} + +func (t *dummyTask) GetCDCProfile() *CDCProfile { + return &CDCProfile{ + PDUri: "http://upstream-pd:2379", + SinkURI: "kafka://kafka:9092/testdb_test?protocol=avro", + Opts: map[string]string{"registry": "http://schema-registry:8081"}, + } +} + +func (t *dummyTask) Name() string { + return "Dummy" +} + +func (t *dummyTask) Run(taskContext *TaskContext) error { + err := taskContext.Upstream.Ping() + require.NoError(t.test, err, "Pinging upstream failed") + + err = taskContext.Downstream.Ping() + require.NoError(t.test, err, "Pinging downstream failed") + + err = taskContext.CreateDB("testdb") + require.NoError(t.test, err) + + log.Info("Running task") + return nil +} + +func TestAvroKafkaDockerEnv_RunTest(t *testing.T) { + env := NewAvroKafkaDockerEnv("") + require.NotNil(t, env) + + env.Setup() + env.RunTest(&dummyTask{test: t}) + env.TearDown() +} diff --git a/integration/framework/avro_kafka_single_table.go b/integration/framework/avro_kafka_single_table.go new file mode 100644 index 00000000000..c73d6e3b18d --- /dev/null +++ b/integration/framework/avro_kafka_single_table.go @@ -0,0 +1,124 @@ +// Copyright 2020 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package framework + +import ( + "bytes" + "database/sql" + "fmt" + "io/ioutil" + "net/http" + + "github.com/pingcap/errors" + "github.com/pingcap/log" + "go.uber.org/zap" +) + +// AvroSingleTableTask provides a basic implementation for an Avro test case +type AvroSingleTableTask struct { + TableName string +} + +// Name implements Task +func (a *AvroSingleTableTask) Name() string { + log.Warn("AvroSingleTableTask should be embedded in another Task") + return "AvroSingleTableTask-" + a.TableName +} + +// GetCDCProfile implements Task +func (a *AvroSingleTableTask) GetCDCProfile() *CDCProfile { + return &CDCProfile{ + PDUri: "http://upstream-pd:2379", + SinkURI: "kafka://kafka:9092/testdb_" + a.TableName + "?protocol=avro", + Opts: map[string]string{"registry": "http://schema-registry:8081"}, + } +} + +// Prepare implements Task +func (a *AvroSingleTableTask) Prepare(taskContext *TaskContext) error { + err := taskContext.CreateDB("testdb") + if err != nil { + return err + } + + _ = taskContext.Upstream.Close() + taskContext.Upstream, err = sql.Open("mysql", upstreamDSN+"testdb") + if err != nil { + return err + } + + _ = taskContext.Downstream.Close() + taskContext.Downstream, err = sql.Open("mysql", downstreamDSN+"testdb") + if err != nil { + return err + } + + // TODO better way to generate JSON + connectorConfigFmt := `{ + "name": "jdbc-sink-connector", + "config": { + "connector.class": "io.confluent.connect.jdbc.JdbcSinkConnector", + "tasks.max": "1", + "topics": "testdb_%s", + "connection.url": "jdbc:mysql://root@downstream-tidb:4000/testdb", + "connection.ds.pool.size": 5, + "table.name.format": "%s", + "insert.mode": "upsert", + "delete.enabled": true, + "pk.mode": "record_key", + "auto.create": true, + "auto.evolve": true + } + }` + connectorConfig := fmt.Sprintf(connectorConfigFmt, a.TableName, a.TableName) + log.Debug("Creating Kafka sink connector", zap.String("config", connectorConfig)) + + resp, err := http.Post( + "http://127.0.0.1:8083/connectors", + "application/json", + bytes.NewReader([]byte(connectorConfig))) + if err != nil { + return err + } + + if resp.Body == nil { + return errors.New("Kafka Connect Rest API returned empty body") + } + defer resp.Body.Close() + + if resp.StatusCode < 200 || resp.StatusCode >= 300 { + str, err := ioutil.ReadAll(resp.Body) + if err != nil { + return err + } + log.Warn( + "Kafka Connect Rest API returned", + zap.Int("status", resp.StatusCode), + zap.ByteString("body", str)) + return errors.Errorf("Kafka Connect Rest API returned status code %d", resp.StatusCode) + } + + if taskContext.waitForReady != nil { + log.Info("Waiting for env to be ready") + return taskContext.waitForReady() + } + + return nil +} + +// Run implements Task +func (a *AvroSingleTableTask) Run(taskContext *TaskContext) error { + log.Warn("AvroSingleTableTask has been run") + return nil +} diff --git a/integration/framework/avro_kafka_single_table_test.go b/integration/framework/avro_kafka_single_table_test.go new file mode 100644 index 00000000000..cba00eb7417 --- /dev/null +++ b/integration/framework/avro_kafka_single_table_test.go @@ -0,0 +1,44 @@ +// Copyright 2020 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package framework + +import ( + "database/sql" + "testing" + + "github.com/stretchr/testify/require" +) + +type emptyAvroSingleTableTask struct { + AvroSingleTableTask +} + +func TestAvroSingleTableTest_Prepare(t *testing.T) { + env := NewAvroKafkaDockerEnv("") + require.NotNil(t, env) + + env.Setup() + env.RunTest(&emptyAvroSingleTableTask{AvroSingleTableTask{TableName: "test"}}) + + _, err := sql.Open("mysql", upstreamDSN+"testdb") + require.NoError(t, err) + + _, err = sql.Open("mysql", downstreamDSN+"testdb") + require.NoError(t, err) + + err = env.healthChecker() + require.NoError(t, err) + + env.TearDown() +} diff --git a/integration/framework/docker_compose_op.go b/integration/framework/docker_compose_op.go new file mode 100644 index 00000000000..e2b8329f154 --- /dev/null +++ b/integration/framework/docker_compose_op.go @@ -0,0 +1,76 @@ +// Copyright 2020 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package framework + +import ( + "os/exec" + "time" + + "github.com/pingcap/log" + "github.com/pingcap/ticdc/pkg/retry" + "go.uber.org/zap" +) + +type dockerComposeOperator struct { + fileName string + controller string + healthChecker func() error +} + +// Setup brings up a docker-compose service +func (d *dockerComposeOperator) Setup() { + cmd := exec.Command("docker-compose", "-f", d.fileName, "up", "--detach") + runCmdHandleError(cmd) + + if d.healthChecker != nil { + err := retry.Run(time.Second, 120, d.healthChecker) + if err != nil { + log.Fatal("Docker service health check failed after max retries", zap.Error(err)) + } + } +} + +func runCmdHandleError(cmd *exec.Cmd) []byte { + log.Info("Start executing command", zap.String("cmd", cmd.String())) + bytes, err := cmd.Output() + if err, ok := err.(*exec.ExitError); ok { + log.Info("Running command failed", zap.ByteString("stderr", err.Stderr)) + } + + if err != nil { + log.Fatal("Running command failed", + zap.Error(err), + zap.String("command", cmd.String()), + zap.ByteString("output", bytes)) + } + + log.Info("Finished executing command", zap.String("cmd", cmd.String())) + return bytes +} + +// TearDown terminates a docker-compose service and remove all volumes +func (d *dockerComposeOperator) TearDown() { + log.Info("Start tearing down docker-compose services") + cmd := exec.Command("docker-compose", "-f", d.fileName, "down", "-v") + runCmdHandleError(cmd) + log.Info("Finished tearing down docker-compose services") +} + +// ExecInController provides a way to execute commands inside a container in the service +func (d *dockerComposeOperator) ExecInController(shellCmd string) ([]byte, error) { + log.Info("Start executing in the controller container", zap.String("shellCmd", shellCmd)) + cmd := exec.Command("docker", "exec", d.controller, "sh", "-c", shellCmd) + defer log.Info("Finished executing in the controller container", zap.String("shellCmd", shellCmd)) + return cmd.Output() +} diff --git a/integration/framework/docker_compose_op_test.go b/integration/framework/docker_compose_op_test.go new file mode 100644 index 00000000000..3c53d4d47cb --- /dev/null +++ b/integration/framework/docker_compose_op_test.go @@ -0,0 +1,33 @@ +// Copyright 2020 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package framework + +import ( + "testing" + + "github.com/integralist/go-findroot/find" + "github.com/stretchr/testify/assert" +) + +func TestDockerComposeOperator_SetupTearDown(t *testing.T) { + st, err := find.Repo() + assert.NoError(t, err) + + d := &dockerComposeOperator{ + fileName: st.Path + "/docker-compose-avro.yml", + controller: "controller0", + } + d.Setup() + d.TearDown() +} diff --git a/integration/framework/dsl.go b/integration/framework/dsl.go new file mode 100644 index 00000000000..547e18bd625 --- /dev/null +++ b/integration/framework/dsl.go @@ -0,0 +1,125 @@ +// Copyright 2020 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package framework + +import ( + "context" + "time" + + backoff2 "github.com/cenkalti/backoff" + "github.com/pingcap/errors" + "github.com/pingcap/log" + "go.uber.org/zap" +) + +const ( + waitMaxPollInterval = time.Second * 5 +) + +// Awaitable represents the handle of an SQL operation that can be waited on +type Awaitable interface { + SetTimeOut(duration time.Duration) Awaitable + Wait() Checkable +} + +// Checkable represents the handle of an SQL operation whose correctness can be checked +type Checkable interface { + Check() error +} + +type pollable interface { + poll(ctx context.Context) (bool, error) +} + +type pollableAndCheckable interface { + pollable + Checkable +} + +type errorCheckableAndAwaitable struct { + error +} + +// Check implements Checkable +func (e *errorCheckableAndAwaitable) Check() error { + return e.error +} + +// Wait implements Awaitable +func (e *errorCheckableAndAwaitable) Wait() Checkable { + return e +} + +// SetTimeOut implements Awaitable +func (e *errorCheckableAndAwaitable) SetTimeOut(duration time.Duration) Awaitable { + return e +} + +type basicAwaitable struct { + pollableAndCheckable + timeout time.Duration +} + +// SetTimeOut implements Awaitable +func (b *basicAwaitable) SetTimeOut(duration time.Duration) Awaitable { + b.timeout = duration + return b +} + +// Wait implements Awaitable +func (b *basicAwaitable) Wait() Checkable { + var ( + ctx context.Context + cancel context.CancelFunc + ) + if b.timeout == 0 { + ctx, cancel = context.WithCancel(context.Background()) + } else { + ctx, cancel = context.WithTimeout(context.Background(), b.timeout) + } + defer cancel() + + backoff := backoff2.NewExponentialBackOff() + backoff.MaxInterval = waitMaxPollInterval + for { + select { + case <-ctx.Done(): + return &errorCheckableAndAwaitable{ctx.Err()} + default: + } + + ok, err := b.poll(ctx) + if err != nil { + return &errorCheckableAndAwaitable{errors.Annotate(err, "Wait() failed with error")} + } + + if ok { + log.Debug("Wait(): pollable finished") + return b + } + + interval := backoff.NextBackOff() + if interval == backoff2.Stop { + return &errorCheckableAndAwaitable{errors.New("Maximum retry interval reached")} + } + log.Debug("Wait(): pollable returned false, backing off", zap.Duration("interval", interval)) + + ch := time.After(interval) + select { + case <-ctx.Done(): + return &errorCheckableAndAwaitable{ctx.Err()} + case <-ch: + } + } +} diff --git a/integration/framework/env.go b/integration/framework/env.go new file mode 100644 index 00000000000..1bf466c7fa4 --- /dev/null +++ b/integration/framework/env.go @@ -0,0 +1,26 @@ +// Copyright 2020 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package framework + +// MqListener represents a callback function for listening on the MQ output +type MqListener func(states interface{}, topic string, key []byte, value []byte) error + +// Environment is an abstract of the CDC-Upstream-Downstream-MQ complex to be tested +type Environment interface { + Setup() + TearDown() + Reset() + RunTest(Task) + SetListener(states interface{}, listener MqListener) +} diff --git a/integration/framework/sql_batch_op.go b/integration/framework/sql_batch_op.go new file mode 100644 index 00000000000..8eb3f5f9476 --- /dev/null +++ b/integration/framework/sql_batch_op.go @@ -0,0 +1,141 @@ +// Copyright 2020 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package framework + +import ( + "context" + "fmt" + "reflect" + "strings" + + "github.com/jmoiron/sqlx" + "github.com/pingcap/errors" + "github.com/pingcap/log" + "go.uber.org/zap" +) + +const ( + selectQueryMaxBatchSize = 256 +) + +type sqlAllAwaiter struct { + helper *SQLHelper + data map[interface{}]sqlRowContainer + retrievedValues []map[string]interface{} + table *Table +} + +// All joins a slice of Awaitable sql requests. The request must be to the same table. +// TODO does not support composite primary key for now! +func All(helper *SQLHelper, awaitables []Awaitable) Awaitable { + if _, ok := awaitables[0].(sqlRowContainer); !ok { + return awaitables[0] + } + + ret := &sqlAllAwaiter{ + helper: helper, + data: make(map[interface{}]sqlRowContainer, len(awaitables)), + retrievedValues: make([]map[string]interface{}, 0), + table: awaitables[0].(sqlRowContainer).getTable(), + } + + for _, row := range awaitables { + rowContainer, ok := row.(sqlRowContainer) + if !ok { + return row + } + key := rowContainer.getData()[rowContainer.getTable().uniqueIndex[0]] + ret.data[normalizeKeys(key)] = rowContainer + } + + return &basicAwaitable{ + pollableAndCheckable: ret, + timeout: 0, + } +} + +func (s *sqlAllAwaiter) poll(ctx context.Context) (bool, error) { + db := sqlx.NewDb(s.helper.downstream, "mysql") + + batchSize := 0 + counter := 0 + indexValues := make([]interface{}, 0) + s.retrievedValues = make([]map[string]interface{}, 0) + for k, v := range s.data { + indexValues = append(indexValues, k) + batchSize++ + counter++ + if batchSize >= selectQueryMaxBatchSize || counter == len(s.data) { + log.Debug("Selecting", zap.String("table", s.table.tableName), zap.Any("keys", indexValues)) + query, args, err := sqlx.In("select distinct * from "+s.table.tableName+" where "+v.getTable().uniqueIndex[0]+" in (?)", indexValues) + if err != nil { + return false, errors.AddStack(err) + } + query = db.Rebind(query) + rows, err := db.QueryContext(ctx, query, args...) + if err != nil { + if strings.Contains(err.Error(), "Error 1146") { + log.Info("table does not exist, will try again", zap.Error(err), zap.String("query", query)) + return false, nil + } + return false, errors.AddStack(err) + } + + for rows.Next() { + m, err := rowsToMap(rows) + if err != nil { + return false, errors.AddStack(err) + } + s.retrievedValues = append(s.retrievedValues, m) + } + batchSize = 0 + indexValues = make([]interface{}, 0) + } + } + + log.Debug("poll finished", zap.Int("total-retrieved", len(s.retrievedValues))) + + if len(s.data) == len(s.retrievedValues) { + return true, nil + } + + return false, nil +} + +func (s *sqlAllAwaiter) Check() error { + for _, row := range s.retrievedValues { + key := row[s.table.uniqueIndex[0]] + expected := s.data[normalizeKeys(key)] + if !compareMaps(row, expected.getData()) { + log.Warn( + "Check failed", + zap.String("expected", fmt.Sprintf("%v", expected)), + zap.String("actual", fmt.Sprintf("%v", row)), + ) + return errors.New("Check failed") + } + } + return nil +} + +func normalizeKeys(key interface{}) interface{} { + switch key.(type) { + case int, int8, int16, int32, int64: + return reflect.ValueOf(key).Int() + case uint, uint8, uint16, uint32, uint64: + return reflect.ValueOf(key).Uint() + default: + return key + } +} diff --git a/integration/framework/sql_helper.go b/integration/framework/sql_helper.go new file mode 100644 index 00000000000..c67191dd139 --- /dev/null +++ b/integration/framework/sql_helper.go @@ -0,0 +1,481 @@ +// Copyright 2020 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package framework + +import ( + "context" + "database/sql" + "fmt" + "strings" + "sync/atomic" + + "github.com/jmoiron/sqlx" + "github.com/pingcap/errors" + "github.com/pingcap/log" + "github.com/pingcap/ticdc/pkg/quotes" + "go.uber.org/zap" + "go.uber.org/zap/zapcore" + "upper.io/db.v3/lib/sqlbuilder" + _ "upper.io/db.v3/mysql" // imported for side effects +) + +// SQLHelper provides basic utilities for manipulating data +type SQLHelper struct { + upstream *sql.DB + downstream *sql.DB + ctx context.Context +} + +// Table represents the handle of a table in the upstream +type Table struct { + err error + tableName string + uniqueIndex []string + helper *SQLHelper +} + +// GetTable returns the handle of the given table +func (h *SQLHelper) GetTable(tableName string) *Table { + db, err := sqlbuilder.New("mysql", h.upstream) + if err != nil { + return &Table{err: errors.AddStack(err)} + } + + idxCol, err := getUniqueIndexColumn(h.ctx, db, "testdb", tableName) + if err != nil { + return &Table{err: errors.AddStack(err)} + } + + return &Table{tableName: tableName, uniqueIndex: idxCol, helper: h} +} + +func (t *Table) makeSQLRequest(requestType sqlRequestType, rowData map[string]interface{}) (*sqlRequest, error) { + if t.err != nil { + return nil, t.err + } + + return &sqlRequest{ + tableName: t.tableName, + data: rowData, + result: nil, + uniqueIndex: t.uniqueIndex, + helper: t.helper, + requestType: requestType, + }, nil +} + +// Insert returns a Sendable object that represents an Insert clause +func (t *Table) Insert(rowData map[string]interface{}) Sendable { + basicReq, err := t.makeSQLRequest(sqlRequestTypeInsert, rowData) + if err != nil { + return &errorSender{err: err} + } + + return &syncSQLRequest{*basicReq} +} + +// Upsert returns a Sendable object that represents a Replace Into clause +func (t *Table) Upsert(rowData map[string]interface{}) Sendable { + basicReq, err := t.makeSQLRequest(sqlRequestTypeUpsert, rowData) + if err != nil { + return &errorSender{err: err} + } + + return &syncSQLRequest{*basicReq} +} + +// Delete returns a Sendable object that represents a Delete from clause +func (t *Table) Delete(rowData map[string]interface{}) Sendable { + basicReq, err := t.makeSQLRequest(sqlRequestTypeDelete, rowData) + if err != nil { + return &errorSender{err: err} + } + + return &syncSQLRequest{*basicReq} +} + +type sqlRowContainer interface { + getData() map[string]interface{} + getComparableKey() string + getTable() *Table +} + +type awaitableSQLRowContainer struct { + Awaitable + sqlRowContainer +} + +type sqlRequestType int32 + +const ( + sqlRequestTypeInsert sqlRequestType = iota + sqlRequestTypeUpsert + sqlRequestTypeDelete +) + +type sqlRequest struct { + tableName string + data map[string]interface{} + result map[string]interface{} + uniqueIndex []string + helper *SQLHelper + requestType sqlRequestType + hasReadBack uint32 +} + +// MarshalLogObjects helps printing the sqlRequest +func (s *sqlRequest) MarshalLogObject(encoder zapcore.ObjectEncoder) error { + encoder.AddString("upstream", fmt.Sprintf("%#v", s.data)) + encoder.AddString("downstream", fmt.Sprintf("%#v", s.result)) + return nil +} + +func (s *sqlRequest) getPrimaryKeyTuple() string { + return makeColumnTuple(s.uniqueIndex) +} + +func (s *sqlRequest) getWhereCondition() []interface{} { + builder := strings.Builder{} + args := make([]interface{}, 1, len(s.uniqueIndex)+1) + builder.WriteString(s.getPrimaryKeyTuple() + " = (") + for i, col := range s.uniqueIndex { + builder.WriteString("?") + if i != len(s.uniqueIndex)-1 { + builder.WriteString(",") + } + + args = append(args, s.data[col]) + } + builder.WriteString(")") + args[0] = builder.String() + return args +} + +func (s *sqlRequest) getComparableKey() string { + if len(s.uniqueIndex) == 1 { + return s.uniqueIndex[0] + } + + ret := make(map[string]interface{}) + for k, v := range s.data { + for _, col := range s.uniqueIndex { + if k == col { + ret[k] = v + } + } + } + return fmt.Sprintf("%v", ret) +} + +func (s *sqlRequest) getData() map[string]interface{} { + return s.data +} + +func (s *sqlRequest) getTable() *Table { + return &Table{ + err: nil, + tableName: s.tableName, + uniqueIndex: s.uniqueIndex, + helper: s.helper, + } +} + +func (s *sqlRequest) getAwaitableSQLRowContainer() *awaitableSQLRowContainer { + return &awaitableSQLRowContainer{ + Awaitable: &basicAwaitable{ + pollableAndCheckable: s, + timeout: 0, + }, + sqlRowContainer: s, + } +} + +// Sendable is a sendable request to the upstream +type Sendable interface { + Send() Awaitable +} + +type errorSender struct { + err error +} + +// Send implements sender +func (s *errorSender) Send() Awaitable { + return &errorCheckableAndAwaitable{s.err} +} + +type syncSQLRequest struct { + sqlRequest +} + +func (r *syncSQLRequest) Send() Awaitable { + atomic.StoreUint32(&r.hasReadBack, 0) + var err error + switch r.requestType { + case sqlRequestTypeInsert: + err = r.insert(r.helper.ctx) + case sqlRequestTypeUpsert: + err = r.upsert(r.helper.ctx) + case sqlRequestTypeDelete: + err = r.delete(r.helper.ctx) + } + + go func() { + db, err := sqlbuilder.New("mysql", r.helper.upstream) + if err != nil { + log.Warn("ReadBack:", zap.Error(err)) + return + } + + cond := r.getWhereCondition() + + rows, err := db.SelectFrom(r.tableName).Where(cond).QueryContext(r.helper.ctx) + if err != nil { + log.Warn("ReadBack:", zap.Error(err)) + return + } + defer rows.Close() + + if !rows.Next() { + log.Warn("ReadBack:", zap.Error(err)) + return + } + r.data, err = rowsToMap(rows) + if err != nil { + log.Warn("ReadBack", zap.Error(err)) + return + } + + atomic.StoreUint32(&r.hasReadBack, 1) + }() + + if err != nil { + return &errorCheckableAndAwaitable{errors.AddStack(err)} + } + return r.getAwaitableSQLRowContainer() +} + +/* +type asyncSQLRequest struct { + sqlRequest +} +*/ + +func (s *sqlRequest) insert(ctx context.Context) error { + db, err := sqlbuilder.New("mysql", s.helper.upstream) + if err != nil { + return errors.AddStack(err) + } + + keys := make([]string, len(s.data)) + values := make([]interface{}, len(s.data)) + i := 0 + for k, v := range s.data { + keys[i] = k + values[i] = v + i++ + } + + _, err = db.InsertInto(s.tableName).Columns(keys...).Values(values...).ExecContext(ctx) + if err != nil { + return errors.AddStack(err) + } + + s.requestType = sqlRequestTypeInsert + return nil +} + +func (s *sqlRequest) upsert(ctx context.Context) error { + db := sqlx.NewDb(s.helper.upstream, "mysql") + + keys := make([]string, len(s.data)) + values := make([]interface{}, len(s.data)) + i := 0 + for k, v := range s.data { + keys[i] = k + values[i] = v + i++ + } + + query, args, err := sqlx.In("replace into `"+s.tableName+"` "+makeColumnTuple(keys)+" values (?)", values) + if err != nil { + return errors.AddStack(err) + } + + query = db.Rebind(query) + _, err = s.helper.upstream.ExecContext(ctx, query, args...) + if err != nil { + return errors.AddStack(err) + } + + s.requestType = sqlRequestTypeUpsert + return nil +} + +func (s *sqlRequest) delete(ctx context.Context) error { + db, err := sqlbuilder.New("mysql", s.helper.downstream) + if err != nil { + return errors.AddStack(err) + } + + _, err = db.DeleteFrom(s.tableName).Where(s.getWhereCondition()).ExecContext(ctx) + if err != nil { + return errors.AddStack(err) + } + + s.requestType = sqlRequestTypeDelete + return nil +} + +func (s *sqlRequest) read(ctx context.Context) (map[string]interface{}, error) { + db, err := sqlbuilder.New("mysql", s.helper.downstream) + if err != nil { + return nil, errors.AddStack(err) + } + + rows, err := db.SelectFrom(s.tableName).Where(s.getWhereCondition()).QueryContext(ctx) + if err != nil { + return nil, errors.AddStack(err) + + } + defer rows.Close() + + if !rows.Next() { + return nil, nil + } + return rowsToMap(rows) +} + +//nolint:unused +func (s *sqlRequest) getBasicAwaitable() basicAwaitable { + return basicAwaitable{ + pollableAndCheckable: s, + timeout: 0, + } +} + +func (s *sqlRequest) poll(ctx context.Context) (bool, error) { + if atomic.LoadUint32(&s.hasReadBack) == 0 { + return false, nil + } + res, err := s.read(ctx) + if err != nil { + if strings.Contains(err.Error(), "Error 1146") { + return false, nil + } + return false, errors.AddStack(err) + } + s.result = res + + switch s.requestType { + case sqlRequestTypeInsert: + if res == nil { + return false, nil + } + return true, nil + case sqlRequestTypeUpsert: + if res == nil { + return false, nil + } + if compareMaps(s.data, res) { + return true, nil + } + log.Debug("Upserted row does not match the expected") + return false, nil + case sqlRequestTypeDelete: + if res == nil { + return true, nil + } + log.Debug("Delete not successful yet", zap.Reflect("where condition", s.getWhereCondition())) + return false, nil + } + return true, nil +} + +func (s *sqlRequest) Check() error { + if s.requestType == sqlRequestTypeUpsert || s.requestType == sqlRequestTypeDelete { + return nil + } + // TODO better comparator + if s.result == nil { + return errors.New("Check: nil result") + } + if compareMaps(s.data, s.result) { + return nil + } + log.Warn("Check failed", zap.Object("request", s)) + return errors.New("Check failed") +} + +func rowsToMap(rows *sql.Rows) (map[string]interface{}, error) { + colNames, err := rows.Columns() + if err != nil { + return nil, errors.AddStack(err) + } + + colData := make([]interface{}, len(colNames)) + colDataPtrs := make([]interface{}, len(colNames)) + for i := range colData { + colDataPtrs[i] = &colData[i] + } + + err = rows.Scan(colDataPtrs...) + if err != nil { + return nil, errors.AddStack(err) + } + + ret := make(map[string]interface{}, len(colNames)) + for i := 0; i < len(colNames); i++ { + ret[colNames[i]] = colData[i] + } + return ret, nil +} + +func getUniqueIndexColumn(ctx context.Context, db sqlbuilder.Database, dbName string, tableName string) ([]string, error) { + row, err := db.QueryRowContext(ctx, ` + SELECT GROUP_CONCAT(COLUMN_NAME SEPARATOR ' ') FROM INFORMATION_SCHEMA.STATISTICS + WHERE TABLE_SCHEMA = ? AND TABLE_NAME = ? + GROUP BY INDEX_NAME + ORDER BY FIELD(INDEX_NAME,'PRIMARY') DESC + `, dbName, tableName) + if err != nil { + return nil, errors.AddStack(err) + } + + colName := "" + err = row.Scan(&colName) + if err != nil { + return nil, errors.AddStack(err) + } + + return strings.Split(colName, " "), nil +} + +func compareMaps(m1 map[string]interface{}, m2 map[string]interface{}) bool { + // TODO better comparator + if m2 == nil { + return false + } + str1 := fmt.Sprintf("%v", m1) + str2 := fmt.Sprintf("%v", m2) + return str1 == str2 +} + +func makeColumnTuple(colNames []string) string { + colNamesQuoted := make([]string, len(colNames)) + for i := range colNames { + colNamesQuoted[i] = quotes.QuoteName(colNames[i]) + } + return "(" + strings.Join(colNamesQuoted, ",") + ")" +} diff --git a/integration/framework/task.go b/integration/framework/task.go new file mode 100644 index 00000000000..acccdba9193 --- /dev/null +++ b/integration/framework/task.go @@ -0,0 +1,104 @@ +// Copyright 2020 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package framework + +import ( + "context" + "database/sql" + "strings" + + _ "github.com/go-sql-driver/mysql" // imported for side effects + "github.com/pingcap/log" + "go.uber.org/zap" +) + +// Task represents a single test case +type Task interface { + Name() string + GetCDCProfile() *CDCProfile + Prepare(taskContext *TaskContext) error + Run(taskContext *TaskContext) error +} + +// TaskContext is passed to the test case to provide basic utilities for testing +type TaskContext struct { + Upstream *sql.DB + Downstream *sql.DB + env Environment + waitForReady func() error + Ctx context.Context +} + +// CDCProfile represents the command line arguments used to create the changefeed +type CDCProfile struct { + PDUri string + SinkURI string + Opts map[string]string +} + +// CreateDB creates a database in both the upstream and the downstream +func (c *TaskContext) CreateDB(name string) error { + log.Debug("Creating database in upstream", zap.String("db", name)) + _, err := c.Upstream.ExecContext(c.Ctx, "create database "+name) + if err != nil { + log.Warn("Failed to create database in upstream", zap.String("db", name), zap.Error(err)) + return err + } + log.Debug("Successfully created database in upstream", zap.String("db", name)) + + log.Debug("Creating database in downstream", zap.String("db", name)) + _, err = c.Downstream.ExecContext(c.Ctx, "create database "+name) + if err != nil { + log.Warn("Failed to create database in downstream", zap.String("db", name), zap.Error(err)) + return err + } + log.Debug("Successfully created database in downstream", zap.String("db", name)) + + return nil +} + +// SQLHelper returns an SQLHelper +func (c *TaskContext) SQLHelper() *SQLHelper { + return &SQLHelper{ + upstream: c.Upstream, + downstream: c.Downstream, + ctx: c.Ctx, + } +} + +// String returns the string representation of the CDCProfile +func (p *CDCProfile) String() string { + builder := strings.Builder{} + builder.WriteString("cli changefeed create ") + if p.PDUri == "" { + p.PDUri = "http://127.0.0.1:2379" + } + + builder.WriteString("--pd=" + p.PDUri + " ") + + if p.SinkURI == "" { + log.Fatal("SinkURI cannot be empty!") + } + + builder.WriteString("--sink-uri=" + p.SinkURI + " ") + + if p.Opts == nil || len(p.Opts) == 0 { + return builder.String() + } + + for k, v := range p.Opts { + builder.WriteString("--opts=\"" + k + "=" + v + "\" ") + } + return builder.String() +} diff --git a/integration/integration.go b/integration/integration.go new file mode 100644 index 00000000000..2d441edac84 --- /dev/null +++ b/integration/integration.go @@ -0,0 +1,47 @@ +// Copyright 2020 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package main + +import ( + "flag" + "github.com/pingcap/log" + "github.com/pingcap/ticdc/integration/framework" + "go.uber.org/zap/zapcore" +) + +func main() { + dockerComposeFile := flag.String("docker-compose-file", "", "the path of the Docker-compose yml file") + + testCases := []framework.Task{ + newSimpleCase(), + newDeleteCase(), + newManyTypesCase(), + newUnsignedCase(), + newCompositePKeyCase(), + newAlterCase(), // this case is slow, so put it last + } + + log.SetLevel(zapcore.DebugLevel) + env := framework.NewAvroKafkaDockerEnv(*dockerComposeFile) + env.Setup() + + for i := range testCases { + env.RunTest(testCases[i]) + if i < len(testCases)-1 { + env.Reset() + } + } + + env.TearDown() +} diff --git a/kafka_consumer/main.go b/kafka_consumer/main.go index 29bdee10673..e77acea55a2 100644 --- a/kafka_consumer/main.go +++ b/kafka_consumer/main.go @@ -28,6 +28,8 @@ import ( "syscall" "time" + "github.com/pingcap/ticdc/pkg/quotes" + "github.com/pingcap/ticdc/pkg/config" "github.com/pingcap/ticdc/pkg/security" @@ -281,7 +283,8 @@ type Consumer struct { } sinksMu sync.Mutex - ddlSink sink.Sink + ddlSink sink.Sink + fakeTableIDGenerator *fakeTableIDGenerator globalResolvedTs uint64 } @@ -303,6 +306,9 @@ func NewConsumer(ctx context.Context) (*Consumer, error) { return nil, errors.Trace(err) } c := new(Consumer) + c.fakeTableIDGenerator = &fakeTableIDGenerator{ + tableIDs: make(map[string]int64), + } c.sinks = make([]*struct { sink.Sink resolvedTs uint64 @@ -399,6 +405,12 @@ ClaimMessages: // FIXME: hack to set start-ts in row changed event, as start-ts // is not contained in TiCDC open protocol row.StartTs = row.CommitTs + var partitionID int64 + if row.Table.IsPartition { + partitionID = row.Table.TableID + } + row.Table.TableID = + c.fakeTableIDGenerator.generateFakeTableID(row.Table.Schema, row.Table.Table, partitionID) err = sink.EmitRowChangedEvents(ctx, row) if err != nil { log.Fatal("emit row changed event failed", zap.Error(err)) @@ -504,7 +516,7 @@ func (c *Consumer) Run(ctx context.Context) error { sink.Sink resolvedTs uint64 }) error { - return sink.FlushRowChangedEvents(ctx, todoDDL.CommitTs) + return syncFlushRowChangedEvents(ctx, sink, todoDDL.CommitTs) }) if err != nil { return errors.Trace(err) @@ -533,10 +545,48 @@ func (c *Consumer) Run(ctx context.Context) error { sink.Sink resolvedTs uint64 }) error { - return sink.FlushRowChangedEvents(ctx, globalResolvedTs) + return syncFlushRowChangedEvents(ctx, sink, globalResolvedTs) }) if err != nil { return errors.Trace(err) } } } + +func syncFlushRowChangedEvents(ctx context.Context, sink sink.Sink, resolvedTs uint64) error { + for { + select { + case <-ctx.Done(): + return ctx.Err() + default: + } + checkpointTs, err := sink.FlushRowChangedEvents(ctx, resolvedTs) + if err != nil { + return err + } + if checkpointTs >= resolvedTs { + return nil + } + } +} + +type fakeTableIDGenerator struct { + tableIDs map[string]int64 + currentTableID int64 + mu sync.Mutex +} + +func (g *fakeTableIDGenerator) generateFakeTableID(schema, table string, partition int64) int64 { + g.mu.Lock() + defer g.mu.Unlock() + key := quotes.QuoteSchema(schema, table) + if partition != 0 { + key = fmt.Sprintf("%s.`%d`", key, partition) + } + if tableID, ok := g.tableIDs[key]; ok { + return tableID + } + g.currentTableID++ + g.tableIDs[key] = g.currentTableID + return g.currentTableID +} diff --git a/metrics/grafana/ticdc.json b/metrics/grafana/ticdc.json index 415a489bbba..b19d9b8bfa1 100644 --- a/metrics/grafana/ticdc.json +++ b/metrics/grafana/ticdc.json @@ -60,7 +60,7 @@ "gnetId": null, "graphTooltip": 1, "id": null, - "iteration": 1596457144174, + "iteration": 1597641819763, "links": [], "panels": [ { @@ -550,43 +550,102 @@ "x": 0, "y": 1 }, - "id": 13, + "id": 11, "panels": [ + { + "columns": [ + { + "text": "Current", + "value": "current" + } + ], + "datasource": "${DS_TEST-CLUSTER}", + "fontSize": "100%", + "gridPos": { + "h": 5, + "w": 4, + "x": 0, + "y": 2 + }, + "id": 4, + "links": [], + "pageSize": null, + "scroll": true, + "showHeader": true, + "sort": { + "col": 0, + "desc": true + }, + "styles": [ + { + "alias": "Time", + "dateFormat": "YYYY-MM-DD HH:mm:ss", + "pattern": "Time", + "type": "date" + }, + { + "alias": "", + "colorMode": null, + "colors": [ + "rgba(245, 54, 54, 0.9)", + "rgba(237, 129, 40, 0.89)", + "rgba(50, 172, 45, 0.97)" + ], + "decimals": 2, + "pattern": "/.*/", + "thresholds": [], + "type": "number", + "unit": "short" + } + ], + "targets": [ + { + "expr": "sum(ticdc_processor_num_of_tables{changefeed=~\"$changefeed\",capture=~\"$capture\"}) by (capture)", + "format": "time_series", + "interval": "", + "intervalFactor": 1, + "legendFormat": "{{capture}}", + "refId": "A" + } + ], + "timeFrom": null, + "timeShift": null, + "title": "changefeed table count", + "transform": "timeseries_aggregations", + "type": "table" + }, { "aliasColors": {}, "bars": false, + "cacheTimeout": null, "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", + "description": "global checkpoint ts of processor", "fill": 1, - "fillGradient": 0, "gridPos": { - "h": 7, - "w": 12, - "x": 0, + "h": 5, + "w": 8, + "x": 4, "y": 2 }, - "hiddenSeries": false, - "id": 15, + "id": 86, "legend": { "alignAsTable": true, "avg": false, "current": true, - "hideEmpty": false, "max": false, "min": false, - "rightSide": true, + "rightSide": false, "show": true, + "sideWidth": null, "total": false, "values": true }, "lines": true, "linewidth": 1, + "links": [], "nullPointMode": "null", - "options": { - "dataLinks": [] - }, - "paceLength": 10, "percentage": false, "pointradius": 2, "points": false, @@ -597,18 +656,28 @@ "steppedLine": false, "targets": [ { - "expr": "ticdc_kvclient_event_feed_count", + "expr": "max(ticdc_processor_checkpoint_ts{changefeed=~\"$changefeed\",capture=~\"$capture\"}) by (changefeed) > 0", "format": "time_series", + "interval": "", "intervalFactor": 1, - "legendFormat": "{{instance}}", + "legendFormat": "{{changefeed}}", "refId": "A" + }, + { + "expr": "max(pd_cluster_tso) * 1000", + "format": "time_series", + "hide": false, + "interval": "", + "intervalFactor": 1, + "legendFormat": "approximate current time (s)", + "refId": "B" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Event Feed Count", + "title": "changefeed checkpoint", "tooltip": { "shared": true, "sort": 0, @@ -617,6 +686,7 @@ "type": "graph", "xaxis": { "buckets": null, + "max": null, "mode": "time", "name": null, "show": true, @@ -624,7 +694,7 @@ }, "yaxes": [ { - "format": "short", + "format": "dateTimeAsIso", "label": null, "logBase": 1, "max": null, @@ -637,7 +707,7 @@ "logBase": 1, "max": null, "min": null, - "show": true + "show": false } ], "yaxis": { @@ -645,29 +715,192 @@ "alignLevel": null } }, + { + "columns": [ + { + "text": "Current", + "value": "current" + } + ], + "datasource": "${DS_TEST-CLUSTER}", + "description": "local resolved ts of processor", + "fontSize": "100%", + "gridPos": { + "h": 5, + "w": 5, + "x": 12, + "y": 2 + }, + "id": 90, + "links": [], + "pageSize": null, + "scroll": true, + "showHeader": true, + "sort": { + "col": 0, + "desc": true + }, + "styles": [ + { + "alias": "table", + "colorMode": null, + "colors": [ + "rgba(245, 54, 54, 0.9)", + "rgba(237, 129, 40, 0.89)", + "rgba(50, 172, 45, 0.97)" + ], + "dateFormat": "YYYY-MM-DD HH:mm:ss", + "decimals": 2, + "mappingType": 1, + "pattern": "Metric", + "thresholds": [], + "type": "string", + "unit": "short" + }, + { + "alias": "resolved ts", + "colorMode": null, + "colors": [ + "rgba(245, 54, 54, 0.9)", + "rgba(237, 129, 40, 0.89)", + "rgba(50, 172, 45, 0.97)" + ], + "dateFormat": "MM-DD HH:mm:ss.SSS", + "decimals": 2, + "pattern": "Current", + "thresholds": [], + "type": "date", + "unit": "short" + } + ], + "targets": [ + { + "expr": "max(ticdc_processor_resolved_ts{changefeed=~\"$changefeed\",capture=~\"$capture\"}) by (capture)", + "format": "time_series", + "interval": "", + "intervalFactor": 1, + "legendFormat": "{{capture}}", + "refId": "A" + }, + { + "expr": "max(ticdc_processor_checkpoint_ts{changefeed=~\"$changefeed\",capture=~\"$capture\"}) by (capture) > 0", + "format": "time_series", + "interval": "", + "intervalFactor": 1, + "legendFormat": "changefeed checkpoint", + "refId": "B" + } + ], + "timeFrom": null, + "timeShift": null, + "title": "processor resolved ts", + "transform": "timeseries_aggregations", + "type": "table" + }, + { + "columns": [ + { + "text": "Current", + "value": "current" + } + ], + "datasource": "${DS_TEST-CLUSTER}", + "description": "local resolved ts of processor", + "fontSize": "100%", + "gridPos": { + "h": 5, + "w": 7, + "x": 17, + "y": 2 + }, + "id": 30, + "links": [], + "pageSize": null, + "scroll": true, + "showHeader": true, + "sort": { + "col": 0, + "desc": true + }, + "styles": [ + { + "alias": "table", + "colorMode": null, + "colors": [ + "rgba(245, 54, 54, 0.9)", + "rgba(237, 129, 40, 0.89)", + "rgba(50, 172, 45, 0.97)" + ], + "dateFormat": "YYYY-MM-DD HH:mm:ss", + "decimals": 2, + "mappingType": 1, + "pattern": "Metric", + "thresholds": [], + "type": "string", + "unit": "short" + }, + { + "alias": "resolved ts", + "colorMode": null, + "colors": [ + "rgba(245, 54, 54, 0.9)", + "rgba(237, 129, 40, 0.89)", + "rgba(50, 172, 45, 0.97)" + ], + "dateFormat": "MM-DD HH:mm:ss.SSS", + "decimals": 2, + "pattern": "Current", + "thresholds": [], + "type": "date", + "unit": "short" + } + ], + "targets": [ + { + "expr": "max(ticdc_processor_table_resolved_ts{changefeed=~\"$changefeed\",capture=~\"$capture\"}) by (capture,table)", + "format": "time_series", + "interval": "", + "intervalFactor": 1, + "legendFormat": "{{capture}}-{{table}}", + "refId": "A" + }, + { + "expr": "max(ticdc_processor_checkpoint_ts{changefeed=~\"$changefeed\",capture=~\"$capture\"}) by (capture) > 0", + "format": "time_series", + "interval": "", + "intervalFactor": 1, + "legendFormat": "changefeed checkpoint", + "refId": "B" + } + ], + "timeFrom": null, + "timeShift": null, + "title": "table resolved ts", + "transform": "timeseries_aggregations", + "type": "table" + }, { "aliasColors": {}, "bars": false, "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", + "description": "global checkpoint ts lag of processor", "fill": 1, "fillGradient": 0, "gridPos": { "h": 7, "w": 12, - "x": 12, - "y": 2 + "x": 0, + "y": 7 }, "hiddenSeries": false, - "id": 28, + "id": 3, "legend": { "alignAsTable": true, "avg": false, "current": true, - "hideEmpty": true, - "hideZero": true, - "max": true, + "max": false, "min": false, "rightSide": true, "show": true, @@ -676,6 +909,7 @@ }, "lines": true, "linewidth": 1, + "links": [], "nullPointMode": "null", "options": { "dataLinks": [] @@ -691,10 +925,11 @@ "steppedLine": false, "targets": [ { - "expr": "sum(rate(ticdc_kvclient_event_feed_error_count[30s])) by (instance, type)", + "expr": "sum(ticdc_processor_checkpoint_ts_lag{changefeed=~\"$changefeed\",capture=~\"$capture\"}) by (capture)", "format": "time_series", + "interval": "", "intervalFactor": 1, - "legendFormat": "{{instance}} - {{type}}", + "legendFormat": "{{capture}}", "refId": "A" } ], @@ -702,7 +937,7 @@ "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Event Feed Error Count", + "title": "changefeed checkpoint lag", "tooltip": { "shared": true, "sort": 0, @@ -718,7 +953,7 @@ }, "yaxes": [ { - "format": "short", + "format": "s", "label": null, "logBase": 1, "max": null, @@ -745,21 +980,23 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "decimals": null, + "description": "local resolved ts lag of processor", "fill": 1, "fillGradient": 0, "gridPos": { "h": 7, "w": 12, - "x": 0, - "y": 9 + "x": 12, + "y": 7 }, "hiddenSeries": false, - "id": 17, + "id": 2, "legend": { "alignAsTable": true, "avg": false, "current": true, + "hideEmpty": false, + "hideZero": false, "max": false, "min": false, "rightSide": true, @@ -769,6 +1006,7 @@ }, "lines": true, "linewidth": 1, + "links": [], "nullPointMode": "null", "options": { "dataLinks": [] @@ -784,10 +1022,11 @@ "steppedLine": false, "targets": [ { - "expr": "histogram_quantile(0.95, sum(rate(ticdc_kvclient_event_size_bytes_bucket[1m])) by (le, instance))", + "expr": "sum(ticdc_processor_resolved_ts_lag{changefeed=~\"$changefeed\",capture=~\"$capture\"}) by (capture)", "format": "time_series", + "interval": "", "intervalFactor": 1, - "legendFormat": "{{instance}} 95", + "legendFormat": "{{capture}}", "refId": "A" } ], @@ -795,7 +1034,7 @@ "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Event Size", + "title": "processor resolved ts lag", "tooltip": { "shared": true, "sort": 0, @@ -811,7 +1050,7 @@ }, "yaxes": [ { - "format": "bytes", + "format": "s", "label": null, "logBase": 1, "max": null, @@ -832,32 +1071,186 @@ "alignLevel": null } }, + { + "cards": { + "cardPadding": 0, + "cardRound": 0 + }, + "color": { + "cardColor": "#FF9830", + "colorScale": "linear", + "colorScheme": "interpolateSpectral", + "exponent": 0.5, + "max": null, + "min": 0, + "mode": "spectrum" + }, + "dataFormat": "tsbuckets", + "datasource": "${DS_TEST-CLUSTER}", + "description": "", + "gridPos": { + "h": 7, + "w": 12, + "x": 0, + "y": 14 + }, + "heatmap": {}, + "hideZeroBuckets": true, + "highlightCards": true, + "id": 93, + "legend": { + "alignAsTable": true, + "avg": false, + "current": true, + "max": true, + "min": false, + "rightSide": true, + "show": true, + "sort": "current", + "sortDesc": true, + "total": false, + "values": true + }, + "links": [], + "reverseYBuckets": false, + "targets": [ + { + "expr": "max(rate(ticdc_processor_flush_event_duration_seconds_bucket{capture=~\"$capture\"}[1m])) by (le)", + "format": "heatmap", + "instant": false, + "intervalFactor": 2, + "legendFormat": "{{le}}", + "refId": "A" + } + ], + "title": "flush sink duration", + "tooltip": { + "show": true, + "showHistogram": true + }, + "tooltipDecimals": 1, + "type": "heatmap", + "xAxis": { + "show": true + }, + "xBucketNumber": null, + "xBucketSize": null, + "yAxis": { + "decimals": 1, + "format": "s", + "logBase": 1, + "max": null, + "min": null, + "show": true, + "splitFactor": null + }, + "yBucketBound": "upper", + "yBucketNumber": null, + "yBucketSize": null + }, + { + "cards": { + "cardPadding": 0, + "cardRound": 0 + }, + "color": { + "cardColor": "#FF9830", + "colorScale": "linear", + "colorScheme": "interpolateSpectral", + "exponent": 0.5, + "min": 0, + "mode": "spectrum" + }, + "dataFormat": "tsbuckets", + "datasource": "${DS_TEST-CLUSTER}", + "description": "", + "gridPos": { + "h": 7, + "w": 12, + "x": 12, + "y": 14 + }, + "heatmap": {}, + "hideZeroBuckets": true, + "highlightCards": true, + "id": 94, + "legend": { + "alignAsTable": true, + "avg": false, + "current": true, + "max": true, + "min": false, + "rightSide": true, + "show": true, + "sort": "current", + "sortDesc": true, + "total": false, + "values": true + }, + "links": [], + "reverseYBuckets": false, + "targets": [ + { + "expr": "max(rate(ticdc_sink_txn_exec_duration_bucket{capture=~\"$capture\"}[1m])) by (le)", + "format": "heatmap", + "instant": false, + "intervalFactor": 2, + "legendFormat": "{{le}}", + "refId": "A" + } + ], + "title": "sink txn execution duration", + "tooltip": { + "show": true, + "showHistogram": true + }, + "tooltipDecimals": 1, + "type": "heatmap", + "xAxis": { + "show": true + }, + "xBucketNumber": null, + "xBucketSize": null, + "yAxis": { + "decimals": 1, + "format": "s", + "logBase": 1, + "max": null, + "min": null, + "show": true, + "splitFactor": null + }, + "yBucketBound": "upper", + "yBucketNumber": null, + "yBucketSize": null + }, { "aliasColors": {}, "bars": false, "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", + "description": "size of the DML bucket", "fill": 1, "fillGradient": 0, "gridPos": { "h": 7, "w": 12, - "x": 12, - "y": 9 + "x": 0, + "y": 21 }, "hiddenSeries": false, - "id": 29, + "id": 95, "legend": { "alignAsTable": true, - "avg": false, + "avg": true, "current": true, - "hideEmpty": true, - "hideZero": true, "max": true, "min": false, "rightSide": true, "show": true, + "sort": "current", + "sortDesc": true, "total": false, "values": true }, @@ -868,32 +1261,77 @@ "options": { "dataLinks": [] }, - "paceLength": 10, "percentage": false, "pointradius": 2, "points": false, "renderer": "flot", "seriesOverrides": [], "spaceLength": 10, - "stack": false, + "stack": true, "steppedLine": false, "targets": [ { - "expr": "sum(rate(ticdc_kvclient_pull_event_count{changefeed=~\"$changefeed\"}[1m])) by (instance, type)", + "expr": "sum(rate(ticdc_sink_bucket_size{changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m])) by (capture,bucket)", "format": "time_series", + "hide": true, + "interval": "", "intervalFactor": 1, - "legendFormat": "{{instance}}-{{type}}", + "legendFormat": "{{capture}}-{{bucket}}", "refId": "A" + }, + { + "expr": "count(rate(ticdc_sink_bucket_size{changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m]) >= 0)", + "format": "time_series", + "hide": true, + "interval": "", + "intervalFactor": 1, + "legendFormat": "total worker", + "refId": "B" + }, + { + "expr": "count(rate(ticdc_sink_bucket_size{changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m]) <= 2)", + "format": "time_series", + "interval": "", + "intervalFactor": 1, + "legendFormat": "0-2 row/s worker", + "refId": "C" + }, + { + "expr": "count(rate(ticdc_sink_bucket_size{changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m]) > 2 and rate(ticdc_sink_bucket_size{changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m]) <= 10)", + "format": "time_series", + "hide": false, + "interval": "", + "intervalFactor": 1, + "legendFormat": "2-10 row/s worker", + "refId": "D" + }, + { + "expr": "count(rate(ticdc_sink_bucket_size{changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m]) > 10 and rate(ticdc_sink_bucket_size{changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m]) <= 100)", + "format": "time_series", + "hide": false, + "interval": "", + "intervalFactor": 1, + "legendFormat": "10-100 row/s worker", + "refId": "E" + }, + { + "expr": "count(rate(ticdc_sink_bucket_size{changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m]) > 100)", + "format": "time_series", + "hide": false, + "interval": "", + "intervalFactor": 1, + "legendFormat": ">100 row/s worker", + "refId": "F" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "EventFeed Receive Event Count By Instance", + "title": "sink mysql worker load", "tooltip": { "shared": true, - "sort": 0, + "sort": 2, "value_type": "individual" }, "type": "graph", @@ -933,26 +1371,27 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", + "description": "size of pending flush rows", "fill": 1, "fillGradient": 0, "gridPos": { "h": 7, "w": 12, - "x": 0, - "y": 16 + "x": 12, + "y": 21 }, "hiddenSeries": false, - "id": 31, + "id": 96, "legend": { "alignAsTable": true, - "avg": false, + "avg": true, "current": true, - "hideEmpty": true, - "hideZero": true, "max": true, "min": false, "rightSide": true, "show": true, + "sort": "current", + "sortDesc": true, "total": false, "values": true }, @@ -963,7 +1402,6 @@ "options": { "dataLinks": [] }, - "paceLength": 10, "percentage": false, "pointradius": 2, "points": false, @@ -974,21 +1412,40 @@ "steppedLine": false, "targets": [ { - "expr": "sum(rate(ticdc_kvclient_send_event_count{changefeed=~\"$changefeed\"}[1m])) by (instance, type)", + "expr": "sum(ticdc_sink_total_rows_count{changefeed=~\"$changefeed\",capture=~\"$capture\"}) by (capture) - sum(ticdc_sink_total_flushed_rows_count{changefeed=~\"$changefeed\",capture=~\"$capture\"}) by (capture)", "format": "time_series", + "interval": "", "intervalFactor": 1, - "legendFormat": "{{instance}}-{{type}}", + "legendFormat": "{{capture}}", "refId": "A" + }, + { + "expr": "sum(ticdc_sink_total_rows_count{changefeed=~\"$changefeed\",capture=~\"$capture\"}) by (capture)", + "format": "time_series", + "hide": true, + "interval": "", + "intervalFactor": 1, + "legendFormat": "{{capture}} - total", + "refId": "B" + }, + { + "expr": "sum(ticdc_sink_total_flushed_rows_count{changefeed=~\"$changefeed\",capture=~\"$capture\"}) by (capture)", + "format": "time_series", + "hide": true, + "interval": "", + "intervalFactor": 1, + "legendFormat": "{{capture}} - flushed", + "refId": "C" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Dispatch Event Count By Instance", + "title": "sink pending flush rows", "tooltip": { "shared": true, - "sort": 0, + "sort": 2, "value_type": "individual" }, "type": "graph", @@ -1028,23 +1485,22 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "Puller event channel size", + "description": "The number of events received from kv client event channel", "fill": 1, "fillGradient": 0, "gridPos": { "h": 7, "w": 12, - "x": 12, - "y": 16 + "x": 0, + "y": 28 }, "hiddenSeries": false, - "id": 40, + "id": 32, "legend": { "alignAsTable": true, "avg": false, "current": true, - "hideEmpty": false, - "max": true, + "max": false, "min": false, "rightSide": true, "show": true, @@ -1069,10 +1525,10 @@ "steppedLine": false, "targets": [ { - "expr": "sum(ticdc_puller_event_chan_size{changefeed=~\"$changefeed\",capture=~\"$capture\"}) by (capture, table)", + "expr": "sum (rate(ticdc_puller_kv_event_count{changefeed=~\"$changefeed\"}[1m])) by (instance, type)", "format": "time_series", "intervalFactor": 1, - "legendFormat": "{{capture}}", + "legendFormat": "{{instance}} - {{type}}", "refId": "A" } ], @@ -1080,7 +1536,7 @@ "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "puller event chan size", + "title": "puller receive kv event from chan", "tooltip": { "shared": true, "sort": 0, @@ -1096,7 +1552,7 @@ }, "yaxes": [ { - "format": "short", + "format": "none", "label": null, "logBase": 1, "max": null, @@ -1123,22 +1579,21 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "Puller entry channel buffer size", + "description": "Bucketed histogram of conflict detect time (s) for single DML statement", "fill": 1, "fillGradient": 0, "gridPos": { "h": 7, "w": 12, - "x": 0, - "y": 23 + "x": 12, + "y": 28 }, "hiddenSeries": false, - "id": 43, + "id": 83, "legend": { "alignAsTable": true, "avg": false, "current": true, - "hideEmpty": false, "max": true, "min": false, "rightSide": true, @@ -1153,7 +1608,6 @@ "options": { "dataLinks": [] }, - "paceLength": 10, "percentage": false, "pointradius": 2, "points": false, @@ -1164,21 +1618,30 @@ "steppedLine": false, "targets": [ { - "expr": "sum(ticdc_puller_output_chan_size{changefeed=~\"$changefeed\",capture=~\"$capture\"}) by (capture, table)", + "expr": "histogram_quantile(0.95,sum(rate(ticdc_sink_conflict_detect_duration_bucket{changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m])) by (le,instance))", "format": "time_series", + "interval": "", "intervalFactor": 1, - "legendFormat": "{{capture}} - {{table}}", + "legendFormat": "{{instance}}-95", "refId": "A" + }, + { + "expr": "histogram_quantile(0.99,sum(rate(ticdc_sink_conflict_detect_duration_bucket{changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m])) by (le,instance))", + "format": "time_series", + "interval": "", + "intervalFactor": 1, + "legendFormat": "{{instance}}-99", + "refId": "B" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "puller output chan size", + "title": "sink mysql conflict detect duration", "tooltip": { "shared": true, - "sort": 0, + "sort": 2, "value_type": "individual" }, "type": "graph", @@ -1191,9 +1654,9 @@ }, "yaxes": [ { - "format": "short", + "format": "s", "label": null, - "logBase": 1, + "logBase": 2, "max": null, "min": null, "show": true @@ -1218,23 +1681,22 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "Puller in memory buffer size", + "description": "The number of events received from kv client event channel", "fill": 1, "fillGradient": 0, "gridPos": { "h": 7, "w": 12, - "x": 12, - "y": 23 + "x": 0, + "y": 35 }, "hiddenSeries": false, - "id": 50, + "id": 35, "legend": { "alignAsTable": true, "avg": false, "current": true, - "hideEmpty": false, - "max": true, + "max": false, "min": false, "rightSide": true, "show": true, @@ -1259,18 +1721,32 @@ "steppedLine": false, "targets": [ { - "expr": "sum(ticdc_puller_mem_buffer_size{changefeed=~\"$changefeed\",capture=~\"$capture\"}) by (capture, table)", + "expr": "histogram_quantile(0.95, sum(rate(ticdc_sink_txn_exec_duration_bucket{changefeed=~\"$changefeed\"}[1m])) by (le,instance))", "format": "time_series", "intervalFactor": 1, - "legendFormat": "{{capture}} - {{table}}", + "legendFormat": "{{instance}}-p95", "refId": "A" + }, + { + "expr": "histogram_quantile(0.99, sum(rate(ticdc_sink_txn_exec_duration_bucket{changefeed=~\"$changefeed\"}[1m])) by (le,instance))", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "{{instance}}-p99", + "refId": "B" + }, + { + "expr": "histogram_quantile(0.999, sum(rate(ticdc_sink_txn_exec_duration_bucket{changefeed=~\"$changefeed\"}[1m])) by (le,instance))", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "{{instance}}-p999", + "refId": "C" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "puller mem buffer size", + "title": "exec txn duration", "tooltip": { "shared": true, "sort": 0, @@ -1286,9 +1762,9 @@ }, "yaxes": [ { - "format": "short", + "format": "s", "label": null, - "logBase": 1, + "logBase": 2, "max": null, "min": null, "show": true @@ -1313,23 +1789,22 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "forwarded resolved ts in a single puller", + "description": "The number of events received from txn collector", "fill": 1, "fillGradient": 0, "gridPos": { "h": 7, "w": 12, - "x": 0, - "y": 30 + "x": 12, + "y": 35 }, "hiddenSeries": false, - "id": 52, + "id": 5, "legend": { "alignAsTable": true, "avg": false, "current": true, - "hideEmpty": false, - "max": true, + "max": false, "min": false, "rightSide": true, "show": true, @@ -1354,10 +1829,10 @@ "steppedLine": false, "targets": [ { - "expr": "sum(ticdc_puller_resolved_ts{changefeed=~\"$changefeed\",capture=~\"$capture\"}) by (capture, table)", + "expr": "sum (rate(ticdc_puller_txn_collect_event_count{changefeed=~\"$changefeed\"}[1m])) by (instance, type)", "format": "time_series", "intervalFactor": 1, - "legendFormat": "{{capture}}-{{table}}", + "legendFormat": "{{instance}} - {{type}}", "refId": "A" } ], @@ -1365,7 +1840,7 @@ "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "puller resolved ts", + "title": "Event Count Received in Txn Collector ", "tooltip": { "shared": true, "sort": 0, @@ -1381,7 +1856,7 @@ }, "yaxes": [ { - "format": "dateTimeAsIso", + "format": "none", "label": null, "logBase": 1, "max": null, @@ -1408,26 +1883,27 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "Puller entry sorter unsoreted items size", - "fill": 1, + "description": "input chan size of table mounter", + "fill": 0, "fillGradient": 0, "gridPos": { "h": 7, "w": 12, - "x": 12, - "y": 30 + "x": 0, + "y": 42 }, "hiddenSeries": false, - "id": 51, + "id": 37, "legend": { "alignAsTable": true, "avg": false, "current": true, - "hideEmpty": false, "max": true, "min": false, "rightSide": true, "show": true, + "sort": "current", + "sortDesc": true, "total": false, "values": true }, @@ -1449,10 +1925,10 @@ "steppedLine": false, "targets": [ { - "expr": "sum(ticdc_puller_entry_sorter_unsorted_size{changefeed=~\"$changefeed\",capture=~\"$capture\"}) by (capture, table)", + "expr": "sum(ticdc_mounter_input_chan_size{changefeed=~\"$changefeed\",capture=~\"$capture\"}) by (capture)", "format": "time_series", "intervalFactor": 1, - "legendFormat": "{{capture}}-{{table}}", + "legendFormat": "{{capture}}", "refId": "A" } ], @@ -1460,10 +1936,10 @@ "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "entry sorter unsorted size", + "title": "mounter input chan size", "tooltip": { "shared": true, - "sort": 0, + "sort": 2, "value_type": "individual" }, "type": "graph", @@ -1476,7 +1952,7 @@ }, "yaxes": [ { - "format": "short", + "format": "none", "label": null, "logBase": 1, "max": null, @@ -1503,23 +1979,20 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "Puller entry sorter resolved channel size", + "description": "counter for processor exits with error", "fill": 1, - "fillGradient": 0, "gridPos": { "h": 7, "w": 12, - "x": 0, - "y": 37 + "x": 12, + "y": 42 }, - "hiddenSeries": false, - "id": 42, + "id": 82, "legend": { "alignAsTable": true, "avg": false, "current": true, - "hideEmpty": false, - "max": true, + "max": false, "min": false, "rightSide": true, "show": true, @@ -1530,10 +2003,6 @@ "linewidth": 1, "links": [], "nullPointMode": "null", - "options": { - "dataLinks": [] - }, - "paceLength": 10, "percentage": false, "pointradius": 2, "points": false, @@ -1544,10 +2013,10 @@ "steppedLine": false, "targets": [ { - "expr": "sum(ticdc_puller_entry_sorter_resolved_chan_size{changefeed=~\"$changefeed\",capture=~\"$capture\"}) by (capture, table)", + "expr": "sum(changes(ticdc_processor_exit_with_error_count{changefeed=~\"$changefeed\",capture=~\"$capture\"}[30m])) by (capture)", "format": "time_series", "intervalFactor": 1, - "legendFormat": "{{capture}}-{{table}}", + "legendFormat": "{{capture}}", "refId": "A" } ], @@ -1555,7 +2024,7 @@ "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "entry sorter resolved chan size", + "title": "exit error count", "tooltip": { "shared": true, "sort": 0, @@ -1598,26 +2067,27 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "Puller entry sorter output channel size", - "fill": 1, + "description": "Bucketed histogram of processing time (s) of unmarshal and mount in mounter.", + "fill": 0, "fillGradient": 0, "gridPos": { "h": 7, "w": 12, - "x": 12, - "y": 37 + "x": 0, + "y": 49 }, "hiddenSeries": false, - "id": 41, + "id": 55, "legend": { "alignAsTable": true, "avg": false, "current": true, - "hideEmpty": false, "max": true, "min": false, "rightSide": true, "show": true, + "sort": "current", + "sortDesc": true, "total": false, "values": true }, @@ -1639,21 +2109,37 @@ "steppedLine": false, "targets": [ { - "expr": "sum(ticdc_puller_entry_sorter_output_chan_size{changefeed=~\"$changefeed\",capture=~\"$capture\"}) by (capture, table)", + "expr": "histogram_quantile(0.99, sum(rate(ticdc_mounter_unmarshal_and_mount_bucket{changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m])) by (le, capture))", "format": "time_series", "intervalFactor": 1, - "legendFormat": "{{capture}}-{{table}}", + "legendFormat": "{{capture}}-p99", "refId": "A" + }, + { + "expr": "histogram_quantile(0.999, sum(rate(ticdc_mounter_unmarshal_and_mount_bucket{changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m])) by (le, capture))", + "format": "time_series", + "hide": true, + "instant": false, + "intervalFactor": 1, + "legendFormat": "{{capture}}-p999", + "refId": "B" + }, + { + "expr": "rate(ticdc_mounter_unmarshal_and_mount_sum{changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m]) / rate(ticdc_mounter_unmarshal_and_mount_count{changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m])", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "{{capture}}-avg", + "refId": "C" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "entry sorter output chan size", + "title": "mounter unmarshal and mount", "tooltip": { "shared": true, - "sort": 0, + "sort": 2, "value_type": "individual" }, "type": "graph", @@ -1666,7 +2152,7 @@ }, "yaxes": [ { - "format": "short", + "format": "s", "label": null, "logBase": 1, "max": null, @@ -1674,7 +2160,7 @@ "show": true }, { - "format": "short", + "format": "none", "label": null, "logBase": 1, "max": null, @@ -1693,22 +2179,21 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "Bucketed histogram of processing time (s) of merge in entry sorter", + "description": "DML count that executed to downstream", "fill": 1, "fillGradient": 0, "gridPos": { "h": 7, "w": 12, - "x": 0, - "y": 44 + "x": 12, + "y": 49 }, "hiddenSeries": false, - "id": 53, + "id": 34, "legend": { "alignAsTable": true, - "avg": false, + "avg": true, "current": true, - "hideEmpty": false, "max": true, "min": false, "rightSide": true, @@ -1734,18 +2219,18 @@ "steppedLine": false, "targets": [ { - "expr": "histogram_quantile(0.99, sum(rate(ticdc_puller_entry_sorter_sort_bucket{changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m])) by (le,table))", + "expr": "sum (rate(ticdc_sink_txn_batch_size_sum{changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m])) by (capture)", "format": "time_series", "intervalFactor": 1, - "legendFormat": "{{table}}-p99", + "legendFormat": "{{capture}}", "refId": "A" }, { - "expr": "histogram_quantile(0.90, sum(rate(ticdc_puller_entry_sorter_sort_bucket{changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m])) by (le,table))", + "expr": "sum (rate(ticdc_sink_txn_batch_size_sum{changefeed=~\"$changefeed\"}[1m])) by (changefeed)", "format": "time_series", - "hide": true, + "interval": "", "intervalFactor": 1, - "legendFormat": "{{table}}-p90", + "legendFormat": "total", "refId": "B" } ], @@ -1753,7 +2238,7 @@ "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "entry sorter sort duration", + "title": "sink exec DML count", "tooltip": { "shared": true, "sort": 0, @@ -1769,9 +2254,9 @@ }, "yaxes": [ { - "format": "short", + "format": "none", "label": null, - "logBase": 2, + "logBase": 1, "max": null, "min": null, "show": true @@ -1796,26 +2281,27 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "Bucketed histogram of processing time (s) of merge in entry sorter.", - "fill": 1, + "description": "checkpoint channel size for mq sink", + "fill": 0, "fillGradient": 0, "gridPos": { "h": 7, "w": 12, - "x": 12, - "y": 44 + "x": 0, + "y": 56 }, "hiddenSeries": false, - "id": 54, + "id": 48, "legend": { "alignAsTable": true, "avg": false, "current": true, - "hideEmpty": false, "max": true, "min": false, "rightSide": true, "show": true, + "sort": "current", + "sortDesc": true, "total": false, "values": true }, @@ -1837,30 +2323,21 @@ "steppedLine": false, "targets": [ { - "expr": "histogram_quantile(0.99, sum(rate(ticdc_puller_entry_sorter_merge_bucket{changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m])) by (le,table))", + "expr": "sum(ticdc_sink_mq_checkpoint_chan_size{changefeed=~\"$changefeed\",capture=~\"$capture\"}) by (capture)", "format": "time_series", - "hide": false, "intervalFactor": 1, - "legendFormat": "{{table}}-p99", + "legendFormat": "{{capture}}", "refId": "A" - }, - { - "expr": "histogram_quantile(0.90, sum(rate(ticdc_puller_entry_sorter_merge_bucket{changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m])) by (le,table))", - "format": "time_series", - "hide": true, - "intervalFactor": 1, - "legendFormat": "{{table}}-p90", - "refId": "B" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "entry sorter merge duration", + "title": "sink mq checkpoint chan size", "tooltip": { "shared": true, - "sort": 0, + "sort": 2, "value_type": "individual" }, "type": "graph", @@ -1873,9 +2350,9 @@ }, "yaxes": [ { - "format": "short", + "format": "none", "label": null, - "logBase": 2, + "logBase": 1, "max": null, "min": null, "show": true @@ -1893,44 +2370,28 @@ "align": false, "alignLevel": null } - } - ], - "title": "Events", - "type": "row" - }, - { - "collapsed": true, - "gridPos": { - "h": 1, - "w": 24, - "x": 0, - "y": 2 - }, - "id": 11, - "panels": [ + }, { "aliasColors": {}, "bars": false, "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "local resolved ts lag of processor", + "description": "The number of events received from kv client event channel", "fill": 1, "fillGradient": 0, "gridPos": { "h": 7, "w": 12, - "x": 0, - "y": 3 + "x": 12, + "y": 56 }, "hiddenSeries": false, - "id": 2, + "id": 36, "legend": { "alignAsTable": true, "avg": false, "current": true, - "hideEmpty": false, - "hideZero": false, "max": false, "min": false, "rightSide": true, @@ -1956,114 +2417,33 @@ "steppedLine": false, "targets": [ { - "expr": "sum(ticdc_processor_resolved_ts_lag{changefeed=~\"$changefeed\",capture=~\"$capture\"}) by (capture)", + "expr": "histogram_quantile(0.90, sum(rate(ticdc_sink_txn_batch_size_bucket{changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m])) by (le,capture))", "format": "time_series", - "interval": "", "intervalFactor": 1, - "legendFormat": "{{capture}}", + "legendFormat": "{{capture}}-p90", "refId": "A" - } - ], - "thresholds": [], - "timeFrom": null, - "timeRegions": [], - "timeShift": null, - "title": "processor resolved ts lag", - "tooltip": { - "shared": true, - "sort": 0, - "value_type": "individual" - }, - "type": "graph", - "xaxis": { - "buckets": null, - "mode": "time", - "name": null, - "show": true, - "values": [] - }, - "yaxes": [ - { - "format": "s", - "label": null, - "logBase": 1, - "max": null, - "min": null, - "show": true }, { - "format": "short", - "label": null, - "logBase": 1, - "max": null, - "min": null, - "show": true - } - ], - "yaxis": { - "align": false, - "alignLevel": null - } - }, - { - "aliasColors": {}, - "bars": false, - "dashLength": 10, - "dashes": false, - "datasource": "${DS_TEST-CLUSTER}", - "description": "global checkpoint ts lag of processor", - "fill": 1, - "fillGradient": 0, - "gridPos": { - "h": 7, - "w": 12, - "x": 12, - "y": 3 - }, - "hiddenSeries": false, - "id": 3, - "legend": { - "alignAsTable": true, - "avg": false, - "current": true, - "max": false, - "min": false, - "rightSide": true, - "show": true, - "total": false, - "values": true - }, - "lines": true, - "linewidth": 1, - "links": [], - "nullPointMode": "null", - "options": { - "dataLinks": [] - }, - "paceLength": 10, - "percentage": false, - "pointradius": 2, - "points": false, - "renderer": "flot", - "seriesOverrides": [], - "spaceLength": 10, - "stack": false, - "steppedLine": false, - "targets": [ + "expr": "histogram_quantile(0.99, sum(rate(ticdc_sink_txn_batch_size_bucket{changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m])) by (le,capture))", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "{{capture}}-p99", + "refId": "B" + }, { - "expr": "sum(ticdc_processor_checkpoint_ts_lag{changefeed=~\"$changefeed\",capture=~\"$capture\"}) by (capture)", + "expr": "histogram_quantile(0.999, sum(rate(ticdc_sink_txn_batch_size_bucket{changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m])) by (le,capture))", "format": "time_series", - "interval": "", + "hide": true, "intervalFactor": 1, - "legendFormat": "{{capture}}", - "refId": "A" + "legendFormat": "{{capture}}-p999", + "refId": "C" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "changefeed checkpoint lag", + "title": "exec txn batch", "tooltip": { "shared": true, "sort": 0, @@ -2079,9 +2459,9 @@ }, "yaxes": [ { - "format": "s", + "format": "none", "label": null, - "logBase": 1, + "logBase": 2, "max": null, "min": null, "show": true @@ -2106,27 +2486,27 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "", - "fill": 1, + "description": "Bucketed histogram of processing time (s) of waiting event prepare in processor.", + "fill": 0, "fillGradient": 0, "gridPos": { "h": 7, "w": 12, "x": 0, - "y": 10 + "y": 63 }, "hiddenSeries": false, - "id": 85, + "id": 56, "legend": { "alignAsTable": true, "avg": false, "current": true, - "hideEmpty": false, - "hideZero": false, - "max": false, + "max": true, "min": false, "rightSide": true, "show": true, + "sort": "current", + "sortDesc": true, "total": false, "values": true }, @@ -2148,22 +2528,28 @@ "steppedLine": false, "targets": [ { - "expr": "sum(ticdc_processor_resolved_ts{changefeed=~\"$changefeed\",capture=~\"$capture\"}) by (capture)", + "expr": "histogram_quantile(0.999, sum(rate(ticdc_processor_wait_event_prepare_bucket{changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m])) by (le, capture))", "format": "time_series", - "interval": "", "intervalFactor": 1, - "legendFormat": "{{capture}}", + "legendFormat": "{{capture}}-p999", "refId": "A" + }, + { + "expr": "rate(ticdc_processor_wait_event_prepare_sum{changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m]) / rate(ticdc_processor_wait_event_prepare_count{changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m])", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "{{capture}}-avg", + "refId": "B" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "processor resolved ts", + "title": "processor wait event prepare", "tooltip": { "shared": true, - "sort": 0, + "sort": 2, "value_type": "individual" }, "type": "graph", @@ -2176,7 +2562,7 @@ }, "yaxes": [ { - "format": "dateTimeAsIso", + "format": "s", "label": null, "logBase": 1, "max": null, @@ -2203,25 +2589,27 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "global checkpoint ts of processor", - "fill": 1, + "description": "size of row changed event output channel from table to processor", + "fill": 0, "fillGradient": 0, "gridPos": { "h": 7, "w": 12, "x": 12, - "y": 10 + "y": 63 }, "hiddenSeries": false, - "id": 86, + "id": 38, "legend": { "alignAsTable": true, "avg": false, "current": true, - "max": false, + "max": true, "min": false, "rightSide": true, "show": true, + "sort": "current", + "sortDesc": true, "total": false, "values": true }, @@ -2243,28 +2631,21 @@ "steppedLine": false, "targets": [ { - "expr": "sum(ticdc_processor_checkpoint_ts{changefeed=~\"$changefeed\",capture=~\"$capture\"}) by (capture)", + "expr": "sum(ticdc_processor_txn_output_chan_size{changefeed=~\"$changefeed\",capture=~\"$capture\"}) by (capture)", "format": "time_series", - "interval": "", "intervalFactor": 1, "legendFormat": "{{capture}}", "refId": "A" - }, - { - "expr": "time()*1000", - "interval": "", - "legendFormat": "", - "refId": "B" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "changefeed checkpoint", + "title": "processor table output chan size", "tooltip": { "shared": true, - "sort": 0, + "sort": 2, "value_type": "individual" }, "type": "graph", @@ -2277,7 +2658,7 @@ }, "yaxes": [ { - "format": "dateTimeAsIso", + "format": "none", "label": null, "logBase": 1, "max": null, @@ -2297,31 +2678,47 @@ "align": false, "alignLevel": null } - }, + } + ], + "title": "Changefeed", + "type": "row" + }, + { + "collapsed": true, + "gridPos": { + "h": 1, + "w": 24, + "x": 0, + "y": 2 + }, + "id": 58, + "panels": [ { "aliasColors": {}, "bars": false, "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", + "decimals": 1, "fill": 1, - "fillGradient": 0, "gridPos": { - "h": 7, + "h": 8, "w": 12, "x": 0, - "y": 17 + "y": 71 }, - "hiddenSeries": false, - "id": 4, + "id": 60, "legend": { "alignAsTable": true, "avg": false, "current": true, - "max": false, + "max": true, "min": false, "rightSide": true, "show": true, + "sideWidth": null, + "sort": "current", + "sortDesc": true, "total": false, "values": true }, @@ -2329,12 +2726,9 @@ "linewidth": 1, "links": [], "nullPointMode": "null", - "options": { - "dataLinks": [] - }, "paceLength": 10, "percentage": false, - "pointradius": 2, + "pointradius": 5, "points": false, "renderer": "flot", "seriesOverrides": [], @@ -2343,19 +2737,19 @@ "steppedLine": false, "targets": [ { - "expr": "sum(ticdc_processor_num_of_tables{changefeed=~\"$changefeed\",capture=~\"$capture\"}) by (capture)", + "expr": "sum(rate(tikv_thread_cpu_seconds_total{instance=~\"$tikv_instance\", name=~\"cdc\"}[1m])) by (instance)", "format": "time_series", - "interval": "", - "intervalFactor": 1, - "legendFormat": "{{capture}}", - "refId": "A" + "intervalFactor": 2, + "legendFormat": "{{instance}}", + "refId": "A", + "step": 4 } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "change feed table count", + "title": "CDC endpoint CPU", "tooltip": { "shared": true, "sort": 0, @@ -2371,7 +2765,7 @@ }, "yaxes": [ { - "format": "none", + "format": "percentunit", "label": null, "logBase": 1, "max": null, @@ -2398,27 +2792,26 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "local resolved ts of processor", + "decimals": 1, "fill": 1, - "fillGradient": 0, "gridPos": { - "h": 7, + "h": 8, "w": 12, "x": 12, - "y": 17 + "y": 71 }, - "hiddenSeries": false, - "id": 30, + "id": 62, "legend": { "alignAsTable": true, "avg": false, "current": true, - "hideEmpty": false, - "hideZero": false, - "max": false, + "max": true, "min": false, "rightSide": true, "show": true, + "sideWidth": null, + "sort": "current", + "sortDesc": true, "total": false, "values": true }, @@ -2426,12 +2819,9 @@ "linewidth": 1, "links": [], "nullPointMode": "null", - "options": { - "dataLinks": [] - }, "paceLength": 10, "percentage": false, - "pointradius": 2, + "pointradius": 5, "points": false, "renderer": "flot", "seriesOverrides": [], @@ -2440,19 +2830,19 @@ "steppedLine": false, "targets": [ { - "expr": "sum(ticdc_processor_table_resolved_ts{changefeed=~\"$changefeed\",capture=~\"$capture\"}) by (capture,table)", + "expr": "sum(rate(tikv_thread_cpu_seconds_total{instance=~\"$tikv_instance\", name=~\"cdcwkr.*\"}[1m])) by (instance)", "format": "time_series", - "interval": "", - "intervalFactor": 1, - "legendFormat": "{{capture}}-{{table}}", - "refId": "A" + "intervalFactor": 2, + "legendFormat": "{{instance}}", + "refId": "A", + "step": 4 } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "table resolved ts", + "title": "CDC worker CPU", "tooltip": { "shared": true, "sort": 0, @@ -2468,7 +2858,7 @@ }, "yaxes": [ { - "format": "dateTimeAsIso", + "format": "percentunit", "label": null, "logBase": 1, "max": null, @@ -2490,482 +2880,217 @@ } }, { - "aliasColors": {}, - "bars": false, - "dashLength": 10, - "dashes": false, - "datasource": "${DS_TEST-CLUSTER}", - "description": "The number of events received from kv client event channel", - "fill": 1, - "fillGradient": 0, - "gridPos": { - "h": 7, - "w": 12, - "x": 0, - "y": 24 - }, - "hiddenSeries": false, - "id": 32, - "legend": { - "alignAsTable": true, - "avg": false, - "current": true, - "max": false, - "min": false, - "rightSide": true, - "show": true, - "total": false, - "values": true - }, - "lines": true, - "linewidth": 1, - "links": [], - "nullPointMode": "null", - "options": { - "dataLinks": [] - }, - "paceLength": 10, - "percentage": false, - "pointradius": 2, - "points": false, - "renderer": "flot", - "seriesOverrides": [], - "spaceLength": 10, - "stack": false, - "steppedLine": false, - "targets": [ - { - "expr": "sum (rate(ticdc_puller_kv_event_count{changefeed=~\"$changefeed\"}[1m])) by (instance, type)", - "format": "time_series", - "intervalFactor": 1, - "legendFormat": "{{instance}} - {{type}}", - "refId": "A" - } - ], - "thresholds": [], - "timeFrom": null, - "timeRegions": [], - "timeShift": null, - "title": "puller receive kv event from chan", - "tooltip": { - "shared": true, - "sort": 0, - "value_type": "individual" - }, - "type": "graph", - "xaxis": { - "buckets": null, - "mode": "time", - "name": null, - "show": true, - "values": [] - }, - "yaxes": [ - { - "format": "none", - "label": null, - "logBase": 1, - "max": null, - "min": null, - "show": true - }, - { - "format": "short", - "label": null, - "logBase": 1, - "max": null, - "min": null, - "show": true - } - ], - "yaxis": { - "align": false, - "alignLevel": null - } - }, - { - "aliasColors": {}, - "bars": false, - "dashLength": 10, - "dashes": false, - "datasource": "${DS_TEST-CLUSTER}", - "description": "The number of events received from txn collector", - "fill": 1, - "fillGradient": 0, - "gridPos": { - "h": 7, - "w": 12, - "x": 12, - "y": 24 - }, - "hiddenSeries": false, - "id": 5, - "legend": { - "alignAsTable": true, - "avg": false, - "current": true, - "max": false, - "min": false, - "rightSide": true, - "show": true, - "total": false, - "values": true - }, - "lines": true, - "linewidth": 1, - "links": [], - "nullPointMode": "null", - "options": { - "dataLinks": [] - }, - "paceLength": 10, - "percentage": false, - "pointradius": 2, - "points": false, - "renderer": "flot", - "seriesOverrides": [], - "spaceLength": 10, - "stack": false, - "steppedLine": false, - "targets": [ - { - "expr": "sum (rate(ticdc_puller_txn_collect_event_count{changefeed=~\"$changefeed\"}[1m])) by (instance, type)", - "format": "time_series", - "intervalFactor": 1, - "legendFormat": "{{instance}} - {{type}}", - "refId": "A" - } - ], - "thresholds": [], - "timeFrom": null, - "timeRegions": [], - "timeShift": null, - "title": "Event Count Received in Txn Collector ", - "tooltip": { - "shared": true, - "sort": 0, - "value_type": "individual" - }, - "type": "graph", - "xaxis": { - "buckets": null, - "mode": "time", - "name": null, - "show": true, - "values": [] - }, - "yaxes": [ + "columns": [ { - "format": "none", - "label": null, - "logBase": 1, - "max": null, - "min": null, - "show": true + "text": "Current", + "value": "current" }, { - "format": "short", - "label": null, - "logBase": 1, - "max": null, - "min": null, - "show": true + "text": "Max", + "value": "max" } ], - "yaxis": { - "align": false, - "alignLevel": null - } - }, - { - "aliasColors": {}, - "bars": false, - "dashLength": 10, - "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "local resolved ts of processor", - "fill": 1, - "fillGradient": 0, + "fontSize": "90%", "gridPos": { - "h": 7, - "w": 12, + "h": 8, + "w": 7, "x": 0, - "y": 31 + "y": 79 }, - "hiddenSeries": false, - "id": 33, - "legend": { - "alignAsTable": true, - "avg": false, - "current": true, - "max": false, - "min": false, - "rightSide": true, - "show": true, - "total": false, - "values": true - }, - "lines": true, - "linewidth": 1, + "hideTimeOverride": true, + "id": 64, "links": [], - "nullPointMode": "null", - "options": { - "dataLinks": [] + "pageSize": null, + "scroll": true, + "showHeader": true, + "sort": { + "col": 2, + "desc": false }, - "paceLength": 10, - "percentage": false, - "pointradius": 2, - "points": false, - "renderer": "flot", - "seriesOverrides": [], - "spaceLength": 10, - "stack": false, - "steppedLine": false, - "targets": [ + "styles": [ { - "expr": "sum (ticdc_processor_table_resolved_ts{changefeed=~\"$changefeed\",capture=~\"$capture\"}) by (capture, table)", - "format": "time_series", - "intervalFactor": 1, - "legendFormat": "{{capture}} - {{table}}", - "refId": "A" - } - ], - "thresholds": [], - "timeFrom": null, - "timeRegions": [], - "timeShift": null, - "title": "processor table resolved ts", - "tooltip": { - "shared": true, - "sort": 0, - "value_type": "individual" - }, - "type": "graph", - "xaxis": { - "buckets": null, - "mode": "time", - "name": null, - "show": true, - "values": [] - }, - "yaxes": [ + "alias": "Instance", + "colorMode": null, + "colors": [ + "rgba(245, 54, 54, 0.9)", + "rgba(237, 129, 40, 0.89)", + "rgba(50, 172, 45, 0.97)" + ], + "dateFormat": "YYYY-MM-DD HH:mm:ss", + "decimals": 2, + "mappingType": 1, + "pattern": "Metric", + "thresholds": [], + "type": "string", + "unit": "short", + "valueMaps": [] + }, { - "format": "dateTimeAsIso", - "label": null, - "logBase": 1, - "max": null, - "min": null, - "show": true + "alias": "Time", + "colorMode": null, + "colors": [ + "rgba(245, 54, 54, 0.9)", + "rgba(237, 129, 40, 0.89)", + "rgba(50, 172, 45, 0.97)" + ], + "dateFormat": "MM-DD HH:mm:ss.SSS", + "decimals": 2, + "mappingType": 1, + "pattern": "Max", + "thresholds": [], + "type": "date", + "unit": "short" + } + ], + "targets": [ + { + "expr": "tikv_cdc_min_resolved_ts{instance=~\"$tikv_instance\"}", + "format": "time_series", + "intervalFactor": 2, + "legendFormat": "{{instance}}", + "refId": "A" }, { - "format": "short", - "label": null, - "logBase": 1, - "max": null, - "min": null, - "show": true + "expr": "max(pd_cluster_tso) * 1000", + "format": "time_series", + "hide": true, + "intervalFactor": 2, + "legendFormat": "current time (s)", + "refId": "B" } ], - "yaxis": { - "align": false, - "alignLevel": null - } + "timeFrom": "1s", + "title": "Min resolved ts", + "transform": "timeseries_aggregations", + "type": "table" }, { - "aliasColors": {}, - "bars": false, - "dashLength": 10, - "dashes": false, + "columns": [ + { + "text": "Current", + "value": "current" + } + ], "datasource": "${DS_TEST-CLUSTER}", - "description": "counter for processor exits with error", - "fill": 1, + "fontSize": "90%", "gridPos": { - "h": 7, - "w": 12, - "x": 12, - "y": 31 - }, - "id": 82, - "legend": { - "alignAsTable": true, - "avg": false, - "current": true, - "max": false, - "min": false, - "rightSide": true, - "show": true, - "total": false, - "values": true + "h": 8, + "w": 5, + "x": 7, + "y": 79 }, - "lines": true, - "linewidth": 1, + "hideTimeOverride": true, + "id": 66, "links": [], - "nullPointMode": "null", - "percentage": false, - "pointradius": 2, - "points": false, - "renderer": "flot", - "seriesOverrides": [], - "spaceLength": 10, - "stack": false, - "steppedLine": false, - "targets": [ + "pageSize": null, + "scroll": true, + "showHeader": true, + "sort": { + "col": 0, + "desc": false + }, + "styles": [ { - "expr": "sum(changes(ticdc_processor_exit_with_error_count{changefeed=~\"$changefeed\",capture=~\"$capture\"}[30m])) by (capture)", - "format": "time_series", - "intervalFactor": 1, - "legendFormat": "{{capture}}", - "refId": "A" + "alias": "Option", + "colorMode": null, + "dateFormat": "YYYY-MM-DD HH:mm:ss", + "mappingType": 2, + "pattern": "Metric", + "preserveFormat": false, + "sanitize": false, + "type": "number" } ], - "thresholds": [], - "timeFrom": null, - "timeRegions": [], - "timeShift": null, - "title": "exit error count", - "tooltip": { - "shared": true, - "sort": 0, - "value_type": "individual" - }, - "type": "graph", - "xaxis": { - "buckets": null, - "mode": "time", - "name": null, - "show": true, - "values": [] - }, - "yaxes": [ - { - "format": "short", - "label": null, - "logBase": 1, - "max": null, - "min": null, - "show": true - }, + "targets": [ { - "format": "short", - "label": null, - "logBase": 1, - "max": null, - "min": null, - "show": true + "expr": "tikv_cdc_min_resolved_ts_region{instance=~\"$tikv_instance\"}", + "format": "time_series", + "intervalFactor": 2, + "legendFormat": "{{instance}}", + "refId": "A" } ], - "yaxis": { - "align": false, - "alignLevel": null - } + "timeFrom": "1s", + "title": "Min resolved Region", + "transform": "timeseries_aggregations", + "type": "table" }, { - "aliasColors": {}, - "bars": false, - "dashLength": 10, - "dashes": false, + "cards": { + "cardPadding": 0, + "cardRound": 0 + }, + "color": { + "cardColor": "#FF9830", + "colorScale": "linear", + "colorScheme": "interpolateSpectral", + "exponent": 0.5, + "min": 0, + "mode": "spectrum" + }, + "dataFormat": "tsbuckets", "datasource": "${DS_TEST-CLUSTER}", - "description": "The number of events received from kv client event channel", - "fill": 1, - "fillGradient": 0, + "description": "The time consumed to CDC incremental scan", "gridPos": { - "h": 7, + "h": 8, "w": 12, - "x": 0, - "y": 38 + "x": 12, + "y": 79 }, - "hiddenSeries": false, - "id": 35, + "heatmap": {}, + "hideZeroBuckets": true, + "highlightCards": true, + "id": 68, "legend": { "alignAsTable": true, "avg": false, "current": true, - "max": false, + "max": true, "min": false, "rightSide": true, "show": true, + "sort": "current", + "sortDesc": true, "total": false, "values": true }, - "lines": true, - "linewidth": 1, "links": [], - "nullPointMode": "null", - "options": { - "dataLinks": [] - }, - "paceLength": 10, - "percentage": false, - "pointradius": 2, - "points": false, - "renderer": "flot", - "seriesOverrides": [], - "spaceLength": 10, - "stack": false, - "steppedLine": false, + "reverseYBuckets": false, "targets": [ { - "expr": "histogram_quantile(0.95, sum(rate(ticdc_sink_txn_exec_duration_bucket{changefeed=~\"$changefeed\"}[1m])) by (le,instance))", - "format": "time_series", - "intervalFactor": 1, - "legendFormat": "{{instance}}-p95", + "expr": "sum(rate(tikv_cdc_scan_duration_seconds_bucket{instance=~\"$tikv_instance\"}[1m])) by (le)", + "format": "heatmap", + "instant": false, + "intervalFactor": 2, + "legendFormat": "{{le}}", "refId": "A" - }, - { - "expr": "histogram_quantile(0.99, sum(rate(ticdc_sink_txn_exec_duration_bucket{changefeed=~\"$changefeed\"}[1m])) by (le,instance))", - "format": "time_series", - "intervalFactor": 1, - "legendFormat": "{{instance}}-p99", - "refId": "B" - }, - { - "expr": "histogram_quantile(0.999, sum(rate(ticdc_sink_txn_exec_duration_bucket{changefeed=~\"$changefeed\"}[1m])) by (le,instance))", - "format": "time_series", - "intervalFactor": 1, - "legendFormat": "{{instance}}-p999", - "refId": "C" } ], - "thresholds": [], - "timeFrom": null, - "timeRegions": [], - "timeShift": null, - "title": "exec txn duration", + "title": "Initial scan duration", "tooltip": { - "shared": true, - "sort": 0, - "value_type": "individual" - }, - "type": "graph", - "xaxis": { - "buckets": null, - "mode": "time", - "name": null, "show": true, - "values": [] + "showHistogram": true }, - "yaxes": [ - { - "format": "s", - "label": null, - "logBase": 2, - "max": null, - "min": null, - "show": true - }, - { - "format": "short", - "label": null, - "logBase": 1, - "max": null, - "min": null, - "show": true - } - ], - "yaxis": { - "align": false, - "alignLevel": null - } + "tooltipDecimals": 1, + "type": "heatmap", + "xAxis": { + "show": true + }, + "xBucketNumber": null, + "xBucketSize": null, + "yAxis": { + "decimals": 1, + "format": "s", + "logBase": 1, + "max": null, + "min": null, + "show": true, + "splitFactor": null + }, + "yBucketBound": "upper", + "yBucketNumber": null, + "yBucketSize": null }, { "aliasColors": {}, @@ -2973,25 +3098,29 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "DML count that executed to downstream", + "decimals": 1, + "description": "", "fill": 1, - "fillGradient": 0, "gridPos": { - "h": 7, + "h": 8, "w": 12, - "x": 12, - "y": 38 + "x": 0, + "y": 87 }, - "hiddenSeries": false, - "id": 34, + "id": 70, "legend": { "alignAsTable": true, - "avg": true, + "avg": false, "current": true, + "hideEmpty": false, + "hideZero": true, "max": true, "min": false, "rightSide": true, "show": true, + "sideWidth": null, + "sort": null, + "sortDesc": null, "total": false, "values": true }, @@ -2999,12 +3128,9 @@ "linewidth": 1, "links": [], "nullPointMode": "null", - "options": { - "dataLinks": [] - }, "paceLength": 10, "percentage": false, - "pointradius": 2, + "pointradius": 5, "points": false, "renderer": "flot", "seriesOverrides": [], @@ -3013,26 +3139,18 @@ "steppedLine": false, "targets": [ { - "expr": "sum (rate(ticdc_sink_txn_batch_size_sum{changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m])) by (capture)", + "expr": "histogram_quantile(0.99999, sum(rate(tikv_cdc_resolved_ts_gap_seconds_bucket{instance=~\"$tikv_instance\"}[1m])) by (le, instance))", "format": "time_series", "intervalFactor": 1, - "legendFormat": "{{capture}}", + "legendFormat": "{{instance}}", "refId": "A" - }, - { - "expr": "sum (rate(ticdc_sink_txn_batch_size_sum{changefeed=~\"$changefeed\"}[1m])) by (changefeed)", - "format": "time_series", - "interval": "", - "intervalFactor": 1, - "legendFormat": "{{changefeed}}", - "refId": "B" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "sink exec DML count", + "title": "99.9% Resolved ts lag duration", "tooltip": { "shared": true, "sort": 0, @@ -3048,9 +3166,9 @@ }, "yaxes": [ { - "format": "none", + "format": "s", "label": null, - "logBase": 1, + "logBase": 10, "max": null, "min": null, "show": true @@ -3075,25 +3193,27 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "input chan size of table mounter", - "fill": 0, - "fillGradient": 0, + "decimals": 1, + "description": "", + "fill": 1, "gridPos": { - "h": 7, + "h": 8, "w": 12, - "x": 0, - "y": 45 + "x": 12, + "y": 87 }, - "hiddenSeries": false, - "id": 37, + "id": 72, "legend": { "alignAsTable": true, "avg": false, "current": true, + "hideEmpty": false, + "hideZero": true, "max": true, "min": false, "rightSide": true, "show": true, + "sideWidth": null, "sort": "current", "sortDesc": true, "total": false, @@ -3103,12 +3223,9 @@ "linewidth": 1, "links": [], "nullPointMode": "null", - "options": { - "dataLinks": [] - }, "paceLength": 10, "percentage": false, - "pointradius": 2, + "pointradius": 5, "points": false, "renderer": "flot", "seriesOverrides": [], @@ -3117,10 +3234,10 @@ "steppedLine": false, "targets": [ { - "expr": "sum(ticdc_mounter_input_chan_size{changefeed=~\"$changefeed\",capture=~\"$capture\"}) by (capture)", + "expr": "histogram_quantile(0.999, sum(rate(tikv_cdc_scan_duration_seconds_bucket{instance=~\"$tikv_instance\"}[1m])) by (le, instance))", "format": "time_series", "intervalFactor": 1, - "legendFormat": "{{capture}}", + "legendFormat": "{{instance}}", "refId": "A" } ], @@ -3128,10 +3245,10 @@ "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "mounter input chan size", + "title": "99.9% Scan duration", "tooltip": { "shared": true, - "sort": 2, + "sort": 0, "value_type": "individual" }, "type": "graph", @@ -3144,7 +3261,7 @@ }, "yaxes": [ { - "format": "none", + "format": "s", "label": null, "logBase": 1, "max": null, @@ -3157,7 +3274,7 @@ "logBase": 1, "max": null, "min": null, - "show": true + "show": false } ], "yaxis": { @@ -3171,25 +3288,30 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "The number of events received from kv client event channel", - "fill": 1, - "fillGradient": 0, + "decimals": 1, + "description": "The memory usage per TiKV instance", + "editable": true, + "error": false, + "fill": 0, + "grid": {}, "gridPos": { - "h": 7, + "h": 8, "w": 12, - "x": 12, - "y": 45 + "x": 0, + "y": 95 }, - "hiddenSeries": false, - "id": 36, + "id": 74, "legend": { "alignAsTable": true, "avg": false, "current": true, - "max": false, + "max": true, "min": false, "rightSide": true, "show": true, + "sideWidth": null, + "sort": "max", + "sortDesc": true, "total": false, "values": true }, @@ -3197,12 +3319,9 @@ "linewidth": 1, "links": [], "nullPointMode": "null", - "options": { - "dataLinks": [] - }, "paceLength": 10, "percentage": false, - "pointradius": 2, + "pointradius": 5, "points": false, "renderer": "flot", "seriesOverrides": [], @@ -3211,34 +3330,40 @@ "steppedLine": false, "targets": [ { - "expr": "histogram_quantile(0.90, sum(rate(ticdc_sink_txn_batch_size_bucket{changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m])) by (le,capture))", + "expr": "avg(process_resident_memory_bytes{instance=~\"$tikv_instance\", job=\"tikv\"}) by (instance)", "format": "time_series", - "intervalFactor": 1, - "legendFormat": "{{capture}}-p90", - "refId": "A" + "hide": true, + "intervalFactor": 2, + "legendFormat": "tikv-{{instance}}", + "refId": "A", + "step": 10 }, { - "expr": "histogram_quantile(0.99, sum(rate(ticdc_sink_txn_batch_size_bucket{changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m])) by (le,capture))", + "expr": "avg(process_resident_memory_bytes{instance=~\"$tikv_instance\", job=\"cdc\"}) by (instance)", "format": "time_series", - "intervalFactor": 1, - "legendFormat": "{{capture}}-p99", - "refId": "B" + "hide": true, + "intervalFactor": 2, + "legendFormat": "cdc-{{instance}}", + "refId": "B", + "step": 10 }, { - "expr": "histogram_quantile(0.999, sum(rate(ticdc_sink_txn_batch_size_bucket{changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m])) by (le,capture))", + "expr": "(avg(process_resident_memory_bytes{instance=~\"$tikv_instance\", job=\"tikv\"}) by (instance)) - (avg(tikv_engine_block_cache_size_bytes{instance=~\"$tikv_instance\", db=\"kv\"}) by(instance))", "format": "time_series", - "hide": true, - "intervalFactor": 1, - "legendFormat": "{{capture}}-p999", - "refId": "C" + "hide": false, + "intervalFactor": 2, + "legendFormat": "tikv-{{instance}}", + "refId": "C", + "step": 10 } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "exec txn batch", + "title": "Memory without block cache", "tooltip": { + "msResolution": false, "shared": true, "sort": 0, "value_type": "individual" @@ -3253,11 +3378,11 @@ }, "yaxes": [ { - "format": "none", + "format": "bytes", "label": null, - "logBase": 2, + "logBase": 1, "max": null, - "min": null, + "min": "0", "show": true }, { @@ -3280,17 +3405,19 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "Bucketed histogram of processing time (s) of unmarshal and mount in mounter.", + "decimals": 1, + "description": "The memory usage per TiKV instance", + "editable": true, + "error": false, "fill": 0, - "fillGradient": 0, + "grid": {}, "gridPos": { - "h": 7, + "h": 8, "w": 12, - "x": 0, - "y": 52 + "x": 12, + "y": 95 }, - "hiddenSeries": false, - "id": 55, + "id": 76, "legend": { "alignAsTable": true, "avg": false, @@ -3299,6 +3426,7 @@ "min": false, "rightSide": true, "show": true, + "sideWidth": null, "sort": "current", "sortDesc": true, "total": false, @@ -3308,12 +3436,9 @@ "linewidth": 1, "links": [], "nullPointMode": "null", - "options": { - "dataLinks": [] - }, "paceLength": 10, "percentage": false, - "pointradius": 2, + "pointradius": 5, "points": false, "renderer": "flot", "seriesOverrides": [], @@ -3322,37 +3447,24 @@ "steppedLine": false, "targets": [ { - "expr": "histogram_quantile(0.99, sum(rate(ticdc_mounter_unmarshal_and_mount_bucket{changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m])) by (le, capture))", - "format": "time_series", - "intervalFactor": 1, - "legendFormat": "{{capture}}-p99", - "refId": "A" - }, - { - "expr": "histogram_quantile(0.999, sum(rate(ticdc_mounter_unmarshal_and_mount_bucket{changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m])) by (le, capture))", - "format": "time_series", - "hide": true, - "instant": false, - "intervalFactor": 1, - "legendFormat": "{{capture}}-p999", - "refId": "B" - }, - { - "expr": "rate(ticdc_mounter_unmarshal_and_mount_sum{changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m]) / rate(ticdc_mounter_unmarshal_and_mount_count{changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m])", + "expr": "avg(tikv_cdc_pending_bytes{instance=~\"$tikv_instance\", job=\"tikv\"}) by (instance)", "format": "time_series", - "intervalFactor": 1, - "legendFormat": "{{capture}}-avg", - "refId": "C" + "hide": false, + "intervalFactor": 2, + "legendFormat": "tikv-{{instance}}", + "refId": "A", + "step": 10 } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "mounter unmarshal and mount", + "title": "CDC pending bytes in memory", "tooltip": { + "msResolution": false, "shared": true, - "sort": 2, + "sort": 0, "value_type": "individual" }, "type": "graph", @@ -3365,15 +3477,15 @@ }, "yaxes": [ { - "format": "s", + "format": "bytes", "label": null, "logBase": 1, "max": null, - "min": null, + "min": "0", "show": true }, { - "format": "none", + "format": "short", "label": null, "logBase": 1, "max": null, @@ -3392,17 +3504,19 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "size of row changed event output channel from table to processor", + "decimals": 1, + "description": "The memory usage per TiKV instance", + "editable": true, + "error": false, "fill": 0, - "fillGradient": 0, + "grid": {}, "gridPos": { - "h": 7, + "h": 8, "w": 12, - "x": 12, - "y": 52 + "x": 0, + "y": 103 }, - "hiddenSeries": false, - "id": 38, + "id": 78, "legend": { "alignAsTable": true, "avg": false, @@ -3411,8 +3525,9 @@ "min": false, "rightSide": true, "show": true, - "sort": "current", - "sortDesc": true, + "sideWidth": null, + "sort": null, + "sortDesc": null, "total": false, "values": true }, @@ -3420,12 +3535,9 @@ "linewidth": 1, "links": [], "nullPointMode": "null", - "options": { - "dataLinks": [] - }, "paceLength": 10, "percentage": false, - "pointradius": 2, + "pointradius": 5, "points": false, "renderer": "flot", "seriesOverrides": [], @@ -3434,21 +3546,24 @@ "steppedLine": false, "targets": [ { - "expr": "sum(ticdc_processor_txn_output_chan_size{changefeed=~\"$changefeed\",capture=~\"$capture\"}) by (capture)", + "expr": "avg(tikv_cdc_captured_region_total{instance=~\"$tikv_instance\"}) by (instance)", "format": "time_series", - "intervalFactor": 1, - "legendFormat": "{{capture}}", - "refId": "A" + "hide": false, + "intervalFactor": 2, + "legendFormat": "tikv-{{instance}}", + "refId": "A", + "step": 10 } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "processor table output chan size", + "title": "Captured region count", "tooltip": { + "msResolution": false, "shared": true, - "sort": 2, + "sort": 0, "value_type": "individual" }, "type": "graph", @@ -3465,7 +3580,7 @@ "label": null, "logBase": 1, "max": null, - "min": null, + "min": "0", "show": true }, { @@ -3481,40 +3596,51 @@ "align": false, "alignLevel": null } - }, + } + ], + "title": "TiKV", + "type": "row" + }, + { + "collapsed": true, + "gridPos": { + "h": 1, + "w": 24, + "x": 0, + "y": 3 + }, + "id": 13, + "panels": [ { "aliasColors": {}, "bars": false, "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "checkpoint channel size for mq sink", - "fill": 0, + "fill": 1, "fillGradient": 0, "gridPos": { "h": 7, "w": 12, "x": 0, - "y": 59 + "y": 2 }, "hiddenSeries": false, - "id": 48, + "id": 15, "legend": { "alignAsTable": true, "avg": false, "current": true, - "max": true, + "hideEmpty": false, + "max": false, "min": false, "rightSide": true, "show": true, - "sort": "current", - "sortDesc": true, "total": false, "values": true }, "lines": true, "linewidth": 1, - "links": [], "nullPointMode": "null", "options": { "dataLinks": [] @@ -3530,10 +3656,10 @@ "steppedLine": false, "targets": [ { - "expr": "sum(ticdc_sink_mq_checkpoint_chan_size{changefeed=~\"$changefeed\",capture=~\"$capture\"}) by (capture)", + "expr": "ticdc_kvclient_event_feed_count", "format": "time_series", "intervalFactor": 1, - "legendFormat": "{{capture}}", + "legendFormat": "{{instance}}", "refId": "A" } ], @@ -3541,10 +3667,10 @@ "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "sink mq checkpoint chan size", + "title": "Event Feed Count", "tooltip": { "shared": true, - "sort": 2, + "sort": 0, "value_type": "individual" }, "type": "graph", @@ -3557,7 +3683,7 @@ }, "yaxes": [ { - "format": "none", + "format": "short", "label": null, "logBase": 1, "max": null, @@ -3584,33 +3710,31 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "Bucketed histogram of processing time (s) of waiting event prepare in processor.", - "fill": 0, + "fill": 1, "fillGradient": 0, "gridPos": { "h": 7, "w": 12, "x": 12, - "y": 59 + "y": 2 }, "hiddenSeries": false, - "id": 56, + "id": 28, "legend": { "alignAsTable": true, "avg": false, "current": true, + "hideEmpty": true, + "hideZero": true, "max": true, "min": false, "rightSide": true, "show": true, - "sort": "current", - "sortDesc": true, "total": false, "values": true }, "lines": true, "linewidth": 1, - "links": [], "nullPointMode": "null", "options": { "dataLinks": [] @@ -3626,28 +3750,21 @@ "steppedLine": false, "targets": [ { - "expr": "histogram_quantile(0.999, sum(rate(ticdc_processor_wait_event_prepare_bucket{changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m])) by (le, capture))", + "expr": "sum(rate(ticdc_kvclient_event_feed_error_count[30s])) by (instance, type)", "format": "time_series", "intervalFactor": 1, - "legendFormat": "{{capture}}-p999", + "legendFormat": "{{instance}} - {{type}}", "refId": "A" - }, - { - "expr": "rate(ticdc_processor_wait_event_prepare_sum{changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m]) / rate(ticdc_processor_wait_event_prepare_count{changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m])", - "format": "time_series", - "intervalFactor": 1, - "legendFormat": "{{capture}}-avg", - "refId": "B" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "processor wait event prepare", + "title": "Event Feed Error Count", "tooltip": { "shared": true, - "sort": 2, + "sort": 0, "value_type": "individual" }, "type": "graph", @@ -3660,7 +3777,7 @@ }, "yaxes": [ { - "format": "s", + "format": "short", "label": null, "logBase": 1, "max": null, @@ -3687,15 +3804,17 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "total count of mysql execution errors", + "decimals": null, "fill": 1, + "fillGradient": 0, "gridPos": { "h": 7, "w": 12, "x": 0, - "y": 66 + "y": 9 }, - "id": 81, + "hiddenSeries": false, + "id": 17, "legend": { "alignAsTable": true, "avg": false, @@ -3709,8 +3828,11 @@ }, "lines": true, "linewidth": 1, - "links": [], "nullPointMode": "null", + "options": { + "dataLinks": [] + }, + "paceLength": 10, "percentage": false, "pointradius": 2, "points": false, @@ -3721,10 +3843,10 @@ "steppedLine": false, "targets": [ { - "expr": "sum(changes(ticdc_sink_mysql_execution_error{changefeed=~\"$changefeed\",capture=~\"$capture\"}[30m])) by (capture)", + "expr": "histogram_quantile(0.95, sum(rate(ticdc_kvclient_event_size_bytes_bucket[1m])) by (le, instance))", "format": "time_series", "intervalFactor": 1, - "legendFormat": "{{capture}}", + "legendFormat": "{{instance}} 95", "refId": "A" } ], @@ -3732,10 +3854,10 @@ "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "sink mysql execution error", + "title": "Event Size", "tooltip": { "shared": true, - "sort": 2, + "sort": 0, "value_type": "individual" }, "type": "graph", @@ -3748,7 +3870,7 @@ }, "yaxes": [ { - "format": "short", + "format": "bytes", "label": null, "logBase": 1, "max": null, @@ -3775,21 +3897,22 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "Bucketed histogram of conflict detect time (s) for single DML statement", "fill": 1, "fillGradient": 0, "gridPos": { "h": 7, "w": 12, "x": 12, - "y": 66 + "y": 9 }, "hiddenSeries": false, - "id": 83, + "id": 29, "legend": { "alignAsTable": true, "avg": false, "current": true, + "hideEmpty": true, + "hideZero": true, "max": true, "min": false, "rightSide": true, @@ -3804,6 +3927,7 @@ "options": { "dataLinks": [] }, + "paceLength": 10, "percentage": false, "pointradius": 2, "points": false, @@ -3814,30 +3938,21 @@ "steppedLine": false, "targets": [ { - "expr": "histogram_quantile(0.95,sum(rate(ticdc_sink_conflict_detect_duration_bucket{changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m])) by (le,instance))", + "expr": "sum(rate(ticdc_kvclient_pull_event_count{changefeed=~\"$changefeed\"}[1m])) by (instance, type)", "format": "time_series", - "interval": "", "intervalFactor": 1, - "legendFormat": "{{instance}}-95", + "legendFormat": "{{instance}}-{{type}}", "refId": "A" - }, - { - "expr": "histogram_quantile(0.99,sum(rate(ticdc_sink_conflict_detect_duration_bucket{changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m])) by (le,instance))", - "format": "time_series", - "interval": "", - "intervalFactor": 1, - "legendFormat": "{{instance}}-99", - "refId": "B" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "sink conflict detect duration", + "title": "EventFeed Receive Event Count By Instance", "tooltip": { "shared": true, - "sort": 2, + "sort": 0, "value_type": "individual" }, "type": "graph", @@ -3850,9 +3965,9 @@ }, "yaxes": [ { - "format": "s", + "format": "short", "label": null, - "logBase": 2, + "logBase": 1, "max": null, "min": null, "show": true @@ -3877,27 +3992,26 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "size of the DML bucket", "fill": 1, "fillGradient": 0, "gridPos": { "h": 7, "w": 12, "x": 0, - "y": 73 + "y": 16 }, "hiddenSeries": false, - "id": 84, + "id": 31, "legend": { "alignAsTable": true, - "avg": true, + "avg": false, "current": true, + "hideEmpty": true, + "hideZero": true, "max": true, "min": false, "rightSide": true, "show": true, - "sort": "current", - "sortDesc": true, "total": false, "values": true }, @@ -3908,6 +4022,7 @@ "options": { "dataLinks": [] }, + "paceLength": 10, "percentage": false, "pointradius": 2, "points": false, @@ -3918,11 +4033,10 @@ "steppedLine": false, "targets": [ { - "expr": "sum(rate(ticdc_sink_bucket_size{changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m])) by (capture,bucket)", + "expr": "sum(rate(ticdc_kvclient_send_event_count{changefeed=~\"$changefeed\"}[1m])) by (instance, type)", "format": "time_series", - "interval": "", "intervalFactor": 1, - "legendFormat": "{{capture}}-{{bucket}}", + "legendFormat": "{{instance}}-{{type}}", "refId": "A" } ], @@ -3930,10 +4044,10 @@ "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "mysql sink bucket_size", + "title": "Dispatch Event Count By Instance", "tooltip": { "shared": true, - "sort": 2, + "sort": 0, "value_type": "individual" }, "type": "graph", @@ -3973,27 +4087,26 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "size of pending flush rows", + "description": "Puller event channel size", "fill": 1, "fillGradient": 0, "gridPos": { "h": 7, "w": 12, "x": 12, - "y": 73 + "y": 16 }, "hiddenSeries": false, - "id": 88, + "id": 40, "legend": { "alignAsTable": true, - "avg": true, + "avg": false, "current": true, + "hideEmpty": false, "max": true, "min": false, "rightSide": true, "show": true, - "sort": "current", - "sortDesc": true, "total": false, "values": true }, @@ -4004,6 +4117,7 @@ "options": { "dataLinks": [] }, + "paceLength": 10, "percentage": false, "pointradius": 2, "points": false, @@ -4014,9 +4128,8 @@ "steppedLine": false, "targets": [ { - "expr": "sum(ticdc_sink_total_rows_count{changefeed=~\"$changefeed\",capture=~\"$capture\"}) by (capture) - sum(ticdc_sink_total_flushed_rows_count{changefeed=~\"$changefeed\",capture=~\"$capture\"}) by (capture)", + "expr": "sum(ticdc_puller_event_chan_size{changefeed=~\"$changefeed\",capture=~\"$capture\"}) by (capture, table)", "format": "time_series", - "interval": "", "intervalFactor": 1, "legendFormat": "{{capture}}", "refId": "A" @@ -4026,10 +4139,10 @@ "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Pending flush rows", + "title": "puller event chan size", "tooltip": { "shared": true, - "sort": 2, + "sort": 0, "value_type": "individual" }, "type": "graph", @@ -4063,122 +4176,32 @@ "alignLevel": null } }, - { - "cards": { - "cardPadding": 0, - "cardRound": 0 - }, - "color": { - "cardColor": "#FF9830", - "colorScale": "linear", - "colorScheme": "interpolateSpectral", - "exponent": 0.5, - "min": 0, - "mode": "spectrum" - }, - "dataFormat": "tsbuckets", - "datasource": "${DS_TEST-CLUSTER}", - "description": "", - "gridPos": { - "h": 8, - "w": 12, - "x": 0, - "y": 80 - }, - "heatmap": {}, - "hideZeroBuckets": true, - "highlightCards": true, - "id": 87, - "legend": { - "alignAsTable": true, - "avg": false, - "current": true, - "max": true, - "min": false, - "rightSide": true, - "show": true, - "sort": "current", - "sortDesc": true, - "total": false, - "values": true - }, - "links": [], - "reverseYBuckets": false, - "targets": [ - { - "expr": "sum(rate(ticdc_sink_txn_exec_duration_bucket{capture=~\"$capture\"}[1m])) by (le)", - "format": "heatmap", - "instant": false, - "intervalFactor": 2, - "legendFormat": "{{le}}", - "refId": "A" - } - ], - "title": "Sink txn execution duration", - "tooltip": { - "show": true, - "showHistogram": true - }, - "tooltipDecimals": 1, - "type": "heatmap", - "xAxis": { - "show": true - }, - "xBucketNumber": null, - "xBucketSize": null, - "yAxis": { - "decimals": 1, - "format": "s", - "logBase": 1, - "max": null, - "min": null, - "show": true, - "splitFactor": null - }, - "yBucketBound": "upper", - "yBucketNumber": null, - "yBucketSize": null - } - ], - "title": "Changefeed", - "type": "row" - }, - { - "collapsed": true, - "gridPos": { - "h": 1, - "w": 24, - "x": 0, - "y": 3 - }, - "id": 58, - "panels": [ { "aliasColors": {}, "bars": false, "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "decimals": 1, + "description": "Puller entry channel buffer size", "fill": 1, + "fillGradient": 0, "gridPos": { - "h": 8, + "h": 7, "w": 12, "x": 0, - "y": 4 + "y": 23 }, - "id": 60, + "hiddenSeries": false, + "id": 43, "legend": { "alignAsTable": true, "avg": false, "current": true, + "hideEmpty": false, "max": true, "min": false, "rightSide": true, "show": true, - "sideWidth": null, - "sort": "current", - "sortDesc": true, "total": false, "values": true }, @@ -4186,9 +4209,12 @@ "linewidth": 1, "links": [], "nullPointMode": "null", + "options": { + "dataLinks": [] + }, "paceLength": 10, "percentage": false, - "pointradius": 5, + "pointradius": 2, "points": false, "renderer": "flot", "seriesOverrides": [], @@ -4197,19 +4223,18 @@ "steppedLine": false, "targets": [ { - "expr": "sum(rate(tikv_thread_cpu_seconds_total{instance=~\"$tikv_instance\", name=~\"cdc\"}[1m])) by (instance)", + "expr": "sum(ticdc_puller_output_chan_size{changefeed=~\"$changefeed\",capture=~\"$capture\"}) by (capture, table)", "format": "time_series", - "intervalFactor": 2, - "legendFormat": "{{instance}}", - "refId": "A", - "step": 4 + "intervalFactor": 1, + "legendFormat": "{{capture}} - {{table}}", + "refId": "A" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "CDC endpoint CPU", + "title": "puller output chan size", "tooltip": { "shared": true, "sort": 0, @@ -4225,7 +4250,7 @@ }, "yaxes": [ { - "format": "percentunit", + "format": "short", "label": null, "logBase": 1, "max": null, @@ -4252,26 +4277,26 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "decimals": 1, + "description": "Puller in memory buffer size", "fill": 1, + "fillGradient": 0, "gridPos": { - "h": 8, + "h": 7, "w": 12, "x": 12, - "y": 4 + "y": 23 }, - "id": 62, + "hiddenSeries": false, + "id": 50, "legend": { "alignAsTable": true, "avg": false, "current": true, + "hideEmpty": false, "max": true, "min": false, "rightSide": true, "show": true, - "sideWidth": null, - "sort": "current", - "sortDesc": true, "total": false, "values": true }, @@ -4279,9 +4304,12 @@ "linewidth": 1, "links": [], "nullPointMode": "null", + "options": { + "dataLinks": [] + }, "paceLength": 10, "percentage": false, - "pointradius": 5, + "pointradius": 2, "points": false, "renderer": "flot", "seriesOverrides": [], @@ -4290,19 +4318,18 @@ "steppedLine": false, "targets": [ { - "expr": "sum(rate(tikv_thread_cpu_seconds_total{instance=~\"$tikv_instance\", name=~\"cdcwkr.*\"}[1m])) by (instance)", + "expr": "sum(ticdc_puller_mem_buffer_size{changefeed=~\"$changefeed\",capture=~\"$capture\"}) by (capture, table)", "format": "time_series", - "intervalFactor": 2, - "legendFormat": "{{instance}}", - "refId": "A", - "step": 4 + "intervalFactor": 1, + "legendFormat": "{{capture}} - {{table}}", + "refId": "A" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "CDC worker CPU", + "title": "puller mem buffer size", "tooltip": { "shared": true, "sort": 0, @@ -4318,7 +4345,7 @@ }, "yaxes": [ { - "format": "percentunit", + "format": "short", "label": null, "logBase": 1, "max": null, @@ -4340,209 +4367,99 @@ } }, { - "columns": [ - { - "text": "Current", - "value": "current" - }, - { - "text": "Max", - "value": "max" - } - ], - "datasource": "${DS_TEST-CLUSTER}", - "fontSize": "90%", - "gridPos": { - "h": 8, - "w": 7, - "x": 0, - "y": 12 - }, - "hideTimeOverride": true, - "id": 64, - "links": [], - "pageSize": null, - "scroll": true, - "showHeader": true, - "sort": { - "col": 2, - "desc": false - }, - "styles": [ - { - "alias": "Instance", - "colorMode": null, - "colors": [ - "rgba(245, 54, 54, 0.9)", - "rgba(237, 129, 40, 0.89)", - "rgba(50, 172, 45, 0.97)" - ], - "dateFormat": "YYYY-MM-DD HH:mm:ss", - "decimals": 2, - "mappingType": 1, - "pattern": "Metric", - "thresholds": [], - "type": "string", - "unit": "short", - "valueMaps": [] - }, - { - "alias": "Time", - "colorMode": null, - "colors": [ - "rgba(245, 54, 54, 0.9)", - "rgba(237, 129, 40, 0.89)", - "rgba(50, 172, 45, 0.97)" - ], - "dateFormat": "MM-DD HH:mm:ss.SSS", - "decimals": 2, - "mappingType": 1, - "pattern": "Max", - "thresholds": [], - "type": "date", - "unit": "short" - } - ], - "targets": [ - { - "expr": "tikv_cdc_min_resolved_ts{instance=~\"$tikv_instance\"}", - "format": "time_series", - "intervalFactor": 2, - "legendFormat": "{{instance}}", - "refId": "A" - } - ], - "timeFrom": "1s", - "title": "Min resolved ts", - "transform": "timeseries_aggregations", - "type": "table" - }, - { - "columns": [ - { - "text": "Current", - "value": "current" - } - ], - "datasource": "${DS_TEST-CLUSTER}", - "fontSize": "90%", - "gridPos": { - "h": 8, - "w": 5, - "x": 7, - "y": 12 - }, - "hideTimeOverride": true, - "id": 66, - "links": [], - "pageSize": null, - "scroll": true, - "showHeader": true, - "sort": { - "col": 0, - "desc": false - }, - "styles": [ - { - "alias": "Option", - "colorMode": null, - "dateFormat": "YYYY-MM-DD HH:mm:ss", - "mappingType": 2, - "pattern": "Metric", - "preserveFormat": false, - "sanitize": false, - "type": "number" - } - ], - "targets": [ - { - "expr": "tikv_cdc_min_resolved_ts_region{instance=~\"$tikv_instance\"}", - "format": "time_series", - "intervalFactor": 2, - "legendFormat": "{{instance}}", - "refId": "A" - } - ], - "timeFrom": "1s", - "title": "Min resolved Region", - "transform": "timeseries_aggregations", - "type": "table" - }, - { - "cards": { - "cardPadding": 0, - "cardRound": 0 - }, - "color": { - "cardColor": "#FF9830", - "colorScale": "linear", - "colorScheme": "interpolateSpectral", - "exponent": 0.5, - "min": 0, - "mode": "spectrum" - }, - "dataFormat": "tsbuckets", + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "The time consumed to CDC incremental scan", + "description": "forwarded resolved ts in a single puller", + "fill": 1, + "fillGradient": 0, "gridPos": { - "h": 8, + "h": 7, "w": 12, - "x": 12, - "y": 12 + "x": 0, + "y": 30 }, - "heatmap": {}, - "hideZeroBuckets": true, - "highlightCards": true, - "id": 68, + "hiddenSeries": false, + "id": 52, "legend": { "alignAsTable": true, "avg": false, "current": true, + "hideEmpty": false, "max": true, "min": false, "rightSide": true, "show": true, - "sort": "current", - "sortDesc": true, "total": false, "values": true }, + "lines": true, + "linewidth": 1, "links": [], - "reverseYBuckets": false, + "nullPointMode": "null", + "options": { + "dataLinks": [] + }, + "paceLength": 10, + "percentage": false, + "pointradius": 2, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, "targets": [ { - "expr": "sum(rate(tikv_cdc_scan_duration_seconds_bucket{instance=~\"$tikv_instance\"}[1m])) by (le)", - "format": "heatmap", - "instant": false, - "intervalFactor": 2, - "legendFormat": "{{le}}", + "expr": "sum(ticdc_puller_resolved_ts{changefeed=~\"$changefeed\",capture=~\"$capture\"}) by (capture, table)", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "{{capture}}-{{table}}", "refId": "A" } ], - "title": "Initial scan duration", + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "puller resolved ts", "tooltip": { - "show": true, - "showHistogram": true - }, - "tooltipDecimals": 1, - "type": "heatmap", - "xAxis": { - "show": true + "shared": true, + "sort": 0, + "value_type": "individual" }, - "xBucketNumber": null, - "xBucketSize": null, - "yAxis": { - "decimals": 1, - "format": "s", - "logBase": 1, - "max": null, - "min": null, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, "show": true, - "splitFactor": null - }, - "yBucketBound": "upper", - "yBucketNumber": null, - "yBucketSize": null + "values": [] + }, + "yaxes": [ + { + "format": "dateTimeAsIso", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + }, + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } }, { "aliasColors": {}, @@ -4550,29 +4467,26 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "decimals": 1, - "description": "", + "description": "Puller entry sorter unsoreted items size", "fill": 1, + "fillGradient": 0, "gridPos": { - "h": 8, + "h": 7, "w": 12, - "x": 0, - "y": 20 + "x": 12, + "y": 30 }, - "id": 70, + "hiddenSeries": false, + "id": 51, "legend": { "alignAsTable": true, "avg": false, "current": true, "hideEmpty": false, - "hideZero": true, "max": true, "min": false, "rightSide": true, "show": true, - "sideWidth": null, - "sort": null, - "sortDesc": null, "total": false, "values": true }, @@ -4580,9 +4494,12 @@ "linewidth": 1, "links": [], "nullPointMode": "null", + "options": { + "dataLinks": [] + }, "paceLength": 10, "percentage": false, - "pointradius": 5, + "pointradius": 2, "points": false, "renderer": "flot", "seriesOverrides": [], @@ -4591,10 +4508,10 @@ "steppedLine": false, "targets": [ { - "expr": "histogram_quantile(0.99999, sum(rate(tikv_cdc_resolved_ts_gap_seconds_bucket{instance=~\"$tikv_instance\"}[1m])) by (le, instance))", + "expr": "sum(ticdc_puller_entry_sorter_unsorted_size{changefeed=~\"$changefeed\",capture=~\"$capture\"}) by (capture, table)", "format": "time_series", "intervalFactor": 1, - "legendFormat": "{{instance}}", + "legendFormat": "{{capture}}-{{table}}", "refId": "A" } ], @@ -4602,7 +4519,7 @@ "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "99.9% Resolved ts lag duration", + "title": "entry sorter unsorted size", "tooltip": { "shared": true, "sort": 0, @@ -4618,9 +4535,9 @@ }, "yaxes": [ { - "format": "s", + "format": "short", "label": null, - "logBase": 10, + "logBase": 1, "max": null, "min": null, "show": true @@ -4645,29 +4562,26 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "decimals": 1, - "description": "", + "description": "Puller entry sorter resolved channel size", "fill": 1, + "fillGradient": 0, "gridPos": { - "h": 8, + "h": 7, "w": 12, - "x": 12, - "y": 20 + "x": 0, + "y": 37 }, - "id": 72, + "hiddenSeries": false, + "id": 42, "legend": { "alignAsTable": true, "avg": false, "current": true, "hideEmpty": false, - "hideZero": true, "max": true, "min": false, "rightSide": true, "show": true, - "sideWidth": null, - "sort": "current", - "sortDesc": true, "total": false, "values": true }, @@ -4675,9 +4589,12 @@ "linewidth": 1, "links": [], "nullPointMode": "null", + "options": { + "dataLinks": [] + }, "paceLength": 10, "percentage": false, - "pointradius": 5, + "pointradius": 2, "points": false, "renderer": "flot", "seriesOverrides": [], @@ -4686,10 +4603,10 @@ "steppedLine": false, "targets": [ { - "expr": "histogram_quantile(0.999, sum(rate(tikv_cdc_scan_duration_seconds_bucket{instance=~\"$tikv_instance\"}[1m])) by (le, instance))", + "expr": "sum(ticdc_puller_entry_sorter_resolved_chan_size{changefeed=~\"$changefeed\",capture=~\"$capture\"}) by (capture, table)", "format": "time_series", "intervalFactor": 1, - "legendFormat": "{{instance}}", + "legendFormat": "{{capture}}-{{table}}", "refId": "A" } ], @@ -4697,7 +4614,7 @@ "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "99.9% Scan duration", + "title": "entry sorter resolved chan size", "tooltip": { "shared": true, "sort": 0, @@ -4713,7 +4630,7 @@ }, "yaxes": [ { - "format": "s", + "format": "short", "label": null, "logBase": 1, "max": null, @@ -4726,7 +4643,7 @@ "logBase": 1, "max": null, "min": null, - "show": false + "show": true } ], "yaxis": { @@ -4740,30 +4657,26 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "decimals": 1, - "description": "The memory usage per TiKV instance", - "editable": true, - "error": false, - "fill": 0, - "grid": {}, + "description": "Puller entry sorter output channel size", + "fill": 1, + "fillGradient": 0, "gridPos": { - "h": 8, + "h": 7, "w": 12, - "x": 0, - "y": 28 + "x": 12, + "y": 37 }, - "id": 74, + "hiddenSeries": false, + "id": 41, "legend": { "alignAsTable": true, "avg": false, "current": true, + "hideEmpty": false, "max": true, "min": false, "rightSide": true, "show": true, - "sideWidth": null, - "sort": "max", - "sortDesc": true, "total": false, "values": true }, @@ -4771,9 +4684,12 @@ "linewidth": 1, "links": [], "nullPointMode": "null", + "options": { + "dataLinks": [] + }, "paceLength": 10, "percentage": false, - "pointradius": 5, + "pointradius": 2, "points": false, "renderer": "flot", "seriesOverrides": [], @@ -4782,40 +4698,19 @@ "steppedLine": false, "targets": [ { - "expr": "avg(process_resident_memory_bytes{instance=~\"$tikv_instance\", job=\"tikv\"}) by (instance)", - "format": "time_series", - "hide": true, - "intervalFactor": 2, - "legendFormat": "tikv-{{instance}}", - "refId": "A", - "step": 10 - }, - { - "expr": "avg(process_resident_memory_bytes{instance=~\"$tikv_instance\", job=\"cdc\"}) by (instance)", - "format": "time_series", - "hide": true, - "intervalFactor": 2, - "legendFormat": "cdc-{{instance}}", - "refId": "B", - "step": 10 - }, - { - "expr": "(avg(process_resident_memory_bytes{instance=~\"$tikv_instance\", job=\"tikv\"}) by (instance)) - (avg(tikv_engine_block_cache_size_bytes{instance=~\"$tikv_instance\", db=\"kv\"}) by(instance))", + "expr": "sum(ticdc_puller_entry_sorter_output_chan_size{changefeed=~\"$changefeed\",capture=~\"$capture\"}) by (capture, table)", "format": "time_series", - "hide": false, - "intervalFactor": 2, - "legendFormat": "tikv-{{instance}}", - "refId": "C", - "step": 10 + "intervalFactor": 1, + "legendFormat": "{{capture}}-{{table}}", + "refId": "A" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Memory without block cache", + "title": "entry sorter output chan size", "tooltip": { - "msResolution": false, "shared": true, "sort": 0, "value_type": "individual" @@ -4830,11 +4725,11 @@ }, "yaxes": [ { - "format": "bytes", + "format": "short", "label": null, "logBase": 1, "max": null, - "min": "0", + "min": null, "show": true }, { @@ -4857,30 +4752,26 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "decimals": 1, - "description": "The memory usage per TiKV instance", - "editable": true, - "error": false, - "fill": 0, - "grid": {}, + "description": "Bucketed histogram of processing time (s) of merge in entry sorter", + "fill": 1, + "fillGradient": 0, "gridPos": { - "h": 8, + "h": 7, "w": 12, - "x": 12, - "y": 28 + "x": 0, + "y": 44 }, - "id": 76, + "hiddenSeries": false, + "id": 53, "legend": { "alignAsTable": true, "avg": false, "current": true, + "hideEmpty": false, "max": true, "min": false, "rightSide": true, "show": true, - "sideWidth": null, - "sort": "current", - "sortDesc": true, "total": false, "values": true }, @@ -4888,9 +4779,12 @@ "linewidth": 1, "links": [], "nullPointMode": "null", + "options": { + "dataLinks": [] + }, "paceLength": 10, "percentage": false, - "pointradius": 5, + "pointradius": 2, "points": false, "renderer": "flot", "seriesOverrides": [], @@ -4899,22 +4793,27 @@ "steppedLine": false, "targets": [ { - "expr": "avg(tikv_cdc_pending_bytes{instance=~\"$tikv_instance\", job=\"tikv\"}) by (instance)", + "expr": "histogram_quantile(0.99, sum(rate(ticdc_puller_entry_sorter_sort_bucket{changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m])) by (le,table))", "format": "time_series", - "hide": false, - "intervalFactor": 2, - "legendFormat": "tikv-{{instance}}", - "refId": "A", - "step": 10 + "intervalFactor": 1, + "legendFormat": "{{table}}-p99", + "refId": "A" + }, + { + "expr": "histogram_quantile(0.90, sum(rate(ticdc_puller_entry_sorter_sort_bucket{changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m])) by (le,table))", + "format": "time_series", + "hide": true, + "intervalFactor": 1, + "legendFormat": "{{table}}-p90", + "refId": "B" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "CDC pending bytes in memory", + "title": "entry sorter sort duration", "tooltip": { - "msResolution": false, "shared": true, "sort": 0, "value_type": "individual" @@ -4929,11 +4828,11 @@ }, "yaxes": [ { - "format": "bytes", + "format": "short", "label": null, - "logBase": 1, + "logBase": 2, "max": null, - "min": "0", + "min": null, "show": true }, { @@ -4956,30 +4855,26 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "decimals": 1, - "description": "The memory usage per TiKV instance", - "editable": true, - "error": false, - "fill": 0, - "grid": {}, + "description": "Bucketed histogram of processing time (s) of merge in entry sorter.", + "fill": 1, + "fillGradient": 0, "gridPos": { - "h": 8, + "h": 7, "w": 12, - "x": 0, - "y": 36 + "x": 12, + "y": 44 }, - "id": 78, + "hiddenSeries": false, + "id": 54, "legend": { "alignAsTable": true, "avg": false, "current": true, + "hideEmpty": false, "max": true, "min": false, "rightSide": true, "show": true, - "sideWidth": null, - "sort": null, - "sortDesc": null, "total": false, "values": true }, @@ -4987,9 +4882,12 @@ "linewidth": 1, "links": [], "nullPointMode": "null", + "options": { + "dataLinks": [] + }, "paceLength": 10, "percentage": false, - "pointradius": 5, + "pointradius": 2, "points": false, "renderer": "flot", "seriesOverrides": [], @@ -4998,22 +4896,28 @@ "steppedLine": false, "targets": [ { - "expr": "avg(tikv_cdc_captured_region_total{instance=~\"$tikv_instance\"}) by (instance)", + "expr": "histogram_quantile(0.99, sum(rate(ticdc_puller_entry_sorter_merge_bucket{changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m])) by (le,table))", "format": "time_series", "hide": false, - "intervalFactor": 2, - "legendFormat": "tikv-{{instance}}", - "refId": "A", - "step": 10 + "intervalFactor": 1, + "legendFormat": "{{table}}-p99", + "refId": "A" + }, + { + "expr": "histogram_quantile(0.90, sum(rate(ticdc_puller_entry_sorter_merge_bucket{changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m])) by (le,table))", + "format": "time_series", + "hide": true, + "intervalFactor": 1, + "legendFormat": "{{table}}-p90", + "refId": "B" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Captured region count", + "title": "entry sorter merge duration", "tooltip": { - "msResolution": false, "shared": true, "sort": 0, "value_type": "individual" @@ -5028,11 +4932,11 @@ }, "yaxes": [ { - "format": "none", + "format": "short", "label": null, - "logBase": 1, + "logBase": 2, "max": null, - "min": "0", + "min": null, "show": true }, { @@ -5050,7 +4954,7 @@ } } ], - "title": "TiKV", + "title": "Events", "type": "row" } ], @@ -5067,7 +4971,7 @@ "definition": "label_values(ticdc_processor_resolved_ts, changefeed)", "hide": 0, "includeAll": true, - "label": null, + "label": "Changefeed", "multi": false, "name": "changefeed", "options": [], @@ -5089,7 +4993,7 @@ "definition": "label_values(ticdc_processor_resolved_ts, capture)", "hide": 0, "includeAll": true, - "label": null, + "label": "Capture", "multi": false, "name": "capture", "options": [], @@ -5111,7 +5015,7 @@ "definition": "label_values(tikv_engine_size_bytes, instance)", "hide": 0, "includeAll": true, - "label": "TiKVInstance", + "label": "TiKV", "multi": false, "name": "tikv_instance", "options": [], @@ -5159,6 +5063,6 @@ }, "timezone": "browser", "title": "Test-Cluster-CDC", - "uid": "YiGL8hBZa", - "version": 6 + "uid": "YiGL8hBZ1", + "version": 7 } \ No newline at end of file diff --git a/pkg/cyclic/filter.go b/pkg/cyclic/filter.go index 49be509e499..1eac03e2762 100644 --- a/pkg/cyclic/filter.go +++ b/pkg/cyclic/filter.go @@ -22,11 +22,16 @@ import ( // ExtractReplicaID extracts replica ID from the given mark row. func ExtractReplicaID(markRow *model.RowChangedEvent) uint64 { - val, ok := markRow.Columns[mark.CyclicReplicaIDCol] - if !ok { - panic("bad mark table, " + mark.CyclicReplicaIDCol + " not found") + for _, c := range markRow.Columns { + if c == nil { + continue + } + if c.Name == mark.CyclicReplicaIDCol { + return c.Value.(uint64) + } } - return val.Value.(uint64) + log.Fatal("bad mark table, " + mark.CyclicReplicaIDCol + " not found") + return 0 } // TxnMap maps start ts to txn may cross multiple tables. @@ -58,10 +63,12 @@ func (m MarkMap) shouldFilterTxn(startTs uint64, filterReplicaIDs []uint64, repl // FilterAndReduceTxns filters duplicate txns bases on filterReplicaIDs and // if the mark table dml is exist in the txn, this functiong will set the replicaID by mark table dml // if the mark table dml is not exist, this function will set the replicaID by config -func FilterAndReduceTxns(txnsMap map[model.TableName][]*model.Txn, filterReplicaIDs []uint64, replicaID uint64) { +func FilterAndReduceTxns( + txnsMap map[model.TableID][]*model.SingleTableTxn, filterReplicaIDs []uint64, replicaID uint64, +) (skippedRowCount int) { markMap := make(MarkMap) - for table, txns := range txnsMap { - if !mark.IsMarkTable(table.Schema, table.Table) { + for _, txns := range txnsMap { + if !mark.IsMarkTable(txns[0].Table.Schema, txns[0].Table.Table) { continue } for _, txn := range txns { @@ -83,17 +90,22 @@ func FilterAndReduceTxns(txnsMap map[model.TableName][]*model.Txn, filterReplica } } for table, txns := range txnsMap { - if mark.IsMarkTable(table.Schema, table.Table) { + if mark.IsMarkTable(txns[0].Table.Schema, txns[0].Table.Table) { delete(txnsMap, table) + for i := range txns { + // For simplicity, we do not count mark table rows in statistics. + skippedRowCount += len(txns[i].Rows) + } continue } - filteredTxns := make([]*model.Txn, 0, len(txns)) + filteredTxns := make([]*model.SingleTableTxn, 0, len(txns)) for _, txn := range txns { // Check if we should skip this event markRow, needSkip := markMap.shouldFilterTxn(txn.StartTs, filterReplicaIDs, replicaID) if needSkip { // Found cyclic mark, skip this event as it originly created from // downstream. + skippedRowCount += len(txn.Rows) continue } txn.ReplicaID = replicaID @@ -108,4 +120,5 @@ func FilterAndReduceTxns(txnsMap map[model.TableName][]*model.Txn, filterReplica txnsMap[table] = filteredTxns } } + return } diff --git a/pkg/cyclic/filter_test.go b/pkg/cyclic/filter_test.go index fdcb8507967..f93c6309641 100644 --- a/pkg/cyclic/filter_test.go +++ b/pkg/cyclic/filter_test.go @@ -31,159 +31,167 @@ func TestCyclic(t *testing.T) { check.TestingT(t) } func (s *markSuit) TestFilterAndReduceTxns(c *check.C) { rID := mark.CyclicReplicaIDCol testCases := []struct { - input map[model.TableName][]*model.Txn - output map[model.TableName][]*model.Txn + input map[model.TableID][]*model.SingleTableTxn + output map[model.TableID][]*model.SingleTableTxn filterID []uint64 replicaID uint64 }{ { - input: map[model.TableName][]*model.Txn{}, - output: map[model.TableName][]*model.Txn{}, + input: map[model.TableID][]*model.SingleTableTxn{}, + output: map[model.TableID][]*model.SingleTableTxn{}, filterID: []uint64{}, replicaID: 0, }, { - input: map[model.TableName][]*model.Txn{{Table: "a"}: {{StartTs: 1}}}, - output: map[model.TableName][]*model.Txn{{Table: "a"}: {{StartTs: 1, ReplicaID: 1}}}, + input: map[model.TableID][]*model.SingleTableTxn{1: {{Table: &model.TableName{Table: "a"}, StartTs: 1}}}, + output: map[model.TableID][]*model.SingleTableTxn{1: {{Table: &model.TableName{Table: "a"}, StartTs: 1, ReplicaID: 1}}}, filterID: []uint64{}, replicaID: 1, }, { - input: map[model.TableName][]*model.Txn{ - {Schema: "tidb_cdc"} /* cyclic.SchemaName */ : { + input: map[model.TableID][]*model.SingleTableTxn{ + 2: { { + Table: &model.TableName{Schema: "tidb_cdc"}, /* cyclic.SchemaName */ StartTs: 1, - Rows: []*model.RowChangedEvent{{StartTs: 1, Columns: map[string]*model.Column{rID: {Value: uint64(10)}}}}, + Rows: []*model.RowChangedEvent{{StartTs: 1, Columns: []*model.Column{{Name: rID, Value: uint64(10)}}}}, }, }, }, - output: map[model.TableName][]*model.Txn{}, + output: map[model.TableID][]*model.SingleTableTxn{}, filterID: []uint64{}, replicaID: 1, }, { - input: map[model.TableName][]*model.Txn{ - {Table: "a"}: {{StartTs: 1}}, - {Schema: "tidb_cdc"}: { + input: map[model.TableID][]*model.SingleTableTxn{ + 1: {{Table: &model.TableName{Table: "a"}, StartTs: 1}}, + 2: { { + Table: &model.TableName{Schema: "tidb_cdc"}, /* cyclic.SchemaName */ StartTs: 1, - Rows: []*model.RowChangedEvent{{StartTs: 1, Columns: map[string]*model.Column{rID: {Value: uint64(10)}}}}, + Rows: []*model.RowChangedEvent{{StartTs: 1, Columns: []*model.Column{{Name: rID, Value: uint64(10)}}}}, }, }, }, - output: map[model.TableName][]*model.Txn{}, + output: map[model.TableID][]*model.SingleTableTxn{}, filterID: []uint64{10}, replicaID: 1, }, { - input: map[model.TableName][]*model.Txn{ - {Table: "a"}: {{StartTs: 1}}, - {Schema: "tidb_cdc", Table: "1"}: { + input: map[model.TableID][]*model.SingleTableTxn{ + 1: {{Table: &model.TableName{Table: "a"}, StartTs: 1}}, + 2: { { + Table: &model.TableName{Schema: "tidb_cdc", Table: "1"}, StartTs: 1, - Rows: []*model.RowChangedEvent{{StartTs: 1, Columns: map[string]*model.Column{rID: {Value: uint64(10)}}}}, + Rows: []*model.RowChangedEvent{{StartTs: 1, Columns: []*model.Column{{Name: rID, Value: uint64(10)}}}}, }, }, - {Schema: "tidb_cdc", Table: "2"}: { + 3: { { + Table: &model.TableName{Schema: "tidb_cdc", Table: "2"}, StartTs: 2, - Rows: []*model.RowChangedEvent{{StartTs: 2, Columns: map[string]*model.Column{rID: {Value: uint64(10)}}}}, + Rows: []*model.RowChangedEvent{{StartTs: 2, Columns: []*model.Column{{Name: rID, Value: uint64(10)}}}}, }, }, - {Schema: "tidb_cdc", Table: "3"}: { + 4: { { + Table: &model.TableName{Schema: "tidb_cdc", Table: "3"}, StartTs: 3, - Rows: []*model.RowChangedEvent{{StartTs: 3, Columns: map[string]*model.Column{rID: {Value: uint64(10)}}}}, + Rows: []*model.RowChangedEvent{{StartTs: 3, Columns: []*model.Column{{Name: rID, Value: uint64(10)}}}}, }, }, }, - output: map[model.TableName][]*model.Txn{}, + output: map[model.TableID][]*model.SingleTableTxn{}, filterID: []uint64{10}, replicaID: 1, }, { - input: map[model.TableName][]*model.Txn{ - {Table: "a"}: {{StartTs: 1}}, - {Table: "b2"}: {{StartTs: 2}}, - {Table: "b2_1"}: {{StartTs: 2}}, - {Schema: "tidb_cdc", Table: "1"}: { + input: map[model.TableID][]*model.SingleTableTxn{ + 1: {{Table: &model.TableName{Table: "a"}, StartTs: 1}}, + 2: {{Table: &model.TableName{Table: "b2"}, StartTs: 2}}, + 3: {{Table: &model.TableName{Table: "b2_1"}, StartTs: 2}}, + 4: { { + Table: &model.TableName{Schema: "tidb_cdc", Table: "1"}, StartTs: 1, - Rows: []*model.RowChangedEvent{{StartTs: 1, Columns: map[string]*model.Column{rID: {Value: uint64(10)}}}}, + Rows: []*model.RowChangedEvent{{StartTs: 1, Columns: []*model.Column{{Name: rID, Value: uint64(10)}}}}, }, }, }, - output: map[model.TableName][]*model.Txn{ - {Table: "b2"}: {{StartTs: 2, ReplicaID: 1}}, - {Table: "b2_1"}: {{StartTs: 2, ReplicaID: 1}}, + output: map[model.TableID][]*model.SingleTableTxn{ + 2: {{Table: &model.TableName{Table: "b2"}, StartTs: 2, ReplicaID: 1}}, + 3: {{Table: &model.TableName{Table: "b2_1"}, StartTs: 2, ReplicaID: 1}}, }, filterID: []uint64{10}, replicaID: 1, }, { - input: map[model.TableName][]*model.Txn{ - {Table: "a"}: {{StartTs: 1}}, - {Table: "b2"}: {{StartTs: 2}}, - {Table: "b2_1"}: {{StartTs: 2}}, - {Table: "b3"}: {{StartTs: 3}}, - {Table: "b3_1"}: {{StartTs: 3}}, - {Schema: "tidb_cdc", Table: "1"}: { + input: map[model.TableID][]*model.SingleTableTxn{ + 1: {{Table: &model.TableName{Table: "a"}, StartTs: 1}}, + 2: {{Table: &model.TableName{Table: "b2"}, StartTs: 2}}, + 3: {{Table: &model.TableName{Table: "b2_1"}, StartTs: 2}}, + 4: {{Table: &model.TableName{Table: "b3"}, StartTs: 3}}, + 5: {{Table: &model.TableName{Table: "b3_1"}, StartTs: 3}}, + 6: { { + Table: &model.TableName{Schema: "tidb_cdc", Table: "1"}, StartTs: 2, - Rows: []*model.RowChangedEvent{{StartTs: 2, Columns: map[string]*model.Column{rID: {Value: uint64(10)}}}}, + Rows: []*model.RowChangedEvent{{StartTs: 2, Columns: []*model.Column{{Name: rID, Value: uint64(10)}}}}, }, { + Table: &model.TableName{Schema: "tidb_cdc", Table: "1"}, StartTs: 3, - Rows: []*model.RowChangedEvent{{StartTs: 3, Columns: map[string]*model.Column{rID: {Value: uint64(11)}}}}, + Rows: []*model.RowChangedEvent{{StartTs: 3, Columns: []*model.Column{{Name: rID, Value: uint64(11)}}}}, }, }, }, - output: map[model.TableName][]*model.Txn{ - {Table: "a"}: {{StartTs: 1, ReplicaID: 1}}, - {Table: "b3"}: {{StartTs: 3, ReplicaID: 11}}, - {Table: "b3_1"}: {{StartTs: 3, ReplicaID: 11}}, + output: map[model.TableID][]*model.SingleTableTxn{ + 1: {{Table: &model.TableName{Table: "a"}, StartTs: 1, ReplicaID: 1}}, + 4: {{Table: &model.TableName{Table: "b3"}, StartTs: 3, ReplicaID: 11}}, + 5: {{Table: &model.TableName{Table: "b3_1"}, StartTs: 3, ReplicaID: 11}}, }, filterID: []uint64{10}, // 10 -> 2, filter start ts 2 replicaID: 1, }, { - input: map[model.TableName][]*model.Txn{ - {Table: "b2"}: {{StartTs: 2, CommitTs: 2}}, - {Table: "b3"}: { - {StartTs: 2, CommitTs: 2}, - {StartTs: 3, CommitTs: 3}, - {StartTs: 3, CommitTs: 3}, - {StartTs: 4, CommitTs: 4}, - }, - {Schema: "tidb_cdc", Table: "1"}: { + input: map[model.TableID][]*model.SingleTableTxn{ + 2: {{Table: &model.TableName{Table: "b2"}, StartTs: 2, CommitTs: 2}}, + 3: {{Table: &model.TableName{Table: "b3"}, StartTs: 2, CommitTs: 2}, + {Table: &model.TableName{Table: "b3"}, StartTs: 3, CommitTs: 3}, + {Table: &model.TableName{Table: "b3"}, StartTs: 3, CommitTs: 3}, + {Table: &model.TableName{Table: "b3"}, StartTs: 4, CommitTs: 4}}, + 6: { { + Table: &model.TableName{Schema: "tidb_cdc", Table: "1"}, StartTs: 2, - Rows: []*model.RowChangedEvent{{StartTs: 2, Columns: map[string]*model.Column{rID: {Value: uint64(10)}}}}, + Rows: []*model.RowChangedEvent{{StartTs: 2, Columns: []*model.Column{{Name: rID, Value: uint64(10)}}}}, }, { + Table: &model.TableName{Schema: "tidb_cdc", Table: "1"}, StartTs: 3, - Rows: []*model.RowChangedEvent{{StartTs: 3, Columns: map[string]*model.Column{rID: {Value: uint64(11)}}}}, + Rows: []*model.RowChangedEvent{{StartTs: 3, Columns: []*model.Column{{Name: rID, Value: uint64(11)}}}}, }, }, }, - output: map[model.TableName][]*model.Txn{ - {Table: "b3"}: { - {StartTs: 3, CommitTs: 3, ReplicaID: 11}, - {StartTs: 3, CommitTs: 3, ReplicaID: 11}, - {StartTs: 4, CommitTs: 4, ReplicaID: 1}, - }, + output: map[model.TableID][]*model.SingleTableTxn{ + 3: {{Table: &model.TableName{Table: "b3"}, StartTs: 3, CommitTs: 3, ReplicaID: 11}, + {Table: &model.TableName{Table: "b3"}, StartTs: 3, CommitTs: 3, ReplicaID: 11}, + {Table: &model.TableName{Table: "b3"}, StartTs: 4, CommitTs: 4, ReplicaID: 1}}, }, filterID: []uint64{10}, // 10 -> 2, filter start ts 2 replicaID: 1, - }, { - input: map[model.TableName][]*model.Txn{ - {Table: "b2"}: {{StartTs: 2}}, - {Schema: "tidb_cdc", Table: "1"}: { - // Duplicate mark table changes. + }, + { + input: map[model.TableID][]*model.SingleTableTxn{ + 2: {{Table: &model.TableName{Table: "b2"}, StartTs: 2}}, + 6: { { + Table: &model.TableName{Schema: "tidb_cdc", Table: "1"}, StartTs: 2, - Rows: []*model.RowChangedEvent{{StartTs: 2, Columns: map[string]*model.Column{rID: {Value: uint64(10)}}}}, + Rows: []*model.RowChangedEvent{{StartTs: 2, Columns: []*model.Column{{Name: rID, Value: uint64(10)}}}}, }, { + Table: &model.TableName{Schema: "tidb_cdc", Table: "1"}, StartTs: 2, - Rows: []*model.RowChangedEvent{{StartTs: 2, Columns: map[string]*model.Column{rID: {Value: uint64(10)}}}}, + Rows: []*model.RowChangedEvent{{StartTs: 2, Columns: []*model.Column{{Name: rID, Value: uint64(10)}}}}, }, }, }, - output: map[model.TableName][]*model.Txn{}, + output: map[model.TableID][]*model.SingleTableTxn{}, filterID: []uint64{10}, // 10 -> 2, filter start ts 2 replicaID: 1, }, diff --git a/pkg/filter/filter.go b/pkg/filter/filter.go index d10e4eaac71..f1718a3430c 100644 --- a/pkg/filter/filter.go +++ b/pkg/filter/filter.go @@ -14,18 +14,17 @@ package filter import ( - "strings" - "github.com/pingcap/errors" "github.com/pingcap/parser/model" "github.com/pingcap/ticdc/pkg/config" "github.com/pingcap/ticdc/pkg/cyclic/mark" - filter "github.com/pingcap/tidb-tools/pkg/table-filter" + filterV1 "github.com/pingcap/tidb-tools/pkg/filter" + filterV2 "github.com/pingcap/tidb-tools/pkg/table-filter" ) // Filter is a event filter implementation type Filter struct { - filter filter.Filter + filter filterV2.Filter ignoreTxnStartTs []uint64 ddlAllowlist []model.ActionType isCyclicEnabled bool @@ -33,22 +32,22 @@ type Filter struct { // NewFilter creates a filter func NewFilter(cfg *config.ReplicaConfig) (*Filter, error) { - var f filter.Filter + var f filterV2.Filter var err error if len(cfg.Filter.Rules) == 0 && cfg.Filter.MySQLReplicationRules != nil { - f, err = filter.ParseMySQLReplicationRules(cfg.Filter.MySQLReplicationRules) + f, err = filterV2.ParseMySQLReplicationRules(cfg.Filter.MySQLReplicationRules) } else { rules := cfg.Filter.Rules if len(rules) == 0 { rules = []string{"*.*"} } - f, err = filter.Parse(rules) + f, err = filterV2.Parse(rules) } if err != nil { return nil, errors.Trace(err) } if !cfg.CaseSensitive { - f = filter.CaseInsensitive(f) + f = filterV2.CaseInsensitive(f) } return &Filter{ filter: f, @@ -88,8 +87,16 @@ func (f *Filter) ShouldIgnoreDMLEvent(ts uint64, schema, table string) bool { // ShouldIgnoreDDLEvent removes DDLs that's not wanted by this change feed. // CDC only supports filtering by database/table now. -func (f *Filter) ShouldIgnoreDDLEvent(ts uint64, schema, table string) bool { - return f.shouldIgnoreStartTs(ts) || f.ShouldIgnoreTable(schema, table) +func (f *Filter) ShouldIgnoreDDLEvent(ts uint64, ddlType model.ActionType, schema, table string) bool { + var shouldIgnoreTableOrSchema bool + switch ddlType { + case model.ActionCreateSchema, model.ActionDropSchema, + model.ActionModifySchemaCharsetAndCollate: + shouldIgnoreTableOrSchema = !f.filter.MatchSchema(schema) + default: + shouldIgnoreTableOrSchema = f.ShouldIgnoreTable(schema, table) + } + return f.shouldIgnoreStartTs(ts) || shouldIgnoreTableOrSchema } // ShouldDiscardDDL returns true if this DDL should be discarded @@ -144,7 +151,9 @@ func (f *Filter) shouldDiscardByBuiltInDDLAllowlist(ddlType model.ActionType) bo model.ActionRecoverTable, model.ActionModifySchemaCharsetAndCollate, model.ActionAddPrimaryKey, - model.ActionDropPrimaryKey: + model.ActionDropPrimaryKey, + model.ActionAddColumns, + model.ActionDropColumns: return false } return true @@ -152,11 +161,5 @@ func (f *Filter) shouldDiscardByBuiltInDDLAllowlist(ddlType model.ActionType) bo // IsSysSchema returns true if the given schema is a system schema func IsSysSchema(db string) bool { - db = strings.ToUpper(db) - for _, schema := range []string{"INFORMATION_SCHEMA", "PERFORMANCE_SCHEMA", "MYSQL", "METRIC_SCHEMA"} { - if schema == db { - return true - } - } - return false + return filterV1.IsSystemSchema(db) } diff --git a/pkg/filter/filter_test.go b/pkg/filter/filter_test.go index 01de50e39f5..5c0f1b3c098 100644 --- a/pkg/filter/filter_test.go +++ b/pkg/filter/filter_test.go @@ -34,7 +34,7 @@ func (s *filterSuite) TestShouldUseDefaultRules(c *check.C) { 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) - c.Assert(filter.ShouldIgnoreTable("metric_schema", "query_duration"), check.IsTrue) + c.Assert(filter.ShouldIgnoreTable("metric_schema", "query_duration"), check.IsFalse) c.Assert(filter.ShouldIgnoreTable("sns", "user"), check.IsFalse) c.Assert(filter.ShouldIgnoreTable("tidb_cdc", "repl_mark_a_a"), check.IsFalse) } @@ -62,33 +62,58 @@ func (s *filterSuite) TestShouldUseCustomRules(c *check.C) { } func (s *filterSuite) TestShouldIgnoreTxn(c *check.C) { - filter, err := NewFilter(&config.ReplicaConfig{ - Filter: &config.FilterConfig{ - IgnoreTxnStartTs: []uint64{1, 3}, - Rules: []string{"sns.*", "ecom.*", "!sns.log", "!ecom.test"}, - }, - }) - c.Assert(err, check.IsNil) testCases := []struct { - schema string - table string - ts uint64 - ignore bool + cases []struct { + schema string + table string + ts uint64 + ignore bool + } + ignoreTxnStartTs []uint64 + rules []string }{ - {"sns", "ttta", 1, true}, - {"ecom", "aabb", 2, false}, - {"sns", "log", 3, true}, - {"sns", "log", 4, true}, - {"ecom", "test", 5, true}, - {"test", "test", 6, true}, - {"ecom", "log", 6, false}, + {cases: []struct { + schema string + table string + ts uint64 + ignore bool + }{{"sns", "ttta", 1, true}, + {"ecom", "aabb", 2, false}, + {"sns", "log", 3, true}, + {"sns", "log", 4, true}, + {"ecom", "test", 5, true}, + {"test", "test", 6, true}, + {"ecom", "log", 6, false}}, + ignoreTxnStartTs: []uint64{1, 3}, + rules: []string{"sns.*", "ecom.*", "!sns.log", "!ecom.test"}, + }, + {cases: []struct { + schema string + table string + ts uint64 + ignore bool + }{{"S", "D1", 1, true}, + {"S", "Da", 1, false}, + {"S", "Db", 1, false}, + {"S", "Daa", 1, false}}, + ignoreTxnStartTs: []uint64{}, + rules: []string{"*.*", "!S.D[!a-d]"}, + }, } - for _, tc := range testCases { - c.Assert(filter.ShouldIgnoreDMLEvent(tc.ts, tc.schema, tc.table), check.Equals, tc.ignore) - c.Assert(filter.ShouldIgnoreDDLEvent(tc.ts, tc.schema, tc.table), check.Equals, tc.ignore) + for _, ftc := range testCases { + filter, err := NewFilter(&config.ReplicaConfig{ + Filter: &config.FilterConfig{ + IgnoreTxnStartTs: ftc.ignoreTxnStartTs, + Rules: ftc.rules, + }, + }) + c.Assert(err, check.IsNil) + for _, tc := range ftc.cases { + c.Assert(filter.ShouldIgnoreDMLEvent(tc.ts, tc.schema, tc.table), check.Equals, tc.ignore) + c.Assert(filter.ShouldIgnoreDDLEvent(tc.ts, model.ActionCreateTable, tc.schema, tc.table), check.Equals, tc.ignore) + } } - } func (s *filterSuite) TestShouldDiscardDDL(c *check.C) { @@ -103,3 +128,52 @@ func (s *filterSuite) TestShouldDiscardDDL(c *check.C) { c.Assert(filter.ShouldDiscardDDL(model.ActionAddForeignKey), check.IsFalse) c.Assert(filter.ShouldDiscardDDL(model.ActionCreateSequence), check.IsTrue) } + +func (s *filterSuite) TestShouldIgnoreDDL(c *check.C) { + testCases := []struct { + cases []struct { + schema string + table string + ddlType model.ActionType + ignore bool + } + rules []string + }{{cases: []struct { + schema string + table string + ddlType model.ActionType + ignore bool + }{{"sns", "", model.ActionCreateSchema, false}, + {"sns", "", model.ActionDropSchema, false}, + {"sns", "", model.ActionModifySchemaCharsetAndCollate, false}, + {"ecom", "", model.ActionCreateSchema, false}, + {"ecom", "aa", model.ActionCreateTable, false}, + {"ecom", "", model.ActionCreateSchema, false}, + {"test", "", model.ActionCreateSchema, true}}, + rules: []string{"sns.*", "ecom.*", "!sns.log", "!ecom.test"}, + }, {cases: []struct { + schema string + table string + ddlType model.ActionType + ignore bool + }{{"sns", "", model.ActionCreateSchema, false}, + {"sns", "", model.ActionDropSchema, false}, + {"sns", "", model.ActionModifySchemaCharsetAndCollate, false}, + {"sns", "aa", model.ActionCreateTable, true}, + {"sns", "C1", model.ActionCreateTable, false}, + {"sns", "", model.ActionCreateTable, true}}, + rules: []string{"sns.C1"}, + }} + for _, ftc := range testCases { + filter, err := NewFilter(&config.ReplicaConfig{ + Filter: &config.FilterConfig{ + IgnoreTxnStartTs: []uint64{}, + Rules: ftc.rules, + }, + }) + c.Assert(err, check.IsNil) + for _, tc := range ftc.cases { + c.Assert(filter.ShouldIgnoreDDLEvent(1, tc.ddlType, tc.schema, tc.table), check.Equals, tc.ignore, check.Commentf("%#v", tc)) + } + } +} diff --git a/pkg/scheduler/table_number.go b/pkg/scheduler/table_number.go index f5658e5da54..99ae38c5f18 100644 --- a/pkg/scheduler/table_number.go +++ b/pkg/scheduler/table_number.go @@ -49,12 +49,12 @@ func (t *TableNumberScheduler) CalRebalanceOperates(targetSkewness float64) ( for _, captureWorkloads := range t.workloads { totalTableNumber += uint64(len(captureWorkloads)) } - limitTableNumber := (totalTableNumber / uint64(len(t.workloads))) + 1 + limitTableNumber := (float64(totalTableNumber) / float64(len(t.workloads))) + 1 appendTables := make(map[model.TableID]model.Ts) moveTableJobs = make(map[model.TableID]*model.MoveTableJob) for captureID, captureWorkloads := range t.workloads { - for uint64(len(captureWorkloads)) > limitTableNumber { + for float64(len(captureWorkloads)) >= limitTableNumber { for tableID := range captureWorkloads { // find a table in this capture appendTables[tableID] = 0 diff --git a/pkg/scheduler/table_number_test.go b/pkg/scheduler/table_number_test.go index bc15a090c1f..8600725196b 100644 --- a/pkg/scheduler/table_number_test.go +++ b/pkg/scheduler/table_number_test.go @@ -84,4 +84,22 @@ func (s *tableNumberSuite) TestCalRebalanceOperates(c *check.C) { } c.Assert(fmt.Sprintf("%.2f%%", skewness*100), check.Equals, "14.14%") + + scheduler.ResetWorkloads("capture1", model.TaskWorkload{ + 1: model.WorkloadInfo{Workload: 1}, + 2: model.WorkloadInfo{Workload: 1}, + 3: model.WorkloadInfo{Workload: 1}}) + scheduler.ResetWorkloads("capture2", model.TaskWorkload{}) + scheduler.ResetWorkloads("capture3", model.TaskWorkload{}) + c.Assert(fmt.Sprintf("%.2f%%", scheduler.Skewness()*100), check.Equals, "141.42%") + skewness, moveJobs = scheduler.CalRebalanceOperates(0) + + for tableID, job := range moveJobs { + c.Assert(len(job.From), check.Greater, 0) + c.Assert(len(job.To), check.Greater, 0) + c.Assert(job.TableID, check.Equals, tableID) + c.Assert(job.From, check.Not(check.Equals), job.To) + c.Assert(job.Status, check.Equals, model.MoveTableStatusNone) + } + c.Assert(fmt.Sprintf("%.2f%%", skewness*100), check.Equals, "0.00%") } diff --git a/pkg/security/credential.go b/pkg/security/credential.go index 229c9e5c18b..2b538ed3770 100644 --- a/pkg/security/credential.go +++ b/pkg/security/credential.go @@ -31,6 +31,11 @@ type Credential struct { CertAllowedCN []string `toml:"cert-allowed-cn" json:"cert-allowed-cn"` } +// IsTLSEnabled checks whether TLS is enabled or not. +func (s *Credential) IsTLSEnabled() bool { + return len(s.CAPath) != 0 +} + // PDSecurityOption creates a new pd SecurityOption from Security func (s *Credential) PDSecurityOption() pd.SecurityOption { return pd.SecurityOption{ diff --git a/pkg/util/log_test.go b/pkg/util/log_test.go index 9c7f62119d5..cfb830c7b5b 100644 --- a/pkg/util/log_test.go +++ b/pkg/util/log_test.go @@ -15,7 +15,7 @@ package util import ( "context" - "path" + "path/filepath" "testing" "github.com/pingcap/check" @@ -34,7 +34,7 @@ type logSuite struct{} var _ = check.Suite(&logSuite{}) func (s *logSuite) TestInitLogger(c *check.C) { - f := path.Join(c.MkDir(), "test") + f := filepath.Join(c.MkDir(), "test") cfg := &Config{ Level: "warning", File: f, diff --git a/proto/generate-proto.sh b/proto/generate-proto.sh index a7357a8fe1f..6dbd935be72 100755 --- a/proto/generate-proto.sh +++ b/proto/generate-proto.sh @@ -3,6 +3,7 @@ echo "generate canal protocol code..." [ ! -d ./canal ] && mkdir ./canal +[ ! -d ./cdclog ] && mkdir ./cdclog protoc --gofast_out=./canal EntryProtocol.proto protoc --gofast_out=./canal CanalProtocol.proto diff --git a/scripts/avro-local-test.sh b/scripts/avro-local-test.sh index aa223213776..c1e0af7f3b7 100755 --- a/scripts/avro-local-test.sh +++ b/scripts/avro-local-test.sh @@ -49,7 +49,7 @@ sub_up() { sub_down() { sudo docker-compose -f ../docker-compose-avro.yml down - sudo rm -r ../docker/log ../docker/data + sudo rm -r ../docker/logs ../docker/data } subcommand=$1 diff --git a/scripts/fix_lib_zstd.sh b/scripts/fix_lib_zstd.sh new file mode 100755 index 00000000000..9df1fb2e4a4 --- /dev/null +++ b/scripts/fix_lib_zstd.sh @@ -0,0 +1,19 @@ +#!/bin/bash + +user=$(whoami) +GOOS=$(go env GOOS) +GOARCH=$(go env GOARCH) +module="github.com/valyala/gozstd@v1.7.0" + +GO111MODULE=on go mod download ${module} +# In CI environment, the gopath contains multiple dirs, choose the first one +cd $(echo $GOPATH|awk -F':' '{print $1}')/pkg/mod/${module} +sudo MOREFLAGS=-fPIC make clean libzstd.a + +sudo which go +if [[ $? != 0 ]]; then + lib_name=libzstd_${GOOS}_${GOARCH}.a + echo "mv libzstd__.a ${lib_name}" + sudo mv libzstd__.a ${lib_name} + sudo chown ${user}:${user} ${lib_name} +fi diff --git a/scripts/jenkins_ci/integration_test_common.groovy b/scripts/jenkins_ci/integration_test_common.groovy index dcdf4204a75..e8465a93610 100644 --- a/scripts/jenkins_ci/integration_test_common.groovy +++ b/scripts/jenkins_ci/integration_test_common.groovy @@ -34,6 +34,7 @@ def prepare_binaries() { node ("${GO_TEST_SLAVE}") { deleteDir() container("golang") { + println "debug command:\nkubectl -n jenkins-ci exec -ti ${NODE_NAME} bash" def tidb_sha1 = sh(returnStdout: true, script: "curl ${FILE_SERVER_URL}/download/refs/pingcap/tidb/${TIDB_BRANCH}/sha1").trim() def tikv_sha1 = sh(returnStdout: true, script: "curl ${FILE_SERVER_URL}/download/refs/pingcap/tikv/${TIKV_BRANCH}/sha1").trim() def pd_sha1 = sh(returnStdout: true, script: "curl ${FILE_SERVER_URL}/download/refs/pingcap/pd/${PD_BRANCH}/sha1").trim() @@ -44,15 +45,17 @@ def prepare_binaries() { tidb_url="${FILE_SERVER_URL}/download/builds/pingcap/tidb/${tidb_sha1}/centos7/tidb-server.tar.gz" tikv_url="${FILE_SERVER_URL}/download/builds/pingcap/tikv/${tikv_sha1}/centos7/tikv-server.tar.gz" pd_url="${FILE_SERVER_URL}/download/builds/pingcap/pd/${pd_sha1}/centos7/pd-server.tar.gz" + minio_url="${FILE_SERVER_URL}/download/minio.tar.gz" curl \${tidb_url} | tar xz -C ./tmp bin/tidb-server curl \${pd_url} | tar xz -C ./tmp bin/* curl \${tikv_url} | tar xz -C ./tmp bin/tikv-server + curl \${minio_url} | tar xz -C ./tmp/bin minio mv tmp/bin/* third_bin curl http://download.pingcap.org/tiflash-nightly-linux-amd64.tar.gz | tar xz -C third_bin mv third_bin/tiflash-nightly-linux-amd64/* third_bin curl ${FILE_SERVER_URL}/download/builds/pingcap/go-ycsb/test-br/go-ycsb -o third_bin/go-ycsb - curl -L https://github.com/etcd-io/etcd/releases/download/v3.4.7/etcd-v3.4.7-linux-amd64.tar.gz | tar xz -C ./tmp + curl -L http://fileserver.pingcap.net/download/builds/pingcap/cdc/etcd-v3.4.7-linux-amd64.tar.gz | tar xz -C ./tmp mv tmp/etcd-v3.4.7-linux-amd64/etcdctl third_bin curl https://download.pingcap.org/tidb-tools-v2.1.6-linux-amd64.tar.gz | tar xz -C ./tmp tidb-tools-v2.1.6-linux-amd64/bin/sync_diff_inspector mv tmp/tidb-tools-v2.1.6-linux-amd64/bin/* third_bin @@ -70,6 +73,7 @@ def prepare_binaries() { prepares["build binaries"] = { node ("${GO_TEST_SLAVE}") { container("golang") { + println "debug command:\nkubectl -n jenkins-ci exec -ti ${NODE_NAME} bash" def ws = pwd() deleteDir() unstash 'ticdc' @@ -139,6 +143,7 @@ def tests(sink_type, node_label) { sh "mv ${ws}/third_bin/* ./bin/" try { sh """ + sudo pip install s3cmd rm -rf /tmp/tidb_cdc_test mkdir -p /tmp/tidb_cdc_test GO111MODULE=off GOPATH=\$GOPATH:${ws}/go PATH=\$GOPATH/bin:${ws}/go/bin:\$PATH make integration_test_${sink_type} CASE=${case_name} diff --git a/tests/_utils/start_tidb_cluster b/tests/_utils/start_tidb_cluster index d51c3b98b7e..a308179efa4 100755 --- a/tests/_utils/start_tidb_cluster +++ b/tests/_utils/start_tidb_cluster @@ -9,6 +9,7 @@ set -e OUT_DIR= tidb_config= retry_times=3 +multiple_upstream_pd="false" while [[ ${1} ]]; do case "${1}" in @@ -24,6 +25,10 @@ while [[ ${1} ]]; do retry_times=${2} shift ;; + --multiple-upstream-pd) + multiple_upstream_pd=${2} + shift + ;; *) echo "Unknown parameter: ${1}" >&2 exit 1 @@ -45,9 +50,9 @@ do echo "The ${i} times to try to start tidb cluster..." if [[ "$tidb_config" != "" ]]; then - start_tidb_cluster_impl --workdir ${OUT_DIR} --tidb-config ${tidb_config} + start_tidb_cluster_impl --workdir ${OUT_DIR} --multiple-upstream-pd ${multiple_upstream_pd} --tidb-config ${tidb_config} else - start_tidb_cluster_impl --workdir ${OUT_DIR} + start_tidb_cluster_impl --workdir ${OUT_DIR} --multiple-upstream-pd ${multiple_upstream_pd} fi if [ $? -eq 0 ]; then @@ -55,4 +60,4 @@ do fi let i++ echo "start tidb cluster failed" -done \ No newline at end of file +done diff --git a/tests/_utils/start_tidb_cluster_impl b/tests/_utils/start_tidb_cluster_impl index 68ea1b94ba3..ad97baacebb 100755 --- a/tests/_utils/start_tidb_cluster_impl +++ b/tests/_utils/start_tidb_cluster_impl @@ -2,11 +2,13 @@ # --workdir: work directory # --tidb-config: path to tidb config file +# --multiple-upstream-pd: whether to deploy multiple pd severs in upstream set -e OUT_DIR= tidb_config= +multiple_upstream_pd= while [[ ${1} ]]; do case "${1}" in @@ -18,6 +20,10 @@ while [[ ${1} ]]; do tidb_config=${2} shift ;; + --multiple-upstream-pd) + multiple_upstream_pd=${2} + shift + ;; *) echo "Unknown parameter: ${1}" >&2 exit 1 @@ -44,14 +50,28 @@ enable-placement-rules = true EOF echo "Starting Upstream PD..." -pd-server \ - --advertise-client-urls http://${UP_PD_HOST}:${UP_PD_PORT} \ - --client-urls http://0.0.0.0:${UP_PD_PORT} \ - --advertise-peer-urls http://${UP_PD_HOST}:${UP_PD_PEER_PORT} \ - --peer-urls http://0.0.0.0:${UP_PD_PEER_PORT} \ - --config "$OUT_DIR/pd-config.toml" \ - --log-file "$OUT_DIR/pd.log" \ - --data-dir "$OUT_DIR/pd" & +if [[ "$multiple_upstream_pd" == "true" ]]; then + pd_count=3 + initial_cluster="pd1=http://${UP_PD_HOST_1}:${UP_PD_PEER_PORT_1},pd2=http://${UP_PD_HOST_2}:${UP_PD_PEER_PORT_2},pd3=http://${UP_PD_HOST_3}:${UP_PD_PEER_PORT_3}" +else + pd_count=1 + initial_cluster="pd1=http://${UP_PD_HOST_1}:${UP_PD_PEER_PORT_1}" +fi +for idx in $(seq 1 $pd_count); do + host="UP_PD_HOST_$idx" + port="UP_PD_PORT_$idx" + peer_port="UP_PD_PEER_PORT_$idx" + pd-server \ + --advertise-client-urls http://${!host}:${!port} \ + --client-urls http://0.0.0.0:${!port} \ + --advertise-peer-urls http://${!host}:${!peer_port} \ + --peer-urls http://0.0.0.0:${!peer_port} \ + --config "$OUT_DIR/pd-config.toml" \ + --log-file "$OUT_DIR/pd$idx.log" \ + --data-dir "$OUT_DIR/pd$idx" \ + --name="pd$idx" \ + --initial-cluster=${initial_cluster} & +done echo "Starting Downstream PD..." pd-server \ @@ -64,15 +84,20 @@ pd-server \ --data-dir "$OUT_DIR/down_pd" & # wait until PD is online... -while ! curl -o /dev/null -sf http://${UP_PD_HOST}:${UP_PD_PORT}/pd/api/v1/version; do - sleep 1 -done +for idx in $(seq 1 $pd_count); do + host="UP_PD_HOST_$idx" + port="UP_PD_PORT_$idx" -while ! curl -o /dev/null -sf http://${DOWN_PD_HOST}:${DOWN_PD_PORT}/pd/api/v1/version; do - sleep 1 + while ! curl -o /dev/null -sf http://${!host}:${!port}/pd/api/v1/version; do + sleep 1 + done + + while [ -z "$(curl http://${!host}:${!port}/pd/health 2> /dev/null | grep 'health' | grep 'true')" ]; do + sleep 1 + done done -while [ -z "$(curl http://${UP_PD_HOST}:${UP_PD_PORT}/pd/health 2> /dev/null | grep 'health' | grep 'true')" ]; do +while ! curl -o /dev/null -sf http://${DOWN_PD_HOST}:${DOWN_PD_PORT}/pd/api/v1/version; do sleep 1 done @@ -109,7 +134,7 @@ for idx in $(seq 1 3); do port="UP_TIKV_PORT_$idx" status_port="UP_TIKV_STATUS_PORT_$idx" tikv-server \ - --pd ${UP_PD_HOST}:${UP_PD_PORT} \ + --pd ${UP_PD_HOST_1}:${UP_PD_PORT_1} \ -A ${!host}:${!port} \ --status-addr ${!host}:${!status_port} \ --log-file "$OUT_DIR/tikv$idx.log" \ @@ -135,7 +160,7 @@ tidb-server \ -P ${UP_TIDB_PORT} \ -config "$OUT_DIR/tidb-config.toml" \ --store tikv \ - --path ${UP_PD_HOST}:${UP_PD_PORT} \ + --path ${UP_PD_HOST_1}:${UP_PD_PORT_1} \ --status=${UP_TIDB_STATUS} \ --log-file "$OUT_DIR/tidb.log" & @@ -143,7 +168,7 @@ tidb-server \ -P ${UP_TIDB_OTHER_PORT} \ -config "$OUT_DIR/tidb-config.toml" \ --store tikv \ - --path ${UP_PD_HOST}:${UP_PD_PORT} \ + --path ${UP_PD_HOST_1}:${UP_PD_PORT_1} \ --status=${UP_TIDB_OTHER_STATUS} \ --log-file "$OUT_DIR/tidb_other.log" & @@ -233,7 +258,7 @@ runAsDaemon = true [raft] kvstore_path = "${OUT_DIR}/tiflash/kvstore" -pd_addr = "${UP_PD_HOST}:${UP_PD_PORT}" +pd_addr = "${UP_PD_HOST_1}:${UP_PD_PORT_1}" ignore_databases = "system,default" storage_engine = "tmt" EOF diff --git a/tests/_utils/stop_tidb_cluster b/tests/_utils/stop_tidb_cluster index 86da17eafa4..3955354fea4 100755 --- a/tests/_utils/stop_tidb_cluster +++ b/tests/_utils/stop_tidb_cluster @@ -16,8 +16,12 @@ kill -9 $(lsof -i tcp:${UP_TIDB_STATUS} -t 2>/dev/null) &>/dev/null || true kill -9 $(lsof -i tcp:${UP_TIDB_OTHER_STATUS} -t 2>/dev/null) &>/dev/null || true kill -9 $(lsof -i tcp:${DOWN_TIDB_PORT} -t 2>/dev/null) &>/dev/null || true kill -9 $(lsof -i tcp:${DOWN_TIDB_STATUS} -t 2>/dev/null) &>/dev/null || true -kill -9 $(lsof -i tcp:${UP_PD_PORT} -t 2>/dev/null) &>/dev/null || true -kill -9 $(lsof -i tcp:${UP_PD_PEER_PORT} -t 2>/dev/null) &>/dev/null || true +kill -9 $(lsof -i tcp:${UP_PD_PORT_1} -t 2>/dev/null) &>/dev/null || true +kill -9 $(lsof -i tcp:${UP_PD_PEER_PORT_1} -t 2>/dev/null) &>/dev/null || true +kill -9 $(lsof -i tcp:${UP_PD_PORT_2} -t 2>/dev/null) &>/dev/null || true +kill -9 $(lsof -i tcp:${UP_PD_PEER_PORT_2} -t 2>/dev/null) &>/dev/null || true +kill -9 $(lsof -i tcp:${UP_PD_PORT_3} -t 2>/dev/null) &>/dev/null || true +kill -9 $(lsof -i tcp:${UP_PD_PEER_PORT_3} -t 2>/dev/null) &>/dev/null || true kill -9 $(lsof -i tcp:${DOWN_PD_PORT} -t 2>/dev/null) &>/dev/null || true kill -9 $(lsof -i tcp:${DOWN_PD_PEER_PORT} -t 2>/dev/null) &>/dev/null || true kill -9 $(lsof -i tcp:${UP_TIKV_PORT_1} -t 2>/dev/null) &>/dev/null || true diff --git a/tests/_utils/test_prepare b/tests/_utils/test_prepare index e573f2a3dd2..21382b213b8 100644 --- a/tests/_utils/test_prepare +++ b/tests/_utils/test_prepare @@ -10,9 +10,15 @@ TLS_TIDB_HOST=${TLS_TIDB_HOST:-127.0.0.1} TLS_TIDB_PORT=${TLS_TIDB_PORT:-3307} TLS_TIDB_STATUS=${TLS_TIDB_STATUS:-30080} -UP_PD_HOST=${UP_PD_HOST:-127.0.0.1} -UP_PD_PORT=${UP_PD_PORT:-2379} -UP_PD_PEER_PORT=${UP_PD_PEER_PORT:-2380} +UP_PD_HOST_1=${UP_PD_HOST_1:-127.0.0.1} +UP_PD_PORT_1=${UP_PD_PORT_1:-2379} +UP_PD_PEER_PORT_1=${UP_PD_PEER_PORT_1:-2380} +UP_PD_HOST_2=${UP_PD_HOST_2:-127.0.0.1} +UP_PD_PORT_2=${UP_PD_PORT_2:-2679} +UP_PD_PEER_PORT_2=${UP_PD_PEER_PORT_2:-2680} +UP_PD_HOST_3=${UP_PD_HOST_3:-127.0.0.1} +UP_PD_PORT_3=${UP_PD_PORT_3:-2779} +UP_PD_PEER_PORT_3=${UP_PD_PEER_PORT_3:-2780} DOWN_PD_HOST=${DOWN_PD_HOST:-127.0.0.1} DOWN_PD_PORT=${DOWN_PD_PORT:-2479} DOWN_PD_PEER_PORT=${DOWN_PD_PEER_PORT:-2480} diff --git a/tests/autorandom/run.sh b/tests/autorandom/run.sh index e08fa6e2595..a89be6cc865 100644 --- a/tests/autorandom/run.sh +++ b/tests/autorandom/run.sh @@ -16,7 +16,7 @@ function run() { cd $WORK_DIR # record tso before we create tables to skip the system table DDLs - start_ts=$(cdc cli tso query --pd=http://$UP_PD_HOST:$UP_PD_PORT) + start_ts=$(cdc cli tso query --pd=http://$UP_PD_HOST_1:$UP_PD_PORT_1) run_cdc_server --workdir $WORK_DIR --binary $CDC_BINARY diff --git a/tests/availability/run.sh b/tests/availability/run.sh index 9f2e9a232f4..76c0d9c74b5 100644 --- a/tests/availability/run.sh +++ b/tests/availability/run.sh @@ -21,7 +21,7 @@ function prepare() { cd $WORK_DIR # record tso before we create tables to skip the system table DDLs - start_ts=$(run_cdc_cli tso query --pd=http://$UP_PD_HOST:$UP_PD_PORT) + start_ts=$(run_cdc_cli tso query --pd=http://$UP_PD_HOST_1:$UP_PD_PORT_1) run_sql "CREATE table test.availability1(id int primary key, val int);" run_sql "CREATE table test.availability2(id int primary key, val int);" diff --git a/tests/batch_add_table/run.sh b/tests/batch_add_table/run.sh index f53aef53dc1..11443e94fe9 100644 --- a/tests/batch_add_table/run.sh +++ b/tests/batch_add_table/run.sh @@ -18,7 +18,7 @@ function run() { run_sql_file $CUR/data/prepare.sql ${UP_TIDB_HOST} ${UP_TIDB_PORT} run_sql_file $CUR/data/prepare.sql ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} # record tso before we create tables to skip the system table DDLs - start_ts=$(run_cdc_cli tso query --pd=http://$UP_PD_HOST:$UP_PD_PORT) + start_ts=$(run_cdc_cli tso query --pd=http://$UP_PD_HOST_1:$UP_PD_PORT_1) run_cdc_server --workdir $WORK_DIR --binary $CDC_BINARY diff --git a/tests/cdc/run.sh b/tests/cdc/run.sh index 324cb3bfb24..3930d2ac6bd 100755 --- a/tests/cdc/run.sh +++ b/tests/cdc/run.sh @@ -17,7 +17,7 @@ function prepare() { cd $WORK_DIR # record tso before we create tables to skip the system table DDLs - start_ts=$(run_cdc_cli tso query --pd=http://$UP_PD_HOST:$UP_PD_PORT) + start_ts=$(run_cdc_cli tso query --pd=http://$UP_PD_HOST_1:$UP_PD_PORT_1) run_cdc_server --workdir $WORK_DIR --binary $CDC_BINARY diff --git a/tests/cdclog_file/run.sh b/tests/cdclog_file/run.sh new file mode 100644 index 00000000000..2cd62336789 --- /dev/null +++ b/tests/cdclog_file/run.sh @@ -0,0 +1,84 @@ +#!/bin/bash + +set -e + +CUR=$( cd "$( dirname "${BASH_SOURCE[0]}" )" && pwd ) +source $CUR/../_utils/test_prepare +WORK_DIR=$OUT_DIR/$TEST_NAME +CDC_BINARY=cdc.test + + +function prepare() { + rm -rf "$WORK_DIR" + mkdir -p "$WORK_DIR" + stop_tidb_cluster + start_tidb_cluster --workdir $WORK_DIR + + cd $WORK_DIR + + # record tso before we create tables to skip the system table DDLs + start_ts=$(run_cdc_cli tso query --pd=http://$UP_PD_HOST_1:$UP_PD_PORT_1) + + run_cdc_server --workdir $WORK_DIR --binary $CDC_BINARY + + SINK_URI="local://$WORK_DIR/test?" + + run_cdc_cli changefeed create --start-ts=$start_ts --sink-uri="$SINK_URI" +} + +success=0 +function check_cdclog() { + # TODO test rotate file + DATA_DIR="$WORK_DIR/test" + # retrieve table id by log meta + if [ ! -f $DATA_DIR/log.meta ]; then + return + fi + table_id=$(cat $DATA_DIR/log.meta | jq | grep t1 | awk -F '"' '{print $2}') + if [ ! -d $DATA_DIR/t_$table_id ]; then + return + fi + file_count=$(ls -ahl $DATA_DIR/t_$table_id | grep cdclog | wc -l) + if [[ ! "$file_count" -eq "1" ]]; then + echo "$TEST_NAME failed, expect 1 row changed files, obtain $file_count" + return + fi + if [ ! -d $DATA_DIR/ddls ]; then + return + fi + ddl_file_count=$(ls -ahl $DATA_DIR/ddls | grep ddl | wc -l) + if [[ ! "$ddl_file_count" -eq "1" ]]; then + echo "$TEST_NAME failed, expect 1 ddl file, obtain $ddl_file_count" + return + fi + success=1 +} + +function cdclog_test() { + run_sql "drop database if exists $TEST_NAME" + run_sql "create database $TEST_NAME" + run_sql "create table $TEST_NAME.t1 (c0 int primary key, payload varchar(1024));" + run_sql "create table $TEST_NAME.t2 (c0 int primary key, payload varchar(1024));" + + run_sql "insert into $TEST_NAME.t1 values (1, 'a')" + run_sql "insert into $TEST_NAME.t1 values (2, 'b')" + + i=0 + while [ $i -lt 30 ] + do + check_cdclog + if [ "$success" == 1 ]; then + echo "check log successfully" + break + fi + i=$(( $i + 1 )) + echo "check log failed $i-th time, retry later" + sleep 2 + done + cleanup_process $CDC_BINARY +} + +trap stop_tidb_cluster EXIT +prepare $* +cdclog_test $* +echo "[$(date)] <<<<<< run test case $TEST_NAME success! >>>>>>" diff --git a/tests/cdclog_s3/run.sh b/tests/cdclog_s3/run.sh new file mode 100644 index 00000000000..990c2a895c1 --- /dev/null +++ b/tests/cdclog_s3/run.sh @@ -0,0 +1,115 @@ +#!/bin/bash + +set -e + +CUR=$( cd "$( dirname "${BASH_SOURCE[0]}" )" && pwd ) +source $CUR/../_utils/test_prepare +WORK_DIR=$OUT_DIR/$TEST_NAME +CDC_BINARY=cdc.test + +# start the s3 server +export MINIO_ACCESS_KEY=cdcs3accesskey +export MINIO_SECRET_KEY=cdcs3secretkey +export MINIO_BROWSER=off +export AWS_ACCESS_KEY_ID=$MINIO_ACCESS_KEY +export AWS_SECRET_ACCESS_KEY=$MINIO_SECRET_KEY +export S3_ENDPOINT=127.0.0.1:24927 +rm -rf "$WORK_DIR" +mkdir -p "$WORK_DIR" +bin/minio server --address $S3_ENDPOINT "$WORK_DIR/s3" & +MINIO_PID=$! +i=0 +while ! curl -o /dev/null -v -s "http://$S3_ENDPOINT/"; do + i=$(($i+1)) + if [ $i -gt 7 ]; then + echo 'Failed to start minio' + exit 1 + fi + sleep 2 +done + +stop_minio() { + kill -2 $MINIO_PID +} + +stop() { + stop_minio + stop_tidb_cluster +} + +s3cmd --access_key=$MINIO_ACCESS_KEY --secret_key=$MINIO_SECRET_KEY --host=$S3_ENDPOINT --host-bucket=$S3_ENDPOINT --no-ssl mb s3://logbucket + +function prepare() { + stop_tidb_cluster + start_tidb_cluster --workdir $WORK_DIR + + cd $WORK_DIR + + # record tso before we create tables to skip the system table DDLs + start_ts=$(run_cdc_cli tso query --pd=http://$UP_PD_HOST_1:$UP_PD_PORT_1) + + run_cdc_server --workdir $WORK_DIR --binary $CDC_BINARY + + SINK_URI="s3://logbucket/test?endpoint=http://$S3_ENDPOINT/" + + run_cdc_cli changefeed create --start-ts=$start_ts --sink-uri="$SINK_URI" +} + +success=0 +function check_cdclog() { + DATA_DIR="$WORK_DIR/s3/logbucket/test" + # retrieve table id by log meta + if [ ! -f $DATA_DIR/log.meta ]; then + return + fi + table_id=$(cat $DATA_DIR/log.meta | jq | grep t1 | awk -F '"' '{print $2}') + if [ ! -d $DATA_DIR/t_$table_id ]; then + return + fi + file_count=$(ls -ahl $DATA_DIR/t_$table_id | grep cdclog | wc -l) + if [[ ! "$file_count" -eq "2" ]]; then + echo "$TEST_NAME failed, expect 2 row changed files, obtain $file_count" + return + fi + if [ ! -d $DATA_DIR/ddls ]; then + return + fi + ddl_file_count=$(ls -ahl $DATA_DIR/ddls | grep ddl | wc -l) + if [[ ! "$ddl_file_count" -eq "1" ]]; then + echo "$TEST_NAME failed, expect 1 ddl file, obtain $ddl_file_count" + return + fi + success=1 +} + +function cdclog_test() { + run_sql "drop database if exists $TEST_NAME" + run_sql "create database $TEST_NAME" + run_sql "create table $TEST_NAME.t1 (c0 int primary key, payload varchar(1024));" + run_sql "create table $TEST_NAME.t2 (c0 int primary key, payload varchar(1024));" + + run_sql "insert into $TEST_NAME.t1 values (1, 'a')" + # because flush row changed events interval is 5 second + # so sleep 20 second will generate two files + sleep 20 + run_sql "insert into $TEST_NAME.t1 values (2, 'b')" + + i=0 + while [ $i -lt 30 ] + do + check_cdclog + if [ "$success" == 1 ]; then + echo "check log successfully" + break + fi + i=$(( $i + 1 )) + echo "check log failed $i-th time, retry later" + sleep 2 + done + cleanup_process $CDC_BINARY +} + +trap stop EXIT +prepare $* +cdclog_test $* +echo "[$(date)] <<<<<< run test case $TEST_NAME success! >>>>>>" diff --git a/tests/changefeed_auto_stop/run.sh b/tests/changefeed_auto_stop/run.sh index 84730de2130..07ba4bb4db8 100755 --- a/tests/changefeed_auto_stop/run.sh +++ b/tests/changefeed_auto_stop/run.sh @@ -34,7 +34,7 @@ function run() { rm -rf $WORK_DIR && mkdir -p $WORK_DIR start_tidb_cluster --workdir $WORK_DIR cd $WORK_DIR - start_ts=$(cdc cli tso query --pd=http://$UP_PD_HOST:$UP_PD_PORT) + start_ts=$(cdc cli tso query --pd=http://$UP_PD_HOST_1:$UP_PD_PORT_1) for i in $(seq $DB_COUNT); do db="changefeed_auto_stop_$i" @@ -42,9 +42,9 @@ function run() { go-ycsb load mysql -P $CUR/conf/workload -p mysql.host=${UP_TIDB_HOST} -p mysql.port=${UP_TIDB_PORT} -p mysql.user=root -p mysql.db=$db done - run_cdc_server --workdir $WORK_DIR --binary $CDC_BINARY --logsuffix "1" --addr "127.0.0.1:8301" --pd "http://${UP_PD_HOST}:${UP_PD_PORT}" + run_cdc_server --workdir $WORK_DIR --binary $CDC_BINARY --logsuffix "1" --addr "127.0.0.1:8301" --pd "http://${UP_PD_HOST_1}:${UP_PD_PORT_1}" export GO_FAILPOINTS='github.com/pingcap/ticdc/cdc/ProcessorSyncResolvedError=1*return(true);github.com/pingcap/ticdc/cdc/ProcessorUpdatePositionDelaying=return(true)' - run_cdc_server --workdir $WORK_DIR --binary $CDC_BINARY --logsuffix "2" --addr "127.0.0.1:8302" --pd "http://${UP_PD_HOST}:${UP_PD_PORT}" + run_cdc_server --workdir $WORK_DIR --binary $CDC_BINARY --logsuffix "2" --addr "127.0.0.1:8302" --pd "http://${UP_PD_HOST_1}:${UP_PD_PORT_1}" export GO_FAILPOINTS='' TOPIC_NAME="ticdc-changefeed-auto-stop-test-$RANDOM" @@ -52,14 +52,14 @@ function run() { kafka) SINK_URI="kafka://127.0.0.1:9092/$TOPIC_NAME?partition-num=4";; *) SINK_URI="mysql://root@127.0.0.1:3306/";; esac - changefeedid=$(cdc cli changefeed create --pd="http://${UP_PD_HOST}:${UP_PD_PORT}" --start-ts=$start_ts --sink-uri="$SINK_URI" 2>&1|tail -n2|head -n1|awk '{print $2}') + changefeedid=$(cdc cli changefeed create --pd="http://${UP_PD_HOST_1}:${UP_PD_PORT_1}" --start-ts=$start_ts --sink-uri="$SINK_URI" 2>&1|tail -n2|head -n1|awk '{print $2}') if [ "$SINK_TYPE" == "kafka" ]; then run_kafka_consumer $WORK_DIR "kafka://127.0.0.1:9092/$TOPIC_NAME?partition-num=4" fi - ensure 10 check_changefeed_is_stopped ${UP_PD_HOST}:${UP_PD_PORT} ${changefeedid} + ensure 10 check_changefeed_is_stopped ${UP_PD_HOST_1}:${UP_PD_PORT_1} ${changefeedid} - cdc cli changefeed resume --changefeed-id=${changefeedid} --pd="http://${UP_PD_HOST}:${UP_PD_PORT}" + cdc cli changefeed resume --changefeed-id=${changefeedid} --pd="http://${UP_PD_HOST_1}:${UP_PD_PORT_1}" for i in $(seq $DB_COUNT); do check_table_exists "changefeed_auto_stop_$i.USERTABLE" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} done diff --git a/tests/changefeed_error/run.sh b/tests/changefeed_error/run.sh index be9f8e8953a..70b8d389c9a 100755 --- a/tests/changefeed_error/run.sh +++ b/tests/changefeed_error/run.sh @@ -53,7 +53,7 @@ function run() { cd $WORK_DIR - start_ts=$(run_cdc_cli tso query --pd=http://$UP_PD_HOST:$UP_PD_PORT) + start_ts=$(run_cdc_cli tso query --pd=http://$UP_PD_HOST_1:$UP_PD_PORT_1) run_sql "CREATE DATABASE changefeed_error;" ${UP_TIDB_HOST} ${UP_TIDB_PORT} go-ycsb load mysql -P $CUR/conf/workload -p mysql.host=${UP_TIDB_HOST} -p mysql.port=${UP_TIDB_PORT} -p mysql.user=root -p mysql.db=changefeed_error export GO_FAILPOINTS='github.com/pingcap/ticdc/cdc/NewChangefeedNoRetryError=1*return(true)' @@ -70,10 +70,10 @@ function run() { run_kafka_consumer $WORK_DIR "kafka://127.0.0.1:9092/$TOPIC_NAME?partition-num=4" fi - ensure $MAX_RETRIES check_changefeed_mark_failed http://${UP_PD_HOST}:${UP_PD_PORT} ${changefeedid} "\[tikv:9006\]GC life time is shorter than transaction duration.*" - changefeed_info=$(ETCDCTL_API=3 etcdctl --endpoints=${UP_PD_HOST}:${UP_PD_PORT} get /tidb/cdc/changefeed/info/${changefeedid}|tail -n 1) + ensure $MAX_RETRIES check_changefeed_mark_failed http://${UP_PD_HOST_1}:${UP_PD_PORT_1} ${changefeedid} "\[tikv:9006\]GC life time is shorter than transaction duration.*" + changefeed_info=$(ETCDCTL_API=3 etcdctl --endpoints=${UP_PD_HOST_1}:${UP_PD_PORT_1} get /tidb/cdc/changefeed/info/${changefeedid}|tail -n 1) new_info=$(echo $changefeed_info|sed 's/"state":"failed"/"state":"normal"/g') - ETCDCTL_API=3 etcdctl --endpoints=${UP_PD_HOST}:${UP_PD_PORT} put /tidb/cdc/changefeed/info/${changefeedid} "$new_info" + ETCDCTL_API=3 etcdctl --endpoints=${UP_PD_HOST_1}:${UP_PD_PORT_1} put /tidb/cdc/changefeed/info/${changefeedid} "$new_info" check_table_exists "changefeed_error.USERTABLE" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} check_sync_diff $WORK_DIR $CUR/conf/diff_config.toml @@ -83,12 +83,12 @@ function run() { export GO_FAILPOINTS='github.com/pingcap/ticdc/cdc/NewChangefeedRetryError=return(true)' kill $capture_pid - ensure $MAX_RETRIES check_no_capture http://${UP_PD_HOST}:${UP_PD_PORT} + ensure $MAX_RETRIES check_no_capture http://${UP_PD_HOST_1}:${UP_PD_PORT_1} run_cdc_server --workdir $WORK_DIR --binary $CDC_BINARY - ensure $MAX_RETRIES check_changefeed_mark_failed http://${UP_PD_HOST}:${UP_PD_PORT} ${changefeedid} "failpoint injected retriable error" + ensure $MAX_RETRIES check_changefeed_mark_failed http://${UP_PD_HOST_1}:${UP_PD_PORT_1} ${changefeedid} "failpoint injected retriable error" cdc cli changefeed remove -c $changefeedid - ensure $MAX_RETRIES check_no_changefeed ${UP_PD_HOST}:${UP_PD_PORT} + ensure $MAX_RETRIES check_no_changefeed ${UP_PD_HOST_1}:${UP_PD_PORT_1} export GO_FAILPOINTS='' cleanup_process $CDC_BINARY diff --git a/tests/changefeed_finish/run.sh b/tests/changefeed_finish/run.sh index f220909b3d5..67d2e4b1fd2 100755 --- a/tests/changefeed_finish/run.sh +++ b/tests/changefeed_finish/run.sh @@ -38,7 +38,7 @@ function run() { start_tidb_cluster --workdir $WORK_DIR cd $WORK_DIR - pd_addr="http://$UP_PD_HOST:$UP_PD_PORT" + pd_addr="http://$UP_PD_HOST_1:$UP_PD_PORT_1" TOPIC_NAME="ticdc-changefeed-pause-resume-$RANDOM" case $SINK_TYPE in kafka) SINK_URI="kafka://127.0.0.1:9092/$TOPIC_NAME?partition-num=4";; @@ -48,7 +48,7 @@ function run() { run_kafka_consumer $WORK_DIR "kafka://127.0.0.1:9092/$TOPIC_NAME?partition-num=4" fi run_cdc_server --workdir $WORK_DIR --binary $CDC_BINARY --addr "127.0.0.1:8300" --pd $pd_addr - now=$(cdc cli tso query --pd=http://$UP_PD_HOST:$UP_PD_PORT) + now=$(cdc cli tso query --pd=http://$UP_PD_HOST_1:$UP_PD_PORT_1) # 90s after now target_ts=$(( $now + 90 * 10 ** 3 * 2 ** 18 )) changefeed_id=$(cdc cli changefeed create --sink-uri="$SINK_URI" --target-ts=$target_ts 2>&1|tail -n2|head -n1|awk '{print $2}') diff --git a/tests/changefeed_pause_resume/run.sh b/tests/changefeed_pause_resume/run.sh index 6b0e501a388..da1a44117fe 100755 --- a/tests/changefeed_pause_resume/run.sh +++ b/tests/changefeed_pause_resume/run.sh @@ -14,7 +14,7 @@ function run() { start_tidb_cluster --workdir $WORK_DIR cd $WORK_DIR - pd_addr="http://$UP_PD_HOST:$UP_PD_PORT" + pd_addr="http://$UP_PD_HOST_1:$UP_PD_PORT_1" TOPIC_NAME="ticdc-changefeed-pause-resume-$RANDOM" case $SINK_TYPE in kafka) SINK_URI="kafka://127.0.0.1:9092/$TOPIC_NAME?partition-num=4";; diff --git a/tests/changefeed_reconstruct/run.sh b/tests/changefeed_reconstruct/run.sh index 6761cf0b479..c282004e66c 100755 --- a/tests/changefeed_reconstruct/run.sh +++ b/tests/changefeed_reconstruct/run.sh @@ -37,7 +37,7 @@ function run() { start_tidb_cluster --workdir $WORK_DIR cd $WORK_DIR - pd_addr="http://$UP_PD_HOST:$UP_PD_PORT" + pd_addr="http://$UP_PD_HOST_1:$UP_PD_PORT_1" TOPIC_NAME="ticdc-changefeed-reconstruct-$RANDOM" case $SINK_TYPE in kafka) SINK_URI="kafka://127.0.0.1:9092/$TOPIC_NAME?partition-num=4";; diff --git a/tests/cli/run.sh b/tests/cli/run.sh index cc852ac0f71..dd736436e49 100644 --- a/tests/cli/run.sh +++ b/tests/cli/run.sh @@ -10,7 +10,7 @@ TLS_DIR=$( cd $CUR/../_certificates && pwd ) function check_changefeed_state() { changefeedid=$1 expected=$2 - output=$(cdc cli changefeed query --simple --changefeed-id $changefeedid --pd=http://$UP_PD_HOST:$UP_PD_PORT 2>&1) + output=$(cdc cli changefeed query --simple --changefeed-id $changefeedid --pd=http://$UP_PD_HOST_1:$UP_PD_PORT_1 2>&1) state=$(echo $output | grep -oE "\"state\": \"[a-z]+\""|tr -d '" '|awk -F':' '{print $(NF)}') if [ "$state" != "$expected" ]; then echo "unexpected state $output, expected $expected" @@ -18,15 +18,26 @@ function check_changefeed_state() { fi } +function check_changefeed_count() { + pd_addr=$1 + expected=$2 + feed_count=$(cdc cli changefeed list --pd=$pd_addr|jq '.|length') + if [[ "$feed_count" != "$expected" ]]; then + echo "[$(date)] <<<<< unexpect changefeed count! expect ${expected} got ${feed_count} >>>>>" + exit 1 + fi + echo "changefeed count ${feed_count} check pass, pd_addr: $pd_addr" +} + function run() { rm -rf $WORK_DIR && mkdir -p $WORK_DIR - start_tidb_cluster --workdir $WORK_DIR + start_tidb_cluster --workdir $WORK_DIR --multiple-upstream-pd true cd $WORK_DIR # record tso before we create tables to skip the system table DDLs - start_ts=$(run_cdc_cli tso query --pd=http://$UP_PD_HOST:$UP_PD_PORT) + start_ts=$(run_cdc_cli tso query --pd=http://$UP_PD_HOST_1:$UP_PD_PORT_1) run_sql "CREATE table test.simple(id int primary key, val int);" run_sql "CREATE table test.\`simple-dash\`(id int primary key, val int);" @@ -53,6 +64,11 @@ function run() { check_changefeed_state $uuid "normal" + check_changefeed_count http://${UP_PD_HOST_1}:${UP_PD_PORT_1} 1 + check_changefeed_count http://${UP_PD_HOST_2}:${UP_PD_PORT_2} 1 + check_changefeed_count http://${UP_PD_HOST_3}:${UP_PD_PORT_3} 1 + check_changefeed_count http://${UP_PD_HOST_1}:${UP_PD_PORT_1},http://${UP_PD_HOST_2}:${UP_PD_PORT_2},http://${UP_PD_HOST_3}:${UP_PD_PORT_3} 1 + # Make sure changefeed can not be created if the name is already exists. exists=$(run_cdc_cli changefeed create --start-ts=$start_ts --sink-uri="$SINK_URI" --changefeed-id="$uuid" 2>&1 | grep -oE 'already exists') if [[ -z $exists ]]; then @@ -60,6 +76,17 @@ function run() { exit 1 fi + # Update changefeed failed because changefeed is running +cat - >"$WORK_DIR/changefeed.toml" <&1 | grep 'admin-job-type' | grep -oE '[0-9]' | head -1) @@ -70,11 +97,6 @@ function run() { check_changefeed_state $uuid "stopped" # Update changefeed -cat - >"$WORK_DIR/changefeed.toml" <&1) if [[ ! $changefeed_info == *"\"case-sensitive\": false"* ]]; then @@ -110,6 +132,19 @@ EOF fi check_changefeed_state $uuid "removed" + # Make sure changefeed can not be created if a removed changefeed with the same name exists + create_log=$(run_cdc_cli changefeed create --start-ts=$start_ts --sink-uri="$SINK_URI" --changefeed-id="$uuid" 2>&1) + exists=$(echo $create_log | grep -oE 'already exists') + if [[ -z $exists ]]; then + echo "[$(date)] <<<<< unexpect output got ${create_log} >>>>>" + exit 1 + fi + + # force remove the changefeed, and re create a new one with the same name + run_cdc_cli changefeed --changefeed-id $uuid remove --force && sleep 3 + run_cdc_cli changefeed create --sink-uri="$SINK_URI" --tz="Asia/Shanghai" -c="$uuid" && sleep 3 + check_changefeed_state $uuid "normal" + # Make sure bad sink url fails at creating changefeed. badsink=$(run_cdc_cli changefeed create --start-ts=$start_ts --sink-uri="mysql://badsink" 2>&1 | grep -oE 'fail') if [[ -z $badsink ]]; then diff --git a/tests/common_1/data/test.sql b/tests/common_1/data/test.sql index 8a0f616653b..07857d49172 100644 --- a/tests/common_1/data/test.sql +++ b/tests/common_1/data/test.sql @@ -4,104 +4,203 @@ use `common_1`; -- multi data type test -CREATE TABLE cdc_multi_data_type ( - id INT AUTO_INCREMENT, - t_boolean BOOLEAN, - t_bigint BIGINT, - t_double DOUBLE, - t_decimal DECIMAL(38,19), - t_bit BIT(64), - t_date DATE, - t_datetime DATETIME, - t_timestamp TIMESTAMP NULL, - t_time TIME, - t_year YEAR, - t_char CHAR, - t_varchar VARCHAR(10), - t_blob BLOB, - t_text TEXT, - t_enum ENUM('enum1', 'enum2', 'enum3'), - t_set SET('a', 'b', 'c'), - t_json JSON, - PRIMARY KEY(id) -) ENGINE=InnoDB DEFAULT CHARSET=utf8 COLLATE=utf8_bin; - -INSERT INTO cdc_multi_data_type(t_boolean, t_bigint, t_double, t_decimal, t_bit - ,t_date, t_datetime, t_timestamp, t_time, t_year - ,t_char, t_varchar, t_blob, t_text, t_enum - ,t_set, t_json) VALUES - (true, 9223372036854775807, 123.123, 123456789012.123456789012, b'1000001' - ,'1000-01-01', '9999-12-31 23:59:59', '19731230153000', '23:59:59', 1970 - ,'测', '测试', 'blob', '测试text', 'enum2' - ,'a,b', NULL); - -INSERT INTO cdc_multi_data_type(t_boolean, t_bigint, t_double, t_decimal, t_bit - ,t_date, t_datetime, t_timestamp, t_time, t_year - ,t_char, t_varchar, t_blob, t_text, t_enum - ,t_set, t_json) VALUES - (true, 9223372036854775807, 678, 321, b'1000001' - ,'1000-01-01', '9999-12-31 23:59:59', '19731230153000', '23:59:59', 1970 - ,'测', '测试', 'blob', '测试text', 'enum2' - ,'a,b', NULL); - -INSERT INTO cdc_multi_data_type(t_boolean) VALUES(TRUE); - -INSERT INTO cdc_multi_data_type(t_boolean) VALUES(FALSE); - -INSERT INTO cdc_multi_data_type(t_bigint) VALUES(-9223372036854775808); - -INSERT INTO cdc_multi_data_type(t_bigint) VALUES(9223372036854775807); - -INSERT INTO cdc_multi_data_type(t_json) VALUES('{"key1": "value1", "key2": "value2"}'); +CREATE TABLE cdc_multi_data_type +( + id INT AUTO_INCREMENT, + t_boolean BOOLEAN, + t_bigint BIGINT, + t_double DOUBLE, + t_decimal DECIMAL(38, 19), + t_bit BIT(64), + t_date DATE, + t_datetime DATETIME, + t_timestamp TIMESTAMP NULL, + t_time TIME, + t_year YEAR, + t_char CHAR, + t_varchar VARCHAR(10), + t_blob BLOB, + t_text TEXT, + t_enum ENUM ('enum1', 'enum2', 'enum3'), + t_set SET ('a', 'b', 'c'), + t_json JSON, + PRIMARY KEY (id) +) ENGINE = InnoDB + DEFAULT CHARSET = utf8 + COLLATE = utf8_bin; + +INSERT INTO cdc_multi_data_type( t_boolean, t_bigint, t_double, t_decimal, t_bit + , t_date, t_datetime, t_timestamp, t_time, t_year + , t_char, t_varchar, t_blob, t_text, t_enum + , t_set, t_json) +VALUES ( true, 9223372036854775807, 123.123, 123456789012.123456789012, b'1000001' + , '1000-01-01', '9999-12-31 23:59:59', '19731230153000', '23:59:59', 1970 + , '测', '测试', 'blob', '测试text', 'enum2' + , 'a,b', NULL); + +INSERT INTO cdc_multi_data_type( t_boolean, t_bigint, t_double, t_decimal, t_bit + , t_date, t_datetime, t_timestamp, t_time, t_year + , t_char, t_varchar, t_blob, t_text, t_enum + , t_set, t_json) +VALUES ( true, 9223372036854775807, 678, 321, b'1000001' + , '1000-01-01', '9999-12-31 23:59:59', '19731230153000', '23:59:59', 1970 + , '测', '测试', 'blob', '测试text', 'enum2' + , 'a,b', NULL); + +INSERT INTO cdc_multi_data_type(t_boolean) +VALUES (TRUE); + +INSERT INTO cdc_multi_data_type(t_boolean) +VALUES (FALSE); + +INSERT INTO cdc_multi_data_type(t_bigint) +VALUES (-9223372036854775808); + +INSERT INTO cdc_multi_data_type(t_bigint) +VALUES (9223372036854775807); + +INSERT INTO cdc_multi_data_type(t_json) +VALUES ('{ + "key1": "value1", + "key2": "value2" +}'); -- view test -CREATE TABLE t1 (id INT NOT NULL PRIMARY KEY AUTO_INCREMENT, c1 INT NOT NULL); +CREATE TABLE t1 +( + id INT NOT NULL PRIMARY KEY AUTO_INCREMENT, + c1 INT NOT NULL +); -INSERT INTO t1 (c1) VALUES (1),(2),(3),(4),(5); +INSERT INTO t1 (c1) +VALUES (1), + (2), + (3), + (4), + (5); -CREATE VIEW v1 AS SELECT * FROM t1 WHERE c1 > 2; +CREATE VIEW v1 AS +SELECT * +FROM t1 +WHERE c1 > 2; -- uk without pk -- https://internal.pingcap.net/jira/browse/TOOL-714 -- CDC don't support UK is null -CREATE TABLE uk_without_pk (id INT, a1 INT NOT NULL, a3 INT NOT NULL, UNIQUE KEY dex1(a1, a3)); +CREATE TABLE uk_without_pk +( + id INT, + a1 INT NOT NULL, + a3 INT NOT NULL, + UNIQUE KEY dex1 (a1, a3) +); -INSERT INTO uk_without_pk(id, a1, a3) VALUES(1, 1, 2); +INSERT INTO uk_without_pk(id, a1, a3) +VALUES (1, 1, 2); -INSERT INTO uk_without_pk(id, a1, a3) VALUES(2, 1, 1); +INSERT INTO uk_without_pk(id, a1, a3) +VALUES (2, 1, 1); -UPDATE uk_without_pk SET id = 10, a1 = 2 WHERE a1 = 1; +UPDATE uk_without_pk +SET id = 10, + a1 = 2 +WHERE a1 = 1; -UPDATE uk_without_pk SET id = 100 WHERE a1 = 10; +UPDATE uk_without_pk +SET id = 100 +WHERE a1 = 10; -UPDATE uk_without_pk SET a3 = 4 WHERE a3 = 1; +UPDATE uk_without_pk +SET a3 = 4 +WHERE a3 = 1; -- bit column -- Test issue: TOOL-1346 -CREATE TABLE binlog_insert_bit(a BIT(1) PRIMARY KEY, b BIT(64)); +CREATE TABLE binlog_insert_bit +( + a BIT(1) PRIMARY KEY, + b BIT(64) +); -INSERT INTO binlog_insert_bit VALUES (0x01, 0xffffffff); +INSERT INTO binlog_insert_bit +VALUES (0x01, 0xffffffff); -UPDATE binlog_insert_bit SET a = 0x00, b = 0xfffffffe; +UPDATE binlog_insert_bit +SET a = 0x00, + b = 0xfffffffe; -- recover test -- Test issue: TOOL-1407 -CREATE TABLE recover_and_insert(id INT PRIMARY KEY, a INT); +CREATE TABLE recover_and_insert +( + id INT PRIMARY KEY, + a INT +); -INSERT INTO recover_and_insert(id, a) VALUES(1, -1); +INSERT INTO recover_and_insert(id, a) +VALUES (1, -1); -UPDATE recover_and_insert SET a = -5 WHERE id = 1; +UPDATE recover_and_insert +SET a = -5 +WHERE id = 1; DROP TABLE recover_and_insert; RECOVER TABLE recover_and_insert; -- make sure we can insert data after recovery -INSERT INTO recover_and_insert(id, a) VALUES(2, -3); +INSERT INTO recover_and_insert(id, a) +VALUES (2, -3); + +-- column null test + +CREATE TABLE `column_is_null` +( + `id` int(11) NOT NULL, + `t` datetime DEFAULT CURRENT_TIMESTAMP, + PRIMARY KEY (`id`) +) ENGINE = InnoDB + DEFAULT CHARSET = utf8mb4 + COLLATE = utf8mb4_bin; + +INSERT INTO `column_is_null`(id) +VALUES (1), + (2); +UPDATE `column_is_null` +SET t = NULL +WHERE id = 1; + +-- test add and drop columns + +CREATE TABLE `add_and_drop_columns` +( + `id` int(11) NOT NULL PRIMARY KEY +); + +insert into `add_and_drop_columns` (id) +values (1); + +alter table `add_and_drop_columns` + add col1 int null, + add col2 int null, + add col3 int null; + +insert into `add_and_drop_columns` (id, col1, col2, col3) +values (2, 3, 4, 5); + +alter table `add_and_drop_columns` + drop col1, + drop col2; + +insert into `add_and_drop_columns` (id, col3) +values (3, 4); -- mark finish table -CREATE TABLE finish_mark(a int primary key); \ No newline at end of file +CREATE TABLE finish_mark +( + a int primary key +); diff --git a/tests/common_1/run.sh b/tests/common_1/run.sh index 89d71ceacfd..dce8d86d115 100644 --- a/tests/common_1/run.sh +++ b/tests/common_1/run.sh @@ -16,7 +16,7 @@ function run() { cd $WORK_DIR # record tso before we create tables to skip the system table DDLs - start_ts=$(run_cdc_cli tso query --pd=http://$UP_PD_HOST:$UP_PD_PORT) + start_ts=$(run_cdc_cli tso query --pd=http://$UP_PD_HOST_1:$UP_PD_PORT_1) run_cdc_server --workdir $WORK_DIR --binary $CDC_BINARY diff --git a/tests/cyclic_ab/run.sh b/tests/cyclic_ab/run.sh index 2cf399996fb..44922d88db8 100644 --- a/tests/cyclic_ab/run.sh +++ b/tests/cyclic_ab/run.sh @@ -43,13 +43,13 @@ function run() { } done # record tso after we create tables to not block on waiting mark tables DDLs. - start_ts=$(run_cdc_cli tso query --pd=http://$UP_PD_HOST:$UP_PD_PORT) + start_ts=$(run_cdc_cli tso query --pd=http://$UP_PD_HOST_1:$UP_PD_PORT_1) run_cdc_server \ --workdir $WORK_DIR \ --binary $CDC_BINARY \ --logsuffix "_${TEST_NAME}_upsteam" \ - --pd "http://${UP_PD_HOST}:${UP_PD_PORT}" \ + --pd "http://${UP_PD_HOST_1}:${UP_PD_PORT_1}" \ --addr "127.0.0.1:8300" run_cdc_server \ @@ -62,7 +62,7 @@ function run() { # Echo y to ignore ineligible tables echo "y" | run_cdc_cli changefeed create --start-ts=$start_ts \ --sink-uri="mysql://root@${DOWN_TIDB_HOST}:${DOWN_TIDB_PORT}/" \ - --pd "http://${UP_PD_HOST}:${UP_PD_PORT}" \ + --pd "http://${UP_PD_HOST_1}:${UP_PD_PORT_1}" \ --cyclic-replica-id 1 \ --cyclic-filter-replica-ids 2 \ --cyclic-sync-ddl true diff --git a/tests/cyclic_abc/run.sh b/tests/cyclic_abc/run.sh index 69e9e7b8806..0eb5e97f25e 100644 --- a/tests/cyclic_abc/run.sh +++ b/tests/cyclic_abc/run.sh @@ -47,7 +47,7 @@ function run() { run_cdc_cli changefeed cyclic create-marktables \ --cyclic-upstream-dsn="root@tcp(${UP_TIDB_HOST}:${UP_TIDB_PORT})/" \ - --pd "http://${UP_PD_HOST}:${UP_PD_PORT}" + --pd "http://${UP_PD_HOST_1}:${UP_PD_PORT_1}" run_cdc_cli changefeed cyclic create-marktables \ --cyclic-upstream-dsn="root@tcp(${DOWN_TIDB_HOST}:${DOWN_TIDB_PORT})/" \ @@ -64,13 +64,13 @@ function run() { --cyclic-upstream-ssl-key=$TLS_DIR/server-key.pem # record tso after we create tables to not block on waiting mark tables DDLs. - start_ts=$(run_cdc_cli tso query --pd=http://$UP_PD_HOST:$UP_PD_PORT) + start_ts=$(run_cdc_cli tso query --pd=http://$UP_PD_HOST_1:$UP_PD_PORT_1) run_cdc_server \ --workdir $WORK_DIR \ --binary $CDC_BINARY \ --logsuffix "_${TEST_NAME}_upsteam" \ - --pd "http://${UP_PD_HOST}:${UP_PD_PORT}" \ + --pd "http://${UP_PD_HOST_1}:${UP_PD_PORT_1}" \ --addr "127.0.0.1:8300" run_cdc_server \ @@ -91,7 +91,7 @@ function run() { run_cdc_cli changefeed create --start-ts=$start_ts \ --sink-uri="mysql://root@${DOWN_TIDB_HOST}:${DOWN_TIDB_PORT}/" \ - --pd "http://${UP_PD_HOST}:${UP_PD_PORT}" \ + --pd "http://${UP_PD_HOST_1}:${UP_PD_PORT_1}" \ --cyclic-replica-id 1 \ --cyclic-filter-replica-ids 2 \ --cyclic-sync-ddl true @@ -106,6 +106,7 @@ function run() { run_cdc_cli changefeed create --start-ts=$start_ts \ --sink-uri="mysql://root@${UP_TIDB_HOST}:${UP_TIDB_PORT}/" \ --pd "https://${TLS_PD_HOST}:${TLS_PD_PORT}" \ + --changefeed-id "tls-changefeed" \ --ca=$TLS_DIR/ca.pem \ --cert=$TLS_DIR/client.pem \ --key=$TLS_DIR/client-key.pem \ @@ -163,6 +164,20 @@ function run() { exit 1 fi + # Check cli TLS + run_cdc_cli changefeed list \ + --pd "https://${TLS_PD_HOST}:${TLS_PD_PORT}" \ + --ca=$TLS_DIR/ca.pem \ + --cert=$TLS_DIR/client.pem \ + --key=$TLS_DIR/client-key.pem + + run_cdc_cli changefeed query \ + --changefeed-id "tls-changefeed" \ + --pd "https://${TLS_PD_HOST}:${TLS_PD_PORT}" \ + --ca=$TLS_DIR/ca.pem \ + --cert=$TLS_DIR/client.pem \ + --key=$TLS_DIR/client-key.pem + cleanup_process $CDC_BINARY } diff --git a/tests/ddl_reentrant/run.sh b/tests/ddl_reentrant/run.sh index e82f2070aa5..bccee763d81 100644 --- a/tests/ddl_reentrant/run.sh +++ b/tests/ddl_reentrant/run.sh @@ -28,6 +28,8 @@ ddls=("create database ddl_reentrant" false "drop view ddl_reentrant.t3_view" false "alter table ddl_reentrant.t3 default character set utf8mb4 default collate utf8mb4_unicode_ci" true "alter schema ddl_reentrant default character set utf8mb4 default collate utf8mb4_unicode_ci" true + "alter table ddl_reentrant.t2 add column c1 int, add column c2 int, add column c3 int" false + "alter table ddl_reentrant.t2 drop column c1, drop column c2, drop column c3" false "alter table ddl_reentrant.t2 drop primary key" false "alter table ddl_reentrant.t2 add primary key pk(id)" false "drop table ddl_reentrant.t2" false diff --git a/tests/ddl_sequence/run.sh b/tests/ddl_sequence/run.sh index 77040e8a61b..cb3d17d521a 100644 --- a/tests/ddl_sequence/run.sh +++ b/tests/ddl_sequence/run.sh @@ -16,7 +16,7 @@ function run() { cd $WORK_DIR # record tso before we create tables to skip the system table DDLs - start_ts=$(run_cdc_cli tso query --pd=http://$UP_PD_HOST:$UP_PD_PORT) + start_ts=$(run_cdc_cli tso query --pd=http://$UP_PD_HOST_1:$UP_PD_PORT_1) run_cdc_server --workdir $WORK_DIR --binary $CDC_BINARY diff --git a/tests/drop_many_tables/run.sh b/tests/drop_many_tables/run.sh index cae92384b16..9161b77bc54 100644 --- a/tests/drop_many_tables/run.sh +++ b/tests/drop_many_tables/run.sh @@ -16,7 +16,7 @@ function run() { cd $WORK_DIR # record tso before we create tables to skip the system table DDLs - start_ts=$(run_cdc_cli tso query --pd=http://$UP_PD_HOST:$UP_PD_PORT) + start_ts=$(run_cdc_cli tso query --pd=http://$UP_PD_HOST_1:$UP_PD_PORT_1) run_cdc_server --workdir $WORK_DIR --binary $CDC_BINARY diff --git a/tests/file_sort/run.sh b/tests/file_sort/run.sh index 784b5a6fd3c..1b669bb89b8 100755 --- a/tests/file_sort/run.sh +++ b/tests/file_sort/run.sh @@ -18,8 +18,8 @@ function run() { cd $WORK_DIR - start_ts=$(run_cdc_cli tso query --pd=http://$UP_PD_HOST:$UP_PD_PORT) - run_sql "CREATE DATABASE file_sort;" + start_ts=$(run_cdc_cli tso query --pd=http://$UP_PD_HOST_1:$UP_PD_PORT_1) + run_sql "CREATE DATABASE file_sort;" ${UP_TIDB_HOST} ${UP_TIDB_PORT} go-ycsb load mysql -P $CUR/conf/workload -p mysql.host=${UP_TIDB_HOST} -p mysql.port=${UP_TIDB_PORT} -p mysql.user=root -p mysql.db=file_sort run_cdc_server --workdir $WORK_DIR --binary $CDC_BINARY --loglevel "info" @@ -37,21 +37,27 @@ function run() { # Add a check table to reduce check time, or if we check data with sync diff # directly, there maybe a lot of diff data at first because of the incremental scan - run_sql "CREATE table file_sort.check1(id int primary key);" + run_sql "CREATE table file_sort.check1(id int primary key);" ${UP_TIDB_HOST} ${UP_TIDB_PORT} check_table_exists "file_sort.USERTABLE" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} check_table_exists "file_sort.check1" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} 90 check_sync_diff $WORK_DIR $CUR/conf/diff_config.toml - go-ycsb load mysql -P $CUR/conf/workload -p mysql.host=${UP_TIDB_HOST} -p mysql.port=${UP_TIDB_PORT} -p mysql.user=root -p mysql.db=file_sort - run_sql "CREATE table file_sort.check2(id int primary key);" + run_sql "truncate table file_sort.USERTABLE" ${UP_TIDB_HOST} ${UP_TIDB_PORT} + check_sync_diff $WORK_DIR $CUR/conf/diff_config.toml + run_sql "CREATE table file_sort.check2(id int primary key);" ${UP_TIDB_HOST} ${UP_TIDB_PORT} check_table_exists "file_sort.check2" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} 90 check_sync_diff $WORK_DIR $CUR/conf/diff_config.toml - run_sql "create table file_sort.USERTABLE2 like file_sort.USERTABLE" - run_sql "insert into file_sort.USERTABLE2 select * from file_sort.USERTABLE" - run_sql "create table file_sort.check3(id int primary key);" - check_table_exists "file_sort.USERTABLE2" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} + go-ycsb load mysql -P $CUR/conf/workload -p mysql.host=${UP_TIDB_HOST} -p mysql.port=${UP_TIDB_PORT} -p mysql.user=root -p mysql.db=file_sort + run_sql "CREATE table file_sort.check3(id int primary key);" ${UP_TIDB_HOST} ${UP_TIDB_PORT} check_table_exists "file_sort.check3" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} 90 + check_sync_diff $WORK_DIR $CUR/conf/diff_config.toml + + run_sql "create table file_sort.USERTABLE2 like file_sort.USERTABLE" ${UP_TIDB_HOST} ${UP_TIDB_PORT} + run_sql "insert into file_sort.USERTABLE2 select * from file_sort.USERTABLE" ${UP_TIDB_HOST} ${UP_TIDB_PORT} + run_sql "create table file_sort.check4(id int primary key);" ${UP_TIDB_HOST} ${UP_TIDB_PORT} + check_table_exists "file_sort.USERTABLE2" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} + check_table_exists "file_sort.check4" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} 90 check_sync_diff $WORK_DIR $CUR/conf/diff_config.toml diff --git a/tests/gc_safepoint/run.sh b/tests/gc_safepoint/run.sh index 579765d0d48..bf77cb26fec 100755 --- a/tests/gc_safepoint/run.sh +++ b/tests/gc_safepoint/run.sh @@ -7,19 +7,19 @@ source $CUR/../_utils/test_prepare WORK_DIR=$OUT_DIR/$TEST_NAME CDC_BINARY=cdc.test SINK_TYPE=$1 -MAX_RETRIES=5 +MAX_RETRIES=10 function get_safepoint() { pd_addr=$1 pd_cluster_id=$2 - safe_point=$(etcdctl get --endpoints=$pd_addr /pd/$pd_cluster_id/gc/safe_point/service/ticdc|grep -oE "SafePoint\":[0-9]+"|grep -oE "[0-9]+") + safe_point=$(ETCDCTL_API=3 etcdctl --endpoints=$pd_addr get /pd/$pd_cluster_id/gc/safe_point/service/ticdc|grep -oE "safe_point\":[0-9]+"|grep -oE "[0-9]+") echo $safe_point } function check_safepoint_cleared() { pd_addr=$1 pd_cluster_id=$2 - query=$(etcdctl get --endpoints=$pd_addr /pd/$pd_cluster_id/gc/safe_point/service/ticdc) + query=$(ETCDCTL_API=3 etcdctl --endpoints=$pd_addr get /pd/$pd_cluster_id/gc/safe_point/service/ticdc) if [ ! -z "$query" ]; then echo "gc safepoint is not cleared: $query" fi @@ -73,7 +73,7 @@ function run() { start_tidb_cluster --workdir $WORK_DIR cd $WORK_DIR - pd_addr="http://$UP_PD_HOST:$UP_PD_PORT" + pd_addr="http://$UP_PD_HOST_1:$UP_PD_PORT_1" TOPIC_NAME="ticdc-gc-safepoint-$RANDOM" case $SINK_TYPE in kafka) SINK_URI="kafka://127.0.0.1:9092/$TOPIC_NAME?partition-num=4";; diff --git a/tests/generate_column/run.sh b/tests/generate_column/run.sh index 2e39ff1db13..1501ce10850 100644 --- a/tests/generate_column/run.sh +++ b/tests/generate_column/run.sh @@ -16,7 +16,7 @@ function run() { cd $WORK_DIR # record tso before we create tables to skip the system table DDLs - start_ts=$(run_cdc_cli tso query --pd=http://$UP_PD_HOST:$UP_PD_PORT) + start_ts=$(run_cdc_cli tso query --pd=http://$UP_PD_HOST_1:$UP_PD_PORT_1) run_cdc_server --workdir $WORK_DIR --binary $CDC_BINARY diff --git a/tests/kill_owner_with_ddl/conf/diff_config.toml b/tests/kill_owner_with_ddl/conf/diff_config.toml new file mode 100644 index 00000000000..247c2592935 --- /dev/null +++ b/tests/kill_owner_with_ddl/conf/diff_config.toml @@ -0,0 +1,27 @@ +# diff Configuration. + +log-level = "info" +chunk-size = 10 +check-thread-count = 4 +sample-percent = 100 +use-rowid = false +use-checksum = true +fix-sql-file = "fix.sql" + +# tables need to check. +[[check-tables]] + schema = "kill_owner_with_ddl" + tables = ["~t.*"] + +[[source-db]] + host = "127.0.0.1" + port = 4000 + user = "root" + password = "" + instance-id = "source-1" + +[target-db] + host = "127.0.0.1" + port = 3306 + user = "root" + password = "" diff --git a/tests/kill_owner_with_ddl/run.sh b/tests/kill_owner_with_ddl/run.sh new file mode 100755 index 00000000000..fac0922165c --- /dev/null +++ b/tests/kill_owner_with_ddl/run.sh @@ -0,0 +1,88 @@ +#!/bin/bash + +set -e + +CUR=$( cd "$( dirname "${BASH_SOURCE[0]}" )" && pwd ) +source $CUR/../_utils/test_prepare +WORK_DIR=$OUT_DIR/$TEST_NAME +CDC_BINARY=cdc.test +SINK_TYPE=$1 + +MAX_RETRIES=10 + +function check_capture_count() { + pd=$1 + expected=$2 + count=$(cdc cli capture list --pd=$pd 2>&1|jq '.|length') + if [[ ! "$count" -eq "$expected" ]]; then + echo "count: $count expected: $expected" + exit 1 + fi +} + +function kill_cdc_and_restart() { + pd_addr=$1 + work_dir=$2 + cdc_binary=$3 + MAX_RETRIES=10 + cdc_pid=$(curl -s http://127.0.0.1:8300/status|jq '.pid') + kill $cdc_pid + ensure $MAX_RETRIES check_capture_count $pd_addr 0 + run_cdc_server --workdir $work_dir --binary $cdc_binary --addr "127.0.0.1:8300" --pd $pd_addr + ensure $MAX_RETRIES check_capture_count $pd_addr 1 +} + +export -f check_capture_count +export -f kill_cdc_and_restart + +function run() { + # kafka is not supported yet. + if [ "$SINK_TYPE" == "kafka" ]; then + return + fi + + rm -rf $WORK_DIR && mkdir -p $WORK_DIR + start_tidb_cluster --workdir $WORK_DIR + cd $WORK_DIR + + pd_addr="http://$UP_PD_HOST_1:$UP_PD_PORT_1" + SINK_URI="mysql://root@127.0.0.1:3306/?max-txn-row=1" + + run_cdc_server --workdir $WORK_DIR --binary $CDC_BINARY --addr "127.0.0.1:8300" --pd $pd_addr + cdc cli changefeed create --pd=$pd_addr --sink-uri="$SINK_URI" + run_sql "CREATE DATABASE kill_owner_with_ddl;" ${UP_TIDB_HOST} ${UP_TIDB_PORT} + run_sql "CREATE table kill_owner_with_ddl.t1 (id int primary key auto_increment, val int);" ${UP_TIDB_HOST} ${UP_TIDB_PORT} + check_table_exists "kill_owner_with_ddl.t1" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} + + export GO_FAILPOINTS='github.com/pingcap/ticdc/cdc/sink/MySQLSinkExecDDLDelay=return(true)' + kill_cdc_and_restart $pd_addr $WORK_DIR $CDC_BINARY + + for i in $(seq 2 3); do + run_sql "CREATE table kill_owner_with_ddl.t$i (id int primary key auto_increment, val int);" ${UP_TIDB_HOST} ${UP_TIDB_PORT} + done + for i in $(seq 1 3); do + run_sql "INSERT INTO kill_owner_with_ddl.t$i VALUES (),(),();" ${UP_TIDB_HOST} ${UP_TIDB_PORT} + done + + # sleep to ensure processor has consumed the DDL and flushed checkpoint + sleep 5 + + for i in $(seq 1 3); do + kill_cdc_and_restart $pd_addr $WORK_DIR $CDC_BINARY + sleep 2 + done + + export GO_FAILPOINTS='' + kill_cdc_and_restart $pd_addr $WORK_DIR $CDC_BINARY + + for i in $(seq 1 3); do + check_table_exists "kill_owner_with_ddl.t$i" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} + done + check_sync_diff $WORK_DIR $CUR/conf/diff_config.toml + + cleanup_process $CDC_BINARY +} + +trap stop_tidb_cluster EXIT +run $* +echo "[$(date)] <<<<<< run test case $TEST_NAME success! >>>>>>" diff --git a/tests/many_pk_or_uk/run.sh b/tests/many_pk_or_uk/run.sh index 55d558d316d..dbf860bfcc4 100755 --- a/tests/many_pk_or_uk/run.sh +++ b/tests/many_pk_or_uk/run.sh @@ -17,7 +17,7 @@ function prepare() { cd $WORK_DIR # record tso before we create tables to skip the system table DDLs - start_ts=$(run_cdc_cli tso query --pd=http://$UP_PD_HOST:$UP_PD_PORT) + start_ts=$(run_cdc_cli tso query --pd=http://$UP_PD_HOST_1:$UP_PD_PORT_1) run_cdc_server --workdir $WORK_DIR --binary $CDC_BINARY diff --git a/tests/multi_capture/run.sh b/tests/multi_capture/run.sh index 986615057a4..72b5ff22f09 100755 --- a/tests/multi_capture/run.sh +++ b/tests/multi_capture/run.sh @@ -19,7 +19,7 @@ function run() { cd $WORK_DIR # record tso before we create tables to skip the system table DDLs - start_ts=$(run_cdc_cli tso query --pd=http://$UP_PD_HOST:$UP_PD_PORT) + start_ts=$(run_cdc_cli tso query --pd=http://$UP_PD_HOST_1:$UP_PD_PORT_1) # create $DB_COUNT databases and import initial workload for i in $(seq $DB_COUNT); do diff --git a/tests/multi_source/run.sh b/tests/multi_source/run.sh index a427f90cccf..d14dfc59119 100755 --- a/tests/multi_source/run.sh +++ b/tests/multi_source/run.sh @@ -17,7 +17,7 @@ function prepare() { cd $WORK_DIR # record tso before we create tables to skip the system table DDLs - start_ts=$(run_cdc_cli tso query --pd=http://$UP_PD_HOST:$UP_PD_PORT) + start_ts=$(run_cdc_cli tso query --pd=http://$UP_PD_HOST_1:$UP_PD_PORT_1) run_cdc_server --workdir $WORK_DIR --binary $CDC_BINARY diff --git a/tests/partition_table/run.sh b/tests/partition_table/run.sh index 24624a9deec..b4aed55be52 100644 --- a/tests/partition_table/run.sh +++ b/tests/partition_table/run.sh @@ -16,7 +16,7 @@ function run() { cd $WORK_DIR # record tso before we create tables to skip the system table DDLs - start_ts=$(run_cdc_cli tso query --pd=http://$UP_PD_HOST:$UP_PD_PORT) + start_ts=$(run_cdc_cli tso query --pd=http://$UP_PD_HOST_1:$UP_PD_PORT_1) run_cdc_server --workdir $WORK_DIR --binary $CDC_BINARY diff --git a/tests/processor_panic/run.sh b/tests/processor_panic/run.sh new file mode 100644 index 00000000000..f003ce4f334 --- /dev/null +++ b/tests/processor_panic/run.sh @@ -0,0 +1,106 @@ +#!/bin/bash + +set -e + +CUR=$( cd "$( dirname "${BASH_SOURCE[0]}" )" && pwd ) +source $CUR/../_utils/test_prepare +WORK_DIR=$OUT_DIR/$TEST_NAME +CDC_BINARY=cdc.test +SINK_TYPE=$1 + +function prepare() { + rm -rf $WORK_DIR && mkdir -p $WORK_DIR + + start_tidb_cluster --workdir $WORK_DIR + + cd $WORK_DIR + + # record tso before we create tables to skip the system table DDLs + start_ts=$(run_cdc_cli tso query --pd=http://$UP_PD_HOST_1:$UP_PD_PORT_1) + + run_sql "CREATE table test.simple1(id int primary key, val int);" + run_sql "CREATE table test.simple2(id int primary key, val int);" + + export GO_FAILPOINTS='github.com/pingcap/ticdc/cdc/processorPanic=1*return(true)' + run_cdc_server --workdir $WORK_DIR --binary $CDC_BINARY + + TOPIC_NAME="ticdc-simple-test-$RANDOM" + case $SINK_TYPE in + kafka) SINK_URI="kafka://127.0.0.1:9092/$TOPIC_NAME?partition-num=4&kafka-client-id=cdc_test_simple";; + *) SINK_URI="mysql://root@127.0.0.1:3306/";; + esac + run_cdc_cli changefeed create --start-ts=$start_ts --sink-uri="$SINK_URI" + if [ "$SINK_TYPE" == "kafka" ]; then + run_kafka_consumer $WORK_DIR "kafka://127.0.0.1:9092/$TOPIC_NAME?partition-num=4" + fi +} + +function sql_check() { + # run check in sequence and short circuit principle, if error hanppens, + # the following statement will be not executed + + # check table simple1. + run_sql "SELECT id, val FROM test.simple1;" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} && \ + check_contains "id: 1" && \ + check_contains "val: 1" && \ + check_contains "id: 2" && \ + check_contains "val: 22" && \ + check_not_contains "id: 3" && \ + + # check table simple2. + run_sql "SELECT id, val FROM test.simple2;" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} && \ + check_contains "id: 1" && \ + check_contains "val: 1" && \ + check_contains "id: 2" && \ + check_contains "val: 22" && \ + check_not_contains "id: 3" +} + +function sql_test() { + # test insert/update/delete for two table in the same way. + run_sql "INSERT INTO test.simple1(id, val) VALUES (1, 1);" + run_sql "INSERT INTO test.simple1(id, val) VALUES (2, 2);" + run_sql "INSERT INTO test.simple1(id, val) VALUES (3, 3);" + + # update id = 2 and delete id = 3 + run_sql "UPDATE test.simple1 set val = 22 where id = 2;" + run_sql "DELETE from test.simple1 where id = 3;" + + + # same dml for table simple2 + run_sql "INSERT INTO test.simple2(id, val) VALUES (1, 1);" + run_sql "INSERT INTO test.simple2(id, val) VALUES (2, 2);" + run_sql "INSERT INTO test.simple2(id, val) VALUES (3, 3);" + + run_sql "UPDATE test.simple2 set val = 22 where id = 2;" + run_sql "DELETE from test.simple2 where id = 3;" + + i=0 + check_time=50 + set +e + while [ $i -lt $check_time ] + do + sql_check + ret=$? + if [ "$ret" == 0 ]; then + echo "check data successfully" + break + fi + ((i++)) + echo "check data failed $i-th time, retry later" + sleep 2 + done + set -e + + if [ $i -ge $check_time ]; then + echo "check data failed at last" + exit 1 + fi + + cleanup_process $CDC_BINARY +} + +trap stop_tidb_cluster EXIT +prepare $* +sql_test $* +echo "[$(date)] <<<<<< run test case $TEST_NAME success! >>>>>>" diff --git a/tests/resolve_lock/main.go b/tests/resolve_lock/main.go index 4f4700afe89..4ea7c9aeef6 100644 --- a/tests/resolve_lock/main.go +++ b/tests/resolve_lock/main.go @@ -34,7 +34,6 @@ import ( "github.com/pingcap/parser/model" pd "github.com/pingcap/pd/v4/client" "github.com/pingcap/ticdc/tests/util" - "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/store/tikv" "github.com/pingcap/tidb/store/tikv/oracle" "github.com/pingcap/tidb/store/tikv/tikvrpc" @@ -224,7 +223,7 @@ func (c *Locker) lockKeys(ctx context.Context, rowIDs []int64) error { keyPrefix := tablecodec.GenTableRecordPrefix(c.tableID) for _, rowID := range rowIDs { - key := tablecodec.EncodeRecordKey(keyPrefix, kv.IntHandle(rowID)) + key := tablecodec.EncodeRecordKey(keyPrefix, rowID) keys = append(keys, key) } diff --git a/tests/resolve_lock/run.sh b/tests/resolve_lock/run.sh index 376b54a31ae..c99696e1801 100755 --- a/tests/resolve_lock/run.sh +++ b/tests/resolve_lock/run.sh @@ -17,7 +17,7 @@ function prepare() { cd $WORK_DIR # record tso before we create tables to skip the system table DDLs - start_ts=$(run_cdc_cli tso query --pd=http://$UP_PD_HOST:$UP_PD_PORT) + start_ts=$(run_cdc_cli tso query --pd=http://$UP_PD_HOST_1:$UP_PD_PORT_1) run_cdc_server --workdir $WORK_DIR --binary $CDC_BINARY diff --git a/tests/row_format/data/prepare.sql b/tests/row_format/data/prepare.sql index 0ec72ac4929..44fdf69d440 100644 --- a/tests/row_format/data/prepare.sql +++ b/tests/row_format/data/prepare.sql @@ -73,4 +73,167 @@ SET t_bigint = 888, "key0": "value0", "key2": "value2" }' -WHERE id = 2; \ No newline at end of file +WHERE id = 2; + +create table tp_int +( + id int auto_increment, + c_tinyint tinyint null, + c_smallint smallint null, + c_mediumint mediumint null, + c_int int null, + c_bigint bigint null, + constraint pk + primary key (id) +); + +create table tp_text +( + id int auto_increment, + c_tinytext tinytext null, + c_text text null, + c_mediumtext mediumtext null, + c_longtext longtext null, + c_varchar varchar(16) null, + c_char char(16) null, + c_tinyblob tinyblob null, + c_blob blob null, + c_mediumblob mediumblob null, + c_longblob longblob null, + c_binary binary(16) null, + c_varbinary varbinary(16) null, + constraint pk + primary key (id) +); + +create table tp_time +( + id int auto_increment, + c_date date null, + c_datetime datetime null, + c_timestamp timestamp null, + c_time time null, + c_year year null, + constraint pk + primary key (id) +); + +create table tp_real +( + id int auto_increment, + c_float float null, + c_double double null, + c_decimal decimal null, + constraint pk + primary key (id) +); + +create table tp_other +( + id int auto_increment, + c_enum enum ('a','b','c') null, + c_set set ('a','b','c') null, + c_bit bit(64) null, + c_json json null, + constraint pk + primary key (id) +); + +SET GLOBAL tidb_row_format_version = 2; + +insert into tp_int() +values (); + +insert into tp_int(c_tinyint, c_smallint, c_mediumint, c_int, c_bigint) +values (1, 2, 3, 4, 5); + +-- insert max value +insert into tp_int(c_tinyint, c_smallint, c_mediumint, c_int, c_bigint) +values (127, 32767, 8388607, 2147483647, 9223372036854775807); + +-- insert max value +insert into tp_int(c_tinyint, c_smallint, c_mediumint, c_int, c_bigint) +values (-128, -32768, -8388608, -2147483648, -9223372036854775808); + +insert into tp_text() +values (); + +insert into tp_text(c_tinytext, c_text, c_mediumtext, c_longtext, c_varchar, c_char, c_tinyblob, c_blob, c_mediumblob, + c_longblob, c_binary, c_varbinary) +values ('89504E470D0A1A0A', '89504E470D0A1A0A', '89504E470D0A1A0A', '89504E470D0A1A0A', '89504E470D0A1A0A', + '89504E470D0A1A0A', x'89504E470D0A1A0A', x'89504E470D0A1A0A', x'89504E470D0A1A0A' + , x'89504E470D0A1A0A', x'89504E470D0A1A0A', x'89504E470D0A1A0A'); + +insert into tp_time() +values (); + +insert into tp_time(c_date, c_datetime, c_timestamp, c_time, c_year) +values ('2020-02-20', '2020-02-20 02:20:20', '2020-02-20 02:20:20', '02:20:20', '2020'); + +insert into tp_real() +values (); + +insert into tp_real(c_float, c_double, c_decimal) +values (2020.0202, 2020.0303, 2020.0404); + +insert into tp_other() +values (); + +insert into tp_other(c_enum, c_set, c_bit, c_json) +values ('a', 'a,b', b'1000001', '{ + "key1": "value1", + "key2": "value2" +}'); + +SET GLOBAL tidb_row_format_version = 1; + + +insert into tp_int() +values (); + +insert into tp_int(c_tinyint, c_smallint, c_mediumint, c_int, c_bigint) +values (1, 2, 3, 4, 5); + +-- insert max value +insert into tp_int(c_tinyint, c_smallint, c_mediumint, c_int, c_bigint) +values (127, 32767, 8388607, 2147483647, 9223372036854775807); + +-- insert max value +insert into tp_int(c_tinyint, c_smallint, c_mediumint, c_int, c_bigint) +values (-128, -32768, -8388608, -2147483648, -9223372036854775808); + +insert into tp_text() +values (); + +insert into tp_text(c_tinytext, c_text, c_mediumtext, c_longtext, c_varchar, c_char, c_tinyblob, c_blob, c_mediumblob, + c_longblob, c_binary, c_varbinary) +values ('89504E470D0A1A0A', '89504E470D0A1A0A', '89504E470D0A1A0A', '89504E470D0A1A0A', '89504E470D0A1A0A', + '89504E470D0A1A0A', x'89504E470D0A1A0A', x'89504E470D0A1A0A', x'89504E470D0A1A0A' + , x'89504E470D0A1A0A', x'89504E470D0A1A0A', x'89504E470D0A1A0A'); + +insert into tp_time() +values (); + +insert into tp_time(c_date, c_datetime, c_timestamp, c_time, c_year) +values ('2020-02-20', '2020-02-20 02:20:20', '2020-02-20 02:20:20', '02:20:20', '2020'); + +insert into tp_real() +values (); + +insert into tp_real(c_float, c_double, c_decimal) +values (2020.0202, 2020.0303, 2020.0404); + +insert into tp_other() +values (); + +insert into tp_other(c_enum, c_set, c_bit, c_json) +values ('a', 'a,b', b'1000001', '{ + "key1": "value1", + "key2": "value2" +}'); + + +create table finish_mark +( + id int PRIMARY KEY +); diff --git a/tests/row_format/run.sh b/tests/row_format/run.sh index 4a934501274..2d76b8ba68d 100644 --- a/tests/row_format/run.sh +++ b/tests/row_format/run.sh @@ -16,7 +16,7 @@ function run() { cd $WORK_DIR # record tso before we create tables to skip the system table DDLs - start_ts=$(run_cdc_cli tso query --pd=http://$UP_PD_HOST:$UP_PD_PORT) + start_ts=$(run_cdc_cli tso query --pd=http://$UP_PD_HOST_1:$UP_PD_PORT_1) run_cdc_server --workdir $WORK_DIR --binary $CDC_BINARY @@ -31,7 +31,7 @@ function run() { fi run_sql_file $CUR/data/prepare.sql ${UP_TIDB_HOST} ${UP_TIDB_PORT} # sync_diff can't check non-exist table, so we check expected tables are created in downstream first - check_table_exists row_format.multi_data_type ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} + check_table_exists row_format.finish_mark ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} check_sync_diff $WORK_DIR $CUR/conf/diff_config.toml cleanup_process $CDC_BINARY diff --git a/tests/simple/run.sh b/tests/simple/run.sh index d809f8ad23f..9ee2bcabc88 100644 --- a/tests/simple/run.sh +++ b/tests/simple/run.sh @@ -16,7 +16,7 @@ function prepare() { cd $WORK_DIR # record tso before we create tables to skip the system table DDLs - start_ts=$(run_cdc_cli tso query --pd=http://$UP_PD_HOST:$UP_PD_PORT) + start_ts=$(run_cdc_cli tso query --pd=http://$UP_PD_HOST_1:$UP_PD_PORT_1) run_sql "CREATE table test.simple1(id int primary key, val int);" run_sql "CREATE table test.simple2(id int primary key, val int);" @@ -25,8 +25,8 @@ function prepare() { TOPIC_NAME="ticdc-simple-test-$RANDOM" case $SINK_TYPE in - kafka) SINK_URI="kafka://127.0.0.1:9092/$TOPIC_NAME?partition-num=4&kafka-client-id=cdc_test_simple";; - *) SINK_URI="mysql://root@127.0.0.1:3306/";; + kafka) SINK_URI="kafka+ssl://127.0.0.1:9092/$TOPIC_NAME?partition-num=4&kafka-client-id=cdc_test_simple";; + *) SINK_URI="mysql+ssl://root@127.0.0.1:3306/";; esac run_cdc_cli changefeed create --start-ts=$start_ts --sink-uri="$SINK_URI" if [ "$SINK_TYPE" == "kafka" ]; then diff --git a/tests/sink_hang/run.sh b/tests/sink_hang/run.sh index bdfb7e6a3a0..f7521228536 100644 --- a/tests/sink_hang/run.sh +++ b/tests/sink_hang/run.sh @@ -10,7 +10,7 @@ SINK_TYPE=$1 CDC_COUNT=3 DB_COUNT=4 -MAX_RETRIES=10 +MAX_RETRIES=20 function check_changefeed_state() { pd_addr=$1 @@ -35,7 +35,7 @@ function run() { start_tidb_cluster --workdir $WORK_DIR cd $WORK_DIR - pd_addr="http://$UP_PD_HOST:$UP_PD_PORT" + pd_addr="http://$UP_PD_HOST_1:$UP_PD_PORT_1" TOPIC_NAME="ticdc-sink-hang-test-$RANDOM" case $SINK_TYPE in kafka) SINK_URI="kafka://127.0.0.1:9092/$TOPIC_NAME?partition-num=4";; diff --git a/tests/sink_retry/run.sh b/tests/sink_retry/run.sh index 139b164d7b9..669c173d39d 100755 --- a/tests/sink_retry/run.sh +++ b/tests/sink_retry/run.sh @@ -18,7 +18,7 @@ function run() { cd $WORK_DIR - start_ts=$(run_cdc_cli tso query --pd=http://$UP_PD_HOST:$UP_PD_PORT) + start_ts=$(run_cdc_cli tso query --pd=http://$UP_PD_HOST_1:$UP_PD_PORT_1) run_sql "CREATE DATABASE sink_retry;" go-ycsb load mysql -P $CUR/conf/workload -p mysql.host=${UP_TIDB_HOST} -p mysql.port=${UP_TIDB_PORT} -p mysql.user=root -p mysql.db=sink_retry export GO_FAILPOINTS='github.com/pingcap/ticdc/cdc/sink/MySQLSinkTxnRandomError=25%return(true)' diff --git a/tests/split_region/run.sh b/tests/split_region/run.sh index c69c179366b..0667c80fd87 100755 --- a/tests/split_region/run.sh +++ b/tests/split_region/run.sh @@ -16,7 +16,7 @@ function run() { cd $WORK_DIR # record tso before we create tables to skip the system table DDLs - start_ts=$(run_cdc_cli tso query --pd=http://$UP_PD_HOST:$UP_PD_PORT) + start_ts=$(run_cdc_cli tso query --pd=http://$UP_PD_HOST_1:$UP_PD_PORT_1) run_sql_file $CUR/data/prepare.sql ${UP_TIDB_HOST} ${UP_TIDB_PORT} diff --git a/tests/tiflash/run.sh b/tests/tiflash/run.sh index a8427a2ef55..e80b04521a7 100644 --- a/tests/tiflash/run.sh +++ b/tests/tiflash/run.sh @@ -16,7 +16,7 @@ function run() { cd $WORK_DIR # record tso before we create tables to skip the system table DDLs - start_ts=$(run_cdc_cli tso query --pd=http://$UP_PD_HOST:$UP_PD_PORT) + start_ts=$(run_cdc_cli tso query --pd=http://$UP_PD_HOST_1:$UP_PD_PORT_1) run_cdc_server --workdir $WORK_DIR --binary $CDC_BINARY